From 758d19c55093633fc074424ec9a3f361c211599f Mon Sep 17 00:00:00 2001 From: Peixin Date: Fri, 29 Jul 2022 08:48:34 +0800 Subject: [PATCH 001/190] Init 22.10.0-SNAPSHOT (#6129) Signed-off-by: Peixin Li --- README.md | 2 +- aggregator/pom.xml | 4 ++-- api_validation/pom.xml | 4 ++-- common/pom.xml | 4 ++-- dist/pom.xml | 4 ++-- docs/FAQ.md | 2 +- docs/additional-functionality/ml-integration.md | 2 +- docs/additional-functionality/rapids-udfs.md | 2 +- docs/configs.md | 2 +- docs/dev/shims.md | 12 ++++++------ docs/get-started/getting-started-gcp.md | 4 ++-- docs/get-started/getting-started-on-prem.md | 8 ++++---- integration_tests/README.md | 6 +++--- integration_tests/pom.xml | 4 ++-- jenkins/databricks/create.py | 2 +- jenkins/databricks/init_cudf_udf.sh | 2 +- jenkins/version-def.sh | 8 ++++---- pom.xml | 4 ++-- shuffle-plugin/pom.xml | 4 ++-- spark2-sql-plugin/pom.xml | 4 ++-- .../scala/com/nvidia/spark/rapids/RapidsConf.scala | 2 +- sql-plugin/pom.xml | 4 ++-- .../scala/com/nvidia/spark/rapids/RapidsConf.scala | 2 +- .../scala/com/nvidia/spark/rapids/ShimLoader.scala | 8 ++++---- tests/pom.xml | 4 ++-- tools/pom.xml | 4 ++-- udf-compiler/pom.xml | 4 ++-- 27 files changed, 56 insertions(+), 56 deletions(-) diff --git a/README.md b/README.md index e0faeb1a960..237c06d8987 100644 --- a/README.md +++ b/README.md @@ -75,7 +75,7 @@ as a `provided` dependency. com.nvidia rapids-4-spark_2.12 - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT provided ``` diff --git a/aggregator/pom.xml b/aggregator/pom.xml index d5c608d3170..2189c91af5e 100644 --- a/aggregator/pom.xml +++ b/aggregator/pom.xml @@ -22,12 +22,12 @@ com.nvidia rapids-4-spark-parent - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT rapids-4-spark-aggregator_2.12 RAPIDS Accelerator for Apache Spark Aggregator Creates an aggregated shaded package of the RAPIDS plugin for Apache Spark - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT -Source code for examples of RAPIDS accelerated UDFs is provided in the [udf-examples](https://github.com/NVIDIA/spark-rapids-examples/tree/branch-22.08/examples/UDF-Examples/RAPIDS-accelerated-UDFs) project. +Source code for examples of RAPIDS accelerated UDFs is provided in the [udf-examples](https://github.com/NVIDIA/spark-rapids-examples/tree/branch-22.10/examples/UDF-Examples/RAPIDS-accelerated-UDFs) project. ## GPU Support for Pandas UDF diff --git a/docs/configs.md b/docs/configs.md index e5708ccf8b5..5de5071933c 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -10,7 +10,7 @@ The following is the list of options that `rapids-plugin-4-spark` supports. On startup use: `--conf [conf key]=[conf value]`. For example: ``` -${SPARK_HOME}/bin/spark --jars rapids-4-spark_2.12-22.08.0-SNAPSHOT-cuda11.jar \ +${SPARK_HOME}/bin/spark --jars rapids-4-spark_2.12-22.10.0-SNAPSHOT-cuda11.jar \ --conf spark.plugins=com.nvidia.spark.SQLPlugin \ --conf spark.rapids.sql.incompatibleOps.enabled=true ``` diff --git a/docs/dev/shims.md b/docs/dev/shims.md index 720ca011b11..844ff2cd007 100644 --- a/docs/dev/shims.md +++ b/docs/dev/shims.md @@ -62,16 +62,16 @@ Using JarURLConnection URLs we create a Parallel World of the current version wi Spark 3.0.2's URLs: ``` -jar:file:/home/spark/rapids-4-spark_2.12-22.08.0.jar!/ -jar:file:/home/spark/rapids-4-spark_2.12-22.08.0.jar!/spark3xx-common/ -jar:file:/home/spark/rapids-4-spark_2.12-22.08.0.jar!/spark302/ +jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/ +jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/spark3xx-common/ +jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/spark302/ ``` Spark 3.2.0's URLs : ``` -jar:file:/home/spark/rapids-4-spark_2.12-22.08.0.jar!/ -jar:file:/home/spark/rapids-4-spark_2.12-22.08.0.jar!/spark3xx-common/ -jar:file:/home/spark/rapids-4-spark_2.12-22.08.0.jar!/spark320/ +jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/ +jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/spark3xx-common/ +jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/spark320/ ``` ### Late Inheritance in Public Classes diff --git a/docs/get-started/getting-started-gcp.md b/docs/get-started/getting-started-gcp.md index 802b0d09c1c..6f6ab2e1621 100644 --- a/docs/get-started/getting-started-gcp.md +++ b/docs/get-started/getting-started-gcp.md @@ -186,9 +186,9 @@ val (xgbClassificationModel, _) = benchmark("train") { ## Submit Spark jobs to a Dataproc Cluster Accelerated by GPUs Similar to spark-submit for on-prem clusters, Dataproc supports a Spark application job to be submitted as a Dataproc job. The mortgage examples we use above are also available as a [spark -application](https://github.com/NVIDIA/spark-rapids-examples/tree/branch-22.08/examples/XGBoost-Examples). +application](https://github.com/NVIDIA/spark-rapids-examples/tree/branch-22.10/examples/XGBoost-Examples). After [building the jar -files](https://github.com/NVIDIA/spark-rapids-examples/blob/branch-22.08/docs/get-started/xgboost-examples/building-sample-apps/scala.md) +files](https://github.com/NVIDIA/spark-rapids-examples/blob/branch-22.10/docs/get-started/xgboost-examples/building-sample-apps/scala.md) . Place the jar file `sample_xgboost_apps--jar-with-dependencies.jar` under the diff --git a/docs/get-started/getting-started-on-prem.md b/docs/get-started/getting-started-on-prem.md index 8ca39895960..b83b9a5f169 100644 --- a/docs/get-started/getting-started-on-prem.md +++ b/docs/get-started/getting-started-on-prem.md @@ -53,13 +53,13 @@ CUDA and will not run on other versions. The jars use a classifier to keep them - CUDA 11.x => classifier cuda11 For example, here is a sample version of the jar with CUDA 11.x support: -- rapids-4-spark_2.12-22.08.0-SNAPSHOT-cuda11.jar +- rapids-4-spark_2.12-22.10.0-SNAPSHOT-cuda11.jar For simplicity export the location to this jar. This example assumes the sample jar above has been placed in the `/opt/sparkRapidsPlugin` directory: ```shell export SPARK_RAPIDS_DIR=/opt/sparkRapidsPlugin -export SPARK_RAPIDS_PLUGIN_JAR=${SPARK_RAPIDS_DIR}/rapids-4-spark_2.12-22.08.0-SNAPSHOT-cuda11.jar +export SPARK_RAPIDS_PLUGIN_JAR=${SPARK_RAPIDS_DIR}/rapids-4-spark_2.12-22.10.0-SNAPSHOT-cuda11.jar ``` ## Install the GPU Discovery Script @@ -304,7 +304,7 @@ are using. #### YARN version 3.3.0+ YARN version 3.3.0 and newer support a pluggable device framework which allows adding support for MIG devices via a plugin. See -[NVIDIA GPU Plugin for YARN with MIG support for YARN 3.3.0+](https://github.com/NVIDIA/spark-rapids-examples/tree/branch-22.08/examples/MIG-Support/device-plugins/gpu-mig). +[NVIDIA GPU Plugin for YARN with MIG support for YARN 3.3.0+](https://github.com/NVIDIA/spark-rapids-examples/tree/branch-22.10/examples/MIG-Support/device-plugins/gpu-mig). If you are using that plugin with a Spark version older than 3.2.1 and/or specifying the resource as `nvidia/miggpu` you will also need to specify the config: @@ -321,7 +321,7 @@ required. If you are using YARN version from 3.1.2 up until 3.3.0, it requires making modifications to YARN and deploying a version that adds support for MIG to the built-in YARN GPU resource plugin. -See [NVIDIA Support for GPU for YARN with MIG support for YARN 3.1.2 until YARN 3.3.0](https://github.com/NVIDIA/spark-rapids-examples/tree/branch-22.08/examples/MIG-Support/resource-types/gpu-mig) +See [NVIDIA Support for GPU for YARN with MIG support for YARN 3.1.2 until YARN 3.3.0](https://github.com/NVIDIA/spark-rapids-examples/tree/branch-22.10/examples/MIG-Support/resource-types/gpu-mig) for details. ## Running on Kubernetes diff --git a/integration_tests/README.md b/integration_tests/README.md index 8e147f4ca47..16f9c708c5f 100644 --- a/integration_tests/README.md +++ b/integration_tests/README.md @@ -253,7 +253,7 @@ individually, so you don't risk running unit tests along with the integration te http://www.scalatest.org/user_guide/using_the_scalatest_shell ```shell -spark-shell --jars rapids-4-spark-tests_2.12-22.08.0-SNAPSHOT-tests.jar,rapids-4-spark-integration-tests_2.12-22.08.0-SNAPSHOT-tests.jar,scalatest_2.12-3.0.5.jar,scalactic_2.12-3.0.5.jar +spark-shell --jars rapids-4-spark-tests_2.12-22.10.0-SNAPSHOT-tests.jar,rapids-4-spark-integration-tests_2.12-22.10.0-SNAPSHOT-tests.jar,scalatest_2.12-3.0.5.jar,scalactic_2.12-3.0.5.jar ``` First you import the `scalatest_shell` and tell the tests where they can find the test files you @@ -276,7 +276,7 @@ If you just want to verify the SQL replacement is working you will need to add t assumes CUDA 11.0 is being used. ``` -$SPARK_HOME/bin/spark-submit --jars "rapids-4-spark_2.12-22.08.0-SNAPSHOT-cuda11.jar" ./runtests.py +$SPARK_HOME/bin/spark-submit --jars "rapids-4-spark_2.12-22.10.0-SNAPSHOT-cuda11.jar" ./runtests.py ``` You don't have to enable the plugin for this to work, the test framework will do that for you. @@ -375,7 +375,7 @@ To run cudf_udf tests, need following configuration changes: As an example, here is the `spark-submit` command with the cudf_udf parameter on CUDA 11.0: ``` -$SPARK_HOME/bin/spark-submit --jars "rapids-4-spark_2.12-22.08.0-SNAPSHOT-cuda11.jar,rapids-4-spark-tests_2.12-22.08.0-SNAPSHOT.jar" --conf spark.rapids.memory.gpu.allocFraction=0.3 --conf spark.rapids.python.memory.gpu.allocFraction=0.3 --conf spark.rapids.python.concurrentPythonWorkers=2 --py-files "rapids-4-spark_2.12-22.08.0-SNAPSHOT-cuda11.jar" --conf spark.executorEnv.PYTHONPATH="rapids-4-spark_2.12-22.08.0-SNAPSHOT-cuda11.jar" ./runtests.py --cudf_udf +$SPARK_HOME/bin/spark-submit --jars "rapids-4-spark_2.12-22.10.0-SNAPSHOT-cuda11.jar,rapids-4-spark-tests_2.12-22.10.0-SNAPSHOT.jar" --conf spark.rapids.memory.gpu.allocFraction=0.3 --conf spark.rapids.python.memory.gpu.allocFraction=0.3 --conf spark.rapids.python.concurrentPythonWorkers=2 --py-files "rapids-4-spark_2.12-22.10.0-SNAPSHOT-cuda11.jar" --conf spark.executorEnv.PYTHONPATH="rapids-4-spark_2.12-22.10.0-SNAPSHOT-cuda11.jar" ./runtests.py --cudf_udf ``` ### Enabling fuzz tests diff --git a/integration_tests/pom.xml b/integration_tests/pom.xml index edb5d127e5b..27c8b35bc2d 100644 --- a/integration_tests/pom.xml +++ b/integration_tests/pom.xml @@ -22,10 +22,10 @@ com.nvidia rapids-4-spark-parent - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT rapids-4-spark-integration-tests_2.12 - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT diff --git a/jenkins/databricks/create.py b/jenkins/databricks/create.py index 9ce54f97f10..9dcde35c067 100644 --- a/jenkins/databricks/create.py +++ b/jenkins/databricks/create.py @@ -27,7 +27,7 @@ def main(): workspace = 'https://dbc-9ff9942e-a9c4.cloud.databricks.com' token = '' sshkey = '' - cluster_name = 'CI-GPU-databricks-22.08.0-SNAPSHOT' + cluster_name = 'CI-GPU-databricks-22.10.0-SNAPSHOT' idletime = 240 runtime = '7.0.x-gpu-ml-scala2.12' num_workers = 1 diff --git a/jenkins/databricks/init_cudf_udf.sh b/jenkins/databricks/init_cudf_udf.sh index 8eb51e0b85c..b8d43cf3eef 100644 --- a/jenkins/databricks/init_cudf_udf.sh +++ b/jenkins/databricks/init_cudf_udf.sh @@ -18,7 +18,7 @@ # The initscript to set up environment for the cudf_udf tests on Databricks # Will be automatically pushed into the dbfs:/databricks/init_scripts once it is updated. -CUDF_VER=${CUDF_VER:-22.08} +CUDF_VER=${CUDF_VER:-22.10} # Need to explictly add conda into PATH environment, to activate conda environment. export PATH=/databricks/conda/bin:$PATH diff --git a/jenkins/version-def.sh b/jenkins/version-def.sh index 44eaa0df22e..02af77e6f42 100755 --- a/jenkins/version-def.sh +++ b/jenkins/version-def.sh @@ -26,10 +26,10 @@ for VAR in $OVERWRITE_PARAMS; do done IFS=$PRE_IFS -CUDF_VER=${CUDF_VER:-"22.08.0-SNAPSHOT"} +CUDF_VER=${CUDF_VER:-"22.10.0-SNAPSHOT"} CUDA_CLASSIFIER=${CUDA_CLASSIFIER:-"cuda11"} -PROJECT_VER=${PROJECT_VER:-"22.08.0-SNAPSHOT"} -PROJECT_TEST_VER=${PROJECT_TEST_VER:-"22.08.0-SNAPSHOT"} +PROJECT_VER=${PROJECT_VER:-"22.10.0-SNAPSHOT"} +PROJECT_TEST_VER=${PROJECT_TEST_VER:-"22.10.0-SNAPSHOT"} SPARK_VER=${SPARK_VER:-"3.1.1"} # Make a best attempt to set the default value for the shuffle shim. # Note that SPARK_VER for non-Apache Spark flavors (i.e. databricks, @@ -49,7 +49,7 @@ echo "CUDF_VER: $CUDF_VER, CUDA_CLASSIFIER: $CUDA_CLASSIFIER, PROJECT_VER: $PROJ SPARK_VER: $SPARK_VER, SCALA_BINARY_VER: $SCALA_BINARY_VER" -SPARK_SHIM_VERSIONS_STR=${SPARK_SHIM_VERSIONS_STR:-"311 321cdh 312 313 320 321 322 330"} +SPARK_SHIM_VERSIONS_STR=${SPARK_SHIM_VERSIONS_STR:-"311 321cdh 312 313 314 320 321 322 330 331"} IFS=" " <<< $SPARK_SHIM_VERSIONS_STR read -r -a SPARK_SHIM_VERSIONS diff --git a/pom.xml b/pom.xml index 8e825c250c8..bbf7e16db82 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ rapids-4-spark-parent RAPIDS Accelerator for Apache Spark Root Project The root project of the RAPIDS Accelerator for Apache Spark - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT pom https://nvidia.github.io/spark-rapids/ @@ -987,7 +987,7 @@ 1.10.1 spark${buildver} cuda11 - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT 2.12 incremental 2.12.15 diff --git a/shuffle-plugin/pom.xml b/shuffle-plugin/pom.xml index e20381c3668..4877e8d1c63 100644 --- a/shuffle-plugin/pom.xml +++ b/shuffle-plugin/pom.xml @@ -22,13 +22,13 @@ com.nvidia rapids-4-spark-parent - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT rapids-4-spark-shuffle_2.12 RAPIDS Accelerator for Apache Spark Shuffle Plugin Accelerated shuffle plugin for the RAPIDS plugin for Apache Spark - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT diff --git a/spark2-sql-plugin/pom.xml b/spark2-sql-plugin/pom.xml index 537b639d530..313b0dcd6f4 100644 --- a/spark2-sql-plugin/pom.xml +++ b/spark2-sql-plugin/pom.xml @@ -22,12 +22,12 @@ com.nvidia rapids-4-spark-parent - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT rapids-4-spark-sql-meta_2.11 RAPIDS Accelerator for Apache Spark SQL Plugin Base Meta The RAPIDS SQL plugin for Apache Spark Base Meta Information - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT 2.11 diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 7f394e3f97c..e678811d3b3 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -1460,7 +1460,7 @@ object RapidsConf { |On startup use: `--conf [conf key]=[conf value]`. For example: | |``` - |$SPARK_HOME/bin/spark --jars 'rapids-4-spark_2.12-22.08.0-SNAPSHOT.jar,cudf-22.08.0-SNAPSHOT-cuda11.jar' \ + |$SPARK_HOME/bin/spark --jars 'rapids-4-spark_2.12-22.10.0-SNAPSHOT.jar,cudf-22.10.0-SNAPSHOT-cuda11.jar' \ |--conf spark.plugins=com.nvidia.spark.SQLPlugin \ |--conf spark.rapids.sql.incompatibleOps.enabled=true |``` diff --git a/sql-plugin/pom.xml b/sql-plugin/pom.xml index 0d16d94a41e..92f9d233a72 100644 --- a/sql-plugin/pom.xml +++ b/sql-plugin/pom.xml @@ -22,12 +22,12 @@ com.nvidia rapids-4-spark-parent - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT rapids-4-spark-sql_2.12 RAPIDS Accelerator for Apache Spark SQL Plugin The RAPIDS SQL plugin for Apache Spark - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 09ed723c791..edf7a58cce0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -1546,7 +1546,7 @@ object RapidsConf { |On startup use: `--conf [conf key]=[conf value]`. For example: | |``` - |${SPARK_HOME}/bin/spark --jars rapids-4-spark_2.12-22.08.0-SNAPSHOT-cuda11.jar \ + |${SPARK_HOME}/bin/spark --jars rapids-4-spark_2.12-22.10.0-SNAPSHOT-cuda11.jar \ |--conf spark.plugins=com.nvidia.spark.SQLPlugin \ |--conf spark.rapids.sql.incompatibleOps.enabled=true |``` diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala index bcdc0b1f997..a1b61c13b15 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala @@ -55,13 +55,13 @@ import org.apache.spark.util.MutableURLClassLoader E.g., Spark 3.2.0 Shim will use - jar:file:/home/spark/rapids-4-spark_2.12-22.08.0.jar!/spark3xx-common/ - jar:file:/home/spark/rapids-4-spark_2.12-22.08.0.jar!/spark320/ + jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/spark3xx-common/ + jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/spark320/ Spark 3.1.1 will use - jar:file:/home/spark/rapids-4-spark_2.12-22.08.0.jar!/spark3xx-common/ - jar:file:/home/spark/rapids-4-spark_2.12-22.08.0.jar!/spark311/ + jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/spark3xx-common/ + jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/spark311/ Using these Jar URL's allows referencing different bytecode produced from identical sources by incompatible Scala / Spark dependencies. diff --git a/tests/pom.xml b/tests/pom.xml index cb39efab5b7..590dff20430 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -22,12 +22,12 @@ com.nvidia rapids-4-spark-parent - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT rapids-4-spark-tests_2.12 RAPIDS Accelerator for Apache Spark Tests RAPIDS plugin for Apache Spark integration tests - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT diff --git a/tools/pom.xml b/tools/pom.xml index 3cd93c200b0..b67097410c5 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -22,13 +22,13 @@ com.nvidia rapids-4-spark-parent - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT com.nvidia rapids-4-spark-tools_2.12 RAPIDS Accelerator for Apache Spark tools RAPIDS Accelerator for Apache Spark tools - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT jar diff --git a/udf-compiler/pom.xml b/udf-compiler/pom.xml index e847505f62c..76de8bee583 100644 --- a/udf-compiler/pom.xml +++ b/udf-compiler/pom.xml @@ -22,12 +22,12 @@ com.nvidia rapids-4-spark-parent - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT rapids-4-spark-udf_2.12 RAPIDS Accelerator for Apache Spark Scala UDF Plugin The RAPIDS Scala UDF plugin for Apache Spark - 22.08.0-SNAPSHOT + 22.10.0-SNAPSHOT From fe56df82d4741ab92b3eee46a8bcd1356426aefc Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Mon, 1 Aug 2022 09:21:06 +0800 Subject: [PATCH 002/190] Add tests for .count() in the file readers (#6055) * Added tests for .count() in the file readers * Removed unused import in file read count tests * Refactored to assert_gpu_and_cpu_row_counts_equal * Code style changes * Code style changes * Add tests to assert spark plan * Code style changes * Refactored some test cases * Added comments * Add json test * Remove unused import Signed-off-by: thirtiseven --- integration_tests/src/main/python/csv_test.py | 12 +++++++++++- integration_tests/src/main/python/json_test.py | 15 ++++++++++++++- integration_tests/src/main/python/orc_test.py | 17 ++++++++++++++++- .../src/main/python/parquet_test.py | 16 +++++++++++++++- 4 files changed, 56 insertions(+), 4 deletions(-) diff --git a/integration_tests/src/main/python/csv_test.py b/integration_tests/src/main/python/csv_test.py index 70168757e7b..272b4fd777d 100644 --- a/integration_tests/src/main/python/csv_test.py +++ b/integration_tests/src/main/python/csv_test.py @@ -14,7 +14,7 @@ import pytest -from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error, assert_gpu_fallback_write, \ +from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error, assert_gpu_and_cpu_row_counts_equal, assert_gpu_fallback_write, \ assert_cpu_and_gpu_are_equal_collect_with_capture, assert_gpu_fallback_collect from conftest import get_non_gpu_allowed from datetime import datetime, timezone @@ -540,3 +540,13 @@ def test_csv_read_case_insensitivity(spark_tmp_path): lambda spark: spark.read.option('header', True).csv(data_path).select('one', 'two', 'three'), {'spark.sql.caseSensitive': 'false'} ) + +@allow_non_gpu('FileSourceScanExec', 'CollectLimitExec', 'DeserializeToObjectExec') +def test_csv_read_count(spark_tmp_path): + data_gens = [byte_gen, short_gen, int_gen, long_gen, boolean_gen, date_gen] + gen_list = [('_c' + str(i), gen) for i, gen in enumerate(data_gens)] + data_path = spark_tmp_path + '/CSV_DATA' + + with_cpu_session(lambda spark: gen_df(spark, gen_list).write.csv(data_path)) + + assert_gpu_and_cpu_row_counts_equal(lambda spark: spark.read.csv(data_path)) diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index 94c6c604d62..6fe5e800291 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -14,7 +14,7 @@ import pytest -from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error, assert_gpu_fallback_collect +from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error, assert_gpu_and_cpu_row_counts_equal, assert_gpu_fallback_collect from data_gen import * from conftest import is_databricks_runtime from marks import approximate_float, allow_non_gpu, ignore_order @@ -346,3 +346,16 @@ def test_json_read_with_corrupt_files(spark_tmp_path, v1_enabled_list): .json([first_data_path, second_data_path, third_data_path]) .collect(), conf=all_confs) + +@pytest.mark.parametrize('v1_enabled_list', ["", "json"]) +def test_json_read_count(spark_tmp_path, v1_enabled_list): + gen_list = [byte_gen, short_gen, int_gen, long_gen, boolean_gen] + gen = StructGen([('_c' + str(i), gen) for i, gen in enumerate(gen_list)], nullable=False) + data_path = spark_tmp_path + '/JSON_DATA' + schema = gen.data_type + updated_conf = copy_and_update(_enable_all_types_conf, {'spark.sql.sources.useV1SourceList': v1_enabled_list}) + with_cpu_session( + lambda spark : gen_df(spark, gen).write.json(data_path)) + assert_gpu_and_cpu_row_counts_equal( + lambda spark : spark.read.schema(schema).json(data_path), + conf=updated_conf) diff --git a/integration_tests/src/main/python/orc_test.py b/integration_tests/src/main/python/orc_test.py index 83679a09cb0..cf58ca053eb 100644 --- a/integration_tests/src/main/python/orc_test.py +++ b/integration_tests/src/main/python/orc_test.py @@ -14,7 +14,8 @@ import pytest -from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_fallback_collect, assert_cpu_and_gpu_are_equal_collect_with_capture +from asserts import assert_cpu_and_gpu_are_equal_sql_with_capture, assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_row_counts_equal, assert_gpu_fallback_collect, \ + assert_cpu_and_gpu_are_equal_collect_with_capture from data_gen import * from marks import * from pyspark.sql.types import * @@ -663,3 +664,17 @@ def test_read_type_casting_integral(spark_tmp_path, offset, reader_confs, v1_ena assert_gpu_and_cpu_are_equal_collect( lambda spark: spark.read.schema(rs).orc(data_path), conf=all_confs) + +def test_orc_read_count(spark_tmp_path): + data_path = spark_tmp_path + '/ORC_DATA' + orc_gens = [int_gen, string_gen, double_gen] + gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gens)] + + with_cpu_session(lambda spark: gen_df(spark, gen_list).write.orc(data_path)) + + assert_gpu_and_cpu_row_counts_equal(lambda spark: spark.read.orc(data_path)) + + # assert the spark plan of the equivalent SQL query contains no column in read schema + assert_cpu_and_gpu_are_equal_sql_with_capture( + lambda spark: spark.read.orc(data_path), "SELECT COUNT(*) FROM tab", "tab", + exist_classes=r'GpuFileGpuScan orc .* ReadSchema: struct<>') diff --git a/integration_tests/src/main/python/parquet_test.py b/integration_tests/src/main/python/parquet_test.py index aed92bb6a9e..da166118e3d 100644 --- a/integration_tests/src/main/python/parquet_test.py +++ b/integration_tests/src/main/python/parquet_test.py @@ -15,7 +15,7 @@ import pytest -from asserts import assert_cpu_and_gpu_are_equal_collect_with_capture, assert_gpu_and_cpu_are_equal_collect, \ +from asserts import assert_cpu_and_gpu_are_equal_collect_with_capture, assert_cpu_and_gpu_are_equal_sql_with_capture, assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_row_counts_equal, \ assert_gpu_fallback_collect, assert_gpu_and_cpu_are_equal_sql, assert_gpu_and_cpu_error, assert_py4j_exception from data_gen import * from marks import * @@ -1145,3 +1145,17 @@ def test_parquet_read_encryption(spark_tmp_path, reader_confs, v1_enabled_list): lambda: with_gpu_session( lambda spark: spark.read.parquet(data_path).collect(), conf=conf), error_message='The GPU does not support reading encrypted Parquet files') + +def test_parquet_read_count(spark_tmp_path): + parquet_gens = [int_gen, string_gen, double_gen] + gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] + data_path = spark_tmp_path + '/PARQUET_DATA' + + with_cpu_session(lambda spark: gen_df(spark, gen_list).write.parquet(data_path)) + + assert_gpu_and_cpu_row_counts_equal(lambda spark: spark.read.parquet(data_path)) + + # assert the spark plan of the equivalent SQL query contains no column in read schema + assert_cpu_and_gpu_are_equal_sql_with_capture( + lambda spark: spark.read.parquet(data_path), "SELECT COUNT(*) FROM tab", "tab", + exist_classes=r'GpuFileGpuScan parquet .* ReadSchema: struct<>') From 1c70a348102eeeb23f30e1ec5e81bd76a3126362 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Mon, 1 Aug 2022 10:28:55 -0700 Subject: [PATCH 003/190] Remove hard-coded versions from buildall [skip ci] (#6162) Use Maven profiles to extract the Spark version list to build Shims for. Signed-off-by: Gera Shegalov --- build/buildall | 43 ++++++++++++++----------------------------- 1 file changed, 14 insertions(+), 29 deletions(-) diff --git a/build/buildall b/build/buildall index 95cd0b59a72..73d39bee94b 100755 --- a/build/buildall +++ b/build/buildall @@ -89,6 +89,13 @@ function bloopInstall() { ) } +function versionsFromDistProfile() { + [[ BUILD_ALL_DEBUG == "1" ]] && set -x + versionRawStr=$(mvn -B help:evaluate -q -pl dist -P"$1" -Dexpression=included_buildvers -DforceStdout) + versionStr=${versionRawStr//[$'\n',]/} + echo -n $versionStr +} + FINAL_OP="package" while [[ "$1" != "" ]] ; do @@ -118,6 +125,7 @@ case "$1" in --debug) set -x + BUILD_ALL_DEBUG="1" ;; --clean) @@ -150,40 +158,15 @@ DIST_PROFILE=${DIST_PROFILE:-"noSnapshots"} case $DIST_PROFILE in snapshots?(WithDatabricks)) - SPARK_SHIM_VERSIONS=( - 311 - 321cdh - 312 - 313 - 314 - 320 - 321 - 322 - 330 - 331 - ) + SPARK_SHIM_VERSIONS=($(versionsFromDistProfile "snapshots")) ;; noSnapshots?(WithDatabricks)) - SPARK_SHIM_VERSIONS=( - 311 - 321cdh - 312 - 313 - 320 - 321 - 322 - 330 - ) + SPARK_SHIM_VERSIONS=($(versionsFromDistProfile "noSnapshots")) ;; minimumFeatureVersionMix) - SPARK_SHIM_VERSIONS=( - 321cdh - 312 - 320 - 330 - ) + SPARK_SHIM_VERSIONS=($(versionsFromDistProfile "minimumFeatureVersionMix")) ;; 3*) @@ -198,6 +181,8 @@ case $DIST_PROFILE in esac +echo "Spark versions involved: ${SPARK_SHIM_VERSIONS[@]} ..." + if [[ "$GEN_BLOOP" == "true" ]]; then bloopInstall exit 0 @@ -220,7 +205,7 @@ echo "Building a combined dist jar with Shims for ${SPARK_SHIM_VERSIONS[@]} ..." export MVN_BASE_DIR=$($MVN help:evaluate -Dexpression=project.basedir -q -DforceStdout) function build_single_shim() { - set -x + [[ BUILD_ALL_DEBUG == "1" ]] && set -x BUILD_VER=$1 mkdir -p "$MVN_BASE_DIR/target" (( BUILD_PARALLEL == 1 || NUM_SHIMS == 1 )) && LOG_FILE="/dev/tty" || \ From f82c55a3075eb0aa2215cf6fc49ac6053c94385c Mon Sep 17 00:00:00 2001 From: sinkinben Date: Tue, 2 Aug 2022 17:50:47 +0800 Subject: [PATCH 004/190] GpuGlobalLimitExec and GpuCollectLimitExec support offset (#5946) Support `offset` parameter (which is introduced in spark-3.4.0) in `GpuGlobalLimit` and `GpuCollectLimit`. --- .../src/main/python/limit_test.py | 63 ++++++---- .../com/nvidia/spark/rapids/SparkShims.scala | 12 -- .../spark/rapids/shims/GlobalLimitShims.scala | 19 ++- .../scala/com/nvidia/spark/rapids/limit.scala | 109 +++++++++++------- 4 files changed, 128 insertions(+), 75 deletions(-) diff --git a/integration_tests/src/main/python/limit_test.py b/integration_tests/src/main/python/limit_test.py index 0be13746f22..ded8c3cf629 100644 --- a/integration_tests/src/main/python/limit_test.py +++ b/integration_tests/src/main/python/limit_test.py @@ -14,35 +14,58 @@ import pytest -from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_fallback_collect +from asserts import assert_gpu_and_cpu_are_equal_collect from data_gen import * -from marks import allow_non_gpu from spark_session import is_before_spark_340 @pytest.mark.parametrize('data_gen', all_basic_gens + decimal_gens + array_gens_sample + map_gens_sample + struct_gens_sample, ids=idfn) def test_simple_limit(data_gen): assert_gpu_and_cpu_are_equal_collect( - # We need some processing after the limit to avoid a CollectLimitExec - lambda spark : unary_op_df(spark, data_gen, num_slices=1).limit(10).repartition(1), - conf = {'spark.sql.execution.sortBeforeRepartition': 'false'}) + # We need some processing after the limit to avoid a CollectLimitExec + lambda spark : unary_op_df(spark, data_gen, num_slices=1).limit(10).repartition(1), + conf = {'spark.sql.execution.sortBeforeRepartition': 'false'}) + + +def offset_test_wrapper(sql): + conf = {'spark.rapids.sql.exec.CollectLimitExec': 'true'} + + # Create dataframe to test CollectLimit + def spark_df(spark): + unary_op_df(spark, int_gen, length=2048, num_slices=1).createOrReplaceTempView("tmp_table") + return spark.sql(sql) + assert_gpu_and_cpu_are_equal_collect(spark_df, conf) + + # Create dataframe to test GlobalLimit + def spark_df_repartition(spark): + return spark_df(spark).repartition(1) + assert_gpu_and_cpu_are_equal_collect(spark_df_repartition, conf) -@allow_non_gpu('CollectLimitExec', 'GlobalLimitExec', 'ShuffleExchangeExec') +@pytest.mark.parametrize('offset', [1024, 2048, 4096]) @pytest.mark.skipif(is_before_spark_340(), reason='offset is introduced from Spark 3.4.0') -def test_non_zero_offset_fallback(): - def test_fn(spark): - unary_op_df(spark, int_gen, num_slices=1).createOrReplaceTempView("offset_tmp") - # `offset` is not exposed to Pyspark, so use the sql string. - return spark.sql("select * from offset_tmp limit 10 offset 1") +def test_non_zero_offset(offset): + # offset is used in the test cases having no limit, that is limit = -1 + # 1024: offset < df.numRows + # 2048: offset = df.numRows + # 4096: offset > df.numRows - assert_gpu_fallback_collect( - # We need some processing after the limit to avoid a CollectLimitExec - lambda spark: test_fn(spark).repartition(1), - 'GlobalLimitExec', - conf = {'spark.sql.execution.sortBeforeRepartition': 'false'}) + sql = "select * from tmp_table offset {}".format(offset) + offset_test_wrapper(sql) - assert_gpu_fallback_collect( - test_fn, - 'CollectLimitExec', - conf = {'spark.sql.execution.sortBeforeRepartition': 'false'}) + +@pytest.mark.parametrize('limit, offset', [(0, 0), (0, 10), (1024, 500), (2048, 456), (3000, 111), (500, 500), (100, 600)]) +@pytest.mark.skipif(is_before_spark_340(), reason='offset is introduced from Spark 3.4.0') +def test_non_zero_offset_with_limit(limit, offset): + # In CPU version of spark, (limit, offset) can not be negative number. + # Test case description: + # (0, 0): Corner case: both limit and offset are 0 + # (0, 10): Corner case: limit = 0, offset > 0 + # (1024, 500): offset < limit && limit < df.numRows + # (2048, 456): offset < limit && limit = df.numRows + # (3000, 111): offset < limit && limit > df.numRows + # (500, 500): offset = limit + # (100, 600): offset > limit + + sql = "select * from tmp_table limit {} offset {}".format(limit, offset) + offset_test_wrapper(sql) diff --git a/sql-plugin/src/main/340/scala/com/nvidia/spark/rapids/SparkShims.scala b/sql-plugin/src/main/340/scala/com/nvidia/spark/rapids/SparkShims.scala index 829e5576260..b472234b181 100644 --- a/sql-plugin/src/main/340/scala/com/nvidia/spark/rapids/SparkShims.scala +++ b/sql-plugin/src/main/340/scala/com/nvidia/spark/rapids/SparkShims.scala @@ -35,12 +35,6 @@ object SparkShimImpl extends Spark330PlusShims { TypeSig.all), (globalLimit, conf, p, r) => new SparkPlanMeta[GlobalLimitExec](globalLimit, conf, p, r) { - override def tagPlanForGpu(): Unit = { - if (globalLimit.offset != 0) { - willNotWorkOnGpu("non-zero offset is not supported") - } - } - override def convertToGpu(): GpuExec = GpuGlobalLimitExec( globalLimit.limit, childPlans.head.convertIfNeeded(), globalLimit.offset) @@ -51,12 +45,6 @@ object SparkShimImpl extends Spark330PlusShims { TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP).nested(), TypeSig.all), (collectLimit, conf, p, r) => new GpuCollectLimitMeta(collectLimit, conf, p, r) { - override def tagPlanForGpu(): Unit = { - if (collectLimit.offset != 0) { - willNotWorkOnGpu("non-zero offset is not supported") - } - } - override def convertToGpu(): GpuExec = GpuGlobalLimitExec(collectLimit.limit, GpuShuffleExchangeExec( diff --git a/sql-plugin/src/main/340/scala/com/nvidia/spark/rapids/shims/GlobalLimitShims.scala b/sql-plugin/src/main/340/scala/com/nvidia/spark/rapids/shims/GlobalLimitShims.scala index 5011111d0a9..f12809f78e2 100644 --- a/sql-plugin/src/main/340/scala/com/nvidia/spark/rapids/shims/GlobalLimitShims.scala +++ b/sql-plugin/src/main/340/scala/com/nvidia/spark/rapids/shims/GlobalLimitShims.scala @@ -26,10 +26,21 @@ object GlobalLimitShims { * Estimate the number of rows for a GlobalLimitExec. */ def visit(plan: SparkPlanMeta[GlobalLimitExec]): Option[BigInt] = { - // offset is ignored now, but we should support it. - // See https://github.com/NVIDIA/spark-rapids/issues/5589 - //val offset = plan.wrapped.limit + // offset is introduce in spark-3.4.0, and it ensures that offset >= 0. And limit can be -1, + // such case happens only when we execute sql like 'select * from table offset 10' + val offset = plan.wrapped.offset val limit = plan.wrapped.limit - RowCountPlanVisitor.visit(plan.childPlans.head).map(_.min(limit)).orElse(Some(limit)) + val sliced = if (limit >= 0) { + Some(BigInt(limit - offset).max(0)) + } else { + // limit can be -1, meaning no limit + None + } + RowCountPlanVisitor.visit(plan.childPlans.head) + .map { rowNum => + val remaining = (rowNum - offset).max(0) + sliced.map(_.min(remaining)).getOrElse(remaining) + } + .orElse(sliced) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/limit.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/limit.scala index 6249b2e1f69..bb7fbf0d19e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/limit.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/limit.scala @@ -58,24 +58,60 @@ trait GpuBaseLimitExec extends LimitExec with GpuExec with ShimUnaryExecNode { throw new IllegalStateException(s"Row-based execution should not occur for $this") override def doExecuteColumnar(): RDD[ColumnarBatch] = { + sliceRDD(child.executeColumnar(), limit, 0) + } + + def sliceRDD(rdd: RDD[ColumnarBatch], limit: Int, offset: Int): RDD[ColumnarBatch] = { + val opTime = gpuLongMetric(OP_TIME) val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) - val opTime = gpuLongMetric(OP_TIME) - - val crdd = child.executeColumnar() - crdd.mapPartitions { cbIter => + rdd.mapPartitions { iter => new Iterator[ColumnarBatch] { - private var remainingLimit = limit + private var remainingLimit = limit - offset + private var remainingOffset = offset - override def hasNext: Boolean = remainingLimit > 0 && cbIter.hasNext + override def hasNext: Boolean = (limit == -1 || remainingLimit > 0) && iter.hasNext override def next(): ColumnarBatch = { - val batch = cbIter.next() - withResource(new NvtxWithMetrics("limit", NvtxColor.ORANGE, opTime)) { _ => - val result = if (batch.numRows() > remainingLimit) { - sliceBatch(batch) + if (!this.hasNext) { + throw new NoSuchElementException("Next on empty iterator") + } + + var batch = iter.next() + val numCols = batch.numCols() + + // In each partition, we need to skip `offset` rows + while (batch != null && remainingOffset >= batch.numRows()) { + remainingOffset -= batch.numRows() + batch.safeClose() + batch = if (this.hasNext) { iter.next() } else { null } + } + + // If the last batch is null, then we have offset >= numRows in this partition. + // In such case, we should return an empty batch + if (batch == null || batch.numRows() == 0) { + return new ColumnarBatch(new ArrayBuffer[GpuColumnVector](numCols).toArray, 0) + } + + // Here 0 <= remainingOffset < batch.numRow(), we need to get batch[remainingOffset:] + withResource(new NvtxWithMetrics("limit and offset", NvtxColor.ORANGE, opTime)) { _ => + var result: ColumnarBatch = null + // limit < 0 (limit == -1) denotes there is no limitation, so when + // (remainingOffset == 0 && (remainingLimit >= batch.numRows() || limit < 0)) is true, + // we can take this batch completely + if (remainingOffset == 0 && (remainingLimit >= batch.numRows() || limit < 0)) { + result = batch } else { - batch + // otherwise, we need to slice batch with (remainingOffset, remainingLimit). + // And remainingOffset > 0 will be used only once, for the latter batches in this + // partition, set remainingOffset = 0 + val length = if (remainingLimit >= batch.numRows() || limit < 0) { + batch.numRows() + } else { + remainingLimit + } + result = sliceBatchWithOffset(batch, remainingOffset, length) + remainingOffset = 0 } numOutputBatches += 1 numOutputRows += result.numRows() @@ -84,35 +120,23 @@ trait GpuBaseLimitExec extends LimitExec with GpuExec with ShimUnaryExecNode { } } - def sliceBatch(batch: ColumnarBatch): ColumnarBatch = { - val numColumns = batch.numCols() - val resultCVs = new ArrayBuffer[GpuColumnVector](numColumns) - var exception: Throwable = null - var table: Table = null - try { - if (numColumns > 0) { - table = GpuColumnVector.from(batch) - (0 until numColumns).zip(output).foreach{ case (i, attr) => - val subVector = table.getColumn(i).subVector(0, remainingLimit) - assert(subVector != null) - resultCVs.append(GpuColumnVector.from(subVector, attr.dataType)) - assert(subVector.getRowCount == remainingLimit, - s"result rowcount ${subVector.getRowCount} is not equal to the " + - s"remainingLimit $remainingLimit") + def sliceBatchWithOffset(batch: ColumnarBatch, offset: Int, limit: Int): ColumnarBatch = { + val numCols = batch.numCols() + val end = Math.min(offset + limit, batch.numRows()) + withResource(batch) { _ => + // result buffer need to be closed when there is an exception + closeOnExcept(new ArrayBuffer[GpuColumnVector](numCols)) { result => + if (numCols > 0) { + withResource(GpuColumnVector.from(batch)) { table => + (0 until numCols).zip(output).foreach{ case (i, attr) => + val subVector = table.getColumn(i).subVector(offset, end) + assert(subVector != null) + result.append(GpuColumnVector.from(subVector, attr.dataType)) + } + } } + new ColumnarBatch(result.toArray, end - offset) } - new ColumnarBatch(resultCVs.toArray, remainingLimit) - } catch { - case e: Throwable => exception = e - throw e - } finally { - if (exception != null) { - resultCVs.foreach(gpuVector => gpuVector.safeClose(exception)) - } - if (table != null) { - table.safeClose(exception) - } - batch.safeClose(exception) } } } @@ -128,8 +152,15 @@ case class GpuLocalLimitExec(limit: Int, child: SparkPlan) extends GpuBaseLimitE /** * Take the first `limit` elements of the child's single output partition. */ -case class GpuGlobalLimitExec(limit: Int, child: SparkPlan, offset: Int) extends GpuBaseLimitExec { +case class GpuGlobalLimitExec(limit: Int = -1, child: SparkPlan, + offset: Int = 0) extends GpuBaseLimitExec { + // In CPU code of spark, there is an assertion 'limit >= 0 || (limit == -1 && offset > 0)'. + override def requiredChildDistribution: List[Distribution] = AllTuples :: Nil + + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + super.sliceRDD(child.executeColumnar(), limit, offset) + } } class GpuCollectLimitMeta( From fd50d6e4e34e628fc965dd4e58c567942f16cd67 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Tue, 2 Aug 2022 07:37:57 -0700 Subject: [PATCH 005/190] Fix Bloop project generation in buildall [skip ci] (#6156) This PR fixes #5706 - Generates a Bloop project for each Spark dependency specified via `--profile` - Explains how to activate a particular project - No longer tries to have multiple Spark dependencies in the same .bloop directory - Fixes incorrect BUILD_ALL_DEBUG tests - Fixes match for .bloop in gitignore Signed-off-by: Gera Shegalov --- .gitignore | 2 +- CONTRIBUTING.md | 12 +++++++++++ build/buildall | 53 ++++++++++++++++++++++++------------------------- 3 files changed, 39 insertions(+), 28 deletions(-) diff --git a/.gitignore b/.gitignore index 17413a26cf9..867c8af50aa 100644 --- a/.gitignore +++ b/.gitignore @@ -1,7 +1,7 @@ # keep the lines below sorted # standard .bloop and version-specific .bloop3xy generated by buildall -gb -.bloop*/ +.bloop* .cache .classpath .DS_Store diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 150d9e56f39..0fe64e16ddc 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -218,9 +218,21 @@ not clobbered by repeated `bloopInstall` Maven plugin invocations, and it uses [jq](https://stedolan.github.io/jq/) to post-process JSON-formatted project files such that they compile project classes into non-overlapping set of output directories. +To activate the Spark dependency version 3XY you currently are working with update +the symlink `.bloop` to point to the corresponding directory `.bloop-spark3XY` + +Example usage: +```Bash +./build/buildall --generate-bloop --profile=311,330 +rm -vf .bloop +ln -s .bloop-spark330 .bloop +``` + You can now open the spark-rapids as a [BSP project in IDEA](https://www.jetbrains.com/help/idea/bsp-support.html) +Read on for VS Code Scala Metals instructions. + # Bloop, Scala Metals, and Visual Studio Code _Last tested with 1.63.0-insider (Universal) Commit: bedf867b5b02c1c800fbaf4d6ce09cefba_ diff --git a/build/buildall b/build/buildall index 73d39bee94b..fc3ddb85fe5 100755 --- a/build/buildall +++ b/build/buildall @@ -22,6 +22,7 @@ shopt -s extglob BLOOP_VERSION=${BLOOP_VERSION:-"1.4.13"} BLOOP_SCALA_VERSION=${BLOOP_SCALA_VERSION:-"2.13"} SKIP_CLEAN=1 +BUILD_ALL_DEBUG=0 function print_usage() { echo "Usage: buildall [OPTION]" @@ -52,18 +53,20 @@ function print_usage() { } function bloopInstall() { - BLOOP_DIR="${BLOOP_DIR:-$PWD/.bloop}" - mkdir -p $BLOOP_DIR - rm -f $BLOOP_DIR/* - time ( - for bv in $SPARK_SHIM_VERSIONS; do - bloop_config_dir="$PWD/.bloop$bv" - mkdir -p "$bloop_config_dir" - rm -f "$bloop_config_dir"/* + [[ "$BUILD_ALL_DEBUG" == "1" ]] && set -x + + local bloopTmpDir=$(mktemp -d /tmp/tmp.bloop.XXXXXX) - $MVN install ch.epfl.scala:maven-bloop_${BLOOP_SCALA_VERSION}:${BLOOP_VERSION}:bloopInstall -pl dist -am \ - -Dbloop.configDirectory="$bloop_config_dir" \ + time ( + bloopDirsGenerated=() + for bv in "${SPARK_SHIM_VERSIONS[@]}"; do + bloopTmpConfigDir="$bloopTmpDir/.bloop$bv" + mkdir -p $bloopTmpConfigDir + $MVN -B clean install \ + ch.epfl.scala:maven-bloop_${BLOOP_SCALA_VERSION}:${BLOOP_VERSION}:bloopInstall \ + -pl aggregator -am \ + -Dbloop.configDirectory="$bloopTmpConfigDir" \ -DdownloadSources=true \ -Dbuildver="$bv" \ -DskipTests \ @@ -73,24 +76,21 @@ function bloopInstall() { -Dmaven.updateconfig.skip=true specifier="spark$bv" - for bloop_json in $(echo $bloop_config_dir/*.json); do - IFS="/" <<< "$bloop_json" read -ra bloop_json_parts - last_idx=$((${#bloop_json_parts[@]} - 1)) - file="${bloop_json_parts[$last_idx]}" - project="${file%.json}-$specifier" - < $bloop_json jq \ - --arg specifier "$specifier" \ - '.project.out=.project.out + "/" + $specifier | .project.name=.project.name + "-" + $specifier' \ - > "$BLOOP_DIR/$project.json" - done + bloopDir=$PWD/.bloop-$specifier + rm -rf $bloopDir + mv $bloopTmpConfigDir $bloopDir + echo "generated bloop files under $bloopDir" + bloopDirsGenerated+=($bloopDir) done - - echo "Generated Bloop files under $BLOOP_DIR" + echo "#### Created bloop projects ${bloopDirsGenerated[@]}" + echo "Execute" + echo " ln -s .bloop-spark3XY .bloop" + echo "to make it an active Bloop project in VS Code Scala Metals" ) } function versionsFromDistProfile() { - [[ BUILD_ALL_DEBUG == "1" ]] && set -x + [[ "$BUILD_ALL_DEBUG" == "1" ]] && set -x versionRawStr=$(mvn -B help:evaluate -q -pl dist -P"$1" -Dexpression=included_buildvers -DforceStdout) versionStr=${versionRawStr//[$'\n',]/} echo -n $versionStr @@ -124,8 +124,8 @@ case "$1" in ;; --debug) + BUILD_ALL_DEBUG=1 set -x - BUILD_ALL_DEBUG="1" ;; --clean) @@ -182,6 +182,7 @@ case $DIST_PROFILE in esac echo "Spark versions involved: ${SPARK_SHIM_VERSIONS[@]} ..." +export MVN_BASE_DIR=$($MVN help:evaluate -Dexpression=project.basedir -q -DforceStdout) if [[ "$GEN_BLOOP" == "true" ]]; then bloopInstall @@ -202,10 +203,8 @@ fi echo "Building a combined dist jar with Shims for ${SPARK_SHIM_VERSIONS[@]} ..." -export MVN_BASE_DIR=$($MVN help:evaluate -Dexpression=project.basedir -q -DforceStdout) - function build_single_shim() { - [[ BUILD_ALL_DEBUG == "1" ]] && set -x + [[ "$BUILD_ALL_DEBUG" == "1" ]] && set -x BUILD_VER=$1 mkdir -p "$MVN_BASE_DIR/target" (( BUILD_PARALLEL == 1 || NUM_SHIMS == 1 )) && LOG_FILE="/dev/tty" || \ From 5d50135b7d9a4deb3087029fe76c20a0d1a108e3 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Tue, 2 Aug 2022 10:49:44 -0700 Subject: [PATCH 006/190] Improve coverage in mvn verify check github workflow (#6152) Addresses the issue slipped into #6141 . Workflow improvements: - Dynamically generate builds for all shims without hardcoding versions using https://docs.github.com/en/actions/learn-github-actions/expressions#fromjson - Imitate `buildall`. Verify shim-specific code is buildable in parallel. In another parallel check RAT, scalastyle, and verify all modules with the default spark version - add default value `false` for maven.scalastyle.skip Signed-off-by: Gera Shegalov --- .github/workflows/mvn-verify-check.yml | 68 ++++++++++++++++++++++++-- pom.xml | 1 + 2 files changed, 65 insertions(+), 4 deletions(-) diff --git a/.github/workflows/mvn-verify-check.yml b/.github/workflows/mvn-verify-check.yml index 1742497151c..44038eee414 100644 --- a/.github/workflows/mvn-verify-check.yml +++ b/.github/workflows/mvn-verify-check.yml @@ -13,7 +13,7 @@ # limitations under the License. # A workflow to run mvn verify check -name: Maven verify checks (Scala style, Compile and Doc-gen w/ base Spark version) +name: mvn[compile,RAT,scalastyle,docgen] on: pull_request: @@ -24,7 +24,61 @@ concurrency: cancel-in-progress: true jobs: - mvn-verify-check: + get-noSnapshot-versions-from-dist: + runs-on: ubuntu-latest + outputs: + sparkHeadVersion: ${{ steps.noSnapshotVersionsStep.outputs.headVersion }} + sparkTailVersions: ${{ steps.noSnapshotVersionsStep.outputs.tailVersions }} + steps: + - uses: actions/checkout@v2 # refs/pull/:prNumber/merge + + - name: Setup Java and Maven Env + uses: actions/setup-java@v3 + with: + distribution: adopt + java-version: 8 + + - name: all noSnapshot versions + id: noSnapshotVersionsStep + run: | + set -x + noSnapshotVersionsStr=$(mvn -B help:evaluate -q -pl dist -PnoSnapshots -Dexpression=included_buildvers -DforceStdout) + noSnapshotVersionsStr=$(echo $noSnapshotVersionsStr) + noSnapshotVersionsArr=($(IFS=", "; echo $noSnapshotVersionsStr)) + tailNoSnapshotVersionsArr=(${noSnapshotVersionsArr[@]:1}) + svArrBody=$(printf ",{\"spark-version\":\"%s\"}" "${tailNoSnapshotVersionsArr[@]}") + svArrBody=${svArrBody:1} + svJsonStr=$(printf {\"include\":[%s]} $svArrBody) + echo ::set-output name=headVersion::${noSnapshotVersionsArr[0]} + echo ::set-output name=tailVersions::$svJsonStr + + package-aggregator: + needs: get-noSnapshot-versions-from-dist + strategy: + matrix: ${{ fromJSON(needs.get-noSnapshot-versions-from-dist.outputs.sparkTailVersions) }} + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v2 # refs/pull/:prNumber/merge + + - name: Setup Java and Maven Env + uses: actions/setup-java@v3 + with: + distribution: adopt + java-version: 8 + + - name: package aggregator check + run: > + mvn -B package -pl aggregator -am + -P 'individual,pre-merge' + -Dbuildver=${{ matrix.spark-version }} + -DskipTests + -Dskip + -Dmaven.javadoc.skip + -Dmaven.scalastyle.skip=true + -Drat.skip=true + + verify-all-modules-with-headSparkVersion: + needs: get-noSnapshot-versions-from-dist runs-on: ubuntu-latest steps: - uses: actions/checkout@v2 # refs/pull/:prNumber/merge @@ -36,5 +90,11 @@ jobs: java-version: 8 # includes RAT, code style and doc-gen checks of default shim - - name: mvn verify check - run: mvn verify -P 'individual,pre-merge' -pl dist -am -DskipTests -Dskip -Dmaven.javadoc.skip + - name: verify all modules with lowest-supported Spark version + run: > + mvn -B verify + -P 'individual,pre-merge' + -Dbuildver=${{ needs.get-noSnapshot-versions-from-dist.outputs.sparkHeadVersion }} + -DskipTests + -Dskip + -Dmaven.javadoc.skip diff --git a/pom.xml b/pom.xml index bbf7e16db82..cf435d29dfa 100644 --- a/pom.xml +++ b/pom.xml @@ -1039,6 +1039,7 @@ org/scala-lang/scala-library/${scala.version}/scala-library-${scala.version}.jar ${spark.version.classifier} 3.1.0 + false From 8d6f6f5cc115e44d3b6afcdf85f4207f35912a69 Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Thu, 4 Aug 2022 08:59:42 -0500 Subject: [PATCH 007/190] Disable overflow tableInsert tests for 331+ (#6218) Signed-off-by: Ahmed Hussein (amahussein) --- .../nvidia/spark/rapids/AnsiCastOpSuite.scala | 69 +++++++++++-------- 1 file changed, 41 insertions(+), 28 deletions(-) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/AnsiCastOpSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/AnsiCastOpSuite.scala index def98c9e5d0..bd39a17fc24 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/AnsiCastOpSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/AnsiCastOpSuite.scala @@ -559,6 +559,11 @@ class AnsiCastOpSuite extends GpuExpressionTestSuite { // Writing to Hive tables, which has special rules /////////////////////////////////////////////////////////////////////////// + // Disable tests for Spark-3.3.1+ until new support to CheckOverflowInTableInsert is added + def before3_3_1(s: SparkSession): (Boolean, String) = { + (s.version < "3.3.1", s"Spark version must be prior to 3.3.1") + } + testSparkResultsAreEqual("Write bytes to string", testBytes, sparkConf) { frame => doTableInsert(frame, HIVE_STRING_SQL_TYPE) } @@ -580,72 +585,72 @@ class AnsiCastOpSuite extends GpuExpressionTestSuite { } testSparkResultsAreEqual("Write longs to int (values within range)", intsAsLongs, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_INT_SQL_TYPE) } testSparkResultsAreEqual("Write longs to short (values within range)", shortsAsLongs, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_SHORT_SQL_TYPE) } testSparkResultsAreEqual("Write longs to byte (values within range)", bytesAsLongs, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } testSparkResultsAreEqual("Write ints to short (values within range)", shortsAsInts, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_SHORT_SQL_TYPE) } testSparkResultsAreEqual("Write ints to byte (values within range)", bytesAsInts, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } testSparkResultsAreEqual("Write shorts to byte (values within range)", bytesAsShorts, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } testSparkResultsAreEqual("Write floats to long (values within range)", longsAsFloats, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_LONG_SQL_TYPE) } testSparkResultsAreEqual("Write floats to int (values within range)", intsAsFloats, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_INT_SQL_TYPE) } testSparkResultsAreEqual("Write floats to short (values within range)", shortsAsFloats, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_SHORT_SQL_TYPE) } testSparkResultsAreEqual("Write floats to byte (values within range)", bytesAsFloats, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } testSparkResultsAreEqual("Write doubles to long (values within range)", longsAsDoubles, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_LONG_SQL_TYPE) } testSparkResultsAreEqual("Write doubles to int (values within range)", intsAsDoubles, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_LONG_SQL_TYPE) } testSparkResultsAreEqual("Write doubles to short (values within range)", - shortsAsDoubles, sparkConf) { + shortsAsDoubles, sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_LONG_SQL_TYPE) } testSparkResultsAreEqual("Write doubles to byte (values within range)", bytesAsDoubles, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_LONG_SQL_TYPE) } @@ -653,65 +658,73 @@ class AnsiCastOpSuite extends GpuExpressionTestSuite { // Test for exceptions when casting out of range values /////////////////////////////////////////////////////////////////////////// - testCastFailsForBadInputs("Detect overflow from long to int", testLongs, sparkConf) { + testCastFailsForBadInputs("Detect overflow from long to int", testLongs, sparkConf, + assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_INT_SQL_TYPE) } - testCastFailsForBadInputs("Detect overflow from long to short", testLongs, sparkConf) { + testCastFailsForBadInputs("Detect overflow from long to short", testLongs, sparkConf, + assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_SHORT_SQL_TYPE) } - testCastFailsForBadInputs("Detect overflow from long to byte", testLongs, sparkConf) { + testCastFailsForBadInputs("Detect overflow from long to byte", testLongs, sparkConf, + assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } - testCastFailsForBadInputs("Detect overflow from int to short", testInts, sparkConf) { + testCastFailsForBadInputs("Detect overflow from int to short", testInts, sparkConf, + assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_SHORT_SQL_TYPE) } - testCastFailsForBadInputs("Detect overflow from int to byte", testInts, sparkConf) { + testCastFailsForBadInputs("Detect overflow from int to byte", testInts, sparkConf, + assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } - testCastFailsForBadInputs("Detect overflow from short to byte", testShorts, sparkConf) { + testCastFailsForBadInputs("Detect overflow from short to byte", testShorts, sparkConf, + assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } - testCastFailsForBadInputs("Detect overflow from float to long", testFloats, sparkConf) { + testCastFailsForBadInputs("Detect overflow from float to long", testFloats, sparkConf, + assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_INT_SQL_TYPE) } - testCastFailsForBadInputs("Detect overflow from float to int", testFloats, sparkConf) { + testCastFailsForBadInputs("Detect overflow from float to int", testFloats, sparkConf, + assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_SHORT_SQL_TYPE) } testCastFailsForBadInputs("Detect overflow from float to short", testFloats, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_LONG_SQL_TYPE) } testCastFailsForBadInputs("Detect overflow from float to byte", testFloats, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } testCastFailsForBadInputs("Detect overflow from double to long", testDoubles, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_LONG_SQL_TYPE) } testCastFailsForBadInputs("Detect overflow from double to int", testDoubles, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_INT_SQL_TYPE) } testCastFailsForBadInputs("Detect overflow from double to short", testDoubles, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_SHORT_SQL_TYPE) } testCastFailsForBadInputs("Detect overflow from double to byte", testDoubles, - sparkConf) { + sparkConf, assumeCondition = before3_3_1) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } From 8b497c5b25ca80d410dee64d45d5592eba1ba307 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 4 Aug 2022 13:35:38 -0500 Subject: [PATCH 008/190] Added in very specific support for from_json to a Map (#6211) Signed-off-by: Robert (Bobby) Evans --- docs/configs.md | 1 + docs/supported_ops.md | 67 ++++++-- .../src/main/python/json_test.py | 11 ++ .../nvidia/spark/rapids/GpuOverrides.scala | 14 ++ .../catalyst/json/rapids/GpuJsonScan.scala | 90 ++++++---- .../spark/sql/rapids/GpuJsonToStructs.scala | 161 ++++++++++++++++++ tools/src/main/resources/operatorsScore.csv | 1 + tools/src/main/resources/supportedExprs.csv | 2 + 8 files changed, 301 insertions(+), 46 deletions(-) create mode 100644 sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala diff --git a/docs/configs.md b/docs/configs.md index cde08bd9f52..2c4df908489 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -247,6 +247,7 @@ Name | SQL Function(s) | Description | Default Value | Notes spark.rapids.sql.expression.IsNaN|`isnan`|Checks if a value is NaN|true|None| spark.rapids.sql.expression.IsNotNull|`isnotnull`|Checks if a value is not null|true|None| spark.rapids.sql.expression.IsNull|`isnull`|Checks if a value is null|true|None| +spark.rapids.sql.expression.JsonToStructs|`from_json`|Returns a struct value with the given `jsonStr` and `schema`|false|This is disabled by default because parsing JSON from a column has a large number of issues and should be considered beta quality right now.| spark.rapids.sql.expression.KnownFloatingPointNormalized| |Tag to prevent redundant normalization|true|None| spark.rapids.sql.expression.KnownNotNull| |Tag an expression as known to not be null|true|None| spark.rapids.sql.expression.Lag|`lag`|Window function that returns N entries behind this one|true|None| diff --git a/docs/supported_ops.md b/docs/supported_ops.md index fa4da943b1c..c5cab65d590 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -7590,22 +7590,22 @@ are limited. -KnownFloatingPointNormalized - -Tag to prevent redundant normalization -None +JsonToStructs +`from_json` +Returns a struct value with the given `jsonStr` and `schema` +This is disabled by default because parsing JSON from a column has a large number of issues and should be considered beta quality right now. project -input +jsonStr + -S -S +S @@ -7622,9 +7622,6 @@ are limited. -S -S - @@ -7634,6 +7631,9 @@ are limited. +NS +PS
unsupported child types BOOLEAN, BYTE, SHORT, INT, LONG, FLOAT, DOUBLE, DATE, TIMESTAMP, DECIMAL, NULL, BINARY, CALENDAR, ARRAY, MAP, STRUCT, UDT
+NS @@ -7663,6 +7663,53 @@ are limited. UDT +KnownFloatingPointNormalized + +Tag to prevent redundant normalization +None +project +input + + + + + +S +S + + + + + + + + + + + + + +result + + + + + +S +S + + + + + + + + + + + + + KnownNotNull Tag an expression as known to not be null diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index 6fe5e800291..18d1eadbd7c 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -359,3 +359,14 @@ def test_json_read_count(spark_tmp_path, v1_enabled_list): assert_gpu_and_cpu_row_counts_equal( lambda spark : spark.read.schema(schema).json(data_path), conf=updated_conf) + +def test_from_json_map(): + # The test here is working around some inconsistencies in how the keys are parsed for maps + # on the GPU the keys are dense, but on the CPU they are sparse + json_string_gen = StringGen("{\"a\": \"[0-9]{0,5}\"(, \"b\": \"[A-Z]{0,5}\")?}") + assert_gpu_and_cpu_are_equal_collect( + lambda spark : unary_op_df(spark, json_string_gen)\ + .selectExpr("from_json(a, \"MAP\") as parsed")\ + .selectExpr("parsed[\"a\"] as pa", "parsed[\"b\"] as pb"), + conf={"spark.rapids.sql.expression.JsonToStructs": "true"}) + diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 11c91077c85..9722982ebbb 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -3560,6 +3560,20 @@ object GpuOverrides extends Logging { GpuGetJsonObject(lhs, rhs) } ), + expr[JsonToStructs]( + "Returns a struct value with the given `jsonStr` and `schema`", + ExprChecks.projectOnly( + TypeSig.MAP.nested(TypeSig.STRING), + (TypeSig.STRUCT + TypeSig.MAP + TypeSig.ARRAY).nested(TypeSig.all), + Seq(ParamCheck("jsonStr", TypeSig.STRING, TypeSig.STRING))), + (a, conf, p, r) => new UnaryExprMeta[JsonToStructs](a, conf, p, r) { + override def tagExprForGpu(): Unit = + GpuJsonScan.tagJsonToStructsSupport(a.options, this) + + override def convertToGpu(child: Expression): GpuExpression = + GpuJsonToStructs(a.schema, a.options, child, a.timeZoneId) + }).disabledByDefault("parsing JSON from a column has a large number of issues and " + + "should be considered beta quality right now."), expr[org.apache.spark.sql.execution.ScalarSubquery]( "Subquery that will return only one row and one column", ExprChecks.projectOnly( diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala index 41a32ffd246..4dabbe52a9f 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala @@ -56,71 +56,89 @@ object GpuJsonScan { scanMeta) } - def tagSupport( - sparkSession: SparkSession, - dataSchema: StructType, - readSchema: StructType, - options: Map[String, String], + def tagSupportOptions( + options: JSONOptionsInRead, meta: RapidsMeta[_, _, _]): Unit = { - val parsedOptions = new JSONOptionsInRead( - options, - sparkSession.sessionState.conf.sessionLocalTimeZone, - sparkSession.sessionState.conf.columnNameOfCorruptRecord) - - if (!meta.conf.isJsonEnabled) { - meta.willNotWorkOnGpu("JSON input and output has been disabled. To enable set " + - s"${RapidsConf.ENABLE_JSON} to true") - } - - if (!meta.conf.isJsonReadEnabled) { - meta.willNotWorkOnGpu("JSON input has been disabled. To enable set " + - s"${RapidsConf.ENABLE_JSON_READ} to true. Please note that, currently json reader does " + - s"not support column prune, so user must specify the full schema or just let spark to " + - s"infer the schema") - } - - if (parsedOptions.multiLine) { + if (options.multiLine) { meta.willNotWorkOnGpu("GpuJsonScan does not support multiLine") } // {"name": /* hello */ "Reynold Xin"} is not supported by CUDF - if (parsedOptions.allowComments) { + if (options.allowComments) { meta.willNotWorkOnGpu("GpuJsonScan does not support allowComments") } // {name: 'Reynold Xin'} is not supported by CUDF - if (parsedOptions.allowUnquotedFieldNames) { + if (options.allowUnquotedFieldNames) { meta.willNotWorkOnGpu("GpuJsonScan does not support allowUnquotedFieldNames") } // {'name': 'Reynold Xin'} is not supported by CUDF - if (options.get("allowSingleQuotes").map(_.toBoolean).getOrElse(false)) { + // This is different because the default for this is true, but we don't support it so we lie... + if (options.parameters.get("allowSingleQuotes").map(_.toBoolean).getOrElse(false)) { meta.willNotWorkOnGpu("GpuJsonScan does not support allowSingleQuotes") } // {"name": "Cazen Lee", "price": "\$10"} is not supported by CUDF - if (parsedOptions.allowBackslashEscapingAnyCharacter) { + if (options.allowBackslashEscapingAnyCharacter) { meta.willNotWorkOnGpu("GpuJsonScan does not support allowBackslashEscapingAnyCharacter") } // {"a":null, "b":1, "c":3.0}, Spark will drop column `a` if dropFieldIfAllNull is enabled. - if (parsedOptions.dropFieldIfAllNull) { + if (options.dropFieldIfAllNull) { meta.willNotWorkOnGpu("GpuJsonScan does not support dropFieldIfAllNull") } - if (parsedOptions.parseMode != PermissiveMode) { + if (options.parseMode != PermissiveMode) { meta.willNotWorkOnGpu("GpuJsonScan only supports Permissive JSON parsing") } - if (parsedOptions.lineSeparator.getOrElse("\n") != "\n") { + if (options.lineSeparator.getOrElse("\n") != "\n") { meta.willNotWorkOnGpu("GpuJsonScan only supports \"\\n\" as a line separator") } - parsedOptions.encoding.foreach(enc => + options.encoding.foreach(enc => if (enc != StandardCharsets.UTF_8.name() && enc != StandardCharsets.US_ASCII.name()) { - meta.willNotWorkOnGpu("GpuJsonScan only supports UTF8 or US-ASCII encoded data") - }) + meta.willNotWorkOnGpu("GpuJsonScan only supports UTF8 or US-ASCII encoded data") + }) + } + + def tagJsonToStructsSupport(options:Map[String, String], + meta: RapidsMeta[_, _, _]): Unit = { + val parsedOptions = new JSONOptionsInRead( + options, + SQLConf.get.sessionLocalTimeZone, + SQLConf.get.columnNameOfCorruptRecord) + + tagSupportOptions(parsedOptions, meta) + } + + def tagSupport( + sparkSession: SparkSession, + dataSchema: StructType, + readSchema: StructType, + options: Map[String, String], + meta: RapidsMeta[_, _, _]): Unit = { + + val parsedOptions = new JSONOptionsInRead( + options, + sparkSession.sessionState.conf.sessionLocalTimeZone, + sparkSession.sessionState.conf.columnNameOfCorruptRecord) + + if (!meta.conf.isJsonEnabled) { + meta.willNotWorkOnGpu("JSON input and output has been disabled. To enable set " + + s"${RapidsConf.ENABLE_JSON} to true") + } + + if (!meta.conf.isJsonReadEnabled) { + meta.willNotWorkOnGpu("JSON input has been disabled. To enable set " + + s"${RapidsConf.ENABLE_JSON_READ} to true. Please note that, currently json reader does " + + s"not support column prune, so user must specify the full schema or just let spark to " + + s"infer the schema") + } + + tagSupportOptions(parsedOptions, meta) val types = readSchema.map(_.dataType) if (types.contains(DateType)) { @@ -136,17 +154,17 @@ object GpuJsonScan { if (!meta.conf.isJsonFloatReadEnabled && types.contains(FloatType)) { meta.willNotWorkOnGpu("JSON reading is not 100% compatible when reading floats. " + - s"To enable it please set ${RapidsConf.ENABLE_READ_JSON_FLOATS} to true.") + s"To enable it please set ${RapidsConf.ENABLE_READ_JSON_FLOATS} to true.") } if (!meta.conf.isJsonDoubleReadEnabled && types.contains(DoubleType)) { meta.willNotWorkOnGpu("JSON reading is not 100% compatible when reading doubles. " + - s"To enable it please set ${RapidsConf.ENABLE_READ_JSON_DOUBLES} to true.") + s"To enable it please set ${RapidsConf.ENABLE_READ_JSON_DOUBLES} to true.") } if (!meta.conf.isJsonDecimalReadEnabled && types.exists(_.isInstanceOf[DecimalType])) { meta.willNotWorkOnGpu("JSON reading is not 100% compatible when reading decimals. " + - s"To enable it please set ${RapidsConf.ENABLE_READ_JSON_DECIMALS} to true.") + s"To enable it please set ${RapidsConf.ENABLE_READ_JSON_DECIMALS} to true.") } dataSchema.getFieldIndex(parsedOptions.columnNameOfCorruptRecord).foreach { corruptFieldIndex => diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala new file mode 100644 index 00000000000..d12f8030c38 --- /dev/null +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala @@ -0,0 +1,161 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.rapids + +import ai.rapids.cudf +import com.nvidia.spark.rapids.{GpuColumnVector, GpuScalar, GpuUnaryExpression} +import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableProducingSeq + +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, NullIntolerant, TimeZoneAwareExpression} +import org.apache.spark.sql.types.{AbstractDataType, DataType, StringType} + +case class GpuJsonToStructs( + schema: DataType, + options: Map[String, String], + child: Expression, + timeZoneId: Option[String] = None) + extends GpuUnaryExpression with TimeZoneAwareExpression with ExpectsInputTypes + with NullIntolerant { + + private def cleanAndConcat(input: cudf.ColumnVector): (cudf.ColumnVector, cudf.ColumnVector) ={ + withResource(cudf.Scalar.fromString("{}")) { emptyRow => + val stripped = withResource(cudf.Scalar.fromString(" ")) { space => + input.strip(space) + } + withResource(stripped) { stripped => + val isNullOrEmptyInput = withResource(input.isNull) { isNull => + val isEmpty = withResource(stripped.getCharLengths) { lengths => + withResource(cudf.Scalar.fromInt(0)) { zero => + lengths.lessOrEqualTo(zero) + } + } + withResource(isEmpty) { isEmpty => + isNull.binaryOp(cudf.BinaryOp.NULL_LOGICAL_OR, isEmpty, cudf.DType.BOOL8) + } + } + closeOnExcept(isNullOrEmptyInput) { _ => + withResource(isNullOrEmptyInput.ifElse(emptyRow, stripped)) { cleaned => + withResource(cudf.Scalar.fromString("\n")) { lineSep => + withResource(cleaned.stringContains(lineSep)) { inputHas => + withResource(inputHas.any()) { anyLineSep => + if (anyLineSep.isValid && anyLineSep.getBoolean) { + throw new IllegalArgumentException("We cannot currently support parsing " + + "JSON that contains a line separator in it") + } + } + } + (isNullOrEmptyInput, cleaned.joinStrings(lineSep, emptyRow)) + } + } + } + } + } + } + + private def castToStrings(rawTable: cudf.Table): Seq[cudf.ColumnVector] = { + (0 until rawTable.getNumberOfColumns).safeMap { i => + val col = rawTable.getColumn(i) + if (!cudf.DType.STRING.equals(col.getType)) { + col.castTo(cudf.DType.STRING) + } else { + col.incRefCount() + } + } + } + + private def makeMap(names: Seq[String], values: Seq[cudf.ColumnVector], + numRows: Int): cudf.ColumnVector = { + val nameCols = names.safeMap { name => + withResource(cudf.Scalar.fromString(name)) { scalarName => + cudf.ColumnVector.fromScalar(scalarName, numRows) + } + } + withResource(nameCols) { nameCols => + val structViews = values.zip(nameCols).safeMap { + case (dataCol, nameCol) => cudf.ColumnView.makeStructView(nameCol, dataCol) + } + withResource(structViews) { structViews => + cudf.ColumnVector.makeList(numRows, cudf.DType.STRUCT, structViews: _*) + } + } + } + + override protected def doColumnar(input: GpuColumnVector): cudf.ColumnVector = { + // We cannot handle all corner cases with this right now. The parser just isn't + // good enough, but we will try to handle a few common ones. + val numRows = input.getRowCount.toInt + + // Step 1: verify and preprocess the data to clean it up and normalize a few things. + // Step 2: Concat the data into a single buffer. + val (isNullOrEmpty, combined) = cleanAndConcat(input.getBase) + withResource(isNullOrEmpty) { isNullOrEmpty => + // Step 3: copy the data back to the host so we can parse it. + val combinedHost = withResource(combined) { combined => + combined.copyToHost() + } + // Step 4: Have cudf parse the JSON data + val (names, rawTable) = withResource(combinedHost) { combinedHost => + val data = combinedHost.getData + val start = combinedHost.getStartListOffset(0) + val end = combinedHost.getEndListOffset(0) + val length = end - start + + withResource(cudf.Table.readJSON(cudf.JSONOptions.DEFAULT, data, start, + length)) { tableWithMeta => + val names = tableWithMeta.getColumnNames + (names, tableWithMeta.releaseTable()) + } + } + + val updatedCols = withResource(rawTable) { rawTable => + // Step 5 verify that the data looks correct. + if (rawTable.getRowCount != numRows) { + throw new IllegalStateException("The input data didn't parse correctly and we read a " + + s"different number of rows than was expected. Expected $numRows, " + + s"but got ${rawTable.getRowCount}") + } + if (names.toSet.size != names.size) { + throw new IllegalStateException("Internal Error: found duplicate key names...") + } + + // Step 6: convert any non-string columns back to strings + castToStrings(rawTable) + } + + // Step 7: turn the data into a Map + val mapData = withResource(updatedCols) { updatedCols => + makeMap(names, updatedCols, numRows) + } + + // Step 8: put nulls back in for nulls and empty strings + withResource(mapData) { mapData => + withResource(GpuScalar.from(null, dataType)) { nullVal => + isNullOrEmpty.ifElse(nullVal, mapData) + } + } + } + } + + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + + override def inputTypes: Seq[AbstractDataType] = StringType :: Nil + + override def dataType: DataType = schema.asNullable + + override def nullable: Boolean = true +} \ No newline at end of file diff --git a/tools/src/main/resources/operatorsScore.csv b/tools/src/main/resources/operatorsScore.csv index cbb5bbe2867..aeaf6fd9fb4 100644 --- a/tools/src/main/resources/operatorsScore.csv +++ b/tools/src/main/resources/operatorsScore.csv @@ -128,6 +128,7 @@ IntegralDivide,4 IsNaN,4 IsNotNull,4 IsNull,4 +JsonToStructs,4 KnownFloatingPointNormalized,4 KnownNotNull,4 Lag,4 diff --git a/tools/src/main/resources/supportedExprs.csv b/tools/src/main/resources/supportedExprs.csv index 9f86c960f49..147d89b0be4 100644 --- a/tools/src/main/resources/supportedExprs.csv +++ b/tools/src/main/resources/supportedExprs.csv @@ -258,6 +258,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS, IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +JsonToStructs,NS,`from_json`,This is disabled by default because parsing JSON from a column has a large number of issues and should be considered beta quality right now.,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +JsonToStructs,NS,`from_json`,This is disabled by default because parsing JSON from a column has a large number of issues and should be considered beta quality right now.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,NS,NA KnownFloatingPointNormalized,S, ,None,project,input,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA KnownFloatingPointNormalized,S, ,None,project,result,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA KnownNotNull,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,NS,S,S,PS,PS,PS,NS From 26d9a65df73e245de209704ad978d4083e11d93e Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Thu, 4 Aug 2022 14:04:15 -0700 Subject: [PATCH 009/190] Make dist jar compression opt-out optional (#6194) This PR speeds up iterating on features especially in the realm of [spark-rapids-jni / cudf](https://github.com/NVIDIA/spark-rapids-jni/blob/branch-22.08/CONTRIBUTING.md#local-testing-of-cross-repo-contributions-cudf-spark-rapids-jni-and-spark-rapids). After the initial build of `spark-rapids` when the only code changed is outside spark-rapids it suffices to execute ``` mvn package -pl dist -Dbuildver=330 -DskipTests -Dskip -Dmaven.scalastyle.skip -Dmaven.javadoc.skip ... [INFO] Total time: 35.624 s ``` to repackage all the dependencies By disabling jar compression for local testing the same command speeds up from 35+ seconds to 10+ seconds on my machine ``` mvn package -pl dist -Dbuildver=330 -DskipTests -Dskip -Dmaven.scalastyle.skip -Dmaven.javadoc.skip -Ddist.jar.compress=false ... [INFO] Total time: 10.775 s ``` ### Other changes - Add options advertised in buildall help without implementation to the parser - Add rebuild-dist-only Signed-off-by: Gera Shegalov --- CONTRIBUTING.md | 29 +++++++++++++++++++++++++++++ build/buildall | 19 ++++++++++++++++--- dist/pom.xml | 1 + pom.xml | 1 + 4 files changed, 47 insertions(+), 3 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 0fe64e16ddc..ae81930b9d4 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -125,6 +125,35 @@ mvn clean verify -Dbuildver=321 \ -DallowConventionalDistJar=true ``` +### Iterative development during local testing + +When iterating on changes impacting the `dist` module artifact directly or via +dependencies you might find the jar creation step unacceptably slow. Due to the +current size of the artifact `rapids-4-spark_2.12` Maven Jar Plugin spends the +bulk of the time compressing the artifact content. +Since the JAR file specification focusses on the file entry layout in a ZIP +archive without requiring file entries to be compressed it is possible to skip +compression, and increase the speed of creating `rapids-4-spark_2.12` jar ~3x +for a single Spark version Shim alone. + +To this end in a pre-production build you can set the Boolean property +`dist.jar.compress` to `false`, its default value is `true`. + +The time saved is more significant if you are merely changing +the `aggregator` module, or the `dist` module, or just incorporating changes from +[spark-rapids-jni](https://github.com/NVIDIA/spark-rapids-jni/blob/branch-22.10/CONTRIBUTING.md#local-testing-of-cross-repo-contributions-cudf-spark-rapids-jni-and-spark-rapids) + +For example, to quickly repackage `rapids-4-spark` after the +initial `./build/buildall` you can iterate by invoking +```Bash +mvn package -pl dist -PnoSnapshots -Ddist.jar.compress=false +``` + +or similarly +```Bash + ./build/buildall --rebuild-dist-only --option="-Ddist.jar.compress=false" +``` + ## Code contributions ### Source code layout diff --git a/build/buildall b/build/buildall index fc3ddb85fe5..f767ce3e5e7 100755 --- a/build/buildall +++ b/build/buildall @@ -50,6 +50,8 @@ function print_usage() { echo " Intall the resulting jar instead of just building it" echo " -o=MVN_OPT, --option=MVN_OPT" echo " use this option to build project with maven. E.g., --option='-Dcudf.version=cuda11'" + echo " --rebuild-dist-only" + echo " repackage the dist module artifact using installed dependencies" } function bloopInstall() { @@ -136,6 +138,15 @@ case "$1" in FINAL_OP="install" ;; +--rebuild-dist-only) + SKIP_DIST_DEPS="1" + MODULE="dist" + ;; + +-o=*|--option=*) + MVN_OPT="${1#*=}" + ;; + *) echo >&2 "Unknown arg: $1" print_usage @@ -253,9 +264,11 @@ export -f build_single_shim # Then resume maven build from the dist module now that shims have been installed time ( # printf a single buildver array element per line - printf "%s\n" "${SPARK_SHIM_VERSIONS[@]}" | \ - xargs -t -I% -P "$BUILD_PARALLEL" -n 1 \ - bash -c 'build_single_shim "$@"' _ % + if [[ "$SKIP_DIST_DEPS" != "1" ]]; then + printf "%s\n" "${SPARK_SHIM_VERSIONS[@]}" | \ + xargs -t -I% -P "$BUILD_PARALLEL" -n 1 \ + bash -c 'build_single_shim "$@"' _ % + fi # This used to resume from dist. However, without including aggregator in the build # the build does not properly initialize spark.version property via buildver profiles # in the root pom, and we get a missing spark311 dependency even for --profile=312,321 diff --git a/dist/pom.xml b/dist/pom.xml index 51bdd05a0c4..f2b3d3575ea 100644 --- a/dist/pom.xml +++ b/dist/pom.xml @@ -264,6 +264,7 @@ jar + ${dist.jar.compress} ${project.build.directory}/parallel-world ${cuda.version} diff --git a/pom.xml b/pom.xml index cf435d29dfa..e78c8ba1fc6 100644 --- a/pom.xml +++ b/pom.xml @@ -1040,6 +1040,7 @@ ${spark.version.classifier} 3.1.0 false + true
From 1d920c451f557c01b3742762b7dd2b0e20473334 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Thu, 4 Aug 2022 14:05:56 -0700 Subject: [PATCH 010/190] Configurable max task faiures for integration tests (#6227) Running integration tests with the [fault injection tool][1] requires tolerating falures. [1]: https://github.com/NVIDIA/spark-rapids-jni/tree/branch-22.08/src/main/cpp/faultinj Signed-off-by: Gera Shegalov --- integration_tests/run_pyspark_from_build.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/integration_tests/run_pyspark_from_build.sh b/integration_tests/run_pyspark_from_build.sh index 38034eb9327..af61f324be8 100755 --- a/integration_tests/run_pyspark_from_build.sh +++ b/integration_tests/run_pyspark_from_build.sh @@ -114,11 +114,11 @@ else # free memory. We use free memory to try and avoid issues if the GPU also is working # on graphics, which happens with many workstation GPUs. We also reserve 2 GiB for # CUDA/CUDF overhead which can be used because of JIT or launching large kernels. - + # If you need to increase the amount of GPU memory you need to change it here and # below where the processes are launched. GPU_MEM_PARALLEL=`nvidia-smi --query-gpu=memory.free --format=csv,noheader | awk '{if (MAX < $1){ MAX = $1}} END {print int((MAX - 2 * 1024) / ((1.5 * 1024) + 750))}'` - CPU_CORES=`nproc` + CPU_CORES=`nproc` HOST_MEM_PARALLEL=`cat /proc/meminfo | grep MemAvailable | awk '{print int($2 / (5 * 1024))}'` TMP_PARALLEL=$(( $GPU_MEM_PARALLEL > $CPU_CORES ? $CPU_CORES : $GPU_MEM_PARALLEL )) TMP_PARALLEL=$(( $TMP_PARALLEL > $HOST_MEM_PARALLEL ? $HOST_MEM_PARALLEL : $TMP_PARALLEL )) @@ -192,7 +192,7 @@ else MB_PER_EXEC=${MB_PER_EXEC:-1024} CORES_PER_EXEC=${CORES_PER_EXEC:-1} - SPARK_TASK_MAXFAILURES=1 + SPARK_TASK_MAXFAILURES=${SPARK_TASK_MAXFAILURES:-1} [[ "$VERSION_STRING" < "3.1.1" ]] && SPARK_TASK_MAXFAILURES=4 export PYSP_TEST_spark_driver_extraClassPath="${ALL_JARS// /:}" @@ -234,7 +234,7 @@ else else # If a master is not specified, use "local[cores, $SPARK_TASK_MAXFAILURES]" if [ -z "${PYSP_TEST_spark_master}" ] && [[ "$SPARK_SUBMIT_FLAGS" != *"--master"* ]]; then - CPU_CORES=`nproc` + CPU_CORES=`nproc` # We are limiting the number of tasks in local mode to 4 because it helps to reduce the # total memory usage, especially host memory usage because when copying data to the GPU # buffers as large as batchSizeBytes can be allocated, and the fewer of them we have the better. @@ -243,7 +243,7 @@ else fi fi - # If you want to change the amount of GPU memory allocated you have to change it here + # If you want to change the amount of GPU memory allocated you have to change it here # and where TEST_PARALLEL is calculated export PYSP_TEST_spark_rapids_memory_gpu_allocSize='1536m' From d102caf0c9f99de3ae9081611365adf5d57ce687 Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Fri, 5 Aug 2022 10:15:44 +0800 Subject: [PATCH 011/190] Iceberg Parquet supports multi-threaded reading. (#6110) * Iceberg Parquet supports multi-threaded reading. Signed-off-by: Firestarman --- .../src/main/python/iceberg_test.py | 126 ++++++--- .../rapids/iceberg/parquet/GpuParquet.java | 41 +-- .../iceberg/parquet/GpuParquetReader.java | 82 +++--- .../spark/source/GpuIcebergReader.java | 17 +- .../spark/source/GpuMultiFileBatchReader.java | 253 ++++++++++++++++++ .../iceberg/spark/source/GpuSparkScan.java | 75 +++++- .../iceberg/spark/source/SparkBatch.java | 2 +- .../spark/rapids/GpuMultiFileReader.scala | 68 ++--- .../nvidia/spark/rapids/GpuParquetScan.scala | 82 +++--- .../spark/sql/rapids/GpuInputFileBlock.scala | 4 +- 10 files changed, 571 insertions(+), 179 deletions(-) create mode 100644 sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java diff --git a/integration_tests/src/main/python/iceberg_test.py b/integration_tests/src/main/python/iceberg_test.py index 7bfa92b01d6..c84a5a8c9bf 100644 --- a/integration_tests/src/main/python/iceberg_test.py +++ b/integration_tests/src/main/python/iceberg_test.py @@ -53,7 +53,8 @@ def setup_iceberg_table(spark): @ignore_order(local=True) @pytest.mark.skipif(is_before_spark_320() or is_databricks_runtime(), reason="AQE+DPP not supported until Spark 3.2.0+ and AQE+DPP not supported on Databricks") -def test_iceberg_aqe_dpp(spark_tmp_table_factory): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_aqe_dpp(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() def setup_iceberg_table(spark): @@ -65,12 +66,14 @@ def setup_iceberg_table(spark): assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.sql("SELECT * from {} as X JOIN {} as Y ON X.a = Y.a WHERE Y.a > 0".format(table, table)), conf={"spark.sql.adaptive.enabled": "true", + "spark.rapids.sql.format.parquet.reader.type": reader_type, "spark.sql.optimizer.dynamicPartitionPruning.enabled": "true"}) @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering @pytest.mark.parametrize("data_gens", iceberg_gens_list, ids=idfn) -def test_iceberg_parquet_read_round_trip(spark_tmp_table_factory, data_gens): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_parquet_read_round_trip(spark_tmp_table_factory, data_gens, reader_type): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(data_gens)] table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -80,12 +83,14 @@ def setup_iceberg_table(spark): spark.sql("CREATE TABLE {} USING ICEBERG AS SELECT * FROM {}".format(table, tmpview)) with_cpu_session(setup_iceberg_table) assert_gpu_and_cpu_are_equal_collect( - lambda spark : spark.sql("SELECT * FROM {}".format(table))) + lambda spark : spark.sql("SELECT * FROM {}".format(table)), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}) @iceberg @pytest.mark.parametrize("data_gens", [[long_gen]], ids=idfn) @pytest.mark.parametrize("iceberg_format", ["orc", "avro"], ids=idfn) -def test_iceberg_unsupported_formats(spark_tmp_table_factory, data_gens, iceberg_format): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_unsupported_formats(spark_tmp_table_factory, data_gens, iceberg_format, reader_type): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(data_gens)] table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -97,7 +102,9 @@ def setup_iceberg_table(spark): "AS SELECT * FROM {}".format(tmpview)) with_cpu_session(setup_iceberg_table) assert_py4j_exception( - lambda : with_gpu_session(lambda spark : spark.sql("SELECT * FROM {}".format(table)).collect()), + lambda : with_gpu_session( + lambda spark : spark.sql("SELECT * FROM {}".format(table)).collect(), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}), "UnsupportedOperationException") @iceberg @@ -129,7 +136,8 @@ def setup_iceberg_table(spark): marks=pytest.mark.skipif(is_before_spark_320(), reason="Hadoop with Spark 3.1.x does not support lz4 by default")), ("zstd", None)], ids=idfn) -def test_iceberg_read_parquet_compression_codec(spark_tmp_table_factory, codec_info): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_read_parquet_compression_codec(spark_tmp_table_factory, codec_info, reader_type): codec, error_msg = codec_info table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -141,16 +149,19 @@ def setup_iceberg_table(spark): spark.sql("INSERT INTO {} SELECT * FROM {}".format(table, tmpview)) with_cpu_session(setup_iceberg_table) query = "SELECT * FROM {}".format(table) + read_conf = {'spark.rapids.sql.format.parquet.reader.type': reader_type} if error_msg: assert_py4j_exception( - lambda : with_gpu_session(lambda spark : spark.sql(query).collect()), error_msg) + lambda : with_gpu_session(lambda spark : spark.sql(query).collect(), conf=read_conf), + error_msg) else: - assert_gpu_and_cpu_are_equal_collect(lambda spark : spark.sql(query)) + assert_gpu_and_cpu_are_equal_collect(lambda spark : spark.sql(query), conf=read_conf) @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering @pytest.mark.parametrize("key_gen", [int_gen, long_gen, string_gen, boolean_gen, date_gen, timestamp_gen, decimal_gen_64bit], ids=idfn) -def test_iceberg_read_partition_key(spark_tmp_table_factory, key_gen): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_read_partition_key(spark_tmp_table_factory, key_gen, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() def setup_iceberg_table(spark): @@ -160,11 +171,13 @@ def setup_iceberg_table(spark): "AS SELECT * FROM {}".format(tmpview)) with_cpu_session(setup_iceberg_table) assert_gpu_and_cpu_are_equal_collect( - lambda spark : spark.sql("SELECT a FROM {}".format(table))) + lambda spark : spark.sql("SELECT a FROM {}".format(table)), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}) @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -def test_iceberg_input_meta(spark_tmp_table_factory): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_input_meta(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() def setup_iceberg_table(spark): @@ -176,11 +189,13 @@ def setup_iceberg_table(spark): assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.sql( "SELECT a, input_file_name(), input_file_block_start(), input_file_block_length() " + \ - "FROM {}".format(table))) + "FROM {}".format(table)), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}) @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -def test_iceberg_disorder_read_schema(spark_tmp_table_factory): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_disorder_read_schema(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() def setup_iceberg_table(spark): @@ -190,7 +205,8 @@ def setup_iceberg_table(spark): "AS SELECT * FROM {}".format(tmpview)) with_cpu_session(setup_iceberg_table) assert_gpu_and_cpu_are_equal_collect( - lambda spark : spark.sql("SELECT b,c,a FROM {}".format(table))) + lambda spark : spark.sql("SELECT b,c,a FROM {}".format(table)), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}) @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering @@ -236,7 +252,8 @@ def setup_iceberg_table(spark): @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering @pytest.mark.skipif(is_before_spark_320(), reason="Spark 3.1.x has a catalog bug precluding scope prefix in table names") -def test_iceberg_read_timetravel(spark_tmp_table_factory): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_read_timetravel(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() def setup_snapshots(spark): @@ -253,12 +270,14 @@ def setup_snapshots(spark): first_snapshot_id = with_cpu_session(setup_snapshots) assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.read.option("snapshot-id", first_snapshot_id) \ - .format("iceberg").load("default.{}".format(table))) + .format("iceberg").load("default.{}".format(table)), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}) @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering @pytest.mark.skipif(is_before_spark_320(), reason="Spark 3.1.x has a catalog bug precluding scope prefix in table names") -def test_iceberg_incremental_read(spark_tmp_table_factory): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_incremental_read(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() def setup_snapshots(spark): @@ -282,11 +301,13 @@ def setup_snapshots(spark): lambda spark : spark.read \ .option("start-snapshot-id", start_snapshot) \ .option("end-snapshot-id", end_snapshot) \ - .format("iceberg").load("default.{}".format(table))) + .format("iceberg").load("default.{}".format(table)), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}) @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -def test_iceberg_reorder_columns(spark_tmp_table_factory): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_reorder_columns(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() def setup_iceberg_table(spark): @@ -300,11 +321,14 @@ def setup_iceberg_table(spark): spark.sql("INSERT INTO {} ".format(table) + \ "SELECT * FROM {}".format(tmpview)) with_cpu_session(setup_iceberg_table) - assert_gpu_and_cpu_are_equal_collect(lambda spark : spark.sql("SELECT * FROM {}".format(table))) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT * FROM {}".format(table)), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}) @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -def test_iceberg_rename_column(spark_tmp_table_factory): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_rename_column(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() def setup_iceberg_table(spark): @@ -318,11 +342,14 @@ def setup_iceberg_table(spark): spark.sql("INSERT INTO {} ".format(table) + \ "SELECT * FROM {}".format(tmpview)) with_cpu_session(setup_iceberg_table) - assert_gpu_and_cpu_are_equal_collect(lambda spark : spark.sql("SELECT * FROM {}".format(table))) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT * FROM {}".format(table)), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}) @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -def test_iceberg_column_names_swapped(spark_tmp_table_factory): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_column_names_swapped(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() def setup_iceberg_table(spark): @@ -338,11 +365,14 @@ def setup_iceberg_table(spark): spark.sql("INSERT INTO {} ".format(table) + \ "SELECT * FROM {}".format(tmpview)) with_cpu_session(setup_iceberg_table) - assert_gpu_and_cpu_are_equal_collect(lambda spark : spark.sql("SELECT * FROM {}".format(table))) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT * FROM {}".format(table)), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}) @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -def test_iceberg_alter_column_type(spark_tmp_table_factory): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_alter_column_type(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() def setup_iceberg_table(spark): @@ -358,11 +388,14 @@ def setup_iceberg_table(spark): spark.sql("INSERT INTO {} ".format(table) + \ "SELECT * FROM {}".format(tmpview)) with_cpu_session(setup_iceberg_table) - assert_gpu_and_cpu_are_equal_collect(lambda spark : spark.sql("SELECT * FROM {}".format(table))) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT * FROM {}".format(table)), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}) @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -def test_iceberg_add_column(spark_tmp_table_factory): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_add_column(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() def setup_iceberg_table(spark): @@ -376,11 +409,14 @@ def setup_iceberg_table(spark): spark.sql("INSERT INTO {} ".format(table) + \ "SELECT * FROM {}".format(tmpview)) with_cpu_session(setup_iceberg_table) - assert_gpu_and_cpu_are_equal_collect(lambda spark : spark.sql("SELECT * FROM {}".format(table))) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT * FROM {}".format(table)), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}) @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -def test_iceberg_remove_column(spark_tmp_table_factory): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_remove_column(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() def setup_iceberg_table(spark): @@ -394,11 +430,14 @@ def setup_iceberg_table(spark): spark.sql("INSERT INTO {} ".format(table) + \ "SELECT * FROM {}".format(tmpview)) with_cpu_session(setup_iceberg_table) - assert_gpu_and_cpu_are_equal_collect(lambda spark : spark.sql("SELECT * FROM {}".format(table))) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT * FROM {}".format(table)), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}) @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -def test_iceberg_add_partition_field(spark_tmp_table_factory): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_add_partition_field(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() def setup_iceberg_table(spark): @@ -412,11 +451,14 @@ def setup_iceberg_table(spark): spark.sql("INSERT INTO {} ".format(table) + \ "SELECT * FROM {} ORDER BY b".format(tmpview)) with_cpu_session(setup_iceberg_table) - assert_gpu_and_cpu_are_equal_collect(lambda spark : spark.sql("SELECT * FROM {}".format(table))) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT * FROM {}".format(table)), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}) @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -def test_iceberg_drop_partition_field(spark_tmp_table_factory): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_drop_partition_field(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() def setup_iceberg_table(spark): @@ -430,11 +472,14 @@ def setup_iceberg_table(spark): spark.sql("INSERT INTO {} ".format(table) + \ "SELECT * FROM {}".format(tmpview)) with_cpu_session(setup_iceberg_table) - assert_gpu_and_cpu_are_equal_collect(lambda spark : spark.sql("SELECT * FROM {}".format(table))) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT * FROM {}".format(table)), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}) @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -def test_iceberg_v1_delete(spark_tmp_table_factory): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_v1_delete(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() def setup_iceberg_table(spark): @@ -444,11 +489,14 @@ def setup_iceberg_table(spark): "AS SELECT * FROM {}".format(tmpview)) spark.sql("DELETE FROM {} WHERE a < 0".format(table)) with_cpu_session(setup_iceberg_table) - assert_gpu_and_cpu_are_equal_collect(lambda spark : spark.sql("SELECT * FROM {}".format(table))) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT * FROM {}".format(table)), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}) @iceberg @pytest.mark.skipif(is_before_spark_320(), reason="merge-on-read not supported on Spark 3.1.x") -def test_iceberg_v2_delete_unsupported(spark_tmp_table_factory): +@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +def test_iceberg_v2_delete_unsupported(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() def setup_iceberg_table(spark): @@ -460,5 +508,7 @@ def setup_iceberg_table(spark): spark.sql("DELETE FROM {} WHERE a < 0".format(table)) with_cpu_session(setup_iceberg_table) assert_py4j_exception( - lambda : with_gpu_session(lambda spark : spark.sql("SELECT * FROM {}".format(table)).collect()), + lambda : with_gpu_session( + lambda spark : spark.sql("SELECT * FROM {}".format(table)).collect(), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}), "UnsupportedOperationException: Delete filter is not supported") diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/parquet/GpuParquet.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/parquet/GpuParquet.java index cb4963aea3b..7790814f9cb 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/parquet/GpuParquet.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/parquet/GpuParquet.java @@ -143,25 +143,7 @@ public ReadBuilder withMetrics(scala.collection.immutable.Map } public CloseableIterable build() { - ParquetReadOptions.Builder optionsBuilder; - if (file instanceof HadoopInputFile) { - // remove read properties already set that may conflict with this read - Configuration conf = new Configuration(((HadoopInputFile) file).getConf()); - for (String property : READ_PROPERTIES_TO_REMOVE) { - conf.unset(property); - } - optionsBuilder = HadoopReadOptions.builder(conf); - } else { - //optionsBuilder = ParquetReadOptions.builder(); - throw new UnsupportedOperationException("Only Hadoop files are supported for now"); - } - - if (start != null) { - optionsBuilder.withRange(start, start + length); - } - - ParquetReadOptions options = optionsBuilder.build(); - + ParquetReadOptions options = buildReaderOptions(file, start, length); PartitionedFile partFile = PartitionedFileUtils.newPartitionedFile( InternalRow.empty(), file.location(), start, length); return new GpuParquetReader(file, projectSchema, options, nameMapping, filter, caseSensitive, @@ -169,4 +151,25 @@ public CloseableIterable build() { debugDumpPrefix, metrics); } } + + public static ParquetReadOptions buildReaderOptions(InputFile file, Long start, Long length) { + ParquetReadOptions.Builder optionsBuilder; + if (file instanceof HadoopInputFile) { + // remove read properties already set that may conflict with this read + Configuration conf = new Configuration(((HadoopInputFile) file).getConf()); + for (String property : READ_PROPERTIES_TO_REMOVE) { + conf.unset(property); + } + optionsBuilder = HadoopReadOptions.builder(conf); + } else { + //optionsBuilder = ParquetReadOptions.builder(); + throw new UnsupportedOperationException("Only Hadoop files are supported for now"); + } + + if (start != null) { + optionsBuilder.withRange(start, start + length); + } + + return optionsBuilder.build(); + } } diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/parquet/GpuParquetReader.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/parquet/GpuParquetReader.java index 584cb94a447..f35a669241c 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/parquet/GpuParquetReader.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/parquet/GpuParquetReader.java @@ -115,38 +115,8 @@ public GpuParquetReader( public org.apache.iceberg.io.CloseableIterator iterator() { try (ParquetFileReader reader = newReader(input, options)) { MessageType fileSchema = reader.getFileMetaData().getSchema(); - - MessageType typeWithIds; - if (ParquetSchemaUtil.hasIds(fileSchema)) { - typeWithIds = fileSchema; - } else if (nameMapping != null) { - typeWithIds = ParquetSchemaUtil.applyNameMapping(fileSchema, nameMapping); - } else { - typeWithIds = ParquetSchemaUtil.addFallbackIds(fileSchema); - } - - List rowGroups = reader.getRowGroups(); - List filteredRowGroups = Lists.newArrayListWithCapacity(rowGroups.size()); - - if (expectedSchema.findField(MetadataColumns.ROW_POSITION.fieldId()) != null) { - throw new UnsupportedOperationException("row position meta column not implemented"); - } - - ParquetMetricsRowGroupFilter statsFilter = null; - ParquetDictionaryRowGroupFilter dictFilter = null; - if (filter != null) { - statsFilter = new ParquetMetricsRowGroupFilter(expectedSchema, filter, caseSensitive); - dictFilter = new ParquetDictionaryRowGroupFilter(expectedSchema, filter, caseSensitive); - } - - for (BlockMetaData rowGroup : rowGroups) { - boolean shouldRead = filter == null || ( - statsFilter.shouldRead(typeWithIds, rowGroup) && - dictFilter.shouldRead(typeWithIds, rowGroup, reader.getDictionaryReader(rowGroup))); - if (shouldRead) { - filteredRowGroups.add(rowGroup); - } - } + List filteredRowGroups = filterRowGroups(reader, nameMapping, + expectedSchema, filter, caseSensitive); ReorderColumns reorder = ParquetSchemaUtil.hasIds(fileSchema) ? new ReorderColumns(idToConstant) : new ReorderColumnsFallback(idToConstant); @@ -176,7 +146,45 @@ public org.apache.iceberg.io.CloseableIterator iterator() { } } - private static ParquetFileReader newReader(InputFile file, ParquetReadOptions options) { + public static List filterRowGroups(ParquetFileReader reader, + NameMapping nameMapping, Schema expectedSchema, Expression filter, boolean caseSensitive) { + MessageType fileSchema = reader.getFileMetaData().getSchema(); + + MessageType typeWithIds; + if (ParquetSchemaUtil.hasIds(fileSchema)) { + typeWithIds = fileSchema; + } else if (nameMapping != null) { + typeWithIds = ParquetSchemaUtil.applyNameMapping(fileSchema, nameMapping); + } else { + typeWithIds = ParquetSchemaUtil.addFallbackIds(fileSchema); + } + + List rowGroups = reader.getRowGroups(); + List filteredRowGroups = Lists.newArrayListWithCapacity(rowGroups.size()); + + if (expectedSchema.findField(MetadataColumns.ROW_POSITION.fieldId()) != null) { + throw new UnsupportedOperationException("row position meta column not implemented"); + } + + ParquetMetricsRowGroupFilter statsFilter = null; + ParquetDictionaryRowGroupFilter dictFilter = null; + if (filter != null) { + statsFilter = new ParquetMetricsRowGroupFilter(expectedSchema, filter, caseSensitive); + dictFilter = new ParquetDictionaryRowGroupFilter(expectedSchema, filter, caseSensitive); + } + + for (BlockMetaData rowGroup : rowGroups) { + boolean shouldRead = filter == null || ( + statsFilter.shouldRead(typeWithIds, rowGroup) && + dictFilter.shouldRead(typeWithIds, rowGroup, reader.getDictionaryReader(rowGroup))); + if (shouldRead) { + filteredRowGroups.add(rowGroup); + } + } + return filteredRowGroups; + } + + public static ParquetFileReader newReader(InputFile file, ParquetReadOptions options) { try { return ParquetFileReader.open(ParquetIO.file(file), options); } catch (IOException e) { @@ -184,7 +192,7 @@ private static ParquetFileReader newReader(InputFile file, ParquetReadOptions op } } - private static Map addNullsForMissingFields(Map idToConstant, + public static Map addNullsForMissingFields(Map idToConstant, Set missingFields) { if (missingFields.isEmpty()) { return idToConstant; @@ -197,7 +205,7 @@ private static ParquetFileReader newReader(InputFile file, ParquetReadOptions op } /** Generate the Spark schema corresponding to a Parquet schema and expected Iceberg schema */ - private static class SparkSchemaConverter extends TypeWithSchemaVisitor { + public static class SparkSchemaConverter extends TypeWithSchemaVisitor { @Override public DataType message(Types.StructType iStruct, MessageType message, List fields) { return struct(iStruct, message, fields); @@ -273,7 +281,7 @@ public DataType primitive(org.apache.iceberg.types.Type.PrimitiveType iPrimitive } } - private static class ReorderColumns extends TypeWithSchemaVisitor { + public static class ReorderColumns extends TypeWithSchemaVisitor { private final Map idToConstant; private final Set missingFields = Sets.newHashSet(); @@ -373,7 +381,7 @@ private List filterAndReorder(Types.StructType expected, List fields } } - private static class ReorderColumnsFallback extends ReorderColumns { + public static class ReorderColumnsFallback extends ReorderColumns { public ReorderColumnsFallback(Map idToConstant) { super(idToConstant); } diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuIcebergReader.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuIcebergReader.java index 228a03cf483..e624852e19d 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuIcebergReader.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuIcebergReader.java @@ -97,6 +97,16 @@ public ColumnarBatch next() { } private ColumnarBatch addConstantColumns(ColumnarBatch batch) { + return addConstantColumns(batch, expectedSchema, idToConstant); + } + + private ColumnarBatch addUpcastsIfNeeded(ColumnarBatch batch) { + return addUpcastsIfNeeded(batch, expectedSchema); + } + + /** (package accessible for GpuBatchRapidsReader) */ + static ColumnarBatch addConstantColumns(ColumnarBatch batch, + Schema expectedSchema, Map idToConstant) { ColumnVector[] columns = new ColumnVector[expectedSchema.columns().size()]; ColumnarBatch result = null; final ConstantDetector constantDetector = new ConstantDetector(idToConstant); @@ -135,7 +145,8 @@ private ColumnarBatch addConstantColumns(ColumnarBatch batch) { return result; } - private ColumnarBatch addUpcastsIfNeeded(ColumnarBatch batch) { + /** (package accessible for GpuBatchRapidsReader) */ + static ColumnarBatch addUpcastsIfNeeded(ColumnarBatch batch, Schema expectedSchema) { GpuColumnVector[] columns = null; try { List expectedColumnTypes = expectedSchema.columns(); @@ -146,8 +157,8 @@ private ColumnarBatch addUpcastsIfNeeded(ColumnarBatch batch) { DataType expectedSparkType = SparkSchemaUtil.convert(expectedColumnTypes.get(i).type()); GpuColumnVector oldColumn = columns[i]; columns[i] = GpuColumnVector.from( - GpuCast.doCast(oldColumn.getBase(), oldColumn.dataType(), expectedSparkType, false, false, false), - expectedSparkType); + GpuCast.doCast(oldColumn.getBase(), oldColumn.dataType(), expectedSparkType, + false, false, false), expectedSparkType); } ColumnarBatch newBatch = new ColumnarBatch(columns, batch.numRows()); columns = null; diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java new file mode 100644 index 00000000000..9b958cbaa8b --- /dev/null +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java @@ -0,0 +1,253 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.iceberg.spark.source; + +import com.nvidia.spark.rapids.*; +import com.nvidia.spark.rapids.iceberg.data.GpuDeleteFilter; +import com.nvidia.spark.rapids.iceberg.parquet.GpuParquet; +import com.nvidia.spark.rapids.iceberg.parquet.GpuParquetReader; +import com.nvidia.spark.rapids.iceberg.parquet.ParquetSchemaUtil; +import com.nvidia.spark.rapids.iceberg.parquet.TypeWithSchemaVisitor; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.*; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.schema.MessageType; +import org.apache.spark.TaskContext; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.execution.datasources.PartitionedFile; +import org.apache.spark.sql.rapids.InputFileUtils; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.collection.Seq; +import scala.Tuple2; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +/** The wrapper of the GPU multi-threaded and coalescing(TBD) reader for Iceberg */ +class GpuMultiFileBatchReader extends BaseDataReader { + private static final Logger LOG = LoggerFactory.getLogger(GpuMultiFileBatchReader.class); + private final Map, Schema>> constsSchemaMap = + Maps.newConcurrentMap(); + private final LinkedHashMap files; + private final Schema expectedSchema; + private final boolean caseSensitive; + private final Configuration conf; + private final int maxBatchSizeRows; + private final long maxBatchSizeBytes; + private final String parquetDebugDumpPrefix; + private final scala.collection.immutable.Map metrics; + private final boolean useMultiThread; + private final FileFormat fileFormat; + private final int numThreads; + private final int maxNumFileProcessed; + + private NameMapping nameMapping = null; + private boolean needNext = true; + private boolean isBatchPending; + // lazy variables + private FilePartitionReaderBase rapidsReader = null; + + GpuMultiFileBatchReader(CombinedScanTask task, Table table, Schema expectedSchema, + boolean caseSensitive, Configuration conf, int maxBatchSizeRows, long maxBatchSizeBytes, + String parquetDebugDumpPrefix, int numThreads, int maxNumFileProcessed, + boolean useMultiThread, FileFormat fileFormat, + scala.collection.immutable.Map metrics) { + super(table, task); + this.expectedSchema = expectedSchema; + this.caseSensitive = caseSensitive; + this.conf = conf; + this.maxBatchSizeRows = maxBatchSizeRows; + this.maxBatchSizeBytes = maxBatchSizeBytes; + this.parquetDebugDumpPrefix = parquetDebugDumpPrefix; + this.useMultiThread = useMultiThread; + this.fileFormat = fileFormat; + this.metrics = metrics; + this.numThreads = numThreads; + this.maxNumFileProcessed = maxNumFileProcessed; + String nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); + if (nameMapping != null) { + this.nameMapping = NameMappingParser.fromJson(nameMapping); + } + files = Maps.newLinkedHashMapWithExpectedSize(task.files().size()); + task.files().forEach(fst -> this.files.putIfAbsent(fst.file().path().toString(), fst)); + } + + @Override + public ColumnarBatch get() { + if (rapidsReader == null) { + // Not initialized, return null to align with PerFile reader. + return null; + } + needNext = true; + isBatchPending = false; + // The same post-process with PerFile reader. + try (ColumnarBatch batch = rapidsReader.get()) { + // The Rapids reader should already set the current file. + String curFile = InputFileUtils.getCurInputFilePath(); + Tuple2, Schema> constsSchema = constsSchemaMap.get(curFile); + Map idToConsts = constsSchema._1(); + Schema updatedReadSchema = constsSchema._2(); + return GpuIcebergReader.addUpcastsIfNeeded( + GpuIcebergReader.addConstantColumns(batch, updatedReadSchema, idToConsts), + updatedReadSchema); + } + } + + @Override + public boolean next() throws IOException { + ensureRapidsReader(); + if (needNext) { + needNext = false; + isBatchPending = rapidsReader.next(); + } + return isBatchPending; + } + + @Override + public void close() throws IOException { + if (rapidsReader != null) rapidsReader.close(); + super.close(); + } + + @Override + CloseableIterator open(FileScanTask task) { + // Stub for the required implementation. + // This method will never be called after overriding the "next" method. + throw new IllegalStateException(); + } + + private void ensureRapidsReader() { + if (rapidsReader == null) { + if (FileFormat.PARQUET.equals(fileFormat)) { + if (useMultiThread) { + rapidsReader = createParquetMultiThreadReader(); + } else { + // TODO Support coalescing reading, tracked by + // https://github.com/NVIDIA/spark-rapids/issues/5942 + throw new UnsupportedOperationException( + "Coalescing reading is not supported for Parquet reads yet"); + } + } else { + throw new UnsupportedOperationException( + "Format: " + fileFormat + " is not supported for batched reads"); + } + } + } + + private FilePartitionReaderBase createParquetMultiThreadReader() { + LOG.debug("Using multi-threaded Iceberg Parquet reader, task attempt ID: " + + TaskContext.get().taskAttemptId()); + // Iceberg will handle partition values itself. + StructType emptyPartSchema = new StructType(); + InternalRow emptyPartValue = InternalRow.empty(); + + PartitionedFile[] files = this.files.values().stream() + .map(fst -> PartitionedFileUtils.newPartitionedFile(emptyPartValue, + fst.file().path().toString(), fst.start(), fst.length())) + .toArray(PartitionedFile[]::new); + + return new MultiFileCloudParquetPartitionReader(conf, files, this::filterParquetBlocks, + caseSensitive, parquetDebugDumpPrefix, maxBatchSizeRows, maxBatchSizeBytes, + metrics, emptyPartSchema, numThreads, maxNumFileProcessed, + false, // ignoreMissingFiles + false, // ignoreCorruptFiles + false // useFieldId + ); + } + + /** The filter function for the Parquet multi-file reader */ + private ParquetFileInfoWithBlockMeta filterParquetBlocks(PartitionedFile file) { + FileScanTask fst = this.files.get(file.filePath()); + GpuDeleteFilter deleteFilter = deleteFilter(fst); + if (deleteFilter != null) { + throw new UnsupportedOperationException("Delete filter is not supported"); + } + Schema updatedSchema = requiredSchema(deleteFilter); + Map idToConstant = constantsMap(fst, updatedSchema); + InputFile inFile = getInputFile(fst); + ParquetReadOptions readOptions = + GpuParquet.buildReaderOptions(inFile, fst.start(), fst.length()); + try (ParquetFileReader reader = GpuParquetReader.newReader(inFile, readOptions)) { + MessageType fileSchema = reader.getFileMetaData().getSchema(); + + List filteredRowGroups = GpuParquetReader.filterRowGroups(reader, + nameMapping, updatedSchema, fst.residual(), caseSensitive); + + GpuParquetReader.ReorderColumns reorder = ParquetSchemaUtil.hasIds(fileSchema) ? + new GpuParquetReader.ReorderColumns(idToConstant) : + new GpuParquetReader.ReorderColumnsFallback(idToConstant); + + MessageType fileReadSchema = (MessageType) TypeWithSchemaVisitor.visit( + updatedSchema.asStruct(), fileSchema, reorder); + Seq clippedBlocks = GpuParquetUtils.clipBlocksToSchema( + fileReadSchema, filteredRowGroups, caseSensitive); + StructType partReaderSparkSchema = (StructType) TypeWithSchemaVisitor.visit( + updatedSchema.asStruct(), fileReadSchema, new GpuParquetReader.SparkSchemaConverter()); + + // cache the updated constants + Map updatedConstants = + GpuParquetReader.addNullsForMissingFields(idToConstant, reorder.getMissingFields()); + constsSchemaMap.put(file.filePath(), Tuple2.apply(updatedConstants, updatedSchema)); + + return ParquetFileInfoWithBlockMeta.apply(new Path(new URI(file.filePath())), + clippedBlocks, InternalRow.empty(), fileReadSchema, partReaderSparkSchema, + true, // isCorrectedInt96RebaseMode + true, // isCorrectedRebaseMode + true // hasInt96Timestamps + ); + } catch (IOException e) { + throw new UncheckedIOException("Failed to open file: " + inFile, e); + } catch (URISyntaxException ue) { + throw new IllegalArgumentException("Invalid file path: " + inFile, ue); + } + } + + private GpuDeleteFilter deleteFilter(FileScanTask task) { + if (task.deletes().isEmpty()) { + return null; + } + return new GpuDeleteFilter( + task.file().path().toString(), + task.deletes(), + table().schema(), + expectedSchema); + } + + private Schema requiredSchema(GpuDeleteFilter deleteFilter) { + if (deleteFilter != null && deleteFilter.hasEqDeletes()) { + return deleteFilter.requiredSchema(); + } else { + return expectedSchema; + } + } +} diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java index df33fa4e165..2b956d33a7e 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java @@ -23,6 +23,7 @@ import java.util.stream.Collectors; import com.nvidia.spark.rapids.GpuMetric; +import com.nvidia.spark.rapids.MultiFileReaderUtils; import com.nvidia.spark.rapids.RapidsConf; import com.nvidia.spark.rapids.ScanWithMetricsWrapper; import com.nvidia.spark.rapids.iceberg.spark.Spark3Util; @@ -31,6 +32,7 @@ import com.nvidia.spark.rapids.iceberg.spark.SparkUtil; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; @@ -179,9 +181,17 @@ public String description() { static class ReaderFactory implements PartitionReaderFactory { private final scala.collection.immutable.Map metrics; + private final scala.collection.immutable.Set allCloudSchemes; + private final boolean canUseParquetMultiThread; + private final boolean canUseParquetCoalescing; - public ReaderFactory(scala.collection.immutable.Map metrics) { + public ReaderFactory(scala.collection.immutable.Map metrics, + RapidsConf rapidsConf) { this.metrics = metrics; + this.allCloudSchemes = rapidsConf.getCloudSchemes().toSet(); + // Only multi-threaded Parquet is supported. + this.canUseParquetMultiThread = rapidsConf.isParquetMultiThreadReadEnabled(); + this.canUseParquetCoalescing = false; } @Override @@ -192,7 +202,17 @@ public PartitionReader createReader(InputPartition partition) { @Override public PartitionReader createColumnarReader(InputPartition partition) { if (partition instanceof ReadTask) { - return new BatchReader((ReadTask) partition, metrics); + ReadTask rTask = (ReadTask) partition; + // ret = (canAccelerateRead, isMultiThread, fileFormat) = (_1(), _2(), _3()) + scala.Tuple3 ret = multiFileReadCheck(rTask); + boolean canAccelerateRead = ret._1(); + if (canAccelerateRead) { + boolean isMultiThread = ret._2(); + FileFormat ff = ret._3(); + return new MultiFileBatchReader(rTask, isMultiThread, ff, metrics); + } else { + return new BatchReader(rTask, metrics); + } } else { throw new UnsupportedOperationException("Incorrect input partition type: " + partition); } @@ -202,6 +222,45 @@ public PartitionReader createColumnarReader(InputPartition partit public boolean supportColumnarReads(InputPartition partition) { return true; } + + /** + * Return a tuple as (canAccelerateRead, isMultiThread, fileFormat). + * - "canAccelerateRead" Whether the input read task can be accelerated by + * multi-threaded or coalescing reading. + * - "isMultiThread" Whether to use the multi-threaded reading. + * - "fileFormat" The file format of this combined task. Acceleration requires + * all the files in a combined task have the same format. + */ + private scala.Tuple3 multiFileReadCheck(ReadTask readTask) { + Collection scans = readTask.files(); + boolean canUseMultiThread = false, canUseCoalescing = false; + FileFormat ff = null; + // Require all the files in a partition have the same file format. + if (scans.stream().allMatch(t -> t.file().format().equals(FileFormat.PARQUET))) { + // Now only Parquet is supported. + canUseMultiThread = canUseParquetMultiThread; + canUseCoalescing = canUseParquetCoalescing; + ff = FileFormat.PARQUET; + } + boolean canAccelerateRead = canUseMultiThread || canUseCoalescing; + String[] files = scans.stream().map(f -> f.file().path().toString()) + .toArray(String[]::new); + // Get the final decision for the subtype of the Rapids reader. + boolean useMultiThread = MultiFileReaderUtils.useMultiThreadReader( + canUseCoalescing, canUseMultiThread, files, allCloudSchemes); + return scala.Tuple3.apply(canAccelerateRead, useMultiThread, ff); + } + } + + private static class MultiFileBatchReader + extends GpuMultiFileBatchReader implements PartitionReader { + MultiFileBatchReader(ReadTask task, boolean useMultiThread, FileFormat ff, + scala.collection.immutable.Map metrics) { + super(task.task, task.table(), task.expectedSchema(), task.isCaseSensitive(), + task.getConfiguration(), task.getMaxBatchSizeRows(), task.getMaxBatchSizeBytes(), + task.getParquetDebugDumpPrefix(), task.getNumThreads(), task.getMaxNumFileProcessed(), + useMultiThread, ff, metrics); + } } private static class BatchReader extends GpuBatchDataReader implements PartitionReader { @@ -222,6 +281,8 @@ static class ReadTask implements InputPartition, Serializable { private final int maxBatchSizeRows; private final long maxBatchSizeBytes; private final String parquetDebugDumpPrefix; + private final int numThreads; + private final int maxNumFileProcessed; private transient Schema expectedSchema = null; private transient String[] preferredLocations = null; @@ -243,6 +304,8 @@ static class ReadTask implements InputPartition, Serializable { this.maxBatchSizeRows = rapidsConf.maxReadBatchSizeRows(); this.maxBatchSizeBytes = rapidsConf.maxReadBatchSizeBytes(); this.parquetDebugDumpPrefix = rapidsConf.parquetDebugDumpPrefix(); + this.numThreads = rapidsConf.multiThreadReadNumThreads(); + this.maxNumFileProcessed = rapidsConf.maxNumParquetFilesParallel(); } @Override @@ -278,6 +341,14 @@ public String getParquetDebugDumpPrefix() { return parquetDebugDumpPrefix; } + public int getNumThreads() { + return numThreads; + } + + public int getMaxNumFileProcessed() { + return maxNumFileProcessed; + } + private Schema expectedSchema() { if (expectedSchema == null) { this.expectedSchema = SchemaParser.fromJson(expectedSchemaString); diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/SparkBatch.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/SparkBatch.java index bc00688ca31..413d07694b0 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/SparkBatch.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/SparkBatch.java @@ -84,7 +84,7 @@ public InputPartition[] planInputPartitions() { @Override public PartitionReaderFactory createReaderFactory() { - return new GpuSparkScan.ReaderFactory(parentScan.metrics()); + return new GpuSparkScan.ReaderFactory(parentScan.metrics(), rapidsConf); } @Override diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala index 5e30295dc6c..b83751ee8fc 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala @@ -23,6 +23,7 @@ import java.util.concurrent.{Callable, ConcurrentLinkedQueue, Future, LinkedBloc import scala.annotation.tailrec import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, LinkedHashMap, Queue} +import scala.language.implicitConversions import scala.math.max import ai.rapids.cudf.{ColumnVector, HostMemoryBuffer, NvtxColor, NvtxRange, Table} @@ -128,6 +129,35 @@ object MultiFileReaderThreadPool extends Logging { } } +object MultiFileReaderUtils { + + private implicit def toURI(path: String): URI = { + try { + val uri = new URI(path) + if (uri.getScheme != null) { + return uri + } + } catch { + case _: URISyntaxException => + } + new File(path).getAbsoluteFile.toURI + } + + private def hasPathInCloud(filePaths: Array[String], cloudSchemes: Set[String]): Boolean = { + // Assume the `filePath` always has a scheme, if not try using the local filesystem. + // If that doesn't work for some reasons, users need to configure it directly. + filePaths.exists(fp => cloudSchemes.contains(fp.getScheme)) + } + + def useMultiThreadReader( + coalescingEnabled: Boolean, + multiThreadEnabled: Boolean, + files: Array[String], + cloudSchemes: Set[String]): Boolean = + !coalescingEnabled || (multiThreadEnabled && hasPathInCloud(files, cloudSchemes)) + +} + /** * The base multi-file partition reader factory to create the cloud reading or * coalescing reading respectively. @@ -210,39 +240,9 @@ abstract class MultiFilePartitionReaderFactoryBase( } /** for testing */ - private[rapids] def useMultiThread(filePaths: Array[String]): Boolean = { - !canUseCoalesceFilesReader || (canUseMultiThreadReader && arePathsInCloud(filePaths)) - } - - private def resolveURI(path: String): URI = { - try { - val uri = new URI(path) - if (uri.getScheme() != null) { - return uri - } - } catch { - case _: URISyntaxException => - } - new File(path).getAbsoluteFile().toURI() - } - - // We expect the filePath here to always have a scheme on it, - // if it doesn't we try using the local filesystem. If that - // doesn't work for some reason user would need to configure - // it directly. - private def isCloudFileSystem(filePath: String): Boolean = { - val uri = resolveURI(filePath) - val scheme = uri.getScheme - if (allCloudSchemes.contains(scheme)) { - true - } else { - false - } - } - - private def arePathsInCloud(filePaths: Array[String]): Boolean = { - filePaths.exists(isCloudFileSystem) - } + private[rapids] def useMultiThread(filePaths: Array[String]): Boolean = + MultiFileReaderUtils.useMultiThreadReader( + canUseCoalesceFilesReader, canUseMultiThreadReader, filePaths, allCloudSchemes) } /** @@ -387,7 +387,7 @@ abstract class MultiFileCloudPartitionReaderBase( override def next(): Boolean = { withResource(new NvtxRange(getFileFormatShortName + " readBatch", NvtxColor.GREEN)) { _ => - if (isInitted == false) { + if (!isInitted) { initAndStartReaders() } batch.foreach(_.close()) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index c144be4b41b..708822618f2 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -340,9 +340,10 @@ object GpuParquetPartitionReaderFactoryBase { } // contains meta about all the blocks in a file -private case class ParquetFileInfoWithBlockMeta(filePath: Path, blocks: Seq[BlockMetaData], - partValues: InternalRow, schema: MessageType, isCorrectedInt96RebaseMode: Boolean, - isCorrectedRebaseMode: Boolean, hasInt96Timestamps: Boolean) +case class ParquetFileInfoWithBlockMeta(filePath: Path, blocks: Seq[BlockMetaData], + partValues: InternalRow, schema: MessageType, readSchema: StructType, + isCorrectedInt96RebaseMode: Boolean, isCorrectedRebaseMode: Boolean, + hasInt96Timestamps: Boolean) /** * A parquet compatible stream that allows reading from a HostMemoryBuffer to Parquet. @@ -631,7 +632,7 @@ private case class GpuParquetFileFilterHandler(@transient sqlConf: SQLConf) exte block.setRowCount(numRows) val schema = new MessageType("root") return ParquetFileInfoWithBlockMeta(filePath, Seq(block), file.partitionValues, - schema, false, false, false) + schema, readDataSchema, false, false, false) } tableFooter.serializeThriftFile() @@ -706,8 +707,8 @@ private case class GpuParquetFileFilterHandler(@transient sqlConf: SQLConf) exte } ParquetFileInfoWithBlockMeta(filePath, clipped, file.partitionValues, - clippedSchema, isCorrectedInt96RebaseForThisFile, isCorrectedRebaseForThisFile, - hasInt96Timestamps) + clippedSchema, readDataSchema, isCorrectedInt96RebaseForThisFile, + isCorrectedRebaseForThisFile, hasInt96Timestamps) } } @@ -955,10 +956,12 @@ case class GpuParquetMultiFilePartitionReaderFactory( override def buildBaseColumnarReaderForCloud( files: Array[PartitionedFile], conf: Configuration): PartitionReader[ColumnarBatch] = { - new MultiFileCloudParquetPartitionReader(conf, footerReadType, files, - isCaseSensitive, readDataSchema, debugDumpPrefix, - maxReadBatchSizeRows, maxReadBatchSizeBytes, metrics, partitionSchema, - numThreads, maxNumFileProcessed, filterHandler, filters, + val filterFunc = (file: PartitionedFile) => { + filterHandler.filterBlocks(footerReadType, file, conf, filters, readDataSchema) + } + new MultiFileCloudParquetPartitionReader(conf, files, filterFunc, isCaseSensitive, + debugDumpPrefix, maxReadBatchSizeRows, maxReadBatchSizeBytes, + metrics, partitionSchema, numThreads, maxNumFileProcessed, ignoreMissingFiles, ignoreCorruptFiles, readUseFieldId) } @@ -980,7 +983,7 @@ case class GpuParquetMultiFilePartitionReaderFactory( case e: FileNotFoundException if ignoreMissingFiles => logWarning(s"Skipped missing file: ${file.filePath}", e) ParquetFileInfoWithBlockMeta(new Path(new URI(file.filePath)), Seq.empty, - file.partitionValues, null, false, false, false) + file.partitionValues, null, null, false, false, false) // Throw FileNotFoundException even if `ignoreCorruptFiles` is true case e: FileNotFoundException if !ignoreMissingFiles => throw e // If ignoreMissingFiles=true, this case will never be reached. But it's ok @@ -989,7 +992,7 @@ case class GpuParquetMultiFilePartitionReaderFactory( logWarning( s"Skipped the rest of the content in the corrupted file: ${file.filePath}", e) ParquetFileInfoWithBlockMeta(new Path(new URI(file.filePath)), Seq.empty, - file.partitionValues, null, false, false, false) + file.partitionValues, null, null, false, false, false) } clippedBlocks ++= singleFileInfo.blocks.map(block => ParquetSingleDataBlockMeta( @@ -1071,9 +1074,6 @@ trait ParquetPartitionReaderBase extends Logging with Arm with ScanWithMetrics def conf: Configuration def execMetrics: Map[String, GpuMetric] - // Schema to read - def readDataSchema: StructType - def isSchemaCaseSensitive: Boolean val copyBufferSize = conf.getInt("parquet.read.allocation.size", 8 * 1024 * 1024) @@ -1287,7 +1287,8 @@ trait ParquetPartitionReaderBase extends Logging with Arm with ScanWithMetrics protected def populateCurrentBlockChunk( blockIter: BufferedIterator[BlockMetaData], maxReadBatchSizeRows: Int, - maxReadBatchSizeBytes: Long): Seq[BlockMetaData] = { + maxReadBatchSizeBytes: Long, + readDataSchema: StructType): Seq[BlockMetaData] = { val currentChunk = new ArrayBuffer[BlockMetaData] var numRows: Long = 0 var numBytes: Long = 0 @@ -1442,7 +1443,7 @@ class MultiFileParquetPartitionReader( splits: Array[PartitionedFile], clippedBlocks: Seq[ParquetSingleDataBlockMeta], override val isSchemaCaseSensitive: Boolean, - override val readDataSchema: StructType, + readDataSchema: StructType, debugDumpPrefix: String, maxReadBatchSizeRows: Integer, maxReadBatchSizeBytes: Long, @@ -1635,8 +1636,8 @@ class MultiFileParquetPartitionReader( * * @param conf the Hadoop configuration * @param files the partitioned files to read + * @param filterFunc a function to filter the necessary blocks from a given file * @param isSchemaCaseSensitive whether schema is case sensitive - * @param readDataSchema the Spark schema describing what will be read * @param debugDumpPrefix a path prefix to use for dumping the fabricated Parquet data or null * @param maxReadBatchSizeRows soft limit on the maximum number of rows the reader reads per batch * @param maxReadBatchSizeBytes soft limit on the maximum number of bytes the reader reads per batch @@ -1645,17 +1646,15 @@ class MultiFileParquetPartitionReader( * @param numThreads the size of the threadpool * @param maxNumFileProcessed the maximum number of files to read on the CPU side and waiting to be * processed on the GPU. This affects the amount of host memory used. - * @param filterHandler GpuParquetFileFilterHandler used to filter the parquet blocks - * @param filters filters passed into the filterHandler * @param ignoreMissingFiles Whether to ignore missing files * @param ignoreCorruptFiles Whether to ignore corrupt files + * @param useFieldId Whether to ignore corrupt files */ class MultiFileCloudParquetPartitionReader( override val conf: Configuration, - footerReadType: ParquetFooterReaderType.Value, files: Array[PartitionedFile], + filterFunc: PartitionedFile => ParquetFileInfoWithBlockMeta, override val isSchemaCaseSensitive: Boolean, - override val readDataSchema: StructType, debugDumpPrefix: String, maxReadBatchSizeRows: Integer, maxReadBatchSizeBytes: Long, @@ -1663,15 +1662,12 @@ class MultiFileCloudParquetPartitionReader( partitionSchema: StructType, numThreads: Int, maxNumFileProcessed: Int, - filterHandler: GpuParquetFileFilterHandler, - filters: Array[Filter], ignoreMissingFiles: Boolean, ignoreCorruptFiles: Boolean, useFieldId: Boolean) - extends MultiFileCloudPartitionReaderBase(conf, files, numThreads, maxNumFileProcessed, filters, + extends MultiFileCloudPartitionReaderBase(conf, files, numThreads, maxNumFileProcessed, null, execMetrics, ignoreCorruptFiles) with ParquetPartitionReaderBase { - private case class HostMemoryEmptyMetaData( override val partitionedFile: PartitionedFile, bufferSize: Long, @@ -1693,15 +1689,13 @@ class MultiFileCloudParquetPartitionReader( isCorrectRebaseMode: Boolean, isCorrectInt96RebaseMode: Boolean, hasInt96Timestamps: Boolean, - clippedSchema: MessageType) extends HostMemoryBuffersWithMetaDataBase + clippedSchema: MessageType, + readSchema: StructType) extends HostMemoryBuffersWithMetaDataBase private class ReadBatchRunner( - footerReadType: ParquetFooterReaderType.Value, - taskContext: TaskContext, - filterHandler: GpuParquetFileFilterHandler, file: PartitionedFile, - conf: Configuration, - filters: Array[Filter]) extends Callable[HostMemoryBuffersWithMetaDataBase] with Logging { + filterFunc: PartitionedFile => ParquetFileInfoWithBlockMeta, + taskContext: TaskContext) extends Callable[HostMemoryBuffersWithMetaDataBase] with Logging { private var blockChunkIter: BufferedIterator[BlockMetaData] = null @@ -1736,14 +1730,13 @@ class MultiFileCloudParquetPartitionReader( val startingBytesRead = fileSystemBytesRead() val hostBuffers = new ArrayBuffer[(HostMemoryBuffer, Long)] try { - val fileBlockMeta = filterHandler.filterBlocks(footerReadType, file, conf, filters, - readDataSchema) + val fileBlockMeta = filterFunc(file) if (fileBlockMeta.blocks.isEmpty) { val bytesRead = fileSystemBytesRead() - startingBytesRead // no blocks so return null buffer and size 0 return HostMemoryEmptyMetaData(file, 0, bytesRead, fileBlockMeta.isCorrectedRebaseMode, fileBlockMeta.isCorrectedInt96RebaseMode, - fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, readDataSchema) + fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema) } blockChunkIter = fileBlockMeta.blocks.iterator.buffered if (isDone) { @@ -1751,7 +1744,7 @@ class MultiFileCloudParquetPartitionReader( // got close before finishing HostMemoryEmptyMetaData(file, 0, bytesRead, fileBlockMeta.isCorrectedRebaseMode, fileBlockMeta.isCorrectedInt96RebaseMode, - fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, readDataSchema) + fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema) } else { if (fileBlockMeta.schema.getFieldCount == 0) { val bytesRead = fileSystemBytesRead() - startingBytesRead @@ -1759,12 +1752,12 @@ class MultiFileCloudParquetPartitionReader( // overload size to be number of rows with null buffer HostMemoryEmptyMetaData(file, numRows, bytesRead, fileBlockMeta.isCorrectedRebaseMode, fileBlockMeta.isCorrectedInt96RebaseMode, - fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, readDataSchema) + fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema) } else { val filePath = new Path(new URI(file.filePath)) while (blockChunkIter.hasNext) { val blocksToRead = populateCurrentBlockChunk(blockChunkIter, - maxReadBatchSizeRows, maxReadBatchSizeBytes) + maxReadBatchSizeRows, maxReadBatchSizeBytes, fileBlockMeta.readSchema) hostBuffers += readPartFile(blocksToRead, fileBlockMeta.schema, filePath) } val bytesRead = fileSystemBytesRead() - startingBytesRead @@ -1773,11 +1766,11 @@ class MultiFileCloudParquetPartitionReader( hostBuffers.foreach(_._1.safeClose()) HostMemoryEmptyMetaData(file, 0, bytesRead, fileBlockMeta.isCorrectedRebaseMode, fileBlockMeta.isCorrectedInt96RebaseMode, - fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, readDataSchema) + fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema) } else { HostMemoryBuffersWithMetaData(file, hostBuffers.toArray, bytesRead, fileBlockMeta.isCorrectedRebaseMode, fileBlockMeta.isCorrectedInt96RebaseMode, - fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema) + fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema) } } } @@ -1803,7 +1796,7 @@ class MultiFileCloudParquetPartitionReader( file: PartitionedFile, conf: Configuration, filters: Array[Filter]): Callable[HostMemoryBuffersWithMetaDataBase] = { - new ReadBatchRunner(footerReadType, tc, filterHandler, file, conf, filters) + new ReadBatchRunner(file, filterFunc, tc) } /** @@ -1842,7 +1835,7 @@ class MultiFileCloudParquetPartitionReader( val (hostBuffer, size) = memBuffersAndSize.head val nextBatch = readBufferToTable(buffer.isCorrectRebaseMode, buffer.isCorrectInt96RebaseMode, buffer.hasInt96Timestamps, buffer.clippedSchema, - buffer.partitionedFile.partitionValues, + buffer.readSchema, buffer.partitionedFile.partitionValues, hostBuffer, size, buffer.partitionedFile.filePath) if (memBuffersAndSize.length > 1) { val updatedBuffers = memBuffersAndSize.drop(1) @@ -1860,6 +1853,7 @@ class MultiFileCloudParquetPartitionReader( isCorrectInt96RebaseMode: Boolean, hasInt96Timestamps: Boolean, clippedSchema: MessageType, + readDataSchema: StructType, partValues: InternalRow, hostBuffer: HostMemoryBuffer, dataSize: Long, @@ -1934,7 +1928,7 @@ class ParquetPartitionReader( clippedBlocks: Iterable[BlockMetaData], clippedParquetSchema: MessageType, override val isSchemaCaseSensitive: Boolean, - override val readDataSchema: StructType, + readDataSchema: StructType, debugDumpPrefix: String, maxReadBatchSizeRows: Integer, maxReadBatchSizeBytes: Long, @@ -1971,7 +1965,7 @@ class ParquetPartitionReader( private def readBatch(): Option[ColumnarBatch] = { withResource(new NvtxRange("Parquet readBatch", NvtxColor.GREEN)) { _ => val currentChunkedBlocks = populateCurrentBlockChunk(blockIterator, - maxReadBatchSizeRows, maxReadBatchSizeBytes) + maxReadBatchSizeRows, maxReadBatchSizeBytes, readDataSchema) if (clippedParquetSchema.getFieldCount == 0) { // not reading any data, so return a degenerate ColumnarBatch with the row count val numRows = currentChunkedBlocks.map(_.getRowCount).sum.toInt diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInputFileBlock.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInputFileBlock.scala index 53b6409230e..b23bc138618 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInputFileBlock.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInputFileBlock.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2022, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -54,6 +54,8 @@ object InputFileUtils { def setInputFileBlock(filePath: String, start: Long, length: Long): Unit = { InputFileBlockHolder.set(filePath, start, length) } + + def getCurInputFilePath(): String = InputFileBlockHolder.getInputFilePath.toString } /** From c24f790fec92135066ca832805a04de9c969917e Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Fri, 5 Aug 2022 12:54:17 -0500 Subject: [PATCH 012/190] Fix build error after recent auto-merge Signed-off-by: Jason Lowe --- .../com/nvidia/spark/rapids/GpuParquetScan.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index ffb10742fa6..10e41cfa245 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -1391,7 +1391,10 @@ trait ParquetPartitionReaderBase extends Logging with Arm with ScanWithMetrics } val sparkToParquetSchema = new SparkToParquetSchemaConverter() - def getParquetOptions(clippedSchema: MessageType, useFieldId: Boolean): ParquetOptions = { + def getParquetOptions( + readDataSchema: StructType, + clippedSchema: MessageType, + useFieldId: Boolean): ParquetOptions = { val includeColumns = toCudfColumnNamesAndDataTypes(readDataSchema, clippedSchema, isSchemaCaseSensitive, useFieldId) val builder = ParquetOptions.builder().withTimeUnit(DType.TIMESTAMP_MICROSECONDS) @@ -1582,7 +1585,7 @@ class MultiFileParquetPartitionReader( // Dump parquet data into a file dumpDataToFile(dataBuffer, dataSize, splits, Option(debugDumpPrefix), Some("parquet")) - val parseOpts = getParquetOptions(clippedSchema, useFieldId) + val parseOpts = getParquetOptions(readDataSchema, clippedSchema, useFieldId) // About to start using the GPU GpuSemaphore.acquireIfNecessary(TaskContext.get(), metrics(SEMAPHORE_WAIT_TIME)) @@ -1874,7 +1877,7 @@ class MultiFileCloudParquetPartitionReader( // Dump parquet data into a file dumpDataToFile(hostBuffer, dataSize, files, Option(debugDumpPrefix), Some("parquet")) - val parseOpts = getParquetOptions(clippedSchema, useFieldId) + val parseOpts = getParquetOptions(readDataSchema, clippedSchema, useFieldId) // about to start using the GPU GpuSemaphore.acquireIfNecessary(TaskContext.get(), metrics(SEMAPHORE_WAIT_TIME)) @@ -2013,7 +2016,7 @@ class ParquetPartitionReader( // Dump parquet data into a file dumpDataToFile(dataBuffer, dataSize, Array(split), Option(debugDumpPrefix), Some("parquet")) - val parseOpts = getParquetOptions(clippedParquetSchema, useFieldId) + val parseOpts = getParquetOptions(readDataSchema, clippedParquetSchema, useFieldId) // about to start using the GPU GpuSemaphore.acquireIfNecessary(TaskContext.get(), metrics(SEMAPHORE_WAIT_TIME)) From d79865634028d6e76de5176060b33e5ecef494e2 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Fri, 5 Aug 2022 13:50:37 -0700 Subject: [PATCH 013/190] Preserve underscore in executorEnv in integration tests (#6237) This PR fixes #6236. - don't replace '_' in the environment variable part of spark.executorEnv - refactoring Signed-off-by: Gera Shegalov --- .../src/main/python/spark_init_internal.py | 27 ++++++++++++++----- 1 file changed, 21 insertions(+), 6 deletions(-) diff --git a/integration_tests/src/main/python/spark_init_internal.py b/integration_tests/src/main/python/spark_init_internal.py index 26a6870243c..e36cc3d282b 100644 --- a/integration_tests/src/main/python/spark_init_internal.py +++ b/integration_tests/src/main/python/spark_init_internal.py @@ -21,7 +21,22 @@ findspark.init() import pyspark -_DRIVER_ENV = 'PYSP_TEST_spark_driver_extraJavaOptions' +_CONF_ENV_PREFIX = 'PYSP_TEST_' +_EXECUTOR_ENV_PREFIX = 'spark_executorEnv_' + +def env_for_conf(spark_conf_name): + return _CONF_ENV_PREFIX + spark_conf_name.replace('.', '_') + +def conf_for_env(env_name): + conf_key = env_name[len(_CONF_ENV_PREFIX):] + if conf_key.startswith(_EXECUTOR_ENV_PREFIX): + res = _EXECUTOR_ENV_PREFIX.replace('_', '.') + conf_key[len(_EXECUTOR_ENV_PREFIX):] + else: + res = conf_key.replace('_', '.') + return res + +_DRIVER_ENV = env_for_conf('spark.driver.extraJavaOptions') + def _spark__init(): #Force the RapidsPlugin to be enabled, so it blows up if the classpath is not set properly @@ -34,8 +49,8 @@ def _spark__init(): .config('spark.sql.queryExecutionListeners', 'com.nvidia.spark.rapids.ExecutionPlanCaptureCallback') for key, value in os.environ.items(): - if key.startswith('PYSP_TEST_') and key != _DRIVER_ENV: - _sb.config(key[10:].replace('_', '.'), value) + if key.startswith(_CONF_ENV_PREFIX) and key != _DRIVER_ENV: + _sb.config(conf_for_env(key), value) driver_opts = os.environ.get(_DRIVER_ENV, "") @@ -71,11 +86,11 @@ def _handle_event_log_dir(sb, wid): return spark_conf = pyspark.SparkConf() - master_url = os.environ.get('PYSP_TEST_spark_master', + master_url = os.environ.get(env_for_conf('spark.master'), spark_conf.get("spark.master", 'local')) - event_log_config = os.environ.get('PYSP_TEST_spark_eventLog_enabled', + event_log_config = os.environ.get(env_for_conf('spark.eventLog.enabled'), spark_conf.get('spark.eventLog.enabled', str(False).lower())) - event_log_codec = os.environ.get('PYSP_TEST_spark_eventLog_compression_codec', 'zstd') + event_log_codec = os.environ.get(env_for_conf('spark.eventLog.compression.codec'), 'zstd') if not master_url.startswith('local') or event_log_config != str(False).lower(): print("SPARK_EVENTLOG_ENABLED is ignored for non-local Spark master and when " From 61bf5fc2fa32dc599a4c1a0786607f8c8ee5e5b5 Mon Sep 17 00:00:00 2001 From: Anthony Chang <54450499+anthony-chang@users.noreply.github.com> Date: Fri, 5 Aug 2022 18:01:38 -0400 Subject: [PATCH 014/190] regexp: Catch "nothing to repeat" errors nested in groups (#6247) * Catch cases where the nothing is nested in groups, and for the ? quantifier case Signed-off-by: Anthony Chang * Move commented code Signed-off-by: Anthony Chang * Fix build error after recent auto-merge Signed-off-by: Jason Lowe (cherry picked from commit c24f790fec92135066ca832805a04de9c969917e) Co-authored-by: Jason Lowe --- .../nvidia/spark/rapids/GpuParquetScan.scala | 11 ++++--- .../com/nvidia/spark/rapids/RegexParser.scala | 32 ++++++++++++++++++- 2 files changed, 38 insertions(+), 5 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index ffb10742fa6..10e41cfa245 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -1391,7 +1391,10 @@ trait ParquetPartitionReaderBase extends Logging with Arm with ScanWithMetrics } val sparkToParquetSchema = new SparkToParquetSchemaConverter() - def getParquetOptions(clippedSchema: MessageType, useFieldId: Boolean): ParquetOptions = { + def getParquetOptions( + readDataSchema: StructType, + clippedSchema: MessageType, + useFieldId: Boolean): ParquetOptions = { val includeColumns = toCudfColumnNamesAndDataTypes(readDataSchema, clippedSchema, isSchemaCaseSensitive, useFieldId) val builder = ParquetOptions.builder().withTimeUnit(DType.TIMESTAMP_MICROSECONDS) @@ -1582,7 +1585,7 @@ class MultiFileParquetPartitionReader( // Dump parquet data into a file dumpDataToFile(dataBuffer, dataSize, splits, Option(debugDumpPrefix), Some("parquet")) - val parseOpts = getParquetOptions(clippedSchema, useFieldId) + val parseOpts = getParquetOptions(readDataSchema, clippedSchema, useFieldId) // About to start using the GPU GpuSemaphore.acquireIfNecessary(TaskContext.get(), metrics(SEMAPHORE_WAIT_TIME)) @@ -1874,7 +1877,7 @@ class MultiFileCloudParquetPartitionReader( // Dump parquet data into a file dumpDataToFile(hostBuffer, dataSize, files, Option(debugDumpPrefix), Some("parquet")) - val parseOpts = getParquetOptions(clippedSchema, useFieldId) + val parseOpts = getParquetOptions(readDataSchema, clippedSchema, useFieldId) // about to start using the GPU GpuSemaphore.acquireIfNecessary(TaskContext.get(), metrics(SEMAPHORE_WAIT_TIME)) @@ -2013,7 +2016,7 @@ class ParquetPartitionReader( // Dump parquet data into a file dumpDataToFile(dataBuffer, dataSize, Array(split), Option(debugDumpPrefix), Some("parquet")) - val parseOpts = getParquetOptions(clippedParquetSchema, useFieldId) + val parseOpts = getParquetOptions(readDataSchema, clippedParquetSchema, useFieldId) // about to start using the GPU GpuSemaphore.acquireIfNecessary(TaskContext.get(), metrics(SEMAPHORE_WAIT_TIME)) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala index 676450069ca..917c30d6d29 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala @@ -1377,7 +1377,11 @@ class CudfRegexTranspiler(mode: RegexMode) { s"cuDF does not support repetition of group containing: " + s"${unsupportedTerm.toRegexString}", term.position) } - case (RegexGroup(_, _), SimpleQuantifier(ch)) if ch == '?' => + case (RegexGroup(_, term), SimpleQuantifier(ch)) if ch == '?' => + if (isEntirelyWordBoundary(term) || isEntirelyLineAnchor(term)) { + throw new RegexUnsupportedException( + s"cuDF does not support repetition of: ${term.toRegexString}", term.position) + } RegexRepetition(rewrite(base, replacement, None), quantifier) case (RegexEscaped(ch), SimpleQuantifier('+')) if "AZ".contains(ch) => // \A+ can be transpiled to \A (dropping the repetition) @@ -1498,6 +1502,30 @@ class CudfRegexTranspiler(mode: RegexMode) { } } + private def isEntirely(regex: RegexAST, f: RegexAST => Boolean): Boolean = { + regex match { + case RegexSequence(parts) if parts.nonEmpty => + parts.forall(f) + case RegexGroup(_, term) => + isEntirely(term, f) + case _ => f(regex) + } + } + + private def isEntirelyWordBoundary(regex: RegexAST): Boolean = { + isEntirely(regex, { + case RegexEscaped(ch) if "bB".contains(ch) => true + case _ => false + }) + } + + private def isEntirelyLineAnchor(regex: RegexAST): Boolean = { + isEntirely(regex, { + case RegexEscaped('A') => true + case other => isBeginOrEndLineAnchor(other) + }) + } + private def endsWith(regex: RegexAST, f: RegexAST => Boolean): Boolean = { regex match { case RegexSequence(parts) if parts.nonEmpty => @@ -1506,6 +1534,8 @@ class CudfRegexTranspiler(mode: RegexMode) { case _ => true } endsWith(parts(j), f) + case RegexGroup(_, term) => + endsWith(term, f) case _ => f(regex) } } From c2063860d6a3ecb316e16ae3ccc77dd85ee559cd Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 8 Aug 2022 09:27:37 -0600 Subject: [PATCH 015/190] Add config option to log all query transformations (#6203) * add config option to log all query transformations * Add description for config option Signed-off-by: Andy Grove * log transformations in GpuTransitionOverrides * fix indent * remove thread local * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala Co-authored-by: Gera Shegalov Co-authored-by: Gera Shegalov --- .../com/nvidia/spark/rapids/GpuOverrides.scala | 15 ++++++++++++--- .../spark/rapids/GpuTransitionOverrides.scala | 6 ++++++ .../com/nvidia/spark/rapids/RapidsConf.scala | 8 ++++++++ 3 files changed, 26 insertions(+), 3 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index d725d174fff..93a2da1d489 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -4393,7 +4393,7 @@ case class GpuQueryStagePrepOverrides() extends Rule[SparkPlan] with Logging { override def apply(sparkPlan: SparkPlan): SparkPlan = GpuOverrideUtil.tryOverride { plan => // Note that we disregard the GPU plan returned here and instead rely on side effects of // tagging the underlying SparkPlan. - GpuOverrides().apply(plan) + GpuOverrides().applyWithContext(plan, Some("AQE Query Stage Prep")) // return the original plan which is now modified as a side-effect of invoking GpuOverrides plan }(sparkPlan) @@ -4403,13 +4403,22 @@ case class GpuOverrides() extends Rule[SparkPlan] with Logging { // Spark calls this method once for the whole plan when AQE is off. When AQE is on, it // gets called once for each query stage (where a query stage is an `Exchange`). - override def apply(sparkPlan: SparkPlan): SparkPlan = GpuOverrideUtil.tryOverride { plan => + override def apply(sparkPlan: SparkPlan): SparkPlan = applyWithContext(sparkPlan, None) + + def applyWithContext(sparkPlan: SparkPlan, context: Option[String]): SparkPlan = + GpuOverrideUtil.tryOverride { plan => val conf = new RapidsConf(plan.conf) if (conf.isSqlEnabled && conf.isSqlExecuteOnGPU) { GpuOverrides.logDuration(conf.shouldExplain, t => f"Plan conversion to the GPU took $t%.2f ms") { val updatedPlan = updateForAdaptivePlan(plan, conf) - applyOverrides(updatedPlan, conf) + val newPlan = applyOverrides(updatedPlan, conf) + if (conf.logQueryTransformations) { + val logPrefix = context.map(str => s"[$str]").getOrElse("") + logWarning(s"${logPrefix}Transformed query:" + + s"\nOriginal Plan:\n$plan\nTransformed Plan:\n$newPlan") + } + newPlan } } else if (conf.isSqlEnabled && conf.isSqlExplainOnlyEnabled) { // this mode logs the explain output and returns the original CPU plan diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala index 8c2253995d5..55ebeb2f57d 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala @@ -590,6 +590,12 @@ class GpuTransitionOverrides extends Rule[SparkPlan] { updatedPlan.canonicalized validateExecsInGpuPlan(updatedPlan, rapidsConf) } + + if (rapidsConf.logQueryTransformations) { + logWarning(s"Transformed query:" + + s"\nOriginal Plan:\n$plan\nTransformed Plan:\n$updatedPlan") + } + updatedPlan } } else { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index b00f4226c82..54b735a0264 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -1093,6 +1093,12 @@ object RapidsConf { .toSequence .createWithDefault(Nil) + val LOG_TRANSFORMATIONS = conf("spark.rapids.sql.debug.logTransformations") + .doc("When enabled, all query transformations will be logged.") + .internal() + .booleanConf + .createWithDefault(false) + val PARQUET_DEBUG_DUMP_PREFIX = conf("spark.rapids.sql.parquet.debug.dumpPrefix") .doc("A path prefix where Parquet split file data is dumped for debugging.") .internal() @@ -1676,6 +1682,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val validateExecsInGpuPlan: Seq[String] = get(TEST_VALIDATE_EXECS_ONGPU) + lazy val logQueryTransformations: Boolean = get(LOG_TRANSFORMATIONS) + lazy val rmmDebugLocation: String = get(RMM_DEBUG) lazy val gpuOomDumpDir: Option[String] = get(GPU_OOM_DUMP_DIR) From fa282aac0eb3401ee9eb86d878f5727da4d5dc88 Mon Sep 17 00:00:00 2001 From: Remzi Yang <59198230+HaoYang670@users.noreply.github.com> Date: Tue, 9 Aug 2022 22:26:36 +0800 Subject: [PATCH 016/190] Add `NaN` handling in `GpuMax` (#5989) Signed-off-by: remzi <13716567376yh@gmail.com> --- docs/supported_ops.md | 12 +-- .../src/main/python/hash_aggregate_test.py | 28 +++--- .../src/main/python/window_function_test.py | 8 ++ .../nvidia/spark/rapids/GpuOverrides.scala | 11 +-- .../spark/sql/rapids/AggregateFunctions.scala | 89 ++++++++++++++++++- .../spark/rapids/HashAggregatesSuite.scala | 15 ---- tools/src/main/resources/supportedExprs.csv | 6 +- 7 files changed, 121 insertions(+), 48 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 32ce24997cc..a3ce1bdf3db 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -16353,8 +16353,8 @@ are limited. S S S -PS
Input must not contain NaNs and spark.rapids.sql.hasNans must be false.
-PS
Input must not contain NaNs and spark.rapids.sql.hasNans must be false.
+S +S S PS
UTC is only supported TZ for TIMESTAMP
S @@ -16396,8 +16396,8 @@ are limited. S S S -PS
Input must not contain NaNs and spark.rapids.sql.hasNans must be false.
-PS
Input must not contain NaNs and spark.rapids.sql.hasNans must be false.
+S +S S PS
UTC is only supported TZ for TIMESTAMP
S @@ -16439,8 +16439,8 @@ are limited. S S S -PS
Input must not contain NaNs and spark.rapids.sql.hasNans must be false.
-PS
Input must not contain NaNs and spark.rapids.sql.hasNans must be false.
+S +S S PS
UTC is only supported TZ for TIMESTAMP
S diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index 09e4abd89a5..b2c67046e5a 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -623,6 +623,20 @@ def test_decimal128_min_max_group_by(data_gen): .groupby('a') .agg(f.min('b'), f.max('b'))) +@ignore_order(local=True) +@pytest.mark.parametrize('data_gen', [float_gen, double_gen], ids=idfn) +def test_float_max_reduction_with_nan(data_gen): + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, data_gen).selectExpr('max(a)')) + +@ignore_order(local=True) +@pytest.mark.parametrize('data_gen', [float_gen, double_gen], ids=idfn) +def test_float_max_group_by_with_nan(data_gen): + assert_gpu_and_cpu_are_equal_collect( + lambda spark: two_col_df(spark, byte_gen, data_gen) + .groupby('a') + .agg(f.max('b'))) + # to avoid ordering issues with collect_list we do it all in a single task @ignore_order(local=True) @pytest.mark.parametrize('data_gen', _gen_data_for_collect_list_op, ids=idfn) @@ -975,18 +989,6 @@ def test_hash_multiple_filters(data_gen, conf): 'min(a), max(b) filter (where c > 250) from hash_agg_table group by a', conf) - -@ignore_order -@allow_non_gpu('HashAggregateExec', 'AggregateExpression', 'AttributeReference', 'Alias', 'Max', - 'KnownFloatingPointNormalized', 'NormalizeNaNAndZero', 'ShuffleExchangeExec', - 'HashPartitioning') -@pytest.mark.parametrize('data_gen', struct_gens_xfail, ids=idfn) -def test_hash_query_max_nan_fallback(data_gen): - print_params(data_gen) - assert_gpu_fallback_collect( - lambda spark: gen_df(spark, data_gen, length=100).groupby('a').agg(f.max('b')), - "Max") - @approximate_float @ignore_order @pytest.mark.parametrize('data_gen', [_grpkey_floats_with_nan_zero_grouping_keys, @@ -1837,4 +1839,4 @@ def test_min_max_for_single_level_struct(data_gen): lambda spark : gen_df(spark, df_gen, length=1024), "hash_agg_table", 'select min(a) from hash_agg_table', - _no_nans_float_conf) + _no_nans_float_conf) \ No newline at end of file diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py index 03002fc1570..7869122fe43 100644 --- a/integration_tests/src/main/python/window_function_test.py +++ b/integration_tests/src/main/python/window_function_test.py @@ -144,6 +144,14 @@ 'spark.rapids.sql.castStringToFloat.enabled': 'true' } +@ignore_order(local=True) +@pytest.mark.parametrize('data_gen', [float_gen, double_gen], ids=idfn) +def test_float_window_max_with_nan(data_gen): + w = Window().partitionBy('a') + assert_gpu_and_cpu_are_equal_collect( + lambda spark: two_col_df(spark, byte_gen, data_gen) + .withColumn("max_b", f.max('a').over(w))) + @ignore_order @pytest.mark.parametrize('data_gen', [decimal_gen_128bit], ids=idfn) def test_decimal128_count_window(data_gen): diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 93a2da1d489..46d0eb1c553 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -2285,23 +2285,16 @@ object GpuOverrides extends Logging { TypeSig.orderable, Seq(ParamCheck("input", (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + TypeSig.STRUCT) - .nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL) - .withPsNote(Seq(TypeEnum.DOUBLE, TypeEnum.FLOAT), nanAggPsNote), + .nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL), TypeSig.orderable))).asInstanceOf[ExprChecksImpl].contexts ++ ExprChecks.windowOnly( (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL), TypeSig.orderable, Seq(ParamCheck("input", - (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL) - .withPsNote(Seq(TypeEnum.DOUBLE, TypeEnum.FLOAT), nanAggPsNote), + (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL), TypeSig.orderable))).asInstanceOf[ExprChecksImpl].contexts), (max, conf, p, r) => new AggExprMeta[Max](max, conf, p, r) { - override def tagAggForGpu(): Unit = { - val dataType = max.child.dataType - checkAndTagFloatNanAgg("Max", dataType, conf, this) - } - override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = GpuMax(childExprs.head) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala index 94e76a3ffb6..c064a41c05c 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala @@ -343,6 +343,15 @@ class CudfMax(override val dataType: DataType) extends CudfAggregate { override val name: String = "CudfMax" } +/** + * Check if there is a `true` value in a boolean column. + * The CUDF any aggregation does not work for reductions or group by aggregations + * so we use Max as a workaround for this. + */ +object CudfAny { + def apply(): CudfAggregate = new CudfMax(BooleanType) +} + class CudfMin(override val dataType: DataType) extends CudfAggregate { override lazy val reductionAggregate: cudf.ColumnVector => cudf.Scalar = (col: cudf.ColumnVector) => col.min @@ -513,10 +522,20 @@ case class GpuMin(child: Expression) extends GpuAggregateFunction } } -case class GpuMax(child: Expression) extends GpuAggregateFunction +object GpuMax { + def apply(child: Expression): GpuMax = { + child.dataType match { + case FloatType | DoubleType => GpuFloatMax(child) + case _ => GpuBasicMax(child) + } + } +} + +abstract class GpuMax(child: Expression) extends GpuAggregateFunction with GpuBatchedRunningWindowWithFixer with GpuAggregateWindowFunction - with GpuRunningWindowFunction { + with GpuRunningWindowFunction + with Serializable { override lazy val initialValues: Seq[GpuLiteral] = Seq(GpuLiteral(null, child.dataType)) override lazy val inputProjection: Seq[Expression] = Seq(child) override lazy val updateAggregates: Seq[CudfAggregate] = Seq(new CudfMax(dataType)) @@ -565,6 +584,72 @@ case class GpuMax(child: Expression) extends GpuAggregateFunction } } +/** Max aggregation without `NaN` handling */ +case class GpuBasicMax(child: Expression) extends GpuMax(child) + +/** Max aggregation for FloatType and DoubleType to handle `NaN`s. + * + * In Spark, `Nan` is the max float value, however in cuDF, `Infinity` is. + * We design a workaround method here to match the Spark's behaviour. + * The high level idea is that, in the projection stage, we create another + * column `isNan`. If any value in this column is true, return `Nan`, + * Else, return what `GpuBasicMax` returns. + */ +case class GpuFloatMax(child: Expression) extends GpuMax(child) + with GpuReplaceWindowFunction{ + + override val dataType: DataType = child.dataType match { + case FloatType | DoubleType => child.dataType + case t => throw new IllegalStateException(s"child type $t is not FloatType or DoubleType") + } + protected val nan: Any = child.dataType match { + case FloatType => Float.NaN + case DoubleType => Double.NaN + case t => throw new IllegalStateException(s"child type $t is not FloatType or DoubleType") + } + + protected lazy val updateIsNan = CudfAny() + protected lazy val updateMaxVal = new CudfMax(dataType) + protected lazy val mergeIsNan = CudfAny() + protected lazy val mergeMaxVal = new CudfMax(dataType) + + // Project 2 columns. The first one is the target column, second one is a + // Boolean column indicating whether the values in the target column are` Nan`s. + override lazy val inputProjection: Seq[Expression] = Seq(child, GpuIsNan(child)) + // Execute the `CudfMax` on the target column. At the same time, + // execute the `CudfAny` on the `isNan` column. + override lazy val updateAggregates: Seq[CudfAggregate] = Seq(updateMaxVal, updateIsNan) + // If there is `Nan` value in the target column, return `Nan` + // else return what the `CudfMax` returns + override lazy val postUpdate: Seq[Expression] = + Seq( + GpuIf(updateIsNan.attr, GpuLiteral(nan, dataType), updateMaxVal.attr) + ) + + // Same logic as the `inputProjection` stage. + override lazy val preMerge: Seq[Expression] = + Seq(evaluateExpression, GpuIsNan(evaluateExpression)) + // Same logic as the `updateAggregates` stage. + override lazy val mergeAggregates: Seq[CudfAggregate] = Seq(mergeMaxVal, mergeIsNan) + // Same logic as the `postUpdate` stage. + override lazy val postMerge: Seq[Expression] = + Seq( + GpuIf(mergeIsNan.attr, GpuLiteral(nan, dataType), mergeMaxVal.attr) + ) + + // We should always override the windowing expression to handle `Nan`. + override def shouldReplaceWindow(spec: GpuWindowSpecDefinition): Boolean = true + + override def windowReplacement(spec: GpuWindowSpecDefinition): Expression = { + // The `GpuBasicMax` here has the same functionality as `CudfAny`, + // as `true > false` in cuDF. + val isNan = GpuWindowExpression(GpuBasicMax(GpuIsNan(child)), spec) + // We use `GpuBasicMax` but not `GpuMax` to avoid self recursion. + val max = GpuWindowExpression(GpuBasicMax(child), spec) + GpuIf(isNan, GpuLiteral(nan, dataType), max) + } +} + /** * Extracts a 32-bit chunk from a 128-bit value * @param data expression producing 128-bit values diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala index 819150ae693..ede7fba002a 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala @@ -1618,21 +1618,6 @@ class HashAggregatesSuite extends SparkQueryCompareTestSuite { frame => frame.groupBy("large_longs").agg(avg("large_longs")) } { (_, gpuPlan) => checkExecPlan(gpuPlan) } - ALLOW_NON_GPU_testSparkResultsAreEqualWithCapture( - "max_with_nans_fall_back", - nanDf, - Seq("HashAggregateExec", "AggregateExpression", - "AttributeReference", "Alias", "Max", "ShuffleExchangeExec"), - conf = enableCsvConf()) { - frame => frame.agg(max("doubles")) - } { (_, gpuPlan) => { - // verify nothing ran on the gpu - if (gpuPlan.conf.getAllConfs(RapidsConf.SQL_ENABLED.key).toBoolean) { - val execNode = gpuPlan.find(_.isInstanceOf[GpuHashAggregateExec]) - assert(execNode.isEmpty) - } - }} - ALLOW_NON_GPU_testSparkResultsAreEqualWithCapture( "min_with_nans_fall_back", nanDf, diff --git a/tools/src/main/resources/supportedExprs.csv b/tools/src/main/resources/supportedExprs.csv index 1a58bfcb662..2e3ff5ae942 100644 --- a/tools/src/main/resources/supportedExprs.csv +++ b/tools/src/main/resources/supportedExprs.csv @@ -619,11 +619,11 @@ Last,S,`last`; `last_value`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS, Last,S,`last`; `last_value`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS Last,S,`last`; `last_value`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS Last,S,`last`; `last_value`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS -Max,S,`max`,None,aggregation,input,S,S,S,S,S,PS,PS,S,PS,S,S,S,NS,NS,NS,NA,PS,NS +Max,S,`max`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS Max,S,`max`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS -Max,S,`max`,None,reduction,input,S,S,S,S,S,PS,PS,S,PS,S,S,S,NS,NS,NS,NA,PS,NS +Max,S,`max`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS Max,S,`max`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS -Max,S,`max`,None,window,input,S,S,S,S,S,PS,PS,S,PS,S,S,S,NS,NS,NS,NA,NS,NS +Max,S,`max`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS Max,S,`max`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS Min,S,`min`,None,aggregation,input,S,S,S,S,S,PS,PS,S,PS,S,S,S,NS,NS,NS,NA,PS,NS Min,S,`min`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS From 341cfea514c5ec3f1bbea6a7915e40939fb33f40 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Wed, 10 Aug 2022 10:28:08 +0800 Subject: [PATCH 017/190] Read metadata only when read schema is empty in Avro multi-threaded reading (#6241) * Read metadata only when readDataSchema is empty in Avro multi-threaded reading * Better solution with skipCurrentBlock in file reader and add tests * Add skip to AvroFileReader * Code style changes * Code style changes * Code style changes * Code style changes * Code style changes Signed-off-by: thirtiseven --- .../src/main/python/avro_test.py | 16 +++++++++-- .../spark/rapids/AvroDataFileReader.scala | 18 ++++++++++++ .../apache/spark/sql/rapids/GpuAvroScan.scala | 28 +++++++++++++------ 3 files changed, 51 insertions(+), 11 deletions(-) diff --git a/integration_tests/src/main/python/avro_test.py b/integration_tests/src/main/python/avro_test.py index 05e0092f273..c5a7765af9f 100644 --- a/integration_tests/src/main/python/avro_test.py +++ b/integration_tests/src/main/python/avro_test.py @@ -138,15 +138,25 @@ def test_avro_read_with_corrupt_files(spark_tmp_path, reader_type, v1_enabled_li conf=all_confs) +# 10 rows is a small batch size for multi-batch read test, 2147483647 is the default value +@pytest.mark.parametrize('batch_size_rows', [10, 2147483647]) @pytest.mark.parametrize('v1_enabled_list', ["avro", ""], ids=["v1", "v2"]) @pytest.mark.parametrize('reader_type', rapids_reader_types) -def test_read_count(spark_tmp_path, v1_enabled_list, reader_type): +def test_read_count(spark_tmp_path, v1_enabled_list, reader_type, batch_size_rows): data_path = spark_tmp_path + '/AVRO_DATA' - gen_avro_files([('_c0', int_gen)], data_path) + + # the default block size of the avro file is about 64kb, so we need to generate a larger file + # to test multi-batch read. length=30000 will generate 2 blocks in each partition. + with_cpu_session( + lambda spark: gen_df(spark, [('_c0', int_gen)], length=30000) + .repartition(2).write.format("avro").save(data_path) + ) all_confs = copy_and_update(_enable_all_types_conf, { 'spark.rapids.sql.format.avro.reader.type': reader_type, - 'spark.sql.sources.useV1SourceList': v1_enabled_list}) + 'spark.sql.sources.useV1SourceList': v1_enabled_list, + 'spark.rapids.sql.reader.batchSizeRows': batch_size_rows}) + assert_gpu_and_cpu_row_counts_equal( lambda spark: spark.read.format("avro").load(data_path), conf=all_confs) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AvroDataFileReader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AvroDataFileReader.scala index 86e37393d48..ef591c79b45 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AvroDataFileReader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AvroDataFileReader.scala @@ -205,6 +205,11 @@ abstract class AvroFileReader(si: SeekableInput) extends AutoCloseable { (curBlockStart >= position + SYNC_SIZE) || (curBlockStart >= sin.length()) } + /** Skip next length bytes */ + def skip(length: Int): Unit = { + vin.skipFixed(length) + } + /** * Move to the next synchronization point after a position. To process a range * of file entries, call this with the starting position, then check @@ -436,6 +441,19 @@ class AvroDataFileReader(si: SeekableInput) extends AvroFileReader(si) { curBlockReady = false } + /** + * Skip the current raw block + */ + def skipCurrentBlock(): Unit = { + if (!hasNextBlock) { + throw new NoSuchElementException + } + val dataSize = curDataSize.toInt + skip(dataSize + SYNC_SIZE) + curBlockStart = sin.tell - vin.inputStream.available + curBlockReady = false + } + } object AvroFileReader extends Arm { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala index 1add52ae27d..87ceaab9bc5 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala @@ -745,18 +745,31 @@ class GpuMultiFileCloudAvroPartitionReader( } else { estBlocksSize } + val optHmb = if (readDataSchema.nonEmpty) { + Some(HostMemoryBuffer.allocate(headerSize + estSizeToRead)) + } else None // Allocate the buffer for the header and blocks for a batch - closeOnExcept(HostMemoryBuffer.allocate(headerSize + estSizeToRead)) { hmb => - val out = new HostMemoryOutputStream(hmb) - // Write the header to the output stream - AvroFileWriter(out).writeHeader(reader.header) + closeOnExcept(optHmb) { _ => + val optOut = optHmb.map { hmb => + val out = new HostMemoryOutputStream(hmb) + // Write the header to the output stream + AvroFileWriter(out).writeHeader(reader.header) + out + } // Read the block data to the output stream var batchRowsNum: Int = 0 + var batchSize: Long = 0 var hasNextBlock = true do { - reader.readNextRawBlock(out) + if (optOut.nonEmpty) { + reader.readNextRawBlock(optOut.get) + } else { + // skip the current block + reader.skipCurrentBlock() + } batchRowsNum += curBlock.count.toInt estSizeToRead -= curBlock.blockSize + batchSize += curBlock.blockSize // Continue reading the next block into the current batch when // - the next block exists, and // - the remaining buffer is enough to hold the next block, and @@ -769,14 +782,13 @@ class GpuMultiFileCloudAvroPartitionReader( batchRowsNum <= maxReadBatchSizeRows) // One batch is done - hostBuffers += ((hmb, out.getPos)) + optOut.foreach(out => hostBuffers += ((optHmb.get, out.getPos))) totalRowsNum += batchRowsNum - estBlocksSize -= (out.getPos - headerSize) + estBlocksSize -= batchSize } } // end of while val bufAndSize: Array[(HostMemoryBuffer, Long)] = if (readDataSchema.isEmpty) { - // Overload the size to be the number of rows with null buffer hostBuffers.foreach(_._1.safeClose(new Exception)) Array((null, totalRowsNum)) } else if (isDone) { From 2b9fa502caac77b451e8a6dc84842a3b182592c8 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Thu, 11 Aug 2022 18:38:22 -0500 Subject: [PATCH 018/190] Add BinaryType support to operations that already support arrays (#6298) Signed-off-by: Jason Lowe Signed-off-by: Jason Lowe --- docs/supported_ops.md | 282 +++++++++--------- .../src/main/python/array_test.py | 8 +- integration_tests/src/main/python/cmp_test.py | 6 +- .../src/main/python/collection_ops_test.py | 4 +- .../src/main/python/conditionals_test.py | 4 +- integration_tests/src/main/python/data_gen.py | 4 + .../src/main/python/generate_expr_test.py | 46 +-- .../src/main/python/join_test.py | 8 +- integration_tests/src/main/python/map_test.py | 9 +- .../src/main/python/repart_test.py | 8 +- .../src/main/python/struct_test.py | 5 +- .../nvidia/spark/rapids/GpuOverrides.scala | 113 +++---- .../com/nvidia/spark/rapids/TypeChecks.scala | 6 +- .../com/nvidia/spark/rapids/literals.scala | 16 + .../sql/rapids/execution/GpuHashJoin.scala | 3 +- tools/src/main/resources/supportedExecs.csv | 22 +- tools/src/main/resources/supportedExprs.csv | 38 +-- 17 files changed, 308 insertions(+), 274 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index a3ce1bdf3db..603fd74a128 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -146,11 +146,11 @@ Accelerator supports are described below. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -242,11 +242,11 @@ Accelerator supports are described below. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -266,11 +266,11 @@ Accelerator supports are described below. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -338,11 +338,11 @@ Accelerator supports are described below. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -698,7 +698,7 @@ Accelerator supports are described below. S S S -NS +S NS PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, ARRAY, MAP, UDT
PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, ARRAY, MAP, UDT
@@ -722,11 +722,11 @@ Accelerator supports are described below. S S S +S NS -NS -PS
Round-robin partitioning is not supported if spark.sql.execution.sortBeforeRepartition is true;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
Round-robin partitioning is not supported if spark.sql.execution.sortBeforeRepartition is true;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
Round-robin partitioning is not supported for nested structs if spark.sql.execution.sortBeforeRepartition is true;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
Round-robin partitioning is not supported if spark.sql.execution.sortBeforeRepartition is true;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
Round-robin partitioning is not supported if spark.sql.execution.sortBeforeRepartition is true;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
Round-robin partitioning is not supported for nested structs if spark.sql.execution.sortBeforeRepartition is true;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -809,11 +809,11 @@ Accelerator supports are described below. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -854,11 +854,11 @@ Accelerator supports are described below. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -902,11 +902,11 @@ Accelerator supports are described below. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -989,11 +989,11 @@ Accelerator supports are described below. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -1076,11 +1076,11 @@ Accelerator supports are described below. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -1758,11 +1758,11 @@ are limited. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -1779,11 +1779,11 @@ are limited. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -2672,7 +2672,7 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -2693,7 +2693,7 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -2923,11 +2923,11 @@ are limited. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -3872,11 +3872,11 @@ are limited. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -3893,11 +3893,11 @@ are limited. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -4129,11 +4129,11 @@ are limited. S S S +S NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, MAP, UDT
NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
-NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, MAP, UDT
NS @@ -4150,11 +4150,11 @@ are limited. S S S +S NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, MAP, UDT
NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
-NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, MAP, UDT
NS @@ -4178,7 +4178,7 @@ are limited. NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -4199,7 +4199,7 @@ are limited. NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -4634,11 +4634,11 @@ are limited. S S S +S NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, MAP, UDT
NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
-NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, MAP, UDT
NS @@ -4657,7 +4657,7 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, MAP, UDT
@@ -4749,11 +4749,11 @@ are limited. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -4774,7 +4774,7 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -5472,8 +5472,8 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
If it's map, only primitive key types are supported.;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
If it's map, only primitive key types are supported.;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -5512,11 +5512,11 @@ are limited. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -5924,8 +5924,8 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -5945,7 +5945,7 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -6202,7 +6202,7 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -6242,11 +6242,11 @@ are limited. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -6412,7 +6412,7 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -6451,11 +6451,11 @@ are limited. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -6481,7 +6481,7 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -6498,11 +6498,11 @@ are limited. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -7065,11 +7065,11 @@ are limited. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -7086,11 +7086,11 @@ are limited. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -7107,11 +7107,11 @@ are limited. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -7514,11 +7514,11 @@ are limited. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -7561,11 +7561,11 @@ are limited. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -8546,11 +8546,11 @@ are limited. S S S -NS S -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+S +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -9020,7 +9020,7 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -9040,7 +9040,7 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -9135,7 +9135,7 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -9155,7 +9155,7 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -9182,7 +9182,7 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -9202,7 +9202,7 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -10198,8 +10198,8 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -10219,7 +10219,7 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -12234,8 +12234,8 @@ are limited. -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
@@ -17955,7 +17955,7 @@ and the accelerator produces the same result. NS -NS +S @@ -18359,7 +18359,7 @@ and the accelerator produces the same result. NS -NS +S diff --git a/integration_tests/src/main/python/array_test.py b/integration_tests/src/main/python/array_test.py index 79b01df0023..c6d7178558f 100644 --- a/integration_tests/src/main/python/array_test.py +++ b/integration_tests/src/main/python/array_test.py @@ -33,7 +33,8 @@ array_all_null_gen = ArrayGen(int_gen, all_null=True) array_item_test_gens = array_gens_sample + [array_all_null_gen, - ArrayGen(MapGen(StringGen(pattern='key_[0-9]', nullable=False), StringGen(), max_length=10), max_length=10)] + ArrayGen(MapGen(StringGen(pattern='key_[0-9]', nullable=False), StringGen(), max_length=10), max_length=10), + ArrayGen(BinaryGen(max_length=10), max_length=10)] # Need these for set-based operations @@ -153,7 +154,7 @@ def test_array_item_ansi_not_fail_all_null_data(): @pytest.mark.parametrize('data_gen', all_basic_gens + [ - decimal_gen_32bit, decimal_gen_64bit, decimal_gen_128bit, + decimal_gen_32bit, decimal_gen_64bit, decimal_gen_128bit, binary_gen, StructGen([['child0', StructGen([['child01', IntegerGen()]])], ['child1', string_gen], ['child2', float_gen]], nullable=False), StructGen([['child0', byte_gen], ['child1', string_gen], ['child2', float_gen]], nullable=False)], ids=idfn) def test_make_array(data_gen): @@ -415,7 +416,8 @@ def do_it(spark): }) -array_zips_gen = array_gens_sample + [ArrayGen(map_string_string_gen[0], max_length=5)] +array_zips_gen = array_gens_sample + [ArrayGen(map_string_string_gen[0], max_length=5), + ArrayGen(BinaryGen(max_length=5), max_length=5)] @pytest.mark.parametrize('data_gen', array_zips_gen, ids=idfn) diff --git a/integration_tests/src/main/python/cmp_test.py b/integration_tests/src/main/python/cmp_test.py index 3bc99d85a8c..2d414bbdd5b 100644 --- a/integration_tests/src/main/python/cmp_test.py +++ b/integration_tests/src/main/python/cmp_test.py @@ -196,7 +196,7 @@ def test_gte_for_interval(): f.col('b') >= f.lit(None).cast(data_type), f.col('a') >= f.col('b'))) -@pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + array_gens_sample + struct_gens_sample + map_gens_sample, ids=idfn) +@pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + [binary_gen] + array_gens_sample + struct_gens_sample + map_gens_sample, ids=idfn) def test_isnull(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).select( @@ -215,12 +215,12 @@ def test_isnan(data_gen): lambda spark : unary_op_df(spark, data_gen).select( f.isnan(f.col('a')))) -@pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + array_gens_sample + struct_gens_sample + map_gens_sample, ids=idfn) +@pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + [binary_gen] + array_gens_sample + struct_gens_sample + map_gens_sample, ids=idfn) def test_dropna_any(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : binary_op_df(spark, data_gen).dropna()) -@pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + array_gens_sample + struct_gens_sample + map_gens_sample, ids=idfn) +@pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + [binary_gen] + array_gens_sample + struct_gens_sample + map_gens_sample, ids=idfn) def test_dropna_all(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : binary_op_df(spark, data_gen).dropna(how='all')) diff --git a/integration_tests/src/main/python/collection_ops_test.py b/integration_tests/src/main/python/collection_ops_test.py index 7e31c512f49..17b57156bf0 100644 --- a/integration_tests/src/main/python/collection_ops_test.py +++ b/integration_tests/src/main/python/collection_ops_test.py @@ -22,7 +22,9 @@ nested_gens = [ArrayGen(LongGen()), ArrayGen(decimal_gen_128bit), StructGen([("a", LongGen()), ("b", decimal_gen_128bit)]), - MapGen(StringGen(pattern='key_[0-9]', nullable=False), StringGen())] + MapGen(StringGen(pattern='key_[0-9]', nullable=False), StringGen()), + ArrayGen(BinaryGen(max_length=5)), + MapGen(IntegerGen(nullable=False), BinaryGen(max_length=5))] # additional test for NonNull Array because of https://github.com/rapidsai/cudf/pull/8181 non_nested_array_gens = [ArrayGen(sub_gen, nullable=nullable) for nullable in [True, False] diff --git a/integration_tests/src/main/python/conditionals_test.py b/integration_tests/src/main/python/conditionals_test.py index 33c2d8ca31b..19271c3961d 100644 --- a/integration_tests/src/main/python/conditionals_test.py +++ b/integration_tests/src/main/python/conditionals_test.py @@ -23,8 +23,8 @@ def mk_str_gen(pattern): return StringGen(pattern).with_special_case('').with_special_pattern('.{0,10}') -all_gens = all_gen + [NullGen()] -all_nested_gens = array_gens_sample + struct_gens_sample + map_gens_sample +all_gens = all_gen + [NullGen(), binary_gen] +all_nested_gens = array_gens_sample + [ArrayGen(BinaryGen(max_length=10), max_length=10)] + struct_gens_sample + map_gens_sample all_nested_gens_nonempty_struct = array_gens_sample + nonempty_struct_gens_sample # Create dedicated data gens of nested type for 'if' tests here with two exclusions: diff --git a/integration_tests/src/main/python/data_gen.py b/integration_tests/src/main/python/data_gen.py index 7d1ad0856ac..20db0478ac6 100644 --- a/integration_tests/src/main/python/data_gen.py +++ b/integration_tests/src/main/python/data_gen.py @@ -840,6 +840,8 @@ def to_cast_string(spark_type): elif isinstance(spark_type, StructType): children = [fd.name + ':' + to_cast_string(fd.dataType) for fd in spark_type.fields] return 'STRUCT<{}>'.format(','.join(children)) + elif isinstance(spark_type, BinaryType): + return 'BINARY' else: raise RuntimeError('CAST TO TYPE {} NOT SUPPORTED YET'.format(spark_type)) @@ -866,6 +868,8 @@ def _convert_to_sql(spark_type, data): children = ["'{}'".format(fd.name) + ',' + _convert_to_sql(fd.dataType, x) for fd, x in zip(spark_type.fields, data)] d = "named_struct({})".format(','.join(children)) + elif isinstance(data, bytearray) or isinstance(data, bytes): + d = "X'{}'".format(data.hex()) elif not data: # data is None d = "null" diff --git a/integration_tests/src/main/python/generate_expr_test.py b/integration_tests/src/main/python/generate_expr_test.py index a0e2cbd8f33..03c3257be40 100644 --- a/integration_tests/src/main/python/generate_expr_test.py +++ b/integration_tests/src/main/python/generate_expr_test.py @@ -22,6 +22,10 @@ pytestmark = pytest.mark.nightly_resource_consuming_test +explode_gens = all_gen + [binary_gen] +arrays_with_binary = [ArrayGen(BinaryGen(max_length=5))] +maps_with_binary = [MapGen(IntegerGen(nullable=False), BinaryGen(max_length=5))] + def four_op_df(spark, gen, length=2048, seed=0): return gen_df(spark, StructGen([ ('a', gen), @@ -32,7 +36,7 @@ def four_op_df(spark, gen, length=2048, seed=0): #sort locally because of https://github.com/NVIDIA/spark-rapids/issues/84 # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) -@pytest.mark.parametrize('data_gen', all_gen, ids=idfn) +@pytest.mark.parametrize('data_gen', explode_gens, ids=idfn) def test_explode_makearray(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : four_op_df(spark, data_gen).selectExpr('a', 'explode(array(b, c, d))')) @@ -40,7 +44,7 @@ def test_explode_makearray(data_gen): #sort locally because of https://github.com/NVIDIA/spark-rapids/issues/84 # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) -@pytest.mark.parametrize('data_gen', all_gen, ids=idfn) +@pytest.mark.parametrize('data_gen', explode_gens, ids=idfn) def test_explode_litarray(data_gen): array_lit = gen_scalar(ArrayGen(data_gen, min_length=3, max_length=3, nullable=False)) assert_gpu_and_cpu_are_equal_collect( @@ -52,8 +56,8 @@ def test_explode_litarray(data_gen): @ignore_order(local=True) @pytest.mark.order(1) # at the head of xdist worker queue if pytest-order is installed -@pytest.mark.parametrize('data_gen', all_gen + struct_gens_sample_with_decimal128 + - array_gens_sample + map_gens_sample, +@pytest.mark.parametrize('data_gen', explode_gens + struct_gens_sample_with_decimal128 + + array_gens_sample + map_gens_sample + arrays_with_binary + maps_with_binary, ids=idfn) def test_explode_array_data(data_gen): data_gen = [int_gen, ArrayGen(data_gen)] @@ -64,7 +68,7 @@ def test_explode_array_data(data_gen): #sort locally because of https://github.com/NVIDIA/spark-rapids/issues/84 # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) -@pytest.mark.parametrize('map_gen', map_gens_sample + decimal_128_map_gens, ids=idfn) +@pytest.mark.parametrize('map_gen', map_gens_sample + decimal_128_map_gens + maps_with_binary, ids=idfn) def test_explode_map_data(map_gen): data_gen = [int_gen, map_gen] assert_gpu_and_cpu_are_equal_collect( @@ -74,7 +78,7 @@ def test_explode_map_data(map_gen): #sort locally because of https://github.com/NVIDIA/spark-rapids/issues/84 # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) -@pytest.mark.parametrize('data_gen', all_gen, ids=idfn) +@pytest.mark.parametrize('data_gen', explode_gens, ids=idfn) def test_explode_nested_array_data(data_gen): data_gen = [int_gen, ArrayGen(ArrayGen(data_gen))] assert_gpu_and_cpu_are_equal_collect( @@ -86,10 +90,10 @@ def test_explode_nested_array_data(data_gen): # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) @pytest.mark.order(1) # at the head of xdist worker queue if pytest-order is installed -@pytest.mark.parametrize('data_gen', all_gen + struct_gens_sample_with_decimal128 + - array_gens_sample + map_gens_sample, +@pytest.mark.parametrize('data_gen', explode_gens + struct_gens_sample_with_decimal128 + + array_gens_sample + arrays_with_binary + map_gens_sample + maps_with_binary, ids=idfn) -def test_explode_outer_array_data(spark_tmp_path, data_gen): +def test_explode_outer_array_data(data_gen): data_gen = [int_gen, ArrayGen(data_gen)] assert_gpu_and_cpu_are_equal_collect( lambda spark: two_col_df(spark, *data_gen).selectExpr('a', 'explode_outer(b)'), @@ -98,7 +102,7 @@ def test_explode_outer_array_data(spark_tmp_path, data_gen): #sort locally because of https://github.com/NVIDIA/spark-rapids/issues/84 # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) -@pytest.mark.parametrize('map_gen', map_gens_sample + decimal_128_map_gens, ids=idfn) +@pytest.mark.parametrize('map_gen', map_gens_sample + decimal_128_map_gens + maps_with_binary, ids=idfn) def test_explode_outer_map_data(map_gen): data_gen = [int_gen, map_gen] assert_gpu_and_cpu_are_equal_collect( @@ -108,7 +112,7 @@ def test_explode_outer_map_data(map_gen): #sort locally because of https://github.com/NVIDIA/spark-rapids/issues/84 # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) -@pytest.mark.parametrize('data_gen', all_gen, ids=idfn) +@pytest.mark.parametrize('data_gen', explode_gens, ids=idfn) def test_explode_outer_nested_array_data(data_gen): data_gen = [int_gen, ArrayGen(ArrayGen(data_gen))] assert_gpu_and_cpu_are_equal_collect( @@ -119,7 +123,7 @@ def test_explode_outer_nested_array_data(data_gen): #sort locally because of https://github.com/NVIDIA/spark-rapids/issues/84 # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) -@pytest.mark.parametrize('data_gen', all_gen, ids=idfn) +@pytest.mark.parametrize('data_gen', explode_gens, ids=idfn) def test_posexplode_makearray(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : four_op_df(spark, data_gen).selectExpr('posexplode(array(b, c, d))', 'a')) @@ -127,7 +131,7 @@ def test_posexplode_makearray(data_gen): #sort locally because of https://github.com/NVIDIA/spark-rapids/issues/84 # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) -@pytest.mark.parametrize('data_gen', all_gen, ids=idfn) +@pytest.mark.parametrize('data_gen', explode_gens, ids=idfn) def test_posexplode_litarray(data_gen): array_lit = gen_scalar(ArrayGen(data_gen, min_length=3, max_length=3, nullable=False)) assert_gpu_and_cpu_are_equal_collect( @@ -138,8 +142,8 @@ def test_posexplode_litarray(data_gen): # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) @pytest.mark.order(1) # at the head of xdist worker queue if pytest-order is installed -@pytest.mark.parametrize('data_gen', all_gen + struct_gens_sample_with_decimal128 + - array_gens_sample + map_gens_sample, +@pytest.mark.parametrize('data_gen', explode_gens + struct_gens_sample_with_decimal128 + + array_gens_sample + arrays_with_binary + map_gens_sample + maps_with_binary, ids=idfn) def test_posexplode_array_data(data_gen): data_gen = [int_gen, ArrayGen(data_gen)] @@ -150,7 +154,7 @@ def test_posexplode_array_data(data_gen): #sort locally because of https://github.com/NVIDIA/spark-rapids/issues/84 # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) -@pytest.mark.parametrize('map_gen', map_gens_sample + decimal_128_map_gens, ids=idfn) +@pytest.mark.parametrize('map_gen', map_gens_sample + decimal_128_map_gens + maps_with_binary, ids=idfn) def test_posexplode_map_data(map_gen): data_gen = [int_gen, map_gen] assert_gpu_and_cpu_are_equal_collect( @@ -160,7 +164,7 @@ def test_posexplode_map_data(map_gen): #sort locally because of https://github.com/NVIDIA/spark-rapids/issues/84 # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) -@pytest.mark.parametrize('data_gen', all_gen, ids=idfn) +@pytest.mark.parametrize('data_gen', explode_gens, ids=idfn) def test_posexplode_nested_array_data(data_gen): data_gen = [int_gen, ArrayGen(ArrayGen(data_gen))] assert_gpu_and_cpu_are_equal_collect( @@ -172,8 +176,8 @@ def test_posexplode_nested_array_data(data_gen): # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) @pytest.mark.order(1) # at the head of xdist worker queue if pytest-order is installed -@pytest.mark.parametrize('data_gen', all_gen + struct_gens_sample_with_decimal128 + - array_gens_sample + map_gens_sample, +@pytest.mark.parametrize('data_gen', explode_gens + struct_gens_sample_with_decimal128 + + array_gens_sample + arrays_with_binary + map_gens_sample + maps_with_binary, ids=idfn) def test_posexplode_outer_array_data(data_gen): data_gen = [int_gen, ArrayGen(data_gen)] @@ -184,7 +188,7 @@ def test_posexplode_outer_array_data(data_gen): #sort locally because of https://github.com/NVIDIA/spark-rapids/issues/84 # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) -@pytest.mark.parametrize('map_gen', map_gens_sample + decimal_128_map_gens, ids=idfn) +@pytest.mark.parametrize('map_gen', map_gens_sample + decimal_128_map_gens + maps_with_binary, ids=idfn) def test_posexplode_outer_map_data(map_gen): data_gen = [int_gen, map_gen] assert_gpu_and_cpu_are_equal_collect( @@ -194,7 +198,7 @@ def test_posexplode_outer_map_data(map_gen): #sort locally because of https://github.com/NVIDIA/spark-rapids/issues/84 # After 3.1.0 is the min spark version we can drop this @ignore_order(local=True) -@pytest.mark.parametrize('data_gen', all_gen, ids=idfn) +@pytest.mark.parametrize('data_gen', explode_gens, ids=idfn) def test_posexplode_nested_outer_array_data(data_gen): data_gen = [int_gen, ArrayGen(ArrayGen(data_gen))] assert_gpu_and_cpu_are_equal_collect( diff --git a/integration_tests/src/main/python/join_test.py b/integration_tests/src/main/python/join_test.py index 87df5657391..50c1b1ca70e 100644 --- a/integration_tests/src/main/python/join_test.py +++ b/integration_tests/src/main/python/join_test.py @@ -57,7 +57,7 @@ nested_3d_struct_gens = StructGen([['child0', nested_2d_struct_gens]], nullable=False) struct_gens = [basic_struct_gen, basic_struct_gen_with_no_null_child, nested_2d_struct_gens, nested_3d_struct_gens] -basic_nested_gens = single_level_array_gens + map_string_string_gen + [all_basic_struct_gen] +basic_nested_gens = single_level_array_gens + map_string_string_gen + [all_basic_struct_gen, binary_gen] # data types supported by AST expressions in joins join_ast_gen = [ @@ -194,7 +194,7 @@ def do_join(spark): @allow_non_gpu('SortMergeJoinExec', 'SortExec', 'KnownFloatingPointNormalized', 'ArrayTransform', 'LambdaFunction', 'NamedLambdaVariable', 'NormalizeNaNAndZero', 'ShuffleExchangeExec', 'HashPartitioning') @ignore_order(local=True) -@pytest.mark.parametrize('data_gen', single_level_array_gens, ids=idfn) +@pytest.mark.parametrize('data_gen', single_level_array_gens + [binary_gen], ids=idfn) @pytest.mark.parametrize('join_type', all_join_types, ids=idfn) def test_sortmerge_join_wrong_key_fallback(data_gen, join_type): def do_join(spark): @@ -437,7 +437,7 @@ def do_join(spark): return broadcast(left).join(right, how=join_type).distinct() assert_gpu_and_cpu_are_equal_collect(do_join) -@pytest.mark.parametrize('data_gen', all_gen + single_level_array_gens, ids=idfn) +@pytest.mark.parametrize('data_gen', all_gen + single_level_array_gens + [binary_gen], ids=idfn) @pytest.mark.parametrize('join_type', ['Left', 'LeftSemi', 'LeftAnti'], ids=idfn) def test_right_broadcast_nested_loop_join_condition_missing_count(data_gen, join_type): def do_join(spark): @@ -445,7 +445,7 @@ def do_join(spark): return left.join(broadcast(right), how=join_type).selectExpr('COUNT(*)') assert_gpu_and_cpu_are_equal_collect(do_join) -@pytest.mark.parametrize('data_gen', all_gen + single_level_array_gens, ids=idfn) +@pytest.mark.parametrize('data_gen', all_gen + single_level_array_gens + [binary_gen], ids=idfn) @pytest.mark.parametrize('join_type', ['Right'], ids=idfn) def test_left_broadcast_nested_loop_join_condition_missing_count(data_gen, join_type): def do_join(spark): diff --git a/integration_tests/src/main/python/map_test.py b/integration_tests/src/main/python/map_test.py index e3cc641beb8..36c5a7a6488 100644 --- a/integration_tests/src/main/python/map_test.py +++ b/integration_tests/src/main/python/map_test.py @@ -29,8 +29,9 @@ BooleanGen(), DateGen(), TimestampGen(), null_gen] + decimal_gens)], nullable=False) +maps_with_binary = [MapGen(IntegerGen(nullable=False), BinaryGen(max_length=5))] -@pytest.mark.parametrize('data_gen', map_gens_sample + decimal_64_map_gens + decimal_128_map_gens, ids=idfn) +@pytest.mark.parametrize('data_gen', map_gens_sample + maps_with_binary + decimal_64_map_gens + decimal_128_map_gens, ids=idfn) def test_map_keys(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr( @@ -41,7 +42,7 @@ def test_map_keys(data_gen): 'map_keys(a)')) -@pytest.mark.parametrize('data_gen', map_gens_sample + decimal_64_map_gens + decimal_128_map_gens, ids=idfn) +@pytest.mark.parametrize('data_gen', map_gens_sample + maps_with_binary + decimal_64_map_gens + decimal_128_map_gens, ids=idfn) def test_map_values(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr( @@ -52,7 +53,7 @@ def test_map_values(data_gen): 'map_values(a)')) -@pytest.mark.parametrize('data_gen', map_gens_sample + decimal_64_map_gens + decimal_128_map_gens, ids=idfn) +@pytest.mark.parametrize('data_gen', map_gens_sample + maps_with_binary + decimal_64_map_gens + decimal_128_map_gens, ids=idfn) def test_map_entries(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr( @@ -80,7 +81,7 @@ def decimal_value_gen(): return DecimalGen(precision, scale) return [ByteGen, ShortGen, IntegerGen, LongGen, FloatGen, DoubleGen, - StringGen, DateGen, TimestampGen, decimal_value_gen, + StringGen, DateGen, TimestampGen, decimal_value_gen, BinaryGen, simple_struct_value_gen, nested_struct_value_gen, nested_map_value_gen, array_value_gen] diff --git a/integration_tests/src/main/python/repart_test.py b/integration_tests/src/main/python/repart_test.py index 9910d40d328..7b77b7be426 100644 --- a/integration_tests/src/main/python/repart_test.py +++ b/integration_tests/src/main/python/repart_test.py @@ -160,9 +160,9 @@ def test_union_by_name(data_gen): @pytest.mark.parametrize('data_gen', [ - pytest.param([('basic' + str(i), gen) for i, gen in enumerate(all_basic_gens + decimal_gens)]), + pytest.param([('basic' + str(i), gen) for i, gen in enumerate(all_basic_gens + decimal_gens + [binary_gen])]), pytest.param([('struct' + str(i), gen) for i, gen in enumerate(struct_gens_sample)]), - pytest.param([('array' + str(i), gen) for i, gen in enumerate(array_gens_sample)]), + pytest.param([('array' + str(i), gen) for i, gen in enumerate(array_gens_sample + [ArrayGen(BinaryGen(max_length=5), max_length=5)])]), pytest.param([('map' + str(i), gen) for i, gen in enumerate(map_gens_sample)]), ], ids=idfn) def test_coalesce_types(data_gen): @@ -173,12 +173,12 @@ def test_coalesce_types(data_gen): @pytest.mark.parametrize('length', [0, 2048, 4096], ids=idfn) def test_coalesce_df(num_parts, length): #This should change eventually to be more than just the basic gens - gen_list = [('_c' + str(i), gen) for i, gen in enumerate(all_basic_gens + decimal_gens)] + gen_list = [('_c' + str(i), gen) for i, gen in enumerate(all_basic_gens + decimal_gens + [binary_gen])] assert_gpu_and_cpu_are_equal_collect( lambda spark : gen_df(spark, gen_list, length=length).coalesce(num_parts)) @pytest.mark.parametrize('data_gen', [ - pytest.param([('_c' + str(i), gen) for i, gen in enumerate(all_basic_gens + decimal_gens)]), + pytest.param([('_c' + str(i), gen) for i, gen in enumerate(all_basic_gens + decimal_gens + [binary_gen])]), pytest.param([('s', StructGen([['child0', all_basic_struct_gen]]))]), pytest.param([('a', ArrayGen(string_gen))]), pytest.param([('m', simple_string_to_string_map_gen)]), diff --git a/integration_tests/src/main/python/struct_test.py b/integration_tests/src/main/python/struct_test.py index b6560cab7c4..0e230a95408 100644 --- a/integration_tests/src/main/python/struct_test.py +++ b/integration_tests/src/main/python/struct_test.py @@ -31,7 +31,8 @@ def test_struct_scalar_project(): StructGen([["first", double_gen], ["second", date_gen], ["third", timestamp_gen]]), StructGen([["first", string_gen], ["second", ArrayGen(byte_gen)], ["third", simple_string_to_string_map_gen]]), StructGen([["first", decimal_gen_64bit], ["second", decimal_gen_32bit], ["third", decimal_gen_32bit]]), - StructGen([["first", decimal_gen_128bit], ["second", decimal_gen_128bit], ["third", decimal_gen_128bit]])], ids=idfn) + StructGen([["first", decimal_gen_128bit], ["second", decimal_gen_128bit], ["third", decimal_gen_128bit]]), + StructGen([["first", binary_gen], ["second", ArrayGen(BinaryGen(max_length=10), max_length=10)], ["third", binary_gen]])], ids=idfn) def test_struct_get_item(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr( @@ -40,7 +41,7 @@ def test_struct_get_item(data_gen): 'a.third')) -@pytest.mark.parametrize('data_gen', all_basic_gens + decimal_gens + [ +@pytest.mark.parametrize('data_gen', all_basic_gens + decimal_gens + [binary_gen, null_gen] + single_level_array_gens + struct_gens_sample + map_gens_sample, ids=idfn) def test_make_struct(data_gen): # Spark has no good way to create a map literal without the map function diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 9664d4f5d5e..65a67f1035e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -864,9 +864,9 @@ object GpuOverrides extends Logging { ExprChecks.projectAndAst( TypeSig.astTypes, (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.CALENDAR - + TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT) + + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT) .nested(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT), + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT), TypeSig.all), (lit, conf, p, r) => new LiteralExprMeta(lit, conf, p, r)), expr[Signum]( @@ -880,7 +880,8 @@ object GpuOverrides extends Logging { ExprChecks.unaryProjectAndAstInputMatchesOutput( TypeSig.astTypes + GpuTypeShims.additionalCommonOperatorSupportedTypes, (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.MAP + TypeSig.ARRAY + TypeSig.STRUCT - + TypeSig.DECIMAL_128 + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), + + TypeSig.DECIMAL_128 + TypeSig.BINARY + + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), TypeSig.all), (a, conf, p, r) => new UnaryAstExprMeta[Alias](a, conf, p, r) { override def convertToGpu(child: Expression): GpuExpression = @@ -1386,7 +1387,7 @@ object GpuOverrides extends Logging { "Checks if a value is null", ExprChecks.unaryProject(TypeSig.BOOLEAN, TypeSig.BOOLEAN, (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.MAP + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.DECIMAL_128 + + TypeSig.STRUCT + TypeSig.DECIMAL_128 + TypeSig.BINARY + GpuTypeShims.additionalPredicateSupportedTypes).nested(), TypeSig.all), (a, conf, p, r) => new UnaryExprMeta[IsNull](a, conf, p, r) { @@ -1396,7 +1397,7 @@ object GpuOverrides extends Logging { "Checks if a value is not null", ExprChecks.unaryProject(TypeSig.BOOLEAN, TypeSig.BOOLEAN, (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.MAP + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.DECIMAL_128+ + TypeSig.STRUCT + TypeSig.DECIMAL_128 + TypeSig.BINARY + GpuTypeShims.additionalPredicateSupportedTypes).nested(), TypeSig.all), (a, conf, p, r) => new UnaryExprMeta[IsNotNull](a, conf, p, r) { @@ -1426,8 +1427,8 @@ object GpuOverrides extends Logging { "Checks if number of non null/Nan values is greater than a given value", ExprChecks.projectOnly(TypeSig.BOOLEAN, TypeSig.BOOLEAN, repeatingParamCheck = Some(RepeatingParamCheck("input", - (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP + - TypeSig.ARRAY + TypeSig.STRUCT).nested(), + (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.BINARY + + TypeSig.MAP + TypeSig.ARRAY + TypeSig.STRUCT).nested(), TypeSig.all))), (a, conf, p, r) => new ExprMeta[AtLeastNNonNulls](a, conf, p, r) { def convertToGpu(): GpuExpression = @@ -1524,11 +1525,11 @@ object GpuOverrides extends Logging { expr[Coalesce] ( "Returns the first non-null argument if exists. Otherwise, null", ExprChecks.projectOnly( - (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + + (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.BINARY + GpuTypeShims.additionalArithmeticSupportedTypes).nested(), TypeSig.all, repeatingParamCheck = Some(RepeatingParamCheck("param", - (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT+ + (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.BINARY + GpuTypeShims.additionalArithmeticSupportedTypes).nested(), TypeSig.all))), (a, conf, p, r) => new ExprMeta[Coalesce](a, conf, p, r) { @@ -2097,17 +2098,17 @@ object GpuOverrides extends Logging { expr[If]( "IF expression", ExprChecks.projectOnly( - (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + - TypeSig.MAP + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), + (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + + TypeSig.BINARY + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), TypeSig.all, Seq(ParamCheck("predicate", TypeSig.BOOLEAN, TypeSig.BOOLEAN), ParamCheck("trueValue", - (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + - TypeSig.MAP + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), + (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + + TypeSig.BINARY + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), TypeSig.all), ParamCheck("falseValue", - (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + - TypeSig.MAP + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), + (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + + TypeSig.BINARY + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), TypeSig.all))), (a, conf, p, r) => new ExprMeta[If](a, conf, p, r) { override def convertToGpu(): GpuExpression = { @@ -2590,10 +2591,10 @@ object GpuOverrides extends Logging { "Gets the named field of the struct", ExprChecks.unaryProject( (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.NULL + - TypeSig.DECIMAL_128).nested(), + TypeSig.DECIMAL_128 + TypeSig.BINARY).nested(), TypeSig.all, TypeSig.STRUCT.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.MAP + TypeSig.NULL + TypeSig.DECIMAL_128), + TypeSig.STRUCT + TypeSig.MAP + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.BINARY), TypeSig.STRUCT.nested(TypeSig.all)), (expr, conf, p, r) => new UnaryExprMeta[GetStructField](expr, conf, p, r) { override def convertToGpu(arr: Expression): GpuExpression = @@ -2603,10 +2604,10 @@ object GpuOverrides extends Logging { "Gets the field at `ordinal` in the Array", ExprChecks.binaryProject( (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL + - TypeSig.DECIMAL_128 + TypeSig.MAP).nested(), + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY).nested(), TypeSig.all, ("array", TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP), + TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY), TypeSig.ARRAY.nested(TypeSig.all)), ("ordinal", TypeSig.INT, TypeSig.INT)), (in, conf, p, r) => new BinaryExprMeta[GetArrayItem](in, conf, p, r) { @@ -2617,10 +2618,11 @@ object GpuOverrides extends Logging { "Gets Value from a Map based on a key", ExprChecks.binaryProject( (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL + - TypeSig.DECIMAL_128 + TypeSig.MAP).nested(), + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY).nested(), TypeSig.all, ("map", TypeSig.MAP.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + - TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP), TypeSig.MAP.nested(TypeSig.all)), + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY), + TypeSig.MAP.nested(TypeSig.all)), ("key", TypeSig.commonCudfTypes + TypeSig.DECIMAL_128, TypeSig.all)), (in, conf, p, r) => new BinaryExprMeta[GetMapValue](in, conf, p, r) { override def tagExprForGpu(): Unit = { @@ -2637,11 +2639,11 @@ object GpuOverrides extends Logging { "Returns value for the given key in value if column is map.", ExprChecks.binaryProject( (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL + - TypeSig.DECIMAL_128 + TypeSig.MAP).nested(), TypeSig.all, + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY).nested(), TypeSig.all, ("array/map", TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP) + + TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY) + TypeSig.MAP.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + - TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP) + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY) .withPsNote(TypeEnum.MAP ,"If it's map, only primitive key types are supported."), TypeSig.ARRAY.nested(TypeSig.all) + TypeSig.MAP.nested(TypeSig.all)), ("index/key", (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128) @@ -2658,21 +2660,22 @@ object GpuOverrides extends Logging { // Match exactly with the checks for GetMapValue ExprChecks.binaryProject( (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL + - TypeSig.DECIMAL_128 + TypeSig.MAP).nested(), + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY).nested(), TypeSig.all, ("map", TypeSig.MAP.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + - TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP), + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY), TypeSig.MAP.nested(TypeSig.all)), ("key", TypeSig.commonCudfTypes + TypeSig.DECIMAL_128, TypeSig.all)) case _: ArrayType => // Match exactly with the checks for GetArrayItem ExprChecks.binaryProject( (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL + - TypeSig.DECIMAL_128 + TypeSig.MAP).nested(), + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY).nested(), TypeSig.all, ("array", TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP), + TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP + + TypeSig.BINARY), TypeSig.ARRAY.nested(TypeSig.all)), ("ordinal", TypeSig.INT, TypeSig.INT)) case _ => throw new IllegalStateException("Only Array or Map is supported as input.") @@ -2687,10 +2690,10 @@ object GpuOverrides extends Logging { "Returns an unordered array containing the keys of the map", ExprChecks.unaryProject( TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + - TypeSig.ARRAY + TypeSig.STRUCT).nested(), + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.BINARY).nested(), TypeSig.ARRAY.nested(TypeSig.all - TypeSig.MAP), // Maps cannot have other maps as keys TypeSig.MAP.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY), TypeSig.MAP.nested(TypeSig.all)), (in, conf, p, r) => new UnaryExprMeta[MapKeys](in, conf, p, r) { override def convertToGpu(child: Expression): GpuExpression = @@ -2700,10 +2703,10 @@ object GpuOverrides extends Logging { "Returns an unordered array containing the values of the map", ExprChecks.unaryProject( TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY), TypeSig.ARRAY.nested(TypeSig.all), TypeSig.MAP.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY), TypeSig.MAP.nested(TypeSig.all)), (in, conf, p, r) => new UnaryExprMeta[MapValues](in, conf, p, r) { override def convertToGpu(child: Expression): GpuExpression = @@ -2714,10 +2717,10 @@ object GpuOverrides extends Logging { ExprChecks.unaryProject( // Technically the return type is an array of struct, but we cannot really express that TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY), TypeSig.ARRAY.nested(TypeSig.all), TypeSig.MAP.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY), TypeSig.MAP.nested(TypeSig.all)), (in, conf, p, r) => new UnaryExprMeta[MapEntries](in, conf, p, r) { override def convertToGpu(child: Expression): GpuExpression = @@ -2822,14 +2825,14 @@ object GpuOverrides extends Logging { ExprChecks.projectOnly( TypeSig.ARRAY.nested(TypeSig.gpuNumeric + TypeSig.NULL + TypeSig.STRING + TypeSig.BOOLEAN + TypeSig.DATE + TypeSig.TIMESTAMP + - TypeSig.ARRAY + TypeSig.STRUCT), + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.BINARY), TypeSig.ARRAY.nested(TypeSig.all), repeatingParamCheck = Some(RepeatingParamCheck("arg", TypeSig.gpuNumeric + TypeSig.NULL + TypeSig.STRING + - TypeSig.BOOLEAN + TypeSig.DATE + TypeSig.TIMESTAMP + TypeSig.STRUCT + + TypeSig.BOOLEAN + TypeSig.DATE + TypeSig.TIMESTAMP + TypeSig.STRUCT + TypeSig.BINARY + TypeSig.ARRAY.nested(TypeSig.gpuNumeric + TypeSig.NULL + TypeSig.STRING + TypeSig.BOOLEAN + TypeSig.DATE + TypeSig.TIMESTAMP + TypeSig.STRUCT + - TypeSig.ARRAY), + TypeSig.ARRAY + TypeSig.BINARY), TypeSig.all))), (in, conf, p, r) => new ExprMeta[CreateArray](in, conf, p, r) { @@ -2922,12 +2925,12 @@ object GpuOverrides extends Logging { "Returns a merged array of structs in which the N-th struct contains" + " all N-th values of input arrays.", ExprChecks.projectOnly(TypeSig.ARRAY.nested( - TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + + TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), TypeSig.ARRAY.nested(TypeSig.all), repeatingParamCheck = Some(RepeatingParamCheck("children", TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), TypeSig.ARRAY.nested(TypeSig.all)))), (in, conf, p, r) => new ExprMeta[ArraysZip](in, conf, p, r) { override def convertToGpu(): GpuExpression = { @@ -3200,12 +3203,12 @@ object GpuOverrides extends Logging { "List/String concatenate", ExprChecks.projectOnly((TypeSig.STRING + TypeSig.ARRAY).nested( TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY), (TypeSig.STRING + TypeSig.BINARY + TypeSig.ARRAY).nested(TypeSig.all), repeatingParamCheck = Some(RepeatingParamCheck("input", (TypeSig.STRING + TypeSig.ARRAY).nested( TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY), (TypeSig.STRING + TypeSig.BINARY + TypeSig.ARRAY).nested(TypeSig.all)))), (a, conf, p, r) => new ComplexTypeMergingExprMeta[Concat](a, conf, p, r) { override def convertToGpu(child: Seq[Expression]): GpuExpression = GpuConcat(child) @@ -3325,7 +3328,7 @@ object GpuOverrides extends Logging { "The size of an array or a map", ExprChecks.unaryProject(TypeSig.INT, TypeSig.INT, (TypeSig.ARRAY + TypeSig.MAP).nested(TypeSig.commonCudfTypes + TypeSig.NULL - + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + + TypeSig.DECIMAL_128 + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), (TypeSig.ARRAY + TypeSig.MAP).nested(TypeSig.all)), (a, conf, p, r) => new UnaryExprMeta[Size](a, conf, p, r) { override def convertToGpu(child: Expression): GpuExpression = @@ -3353,10 +3356,10 @@ object GpuOverrides extends Logging { // Here is a walk-around representation, since multi-level nested type is not supported yet. // related issue: https://github.com/NVIDIA/spark-rapids/issues/1901 TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), TypeSig.ARRAY.nested(TypeSig.all), (TypeSig.ARRAY + TypeSig.MAP).nested(TypeSig.commonCudfTypes + TypeSig.NULL + - TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.DECIMAL_128 + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), (TypeSig.ARRAY + TypeSig.MAP).nested(TypeSig.all)), (a, conf, p, r) => new GeneratorExprMeta[Explode](a, conf, p, r) { override val supportOuter: Boolean = true @@ -3368,10 +3371,10 @@ object GpuOverrides extends Logging { // Here is a walk-around representation, since multi-level nested type is not supported yet. // related issue: https://github.com/NVIDIA/spark-rapids/issues/1901 TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), TypeSig.ARRAY.nested(TypeSig.all), (TypeSig.ARRAY + TypeSig.MAP).nested(TypeSig.commonCudfTypes + TypeSig.NULL + - TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.DECIMAL_128 + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), (TypeSig.ARRAY + TypeSig.MAP).nested(TypeSig.all)), (a, conf, p, r) => new GeneratorExprMeta[PosExplode](a, conf, p, r) { override val supportOuter: Boolean = true @@ -3808,15 +3811,15 @@ object GpuOverrides extends Logging { exec[GenerateExec] ( "The backend for operations that generate more output rows than input rows like explode", ExecChecks( - (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.MAP).nested(), + (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.BINARY + + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP).nested(), TypeSig.all), (gen, conf, p, r) => new GpuGenerateExecSparkPlanMeta(gen, conf, p, r)), exec[ProjectExec]( "The backend for most select, withColumn and dropColumn statements", ExecChecks( (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.STRUCT + TypeSig.MAP + - TypeSig.ARRAY + TypeSig.DECIMAL_128 + + TypeSig.ARRAY + TypeSig.DECIMAL_128 + TypeSig.BINARY + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), TypeSig.all), (proj, conf, p, r) => new GpuProjectExecMeta(proj, conf, p, r)), @@ -3846,7 +3849,7 @@ object GpuOverrides extends Logging { exec[CoalesceExec]( "The backend for the dataframe coalesce method", ExecChecks((_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.STRUCT + TypeSig.ARRAY + - TypeSig.MAP + GpuTypeShims.additionalArithmeticSupportedTypes).nested(), + TypeSig.MAP + TypeSig.BINARY + GpuTypeShims.additionalArithmeticSupportedTypes).nested(), TypeSig.all), (coalesce, conf, parent, r) => new SparkPlanMeta[CoalesceExec](coalesce, conf, parent, r) { override def convertToGpu(): GpuExec = @@ -3941,7 +3944,7 @@ object GpuOverrides extends Logging { exec[FilterExec]( "The backend for most filter statements", ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.STRUCT + TypeSig.MAP + - TypeSig.ARRAY + TypeSig.DECIMAL_128 + + TypeSig.ARRAY + TypeSig.DECIMAL_128 + TypeSig.BINARY + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), TypeSig.all), (filter, conf, p, r) => new SparkPlanMeta[FilterExec](filter, conf, p, r) { override def convertToGpu(): GpuExec = @@ -3949,7 +3952,7 @@ object GpuOverrides extends Logging { }), exec[ShuffleExchangeExec]( "The backend for most data being exchanged between processes", - ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + GpuTypeShims.additionalArithmeticSupportedTypes).nested() .withPsNote(TypeEnum.STRUCT, "Round-robin partitioning is not supported for nested " + @@ -3973,7 +3976,7 @@ object GpuOverrides extends Logging { }), exec[BroadcastExchangeExec]( "The backend for broadcast exchange of data", - ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP).nested(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.STRUCT), TypeSig.all), @@ -3990,9 +3993,9 @@ object GpuOverrides extends Logging { (join, conf, p, r) => new GpuBroadcastNestedLoopJoinMeta(join, conf, p, r)), exec[CartesianProductExec]( "Implementation of join using brute force", - ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT) - .nested(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + .nested(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT), TypeSig.all), (join, conf, p, r) => new SparkPlanMeta[CartesianProductExec](join, conf, p, r) { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index ac5b2c90a02..c27aec726ae 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -1120,7 +1120,7 @@ case class ExprChecksImpl(contexts: Map[ExpressionContext, ContextChecks]) */ object CaseWhenCheck extends ExprChecks { val check: TypeSig = (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP).nested() + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY).nested() val sparkSig: TypeSig = TypeSig.all @@ -1251,7 +1251,7 @@ object CreateNamedStructCheck extends ExprChecks { val nameSig: TypeSig = TypeSig.lit(TypeEnum.STRING) val sparkNameSig: TypeSig = TypeSig.lit(TypeEnum.STRING) val valueSig: TypeSig = (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT).nested() + TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT + TypeSig.BINARY).nested() val sparkValueSig: TypeSig = TypeSig.all val resultSig: TypeSig = TypeSig.STRUCT.nested(valueSig) val sparkResultSig: TypeSig = TypeSig.STRUCT.nested(sparkValueSig) @@ -1332,7 +1332,7 @@ class CastChecks extends ExprChecks { val sparkStringSig: TypeSig = cpuNumeric + BOOLEAN + TIMESTAMP + DATE + CALENDAR + STRING + BINARY + GpuTypeShims.additionalTypesStringCanCastTo - val binaryChecks: TypeSig = none + val binaryChecks: TypeSig = BINARY val sparkBinarySig: TypeSig = STRING + BINARY val decimalChecks: TypeSig = gpuNumeric + STRING diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/literals.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/literals.scala index 7b68b2cb50b..42b0eacd967 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/literals.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/literals.scala @@ -143,6 +143,8 @@ object GpuScalar extends Arm with Logging { k.asInstanceOf[Object], v.asInstanceOf[Object]) }.asJava + case BinaryType => + element.asInstanceOf[Array[Byte]].toList.asJava case _ => element } @@ -205,6 +207,10 @@ object GpuScalar extends Arm with Logging { val colType = resolveElementType(elementType) val rows = seq.map(convertElementTo(_, elementType)) ColumnVector.fromLists(colType, rows.asInstanceOf[Seq[JList[_]]]: _*) + case BinaryType => + val colType = GpuColumnVector.convertFrom(elementType, true) + val rows = seq.map(convertElementTo(_, elementType)) + ColumnVector.fromLists(colType, rows.asInstanceOf[Seq[JList[_]]]: _*) case NullType => GpuColumnVector.columnVectorFromNull(seq.size, NullType) case u => @@ -235,6 +241,8 @@ object GpuScalar extends Arm with Logging { Scalar.listFromNull( resolveElementType(StructType( Seq(StructField("key", keyType), StructField("value", valueType))))) + case BinaryType => + Scalar.listFromNull(GpuColumnVector.convertFrom(ByteType, false)) case _ => Scalar.fromNull(GpuColumnVector.getNonNestedRapidsType(nullType)) } case decType: DecimalType => @@ -357,6 +365,14 @@ object GpuScalar extends Arm with Logging { case _ => throw new IllegalArgumentException(s"'$v: ${v.getClass}' is not supported" + s" for MapType, expecting MapData") } + case BinaryType => v match { + case data: Array[Byte] => + withResource(columnVectorFromLiterals(data, ByteType)) { list => + Scalar.listFromColumnView(list) + } + case _ => throw new IllegalArgumentException(s"'$v: ${v.getClass}' is not supported" + + s" for BinaryType, expecting Array[Byte]") + } case GpuUnsignedIntegerType => v match { case i: Int => Scalar.fromUnsignedInt(i) case s: Short => Scalar.fromUnsignedInt(s.toInt) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala index 2d29c7517a0..35cf7f28f42 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala @@ -66,7 +66,8 @@ object JoinTypeChecks { private[this] val sparkSupportedJoinKeyTypes = TypeSig.all - TypeSig.MAP.nested() private[this] val joinRideAlongTypes = - (cudfSupportedKeyTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.MAP).nested() + (cudfSupportedKeyTypes + TypeSig.DECIMAL_128 + TypeSig.BINARY + + TypeSig.ARRAY + TypeSig.MAP).nested() val equiJoinExecChecks: ExecChecks = ExecChecks( joinRideAlongTypes, diff --git a/tools/src/main/resources/supportedExecs.csv b/tools/src/main/resources/supportedExecs.csv index f9fa01672f4..99efa00b74a 100644 --- a/tools/src/main/resources/supportedExecs.csv +++ b/tools/src/main/resources/supportedExecs.csv @@ -1,13 +1,13 @@ Exec,Supported,Notes,Params,BOOLEAN,BYTE,SHORT,INT,LONG,FLOAT,DOUBLE,DATE,TIMESTAMP,STRING,DECIMAL,NULL,BINARY,CALENDAR,ARRAY,MAP,STRUCT,UDT -CoalesceExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +CoalesceExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS CollectLimitExec,NS,This is disabled by default because Collect Limit replacement can be slower on the GPU; if huge number of rows in a batch it could help by limiting the number of rows transferred from GPU to CPU,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS ExpandExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS FileSourceScanExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS -FilterExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS -GenerateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +FilterExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS +GenerateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS GlobalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS -ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS @@ -21,23 +21,23 @@ SortAggregateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,PS,NS,PS,PS,PS,NS InMemoryTableScanExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,PS,PS,PS,NS DataWritingCommandExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,PS,NS,NS,NS,PS,PS,PS,NS BatchScanExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,S,NS,PS,PS,PS,NS -BroadcastExchangeExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS -ShuffleExchangeExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +BroadcastExchangeExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS +ShuffleExchangeExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS BroadcastHashJoinExec,S,None,leftKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS BroadcastHashJoinExec,S,None,rightKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS BroadcastHashJoinExec,S,None,condition,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -BroadcastHashJoinExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +BroadcastHashJoinExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS BroadcastNestedLoopJoinExec,S,None,condition(A non-inner join only is supported if the condition expression can be converted to a GPU AST expression),S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -BroadcastNestedLoopJoinExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS -CartesianProductExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +BroadcastNestedLoopJoinExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS +CartesianProductExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS ShuffledHashJoinExec,S,None,leftKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS ShuffledHashJoinExec,S,None,rightKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS ShuffledHashJoinExec,S,None,condition,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -ShuffledHashJoinExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +ShuffledHashJoinExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS SortMergeJoinExec,S,None,leftKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS SortMergeJoinExec,S,None,rightKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS SortMergeJoinExec,S,None,condition,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -SortMergeJoinExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +SortMergeJoinExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS AggregateInPandasExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS ArrowEvalPythonExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS FlatMapGroupsInPandasExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS diff --git a/tools/src/main/resources/supportedExprs.csv b/tools/src/main/resources/supportedExprs.csv index 2e3ff5ae942..50e7a4f8ebc 100644 --- a/tools/src/main/resources/supportedExprs.csv +++ b/tools/src/main/resources/supportedExprs.csv @@ -17,8 +17,8 @@ Add,S,`+`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA Add,S,`+`,None,AST,lhs,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA Add,S,`+`,None,AST,rhs,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA Add,S,`+`,None,AST,result,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA -Alias,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS -Alias,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +Alias,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS +Alias,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS Alias,S, ,None,AST,input,S,S,S,S,S,S,S,S,PS,NS,NS,NS,NS,NS,NS,NS,NS,NS Alias,S, ,None,AST,result,S,S,S,S,S,S,S,S,PS,NS,NS,NS,NS,NS,NS,NS,NS,NS And,S,`and`,None,project,lhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA @@ -65,7 +65,7 @@ Asinh,S,`asinh`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,N Asinh,S,`asinh`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Asinh,S,`asinh`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Asinh,S,`asinh`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -AtLeastNNonNulls,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +AtLeastNNonNulls,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS AtLeastNNonNulls,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Atan,S,`atan`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Atan,S,`atan`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA @@ -105,8 +105,8 @@ BitwiseXor,S,`^`,None,AST,lhs,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N BitwiseXor,S,`^`,None,AST,rhs,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA BitwiseXor,S,`^`,None,AST,result,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA CaseWhen,S,`when`,None,project,predicate,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -CaseWhen,S,`when`,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS -CaseWhen,S,`when`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +CaseWhen,S,`when`,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS +CaseWhen,S,`when`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS Cbrt,S,`cbrt`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Cbrt,S,`cbrt`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Cbrt,S,`cbrt`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA @@ -115,8 +115,8 @@ Ceil,S,`ceiling`; `ceil`,None,project,input,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA, Ceil,S,`ceiling`; `ceil`,None,project,result,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA CheckOverflow,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA CheckOverflow,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA -Coalesce,S,`coalesce`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS -Coalesce,S,`coalesce`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS +Coalesce,S,`coalesce`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,NS,PS,NS +Coalesce,S,`coalesce`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,NS,PS,NS Concat,S,`concat`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NS,NA,PS,NA,NA,NA Concat,S,`concat`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NS,NA,PS,NA,NA,NA ConcatWs,S,`concat_ws`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,S,NA,NA,NA @@ -136,12 +136,12 @@ Cot,S,`cot`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Cot,S,`cot`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Cot,S,`cot`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Cot,S,`cot`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -CreateArray,S,`array`,None,project,arg,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS +CreateArray,S,`array`,None,project,arg,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,NS,PS,NS CreateArray,S,`array`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA CreateMap,S,`map`,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,S,NA,NA,PS,NA,PS,NA CreateMap,S,`map`,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NA,NA,PS,PS,PS,NA CreateNamedStruct,S,`named_struct`; `struct`,None,project,name,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA -CreateNamedStruct,S,`named_struct`; `struct`,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +CreateNamedStruct,S,`named_struct`; `struct`,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS CreateNamedStruct,S,`named_struct`; `struct`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA CurrentRow$,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA DateAdd,S,`date_add`,None,project,startDate,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA @@ -172,7 +172,7 @@ Divide,S,`/`,None,project,rhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,N Divide,S,`/`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA ElementAt,S,`element_at`,None,project,array/map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,PS,NA,NA ElementAt,S,`element_at`,None,project,index/key,PS,PS,PS,S,PS,PS,PS,PS,PS,PS,PS,NS,NS,NS,NS,NS,NS,NS -ElementAt,S,`element_at`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +ElementAt,S,`element_at`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS EndsWith,S, ,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA EndsWith,S, ,None,project,search,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA EndsWith,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA @@ -202,7 +202,7 @@ FromUnixTime,S,`from_unixtime`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS FromUnixTime,S,`from_unixtime`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA GetArrayItem,S, ,None,project,array,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA GetArrayItem,S, ,None,project,ordinal,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA @@ -210,9 +210,9 @@ GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,P GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS -GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS GetStructField,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA -GetStructField,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +GetStructField,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS GetTimestamp,S, ,None,project,timeExp,NA,NA,NA,NA,NA,NA,NA,S,PS,S,NA,NA,NA,NA,NA,NA,NA,NA GetTimestamp,S, ,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA GetTimestamp,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA @@ -236,9 +236,9 @@ Hypot,S,`hypot`,None,project,lhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA, Hypot,S,`hypot`,None,project,rhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hypot,S,`hypot`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA If,S,`if`,None,project,predicate,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -If,S,`if`,None,project,trueValue,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS -If,S,`if`,None,project,falseValue,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS -If,S,`if`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +If,S,`if`,None,project,trueValue,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS +If,S,`if`,None,project,falseValue,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS +If,S,`if`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS In,S,`in`,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS In,S,`in`,None,project,list,PS,PS,PS,PS,PS,PS,PS,PS,PS,PS,PS,NS,NS,NS,NS,NA,NS,NS In,S,`in`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA @@ -254,9 +254,9 @@ IntegralDivide,S,`div`,None,project,rhs,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,S,NA,NA,NA, IntegralDivide,S,`div`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA IsNaN,S,`isnan`,None,project,input,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA IsNaN,S,`isnan`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA JsonToStructs,NS,`from_json`,This is disabled by default because parsing JSON from a column has a large number of issues and should be considered beta quality right now.,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA JsonToStructs,NS,`from_json`,This is disabled by default because parsing JSON from a column has a large number of issues and should be considered beta quality right now.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,NS,NA @@ -296,7 +296,7 @@ LessThanOrEqual,S,`<=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA, Like,S,`like`,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA Like,S,`like`,None,project,search,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA Like,S,`like`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -Literal,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,S,PS,PS,PS,NS +Literal,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,NS Literal,S, ,None,AST,result,S,S,S,S,S,S,S,S,PS,NS,NS,NS,NS,NS,NS,NS,NS,NS Log,S,`ln`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Log,S,`ln`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA From 9cb38c7a5f4aee402623f255366bb21c04ec31b7 Mon Sep 17 00:00:00 2001 From: Ryan Lee Date: Fri, 12 Aug 2022 11:04:06 -0700 Subject: [PATCH 019/190] Nested struct binary comparison operator support (#6062) * struct binop support * switch to integration tests * avoid DayTimeIntervalGen errors with < 330 Signed-off-by: Ryan Lee Co-authored-by: Ryan Lee --- docs/supported_ops.md | 20 +-- integration_tests/src/main/python/cmp_test.py | 167 +++++++++++------- integration_tests/src/main/python/data_gen.py | 5 + .../nvidia/spark/rapids/GpuOverrides.scala | 40 ++--- tools/src/main/resources/supportedExprs.csv | 20 +-- 5 files changed, 147 insertions(+), 105 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 603fd74a128..44aaa156fb3 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -5704,7 +5704,7 @@ are limited. NS NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, ARRAY, UDT
NS @@ -5725,7 +5725,7 @@ are limited. NS NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, ARRAY, UDT
NS @@ -6596,7 +6596,7 @@ are limited. NS NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, ARRAY, UDT
NS @@ -6617,7 +6617,7 @@ are limited. NS NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, ARRAY, UDT
NS @@ -6728,7 +6728,7 @@ are limited. NS NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, ARRAY, UDT
NS @@ -6749,7 +6749,7 @@ are limited. NS NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, ARRAY, UDT
NS @@ -8192,7 +8192,7 @@ are limited. NS NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, ARRAY, UDT
NS @@ -8213,7 +8213,7 @@ are limited. NS NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, ARRAY, UDT
NS @@ -8324,7 +8324,7 @@ are limited. NS NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, ARRAY, UDT
NS @@ -8345,7 +8345,7 @@ are limited. NS NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, ARRAY, UDT
NS diff --git a/integration_tests/src/main/python/cmp_test.py b/integration_tests/src/main/python/cmp_test.py index 2d414bbdd5b..4d6146beb68 100644 --- a/integration_tests/src/main/python/cmp_test.py +++ b/integration_tests/src/main/python/cmp_test.py @@ -20,7 +20,7 @@ from pyspark.sql.types import * import pyspark.sql.functions as f -@pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen, ids=idfn) +@pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + struct_gens_sample_with_decimal128_no_list, ids=idfn) def test_eq(data_gen): (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) data_type = data_gen.data_type @@ -34,16 +34,23 @@ def test_eq(data_gen): @pytest.mark.skipif(is_before_spark_330(), reason='DayTimeInterval is not supported before Pyspark 3.3.0') def test_eq_for_interval(): - data_gen = DayTimeIntervalGen() - (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) - data_type = data_gen.data_type - assert_gpu_and_cpu_are_equal_collect( - lambda spark : binary_op_df(spark, data_gen).select( - f.col('a') == s1, - s2 == f.col('b'), - f.lit(None).cast(data_type) == f.col('a'), - f.col('b') == f.lit(None).cast(data_type), - f.col('a') == f.col('b'))) + def test_func(data_gen): + (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) + data_type = data_gen.data_type + assert_gpu_and_cpu_are_equal_collect( + lambda spark : binary_op_df(spark, data_gen).select( + f.col('a') == s1, + s2 == f.col('b'), + f.lit(None).cast(data_type) == f.col('a'), + f.col('b') == f.lit(None).cast(data_type), + f.col('a') == f.col('b'))) + # DayTimeIntervalType not supported inside Structs -- issue #6184 + # data_gens = [DayTimeIntervalGen(), + # StructGen([['child0', StructGen([['child2', DayTimeIntervalGen()]])], ['child1', short_gen]])] + data_gens = [DayTimeIntervalGen()] + for data_gen in data_gens: + test_func(data_gen) + @pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen, ids=idfn) def test_eq_ns(data_gen): @@ -70,7 +77,7 @@ def test_eq_ns_for_interval(): f.col('b').eqNullSafe(f.lit(None).cast(data_type)), f.col('a').eqNullSafe(f.col('b')))) -@pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen, ids=idfn) +@pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + struct_gens_sample_with_decimal128_no_list, ids=idfn) def test_ne(data_gen): (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) data_type = data_gen.data_type @@ -84,18 +91,24 @@ def test_ne(data_gen): @pytest.mark.skipif(is_before_spark_330(), reason='DayTimeInterval is not supported before Pyspark 3.3.0') def test_ne_for_interval(): - data_gen = DayTimeIntervalGen() - (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) - data_type = data_gen.data_type - assert_gpu_and_cpu_are_equal_collect( - lambda spark : binary_op_df(spark, data_gen).select( - f.col('a') != s1, - s2 != f.col('b'), - f.lit(None).cast(data_type) != f.col('a'), - f.col('b') != f.lit(None).cast(data_type), - f.col('a') != f.col('b'))) + def test_func(data_gen): + (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) + data_type = data_gen.data_type + assert_gpu_and_cpu_are_equal_collect( + lambda spark : binary_op_df(spark, data_gen).select( + f.col('a') != s1, + s2 != f.col('b'), + f.lit(None).cast(data_type) != f.col('a'), + f.col('b') != f.lit(None).cast(data_type), + f.col('a') != f.col('b'))) + # DayTimeIntervalType not supported inside Structs -- issue #6184 + # data_gens = [DayTimeIntervalGen(), + # StructGen([['child0', StructGen([['child2', DayTimeIntervalGen()]])], ['child1', short_gen]])] + data_gens = [DayTimeIntervalGen()] + for data_gen in data_gens: + test_func(data_gen) -@pytest.mark.parametrize('data_gen', orderable_gens, ids=idfn) +@pytest.mark.parametrize('data_gen', orderable_gens + struct_gens_sample_with_decimal128_no_list, ids=idfn) def test_lt(data_gen): (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) data_type = data_gen.data_type @@ -109,18 +122,24 @@ def test_lt(data_gen): @pytest.mark.skipif(is_before_spark_330(), reason='DayTimeInterval is not supported before Pyspark 3.3.0') def test_lt_for_interval(): - data_gen = DayTimeIntervalGen() - (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) - data_type = data_gen.data_type - assert_gpu_and_cpu_are_equal_collect( - lambda spark : binary_op_df(spark, data_gen).select( - f.col('a') < s1, - s2 < f.col('b'), - f.lit(None).cast(data_type) < f.col('a'), - f.col('b') < f.lit(None).cast(data_type), - f.col('a') < f.col('b'))) + def test_func(data_gen): + (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) + data_type = data_gen.data_type + assert_gpu_and_cpu_are_equal_collect( + lambda spark : binary_op_df(spark, data_gen).select( + f.col('a') < s1, + s2 < f.col('b'), + f.lit(None).cast(data_type) < f.col('a'), + f.col('b') < f.lit(None).cast(data_type), + f.col('a') < f.col('b'))) + # DayTimeIntervalType not supported inside Structs -- issue #6184 + # data_gens = [DayTimeIntervalGen(), + # StructGen([['child0', StructGen([['child2', DayTimeIntervalGen()]])], ['child1', short_gen]])] + data_gens = [DayTimeIntervalGen()] + for data_gen in data_gens: + test_func(data_gen) -@pytest.mark.parametrize('data_gen', orderable_gens, ids=idfn) +@pytest.mark.parametrize('data_gen', orderable_gens + struct_gens_sample_with_decimal128_no_list, ids=idfn) def test_lte(data_gen): (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) data_type = data_gen.data_type @@ -134,16 +153,22 @@ def test_lte(data_gen): @pytest.mark.skipif(is_before_spark_330(), reason='DayTimeInterval is not supported before Pyspark 3.3.0') def test_lte_for_interval(): - data_gen = DayTimeIntervalGen() - (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) - data_type = data_gen.data_type - assert_gpu_and_cpu_are_equal_collect( - lambda spark : binary_op_df(spark, data_gen).select( - f.col('a') <= s1, - s2 <= f.col('b'), - f.lit(None).cast(data_type) <= f.col('a'), - f.col('b') <= f.lit(None).cast(data_type), - f.col('a') <= f.col('b'))) + def test_func(data_gen): + (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) + data_type = data_gen.data_type + assert_gpu_and_cpu_are_equal_collect( + lambda spark : binary_op_df(spark, data_gen).select( + f.col('a') <= s1, + s2 <= f.col('b'), + f.lit(None).cast(data_type) <= f.col('a'), + f.col('b') <= f.lit(None).cast(data_type), + f.col('a') <= f.col('b'))) + # DayTimeIntervalType not supported inside Structs -- issue #6184 + # data_gens = [DayTimeIntervalGen(), + # StructGen([['child0', StructGen([['child2', DayTimeIntervalGen()]])], ['child1', short_gen]])] + data_gens = [DayTimeIntervalGen()] + for data_gen in data_gens: + test_func(data_gen) @pytest.mark.parametrize('data_gen', orderable_gens, ids=idfn) @@ -160,18 +185,24 @@ def test_gt(data_gen): @pytest.mark.skipif(is_before_spark_330(), reason='DayTimeInterval is not supported before Pyspark 3.3.0') def test_gt_interval(): - data_gen = DayTimeIntervalGen() - (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) - data_type = data_gen.data_type - assert_gpu_and_cpu_are_equal_collect( - lambda spark : binary_op_df(spark, data_gen).select( - f.col('a') > s1, - s2 > f.col('b'), - f.lit(None).cast(data_type) > f.col('a'), - f.col('b') > f.lit(None).cast(data_type), - f.col('a') > f.col('b'))) + def test_func(data_gen): + (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) + data_type = data_gen.data_type + assert_gpu_and_cpu_are_equal_collect( + lambda spark : binary_op_df(spark, data_gen).select( + f.col('a') > s1, + s2 > f.col('b'), + f.lit(None).cast(data_type) > f.col('a'), + f.col('b') > f.lit(None).cast(data_type), + f.col('a') > f.col('b'))) + # DayTimeIntervalType not supported inside Structs -- issue #6184 + # data_gens = [DayTimeIntervalGen(), + # StructGen([['child0', StructGen([['child2', DayTimeIntervalGen()]])], ['child1', short_gen]])] + data_gens = [DayTimeIntervalGen()] + for data_gen in data_gens: + test_func(data_gen) -@pytest.mark.parametrize('data_gen', orderable_gens, ids=idfn) +@pytest.mark.parametrize('data_gen', orderable_gens + struct_gens_sample_with_decimal128_no_list, ids=idfn) def test_gte(data_gen): (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) data_type = data_gen.data_type @@ -185,16 +216,22 @@ def test_gte(data_gen): @pytest.mark.skipif(is_before_spark_330(), reason='DayTimeInterval is not supported before Pyspark 3.3.0') def test_gte_for_interval(): - data_gen = DayTimeIntervalGen() - (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) - data_type = data_gen.data_type - assert_gpu_and_cpu_are_equal_collect( - lambda spark : binary_op_df(spark, data_gen).select( - f.col('a') >= s1, - s2 >= f.col('b'), - f.lit(None).cast(data_type) >= f.col('a'), - f.col('b') >= f.lit(None).cast(data_type), - f.col('a') >= f.col('b'))) + def test_func(data_gen): + (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) + data_type = data_gen.data_type + assert_gpu_and_cpu_are_equal_collect( + lambda spark : binary_op_df(spark, data_gen).select( + f.col('a') >= s1, + s2 >= f.col('b'), + f.lit(None).cast(data_type) >= f.col('a'), + f.col('b') >= f.lit(None).cast(data_type), + f.col('a') >= f.col('b'))) + # DayTimeIntervalType not supported inside Structs -- issue #6184 + # data_gens = [DayTimeIntervalGen(), + # StructGen([['child0', StructGen([['child2', DayTimeIntervalGen()]])], ['child1', short_gen]])] + data_gens = [DayTimeIntervalGen()] + for data_gen in data_gens: + test_func(data_gen) @pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + [binary_gen] + array_gens_sample + struct_gens_sample + map_gens_sample, ids=idfn) def test_isnull(data_gen): diff --git a/integration_tests/src/main/python/data_gen.py b/integration_tests/src/main/python/data_gen.py index 20db0478ac6..ac3c812a5ae 100644 --- a/integration_tests/src/main/python/data_gen.py +++ b/integration_tests/src/main/python/data_gen.py @@ -975,11 +975,16 @@ def gen_scalars_for_sql(data_gen, count, seed=0, force_no_nulls=False): nonempty_struct_gens_sample = [all_basic_struct_gen, StructGen([['child0', byte_gen], ['child1', all_basic_struct_gen]]), StructGen([['child0', ArrayGen(short_gen)], ['child1', double_gen]])] +nonempty_struct_gens_sample_no_list = [all_basic_struct_gen, + StructGen([['child0', byte_gen], ['child1', all_basic_struct_gen]]), + StructGen([['child0', short_gen], ['child1', double_gen]])] struct_gens_sample = nonempty_struct_gens_sample + [StructGen([])] +struct_gens_sample_no_list = nonempty_struct_gens_sample_no_list + [StructGen([])] struct_gen_decimal128 = StructGen( [['child' + str(ind), sub_gen] for ind, sub_gen in enumerate([decimal_gen_128bit])]) struct_gens_sample_with_decimal128 = struct_gens_sample + [struct_gen_decimal128] +struct_gens_sample_with_decimal128_no_list = struct_gens_sample_no_list + [struct_gen_decimal128] simple_string_to_string_map_gen = MapGen(StringGen(pattern='key_[0-9]', nullable=False), StringGen(), max_length=10) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 65a67f1035e..649d1e65534 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -1973,11 +1973,11 @@ object GpuOverrides extends Logging { ExprChecks.binaryProjectAndAst( TypeSig.comparisonAstTypes, TypeSig.BOOLEAN, TypeSig.BOOLEAN, - ("lhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("lhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.comparable), - ("rhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("rhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.comparable)), (a, conf, p, r) => new BinaryAstExprMeta[EqualTo](a, conf, p, r) { override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = @@ -1988,11 +1988,11 @@ object GpuOverrides extends Logging { ExprChecks.binaryProjectAndAst( TypeSig.comparisonAstTypes, TypeSig.BOOLEAN, TypeSig.BOOLEAN, - ("lhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("lhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.orderable), - ("rhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("rhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.orderable)), (a, conf, p, r) => new BinaryAstExprMeta[GreaterThan](a, conf, p, r) { override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = @@ -2003,11 +2003,11 @@ object GpuOverrides extends Logging { ExprChecks.binaryProjectAndAst( TypeSig.comparisonAstTypes, TypeSig.BOOLEAN, TypeSig.BOOLEAN, - ("lhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("lhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.orderable), - ("rhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("rhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.orderable)), (a, conf, p, r) => new BinaryAstExprMeta[GreaterThanOrEqual](a, conf, p, r) { override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = @@ -2053,11 +2053,11 @@ object GpuOverrides extends Logging { ExprChecks.binaryProjectAndAst( TypeSig.comparisonAstTypes, TypeSig.BOOLEAN, TypeSig.BOOLEAN, - ("lhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("lhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.orderable), - ("rhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("rhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.orderable)), (a, conf, p, r) => new BinaryAstExprMeta[LessThan](a, conf, p, r) { override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = @@ -2068,11 +2068,11 @@ object GpuOverrides extends Logging { ExprChecks.binaryProjectAndAst( TypeSig.comparisonAstTypes, TypeSig.BOOLEAN, TypeSig.BOOLEAN, - ("lhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("lhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.orderable), - ("rhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("rhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.orderable)), (a, conf, p, r) => new BinaryAstExprMeta[LessThanOrEqual](a, conf, p, r) { override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = diff --git a/tools/src/main/resources/supportedExprs.csv b/tools/src/main/resources/supportedExprs.csv index 50e7a4f8ebc..7214faf2249 100644 --- a/tools/src/main/resources/supportedExprs.csv +++ b/tools/src/main/resources/supportedExprs.csv @@ -179,8 +179,8 @@ EndsWith,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA, EqualNullSafe,S,`<=>`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS EqualNullSafe,S,`<=>`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS EqualNullSafe,S,`<=>`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -EqualTo,S,`=`; `==`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS -EqualTo,S,`=`; `==`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS +EqualTo,S,`=`; `==`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS +EqualTo,S,`=`; `==`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS EqualTo,S,`=`; `==`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA EqualTo,S,`=`; `==`,None,AST,lhs,S,S,S,S,S,NS,NS,S,PS,NS,NS,NS,NS,NS,NS,NA,NS,NS EqualTo,S,`=`; `==`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,NS,NS,NS,NS,NS,NS,NA,NS,NS @@ -216,14 +216,14 @@ GetStructField,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS GetTimestamp,S, ,None,project,timeExp,NA,NA,NA,NA,NA,NA,NA,S,PS,S,NA,NA,NA,NA,NA,NA,NA,NA GetTimestamp,S, ,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA GetTimestamp,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA -GreaterThan,S,`>`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS -GreaterThan,S,`>`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS +GreaterThan,S,`>`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS +GreaterThan,S,`>`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS GreaterThan,S,`>`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GreaterThan,S,`>`,None,AST,lhs,S,S,S,S,S,NS,NS,S,PS,NS,NS,NS,NS,NS,NS,NA,NS,NS GreaterThan,S,`>`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,NS,NS,NS,NS,NS,NS,NA,NS,NS GreaterThan,S,`>`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -GreaterThanOrEqual,S,`>=`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS -GreaterThanOrEqual,S,`>=`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS +GreaterThanOrEqual,S,`>=`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS +GreaterThanOrEqual,S,`>=`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS GreaterThanOrEqual,S,`>=`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA GreaterThanOrEqual,S,`>=`,None,AST,lhs,S,S,S,S,S,NS,NS,S,PS,NS,NS,NS,NS,NS,NS,NA,NS,NS GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,NS,NS,NS,NS,NS,NS,NA,NS,NS @@ -281,14 +281,14 @@ Least,S,`least`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS Least,S,`least`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS Length,S,`length`; `character_length`; `char_length`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NA Length,S,`length`; `character_length`; `char_length`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -LessThan,S,`<`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS -LessThan,S,`<`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS +LessThan,S,`<`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS +LessThan,S,`<`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS LessThan,S,`<`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA LessThan,S,`<`,None,AST,lhs,S,S,S,S,S,NS,NS,S,PS,NS,NS,NS,NS,NS,NS,NA,NS,NS LessThan,S,`<`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,NS,NS,NS,NS,NS,NS,NA,NS,NS LessThan,S,`<`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -LessThanOrEqual,S,`<=`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS -LessThanOrEqual,S,`<=`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS +LessThanOrEqual,S,`<=`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS +LessThanOrEqual,S,`<=`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS LessThanOrEqual,S,`<=`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA LessThanOrEqual,S,`<=`,None,AST,lhs,S,S,S,S,S,NS,NS,S,PS,NS,NS,NS,NS,NS,NS,NA,NS,NS LessThanOrEqual,S,`<=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,NS,NS,NS,NS,NS,NS,NA,NS,NS From e35a7b765df5083d812c6edf06d71942fab9131e Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Fri, 12 Aug 2022 15:09:03 -0500 Subject: [PATCH 020/190] Add BinaryType to some shimmed expressions [databricks] (#6309) * Add BinaryType to some shimmed expressions Signed-off-by: Jason Lowe * Update docs Signed-off-by: Jason Lowe --- docs/supported_ops.md | 8 ++++---- .../com/nvidia/spark/rapids/shims/Spark31XShims.scala | 2 +- .../com/nvidia/spark/rapids/shims/Spark320PlusShims.scala | 7 ++++--- tools/src/main/resources/supportedExecs.csv | 2 +- 4 files changed, 10 insertions(+), 9 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 44aaa156fb3..357be9d19de 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -530,11 +530,11 @@ Accelerator supports are described below. S S S +S NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS diff --git a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/Spark31XShims.scala b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/Spark31XShims.scala index e080e6d9118..4496ea9c93c 100644 --- a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/Spark31XShims.scala +++ b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/Spark31XShims.scala @@ -111,7 +111,7 @@ abstract class Spark31XShims extends SparkShims with Spark31Xuntil33XShims with GpuOverrides.exec[CustomShuffleReaderExec]( "A wrapper of shuffle query stage", ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.MAP).nested(), TypeSig.all), + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY).nested(), TypeSig.all), (exec, conf, p, r) => new GpuCustomShuffleReaderMeta(exec, conf, p, r)) override def findOperators(plan: SparkPlan, predicate: SparkPlan => Boolean): Seq[SparkPlan] = { diff --git a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala index 2ce622e2eb5..ea175b2e6d8 100644 --- a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala +++ b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala @@ -60,7 +60,7 @@ trait Spark320PlusShims extends SparkShims with RebaseShims with Logging { override final def aqeShuffleReaderExec: ExecRule[_ <: SparkPlan] = exec[AQEShuffleReadExec]( "A wrapper of shuffle query stage", ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.MAP).nested(), TypeSig.all), + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY).nested(), TypeSig.all), (exec, conf, p, r) => new GpuCustomShuffleReaderMeta(exec, conf, p, r)) override final def sessionFromPlan(plan: SparkPlan): SparkSession = { @@ -191,8 +191,9 @@ trait Spark320PlusShims extends SparkShims with RebaseShims with Logging { ExprChecks.projectAndAst( TypeSig.astTypes, (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.CALENDAR - + TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT + TypeSig.ansiIntervals) - .nested(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT + + TypeSig.ansiIntervals) + .nested(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT), TypeSig.all), (lit, conf, p, r) => new LiteralExprMeta(lit, conf, p, r)), diff --git a/tools/src/main/resources/supportedExecs.csv b/tools/src/main/resources/supportedExecs.csv index 99efa00b74a..ef75558f809 100644 --- a/tools/src/main/resources/supportedExecs.csv +++ b/tools/src/main/resources/supportedExecs.csv @@ -14,7 +14,7 @@ SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS UnionExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS -CustomShuffleReaderExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS +CustomShuffleReaderExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS HashAggregateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS ObjectHashAggregateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,PS,NS,PS,PS,PS,NS SortAggregateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,PS,NS,PS,PS,PS,NS From 686ac8969f7028a0b2b38b4c433db5b81a6bd419 Mon Sep 17 00:00:00 2001 From: Peixin Date: Mon, 15 Aug 2022 08:47:02 +0800 Subject: [PATCH 021/190] Increase robustness of MVN commands in pre-merge scripts [databricks] (#6305) * Increase robustness of MVN commands in pre-merge scripts Signed-off-by: Peixin Li * fix dollar sign * fix db MVN_OPT Signed-off-by: Peixin Li --- jenkins/databricks/build.sh | 78 ++++++++++++++++----------------- jenkins/spark-premerge-build.sh | 39 ++++++++++------- 2 files changed, 62 insertions(+), 55 deletions(-) diff --git a/jenkins/databricks/build.sh b/jenkins/databricks/build.sh index 82a5d899b21..f2ac625edc5 100755 --- a/jenkins/databricks/build.sh +++ b/jenkins/databricks/build.sh @@ -55,10 +55,10 @@ then fi export WORKSPACE=`pwd` - -SPARK_PLUGIN_JAR_VERSION=`mvn help:evaluate -q -pl dist -Dexpression=project.version -DforceStdout` -SCALA_VERSION=`mvn help:evaluate -q -pl dist -Dexpression=scala.binary.version -DforceStdout` -CUDA_VERSION=`mvn help:evaluate -q -pl dist -Dexpression=cuda.version -DforceStdout` +MVN_CMD="mvn -Dmaven.wagon.http.retryHandler.count=3" +SPARK_PLUGIN_JAR_VERSION=`$MVN_CMD help:evaluate -q -pl dist -Dexpression=project.version -DforceStdout` +SCALA_VERSION=`$MVN_CMD help:evaluate -q -pl dist -Dexpression=scala.binary.version -DforceStdout` +CUDA_VERSION=`$MVN_CMD help:evaluate -q -pl dist -Dexpression=cuda.version -DforceStdout` RAPIDS_BUILT_JAR=rapids-4-spark_$SCALA_VERSION-$SPARK_PLUGIN_JAR_VERSION.jar @@ -173,7 +173,7 @@ fi # Please note we are installing all of these dependencies using the Spark version (SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS) to make it easier # to specify the dependencies in the pom files -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$CATALYSTJAR \ -DgroupId=org.apache.spark \ @@ -181,7 +181,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$SQLJAR \ -DgroupId=org.apache.spark \ @@ -189,7 +189,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$AVROSPARKJAR\ -DgroupId=org.apache.spark \ @@ -197,7 +197,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$AVROMAPRED\ -DgroupId=org.apache.avro\ @@ -205,7 +205,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$AVROJAR \ -DgroupId=org.apache.avro\ @@ -213,7 +213,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$ANNOTJAR \ -DgroupId=org.apache.spark \ @@ -221,7 +221,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar - mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$HIVEEXECJAR \ -DgroupId=org.apache.hive \ @@ -229,7 +229,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$COREJAR \ -DgroupId=org.apache.spark \ @@ -237,7 +237,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$JACKSONCORE\ -DgroupId=com.fasterxml.jackson.core \ @@ -245,7 +245,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$JACKSONANNOTATION\ -DgroupId=com.fasterxml.jackson.core \ @@ -253,7 +253,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$PROTOBUFJAVA \ -DgroupId=com.google.protobuf \ @@ -261,7 +261,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$JAVAASSIST\ -DgroupId=org.javaassist\ @@ -269,7 +269,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$JSON4S \ -DgroupId=org.json4s \ @@ -277,7 +277,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$APACHECOMMONSLANG3 \ -DgroupId=org.apache.commons \ @@ -285,7 +285,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$APACHECOMMONS \ -DgroupId=org.apache.commons \ @@ -293,7 +293,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$KRYO \ -DgroupId=com.esotericsoftware.kryo \ @@ -301,7 +301,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$LAUNCHER \ -DgroupId=org.apache.spark \ @@ -309,7 +309,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$NETWORKCOMMON \ -DgroupId=org.apache.spark \ @@ -317,7 +317,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$NETWORKSHUFFLE \ -DgroupId=org.apache.spark \ @@ -325,7 +325,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$COMMONUNSAFE\ -DgroupId=org.apache.spark \ @@ -333,7 +333,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$HIVESTORAGE \ -DgroupId=org.apache.hive \ @@ -341,7 +341,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$HIVEJAR\ -DgroupId=org.apache.spark \ @@ -349,7 +349,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$HIVESERDEJAR \ -DgroupId=org.apache.hive \ @@ -357,7 +357,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$PARQUETHADOOPJAR \ -DgroupId=org.apache.parquet \ @@ -365,7 +365,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$PARQUETCOMMONJAR \ -DgroupId=org.apache.parquet \ @@ -373,7 +373,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$PARQUETCOLUMNJAR \ -DgroupId=org.apache.parquet \ @@ -381,7 +381,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$PARQUETFORMATJAR \ -DgroupId=org.apache.parquet \ @@ -389,7 +389,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$ARROWFORMATJAR \ -DgroupId=org.apache.arrow \ @@ -397,7 +397,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$ARROWMEMORYJAR \ -DgroupId=org.apache.arrow \ @@ -405,7 +405,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$ARROWVECTORJAR \ -DgroupId=org.apache.arrow \ @@ -413,7 +413,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$ORC_CORE_JAR \ -DgroupId=org.apache.orc \ @@ -421,7 +421,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$ORC_SHIM_JAR \ -DgroupId=org.apache.orc \ @@ -429,7 +429,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$ORC_MAPREDUCE_JAR \ -DgroupId=org.apache.orc \ @@ -437,7 +437,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B install:install-file \ +$MVN_CMD -B install:install-file \ -Dmaven.repo.local=$M2DIR \ -Dfile=$JARDIR/$PROTOBUF_JAR \ -DgroupId=com.google.protobuf \ @@ -445,7 +445,7 @@ mvn -B install:install-file \ -Dversion=$SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS \ -Dpackaging=jar -mvn -B -Ddatabricks -Dbuildver=$BUILDVER clean package -DskipTests $MVN_OPT +$MVN_CMD -B -Ddatabricks -Dbuildver=$BUILDVER clean package -DskipTests $MVN_OPT cd /home/ubuntu tar -zcf spark-rapids-built.tgz spark-rapids diff --git a/jenkins/spark-premerge-build.sh b/jenkins/spark-premerge-build.sh index 9fa86cfb830..a2306c77814 100755 --- a/jenkins/spark-premerge-build.sh +++ b/jenkins/spark-premerge-build.sh @@ -27,6 +27,8 @@ elif [[ $# -gt 1 ]]; then exit 1 fi +MVN_CMD="mvn -Dmaven.wagon.http.retryHandler.count=3" +MVN_BUILD_ARGS="-Drat.skip=true -Dmaven.javadoc.skip=true -Dskip -Dmaven.scalastyle.skip=true -Dcuda.version=$CUDA_CLASSIFIER" mvn_verify() { echo "Run mvn verify..." @@ -36,29 +38,34 @@ mvn_verify() { pre-commit run check-added-large-files --from-ref $BASE_REF --to-ref HEAD # build the Spark 2.x explain jar - env -u SPARK_HOME mvn -B $MVN_URM_MIRROR -Dbuildver=24X clean install -DskipTests + env -u SPARK_HOME $MVN_CMD -B $MVN_URM_MIRROR -Dbuildver=24X clean install -DskipTests - # build all the versions but only run unit tests on one 3.1.X version (base version covers this), and one 3.2.X version. + MVN_INSTALL_CMD="env -u SPARK_HOME $MVN_CMD -U -B $MVN_URM_MIRROR clean install $MVN_BUILD_ARGS -DskipTests -pl aggregator -am" + # build all the versions but only run unit tests on one 3.1.X version (base version covers this), one 3.2.X and one 3.3.X version. # All others shims test should be covered in nightly pipelines - env -u SPARK_HOME mvn -U -B $MVN_URM_MIRROR -Dbuildver=321cdh clean install -Drat.skip=true -DskipTests -Dmaven.javadoc.skip=true -Dskip -Dmaven.scalastyle.skip=true -Dcuda.version=$CUDA_CLASSIFIER -pl aggregator -am - env -u SPARK_HOME mvn -U -B $MVN_URM_MIRROR -Dbuildver=312 clean install -Drat.skip=true -DskipTests -Dmaven.javadoc.skip=true -Dskip -Dmaven.scalastyle.skip=true -Dcuda.version=$CUDA_CLASSIFIER -pl aggregator -am - env -u SPARK_HOME mvn -U -B $MVN_URM_MIRROR -Dbuildver=313 clean install -Drat.skip=true -DskipTests -Dmaven.javadoc.skip=true -Dskip -Dmaven.scalastyle.skip=true -Dcuda.version=$CUDA_CLASSIFIER -pl aggregator -am - [[ $BUILD_MAINTENANCE_VERSION_SNAPSHOTS == "true" ]] && env -u SPARK_HOME mvn -U -B $MVN_URM_MIRROR -Dbuildver=314 clean install -Drat.skip=true -DskipTests -Dmaven.javadoc.skip=true -Dskip -Dmaven.scalastyle.skip=true -Dcuda.version=$CUDA_CLASSIFIER -pl aggregator -am + $MVN_INSTALL_CMD -DskipTests -Dbuildver=321cdh + $MVN_INSTALL_CMD -DskipTests -Dbuildver=312 + $MVN_INSTALL_CMD -DskipTests -Dbuildver=313 + [[ $BUILD_MAINTENANCE_VERSION_SNAPSHOTS == "true" ]] && $MVN_INSTALL_CMD -Dbuildver=314 # don't skip tests - env -u SPARK_HOME mvn -U -B $MVN_URM_MIRROR -Dbuildver=320 clean install -Drat.skip=true -Dmaven.javadoc.skip=true -Dskip -Dmaven.scalastyle.skip=true -Dcuda.version=$CUDA_CLASSIFIER -Dpytest.TEST_TAGS='' -pl '!tools' + env -u SPARK_HOME $MVN_CMD -U -B $MVN_URM_MIRROR -Dbuildver=320 clean install $MVN_BUILD_ARGS \ + -Dpytest.TEST_TAGS='' -pl '!tools' # enable UTF-8 for regular expression tests - env -u SPARK_HOME LC_ALL="en_US.UTF-8" mvn $MVN_URM_MIRROR -Dbuildver=320 test -Drat.skip=true -Dmaven.javadoc.skip=true -Dskip -Dmaven.scalastyle.skip=true -Dcuda.version=$CUDA_CLASSIFIER -Dpytest.TEST_TAGS='' -pl '!tools' -DwildcardSuites=com.nvidia.spark.rapids.ConditionalsSuite,com.nvidia.spark.rapids.RegularExpressionSuite,com.nvidia.spark.rapids.RegularExpressionTranspilerSuite - env -u SPARK_HOME mvn -U -B $MVN_URM_MIRROR -Dbuildver=321 clean install -Drat.skip=true -DskipTests -Dmaven.javadoc.skip=true -Dskip -Dmaven.scalastyle.skip=true -Dcuda.version=$CUDA_CLASSIFIER -pl aggregator -am - env -u SPARK_HOME mvn -U -B $MVN_URM_MIRROR -Dbuildver=322 clean install -Drat.skip=true -DskipTests -Dmaven.javadoc.skip=true -Dskip -Dmaven.scalastyle.skip=true -Dcuda.version=$CUDA_CLASSIFIER -pl aggregator -am - env -u SPARK_HOME mvn -U -B $MVN_URM_MIRROR -Dbuildver=330 clean install -Drat.skip=true -Dmaven.javadoc.skip=true -Dskip -Dmaven.scalastyle.skip=true -Dcuda.version=$CUDA_CLASSIFIER -pl aggregator -am - [[ $BUILD_MAINTENANCE_VERSION_SNAPSHOTS == "true" ]] && env -u SPARK_HOME mvn -U -B $MVN_URM_MIRROR -Dbuildver=331 clean install -Drat.skip=true -DskipTests -Dmaven.javadoc.skip=true -Dskip -Dmaven.scalastyle.skip=true -Dcuda.version=$CUDA_CLASSIFIER -pl aggregator -am + env -u SPARK_HOME LC_ALL="en_US.UTF-8" $MVN_CMD $MVN_URM_MIRROR -Dbuildver=320 test $MVN_BUILD_ARGS \ + -Dpytest.TEST_TAGS='' -pl '!tools' \ + -DwildcardSuites=com.nvidia.spark.rapids.ConditionalsSuite,com.nvidia.spark.rapids.RegularExpressionSuite,com.nvidia.spark.rapids.RegularExpressionTranspilerSuite + $MVN_INSTALL_CMD -DskipTests -Dbuildver=321 + $MVN_INSTALL_CMD -DskipTests -Dbuildver=322 + env -u SPARK_HOME $MVN_CMD -U -B $MVN_URM_MIRROR -Dbuildver=330 clean install $MVN_BUILD_ARGS \ + -Dpytest.TEST_TAGS='' -pl '!tools' + [[ $BUILD_MAINTENANCE_VERSION_SNAPSHOTS == "true" ]] && $MVN_INSTALL_CMD -DskipTests -Dbuildver=331 # TODO: move it to BUILD_MAINTENANCE_VERSION_SNAPSHOTS when we resolve all spark340 build issues - [[ $BUILD_FEATURE_VERSION_SNAPSHOTS == "true" ]] && env -u SPARK_HOME mvn -U -B $MVN_URM_MIRROR -Dbuildver=340 clean install -Drat.skip=true -DskipTests -Dmaven.javadoc.skip=true -Dskip -Dmaven.scalastyle.skip=true -Dcuda.version=$CUDA_CLASSIFIER -pl aggregator -am + [[ $BUILD_FEATURE_VERSION_SNAPSHOTS == "true" ]] && $MVN_INSTALL_CMD -DskipTests -Dbuildver=340 # Here run Python integration tests tagged with 'premerge_ci_1' only, that would help balance test duration and memory # consumption from two k8s pods running in parallel, which executes 'mvn_verify()' and 'ci_2()' respectively. - mvn -B $MVN_URM_MIRROR $PREMERGE_PROFILES clean verify -Dpytest.TEST_TAGS="premerge_ci_1" \ + $MVN_CMD -B $MVN_URM_MIRROR $PREMERGE_PROFILES clean verify -Dpytest.TEST_TAGS="premerge_ci_1" \ -Dpytest.TEST_TYPE="pre-commit" -Dpytest.TEST_PARALLEL=4 -Dcuda.version=$CUDA_CLASSIFIER # The jacoco coverage should have been collected, but because of how the shade plugin @@ -128,7 +135,7 @@ rapids_shuffle_smoke_test() { ci_2() { echo "Run premerge ci 2 testings..." - mvn -U -B $MVN_URM_MIRROR clean package -DskipTests=true -Dcuda.version=$CUDA_CLASSIFIER + $MVN_CMD -U -B $MVN_URM_MIRROR clean package $MVN_BUILD_ARGS -DskipTests=true export TEST_TAGS="not premerge_ci_1" export TEST_TYPE="pre-commit" export TEST_PARALLEL=4 @@ -163,7 +170,7 @@ elif [[ ${PROJECT_VER} =~ ^22\.12\. ]]; then fi ARTF_ROOT="$WORKSPACE/.download" -MVN_GET_CMD="mvn org.apache.maven.plugins:maven-dependency-plugin:2.8:get -B \ +MVN_GET_CMD="$MVN_CMD org.apache.maven.plugins:maven-dependency-plugin:2.8:get -B \ $MVN_URM_MIRROR -DremoteRepositories=$URM_URL \ -Ddest=$ARTF_ROOT" From a1f39e409d7826a9180b4e661805c8ad8ea284ed Mon Sep 17 00:00:00 2001 From: Anthony Chang <54450499+anthony-chang@users.noreply.github.com> Date: Mon, 15 Aug 2022 13:05:39 -0400 Subject: [PATCH 022/190] Estimate and validate regular expression complexities (#6006) * [WIP] initial work on estimating the number of states in a pattern Signed-off-by: Anthony Chang * Change memory estimation formula and add checks Signed-off-by: Anthony Chang * Add check for stringsplit Signed-off-by: Anthony Chang * Return long for gpu memory estimate Signed-off-by: Anthony Chang * Update configs.md Signed-off-by: Anthony Chang * Address feedback Signed-off-by: Anthony Chang * Update configs.md Signed-off-by: Anthony Chang * Address feedback Signed-off-by: Anthony Chang * Apply suggestions from code review Co-authored-by: Andy Grove * Add comment Signed-off-by: Anthony Chang * Set regex state memory in integration tests to not trigger the warning when running integration tests Signed-off-by: Anthony Chang * Add integration tests Signed-off-by: Anthony Chang * Update configs.md Signed-off-by: Anthony Chang * Change regexp_like to rlike Signed-off-by: Anthony Chang Signed-off-by: Anthony Chang Co-authored-by: Andy Grove --- docs/configs.md | 1 + integration_tests/run_pyspark_from_build.sh | 1 + .../src/main/python/regexp_test.py | 46 ++++++++++++ .../spark/rapids/GpuRegExpReplaceMeta.scala | 7 +- .../com/nvidia/spark/rapids/RapidsConf.scala | 20 ++++++ .../rapids/RegexComplexityEstimator.scala | 71 +++++++++++++++++++ .../com/nvidia/spark/rapids/RegexParser.scala | 27 +++++-- .../spark/sql/rapids/stringFunctions.scala | 28 ++++++-- 8 files changed, 186 insertions(+), 15 deletions(-) create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexComplexityEstimator.scala diff --git a/docs/configs.md b/docs/configs.md index c49545f4b12..b4598b315a8 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -130,6 +130,7 @@ Name | Description | Default Value spark.rapids.sql.reader.batchSizeBytes|Soft limit on the maximum number of bytes the reader reads per batch. The readers will read chunks of data until this limit is met or exceeded. Note that the reader may estimate the number of bytes that will be used on the GPU in some cases based on the schema and number of rows in each batch.|2147483647 spark.rapids.sql.reader.batchSizeRows|Soft limit on the maximum number of rows the reader will read per batch. The orc and parquet readers will read row groups until this limit is met or exceeded. The limit is respected by the csv reader.|2147483647 spark.rapids.sql.regexp.enabled|Specifies whether supported regular expressions will be evaluated on the GPU. Unsupported expressions will fall back to CPU. However, there are some known edge cases that will still execute on GPU and produce incorrect results and these are documented in the compatibility guide. Setting this config to false will make all regular expressions run on the CPU instead.|true +spark.rapids.sql.regexp.maxStateMemoryBytes|Specifies the maximum memory on GPU to be used for regular expressions.The memory usage is an estimate based on an upper-bound approximation on the complexity of the regular expression. Note that the actual memory usage may still be higher than this estimate depending on the number of rows in the datacolumn and the input strings themselves. It is recommended to not set this to more than 3 times spark.rapids.sql.batchSizeBytes|2147483647 spark.rapids.sql.replaceSortMergeJoin.enabled|Allow replacing sortMergeJoin with HashJoin|true spark.rapids.sql.rowBasedUDF.enabled|When set to true, optimizes a row-based UDF in a GPU operation by transferring only the data it needs between GPU and CPU inside a query operation, instead of falling this operation back to CPU. This is an experimental feature, and this config might be removed in the future.|false spark.rapids.sql.shuffle.spillThreads|Number of threads used to spill shuffle data to disk in the background.|6 diff --git a/integration_tests/run_pyspark_from_build.sh b/integration_tests/run_pyspark_from_build.sh index af61f324be8..7c1a98a6d4b 100755 --- a/integration_tests/run_pyspark_from_build.sh +++ b/integration_tests/run_pyspark_from_build.sh @@ -208,6 +208,7 @@ else # Not the default 2G but should be large enough for a single batch for all data (we found # 200 MiB being allocated by a single test at most, and we typically have 4 tasks. export PYSP_TEST_spark_rapids_sql_batchSizeBytes='100m' + export PYSP_TEST_spark_rapids_sql_regexp_maxStateMemoryBytes='300m' # Extract Databricks version from deployed configs. This is set automatically on Databricks # notebooks but not when running Spark manually. diff --git a/integration_tests/src/main/python/regexp_test.py b/integration_tests/src/main/python/regexp_test.py index 2cc6610831b..7015b9bdea3 100644 --- a/integration_tests/src/main/python/regexp_test.py +++ b/integration_tests/src/main/python/regexp_test.py @@ -766,3 +766,49 @@ def test_regexp_split_unicode_support(): 'split(a, "[bf]", -1)', 'split(a, "[o]", -2)'), conf=_regexp_conf) + +@allow_non_gpu('ProjectExec', 'RLike') +def test_regexp_memory_fallback(): + gen = StringGen('test') + assert_gpu_fallback_collect( + lambda spark: unary_op_df(spark, gen).selectExpr( + 'a rlike "a{6}"', + 'a rlike "a{6,}"', + 'a rlike "(?:ab){0,3}"', + 'a rlike "(?:12345)?"', + 'a rlike "(?:12345)+"', + 'a rlike "(?:123456)*"', + 'a rlike "a{1,6}"', + 'a rlike "abcdef"', + 'a rlike "(1)(2)(3)"', + 'a rlike "1|2|3|4|5|6"' + ), + cpu_fallback_class_name='RLike', + conf={ + 'spark.rapids.sql.regexp.enabled': 'true', + 'spark.rapids.sql.regexp.maxStateMemoryBytes': '10', + 'spark.rapids.sql.batchSizeBytes': '20' # 1 row in the batch + } + ) + +def test_regexp_memory_ok(): + gen = StringGen('test') + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, gen).selectExpr( + 'a rlike "a{6}"', + 'a rlike "a{6,}"', + 'a rlike "(?:ab){0,3}"', + 'a rlike "(?:12345)?"', + 'a rlike "(?:12345)+"', + 'a rlike "(?:123456)*"', + 'a rlike "a{1,6}"', + 'a rlike "abcdef"', + 'a rlike "(1)(2)(3)"', + 'a rlike "1|2|3|4|5|6"' + ), + conf={ + 'spark.rapids.sql.regexp.enabled': 'true', + 'spark.rapids.sql.regexp.maxStateMemoryBytes': '12', + 'spark.rapids.sql.batchSizeBytes': '20' # 1 row in the batch + } + ) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRegExpReplaceMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRegExpReplaceMeta.scala index cdcbfbdd627..22f5ae78268 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRegExpReplaceMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRegExpReplaceMeta.scala @@ -48,9 +48,10 @@ class GpuRegExpReplaceMeta( try { javaPattern = Some(s.toString()) val (pat, repl) = - new CudfRegexTranspiler(RegexReplaceMode).transpile(s.toString, replacement) - cudfPattern = Some(pat) - repl.map(GpuRegExpUtils.backrefConversion).foreach { + new CudfRegexTranspiler(RegexReplaceMode).getTranspiledAST(s.toString, replacement) + GpuRegExpUtils.validateRegExpComplexity(this, pat) + cudfPattern = Some(pat.toRegexString) + repl.map { r => GpuRegExpUtils.backrefConversion(r.toRegexString) }.foreach { case (hasBackref, convertedRep) => containsBackref = hasBackref replacement = Some(GpuRegExpUtils.unescapeReplaceString(convertedRep)) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 54b735a0264..e2766970aef 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -1068,6 +1068,16 @@ object RapidsConf { .booleanConf .createWithDefault(true) + val REGEXP_MAX_STATE_MEMORY_BYTES = conf("spark.rapids.sql.regexp.maxStateMemoryBytes") + .doc("Specifies the maximum memory on GPU to be used for regular expressions." + + "The memory usage is an estimate based on an upper-bound approximation on the " + + "complexity of the regular expression. Note that the actual memory usage may " + + "still be higher than this estimate depending on the number of rows in the data" + + "column and the input strings themselves. It is recommended to not set this to " + + s"more than 3 times ${GPU_BATCH_SIZE_BYTES.key}") + .bytesConf(ByteUnit.BYTE) + .createWithDefault(Integer.MAX_VALUE) + // INTERNAL TEST AND DEBUG CONFIGS val TEST_CONF = conf("spark.rapids.sql.test.enabled") @@ -2032,6 +2042,16 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isRegExpEnabled: Boolean = get(ENABLE_REGEXP) + lazy val maxRegExpStateMemory: Long = { + val size = get(REGEXP_MAX_STATE_MEMORY_BYTES) + if (size > 3 * gpuTargetBatchSizeBytes) { + logWarning(s"${REGEXP_MAX_STATE_MEMORY_BYTES.key} is more than 3 times " + + s"${GPU_BATCH_SIZE_BYTES.key}. This may cause regular expression operations to " + + s"encounter GPU out of memory errors.") + } + size + } + lazy val getSparkGpuResourceName: String = get(SPARK_GPU_RESOURCE_NAME) lazy val isCpuBasedUDFEnabled: Boolean = get(ENABLE_CPU_BASED_UDF) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexComplexityEstimator.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexComplexityEstimator.scala new file mode 100644 index 00000000000..9046ab00197 --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexComplexityEstimator.scala @@ -0,0 +1,71 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.nvidia.spark.rapids + +import org.apache.spark.sql.types.DataTypes + +object RegexComplexityEstimator { + private def countStates(regex: RegexAST): Int = { + regex match { + case RegexSequence(parts) => + parts.map(countStates).sum + case RegexGroup(true, term) => + 1 + countStates(term) + case RegexGroup(false, term) => + countStates(term) + case RegexCharacterClass(_, _) => + 1 + case RegexChoice(left, right) => + countStates(left) + countStates(right) + case RegexRepetition(term, QuantifierFixedLength(length)) => + length * countStates(term) + case RegexRepetition(term, SimpleQuantifier(ch)) => + ch match { + case '*' => + countStates(term) + case '+' => + 1 + countStates(term) + case '?' => + 1 + countStates(term) + } + case RegexRepetition(term, QuantifierVariableLength(minLength, maxLengthOption)) => + maxLengthOption match { + case Some(maxLength) => + maxLength * countStates(term) + case None => + minLength.max(1) * countStates(term) + } + case RegexChar(_) | RegexEscaped(_) | RegexHexDigit(_) | RegexOctalChar(_) => + 1 + case _ => + 0 + } + } + + private def estimateGpuMemory(numStates: Int, desiredBatchSizeBytes: Long): Long = { + val numRows = GpuBatchUtils.estimateRowCount( + desiredBatchSizeBytes, DataTypes.StringType.defaultSize, 1) + + // cuDF requests num_instructions * num_threads * 2 when allocating the memory on the device + // (ignoring memory alignment). We are trying to reproduce that calculation here: + numStates * numRows * 2 + } + + def isValid(conf: RapidsConf, regex: RegexAST): Boolean = { + val numStates = countStates(regex) + estimateGpuMemory(numStates, conf.gpuTargetBatchSizeBytes) <= conf.maxRegExpStateMemory + } +} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala index 917c30d6d29..ac9d38dec3e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala @@ -173,7 +173,7 @@ class RegexParser(pattern: String) { } private def parseCharacterClass(): RegexCharacterClass = { - val supportedMetaCharacters = "\\^-]+" + val supportedMetaCharacters = "\\^-[]+" def getEscapedComponent(): RegexCharacterClassComponent = { peek() match { @@ -207,7 +207,7 @@ class RegexParser(pattern: String) { RegexEscaped(ch) } else { throw new RegexUnsupportedException( - s"Unsupported escaped character in character class", Some(pos-1)) + s"Unsupported escaped character '$ch' in character class", Some(pos-1)) } } case None => @@ -682,6 +682,22 @@ class CudfRegexTranspiler(mode: RegexMode) { * @return Regular expression and optional replacement in cuDF format */ def transpile(pattern: String, repl: Option[String]): (String, Option[String]) = { + val (cudfRegex, replacement) = getTranspiledAST(pattern, repl) + + // write out to regex string, performing minor transformations + // such as adding additional escaping + (cudfRegex.toRegexString, replacement.map(_.toRegexString)) + } + + /** + * Parse Java regular expression and translate into cuDF regular expression in AST form. + * + * @param pattern Regular expression that is valid in Java's engine + * @param repl Optional replacement pattern + * @return Regular expression AST and optional replacement in cuDF format + */ + def getTranspiledAST( + pattern: String, repl: Option[String]): (RegexAST, Option[RegexReplacement]) = { // parse the source regular expression val regex = new RegexParser(pattern).parse() // if we have a replacement, parse the replacement string using the regex parser to account @@ -690,11 +706,10 @@ class CudfRegexTranspiler(mode: RegexMode) { // validate that the regex is supported by cuDF val cudfRegex = transpile(regex, replacement, None) - // write out to regex string, performing minor transformations - // such as adding additional escaping - (cudfRegex.toRegexString, replacement.map(_.toRegexString)) - } + (cudfRegex, replacement) + } + def transpileToSplittableString(e: RegexAST): Option[String] = { e match { case RegexEscaped(ch) if regexMetaChars.contains(ch) => Some(ch.toString) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala index 2294cfb75d9..92d55681f66 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala @@ -850,6 +850,13 @@ object GpuRegExpUtils { } } + def validateRegExpComplexity(meta: ExprMeta[_], regex: RegexAST): Unit = { + if(!RegexComplexityEstimator.isValid(meta.conf, regex)) { + meta.willNotWorkOnGpu(s"estimated memory needed for regular expression exceeds the maximum." + + s" Set ${RapidsConf.REGEXP_MAX_STATE_MEMORY_BYTES} to change it.") + } + } + /** * Recursively check if pattern contains only zero-match repetitions * ?, *, {0,}, or {0,n} or any combination of them. @@ -905,7 +912,10 @@ class GpuRLikeMeta( case Literal(str: UTF8String, DataTypes.StringType) if str != null => try { // verify that we support this regex and can transpile it to cuDF format - pattern = Some(new CudfRegexTranspiler(RegexFindMode).transpile(str.toString, None)._1) + val (transpiledAST, _) = + new CudfRegexTranspiler(RegexFindMode).getTranspiledAST(str.toString, None) + GpuRegExpUtils.validateRegExpComplexity(this, transpiledAST) + pattern = Some(transpiledAST.toRegexString) } catch { case e: RegexUnsupportedException => willNotWorkOnGpu(e.getMessage) @@ -1092,8 +1102,10 @@ class GpuRegExpExtractMeta( try { val javaRegexpPattern = str.toString // verify that we support this regex and can transpile it to cuDF format - pattern = Some(new CudfRegexTranspiler(RegexFindMode) - .transpile(javaRegexpPattern, None)._1) + val (transpiledAST, _) = + new CudfRegexTranspiler(RegexFindMode).getTranspiledAST(javaRegexpPattern, None) + GpuRegExpUtils.validateRegExpComplexity(this, transpiledAST) + pattern = Some(transpiledAST.toRegexString) numGroups = GpuRegExpUtils.countGroups(javaRegexpPattern) } catch { case e: RegexUnsupportedException => @@ -1211,8 +1223,10 @@ class GpuRegExpExtractAllMeta( try { val javaRegexpPattern = str.toString // verify that we support this regex and can transpile it to cuDF format - pattern = Some(new CudfRegexTranspiler(RegexFindMode) - .transpile(javaRegexpPattern, None)._1) + val (transpiledAST, _) = + new CudfRegexTranspiler(RegexFindMode).getTranspiledAST(javaRegexpPattern, None) + GpuRegExpUtils.validateRegExpComplexity(this, transpiledAST) + pattern = Some(transpiledAST.toRegexString) numGroups = GpuRegExpUtils.countGroups(javaRegexpPattern) } catch { case e: RegexUnsupportedException => @@ -1588,7 +1602,9 @@ abstract class StringSplitRegExpMeta[INPUT <: TernaryExpression](expr: INPUT, pattern = simplified case None => try { - pattern = transpiler.transpile(utf8Str.toString, None)._1 + val (transpiledAST, _) = transpiler.getTranspiledAST(utf8Str.toString, None) + GpuRegExpUtils.validateRegExpComplexity(this, transpiledAST) + pattern = transpiledAST.toRegexString isRegExp = true } catch { case e: RegexUnsupportedException => From 3261cceb80679fcec8e4865e3cd857081645392c Mon Sep 17 00:00:00 2001 From: zengruios <578395184@qq.com> Date: Tue, 16 Aug 2022 02:59:15 +0800 Subject: [PATCH 023/190] [BugFix]Change the RapidsDiskBlockManager in ShuffleBufferCatalog to guarantee the shuffle files can be cleaned successfully (#6318) Signed-off-by: zengruios <578395184@qq.com> Signed-off-by: zengruios <578395184@qq.com> --- .../scala/com/nvidia/spark/rapids/GpuDeviceManager.scala | 6 ++---- .../com/nvidia/spark/rapids/RapidsBufferCatalog.scala | 5 ++++- .../scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala | 7 +++---- .../com/nvidia/spark/rapids/GpuPartitioningSuite.scala | 6 +++--- .../nvidia/spark/rapids/GpuSinglePartitioningSuite.scala | 6 +++--- 5 files changed, 15 insertions(+), 15 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDeviceManager.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDeviceManager.scala index c2a480a9da8..7f34bb1aec3 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDeviceManager.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDeviceManager.scala @@ -232,10 +232,6 @@ object GpuDeviceManager extends Logging { val conf = rapidsConf.getOrElse(new RapidsConf(SparkEnv.get.conf)) val info = Cuda.memGetInfo() - // We need to reserve more memory when RAPIDS shuffle is enabled and we are using the CUDA - // async allocator, so initialize the shuffle environment first. - GpuShuffleEnv.init(conf) - val poolAllocation = computeRmmPoolSize(conf, info) var init = RmmAllocationMode.CUDA_DEFAULT val features = ArrayBuffer[String]() @@ -299,6 +295,8 @@ object GpuDeviceManager extends Logging { Cuda.setDevice(gpuId) Rmm.initialize(init, logConf, poolAllocation) RapidsBufferCatalog.init(conf) + + GpuShuffleEnv.init(conf, RapidsBufferCatalog.getDiskBlockManager()) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala index 8101a7d4f11..d4b0e31a01f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala @@ -158,6 +158,7 @@ object RapidsBufferCatalog extends Logging with Arm { val singleton = new RapidsBufferCatalog private var deviceStorage: RapidsDeviceMemoryStore = _ private var hostStorage: RapidsHostMemoryStore = _ + private var diskBlockManager: RapidsDiskBlockManager = _ private var diskStorage: RapidsDiskStore = _ private var gdsStorage: RapidsGdsStore = _ private var memoryEventHandler: DeviceMemoryEventHandler = _ @@ -178,7 +179,7 @@ object RapidsBufferCatalog extends Logging with Arm { closeImpl() assert(memoryEventHandler == null) deviceStorage = new RapidsDeviceMemoryStore() - val diskBlockManager = new RapidsDiskBlockManager(conf) + diskBlockManager = new RapidsDiskBlockManager(conf) if (rapidsConf.isGdsSpillEnabled) { gdsStorage = new RapidsGdsStore(diskBlockManager, rapidsConf.gdsSpillBatchWriteBufferSize) deviceStorage.setSpillStore(gdsStorage) @@ -298,4 +299,6 @@ object RapidsBufferCatalog extends Logging with Arm { /** Remove a buffer ID from the catalog and release the resources of the registered buffer. */ def removeBuffer(id: RapidsBufferId): Unit = singleton.removeBuffer(id) + + def getDiskBlockManager(): RapidsDiskBlockManager = diskBlockManager } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala index 803c43ee3ca..b9a301a2e27 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala @@ -44,9 +44,8 @@ class GpuShuffleEnv(rapidsConf: RapidsConf) extends Logging { } } - def init(): Unit = { + def init(diskBlockManager: RapidsDiskBlockManager): Unit = { if (isRapidsShuffleConfigured) { - val diskBlockManager = new RapidsDiskBlockManager(conf) shuffleCatalog = new ShuffleBufferCatalog(RapidsBufferCatalog.singleton, diskBlockManager) shuffleReceivedBufferCatalog = @@ -163,9 +162,9 @@ object GpuShuffleEnv extends Logging { // Functions below only get called from the executor // - def init(conf: RapidsConf): Unit = { + def init(conf: RapidsConf, diskBlockManager: RapidsDiskBlockManager): Unit = { val shuffleEnv = new GpuShuffleEnv(conf) - shuffleEnv.init() + shuffleEnv.init(diskBlockManager) env = shuffleEnv } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuPartitioningSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuPartitioningSuite.scala index 61b3a6f9fc3..713d2542e33 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuPartitioningSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuPartitioningSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2021, NVIDIA CORPORATION. + * Copyright (c) 2020-2022, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -112,7 +112,7 @@ class GpuPartitioningSuite extends FunSuite with Arm { TrampolineUtil.cleanupAnyExistingSession() val conf = new SparkConf().set(RapidsConf.SHUFFLE_COMPRESSION_CODEC.key, "none") TestUtils.withGpuSparkSession(conf) { _ => - GpuShuffleEnv.init(new RapidsConf(conf)) + GpuShuffleEnv.init(new RapidsConf(conf), new RapidsDiskBlockManager(conf)) val partitionIndices = Array(0, 2, 2) val gp = new GpuPartitioning { override val numPartitions: Int = partitionIndices.length @@ -144,7 +144,7 @@ class GpuPartitioningSuite extends FunSuite with Arm { val conf = new SparkConf() .set(RapidsConf.SHUFFLE_COMPRESSION_CODEC.key, "lz4") TestUtils.withGpuSparkSession(conf) { _ => - GpuShuffleEnv.init(new RapidsConf(conf)) + GpuShuffleEnv.init(new RapidsConf(conf), new RapidsDiskBlockManager(conf)) val spillPriority = 7L val catalog = RapidsBufferCatalog.singleton withResource(new RapidsDeviceMemoryStore(catalog)) { deviceStore => diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuSinglePartitioningSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuSinglePartitioningSuite.scala index 73889400f9a..f9a66f81f2d 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuSinglePartitioningSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuSinglePartitioningSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2021, NVIDIA CORPORATION. + * Copyright (c) 2020-2022, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -22,7 +22,7 @@ import ai.rapids.cudf.Table import org.scalatest.FunSuite import org.apache.spark.SparkConf -import org.apache.spark.sql.rapids.GpuShuffleEnv +import org.apache.spark.sql.rapids.{GpuShuffleEnv, RapidsDiskBlockManager} import org.apache.spark.sql.types.{DecimalType, DoubleType, IntegerType, StringType} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -44,7 +44,7 @@ class GpuSinglePartitioningSuite extends FunSuite with Arm { val conf = new SparkConf().set("spark.shuffle.manager", GpuShuffleEnv.RAPIDS_SHUFFLE_CLASS) .set(RapidsConf.SHUFFLE_COMPRESSION_CODEC.key, "none") TestUtils.withGpuSparkSession(conf) { _ => - GpuShuffleEnv.init(new RapidsConf(conf)) + GpuShuffleEnv.init(new RapidsConf(conf), new RapidsDiskBlockManager(conf)) val partitioner = GpuSinglePartitioning withResource(buildBatch()) { batch => withResource(GpuColumnVector.from(batch)) { table => From 7724f197797ee71a2d845bf0c20099ceee438866 Mon Sep 17 00:00:00 2001 From: Peixin Date: Tue, 16 Aug 2022 03:05:13 +0800 Subject: [PATCH 024/190] Increase robustness of mvn commands in nightly scripts (#6314) Signed-off-by: Peixin Li Signed-off-by: Peixin Li --- build/buildall | 2 +- jenkins/databricks/deploy.sh | 5 +++-- jenkins/deploy.sh | 6 +++--- jenkins/spark-nightly-build.sh | 2 +- jenkins/spark-tests.sh | 2 +- 5 files changed, 9 insertions(+), 8 deletions(-) diff --git a/build/buildall b/build/buildall index f767ce3e5e7..3bd32e8ae03 100755 --- a/build/buildall +++ b/build/buildall @@ -161,7 +161,7 @@ shift done # include options to mvn command -export MVN="mvn ${MVN_OPT}" +export MVN="mvn -Dmaven.wagon.http.retryHandler.count=3 ${MVN_OPT}" DIST_PROFILE=${DIST_PROFILE:-"noSnapshots"} [[ "$MODULE" != "" ]] && MODULE_OPT="--projects $MODULE --also-make" || MODULE_OPT="" diff --git a/jenkins/databricks/deploy.sh b/jenkins/databricks/deploy.sh index 2ae5b4a60d2..437a57631cb 100755 --- a/jenkins/databricks/deploy.sh +++ b/jenkins/databricks/deploy.sh @@ -32,9 +32,10 @@ SPARK_PLUGIN_JAR_VERSION=`mvn help:evaluate -q -pl dist -Dexpression=project.ver DB_SHIM_NAME=${SPARK_VERSION_STR}db DBJARFPATH=./aggregator/target/${DB_SHIM_NAME}/rapids-4-spark-aggregator_$SCALA_VERSION-$SPARK_PLUGIN_JAR_VERSION.jar echo "Databricks jar is: $DBJARFPATH" -mvn -B deploy:deploy-file $MVN_URM_MIRROR -Durl=$SERVER_URL -DrepositoryId=$SERVER_ID \ +MVN="mvn -Dmaven.wagon.http.retryHandler.count=3 -DretryFailedDeploymentCount=3" +$MVN -B deploy:deploy-file $MVN_URM_MIRROR -Durl=$SERVER_URL -DrepositoryId=$SERVER_ID \ -Dfile=$DBJARFPATH -DpomFile=aggregator/pom.xml -Dclassifier=$DB_SHIM_NAME # install the integration test jar DBINTTESTJARFPATH=./integration_tests/target/rapids-4-spark-integration-tests_$SCALA_VERSION-$SPARK_PLUGIN_JAR_VERSION-${DB_SHIM_NAME}.jar -mvn -B deploy:deploy-file $MVN_URM_MIRROR -Durl=$SERVER_URL -DrepositoryId=$SERVER_ID \ +$MVN -B deploy:deploy-file $MVN_URM_MIRROR -Durl=$SERVER_URL -DrepositoryId=$SERVER_ID \ -Dfile=$DBINTTESTJARFPATH -DpomFile=integration_tests/pom.xml -Dclassifier=$DB_SHIM_NAME diff --git a/jenkins/deploy.sh b/jenkins/deploy.sh index 2a290754782..f5afba5c376 100755 --- a/jenkins/deploy.sh +++ b/jenkins/deploy.sh @@ -62,16 +62,16 @@ echo "Plan to deploy ${FPATH}.jar to $SERVER_URL (ID:$SERVER_ID)" FINAL_AGG_VERSION_TOBUILD=${FINAL_AGG_VERSION_TOBUILD:-'311'} ###### Choose the deploy command ###### - +MVN="mvn -Dmaven.wagon.http.retryHandler.count=3 -DretryFailedDeploymentCount=3" if [ "$SIGN_FILE" == true ]; then # No javadoc and sources jar is generated for shade artifact only. Use 'sql-plugin' instead SQL_ART_ID=`mvn help:evaluate -q -pl $SQL_PL -Dexpression=project.artifactId -DforceStdout` SQL_ART_VER=`mvn help:evaluate -q -pl $SQL_PL -Dexpression=project.version -DforceStdout` JS_FPATH="${SQL_PL}/target/spark${FINAL_AGG_VERSION_TOBUILD}/${SQL_ART_ID}-${SQL_ART_VER}" SRC_DOC_JARS="-Dsources=${JS_FPATH}-sources.jar -Djavadoc=${JS_FPATH}-javadoc.jar" - DEPLOY_CMD="mvn -B gpg:sign-and-deploy-file -s jenkins/settings.xml -Dgpg.passphrase=$GPG_PASSPHRASE" + DEPLOY_CMD="$MVN -B gpg:sign-and-deploy-file -s jenkins/settings.xml -Dgpg.passphrase=$GPG_PASSPHRASE" else - DEPLOY_CMD="mvn -B deploy:deploy-file -s jenkins/settings.xml" + DEPLOY_CMD="$MVN -B deploy:deploy-file -s jenkins/settings.xml" fi echo "Deploy CMD: $DEPLOY_CMD" diff --git a/jenkins/spark-nightly-build.sh b/jenkins/spark-nightly-build.sh index e4016911806..4a11b15cb16 100755 --- a/jenkins/spark-nightly-build.sh +++ b/jenkins/spark-nightly-build.sh @@ -23,7 +23,7 @@ set -ex export M2DIR=${M2DIR:-"$WORKSPACE/.m2"} ## MVN_OPT : maven options environment, e.g. MVN_OPT='-Dspark-rapids-jni.version=xxx' to specify spark-rapids-jni dependency's version. -MVN="mvn ${MVN_OPT}" +MVN="mvn -Dmaven.wagon.http.retryHandler.count=3 -DretryFailedDeploymentCount=3 ${MVN_OPT}" TOOL_PL=${TOOL_PL:-"tools"} DIST_PL="dist" diff --git a/jenkins/spark-tests.sh b/jenkins/spark-tests.sh index 21446e93950..002d19de599 100755 --- a/jenkins/spark-tests.sh +++ b/jenkins/spark-tests.sh @@ -22,7 +22,7 @@ nvidia-smi . jenkins/version-def.sh ARTF_ROOT="$WORKSPACE/jars" -MVN_GET_CMD="mvn org.apache.maven.plugins:maven-dependency-plugin:2.8:get -B \ +MVN_GET_CMD="mvn -Dmaven.wagon.http.retryHandler.count=3 org.apache.maven.plugins:maven-dependency-plugin:2.8:get -B \ -Dmaven.repo.local=$WORKSPACE/.m2 \ $MVN_URM_MIRROR -Ddest=$ARTF_ROOT" From 51d1969dd22547c66ddd4be7a410389430ae1da2 Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Mon, 15 Aug 2022 14:27:40 -0500 Subject: [PATCH 025/190] [Bug] Add Expr OverflowInTableInsert to fix AnsiCastOpSuite (#6256) * Add Expr OverflowInTableInsert to fix AnsiCastSuite Signed-off-by: Ahmed Hussein (amahussein) fixes #6165 --- pom.xml | 2 + .../rapids/shims/Spark331PlusShims.scala | 46 +++++ .../GpuCheckOverflowInTableInsert.scala | 59 ++++++ .../com/nvidia/spark/rapids/SparkShims.scala | 2 +- .../com/nvidia/spark/rapids/SparkShims.scala | 2 +- .../com/nvidia/spark/rapids/GpuCast.scala | 9 +- .../nvidia/spark/rapids/AnsiCastOpSuite.scala | 174 ++++++++++++------ .../com/nvidia/spark/rapids/CastOpSuite.scala | 2 +- 8 files changed, 231 insertions(+), 65 deletions(-) create mode 100644 sql-plugin/src/main/331+/scala/com/nvidia/spark/rapids/shims/Spark331PlusShims.scala create mode 100644 sql-plugin/src/main/331+/scala/org/apache/spark/sql/rapids/GpuCheckOverflowInTableInsert.scala diff --git a/pom.xml b/pom.xml index e78c8ba1fc6..ecccf20e3a3 100644 --- a/pom.xml +++ b/pom.xml @@ -829,6 +829,7 @@ ${project.basedir}/src/main/331/scala + ${project.basedir}/src/main/331+/scala ${project.basedir}/src/main/311+-nondb/scala ${project.basedir}/src/main/311until340-all/scala ${project.basedir}/src/main/320+/java @@ -895,6 +896,7 @@ ${project.basedir}/src/main/340/scala + ${project.basedir}/src/main/331+/scala ${project.basedir}/src/main/311+-nondb/scala ${project.basedir}/src/main/320+/java ${project.basedir}/src/main/320+/scala diff --git a/sql-plugin/src/main/331+/scala/com/nvidia/spark/rapids/shims/Spark331PlusShims.scala b/sql-plugin/src/main/331+/scala/com/nvidia/spark/rapids/shims/Spark331PlusShims.scala new file mode 100644 index 00000000000..868fe0dbddc --- /dev/null +++ b/sql-plugin/src/main/331+/scala/com/nvidia/spark/rapids/shims/Spark331PlusShims.scala @@ -0,0 +1,46 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids.{ExprChecks, ExprRule, GpuCast, GpuExpression, GpuOverrides, TypeSig, UnaryExprMeta} + +import org.apache.spark.sql.catalyst.expressions.{CheckOverflowInTableInsert, Expression} +import org.apache.spark.sql.rapids.GpuCheckOverflowInTableInsert + +trait Spark331PlusShims extends Spark330PlusShims { + override def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = { + val map: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Seq( + // Add expression CheckOverflowInTableInsert starting Spark-3.3.1+ + // Accepts all types as input as the child Cast does the type checking and the calculations. + GpuOverrides.expr[CheckOverflowInTableInsert]( + "Casting a numeric value as another numeric type in store assignment", + ExprChecks.unaryProjectInputMatchesOutput( + TypeSig.all, + TypeSig.all), + (t, conf, p, r) => new UnaryExprMeta[CheckOverflowInTableInsert](t, conf, p, r) { + override def convertToGpu(child: Expression): GpuExpression = { + child match { + case c: GpuCast => GpuCheckOverflowInTableInsert(c, t.columnName) + case _ => + throw new IllegalStateException("Expression child is not of Type GpuCast") + } + } + }) + ).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap + super.getExprs ++ map + } +} diff --git a/sql-plugin/src/main/331+/scala/org/apache/spark/sql/rapids/GpuCheckOverflowInTableInsert.scala b/sql-plugin/src/main/331+/scala/org/apache/spark/sql/rapids/GpuCheckOverflowInTableInsert.scala new file mode 100644 index 00000000000..b80edfb8d2e --- /dev/null +++ b/sql-plugin/src/main/331+/scala/org/apache/spark/sql/rapids/GpuCheckOverflowInTableInsert.scala @@ -0,0 +1,59 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.rapids + +import com.nvidia.spark.rapids.{GpuCast, GpuExpression} +import com.nvidia.spark.rapids.shims.ShimUnaryExpression + +import org.apache.spark.SparkArithmeticException +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.vectorized.ColumnarBatch + +/** + * Handle the new Expression added in Spark-3.3.1+. + * Casting a numeric value as another numeric type in store assignment. It captures the arithmetic + * exception thrown by Cast and shows a relevant error message. + * The current implementation in Spark creates a child of type Cast with AnsiEnabled set to true. + * + * The GPU equivalent columnar evaluation is a delegation to the child's columnar evaluation which + * is a GpuCast. In order to match Spark's Exception, we need to catch the Exception passing it to + * QueryExecutionError. + * The calculation of the sideEffect is delegated to the child Expression "GpuCast". + */ +case class GpuCheckOverflowInTableInsert(child: GpuCast, columnName: String) + extends ShimUnaryExpression with GpuExpression { + + override def dataType: DataType = child.dataType + + override def columnarEval(batch: ColumnarBatch): Any = { + try { + child.columnarEval(batch) + } catch { + // map SparkArithmeticException to SparkArithmeticException("CAST_OVERFLOW_IN_TABLE_INSERT") + case _: SparkArithmeticException => + throw QueryExecutionErrors.castingCauseOverflowErrorInTableInsert( + child.child.dataType, + dataType, + columnName) + } + } + + override def toString: String = child.toString + + override def sql: String = child.sql +} diff --git a/sql-plugin/src/main/331/scala/com/nvidia/spark/rapids/SparkShims.scala b/sql-plugin/src/main/331/scala/com/nvidia/spark/rapids/SparkShims.scala index c384a3da842..2ced31cd978 100644 --- a/sql-plugin/src/main/331/scala/com/nvidia/spark/rapids/SparkShims.scala +++ b/sql-plugin/src/main/331/scala/com/nvidia/spark/rapids/SparkShims.scala @@ -18,6 +18,6 @@ package com.nvidia.spark.rapids.shims import com.nvidia.spark.rapids._ -object SparkShimImpl extends Spark330PlusShims with Spark320until340Shims { +object SparkShimImpl extends Spark331PlusShims with Spark320until340Shims { override def getSparkShimVersion: ShimVersion = ShimLoader.getShimVersion } diff --git a/sql-plugin/src/main/340/scala/com/nvidia/spark/rapids/SparkShims.scala b/sql-plugin/src/main/340/scala/com/nvidia/spark/rapids/SparkShims.scala index b472234b181..b7ac29ab293 100644 --- a/sql-plugin/src/main/340/scala/com/nvidia/spark/rapids/SparkShims.scala +++ b/sql-plugin/src/main/340/scala/com/nvidia/spark/rapids/SparkShims.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.physical.SinglePartition import org.apache.spark.sql.execution.{CollectLimitExec, GlobalLimitExec, SparkPlan} import org.apache.spark.sql.execution.exchange.ENSURE_REQUIREMENTS -object SparkShimImpl extends Spark330PlusShims { +object SparkShimImpl extends Spark331PlusShims { override def getSparkShimVersion: ShimVersion = ShimLoader.getShimVersion private val shimExecs: Map[Class[_ <: SparkPlan], ExecRule[_ <: SparkPlan]] = Seq( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala index efadb5b5f26..ca5d62a1f77 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.{Cast, CastBase, Expression, Nu import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_SECOND import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.GpuToTimestamp.replaceSpecialDates +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.types._ /** Meta-data for cast and ansi_cast. */ @@ -389,7 +390,7 @@ object GpuCast extends Arm { withResource(input.max()) { maxInput => if (minInput.isValid && minInput.getBigDecimal().compareTo(min) == -1 || maxInput.isValid && maxInput.getBigDecimal().compareTo(max) == 1) { - throw new ArithmeticException(GpuCast.INVALID_INPUT_MESSAGE) + throw new ArithmeticException(GpuCast.OVERFLOW_MESSAGE) } } } @@ -634,13 +635,13 @@ object GpuCast extends Arm { maxValue: => Scalar, inclusiveMin: Boolean = true, inclusiveMax: Boolean = true, - errorMsg:String = GpuCast.INVALID_INPUT_MESSAGE): Unit = { + errorMsg:String = GpuCast.OVERFLOW_MESSAGE): Unit = { def throwIfAny(cv: ColumnView): Unit = { withResource(cv) { cv => withResource(cv.any()) { isAny => if (isAny.isValid && isAny.getBoolean) { - throw new IllegalStateException(errorMsg) + throw RapidsErrorUtils.arithmeticOverflowError(errorMsg) } } } @@ -1514,7 +1515,7 @@ object GpuCast extends Arm { if (ansiMode) { withResource(outOfBounds.any()) { isAny => if (isAny.isValid && isAny.getBoolean) { - throw new IllegalStateException(GpuCast.INVALID_INPUT_MESSAGE) + throw RapidsErrorUtils.arithmeticOverflowError(GpuCast.OVERFLOW_MESSAGE) } } input.copyToColumnVector() diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/AnsiCastOpSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/AnsiCastOpSuite.scala index bd39a17fc24..3134857cb04 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/AnsiCastOpSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/AnsiCastOpSuite.scala @@ -25,7 +25,7 @@ import com.nvidia.spark.rapids.shims.SparkShimImpl import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.{DataFrame, Row, SparkSession} -import org.apache.spark.sql.catalyst.expressions.{Alias, CastBase, Expression, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression} import org.apache.spark.sql.execution.ProjectExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -148,23 +148,26 @@ class AnsiCastOpSuite extends GpuExpressionTestSuite { } testCastFailsForBadInputs("ansi_cast overflow decimals to bytes", - generateOutOfRangeDecimalDF(Byte.MinValue, Byte.MaxValue, 10, 0, Byte.MaxValue + 1)) { + generateOutOfRangeDecimalDF(Byte.MinValue, Byte.MaxValue, 10, 0, Byte.MaxValue + 1), + msg = GpuCast.OVERFLOW_MESSAGE) { frame => testCastTo(DataTypes.ByteType)(frame) } testCastFailsForBadInputs("ansi_cast overflow decimals to shorts", - generateOutOfRangeDecimalDF(Short.MinValue, Short.MaxValue, 10, 0, Short.MaxValue + 1)) { + generateOutOfRangeDecimalDF(Short.MinValue, Short.MaxValue, 10, 0, Short.MaxValue + 1), + msg = GpuCast.OVERFLOW_MESSAGE) { frame => testCastTo(DataTypes.ShortType)(frame) } testCastFailsForBadInputs("ansi_cast overflow decimals to Ints", - generateOutOfRangeDecimalDF(Int.MinValue, Int.MaxValue, 10, 0, Int.MaxValue.toLong + 1)) { + generateOutOfRangeDecimalDF(Int.MinValue, Int.MaxValue, 10, 0, Int.MaxValue.toLong + 1), + msg = GpuCast.OVERFLOW_MESSAGE) { frame => testCastTo(DataTypes.IntegerType)(frame) } testCastFailsForBadInputs("ansi_cast overflow decimals to longs", generateOutOfRangeDecimalDF(Long.MinValue, Long.MaxValue, 18, -3, - BigDecimal("999999999999999999000"))) { + BigDecimal("999999999999999999000")), msg = GpuCast.OVERFLOW_MESSAGE) { frame => testCastTo(DataTypes.LongType)(frame) } @@ -474,7 +477,7 @@ class AnsiCastOpSuite extends GpuExpressionTestSuite { private def castToStringExpectedFun[T]: T => Option[String] = (d: T) => Some(String.valueOf(d)) private def testCastToString[T](dataType: DataType, ansiMode: Boolean, - comparisonFunc: Option[(String, String) => Boolean] = None) { + comparisonFunc: Option[(String, String) => Boolean] = None): Unit = { // AnsiCast is merged into Cast from Spark 3.4.0. // Use reflection to avoid shims. val key = Class.forName { @@ -584,73 +587,63 @@ class AnsiCastOpSuite extends GpuExpressionTestSuite { frame => doTableInsert(frame, HIVE_LONG_SQL_TYPE) } - testSparkResultsAreEqual("Write longs to int (values within range)", intsAsLongs, - sparkConf, assumeCondition = before3_3_1) { + testSparkResultsAreEqual("Write longs to int (values within range)", intsAsLongs, sparkConf) { frame => doTableInsert(frame, HIVE_INT_SQL_TYPE) } - testSparkResultsAreEqual("Write longs to short (values within range)", shortsAsLongs, - sparkConf, assumeCondition = before3_3_1) { + testSparkResultsAreEqual("Write longs to short (values within range)", shortsAsLongs, sparkConf) { frame => doTableInsert(frame, HIVE_SHORT_SQL_TYPE) } - testSparkResultsAreEqual("Write longs to byte (values within range)", bytesAsLongs, - sparkConf, assumeCondition = before3_3_1) { + testSparkResultsAreEqual("Write longs to byte (values within range)", bytesAsLongs, sparkConf) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } - testSparkResultsAreEqual("Write ints to short (values within range)", shortsAsInts, - sparkConf, assumeCondition = before3_3_1) { + testSparkResultsAreEqual("Write ints to short (values within range)", shortsAsInts, sparkConf) { frame => doTableInsert(frame, HIVE_SHORT_SQL_TYPE) } - testSparkResultsAreEqual("Write ints to byte (values within range)", bytesAsInts, - sparkConf, assumeCondition = before3_3_1) { + testSparkResultsAreEqual("Write ints to byte (values within range)", bytesAsInts, sparkConf) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } - testSparkResultsAreEqual("Write shorts to byte (values within range)", bytesAsShorts, - sparkConf, assumeCondition = before3_3_1) { + testSparkResultsAreEqual("Write shorts to byte (values within range)", bytesAsShorts, sparkConf) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } - testSparkResultsAreEqual("Write floats to long (values within range)", longsAsFloats, - sparkConf, assumeCondition = before3_3_1) { + testSparkResultsAreEqual("Write floats to long (values within range)", longsAsFloats, sparkConf) { frame => doTableInsert(frame, HIVE_LONG_SQL_TYPE) } - testSparkResultsAreEqual("Write floats to int (values within range)", intsAsFloats, - sparkConf, assumeCondition = before3_3_1) { + testSparkResultsAreEqual("Write floats to int (values within range)", intsAsFloats, sparkConf) { frame => doTableInsert(frame, HIVE_INT_SQL_TYPE) } testSparkResultsAreEqual("Write floats to short (values within range)", shortsAsFloats, - sparkConf, assumeCondition = before3_3_1) { + sparkConf) { frame => doTableInsert(frame, HIVE_SHORT_SQL_TYPE) } - testSparkResultsAreEqual("Write floats to byte (values within range)", bytesAsFloats, - sparkConf, assumeCondition = before3_3_1) { + testSparkResultsAreEqual("Write floats to byte (values within range)", bytesAsFloats, sparkConf) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } testSparkResultsAreEqual("Write doubles to long (values within range)", longsAsDoubles, - sparkConf, assumeCondition = before3_3_1) { + sparkConf) { frame => doTableInsert(frame, HIVE_LONG_SQL_TYPE) } - testSparkResultsAreEqual("Write doubles to int (values within range)", intsAsDoubles, - sparkConf, assumeCondition = before3_3_1) { + testSparkResultsAreEqual("Write doubles to int (values within range)", intsAsDoubles, sparkConf) { frame => doTableInsert(frame, HIVE_LONG_SQL_TYPE) } - testSparkResultsAreEqual("Write doubles to short (values within range)", - shortsAsDoubles, sparkConf, assumeCondition = before3_3_1) { + testSparkResultsAreEqual("Write doubles to short (values within range)", shortsAsDoubles, + sparkConf) { frame => doTableInsert(frame, HIVE_LONG_SQL_TYPE) } testSparkResultsAreEqual("Write doubles to byte (values within range)", bytesAsDoubles, - sparkConf, assumeCondition = before3_3_1) { + sparkConf) { frame => doTableInsert(frame, HIVE_LONG_SQL_TYPE) } @@ -659,72 +652,72 @@ class AnsiCastOpSuite extends GpuExpressionTestSuite { /////////////////////////////////////////////////////////////////////////// testCastFailsForBadInputs("Detect overflow from long to int", testLongs, sparkConf, - assumeCondition = before3_3_1) { + msg = getExMsgForOverflowInTableInsert) { frame => doTableInsert(frame, HIVE_INT_SQL_TYPE) } testCastFailsForBadInputs("Detect overflow from long to short", testLongs, sparkConf, - assumeCondition = before3_3_1) { + msg = getExMsgForOverflowInTableInsert) { frame => doTableInsert(frame, HIVE_SHORT_SQL_TYPE) } testCastFailsForBadInputs("Detect overflow from long to byte", testLongs, sparkConf, - assumeCondition = before3_3_1) { + msg = getExMsgForOverflowInTableInsert) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } testCastFailsForBadInputs("Detect overflow from int to short", testInts, sparkConf, - assumeCondition = before3_3_1) { + msg = getExMsgForOverflowInTableInsert) { frame => doTableInsert(frame, HIVE_SHORT_SQL_TYPE) } testCastFailsForBadInputs("Detect overflow from int to byte", testInts, sparkConf, - assumeCondition = before3_3_1) { + msg = getExMsgForOverflowInTableInsert) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } testCastFailsForBadInputs("Detect overflow from short to byte", testShorts, sparkConf, - assumeCondition = before3_3_1) { + msg = getExMsgForOverflowInTableInsert) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } testCastFailsForBadInputs("Detect overflow from float to long", testFloats, sparkConf, - assumeCondition = before3_3_1) { + msg = getExMsgForOverflowInTableInsert) { frame => doTableInsert(frame, HIVE_INT_SQL_TYPE) } testCastFailsForBadInputs("Detect overflow from float to int", testFloats, sparkConf, - assumeCondition = before3_3_1) { + msg = getExMsgForOverflowInTableInsert) { frame => doTableInsert(frame, HIVE_SHORT_SQL_TYPE) } - testCastFailsForBadInputs("Detect overflow from float to short", testFloats, - sparkConf, assumeCondition = before3_3_1) { + testCastFailsForBadInputs("Detect overflow from float to short", testFloats, sparkConf, + msg = getExMsgForOverflowInTableInsert) { frame => doTableInsert(frame, HIVE_LONG_SQL_TYPE) } - testCastFailsForBadInputs("Detect overflow from float to byte", testFloats, - sparkConf, assumeCondition = before3_3_1) { + testCastFailsForBadInputs("Detect overflow from float to byte", testFloats, sparkConf, + msg = getExMsgForOverflowInTableInsert) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } testCastFailsForBadInputs("Detect overflow from double to long", testDoubles, - sparkConf, assumeCondition = before3_3_1) { + sparkConf, msg = getExMsgForOverflowInTableInsert) { frame => doTableInsert(frame, HIVE_LONG_SQL_TYPE) } testCastFailsForBadInputs("Detect overflow from double to int", testDoubles, - sparkConf, assumeCondition = before3_3_1) { + sparkConf, msg = getExMsgForOverflowInTableInsert) { frame => doTableInsert(frame, HIVE_INT_SQL_TYPE) } testCastFailsForBadInputs("Detect overflow from double to short", testDoubles, - sparkConf, assumeCondition = before3_3_1) { + sparkConf, msg = getExMsgForOverflowInTableInsert) { frame => doTableInsert(frame, HIVE_SHORT_SQL_TYPE) } testCastFailsForBadInputs("Detect overflow from double to byte", testDoubles, - sparkConf, assumeCondition = before3_3_1) { + sparkConf, msg = getExMsgForOverflowInTableInsert) { frame => doTableInsert(frame, HIVE_BYTE_SQL_TYPE) } @@ -820,14 +813,26 @@ class AnsiCastOpSuite extends GpuExpressionTestSuite { } } - private def assertContainsAnsiCast(df: DataFrame, expected: Int = 1): DataFrame = { val projections = SparkShimImpl.findOperators(df.queryExecution.executedPlan, { case _: ProjectExec | _: GpuProjectExec => true case _ => false }) - def isAnsiCast(c: CastBase): Boolean = { + def isSparkCastExpression(c: Expression): Boolean = { + val sparkClzz = Class.forName { + if (cmpSparkVersion(3, 4, 0) < 0) { + "org.apache.spark.sql.catalyst.expressions.AnsiCast" + } else { + "org.apache.spark.sql.catalyst.expressions.Cast" + } + } + c.getClass == sparkClzz + } + + def isAnsiCast(c: Expression): Boolean = { + // prior to Spark 3.4.0 we could use CastBase as argument type, but starting 3.4.0 the type is + // the case class Cast. // prior to Spark 3.3.0 we could use toString to see if the name of // the cast was "cast" or "ansi_cast" but now the name is always "cast" // so we need to use reflection to access the protected field "ansiEnabled" @@ -836,18 +841,52 @@ class AnsiCastOpSuite extends GpuExpressionTestSuite { m.getBoolean(c) } - val count = projections.map { - case p: ProjectExec => p.projectList.count { - case c: CastBase => isAnsiCast(c) - case Alias(c: CastBase, _) => isAnsiCast(c) - case _ => false + def isAnsiCastInTableInsert(expr: Expression, cpuSession: Boolean): Boolean = { + // Use reflection to avoid using shims for Spark-3.3.1+ in the form of: + // CPU: Alias(CheckOverflowInTableInsert(Cast, _), _) + // GPU: GpuAlias(GpuCheckOverflowInTableInsert(GpuCast, _), _) + val exprKlassName = if (cpuSession) { + "org.apache.spark.sql.catalyst.expressions.CheckOverflowInTableInsert" + } else { + "org.apache.spark.sql.rapids.GpuCheckOverflowInTableInsert" } - case p: GpuProjectExec => p.projectList.count { - case c: GpuCast => c.ansiMode - case GpuAlias(c: GpuCast, _) => c.ansiMode + if (expr.getClass.getName.equals(exprKlassName)) { + val exprClzz = Class.forName(exprKlassName) + val childField = exprClzz.getDeclaredField("child") + childField.setAccessible(true) + childField.get(expr) match { + case castExpr: Expression if cpuSession && isSparkCastExpression(castExpr) => + isAnsiCast(castExpr) + case castExpr: GpuCast => castExpr.ansiMode case _ => false } - case _ => 0 + } else { + false + } + } + + val count = projections.map { + case p: ProjectExec => p.projectList.count { + case c: Expression if isSparkCastExpression(c) => isAnsiCast(c) + case expr: Alias => + expr.child match { + case c: Expression if isSparkCastExpression(c) => isAnsiCast(c) + // To support the conversion of CheckOverflowInTableInsert starting Spark-3.3.1+ + case _ => isAnsiCastInTableInsert(expr.child, cpuSession = true) + } + case _ => false + } + case p: GpuProjectExec => p.projectList.count { + case c: GpuCast => c.ansiMode + case c: GpuAlias => + c.child match { + case exp: GpuCast => exp.ansiMode + // To support the conversion of CheckOverflowInTableInsert starting Spark-3.3.1+ + case _ => isAnsiCastInTableInsert(c.child, cpuSession = false) + } + case _ => false + } + case _ => 0 }.sum if (count != expected) { @@ -857,6 +896,25 @@ class AnsiCastOpSuite extends GpuExpressionTestSuite { df } + /** + * Returns the expected pattern in the Exception thrown by an arithmetic overflow during table + * insertion. + * The GPU throws an Spark/ArithmeticException with GpuCast.OVERFLOW_MESSAGE. + * Starting 3.3.1, case during insert throws a new SparkArithmeticException class. + * + * @return the exception message from overflow exception during table insertion, depending on the + * Spark Error Shims loaded. + */ + private def getExMsgForOverflowInTableInsert: String = { + if (cmpSparkVersion(3, 3, 1) < 0) { + // Prior to Spark-3.3.1, GPU throws an ArithmeticException with overflowMessage + GpuCast.OVERFLOW_MESSAGE + } else { + // spark 3.3.1+ shows a different error message for overflow during table insert + "due to an overflow." + } + } + private def testBools = testData(DataTypes.BooleanType)(_) private def testBytes = testData(DataTypes.ByteType)(_) private def testShorts = testData(DataTypes.ShortType)(_) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala index 1efdaefd5cd..3e6ca18eaf1 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala @@ -835,7 +835,7 @@ class CastOpSuite extends GpuExpressionTestSuite { intercept[org.apache.spark.SparkException] { nonOverflowCase(dataType, generator, precision, scale) }, - GpuCast.INVALID_INPUT_MESSAGE) + GpuCast.OVERFLOW_MESSAGE) ) // Compare gpu results with cpu ones when AnsiMode is off (most of them should be null) testCastToDecimal(dataType, From a7a76eb48d091276926e4526b2d9958baa1b0bf8 Mon Sep 17 00:00:00 2001 From: Remzi Yang <59198230+HaoYang670@users.noreply.github.com> Date: Tue, 16 Aug 2022 04:18:54 +0800 Subject: [PATCH 026/190] Add Nan handling for `GpuArrayMax` (#6316) * add Nan handling Signed-off-by: remzi <13716567376yh@gmail.com> * add docs Signed-off-by: remzi <13716567376yh@gmail.com> Signed-off-by: remzi <13716567376yh@gmail.com> --- .../src/main/python/array_test.py | 7 ++- .../nvidia/spark/rapids/GpuOverrides.scala | 7 +-- .../sql/rapids/collectionOperations.scala | 52 ++++++++++++++++++- 3 files changed, 58 insertions(+), 8 deletions(-) diff --git a/integration_tests/src/main/python/array_test.py b/integration_tests/src/main/python/array_test.py index c6d7178558f..c427366638f 100644 --- a/integration_tests/src/main/python/array_test.py +++ b/integration_tests/src/main/python/array_test.py @@ -325,6 +325,11 @@ def test_array_concat_decimal(data_gen): 'concat(a, a)')), conf=no_nans_conf) +@pytest.mark.parametrize('data_gen', [float_gen, double_gen], ids=idfn) +def test_array_max_with_nans(data_gen): + assert_gpu_and_cpu_are_equal_collect( + lambda spark : unary_op_df(spark, ArrayGen(data_gen)).selectExpr( + 'array_max(a)')) @pytest.mark.parametrize('data_gen', array_min_max_gens_no_nan, ids=idfn) def test_array_max(data_gen): @@ -333,7 +338,7 @@ def test_array_max(data_gen): 'array_max(a)'), conf=no_nans_conf) -@pytest.mark.parametrize('data_gen', [ArrayGen(int_gen, all_null=True)], ids=idfn) +@pytest.mark.parametrize('data_gen', [ArrayGen(gen, all_null=True) for gen in [int_gen, float_gen, double_gen]], ids=idfn) def test_array_max_all_nulls(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 649d1e65534..177604c693d 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -2754,14 +2754,9 @@ object GpuOverrides extends Logging { ExprChecks.unaryProject( TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL, TypeSig.orderable, - TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL) - .withPsNote(Seq(TypeEnum.DOUBLE, TypeEnum.FLOAT), nanAggPsNote), + TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL), TypeSig.ARRAY.nested(TypeSig.orderable)), (in, conf, p, r) => new UnaryExprMeta[ArrayMax](in, conf, p, r) { - override def tagExprForGpu(): Unit = { - checkAndTagFloatNanAgg("Max", in.dataType, conf, this) - } - override def convertToGpu(child: Expression): GpuExpression = GpuArrayMax(child) }), diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala index aa5bc939179..4618fe7db80 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala @@ -440,7 +440,19 @@ case class GpuArrayMin(child: Expression) extends GpuUnaryExpression with Implic input.getBase.listReduce(SegmentedReductionAggregation.min()) } -case class GpuArrayMax(child: Expression) extends GpuUnaryExpression with ImplicitCastInputTypes { +object GpuArrayMax { + def apply(child: Expression): GpuArrayMax = { + child.dataType match { + case ArrayType(FloatType | DoubleType, _) => GpuFloatArrayMax(child) + case ArrayType(_, _) => GpuBasicArrayMax(child) + case _ => throw new IllegalStateException(s"array_max accepts only arrays.") + } + } +} + +abstract class GpuArrayMax(child: Expression) extends GpuUnaryExpression + with ImplicitCastInputTypes + with Serializable{ override def nullable: Boolean = true @@ -457,6 +469,44 @@ case class GpuArrayMax(child: Expression) extends GpuUnaryExpression with Implic input.getBase.listReduce(SegmentedReductionAggregation.max()) } +/** ArrayMax without `NaN` handling */ +case class GpuBasicArrayMax(child: Expression) extends GpuArrayMax(child) + +/** ArrayMax for FloatType and DoubleType to handle `NaN`s. + * + * In Spark, `Nan` is the max float value, however in cuDF, the calculation + * involving `Nan` is undefined. + * We design a workaround method here to match the Spark's behaviour. + * The high level idea is that, we firstly check if each array contains `Nan`. + * If it is, the max value is `Nan`, else we use the cuDF kernel to + * calculate the max value. + */ +case class GpuFloatArrayMax(child: Expression) extends GpuArrayMax(child){ + @transient override lazy val dataType: DataType = child.dataType match { + case ArrayType(FloatType, _) => FloatType + case ArrayType(DoubleType, _) => DoubleType + case _ => throw new IllegalStateException( + s"GpuFloatArrayMax accepts only float array and double array." + ) + } + + protected def getNanSalar: Scalar = dataType match { + case FloatType => Scalar.fromFloat(Float.NaN) + case DoubleType => Scalar.fromDouble(Double.NaN) + case t => throw new IllegalStateException(s"dataType $t is not FloatType or DoubleType") + } + + override protected def doColumnar(input: GpuColumnVector): cudf.ColumnVector = { + withResource(getNanSalar){nan => + withResource(input.getBase().listContains(nan)){hasNan => + withResource(input.getBase().listReduce(SegmentedReductionAggregation.max())) {max => + hasNan.ifElse(nan, max) + } + } + } + } +} + case class GpuArrayRepeat(left: Expression, right: Expression) extends GpuBinaryExpression { override def dataType: DataType = ArrayType(left.dataType, left.nullable) From c79f4a9344156f7a0bb965b4baa06a063f2a6a3c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 15 Aug 2022 17:55:47 -0600 Subject: [PATCH 027/190] Re-implement cast timestamp to string and add more tests (#6311) * re-implement cast timestamp to string and add more tests Signed-off-by: Andy Grove * Update tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala Signed-off-by: Andy Grove Co-authored-by: Nghia Truong --- .../src/main/python/cast_test.py | 6 ++++ .../com/nvidia/spark/rapids/GpuCast.scala | 28 ++++++++++++++----- .../com/nvidia/spark/rapids/CastOpSuite.scala | 3 ++ 3 files changed, 30 insertions(+), 7 deletions(-) diff --git a/integration_tests/src/main/python/cast_test.py b/integration_tests/src/main/python/cast_test.py index 1ae36c0d230..d5daf22b04c 100644 --- a/integration_tests/src/main/python/cast_test.py +++ b/integration_tests/src/main/python/cast_test.py @@ -22,6 +22,7 @@ from spark_init_internal import spark_version from datetime import datetime import math +import datetime _decimal_gen_36_5 = DecimalGen(precision=36, scale=5) @@ -473,6 +474,11 @@ def test_cast_timestamp_to_numeric_non_ansi(): .selectExpr("cast(a as byte)", "cast(a as short)", "cast(a as int)", "cast(a as long)", "cast(a as float)", "cast(a as double)")) +def test_cast_timestamp_to_string(): + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, timestamp_gen) + .selectExpr("cast(a as string)")) + @pytest.mark.skipif(is_before_spark_330(), reason='DayTimeInterval is not supported before Pyspark 3.3.0') def test_cast_day_time_interval_to_string(): _assert_cast_to_string_equal(DayTimeIntervalGen(start_field='day', end_field='day', special_cases=[MIN_DAY_TIME_INTERVAL, MAX_DAY_TIME_INTERVAL, timedelta(seconds=0)]), {}) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala index ca5d62a1f77..1e52ff60de2 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala @@ -172,12 +172,6 @@ object GpuCast extends Arm { "\\A\\d{4}\\-\\d{1,2}\\-\\d{1,2}[ T]?(\\d{1,2}:\\d{1,2}:\\d{1,2}\\.\\d{6}Z)\\Z" private val TIMESTAMP_REGEX_NO_DATE = "\\A[T]?(\\d{1,2}:\\d{1,2}:\\d{1,2}\\.\\d{6}Z)\\Z" - /** - * Regex to match timestamps with or without trailing zeros. - */ - private val TIMESTAMP_TRUNCATE_REGEX = "^([0-9]{4}-[0-9]{2}-[0-9]{2} " + - "[0-9]{2}:[0-9]{2}:[0-9]{2})" + - "(.[1-9]*(?:0)?[1-9]+)?(.0*[1-9]+)?(?:.0*)?$" private val BIG_DECIMAL_LONG_MIN = BigDecimal(Long.MinValue) private val BIG_DECIMAL_LONG_MAX = BigDecimal(Long.MaxValue) @@ -708,9 +702,29 @@ object GpuCast extends Arm { } private def castTimestampToString(input: ColumnView): ColumnVector = { + // the complexity in this function is due to Spark's rules for truncating + // the fractional part of the timestamp string. Any trailing decimal place + // or zeroes should be truncated + // ".000000" -> "" + // ".000100" -> ".0001" + // ".100000" -> ".1" + // ".101010" -> ".10101" withResource(input.castTo(DType.TIMESTAMP_MICROSECONDS)) { micros => withResource(micros.asStrings("%Y-%m-%d %H:%M:%S.%6f")) { cv => - cv.stringReplaceWithBackrefs(GpuCast.TIMESTAMP_TRUNCATE_REGEX, "\\1\\2\\3") + // to keep code complexity down, do a first pass that + // removes ".000000" using simple string replace + val firstPass = withResource(Scalar.fromString(".000000")) { search => + withResource(Scalar.fromString("")) { replace => + cv.stringReplace(search, replace) + } + } + // now remove trailing zeroes from any remaining fractional parts + // the first group captures everything between + // the decimal point and the last non-zero digit + // the second group (non-capture) covers the remaining zeroes + withResource(firstPass) { _ => + firstPass.stringReplaceWithBackrefs("(\\.[0-9]*[1-9]+)(?:0+)?$", "\\1") + } } } } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala index 3e6ca18eaf1..a8d5d483c6d 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala @@ -1442,6 +1442,9 @@ object CastOpSuite { "1920-12-31T11:59:59.999", "1969-12-31T23:59:59.999", "1969-12-31T23:59:59.999999", + "1969-12-31T23:59:59.001700", + "1969-12-31T23:59:59.001070", + "1969-12-31T23:59:59.010701", "1970-01-01T00:00:00.000", "1970-01-01T00:00:00.999", "1970-01-01T00:00:00.999111", From f2b6bd0f3724256831ace0041a64886824686553 Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Tue, 16 Aug 2022 10:46:42 +0800 Subject: [PATCH 028/190] [BUG] Fix IT discrepancy which depending on TEST_PARALLEL (#6044) * Update IT: use spark.jars configuration instead of spark.executor.extraClassPath and spark.driver.extraClassPath Signed-off-by: Chong Gao --- integration_tests/run_pyspark_from_build.sh | 28 +++++++++++++++++---- 1 file changed, 23 insertions(+), 5 deletions(-) diff --git a/integration_tests/run_pyspark_from_build.sh b/integration_tests/run_pyspark_from_build.sh index 7c1a98a6d4b..26a9861f225 100755 --- a/integration_tests/run_pyspark_from_build.sh +++ b/integration_tests/run_pyspark_from_build.sh @@ -87,8 +87,12 @@ else AVRO_JARS="" fi - # Only 3 jars: dist.jar integration-test.jar avro.jar - ALL_JARS="$PLUGIN_JARS $TEST_JARS $AVRO_JARS $PARQUET_HADOOP_TESTS" + # ALL_JARS includes dist.jar integration-test.jar avro.jar parquet.jar if they exist + # Remove non-existing paths and canonicalize the paths including get rid of links and `..` + ALL_JARS=$(readlink -e $PLUGIN_JARS $TEST_JARS $AVRO_JARS $PARQUET_HADOOP_TESTS || true) + # `:` separated jars + ALL_JARS="${ALL_JARS//$'\n'/:}" + echo "AND PLUGIN JARS: $ALL_JARS" if [[ "${TEST}" != "" ]]; then @@ -195,8 +199,15 @@ else SPARK_TASK_MAXFAILURES=${SPARK_TASK_MAXFAILURES:-1} [[ "$VERSION_STRING" < "3.1.1" ]] && SPARK_TASK_MAXFAILURES=4 - export PYSP_TEST_spark_driver_extraClassPath="${ALL_JARS// /:}" - export PYSP_TEST_spark_executor_extraClassPath="${ALL_JARS// /:}" + if [[ "${PYSP_TEST_spark_shuffle_manager}" =~ "RapidsShuffleManager" ]]; then + # If specified shuffle manager, set `extraClassPath` due to issue https://github.com/NVIDIA/spark-rapids/issues/5796 + # Remove this line if the issue is fixed + export PYSP_TEST_spark_driver_extraClassPath="${ALL_JARS}" + export PYSP_TEST_spark_executor_extraClassPath="${ALL_JARS}" + else + export PYSP_TEST_spark_jars="${ALL_JARS//:/,}" + fi + export PYSP_TEST_spark_driver_extraJavaOptions="-ea -Duser.timezone=UTC $COVERAGE_SUBMIT_FLAGS" export PYSP_TEST_spark_executor_extraJavaOptions='-ea -Duser.timezone=UTC' export PYSP_TEST_spark_ui_showConsoleProgress='false' @@ -254,7 +265,14 @@ else else # We set the GPU memory size to be a constant value even if only running with a parallelism of 1 # because it helps us have consistent test runs. - exec "$SPARK_HOME"/bin/spark-submit --jars "${ALL_JARS// /,}" \ + if [[ -n "$PYSP_TEST_spark_jars" ]]; then + # `spark.jars` is the same as `--jars`, e.g.: --jars a.jar,b.jar... + jarOpts=(--conf spark.jars="${PYSP_TEST_spark_jars}") + elif [[ -n "$PYSP_TEST_spark_driver_extraClassPath" ]]; then + jarOpts=(--driver-class-path "${PYSP_TEST_spark_driver_extraClassPath}") + fi + + exec "$SPARK_HOME"/bin/spark-submit "${jarOpts[@]}" \ --driver-java-options "$PYSP_TEST_spark_driver_extraJavaOptions" \ $SPARK_SUBMIT_FLAGS \ --conf 'spark.rapids.memory.gpu.allocSize='"$PYSP_TEST_spark_rapids_memory_gpu_allocSize" \ From 65baba1f864162e5ec7e7215e6c0c60571a25863 Mon Sep 17 00:00:00 2001 From: Peixin Date: Tue, 16 Aug 2022 23:24:02 +0800 Subject: [PATCH 029/190] Make run_pyspark to report fail and error as default (#6333) Signed-off-by: Peixin Li Signed-off-by: Peixin Li --- integration_tests/run_pyspark_from_build.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/integration_tests/run_pyspark_from_build.sh b/integration_tests/run_pyspark_from_build.sh index 26a9861f225..c4d5993329b 100755 --- a/integration_tests/run_pyspark_from_build.sh +++ b/integration_tests/run_pyspark_from_build.sh @@ -181,8 +181,9 @@ else "$LOCAL_ROOTDIR" "$LOCAL_ROOTDIR"/src/main/python) + REPORT_CHARS=${REPORT_CHARS:="fE"} # default as (f)ailed, (E)rror TEST_COMMON_OPTS=(-v - -rfExXs + -r"$REPORT_CHARS" "$TEST_TAGS" --std_input_path="$INPUT_PATH"/src/test/resources --color=yes From 3b2986c775f3d10f917347c878d0bea9adc75653 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Tue, 16 Aug 2022 15:32:57 -0700 Subject: [PATCH 030/190] Use Spark's `Utils.getContextOrSparkClassLoader` to load Shims (#5646) Fixes #3851 Spark loads external [datasources][1] using `Utils.getContextOrSparkClassLoader` Trampoline to `Utils.getContextOrSparkClassLoader` to make our current code work with external sources, and to unblock JDK9+ [1]: https://github.com/apache/spark/blob/b63674ea5f746306a96ab8c39c23a230a6cb9566/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala#L652 Signed-off-by: Gera Shegalov --- CONTRIBUTING.md | 4 -- dist/unshimmed-common-from-spark311.txt | 1 + jenkins/databricks/test.sh | 3 +- jenkins/spark-premerge-build.sh | 8 --- jenkins/spark-tests.sh | 1 - .../com/nvidia/spark/rapids/RapidsConf.scala | 8 --- .../nvidia/spark/rapids/GpuOverrides.scala | 8 +-- .../com/nvidia/spark/rapids/RapidsConf.scala | 8 --- .../com/nvidia/spark/rapids/ShimLoader.scala | 66 ++----------------- .../execution/UnshimmedTrampolineUtil.scala | 23 +++++++ 10 files changed, 34 insertions(+), 96 deletions(-) create mode 100644 sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/UnshimmedTrampolineUtil.scala diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index ae81930b9d4..dd7c23fae09 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -105,10 +105,6 @@ You can build against different versions of the CUDA Toolkit by using qone of th We support JDK8 as our main JDK version. However, it's possible to build and run with more modern JDK versions as well. To this end set `JAVA_HOME` in the environment to your JDK root directory. -With JDK9+, you need to disable the default classloader manipulation option and set -spark.rapids.force.caller.classloader=false in your Spark application configuration. There are, however, -known issues with it, e.g. see #5513. - At the time of this writing, the most robust way to run the RAPIDS Accelerator is from a jar dedicated to a single Spark version. To this end please use a single shim and specify `-DallowConventionalDistJar=true` diff --git a/dist/unshimmed-common-from-spark311.txt b/dist/unshimmed-common-from-spark311.txt index 879283a2058..f9d391bd858 100644 --- a/dist/unshimmed-common-from-spark311.txt +++ b/dist/unshimmed-common-from-spark311.txt @@ -28,6 +28,7 @@ com/nvidia/spark/rapids/SparkShimVersion* com/nvidia/spark/rapids/SparkShims* com/nvidia/spark/udf/Plugin* org/apache/spark/sql/rapids/ProxyRapidsShuffleInternalManagerBase* +org/apache/spark/sql/rapids/execution/Unshimmed* org/apache/spark/sql/rapids/RapidsShuffleManagerLike* rapids/*.py rapids4spark-version-info.properties diff --git a/jenkins/databricks/test.sh b/jenkins/databricks/test.sh index d5eb0f1ee36..8cbbb4b526a 100755 --- a/jenkins/databricks/test.sh +++ b/jenkins/databricks/test.sh @@ -82,8 +82,7 @@ ICEBERG_SPARK_VER=$(echo $BASE_SPARK_VER | cut -d. -f1,2) # Classloader config is here to work around classloader issues with # --packages in distributed setups, should be fixed by # https://github.com/NVIDIA/spark-rapids/pull/5646 -ICEBERG_CONFS="--conf spark.rapids.force.caller.classloader=false \ - --packages org.apache.iceberg:iceberg-spark-runtime-${ICEBERG_SPARK_VER}_2.12:${ICEBERG_VERSION} \ +ICEBERG_CONFS="--packages org.apache.iceberg:iceberg-spark-runtime-${ICEBERG_SPARK_VER}_2.12:${ICEBERG_VERSION} \ --conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions \ --conf spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog \ --conf spark.sql.catalog.spark_catalog.type=hadoop \ diff --git a/jenkins/spark-premerge-build.sh b/jenkins/spark-premerge-build.sh index a2306c77814..256f5fdf7e4 100755 --- a/jenkins/spark-premerge-build.sh +++ b/jenkins/spark-premerge-build.sh @@ -85,14 +85,6 @@ mvn_verify() { # Triggering here until we change the jenkins file rapids_shuffle_smoke_test - - # non-caller classloader smoke test in pseudo-distributed - # standalone cluster - echo "Running test_cartesian_join_special_case_count with spark.rapids.force.caller.classloader=false" - PYSP_TEST_spark_rapids_force_caller_classloader=false \ - NUM_LOCAL_EXECS=1 \ - TEST_PARALLEL=0 \ - ./integration_tests/run_pyspark_from_build.sh -k 'test_cartesian_join_special_case_count[100]' } rapids_shuffle_smoke_test() { diff --git a/jenkins/spark-tests.sh b/jenkins/spark-tests.sh index 002d19de599..f3c39af3fcf 100755 --- a/jenkins/spark-tests.sh +++ b/jenkins/spark-tests.sh @@ -183,7 +183,6 @@ run_iceberg_tests() { # --packages in distributed setups, should be fixed by # https://github.com/NVIDIA/spark-rapids/pull/5646 SPARK_SUBMIT_FLAGS="$BASE_SPARK_SUBMIT_ARGS $SEQ_CONF \ - --conf spark.rapids.force.caller.classloader=false \ --packages org.apache.iceberg:iceberg-spark-runtime-${ICEBERG_SPARK_VER}_2.12:${ICEBERG_VERSION} \ --conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions \ --conf spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog \ diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 868743c5cdd..feaf02aebac 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -1479,14 +1479,6 @@ object RapidsConf { .booleanConf .createWithDefault(true) - val FORCE_SHIMCALLER_CLASSLOADER = conf("spark.rapids.force.caller.classloader") - .doc("Option to statically add shim's parallel world classloader URLs to " + - "the classloader of the ShimLoader class, typically Bootstrap classloader. This option" + - " uses reflection with setAccessible true on a classloader that is not created by Spark.") - .internal() - .booleanConf - .createWithDefault(value = true) - val SPARK_GPU_RESOURCE_NAME = conf("spark.rapids.gpu.resourceName") .doc("The name of the Spark resource that represents a GPU that you want the plugin to use " + "if using custom resources with Spark.") diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 177604c693d..edea1e0e7e0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -3953,7 +3953,7 @@ object GpuOverrides extends Logging { .withPsNote(TypeEnum.STRUCT, "Round-robin partitioning is not supported for nested " + s"structs if ${SQLConf.SORT_BEFORE_REPARTITION.key} is true") .withPsNote( - Seq(TypeEnum.ARRAY, TypeEnum.MAP), + Seq(TypeEnum.ARRAY, TypeEnum.MAP), "Round-robin partitioning is not supported if " + s"${SQLConf.SORT_BEFORE_REPARTITION.key} is true"), TypeSig.all), @@ -4017,7 +4017,7 @@ object GpuOverrides extends Logging { (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.ARRAY + TypeSig.STRUCT) .nested() - .withPsNote(Seq(TypeEnum.ARRAY, TypeEnum.MAP), + .withPsNote(Seq(TypeEnum.ARRAY, TypeEnum.MAP), "not allowed for grouping expressions") .withPsNote(TypeEnum.STRUCT, "not allowed for grouping expressions if containing Array or Map as child"), @@ -4033,7 +4033,7 @@ object GpuOverrides extends Logging { .nested() .withPsNote(TypeEnum.BINARY, "only allowed when aggregate buffers can be " + "converted between CPU and GPU") - .withPsNote(Seq(TypeEnum.ARRAY, TypeEnum.MAP), + .withPsNote(Seq(TypeEnum.ARRAY, TypeEnum.MAP), "not allowed for grouping expressions") .withPsNote(TypeEnum.STRUCT, "not allowed for grouping expressions if containing Array or Map as child"), @@ -4051,7 +4051,7 @@ object GpuOverrides extends Logging { .nested() .withPsNote(TypeEnum.BINARY, "only allowed when aggregate buffers can be " + "converted between CPU and GPU") - .withPsNote(Seq(TypeEnum.ARRAY, TypeEnum.MAP), + .withPsNote(Seq(TypeEnum.ARRAY, TypeEnum.MAP), "not allowed for grouping expressions") .withPsNote(TypeEnum.STRUCT, "not allowed for grouping expressions if containing Array or Map as child"), diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index e2766970aef..9c56b96664a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -1497,14 +1497,6 @@ object RapidsConf { .booleanConf .createWithDefault(true) - val FORCE_SHIMCALLER_CLASSLOADER = conf("spark.rapids.force.caller.classloader") - .doc("Option to statically add shim's parallel world classloader URLs to " + - "the classloader of the ShimLoader class, typically Bootstrap classloader. This option" + - " uses reflection with setAccessible true on a classloader that is not created by Spark.") - .internal() - .booleanConf - .createWithDefault(value = true) - val SPARK_GPU_RESOURCE_NAME = conf("spark.rapids.gpu.resourceName") .doc("The name of the Spark resource that represents a GPU that you want the plugin to use " + "if using custom resources with Spark.") diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala index a1b61c13b15..2032cb29a23 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala @@ -23,7 +23,7 @@ import com.nvidia.spark.rapids.iceberg.IcebergProvider import org.apache.commons.lang3.reflect.MethodUtils import scala.annotation.tailrec import scala.collection.JavaConverters._ -import scala.util.{Failure, Success, Try} +import scala.util.Try import org.apache.spark.{SPARK_BRANCH, SPARK_BUILD_DATE, SPARK_BUILD_USER, SPARK_REPO_URL, SPARK_REVISION, SPARK_VERSION, SparkConf, SparkEnv} import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin} @@ -32,6 +32,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ColumnarRule, SparkPlan} +import org.apache.spark.sql.rapids.execution.UnshimmedTrampolineUtil import org.apache.spark.util.MutableURLClassLoader /* @@ -129,52 +130,6 @@ object ShimLoader extends Logging { s"org.apache.spark.sql.rapids.shims.$shimId.RapidsShuffleInternalManager" } - private def serializerClassloader(): Option[ClassLoader] = { - // Hypothesis: serializer is the most universal way to intercept classloaders - - // https://github.com/apache/spark/blob/master/core/src/main/scala/ - // org/apache/spark/serializer/JavaSerializer.scala#L147 - - // https://github.com/apache/spark/blob/master/core/src/main/scala/ - // org/apache/spark/serializer/KryoSerializer.scala#L134 - - Option(SparkEnv.get) - .flatMap { - case env if !env.conf.get("spark.rapids.force.caller.classloader", - true.toString).toBoolean => Option(env.serializer) - case _ if (conventionalSingleShimJarDetected) => None - case _ => - logInfo("Forcing shim caller classloader update (default behavior). " + - "If it causes issues with userClassPathFirst, set " + - "spark.rapids.force.caller.classloader to false!") - None - } - .flatMap { serializer => - logInfo("Looking for a mutable classloader (defaultClassLoader) in SparkEnv.serializer " + - serializer) - // scalac generates accessor methods - val serdeClassLoader = MethodUtils - .invokeMethod(serializer, true, "defaultClassLoader") - .asInstanceOf[Option[ClassLoader]] - .getOrElse { - val threadContextClassLoader = Thread.currentThread().getContextClassLoader - logInfo(s"No defaultClassLoader found in $serializer, falling back " + - s"on Thread context classloader: " + threadContextClassLoader) - threadContextClassLoader - } - - logInfo("Extracted Spark classloader from SparkEnv.serializer " + serdeClassLoader) - findURLClassLoader(serdeClassLoader) - }.orElse { - val shimLoaderCallerCl = getClass.getClassLoader - if (!conventionalSingleShimJarDetected) { - logInfo("Falling back on ShimLoader caller's classloader " + shimLoaderCallerCl) - } - Option(shimLoaderCallerCl) - } - } - - @tailrec private def findURLClassLoader(classLoader: ClassLoader): Option[ClassLoader] = { // walk up the classloader hierarchy until we hit a classloader we can mutate @@ -214,23 +169,12 @@ object ShimLoader extends Logging { } private def updateSparkClassLoader(): Unit = { - // TODO propose a proper addClassPathURL API to Spark similar to addJar but - // accepting non-file-based URI - serializerClassloader().foreach { urlAddable => + findURLClassLoader(UnshimmedTrampolineUtil.sparkClassLoader).foreach { urlAddable => urlsForSparkClassLoader.foreach { url => if (!conventionalSingleShimJarDetected) { logInfo(s"Updating spark classloader $urlAddable with the URLs: " + urlsForSparkClassLoader.mkString(", ")) - Try(MethodUtils.invokeMethod(urlAddable, true, "addURL", url)) - .recoverWith { - case nsm: NoSuchMethodException => - logWarning("JDK8+ detected, consider setting " + - "spark.rapids.force.caller.classloader to false as a workaround") - logDebug(s"JDK8+ detected by catching ${nsm}", nsm) - Success(Unit) - case t => Failure(t) - }.get - + MethodUtils.invokeMethod(urlAddable, true, "addURL", url) logInfo(s"Spark classLoader $urlAddable updated successfully") urlAddable match { case urlCl: java.net.URLClassLoader => @@ -240,7 +184,7 @@ object ShimLoader extends Logging { s"classloader $urlCl although addURL succeeded, maybe pushed up to the " + s"parent classloader ${urlCl.getParent}") } - case _ => () + case _ => Unit } } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/UnshimmedTrampolineUtil.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/UnshimmedTrampolineUtil.scala new file mode 100644 index 00000000000..482cbe1ba53 --- /dev/null +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/UnshimmedTrampolineUtil.scala @@ -0,0 +1,23 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.rapids.execution + +object UnshimmedTrampolineUtil { + def sparkClassLoader: ClassLoader = { + org.apache.spark.util.Utils.getContextOrSparkClassLoader + } +} From aac650e25f10a895803603a60f3d51eb520c1d6a Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 16 Aug 2022 16:40:26 -0600 Subject: [PATCH 031/190] Run Delta Lake tests with Spark 3.2.x (#5981) --- .../src/main/python/delta_lake_test.py | 7 +++-- jenkins/spark-tests.sh | 26 +++++++++++++++++++ 2 files changed, 31 insertions(+), 2 deletions(-) diff --git a/integration_tests/src/main/python/delta_lake_test.py b/integration_tests/src/main/python/delta_lake_test.py index 66792207e9a..edfc5efd369 100644 --- a/integration_tests/src/main/python/delta_lake_test.py +++ b/integration_tests/src/main/python/delta_lake_test.py @@ -16,13 +16,16 @@ from pyspark.sql import Row from asserts import assert_gpu_fallback_collect from marks import allow_non_gpu, delta_lake -from spark_session import with_cpu_session, with_gpu_session, is_databricks91_or_later, is_databricks104_or_later +from spark_session import with_cpu_session, is_databricks91_or_later, spark_version +from spark_session import with_cpu_session, with_gpu_session, is_databricks91_or_later, is_databricks104_or_later, \ + spark_version _conf = {'spark.rapids.sql.explain': 'ALL'} @delta_lake @allow_non_gpu('FileSourceScanExec') -@pytest.mark.skipif(not is_databricks91_or_later(), reason="Delta Lake is already configured on Databricks so we just run these tests there for now") +@pytest.mark.skipif(not (is_databricks91_or_later() or spark_version().startswith("3.2.")), \ + reason="Delta Lake is already configured on Databricks and CI supports Delta Lake OSS with Spark 3.2.x so far") def test_delta_metadata_query_fallback(spark_tmp_table_factory): table = spark_tmp_table_factory.get() def setup_delta_table(spark): diff --git a/jenkins/spark-tests.sh b/jenkins/spark-tests.sh index f3c39af3fcf..5ffeddc7619 100755 --- a/jenkins/spark-tests.sh +++ b/jenkins/spark-tests.sh @@ -168,10 +168,26 @@ export CUDF_UDF_TEST_ARGS="--conf spark.rapids.memory.gpu.allocFraction=0.1 \ --conf spark.pyspark.python=/opt/conda/bin/python \ --py-files ${RAPIDS_PLUGIN_JAR}" + export SCRIPT_PATH="$(pwd -P)" export TARGET_DIR="$SCRIPT_PATH/target" mkdir -p $TARGET_DIR +run_delta_lake_tests() { + echo "run_delta_lake_tests SPARK_VER = $SPARK_VER" + SPARK_321_PATTERN="(3\.2\.[1-9])" + DELTA_LAKE_VER="1.2.1" + if [[ $SPARK_VER =~ $SPARK_321_PATTERN ]]; then + SPARK_SUBMIT_FLAGS="$BASE_SPARK_SUBMIT_ARGS $SEQ_CONF \ + --packages io.delta:delta-core_2.12:$DELTA_LAKE_VER \ + --conf spark.sql.extensions=io.delta.sql.DeltaSparkSessionExtension \ + --conf spark.sql.catalog.spark_catalog=org.apache.spark.sql.delta.catalog.DeltaCatalog" \ + ./run_pyspark_from_build.sh -m delta_lake --delta_lake + else + echo "Skipping Delta Lake tests. Delta Lake does not support Spark version $SPARK_VER" + fi +} + run_iceberg_tests() { ICEBERG_VERSION=${ICEBERG_VERSION:-0.13.2} # get the major/minor version of Spark @@ -214,6 +230,10 @@ run_test_not_parallel() { ./run_pyspark_from_build.sh -k cache_test ;; + delta_lake) + run_delta_lake_tests + ;; + iceberg) run_iceberg_tests ;; @@ -263,6 +283,7 @@ export -f get_tests_by_tags # - DEFAULT: all tests except cudf_udf tests # - CUDF_UDF_ONLY: cudf_udf tests only, requires extra conda cudf-py lib # - ICEBERG_ONLY: iceberg tests only +# - DELTA_LAKE_ONLY: Delta Lake tests only TEST_MODE=${TEST_MODE:-'DEFAULT'} if [[ $TEST_MODE == "DEFAULT" ]]; then # integration tests @@ -315,6 +336,11 @@ if [[ "$TEST_MODE" == "CUDF_UDF_ONLY" ]]; then run_test_not_parallel cudf_udf_test fi +# Delta Lake tests +if [[ "$TEST_MODE" == "DEFAULT" || "$TEST_MODE" == "DELTA_LAKE_ONLY" ]]; then + run_test_not_parallel delta_lake +fi + # Iceberg tests if [[ "$TEST_MODE" == "DEFAULT" || "$TEST_MODE" == "ICEBERG_ONLY" ]]; then run_test_not_parallel iceberg From 24570a097d3afc909e6c4ae9cf5c8c24bd8df9c4 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Wed, 17 Aug 2022 09:38:57 +0800 Subject: [PATCH 032/190] Add more infomation in FileSourceScanExec log when timezone is not UTC (#6307) * Add more infomation in FileSourceScanExec log when timezone is not UTC * Refactor to support custom unsupported message for GPU * Add support for nested type check * Rename functions and add comments * Keep default warning message for nested types * Remove type filter when timezone is not utc * Add default value in a case match Signed-off-by: thirtiseven --- .../com/nvidia/spark/rapids/TypeChecks.scala | 43 +++++++++++++++++-- 1 file changed, 39 insertions(+), 4 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index c27aec726ae..abf9ed64f6d 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -360,8 +360,7 @@ final class TypeSig private( case DoubleType => check.contains(TypeEnum.DOUBLE) case DateType => check.contains(TypeEnum.DATE) case TimestampType if check.contains(TypeEnum.TIMESTAMP) => - TypeChecks.areTimestampsSupported(ZoneId.systemDefault()) && - TypeChecks.areTimestampsSupported(SQLConf.get.sessionLocalTimeZone) + TypeChecks.areTimestampsSupported() case StringType => check.contains(TypeEnum.STRING) case dt: DecimalType => check.contains(TypeEnum.DECIMAL) && @@ -422,8 +421,7 @@ final class TypeSig private( basicNotSupportedMessage(dataType, TypeEnum.DATE, check, isChild) case TimestampType => if (check.contains(TypeEnum.TIMESTAMP) && - (!TypeChecks.areTimestampsSupported(ZoneId.systemDefault()) || - !TypeChecks.areTimestampsSupported(SQLConf.get.sessionLocalTimeZone))) { + !TypeChecks.areTimestampsSupported()) { Seq(withChild(isChild, s"$dataType is not supported with timezone settings: (JVM:" + s" ${ZoneId.systemDefault()}, session: ${SQLConf.get.sessionLocalTimeZone})." + s" Set both of the timezones to UTC to enable $dataType support")) @@ -776,6 +774,36 @@ abstract class TypeChecks[RET] { }.mkString(", ") } + /** + * Original log does not print enough info when timezone is not UTC, + * here check again to add UTC info. + */ + private def tagTimezoneInfoIfHasTimestampType( + unsupportedTypes: Map[DataType, Set[String]], + meta: RapidsMeta[_, _, _] + ): Unit = { + def checkTimestampType(dataType: DataType): Unit = dataType match { + case TimestampType if !TypeChecks.areTimestampsSupported() => { + meta.willNotWorkOnGpu(s"your timezone isn't in UTC (JVM:" + + s" ${ZoneId.systemDefault()}, session: ${SQLConf.get.sessionLocalTimeZone})." + + s" Set both of the timezones to UTC to enable TimestampType support") + return + } + case ArrayType(elementType, _) => + checkTimestampType(elementType) + case MapType(keyType, valueType, _) => + checkTimestampType(keyType) + checkTimestampType(valueType) + case StructType(fields) => + fields.foreach(field => checkTimestampType(field.dataType)) + case _ => + // do nothing + } + unsupportedTypes.foreach { case (dataType, nameSet) => + checkTimestampType(dataType) + } + } + protected def tagUnsupportedTypes( meta: RapidsMeta[_, _, _], sig: TypeSig, @@ -787,6 +815,8 @@ abstract class TypeChecks[RET] { .groupBy(_.dataType) .mapValues(_.map(_.name).toSet) + tagTimezoneInfoIfHasTimestampType(unsupportedTypes, meta) + if (unsupportedTypes.nonEmpty) { meta.willNotWorkOnGpu(msgFormat.format(stringifyTypeAttributeMap(unsupportedTypes))) } @@ -805,6 +835,11 @@ object TypeChecks { val zoneId = DateTimeUtils.getZoneId(zoneIdString) areTimestampsSupported(zoneId) } + + def areTimestampsSupported(): Boolean = { + areTimestampsSupported(ZoneId.systemDefault()) && + areTimestampsSupported(SQLConf.get.sessionLocalTimeZone) + } } /** From 51610de61653bc89ff19377b47c384f9865c3985 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Wed, 17 Aug 2022 12:51:01 -0500 Subject: [PATCH 033/190] Add missing subnet option to dataproc cluster example (#6335) Signed-off-by: Jason Lowe Signed-off-by: Jason Lowe --- docs/get-started/getting-started-gcp.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/get-started/getting-started-gcp.md b/docs/get-started/getting-started-gcp.md index 89d4304b583..c725e189a28 100644 --- a/docs/get-started/getting-started-gcp.md +++ b/docs/get-started/getting-started-gcp.md @@ -83,7 +83,8 @@ gcloud dataproc clusters create $CLUSTER_NAME \ --optional-components=JUPYTER,ZEPPELIN \ --metadata=rapids-runtime=SPARK \ --bucket=$GCS_BUCKET \ - --enable-component-gateway + --enable-component-gateway \ + --subnet=default ``` This may take around 10-15 minutes to complete. You can navigate to the Dataproc clusters tab in the From 593caef7dd61d024e3b144462323a37571a10adc Mon Sep 17 00:00:00 2001 From: Nghia Truong Date: Wed, 17 Aug 2022 11:16:05 -0700 Subject: [PATCH 034/190] Change `TimestampGen` unit in integration test from millisecond to microsecond (#6286) * Change unit from millisecond to microsecond * Add an empty line Signed-off-by: Nghia Truong * Fix `TIMESTAMP_TRUNCATE_REGEX` Signed-off-by: Nghia Truong * Remove `Array(Timestamp)` from cache test Signed-off-by: Nghia Truong * Change timestamp regex Signed-off-by: Nghia Truong * Fix comments Signed-off-by: Nghia Truong * Remove test with nested types containing timestamp Signed-off-by: Nghia Truong * Update comment Signed-off-by: Nghia Truong * Rewrite tests to workaround failure Signed-off-by: Nghia Truong Signed-off-by: Nghia Truong --- .../src/main/python/cache_test.py | 10 +++++++- integration_tests/src/main/python/data_gen.py | 17 +++++++------- integration_tests/src/main/python/orc_test.py | 21 ++++++++++++++++- .../src/main/python/parquet_write_test.py | 23 ++++++++++++++++++- 4 files changed, 60 insertions(+), 11 deletions(-) diff --git a/integration_tests/src/main/python/cache_test.py b/integration_tests/src/main/python/cache_test.py index 4b3bce54fb9..cff9d4e7a07 100644 --- a/integration_tests/src/main/python/cache_test.py +++ b/integration_tests/src/main/python/cache_test.py @@ -150,6 +150,14 @@ def n_fold(spark): # This test doesn't allow negative scale for Decimals as ` df.write.mode('overwrite').parquet(data_path)` # writes parquet which doesn't allow negative decimals +# In addition, `TIMESTAMP_MILLIS` can't be handled correctly when the input is of nested types containing timestamp. +# See issue https://github.com/NVIDIA/spark-rapids/issues/6302. +# Thus, we exclude nested types contaning timestamp from the tests here. +# When the issue is resolved, remove `_cache_single_array_gens_no_null_no_timestamp` and +# use just `_cache_single_array_gens_no_null` for `data_gen` parameter. +_all_basic_gens_no_null_no_timestamp = [gen for gen in all_basic_gens_no_null if gen != timestamp_gen] +_cache_single_array_gens_no_null_no_timestamp = [ArrayGen(gen) for gen in _all_basic_gens_no_null_no_timestamp + + _cache_decimal_gens] @pytest.mark.parametrize('data_gen', [StringGen(), ByteGen(), ShortGen(), IntegerGen(), LongGen(), ArrayGen( StructGen([['child0', StringGen()], @@ -158,7 +166,7 @@ def n_fold(spark): pytest.param(FloatGen(special_cases=[FLOAT_MIN, FLOAT_MAX, 0.0, 1.0, -1.0]), marks=[incompat]), pytest.param(DoubleGen(special_cases=double_special_cases), marks=[incompat]), BooleanGen(), DateGen(), TimestampGen(), decimal_gen_32bit, decimal_gen_64bit, - decimal_gen_128bit] + _cache_single_array_gens_no_null, ids=idfn) + decimal_gen_128bit] + _cache_single_array_gens_no_null_no_timestamp, ids=idfn) @pytest.mark.parametrize('ts_write', ['TIMESTAMP_MICROS', 'TIMESTAMP_MILLIS']) @pytest.mark.parametrize('enable_vectorized', ['true', 'false'], ids=idfn) @ignore_order diff --git a/integration_tests/src/main/python/data_gen.py b/integration_tests/src/main/python/data_gen.py index ac3c812a5ae..f034c39e6c3 100644 --- a/integration_tests/src/main/python/data_gen.py +++ b/integration_tests/src/main/python/data_gen.py @@ -531,23 +531,24 @@ def __init__(self, start=None, end=None, nullable=True): elif not isinstance(start, date): raise RuntimeError('Unsupported type passed in for end {}'.format(end)) - self._start_time = self._to_ms_since_epoch(start) - self._end_time = self._to_ms_since_epoch(end) + self._start_time = self._to_us_since_epoch(start) + self._end_time = self._to_us_since_epoch(end) if (self._epoch >= start and self._epoch <= end): self.with_special_case(self._epoch) _epoch = datetime(1970, 1, 1, tzinfo=timezone.utc) - _ms = timedelta(milliseconds=1) - def _to_ms_since_epoch(self, val): - return int((val - self._epoch)/self._ms) + _us = timedelta(microseconds=1) - def _from_ms_since_epoch(self, ms): - return self._epoch + timedelta(milliseconds=ms) + def _to_us_since_epoch(self, val): + return int((val - self._epoch)/self._us) + + def _from_us_since_epoch(self, us): + return self._epoch + timedelta(microseconds=us) def start(self, rand): start = self._start_time end = self._end_time - self._start(rand, lambda : self._from_ms_since_epoch(rand.randint(start, end))) + self._start(rand, lambda : self._from_us_since_epoch(rand.randint(start, end))) def contains_ts(self): return True diff --git a/integration_tests/src/main/python/orc_test.py b/integration_tests/src/main/python/orc_test.py index 4ac83dae6f9..204b4a127b9 100644 --- a/integration_tests/src/main/python/orc_test.py +++ b/integration_tests/src/main/python/orc_test.py @@ -478,7 +478,26 @@ def test_read_struct_without_stream(spark_tmp_path): lambda spark : spark.read.orc(data_path)) -@pytest.mark.parametrize('orc_gen', flattened_orc_gens, ids=idfn) +# There is an issue when writing timestamp: https://github.com/NVIDIA/spark-rapids/issues/6312 +# Thus, we exclude timestamp types from the tests here. +# When the issue is resolved, remove all these `*_no_timestamp` generators and use just `flattened_orc_gens` +# for `orc_gen` parameter. +orc_basic_gens_no_timestamp = [gen for gen in orc_basic_gens if not isinstance(gen, TimestampGen)] + +orc_array_gens_sample_no_timestamp = [ArrayGen(sub_gen) for sub_gen in orc_basic_gens_no_timestamp] + [ + ArrayGen(ArrayGen(short_gen, max_length=10), max_length=10), + ArrayGen(ArrayGen(string_gen, max_length=10), max_length=10), + ArrayGen(ArrayGen(decimal_gen_64bit, max_length=10), max_length=10), + ArrayGen(StructGen([['child0', byte_gen], ['child1', string_gen], ['child2', float_gen]]))] + +orc_basic_struct_gen_no_timestamp = StructGen([['child'+str(ind), sub_gen] for ind, sub_gen in enumerate(orc_basic_gens_no_timestamp)]) +orc_struct_gens_sample_no_timestamp = [orc_basic_struct_gen_no_timestamp, + StructGen([['child0', byte_gen], ['child1', orc_basic_struct_gen_no_timestamp]]), + StructGen([['child0', ArrayGen(short_gen)], ['child1', double_gen]])] + +flattened_orc_gens_no_timestamp = orc_basic_gens + orc_array_gens_sample_no_timestamp + orc_struct_gens_sample_no_timestamp + +@pytest.mark.parametrize('orc_gen', flattened_orc_gens_no_timestamp, ids=idfn) @pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) @pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) @pytest.mark.parametrize('case_sensitive', ["false", "true"]) diff --git a/integration_tests/src/main/python/parquet_write_test.py b/integration_tests/src/main/python/parquet_write_test.py index f78b3a92e76..6ba4984c64d 100644 --- a/integration_tests/src/main/python/parquet_write_test.py +++ b/integration_tests/src/main/python/parquet_write_test.py @@ -95,12 +95,33 @@ def test_write_round_trip(spark_tmp_path, parquet_gens): data_path, conf=writer_confs) +# `TIMESTAMP_MILLIS` can't be handled correctly when the input is of nested types containing timestamp. +# See issue https://github.com/NVIDIA/spark-rapids/issues/6302. +# Thus, we exclude `TIMESTAMP_MILLIS` from the tests here. +# When the issue is resolved, unify this test with the test below. +parquet_ts_write_options_no_millis = ['INT96', 'TIMESTAMP_MICROS'] @pytest.mark.parametrize('parquet_gens', [[ limited_timestamp(), ArrayGen(limited_timestamp(), max_length=10), MapGen(limited_timestamp(nullable=False), limited_timestamp())]], ids=idfn) +@pytest.mark.parametrize('ts_type', parquet_ts_write_options_no_millis) +def test_timestamp_write_round_trip_no_millis(spark_tmp_path, parquet_gens, ts_type): + gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] + data_path = spark_tmp_path + '/PARQUET_DATA' + all_confs = copy_and_update(writer_confs, {'spark.sql.parquet.outputTimestampType': ts_type}) + assert_gpu_and_cpu_writes_are_equal_collect( + lambda spark, path: gen_df(spark, gen_list).coalesce(1).write.parquet(path), + lambda spark, path: spark.read.parquet(path), + data_path, + conf=all_confs) + +# `TIMESTAMP_MILLIS` can't be handled correctly when the input is of nested types containing timestamp. +# See issue https://github.com/NVIDIA/spark-rapids/issues/6302. +# Thus, we exclude nested types contaning timestamp from the tests here. +# When the issue is resolved, unify this test with the test above. +@pytest.mark.parametrize('parquet_gens', [[limited_timestamp()]], ids=idfn) @pytest.mark.parametrize('ts_type', parquet_ts_write_options) -def test_timestamp_write_round_trip(spark_tmp_path, parquet_gens, ts_type): +def test_timestamp_write_round_trip_no_nested_timestamp(spark_tmp_path, parquet_gens, ts_type): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] data_path = spark_tmp_path + '/PARQUET_DATA' all_confs = copy_and_update(writer_confs, {'spark.sql.parquet.outputTimestampType': ts_type}) From b19441fe8dec648273a110deeff7c881c527d3ba Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Wed, 17 Aug 2022 12:04:31 -0700 Subject: [PATCH 035/190] Avoid passing duplicate conf to spark_init_internal (#6342) Fixes #6344 Avoid passing In non-xdist execution the same config by different means. In spark.jars case it may corrupt the classpath. Verified with ``` SPARK_HOME=~/dist/spark-3.2.2-bin-hadoop3.2 \ NUM_LOCAL_EXECS=1 \ SPARK_SUBMIT_FLAGS="--packages org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.13.2 --conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions --conf spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog --conf spark.sql.catalog.spark_catalog.type=hadoop --conf spark.sql.catalog.spark_catalog.warehouse=$(mktemp -d -p /tmp spark-warehouse-XXXXXX) --conf spark.rapids.force.caller.classloader=false" \ ./integration_tests/run_pyspark_from_build.sh -m iceberg --iceberg \ -k 'test_iceberg_read_partition_key[PERFILE-Long]' ``` Signed-off-by: Gera Shegalov --- integration_tests/run_pyspark_from_build.sh | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/integration_tests/run_pyspark_from_build.sh b/integration_tests/run_pyspark_from_build.sh index c4d5993329b..3ac660c5c45 100755 --- a/integration_tests/run_pyspark_from_build.sh +++ b/integration_tests/run_pyspark_from_build.sh @@ -273,10 +273,20 @@ else jarOpts=(--driver-class-path "${PYSP_TEST_spark_driver_extraClassPath}") fi + driverJavaOpts="$PYSP_TEST_spark_driver_extraJavaOptions" + gpuAllocSize="$PYSP_TEST_spark_rapids_memory_gpu_allocSize" + + # avoid double processing of variables passed to spark in + # spark_conf_init + unset PYSP_TEST_spark_driver_extraClassPath + unset PYSP_TEST_spark_driver_extraJavaOptions + unset PYSP_TEST_spark_jars + unset PYSP_TEST_spark_rapids_memory_gpu_allocSize + exec "$SPARK_HOME"/bin/spark-submit "${jarOpts[@]}" \ - --driver-java-options "$PYSP_TEST_spark_driver_extraJavaOptions" \ + --driver-java-options "$driverJavaOpts" \ $SPARK_SUBMIT_FLAGS \ - --conf 'spark.rapids.memory.gpu.allocSize='"$PYSP_TEST_spark_rapids_memory_gpu_allocSize" \ + --conf 'spark.rapids.memory.gpu.allocSize='"$gpuAllocSize" \ "${RUN_TESTS_COMMAND[@]}" "${TEST_COMMON_OPTS[@]}" fi fi From 91ee6a61cf52afb05a419975d736826a49b10478 Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Thu, 18 Aug 2022 10:11:08 -0500 Subject: [PATCH 036/190] Adding AutoTuner to Profiling Tool (#6338) Signed-off-by: Ahmed Hussein (amahussein) Co-authored-by: Partho Sarthi --- tools/pom.xml | 6 + tools/scripts/discoveryScript.sh | 127 ++++ tools/scripts/getWorkerInfo.sh | 45 ++ .../rapids/tool/profiling/Analysis.scala | 1 + .../rapids/tool/profiling/AutoTuner.scala | 601 ++++++++++++++++++ .../tool/profiling/ClassWarehouse.scala | 14 + .../rapids/tool/profiling/ProfileArgs.scala | 10 + .../rapids/tool/profiling/Profiler.scala | 22 + .../rapids/tool/profiling/AnalysisSuite.scala | 8 +- 9 files changed, 831 insertions(+), 3 deletions(-) create mode 100644 tools/scripts/discoveryScript.sh create mode 100755 tools/scripts/getWorkerInfo.sh create mode 100644 tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala diff --git a/tools/pom.xml b/tools/pom.xml index b67097410c5..deea91935e6 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -83,6 +83,12 @@ scalatest_${scala.binary.version} test
+ + + org.yaml + snakeyaml + 1.30 +
diff --git a/tools/scripts/discoveryScript.sh b/tools/scripts/discoveryScript.sh new file mode 100644 index 00000000000..cff13f7ffc6 --- /dev/null +++ b/tools/scripts/discoveryScript.sh @@ -0,0 +1,127 @@ +#!/bin/bash +# +# Copyright (c) 2022, NVIDIA CORPORATION. All rights reserved. +# +# 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 script generates a YAML file containing system properties. The YAML +# file is used by the AutoTuner for recommending Spark RAPIDS configurations. +# Usage: ./discoveryScript.sh [num-workers] [output-file] + +function usage() { + echo "Usage: ./discoveryScript.sh [num-workers] [output-file]" +} + +if [ "$#" -ne 2 ]; then + echo "Illegal number of parameters" + usage + exit 1 +fi + +NUM_WORKERS=$1 +OUTPUT_FILE=$2 +OS=$(uname -s) + +function echo_warn { + CLEAR='\033[0m' + YELLOW='\033[0;33m' + + echo -e "${YELLOW}[WARN] $1${CLEAR}\n"; +} + +function get_linux_properties() { + local memInKb_=$(cat /proc/meminfo | grep MemTotal) + local memInKb="$(echo $memInKb_ | cut -d' ' -f2)" + memInGb=$((memInKb / (1024 * 1024))) + memInGb="${memInGb}gb" + + local numCores_=$(lscpu | grep "CPU(s)") + numCores="$(echo $numCores_ | cut -d' ' -f2)" + cpuArch=$(arch) + + timeZone=$(cat /etc/timezone) +} + +function get_macos_properties() { + local memInB=$(sysctl -n hw.memsize) + memInGb=$((memInB / (1024 * 1024 * 1024))) + memInGb="${memInGb}gb" + + numCores=$(sysctl -n hw.ncpu) + cpuArch=$(arch) + + timeZone=$(readlink /etc/localtime | sed 's#/var/db/timezone/zoneinfo/##g') +} + +function get_gpu_properties() { + if ! command -v nvidia-smi &> /dev/null; then + echo_warn "nvidia-smi could not be found. Cannot get gpu properties". + return + fi + + local gpuInfo=$(nvidia-smi --query-gpu=count,name,memory.total --format=csv,noheader) + + if [ $? -ne 0 ]; then + echo_warn "nvidia-smi did not exit successfully. Cannot get gpu properties". + return + fi + + IFS=',' read -ra gpuInfoArr <<< "$gpuInfo" + gpuCount=${gpuInfoArr[0]} + gpuName=${gpuInfoArr[1]} + local gpuMemoryInMb="$(echo ${gpuInfoArr[2]} | cut -d' ' -f1)" + gpuMemoryInGb=$((gpuMemoryInMb / 1024)) + gpuMemoryInGb="${gpuMemoryInGb}gb" +} + +function get_disk_space() { + local freeDiskSpaceInKb=$(df -Pk . | sed 1d | grep -v used | awk '{ print $4 "\t" }') + freeDiskSpaceInGb=$((freeDiskSpaceInKb / (1024 * 1024))) + freeDiskSpaceInGb="${freeDiskSpaceInGb}gb" +} + +function read_system_properties() { + if [[ $OS == Linux ]];then + get_linux_properties + elif [[ $OS == Darwin ]];then + get_macos_properties + else + echo_warn "Unsupported OS: $OS. Cannot get cpu and memory properties." + fi + get_gpu_properties + get_disk_space +} + +function write_system_properties() { + cat > "$OUTPUT_FILE" << EOF +system: + num_cores: $numCores + cpu_arch: $cpuArch + memory: $memInGb + free_disk_space: $freeDiskSpaceInGb + time_zone: $timeZone + num_workers: $NUM_WORKERS +gpu: + count: $gpuCount + memory: $gpuMemoryInMb + name: $gpuName +EOF + + echo "YAML file generated at $OUTPUT_FILE" + echo "Contents - " + cat "$OUTPUT_FILE" +} + +read_system_properties +write_system_properties diff --git a/tools/scripts/getWorkerInfo.sh b/tools/scripts/getWorkerInfo.sh new file mode 100755 index 00000000000..bf9aacec849 --- /dev/null +++ b/tools/scripts/getWorkerInfo.sh @@ -0,0 +1,45 @@ +#!/bin/bash +# +# Copyright (c) 2022, NVIDIA CORPORATION. All rights reserved. +# +# 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 script copies and executes discoveryScript.sh to the worker and copies +# back the generated YAML file. The YAML file is used by the AutoTuner for +# recommending Spark RAPIDS configurations. +# Assumption: 'discoveryScript.sh' is present in the same directory as this script. + +# Usage: ./getWorkerInfo.sh [num-workers] [worker-ip] [output-file] + +function usage() { + echo "Usage: ./getWorkerInfo.sh [num-workers] [worker-ip] [output-file]" +} + +if [ "$#" -ne 3 ]; then + echo "Illegal number of parameters" + usage + exit 1 +fi + +NUM_WORKERS=$1 +WORKER_IP=$2 +OUTPUT_FILE_ON_DRIVER=$3 +OUTPUT_FILE_ON_WORKER=/tmp/system_props.yaml +DISCOVERY_SCRIPT=discoveryScript.sh + +echo "Fetching system information from worker - $WORKER_IP" +scp -q ./$DISCOVERY_SCRIPT "$WORKER_IP":/tmp +ssh "$WORKER_IP" "bash /tmp/$DISCOVERY_SCRIPT $NUM_WORKERS $OUTPUT_FILE_ON_WORKER" +scp -q "$WORKER_IP":$OUTPUT_FILE_ON_WORKER $OUTPUT_FILE_ON_DRIVER +echo -e "\nYAML file copied to driver at $OUTPUT_FILE_ON_DRIVER" diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Analysis.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Analysis.scala index d7268efbebc..a4fd9a0d0fa 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Analysis.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Analysis.scala @@ -276,6 +276,7 @@ class Analysis(apps: Seq[ApplicationInfo]) { tasksInSQL.map(_.executorDeserializeTime).sum, execRunTime, tasksInSQL.map(_.input_bytesRead).sum, + tasksInSQL.map(_.input_bytesRead).sum * 1.0 / tasksInSQL.size, tasksInSQL.map(_.input_recordsRead).sum, tasksInSQL.map(_.jvmGCTime).sum, tasksInSQL.map(_.memoryBytesSpilled).sum, diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala new file mode 100644 index 00000000000..5f083688839 --- /dev/null +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala @@ -0,0 +1,601 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.tool.profiling + +import java.io.FileNotFoundException + +import org.yaml.snakeyaml.Yaml +import scala.collection.JavaConverters._ + +import org.apache.spark.internal.Logging + +/** + * A wrapper class that stores all the properties that would be recommended by the Auto-tuner. + * Separate getter and setter methods are specified for each property for ease of access. + */ +class Config { + private val properties: collection.mutable.Map[String, String] = collection.mutable.Map( + "spark.executor.instances" -> null, + "spark.rapids.sql.enabled" -> null, + "spark.executor.cores" -> null, + "spark.executor.memory" -> null, + "spark.rapids.sql.concurrentGpuTasks" -> null, + "spark.task.resource.gpu.amount" -> null, + "spark.sql.shuffle.partitions" -> null, + "spark.sql.files.maxPartitionBytes" -> null, + "spark.rapids.memory.pinnedPool.size" -> null, + "spark.executor.memoryOverhead" -> null + ) + + def getExecutorInstances: Int = properties("spark.executor.instances").toInt + + def getExecutorCores: Int = properties("spark.executor.cores").toInt + + def getExecutorMemory: String = properties("spark.executor.memory") + + def getConcurrentGpuTasks: Int = properties("spark.rapids.sql.concurrentGpuTasks").toInt + + def getTaskResourceGpu: Double = properties("spark.task.resource.gpu.amount").toDouble + + def getShufflePartitions: Int = properties("spark.sql.shuffle.partitions").toInt + + def getMaxPartitionBytes: String = properties("spark.sql.files.maxPartitionBytes") + + def getPinnedPoolSize: String = properties("spark.rapids.memory.pinnedPool.size") + + def getExecutorMemoryOverhead: String = properties("spark.executor.memoryOverhead") + + def getAllProperties: Map[String, String] = properties.toMap + + def setExecutorInstances(numInstances: Int): Unit = { + properties("spark.executor.instances") = numInstances.toString + } + + def setExecutorCores(executorCores: Int): Unit = { + properties("spark.executor.cores") = executorCores.toString + } + + def setExecutorMemory(executorMemory: String): Unit = { + properties("spark.executor.memory") = executorMemory + } + + def setConcurrentGpuTasks(concurrentGpuTasks: Int): Unit = { + properties("spark.rapids.sql.concurrentGpuTasks") = concurrentGpuTasks.toString + } + + def setTaskResourceGpu(taskResourceGpu: Double): Unit = { + properties("spark.task.resource.gpu.amount") = taskResourceGpu.toString + } + + def setShufflePartitions(shufflePartitions: Int): Unit = { + properties("spark.sql.shuffle.partitions") = shufflePartitions.toString + } + + def setMaxPartitionBytes(maxPartitionBytes: String): Unit = { + properties("spark.sql.files.maxPartitionBytes") = maxPartitionBytes + } + + def setPinnedPoolSize(pinnedPoolSize: String): Unit = { + properties("spark.rapids.memory.pinnedPool.size") = pinnedPoolSize + } + + def setExecutorMemoryOverhead(executorMemoryOverhead: String): Unit = { + properties("spark.executor.memoryOverhead") = executorMemoryOverhead + } + + def setExecutorMemoryOverheadFactor(executorMemoryOverheadFactor: Double): Unit = { + properties("spark.executor.memoryOverheadFactor") = executorMemoryOverheadFactor.toString + } +} + +/** + * AutoTuner module that uses event logs and worker's system properties to recommend Spark + * RAPIDS configuration based on heuristics. + * + * Example (Refer to test suite for more cases): + * a. Success: + * Input: + * system: + * num_cores: 64 + * cpu_arch: x86_64 + * memory: 512gb + * free_disk_space: 800gb + * time_zone: America/Los_Angeles + * num_workers: 4 + * gpu: + * count: 8 + * memory: 32gb + * name: NVIDIA V100 + * + * Output: + * Spark Properties: + * --conf spark.executor.cores=8 + * --conf spark.executor.instances=32 + * --conf spark.executor.memory=63.75g + * --conf spark.executor.memoryOverhead=8.38g + * --conf spark.rapids.memory.pinnedPool.size=2g + * --conf spark.rapids.sql.concurrentGpuTasks=4 + * --conf spark.sql.files.maxPartitionBytes=31.67g + * --conf spark.sql.shuffle.partitions=200 + * --conf spark.task.resource.gpu.amount=0.125 + * + * b. Failure: + * Input: Empty or Incorrect System Properties + * Output: + * Comments: + * - 'spark.executor.memory' should be set to at least 2GB/core. + * - 'spark.executor.instances' should be set to 'num_gpus * num_workers'. + * - 'spark.task.resource.gpu.amount' should be set to 1/#cores. + */ +class AutoTuner(app: ApplicationSummaryInfo, workerInfo: String) extends Logging { + import AutoTuner._ + val DEFAULT_SHUFFLE_PARTITION_MULTIPLIER: Int = 2 + val MAX_JVM_GCTIME_FRACTION: Double = 0.3 + + val DEFAULT_CONCURRENT_GPU_TASKS: Int = 2 + val DEFAULT_CONCURRENT_GPU_TASKS_MULTIPLIER: Double = 0.125 // Currently aggressively set to 1/8 + val MAX_CONCURRENT_GPU_TASKS: Int = 4 + + val DEFAULT_MAX_PARTITION_BYTES: String = "512m" + val MAX_PARTITION_BYTES_BOUND: String = "4g" + val MAX_PARTITION_BYTES_RANGE: String = "256m" + val MIN_PARTITION_BYTES_RANGE: String = "128m" + + val DEFAULT_PINNED_POOL_SIZE: String = "2g" + val DEFAULT_MEMORY_OVERHEAD_FACTOR: Double = 0.1 + val DEFAULT_SYSTEM_OVERHEAD: String = "2g" // Overhead of other system processes + + val MAX_PER_EXECUTOR_CORE_COUNT: Int = 16 + val MIN_PER_EXECUTOR_CORE_COUNT: Int = 4 + + val MAX_EXECUTOR_MEMORY: String = "64g" + val MIN_EXECUTOR_MEMORY: String = "8g" + + var comments: Seq[String] = Seq() + + /** + * Recommendation for 'spark.executor.instances' based on number of gpus and workers. + * Assumption - In case GPU properties are not available, it assumes 1 GPU per worker. + */ + private def recommendExecutorInstances(recommendedConfig: Config, + systemProps: SystemProps): Unit = { + systemProps.numWorkers match { + case Some(numWorkers) => + val numInstances = if (systemProps.gpuProps != null) { + numWorkers * systemProps.gpuProps.count + } else { + // Assumption: 1 GPU per worker. + numWorkers + } + + recommendedConfig.setExecutorInstances(numInstances) + case None => + val num_gpus_str = if (systemProps.gpuProps != null) { + systemProps.gpuProps.count.toString + } else { + "num_gpus" + } + + comments :+= s"'spark.executor.instances' should be set to $num_gpus_str * num_workers." + } + } + + /** + * Recommendation for 'spark.executor.cores' based on number of cpu cores and gpus. + */ + private def recommendExecutorCores(recommendedConfig: Config, + systemProps: SystemProps): Unit = { + val numCores: Int = if (systemProps.gpuProps != null) { + Math.min(systemProps.numCores * 1.0 / systemProps.gpuProps.count, + MAX_PER_EXECUTOR_CORE_COUNT).toInt + } else { + systemProps.numCores + } + + if (numCores < MIN_PER_EXECUTOR_CORE_COUNT) { + comments :+= s"Number of cores per executor is very low. " + + s"It is recommended to have at least $MIN_PER_EXECUTOR_CORE_COUNT cores per executor." + } + + if (systemProps.numWorkers.nonEmpty) { + val numInstances = recommendedConfig.getExecutorInstances + if (numCores * numInstances < systemProps.numCores) { + comments :+= "Not all cores in the machine are being used. " + + "It is recommended to use different machine." + } + } + + recommendedConfig.setExecutorCores(numCores) + } + + /** + * Recommendation for 'spark.executor.memory' based on system memory, cluster scheduler + * and num of gpus. + * Note - This method requires recommendedConfig to have 'spark.executor.cores' set. + */ + private def recommendExecutorMemory(recommendedConfig: Config, + systemProps: SystemProps): Unit = { + val systemMemoryNum: Long = convertFromHumanReadableSize(systemProps.memory) + val systemOverhead: Long = convertFromHumanReadableSize(DEFAULT_SYSTEM_OVERHEAD) + val effectiveSystemMemoryNum: Long = systemMemoryNum - systemOverhead - + convertFromHumanReadableSize(DEFAULT_PINNED_POOL_SIZE) + val maxExecutorMemNum: Long = convertFromHumanReadableSize(MAX_EXECUTOR_MEMORY) + + val executorMemory: Long = if (systemProps.gpuProps != null) { + Math.min(effectiveSystemMemoryNum * 1.0 / systemProps.gpuProps.count, + maxExecutorMemNum).toLong + } else { + Math.min(effectiveSystemMemoryNum * 1.0 / recommendedConfig.getExecutorCores, + maxExecutorMemNum).toLong + } + + if(executorMemory < convertFromHumanReadableSize(MIN_EXECUTOR_MEMORY)) { + comments :+= s"Executor memory is very low. " + + s"It is recommended to have at least $MIN_EXECUTOR_MEMORY." + } + + recommendedConfig.setExecutorMemory(convertToHumanReadableSize(executorMemory)) + } + + /** + * Calculate memory overhead as: pinnedPoolSize + (memoryOverheadFactor * executorMemory) + */ + private def calculateMemoryOverhead(pinnedPoolSize: String, executorMemory: String): Long = { + val pinnedPoolSizeNum = convertFromHumanReadableSize(pinnedPoolSize) + val executorMemoryNum = convertFromHumanReadableSize(executorMemory) + (pinnedPoolSizeNum + DEFAULT_MEMORY_OVERHEAD_FACTOR * executorMemoryNum).toLong + } + + /** + * Recommendation for 'spark.rapids.memory.pinnedPool.size' if it is not set. + * Recommendation for memoryOverhead and memoryOverheadFactor based on cluster scheduler and + * spark version used. + * + * Flow: + * if (pinnedPoolSize is not set) -> set pinnedPoolSize, calculate and set memoryOverhead + * else + * if using yarn + * -> if memoryOverhead is not set, calculate and set. + * if using k8s + * -> if memoryOverheadFactor is not set, add comment. + */ + private def recommendMemoryOverheadProperties(recommendedConfig: Config): Unit = { + val pinnedPoolSize = getSparkProperty(app, "spark.rapids.memory.pinnedPool.size") + if (pinnedPoolSize.isEmpty) { + recommendedConfig.setPinnedPoolSize(DEFAULT_PINNED_POOL_SIZE) + val memoryOverhead = calculateMemoryOverhead(DEFAULT_PINNED_POOL_SIZE, + recommendedConfig.getExecutorMemory) + recommendedConfig.setExecutorMemoryOverhead(convertToHumanReadableSize(memoryOverhead)) + } else { + val sparkMaster = getSparkProperty(app, "spark.master") + if (sparkMaster.contains("yarn")) { + if (getSparkProperty(app, "spark.executor.memoryOverhead").isEmpty) { + val memoryOverhead = calculateMemoryOverhead(pinnedPoolSize.get, + recommendedConfig.getExecutorMemory) + recommendedConfig.setExecutorMemoryOverhead( + convertToHumanReadableSize(memoryOverhead)) + } + } else if (sparkMaster.contains("k8s")) { + if (compareSparkVersion(app.appInfo.head.sparkVersion, "3.3.0") > 0) { + if (getSparkProperty(app, "spark.executor.memoryOverheadFactor").isEmpty) { + comments :+= "'spark.executor.memoryOverheadFactor' must be set " + + "if using 'spark.rapids.memory.pinnedPool.size'" + } + } else { + if (getSparkProperty(app, "spark.kubernetes.memoryOverheadFactor").isEmpty) { + comments :+= "'spark.kubernetes.memoryOverheadFactor' must be set " + + "if using 'spark.rapids.memory.pinnedPool.size'" + } + } + } + } + } + + /** + * Recommendation for 'spark.sql.shuffle.partitions' based on spill size. + */ + private def recommendShufflePartitions(recommendedConfig: Config): Unit = { + var shufflePartitions: Int = getSparkProperty(app, "spark.sql.shuffle.partitions") + .getOrElse("200").toInt + + // TODO: Need to look at other metrics for GPU spills (DEBUG mode), and batch sizes metric + val totalSpilledMetrics = app.sqlTaskAggMetrics.map { + task => task.diskBytesSpilledSum + task.memoryBytesSpilledSum + }.sum + if (totalSpilledMetrics > 0) { + shufflePartitions *= DEFAULT_SHUFFLE_PARTITION_MULTIPLIER + // Could be memory instead of partitions + comments :+= "\"spark.sql.shuffle.partitions\" should be increased since spilling occurred." + } + recommendedConfig.setShufflePartitions(shufflePartitions) + } + + /** + * Calculate max partition bytes using task input size. + * Eg, + * MIN_PARTITION_BYTES_RANGE = 128m, MAX_PARTITION_BYTES_RANGE = 256m + * (1) Input: maxPartitionBytes = 512m + * taskInputSize = 12m + * Output: newMaxPartitionBytes = 512m * (128m/12m) = 5g + * (2) Input: maxPartitionBytes = 2g + * taskInputSize = 512m, + * Output: newMaxPartitionBytes = 2g / (512m/128m) = 512m + */ + private def calculateMaxPartitionBytes(maxPartitionBytes: String): String = { + val taskInputSize = + app.sqlTaskAggMetrics.map(_.inputBytesReadAvg).sum / app.sqlTaskAggMetrics.size + val maxPartitionBytesNum = convertFromHumanReadableSize(maxPartitionBytes) + if (taskInputSize > 0 && + taskInputSize < convertFromHumanReadableSize(MIN_PARTITION_BYTES_RANGE)) { + // Increase partition size + val calculatedMaxPartitionBytes = Math.max( + maxPartitionBytesNum * + (convertFromHumanReadableSize(MIN_PARTITION_BYTES_RANGE) / taskInputSize), + convertFromHumanReadableSize(MAX_PARTITION_BYTES_BOUND)) + + convertToHumanReadableSize(calculatedMaxPartitionBytes.toLong) + } else if (taskInputSize > convertFromHumanReadableSize(MAX_PARTITION_BYTES_RANGE)) { + // Decrease partition size + val calculatedMaxPartitionBytes = Math.max( + maxPartitionBytesNum / + (taskInputSize / convertFromHumanReadableSize(MAX_PARTITION_BYTES_RANGE)), + convertFromHumanReadableSize(MAX_PARTITION_BYTES_BOUND)) + + convertToHumanReadableSize(calculatedMaxPartitionBytes.toLong) + } else { + // Do not recommend maxPartitionBytes + null + } + } + + /** + * Recommendation for 'spark.sql.files.maxPartitionBytes' based on input size for each task. + */ + private def recommendMaxPartitionBytes(recommendedConfig: Config): Unit = { + getSparkProperty(app, "spark.sql.files.maxPartitionBytes") match { + case None => + val newMaxPartitionBytes = calculateMaxPartitionBytes(DEFAULT_MAX_PARTITION_BYTES) + recommendedConfig.setMaxPartitionBytes(newMaxPartitionBytes) + + case Some(maxPartitionBytes) => + val newMaxPartitionBytes = calculateMaxPartitionBytes(maxPartitionBytes) + recommendedConfig.setMaxPartitionBytes(newMaxPartitionBytes) + comments :+= s"Although 'spark.sql.files.maxPartitionBytes' was set to $maxPartitionBytes" + + s", recommended value is $newMaxPartitionBytes." + } + } + + private def recommendGeneralProperties() = { + val aqeEnabled = getSparkProperty(app, "spark.sql.adaptive.enabled").getOrElse("False") + if (aqeEnabled == "False") { + comments :+= "'spark.sql.adaptive.enabled' should be enabled for better performance." + } + + val jvmGCFraction = app.sqlTaskAggMetrics.map { + taskMetrics => taskMetrics.jvmGCTimeSum * 1.0 / taskMetrics.executorCpuTime + } + if ((jvmGCFraction.sum / jvmGCFraction.size) > MAX_JVM_GCTIME_FRACTION) { + comments :+= "Average JVM GC time is very high. " + + "Other Garbage Collectors can be used for better performance" + } + } + + /** + * Recommendation for 'spark.task.resource.gpu.amount' based on num of cpu cores. + */ + private def recommendTaskResourceGpu(recommendedConfig: Config): Unit = { + val numCores: Int = recommendedConfig.getExecutorCores + val taskResourceGpu: Double = 1.0 / numCores + recommendedConfig.setTaskResourceGpu(taskResourceGpu) + } + + /** + * Recommendation for 'spark.rapids.sql.concurrentGpuTasks' based on gpu memory. + */ + private def recommendConcurrentGpuTasks(recommendedConfig: Config, + systemProps: SystemProps): Unit = { + val numCores: Int = recommendedConfig.getExecutorCores + val gpuMemoryNum: Long = convertFromHumanReadableSize(systemProps.gpuProps.memory) + val concurrentGpuTasks: Int = Math.min( + gpuMemoryNum * DEFAULT_CONCURRENT_GPU_TASKS_MULTIPLIER, + MAX_CONCURRENT_GPU_TASKS).toInt + recommendedConfig.setConcurrentGpuTasks(concurrentGpuTasks) + + if(numCores < concurrentGpuTasks) { + comments :+= s"For the given GPU, number of CPU cores is very low. It should be" + + s" at least equal to concurrent gpu tasks i.e. $concurrentGpuTasks." + } + } + + /** + * Recommendations for Spark only properties. + */ + private def recommendSparkProperties(recommendedConfig: Config, + systemProps: SystemProps): Unit = { + if (systemProps == null) { + logWarning("System information is not available. Cannot recommend properties.") + comments :+= "'spark.executor.memory' should be set to at least 2GB/core." + comments :+= "'spark.executor.instances' should be set to 'num_gpus * num_workers'." + } else { + recommendExecutorInstances(recommendedConfig, systemProps) + recommendExecutorCores(recommendedConfig, systemProps) + recommendExecutorMemory(recommendedConfig, systemProps) + recommendMemoryOverheadProperties(recommendedConfig) + recommendShufflePartitions(recommendedConfig) + recommendMaxPartitionBytes(recommendedConfig) + recommendGeneralProperties() + } + } + + /** + * Recommendations for GPU (RAPIDS) properties. + */ + private def recommendGpuProperties(recommendedConfig: Config, systemProps: SystemProps): Unit = { + if (systemProps == null || systemProps.gpuProps == null) { + logWarning("GPU information is not available. Cannot recommend properties.") + comments :+= "'spark.task.resource.gpu.amount' should be set to 1/#cores." + comments :+= s"'spark.rapids.sql.concurrentGpuTasks' should be set to" + + s" $DEFAULT_CONCURRENT_GPU_TASKS." + comments :+= s"'spark.rapids.memory.pinnedPool.size' should be set to" + + s" $DEFAULT_PINNED_POOL_SIZE." + } else { + recommendTaskResourceGpu(recommendedConfig) + recommendConcurrentGpuTasks(recommendedConfig, systemProps) + } + } + + /** + * Entry point for generating recommendations. + */ + def getRecommendedProperties: (Seq[RecommendedPropertyResult], + Seq[RecommendedCommentResult]) = { + val (systemProps, message) = parseSystemInfo(workerInfo) + if(message.nonEmpty) { + comments :+= message.get + } + + val recommendedConfig = new Config() + recommendSparkProperties(recommendedConfig, systemProps) + recommendGpuProperties(recommendedConfig, systemProps) + (toPropertyProfileResult(recommendedConfig), toCommentProfileResult(comments)) + } +} + +object AutoTuner extends Logging { + val DEFAULT_WORKER_INFO: String = "." + val SUPPORTED_SIZE_UNITS: Seq[String] = Seq("b", "k", "m", "g", "t", "p") + + /** + * Parses the yaml file and returns system and gpu properties. + * See [[SystemProps]] and [[GpuProps]]. + */ + def parseSystemInfo(yamlFile: String): (SystemProps, Option[String]) = { + try { + val yaml = new Yaml() + val file = scala.io.Source.fromFile(yamlFile) + val text = file.mkString + val rawProps = yaml.load(text).asInstanceOf[java.util.Map[String, Any]] + .asScala.toMap.filter { case (_, v) => v != null } + val rawSystemProps = rawProps("system").asInstanceOf[java.util.Map[String, Any]] + .asScala.toMap.filter { case (_, v) => v != null } + + if (rawSystemProps.nonEmpty) { + val rawGpuProps = rawProps("gpu").asInstanceOf[java.util.Map[String, Any]] + .asScala.toMap.filter { case (_, v) => v != null } + + val gpuProps = if (rawGpuProps.nonEmpty) { + GpuProps( + rawGpuProps("count").toString.toInt, + rawGpuProps("memory").toString, + rawGpuProps("name").toString) + } else { + null + } + + (SystemProps( + rawSystemProps.getOrElse("num_cores", 1).toString.toInt, + rawSystemProps.getOrElse("cpu_arch", "").toString, + rawSystemProps.getOrElse("memory", "0b").toString, + rawSystemProps.getOrElse("free_disk_space", "0b").toString, + rawSystemProps.getOrElse("time_zone", "").toString, + rawSystemProps.get("num_workers").map(_.toString.toInt), + gpuProps), None) + } else { + (null, Some("System properties was empty")) + } + } catch { + case e: FileNotFoundException => + logError("Exception: " + e.getStackTrace.mkString("Array(", ", ", ")")) + (null, Some("System properties file was not found")) + case e: NullPointerException => + logError("Exception: " + e.getStackTrace.mkString("Array(", ", ", ")")) + (null, Some("System properties file was not formatted correctly.")) + case e: Exception => + logError("Exception: " + e.getStackTrace.mkString("Array(", ", ", ")")) + (null, Some(e.toString)) + } + } + + /** + * Returns the value of Spark property from the application summary info. + * [[RapidsPropertyProfileResult]] is defined as (key:key, rows: [key, value]). + * Returns: + * a. If the value is "null" or key is not found: None + * b. Else: Some(value) + */ + private def getSparkProperty(app: ApplicationSummaryInfo, property: String): Option[String] = { + app.sparkProps.collectFirst { + case propertyProfile: RapidsPropertyProfileResult + if propertyProfile.key == property && propertyProfile.rows(1) != "null" => + propertyProfile.rows(1) + } + } + + /** + * Converts size from human readable to bytes. + * Eg, "4m" -> 4194304. + */ + def convertFromHumanReadableSize(size: String): Long = { + val sizesArr = size.toLowerCase.split("(?=[a-z])") + val sizeNum = sizesArr(0).toDouble + val sizeUnit = sizesArr(1) + assert(SUPPORTED_SIZE_UNITS.contains(sizeUnit), s"$size is not a valid human readable size") + (sizeNum * Math.pow(1024, SUPPORTED_SIZE_UNITS.indexOf(sizeUnit))).toLong + } + + /** + * Converts size from bytes to human readable. + * Eg, 4194304 -> "4m", 633554 -> "618.70k". + */ + def convertToHumanReadableSize(size: Long): String = { + if(size < 0) return "0b" + + val unitIndex = (Math.log10(size)/Math.log10(1024)).toInt + assert(unitIndex < SUPPORTED_SIZE_UNITS.size, + s"$size is too large to convert to human readable size") + + val sizeNum = size * 1.0/Math.pow(1024, unitIndex) + val sizeUnit = SUPPORTED_SIZE_UNITS(unitIndex) + + // If sizeNum is an integer omit fraction part + if ((sizeNum % 1) == 0) { + f"${sizeNum.toLong}$sizeUnit" + } else { + f"$sizeNum%.2f$sizeUnit" + } + } + + /** + * Reference - https://stackoverflow.com/a/55246235 + */ + def compareSparkVersion(version1: String, version2: String): Int = { + val paddedVersions = version1.split("\\.").zipAll(version2.split("\\."), "0", "0") + val difference = paddedVersions.find { case (a, b) => a != b } + difference.fold(0) { case (a, b) => a.toInt - b.toInt } + } + + private def toPropertyProfileResult(recommendedConfig:Config): Seq[RecommendedPropertyResult] = { + val properties = recommendedConfig.getAllProperties + properties.collect { + case (property, value) if value != null => RecommendedPropertyResult(property, value) + }.toSeq.sortBy(_.property) + } + + private def toCommentProfileResult(comments:Seq[String]): Seq[RecommendedCommentResult] = { + comments.map(RecommendedCommentResult) + } +} diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ClassWarehouse.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ClassWarehouse.scala index 5b78d11496d..b10786a09e2 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ClassWarehouse.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ClassWarehouse.scala @@ -466,6 +466,8 @@ case class SQLTaskAggMetricsProfileResult( executorDeserializeTimeSum: Long, executorRunTimeSum: Long, inputBytesReadSum: Long, + // Not added to the output since it is used only by the AutoTuner + inputBytesReadAvg: Double, inputRecordsReadSum: Long, jvmGCTimeSum: Long, memoryBytesSpilledSum: Long, @@ -630,3 +632,15 @@ case class WholeStageCodeGenResults( childNodeID.toString) } } + +case class GpuProps(count: Int, memory: String, name: String) +case class SystemProps(numCores: Int, cpuArch: String, memory: String, + diskSpace: String, timeZone: String, numWorkers: Option[Int], gpuProps: GpuProps) + +case class RecommendedPropertyResult(property: String, value: String){ + override def toString: String = "--conf %s=%s".format(property, value) +} + +case class RecommendedCommentResult(comment: String) { + override def toString: String = "- %s".format(comment) +} \ No newline at end of file diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileArgs.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileArgs.scala index 90484c4e0be..21d97ee52dd 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileArgs.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileArgs.scala @@ -86,6 +86,16 @@ Usage: java -cp rapids-4-spark-tools_2.12-.jar:$SPARK_HOME/jars/* descr = "Filter event logs whose application start occurred within the past specified " + "time period. Valid time periods are min(minute),h(hours),d(days),w(weeks)," + "m(months). If a period is not specified it defaults to days.") + val autoTuner: ScallopOption[Boolean] = + opt[Boolean](required = false, + descr = "Toggle auto-tuner module.", + default = Some(false)) + val workerInfo: ScallopOption[String] = + opt[String](required = false, + descr = "File path containing the system information of a worker node. It is assumed " + + "that all workers are homogenous. It requires the AutoTuner to be enabled. Default is" + + "current directory.", + default = Some(AutoTuner.DEFAULT_WORKER_INFO)) validate(filterCriteria) { case crit if (crit.endsWith("-newest-filesystem") || diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala index 76245636b8a..39e7cff46bc 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala @@ -47,6 +47,8 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs) extends Logging private val outputCSV: Boolean = appArgs.csv() private val outputCombined: Boolean = appArgs.combined() + private val useAutoTuner: Boolean = appArgs.autoTuner() + logInfo(s"Threadpool size is $nThreads") /** @@ -446,6 +448,26 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs) extends Logging profileOutputWriter.write("Removed Executors", app.removedExecutors) profileOutputWriter.write("Unsupported SQL Plan", app.unsupportedOps, Some("Unsupported SQL Ops")) + + if (useAutoTuner) { + val workerInfo: String = appArgs.workerInfo.getOrElse(AutoTuner.DEFAULT_WORKER_INFO) + val autoTuner: AutoTuner = new AutoTuner(app, workerInfo) + val (properties, comments) = autoTuner.getRecommendedProperties + profileOutputWriter.writeText("\n### D. Recommended Configuration ###\n") + + if (properties.nonEmpty) { + val propertiesToStr = properties.map(_.toString).reduce(_ + "\n" + _) + profileOutputWriter.writeText("\nSpark Properties:\n" + propertiesToStr + "\n") + } else { + profileOutputWriter.writeText("Cannot recommend properties. See Comments.\n") + } + + // Comments are optional + if (comments.nonEmpty) { + val commentsToStr = comments.map(_.toString).reduce(_ + "\n" + _) + profileOutputWriter.writeText("\nComments:\n" + commentsToStr + "\n") + } + } } } } diff --git a/tools/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala b/tools/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala index a65e2e65888..44c87fc32af 100644 --- a/tools/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala +++ b/tools/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021, NVIDIA CORPORATION. + * Copyright (c) 2021-2022, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -37,6 +37,8 @@ class AnalysisSuite extends FunSuite { private val expRoot = ToolTestUtils.getTestResourceFile("ProfilingExpectations") private val logDir = ToolTestUtils.getTestResourcePath("spark-events-profiling") private val qualLogDir = ToolTestUtils.getTestResourcePath("spark-events-qualification") + // AutoTuner added a field in SQLTaskAggMetricsProfileResult but it is not among the output + private val skippedColumnsInSqlAggProfile = Seq("inputBytesReadAvg") test("test sqlMetricsAggregation simple") { testSqlMetricsAggregation(Array(s"$logDir/rapids_join_eventlog.zstd"), @@ -64,9 +66,9 @@ class AnalysisSuite extends FunSuite { val analysis = new Analysis(apps) val sqlTaskMetrics = analysis.sqlMetricsAggregation() - val resultExpectation = new File(expRoot,expectFile) + val resultExpectation = new File(expRoot, expectFile) import sparkSession.implicits._ - val actualDf = sqlTaskMetrics.toDF + val actualDf = sqlTaskMetrics.toDF.drop(skippedColumnsInSqlAggProfile:_*) val dfExpect = ToolTestUtils.readExpectationCSV(sparkSession, resultExpectation.getPath()) ToolTestUtils.compareDataFrames(actualDf, dfExpect) From dc62d6297470daa809071061b0fa2ccd10e30551 Mon Sep 17 00:00:00 2001 From: Nghia Truong Date: Fri, 19 Aug 2022 08:50:04 -0700 Subject: [PATCH 037/190] Add format `SSS` for `date_format` function (#6271) * Add supported formats Signed-off-by: Nghia Truong * Change unit from millisecond to microsecond * Add an empty line Signed-off-by: Nghia Truong * Remove 9S format Signed-off-by: Nghia Truong * Change conversion from second to microsecond Signed-off-by: Nghia Truong * Change conversion from second to microsecond Signed-off-by: Nghia Truong * Fix `TIMESTAMP_TRUNCATE_REGEX` Signed-off-by: Nghia Truong * Remove `Array(Timestamp)` from cache test Signed-off-by: Nghia Truong * Change timestamp regex Signed-off-by: Nghia Truong * Fix comments Signed-off-by: Nghia Truong * Remove test with nested types containing timestamp Signed-off-by: Nghia Truong * Update comment Signed-off-by: Nghia Truong * Rewrite tests to workaround failure Signed-off-by: Nghia Truong * Reverse code change for `GpuFromUnixTime` Signed-off-by: Nghia Truong * Fix tests Signed-off-by: Nghia Truong Signed-off-by: Nghia Truong --- integration_tests/src/main/python/date_time_test.py | 2 +- .../src/main/scala/com/nvidia/spark/rapids/DateUtils.scala | 4 ++-- .../org/apache/spark/sql/rapids/datetimeExpressions.scala | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/integration_tests/src/main/python/date_time_test.py b/integration_tests/src/main/python/date_time_test.py index 9ece62f321f..722febeb85e 100644 --- a/integration_tests/src/main/python/date_time_test.py +++ b/integration_tests/src/main/python/date_time_test.py @@ -365,7 +365,7 @@ def test_date_format_f_incompat(data_gen, date_format): assert_gpu_fallback_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr("date_format(a, '{}')".format(date_format)), 'ProjectExec', conf) -maybe_supported_date_formats = ['dd-MM-yyyy'] +maybe_supported_date_formats = ['dd-MM-yyyy', 'yyyy-MM-dd HH:mm:ss.SSS', 'yyyy-MM-dd HH:mm:ss.SSSSSS'] @pytest.mark.parametrize('date_format', maybe_supported_date_formats, ids=idfn) @pytest.mark.parametrize('data_gen', date_n_time_gens, ids=idfn) @allow_non_gpu('ProjectExec,Alias,DateFormatClass,Literal,Cast') diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DateUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DateUtils.scala index 4ea21ef92aa..ccb2e91f57a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DateUtils.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DateUtils.scala @@ -40,7 +40,7 @@ object DateUtils { "u", "uu", "uuu", "uuuu", "uuuuu", "uuuuuu", "uuuuuuu", "uuuuuuuu", "uuuuuuuuu", "uuuuuuuuuu", "y", "yyy", "yyyyy", "yyyyyy", "yyyyyyy", "yyyyyyyy", "yyyyyyyyy", "yyyyyyyyyy", "D", "DD", "DDD", "s", "m", "H", "h", "M", "MMM", "MMMM", "MMMMM", "L", "LLL", "LLLL", "LLLLL", - "d", "S", "SS", "SSS", "SSSS", "SSSSS", "SSSSSSSSS", "SSSSSSS", "SSSSSSSS") + "d", "S", "SS", "SSSS", "SSSSS", "SSSSSSS", "SSSSSSSS", "SSSSSSSSS") // we support "yy" in some cases, but not when parsing strings // https://github.com/NVIDIA/spark-rapids/issues/2118 @@ -48,7 +48,7 @@ object DateUtils { val conversionMap = Map( "MM" -> "%m", "LL" -> "%m", "dd" -> "%d", "mm" -> "%M", "ss" -> "%S", "HH" -> "%H", - "yy" -> "%y", "yyyy" -> "%Y", "SSSSSS" -> "%f") + "yy" -> "%y", "yyyy" -> "%Y", "SSS" -> "%3f", "SSSSSS" -> "%6f") val ONE_SECOND_MICROSECONDS = 1000000 diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala index 606c8a91b50..0ec1a295b5c 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala @@ -326,7 +326,7 @@ case class GpuDateFormatClass(timestamp: Expression, override def doColumnar(lhs: GpuColumnVector, rhs: GpuScalar): ColumnVector = { // we aren't using rhs as it was already converted in the GpuOverrides while creating the // expressions map and passed down here as strfFormat - withResource(lhs.getBase.asTimestampSeconds) { tsVector => + withResource(lhs.getBase.asTimestampMicroseconds()) { tsVector => tsVector.asStrings(strfFormat) } } From cbbfe42f8a9dd341c011f7570559d682635247af Mon Sep 17 00:00:00 2001 From: Peixin Date: Sat, 20 Aug 2022 00:12:41 +0800 Subject: [PATCH 038/190] If not delta_lake marker, skip tests (#6365) Signed-off-by: Peixin Li Signed-off-by: Peixin Li --- integration_tests/README.md | 7 +++++++ integration_tests/pytest.ini | 1 + integration_tests/src/main/python/conftest.py | 4 ++++ 3 files changed, 12 insertions(+) diff --git a/integration_tests/README.md b/integration_tests/README.md index 41447c01609..c1151a7d02e 100644 --- a/integration_tests/README.md +++ b/integration_tests/README.md @@ -394,6 +394,13 @@ properly without it. These tests assume Iceberg is not configured and are disabl If Spark has been configured to support Iceberg then these tests can be enabled by adding the `--iceberg` option to the command. +### Enabling Delta Lake tests + +Some tests require that Delta Lake has been configured in the Spark environment and cannot run +properly without it. These tests assume Delta Lake is not configured and are disabled by default. +If Spark has been configured to support Delta Lake then these tests can be enabled by adding the +`--delta_lake` option to the command. + ## Writing tests There are a number of libraries provided to help someone write new tests. diff --git a/integration_tests/pytest.ini b/integration_tests/pytest.ini index f4d9793c5c0..9cd4842a123 100644 --- a/integration_tests/pytest.ini +++ b/integration_tests/pytest.ini @@ -30,6 +30,7 @@ markers = nightly_host_mem_consuming_case: case in nightly_resource_consuming_test that consume much more host memory than normal cases fuzz_test: Mark fuzz tests iceberg: Mark a test that requires Iceberg has been configured, skipping if tests are not configured for Iceberg + delta_lake: Mark a test that requires Delta Lake has been configured, skipping if tests are not configured for Delta Lake regexp: Mark a test that tests regular expressions on the GPU (only works when UTF-8 is enabled) filterwarnings = ignore:.*pytest.mark.order.*:_pytest.warning_types.PytestUnknownMarkWarning diff --git a/integration_tests/src/main/python/conftest.py b/integration_tests/src/main/python/conftest.py index 3fa387496e2..8b880985aa8 100644 --- a/integration_tests/src/main/python/conftest.py +++ b/integration_tests/src/main/python/conftest.py @@ -186,6 +186,10 @@ def pytest_runtest_setup(item): elif is_databricks_runtime(): pytest.skip('Iceberg tests skipped on Databricks') + if item.get_closest_marker('delta_lake'): + if not item.config.getoption('delta_lake'): + pytest.skip('delta lake tests not configured to run') + def pytest_configure(config): global _runtime_env _runtime_env = config.getoption('runtime_env') From 89269c895de4a90af38efe8d5eeb0f342e32377d Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 19 Aug 2022 12:52:50 -0500 Subject: [PATCH 039/190] Allow writing Binary data in Parquet (#6350) Signed-off-by: Robert (Bobby) Evans --- docs/supported_ops.md | 16 ++++++++-------- .../src/main/python/parquet_write_test.py | 8 +++++--- .../com/nvidia/spark/rapids/GpuOverrides.scala | 4 +++- .../com/nvidia/spark/rapids/SchemaUtils.scala | 8 +++++++- tools/src/main/resources/supportedExecs.csv | 2 +- 5 files changed, 24 insertions(+), 14 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 357be9d19de..be50f389580 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -650,11 +650,11 @@ Accelerator supports are described below. S PS
128bit decimal only supported for Orc and Parquet
NS +PS
Only supported for Parquet
NS -NS -PS
Only supported for Parquet;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types NULL, BINARY, CALENDAR, UDT
-PS
Only supported for Parquet;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types NULL, BINARY, CALENDAR, UDT
-PS
Only supported for Parquet;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types NULL, BINARY, CALENDAR, UDT
+PS
Only supported for Parquet;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types NULL, CALENDAR, UDT
+PS
Only supported for Parquet;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types NULL, CALENDAR, UDT
+PS
Only supported for Parquet;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types NULL, CALENDAR, UDT
NS @@ -18884,11 +18884,11 @@ dates or timestamps, or for a lack of type coercion support. S S -NS +S -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types UDT
NS diff --git a/integration_tests/src/main/python/parquet_write_test.py b/integration_tests/src/main/python/parquet_write_test.py index 6ba4984c64d..129036fce54 100644 --- a/integration_tests/src/main/python/parquet_write_test.py +++ b/integration_tests/src/main/python/parquet_write_test.py @@ -51,12 +51,14 @@ def limited_int96(): string_gen, boolean_gen, date_gen, # we are limiting TimestampGen to avoid overflowing the INT96 value # see https://github.com/rapidsai/cudf/issues/8070 - limited_timestamp()] + limited_timestamp(), binary_gen] parquet_basic_map_gens = [MapGen(f(nullable=False), f()) for f in [ BooleanGen, ByteGen, ShortGen, IntegerGen, LongGen, FloatGen, DoubleGen, DateGen, limited_timestamp]] + [simple_string_to_string_map_gen, - MapGen(DecimalGen(20, 2, nullable=False), decimal_gen_128bit)] + MapGen(DecimalGen(20, 2, nullable=False), decimal_gen_128bit), + # python is not happy with binary values being keys of a map + MapGen(StringGen("a{1,5}", nullable=False), binary_gen)] parquet_struct_gen_no_maps = [ StructGen([['child' + str(ind), sub_gen] for ind, sub_gen in enumerate(parquet_basic_gen)]), @@ -81,7 +83,7 @@ def limited_int96(): parquet_map_gens = parquet_map_gens_sample + [ MapGen(StructGen([['child0', StringGen()], ['child1', StringGen()]], nullable=False), FloatGen()), MapGen(StructGen([['child0', StringGen(nullable=True)]], nullable=False), StringGen())] -parquet_write_gens_list = [parquet_basic_gen, decimal_gens] + [ [single_gen] for single_gen in parquet_struct_gen + parquet_array_gen + parquet_map_gens] +parquet_write_gens_list = [[binary_gen], parquet_basic_gen, decimal_gens] + [ [single_gen] for single_gen in parquet_struct_gen + parquet_array_gen + parquet_map_gens] parquet_ts_write_options = ['INT96', 'TIMESTAMP_MICROS', 'TIMESTAMP_MILLIS'] @pytest.mark.order(1) # at the head of xdist worker queue if pytest-order is installed diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index edea1e0e7e0..8766430af52 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -829,7 +829,8 @@ object GpuOverrides extends Logging { TypeSig.ARRAY + TypeSig.MAP + TypeSig.BINARY + GpuTypeShims.additionalParquetSupportedTypes).nested(), cudfWrite = (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.STRUCT + - TypeSig.ARRAY + TypeSig.MAP + GpuTypeShims.additionalParquetSupportedTypes).nested(), + TypeSig.ARRAY + TypeSig.MAP + TypeSig.BINARY + + GpuTypeShims.additionalParquetSupportedTypes).nested(), sparkSig = (TypeSig.cpuAtomics + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP + TypeSig.UDT + GpuTypeShims.additionalParquetSupportedTypes).nested())), (OrcFormatType, FileFormatChecks( @@ -3857,6 +3858,7 @@ object GpuOverrides extends Logging { TypeSig.STRUCT.withPsNote(TypeEnum.STRUCT, "Only supported for Parquet") + TypeSig.MAP.withPsNote(TypeEnum.MAP, "Only supported for Parquet") + TypeSig.ARRAY.withPsNote(TypeEnum.ARRAY, "Only supported for Parquet") + + TypeSig.BINARY.withPsNote(TypeEnum.BINARY, "Only supported for Parquet") + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), TypeSig.all), (p, conf, parent, r) => new SparkPlanMeta[DataWritingCommandExec](p, conf, parent, r) { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SchemaUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SchemaUtils.scala index 236819d9e71..3582d0e8eb0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SchemaUtils.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SchemaUtils.scala @@ -291,8 +291,14 @@ object SchemaUtils extends Arm { writeInt96, fieldMeta, parquetFieldIdWriteEnabled).build().getChildColumnOptions()(0))) + case BinaryType => + if (parquetFieldIdWriteEnabled && parquetFieldId.nonEmpty) { + builder.withBinaryColumn(name, nullable, parquetFieldId.get) + } else { + builder.withBinaryColumn(name, nullable) + } case _ => - if(parquetFieldIdWriteEnabled && parquetFieldId.nonEmpty) { + if (parquetFieldIdWriteEnabled && parquetFieldId.nonEmpty) { builder.withColumn(nullable, name, parquetFieldId.get) } else { builder.withColumns(nullable, name) diff --git a/tools/src/main/resources/supportedExecs.csv b/tools/src/main/resources/supportedExecs.csv index ef75558f809..9809884f315 100644 --- a/tools/src/main/resources/supportedExecs.csv +++ b/tools/src/main/resources/supportedExecs.csv @@ -19,7 +19,7 @@ HashAggregateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS ObjectHashAggregateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,PS,NS,PS,PS,PS,NS SortAggregateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,PS,NS,PS,PS,PS,NS InMemoryTableScanExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,PS,PS,PS,NS -DataWritingCommandExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,PS,NS,NS,NS,PS,PS,PS,NS +DataWritingCommandExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,PS,NS,PS,NS,PS,PS,PS,NS BatchScanExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,S,NS,PS,PS,PS,NS BroadcastExchangeExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS ShuffleExchangeExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS From 608b5b68f5d07083c4bd8a77f70392ae04d5cd99 Mon Sep 17 00:00:00 2001 From: Ryan Lee Date: Fri, 19 Aug 2022 10:54:07 -0700 Subject: [PATCH 040/190] Struct null aware equality comparator <=> support (#6306) * struct binop support * switch to integration tests * avoid DayTimeIntervalGen errors with < 330 * first pass ns_equal * remove extra whitespace and update docs Signed-off-by: Ryan Lee Signed-off-by: Ryan Lee Co-authored-by: Ryan Lee --- docs/supported_ops.md | 4 ++-- integration_tests/src/main/python/cmp_test.py | 4 +--- .../main/scala/com/nvidia/spark/rapids/GpuOverrides.scala | 8 ++++---- tools/src/main/resources/supportedExprs.csv | 4 ++-- 4 files changed, 9 insertions(+), 11 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index be50f389580..a3f9e09f17a 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -5636,7 +5636,7 @@ are limited. NS NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, ARRAY, UDT
NS @@ -5657,7 +5657,7 @@ are limited. NS NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, ARRAY, UDT
NS diff --git a/integration_tests/src/main/python/cmp_test.py b/integration_tests/src/main/python/cmp_test.py index 4d6146beb68..b4c2da7d419 100644 --- a/integration_tests/src/main/python/cmp_test.py +++ b/integration_tests/src/main/python/cmp_test.py @@ -50,9 +50,8 @@ def test_func(data_gen): data_gens = [DayTimeIntervalGen()] for data_gen in data_gens: test_func(data_gen) - -@pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen, ids=idfn) +@pytest.mark.parametrize('data_gen', eq_gens_with_decimal_gen + struct_gens_sample_with_decimal128_no_list, ids=idfn) def test_eq_ns(data_gen): (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) data_type = data_gen.data_type @@ -170,7 +169,6 @@ def test_func(data_gen): for data_gen in data_gens: test_func(data_gen) - @pytest.mark.parametrize('data_gen', orderable_gens, ids=idfn) def test_gt(data_gen): (s1, s2) = gen_scalars(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen)) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 8766430af52..41cc337b228 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -1959,11 +1959,11 @@ object GpuOverrides extends Logging { "Check if the values are equal including nulls <=>", ExprChecks.binaryProject( TypeSig.BOOLEAN, TypeSig.BOOLEAN, - ("lhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("lhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.comparable), - ("rhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("rhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.comparable)), (a, conf, p, r) => new BinaryExprMeta[EqualNullSafe](a, conf, p, r) { override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = diff --git a/tools/src/main/resources/supportedExprs.csv b/tools/src/main/resources/supportedExprs.csv index 7214faf2249..dac588103e5 100644 --- a/tools/src/main/resources/supportedExprs.csv +++ b/tools/src/main/resources/supportedExprs.csv @@ -176,8 +176,8 @@ ElementAt,S,`element_at`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS EndsWith,S, ,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA EndsWith,S, ,None,project,search,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA EndsWith,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -EqualNullSafe,S,`<=>`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS -EqualNullSafe,S,`<=>`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS +EqualNullSafe,S,`<=>`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS +EqualNullSafe,S,`<=>`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS EqualNullSafe,S,`<=>`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA EqualTo,S,`=`; `==`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS EqualTo,S,`=`; `==`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS From 4e400c1b235f8bf9f3bd9690d663558151d91c8b Mon Sep 17 00:00:00 2001 From: Ryan Lee Date: Fri, 19 Aug 2022 10:55:16 -0700 Subject: [PATCH 041/190] Binary type support in Iceberg read (#6361) * binary support, first pass * binary support, first pass Signed-off-by: Ryan Lee Signed-off-by: Ryan Lee Co-authored-by: Ryan Lee --- docs/supported_ops.md | 8 ++++---- integration_tests/src/main/python/iceberg_test.py | 2 +- .../main/scala/com/nvidia/spark/rapids/GpuOverrides.scala | 4 ++-- tools/src/main/resources/supportedDataSource.csv | 2 +- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index a3f9e09f17a..ade5980d647 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -18734,11 +18734,11 @@ dates or timestamps, or for a lack of type coercion support. S S -NS +S -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, UDT
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types UDT
NS diff --git a/integration_tests/src/main/python/iceberg_test.py b/integration_tests/src/main/python/iceberg_test.py index 3e14c937aa8..76ee30fcbe8 100644 --- a/integration_tests/src/main/python/iceberg_test.py +++ b/integration_tests/src/main/python/iceberg_test.py @@ -28,7 +28,7 @@ iceberg_gens_list = [ [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, - string_gen, boolean_gen, date_gen, timestamp_gen, + string_gen, boolean_gen, date_gen, timestamp_gen, binary_gen, ArrayGen(binary_gen), ArrayGen(byte_gen), ArrayGen(long_gen), ArrayGen(string_gen), ArrayGen(date_gen), ArrayGen(timestamp_gen), ArrayGen(decimal_gen_64bit), ArrayGen(ArrayGen(byte_gen)), StructGen([['child0', ArrayGen(byte_gen)], ['child1', byte_gen], ['child2', float_gen], ['child3', decimal_gen_64bit]]), diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 41cc337b228..50fcbd528de 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -853,11 +853,11 @@ object GpuOverrides extends Logging { sparkSig = (TypeSig.cpuAtomics + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP + TypeSig.UDT).nested())), (IcebergFormatType, FileFormatChecks( - cudfRead = (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.STRUCT + + cudfRead = (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.STRUCT + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.MAP + GpuTypeShims.additionalParquetSupportedTypes).nested(), cudfWrite = TypeSig.none, sparkSig = (TypeSig.cpuAtomics + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP + - TypeSig.UDT + GpuTypeShims.additionalParquetSupportedTypes).nested()))) + TypeSig.BINARY + TypeSig.UDT + GpuTypeShims.additionalParquetSupportedTypes).nested()))) val commonExpressions: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Seq( expr[Literal]( diff --git a/tools/src/main/resources/supportedDataSource.csv b/tools/src/main/resources/supportedDataSource.csv index d476923e9e7..68a27a23dec 100644 --- a/tools/src/main/resources/supportedDataSource.csv +++ b/tools/src/main/resources/supportedDataSource.csv @@ -1,7 +1,7 @@ Format,Direction,BOOLEAN,BYTE,SHORT,INT,LONG,FLOAT,DOUBLE,DATE,TIMESTAMP,STRING,DECIMAL,NULL,BINARY,CALENDAR,ARRAY,MAP,STRUCT,UDT Avro,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO CSV,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,NA,NA,NA,NA -Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS +Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA From 87a0270eb30afd3090e85e94e3085a1eb813edc6 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Fri, 19 Aug 2022 16:21:47 -0500 Subject: [PATCH 042/190] Fix datetime name collision in cast_test (#6371) Signed-off-by: Jason Lowe Signed-off-by: Jason Lowe --- integration_tests/src/main/python/cast_test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/integration_tests/src/main/python/cast_test.py b/integration_tests/src/main/python/cast_test.py index d5daf22b04c..2b15e68b2bf 100644 --- a/integration_tests/src/main/python/cast_test.py +++ b/integration_tests/src/main/python/cast_test.py @@ -22,7 +22,6 @@ from spark_init_internal import spark_version from datetime import datetime import math -import datetime _decimal_gen_36_5 = DecimalGen(precision=36, scale=5) From 6631ce8c74ced178cb9b0647c723f207d6033e9c Mon Sep 17 00:00:00 2001 From: Remzi Yang <59198230+HaoYang670@users.noreply.github.com> Date: Mon, 22 Aug 2022 09:03:01 +0800 Subject: [PATCH 043/190] Add `Nan` handling in `GpuArrayMin` (#6349) * test pass, memory leak Signed-off-by: remzi <13716567376yh@gmail.com> * some clean work Signed-off-by: remzi <13716567376yh@gmail.com> * fix memory leak Signed-off-by: remzi <13716567376yh@gmail.com> * fmt Signed-off-by: remzi <13716567376yh@gmail.com> * refactor Signed-off-by: remzi <13716567376yh@gmail.com> * refactor Signed-off-by: remzi <13716567376yh@gmail.com> * refactor Signed-off-by: remzi <13716567376yh@gmail.com> * refactor Signed-off-by: remzi <13716567376yh@gmail.com> * refactor Signed-off-by: remzi <13716567376yh@gmail.com> * refactor Signed-off-by: remzi <13716567376yh@gmail.com> * refactor Signed-off-by: remzi <13716567376yh@gmail.com> * rename Signed-off-by: remzi <13716567376yh@gmail.com> * add docs Signed-off-by: remzi <13716567376yh@gmail.com> * update tests Signed-off-by: remzi <13716567376yh@gmail.com> * fix potential memory leak Signed-off-by: remzi <13716567376yh@gmail.com> * fix potenial memory leak Signed-off-by: remzi <13716567376yh@gmail.com> * fix protential memory leak Signed-off-by: remzi <13716567376yh@gmail.com> * remove duplicate tests Signed-off-by: remzi <13716567376yh@gmail.com> * add all nans test Signed-off-by: remzi <13716567376yh@gmail.com> * Fix spelling mistake Co-authored-by: Jason Lowe * fix spelling mistake Signed-off-by: remzi <13716567376yh@gmail.com> * use SetValuesGen Signed-off-by: remzi <13716567376yh@gmail.com> Signed-off-by: remzi <13716567376yh@gmail.com> Co-authored-by: Jason Lowe --- .../src/main/python/array_test.py | 40 +++---- integration_tests/src/main/python/data_gen.py | 2 +- .../nvidia/spark/rapids/GpuOverrides.scala | 7 +- .../sql/rapids/collectionOperations.scala | 103 +++++++++++++++++- 4 files changed, 117 insertions(+), 35 deletions(-) diff --git a/integration_tests/src/main/python/array_test.py b/integration_tests/src/main/python/array_test.py index c427366638f..58c05f7bf4d 100644 --- a/integration_tests/src/main/python/array_test.py +++ b/integration_tests/src/main/python/array_test.py @@ -307,16 +307,26 @@ def do_it(spark): assert_gpu_and_cpu_are_equal_collect(do_it) # TODO add back in string_gen when https://github.com/rapidsai/cudf/issues/9156 is fixed -array_min_max_gens_no_nan = [byte_gen, short_gen, int_gen, long_gen, FloatGen(no_nans=True), DoubleGen(no_nans=True), +array_min_max_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, date_gen, timestamp_gen, null_gen] + decimal_gens -@pytest.mark.parametrize('data_gen', array_min_max_gens_no_nan, ids=idfn) -def test_array_min(data_gen): +@pytest.mark.parametrize('data_gen', array_min_max_gens, ids=idfn) +def test_array_min_max(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, ArrayGen(data_gen)).selectExpr( - 'array_min(a)'), - conf=no_nans_conf) + 'array_min(a)', 'array_max(a)')) +@pytest.mark.parametrize('data_gen', [ArrayGen(SetValuesGen(datatype, [math.nan, None])) for datatype in [FloatType(), DoubleType()]], ids=idfn) +def test_array_min_max_all_nans(data_gen): + assert_gpu_and_cpu_are_equal_collect( + lambda spark : unary_op_df(spark, data_gen).selectExpr( + 'array_min(a)', 'array_max(a)')) + +@pytest.mark.parametrize('data_gen', [ArrayGen(int_gen, all_null=True)], ids=idfn) +def test_array_min_max_all_nulls(data_gen): + assert_gpu_and_cpu_are_equal_collect( + lambda spark : unary_op_df(spark, data_gen).selectExpr( + 'array_min(a)', 'array_max(a)')) @pytest.mark.parametrize('data_gen', decimal_gens, ids=idfn) def test_array_concat_decimal(data_gen): @@ -325,26 +335,6 @@ def test_array_concat_decimal(data_gen): 'concat(a, a)')), conf=no_nans_conf) -@pytest.mark.parametrize('data_gen', [float_gen, double_gen], ids=idfn) -def test_array_max_with_nans(data_gen): - assert_gpu_and_cpu_are_equal_collect( - lambda spark : unary_op_df(spark, ArrayGen(data_gen)).selectExpr( - 'array_max(a)')) - -@pytest.mark.parametrize('data_gen', array_min_max_gens_no_nan, ids=idfn) -def test_array_max(data_gen): - assert_gpu_and_cpu_are_equal_collect( - lambda spark : unary_op_df(spark, ArrayGen(data_gen)).selectExpr( - 'array_max(a)'), - conf=no_nans_conf) - -@pytest.mark.parametrize('data_gen', [ArrayGen(gen, all_null=True) for gen in [int_gen, float_gen, double_gen]], ids=idfn) -def test_array_max_all_nulls(data_gen): - assert_gpu_and_cpu_are_equal_collect( - lambda spark : unary_op_df(spark, data_gen).selectExpr( - 'array_max(a)'), - conf=no_nans_conf) - @pytest.mark.parametrize('data_gen', orderable_gens + nested_gens_sample, ids=idfn) def test_array_repeat_with_count_column(data_gen): cnt_gen = IntegerGen(min_val=-5, max_val=5, special_cases=[]) diff --git a/integration_tests/src/main/python/data_gen.py b/integration_tests/src/main/python/data_gen.py index f034c39e6c3..db4b54b501f 100644 --- a/integration_tests/src/main/python/data_gen.py +++ b/integration_tests/src/main/python/data_gen.py @@ -307,7 +307,7 @@ def __init__(self, data_type, data): self._vals = data def __repr__(self): - return super().__repr__() + '(' + str(self._child) + ')' + return super().__repr__() +'(' + str(self.data_type) + ',' + str(self._vals) + ')' def start(self, rand): data = self._vals diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 50fcbd528de..33fd8cff3d3 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -2739,14 +2739,9 @@ object GpuOverrides extends Logging { ExprChecks.unaryProject( TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL, TypeSig.orderable, - TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL) - .withPsNote(Seq(TypeEnum.DOUBLE, TypeEnum.FLOAT), nanAggPsNote), + TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL), TypeSig.ARRAY.nested(TypeSig.orderable)), (in, conf, p, r) => new UnaryExprMeta[ArrayMin](in, conf, p, r) { - override def tagExprForGpu(): Unit = { - checkAndTagFloatNanAgg("Min", in.dataType, conf, this) - } - override def convertToGpu(child: Expression): GpuExpression = GpuArrayMin(child) }), diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala index 4618fe7db80..92cf18a9b79 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala @@ -423,7 +423,19 @@ case class GpuSortArray(base: Expression, ascendingOrder: Expression) } } -case class GpuArrayMin(child: Expression) extends GpuUnaryExpression with ImplicitCastInputTypes { +object GpuArrayMin { + def apply(child: Expression): GpuArrayMin = { + child.dataType match { + case ArrayType(FloatType | DoubleType, _) => GpuFloatArrayMin(child) + case ArrayType(_, _) => GpuBasicArrayMin(child) + case _ => throw new IllegalStateException(s"array_min accepts only arrays.") + } + } +} + +abstract class GpuArrayMin(child: Expression) extends GpuUnaryExpression + with ImplicitCastInputTypes + with Serializable { override def nullable: Boolean = true @@ -440,6 +452,91 @@ case class GpuArrayMin(child: Expression) extends GpuUnaryExpression with Implic input.getBase.listReduce(SegmentedReductionAggregation.min()) } +/** ArrayMin without `Nan` handling */ +case class GpuBasicArrayMin(child: Expression) extends GpuArrayMin(child) + +/** ArrayMin for FloatType and DoubleType to handle `Nan`s. + * + * In Spark, `Nan` is the max float value, however in cuDF, the calculation + * involving `Nan` is undefined. + * We design a workaround method here to match the Spark's behaviour. + * The high level idea is: + * if one list contains only `Nan`s or `null`s + * then + if the list contains `Nan` + * then return `Nan` + * else return null + * else + * replace all `Nan`s with nulls; + * use cuDF kernel to find the min value + */ +case class GpuFloatArrayMin(child: Expression) extends GpuArrayMin(child) { + @transient override lazy val dataType: DataType = child.dataType match { + case ArrayType(FloatType, _) => FloatType + case ArrayType(DoubleType, _) => DoubleType + case _ => throw new IllegalStateException( + s"GpuFloatArrayMin accepts only float array and double array." + ) + } + + protected def getNanScalar: Scalar = dataType match { + case FloatType => Scalar.fromFloat(Float.NaN) + case DoubleType => Scalar.fromDouble(Double.NaN) + case t => throw new IllegalStateException(s"dataType $t is not FloatType or DoubleType") + } + + protected def getNullScalar: Scalar = dataType match { + case FloatType => Scalar.fromNull(DType.FLOAT32) + case DoubleType => Scalar.fromNull(DType.FLOAT64) + case t => throw new IllegalStateException(s"dataType $t is not FloatType or DoubleType") + } + + override protected def doColumnar(input: GpuColumnVector): cudf.ColumnVector = { + val listAll = SegmentedReductionAggregation.all() + val listAny = SegmentedReductionAggregation.max() + val base = input.getBase() + + withResource(base.getChildColumnView(0)) { child => + withResource(child.isNan()){ childIsNan => + // if all values in each list are nans or nulls + val allNanOrNull = { + val childIsNanOrNull = withResource(child.isNull()) {_.or(childIsNan)} + val nanOrNullList = withResource(childIsNanOrNull) {base.replaceListChild(_)} + withResource(nanOrNullList) {_.listReduce(listAll)} + } + withResource(allNanOrNull){ allNanOrNull => + // return nan if the list contains nan, else return null + val trueOption = { + val anyNan = withResource(base.replaceListChild(childIsNan)) { + _.listReduce(listAny) + } + withResource(anyNan) { anyNan => + withResource(getNanScalar) { nanScalar => + withResource(getNullScalar) { nullScalar => + anyNan.ifElse(nanScalar, nullScalar) + } + } + } + } + withResource(trueOption){ trueOption => + // replace all nans to nulls, and then find the min value. + val falseOption = withResource(child.nansToNulls()) { nanToNullChild => + withResource(base.replaceListChild(nanToNullChild)) { nanToNullList => + nanToNullList.listReduce(SegmentedReductionAggregation.min()) + } + } + // if a list contains values other than nan or null + // return `trueOption`, else return `falseOption`. + withResource(falseOption){ falseOption => + allNanOrNull.ifElse(trueOption, falseOption) + } + } + } + } + } + } +} + object GpuArrayMax { def apply(child: Expression): GpuArrayMax = { child.dataType match { @@ -472,12 +569,12 @@ abstract class GpuArrayMax(child: Expression) extends GpuUnaryExpression /** ArrayMax without `NaN` handling */ case class GpuBasicArrayMax(child: Expression) extends GpuArrayMax(child) -/** ArrayMax for FloatType and DoubleType to handle `NaN`s. +/** ArrayMax for FloatType and DoubleType to handle `Nan`s. * * In Spark, `Nan` is the max float value, however in cuDF, the calculation * involving `Nan` is undefined. * We design a workaround method here to match the Spark's behaviour. - * The high level idea is that, we firstly check if each array contains `Nan`. + * The high level idea is that, we firstly check if each list contains `Nan`. * If it is, the max value is `Nan`, else we use the cuDF kernel to * calculate the max value. */ From 2c954f5c6445dcbf7434eaa65e40300914fadc56 Mon Sep 17 00:00:00 2001 From: Sameer Raheja Date: Sun, 21 Aug 2022 23:56:26 -0700 Subject: [PATCH 044/190] Remove anthony-chang (#6374) Signed-off-by: Sameer Raheja Signed-off-by: Sameer Raheja --- .github/workflows/blossom-ci.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/blossom-ci.yml b/.github/workflows/blossom-ci.yml index ebfe422a75d..e115736460b 100644 --- a/.github/workflows/blossom-ci.yml +++ b/.github/workflows/blossom-ci.yml @@ -65,7 +65,6 @@ jobs: NVnavkumar,\ amahussein,\ mattahrens,\ - anthony-chang,\ sinkinben,\ thirtiseven,\ ', format('{0},', github.actor)) && github.event.comment.body == 'build' From 294d018e22803f9fd3d016ec72c714509405ffd2 Mon Sep 17 00:00:00 2001 From: Peixin Date: Tue, 23 Aug 2022 20:32:59 +0800 Subject: [PATCH 045/190] Add zip&unzip in pre-merge dockerfile (#6393) Signed-off-by: Peixin Li Signed-off-by: Peixin Li --- integration_tests/run_pyspark_from_build.sh | 1 + jenkins/Dockerfile-blossom.ubuntu | 2 +- jenkins/databricks/test.sh | 3 +++ 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/integration_tests/run_pyspark_from_build.sh b/integration_tests/run_pyspark_from_build.sh index 3ac660c5c45..838237c1143 100755 --- a/integration_tests/run_pyspark_from_build.sh +++ b/integration_tests/run_pyspark_from_build.sh @@ -25,6 +25,7 @@ then >&2 echo "SPARK_HOME IS NOT SET CANNOT RUN PYTHON INTEGRATION TESTS..." else echo "WILL RUN TESTS WITH SPARK_HOME: ${SPARK_HOME}" + [[ ! -x "$(command -v zip)" ]] && { echo "fail to find zip command in $PATH"; exit 1; } # Spark 3.1.1 includes https://github.com/apache/spark/pull/31540 # which helps with spurious task failures as observed in our tests. If you are running # Spark versions before 3.1.1, this sets the spark.max.taskFailures to 4 to allow for diff --git a/jenkins/Dockerfile-blossom.ubuntu b/jenkins/Dockerfile-blossom.ubuntu index cad575dec99..e30551c4a6d 100644 --- a/jenkins/Dockerfile-blossom.ubuntu +++ b/jenkins/Dockerfile-blossom.ubuntu @@ -45,7 +45,7 @@ RUN apt-get update -y && \ RUN add-apt-repository ppa:deadsnakes/ppa && \ apt-get update -y && \ DEBIAN_FRONTEND="noninteractive" apt-get install -y maven \ - openjdk-8-jdk openjdk-11-jdk python3.8 python3.8-distutils python3-setuptools tzdata git + openjdk-8-jdk openjdk-11-jdk python3.8 python3.8-distutils python3-setuptools tzdata git zip unzip RUN python3.8 -m easy_install pip # Set default jdk as 1.8.0 diff --git a/jenkins/databricks/test.sh b/jenkins/databricks/test.sh index 8cbbb4b526a..ec68bfbcea7 100755 --- a/jenkins/databricks/test.sh +++ b/jenkins/databricks/test.sh @@ -22,6 +22,9 @@ SPARK_CONF=${SPARK_CONF:-''} BASE_SPARK_VER=${BASE_SPARK_VER:-'3.1.2'} [[ -z $SPARK_SHIM_VER ]] && export SPARK_SHIM_VER=spark${BASE_SPARK_VER//.}db +# install required packages +sudo apt -y install zip unzip + # Try to use "cudf-udf" conda environment for the python cudf-udf tests. if [ -d "/databricks/conda/envs/cudf-udf" ]; then export PATH=/databricks/conda/envs/cudf-udf/bin:/databricks/conda/bin:$PATH From 295b3cafd8ae530d3e1726bd770fce6947b6b4ab Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Tue, 23 Aug 2022 21:38:12 +0800 Subject: [PATCH 046/190] Include filterblock time in scan time metric for Coalescing readers (#6355) * Include filterblock time in scan time metric for Coalescing readers * Fix scan time metric not exists * Avoided some map double-lookup Signed-off-by: thirtiseven --- .../src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala | 6 +++++- .../main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala | 4 ++++ .../scala/org/apache/spark/sql/rapids/GpuAvroScan.scala | 6 +++++- 3 files changed, 14 insertions(+), 2 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index 45fa3c3b320..4cca3c862b4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -21,7 +21,7 @@ import java.net.URI import java.nio.ByteBuffer import java.nio.channels.{Channels, WritableByteChannel} import java.util -import java.util.concurrent.Callable +import java.util.concurrent.{Callable, TimeUnit} import scala.annotation.tailrec import scala.collection.JavaConverters._ @@ -316,6 +316,7 @@ case class GpuOrcMultiFilePartitionReaderFactory( // we must split the different compress files into different ColumnarBatch. // So here try the best to group the same compression files together before hand. val compressionAndStripes = LinkedHashMap[CompressionKind, ArrayBuffer[OrcSingleStripeMeta]]() + val currentTime = System.nanoTime() files.map { file => val orcPartitionReaderContext = filterHandler.filterStripes(file, dataSchema, readDataSchema, partitionSchema) @@ -329,6 +330,9 @@ case class GpuOrcMultiFilePartitionReaderFactory( OrcSchemaWrapper(orcPartitionReaderContext.updatedReadSchema), OrcExtraInfo(orcPartitionReaderContext.requestedMapping))) } + metrics.get("scanTime").foreach { + _ += TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - currentTime) + } val clippedStripes = compressionAndStripes.values.flatten.toSeq new MultiFileOrcPartitionReader(conf, files, clippedStripes, readDataSchema, debugDumpPrefix, maxReadBatchSizeRows, maxReadBatchSizeBytes, metrics, partitionSchema, numThreads, diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index a7902f37104..ac12b021057 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -976,6 +976,7 @@ case class GpuParquetMultiFilePartitionReaderFactory( files: Array[PartitionedFile], conf: Configuration): PartitionReader[ColumnarBatch] = { val clippedBlocks = ArrayBuffer[ParquetSingleDataBlockMeta]() + val currentTime = System.nanoTime() files.map { file => val singleFileInfo = try { filterHandler.filterBlocks(footerReadType, file, conf, filters, readDataSchema) @@ -1003,6 +1004,9 @@ case class GpuParquetMultiFilePartitionReaderFactory( ParquetExtraInfo(singleFileInfo.isCorrectedRebaseMode, singleFileInfo.isCorrectedInt96RebaseMode, singleFileInfo.hasInt96Timestamps))) } + metrics.get("scanTime").foreach { + _ += TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - currentTime) + } new MultiFileParquetPartitionReader(conf, files, clippedBlocks, isCaseSensitive, readDataSchema, debugDumpPrefix, maxReadBatchSizeRows, maxReadBatchSizeBytes, metrics, diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala index 87ceaab9bc5..a5f79b2e12d 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.rapids import java.io.{FileNotFoundException, IOException, OutputStream} import java.net.URI -import java.util.concurrent.Callable +import java.util.concurrent.{Callable, TimeUnit} import scala.annotation.tailrec import scala.collection.JavaConverters.{asScalaBufferConverter, mapAsScalaMapConverter} @@ -248,6 +248,7 @@ case class GpuAvroMultiFilePartitionReaderFactory( val clippedBlocks = ArrayBuffer[AvroSingleDataBlockInfo]() val mapPathHeader = LinkedHashMap[Path, Header]() val filterHandler = AvroFileFilterHandler(conf, options) + val currentTime = System.nanoTime() files.foreach { file => val singleFileInfo = try { filterHandler.filterBlocks(file) @@ -275,6 +276,9 @@ case class GpuAvroMultiFilePartitionReaderFactory( mapPathHeader.put(fPath, singleFileInfo.header) } } + metrics.get("scanTime").foreach { + _ += TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - currentTime) + } new GpuMultiFileAvroPartitionReader(conf, files, clippedBlocks, readDataSchema, partitionSchema, maxReadBatchSizeRows, maxReadBatchSizeBytes, numThreads, debugDumpPrefix, metrics, mapPathHeader.toMap) From f7936b951d687204feb6b162390d29675f11366c Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Tue, 23 Aug 2022 11:35:54 -0500 Subject: [PATCH 047/190] Use new custom kernel for large decimal multiply (#6387) Signed-off-by: Robert (Bobby) Evans --- .../src/main/python/arithmetic_ops_test.py | 50 ++++++++++--------- .../nvidia/spark/rapids/GpuOverrides.scala | 19 ++----- .../apache/spark/sql/rapids/arithmetic.scala | 43 ++++++++++++++-- 3 files changed, 70 insertions(+), 42 deletions(-) diff --git a/integration_tests/src/main/python/arithmetic_ops_test.py b/integration_tests/src/main/python/arithmetic_ops_test.py index f86f6294180..49852491900 100644 --- a/integration_tests/src/main/python/arithmetic_ops_test.py +++ b/integration_tests/src/main/python/arithmetic_ops_test.py @@ -27,12 +27,17 @@ # No overflow gens here because we just focus on verifying the fallback to CPU when # enabling ANSI mode. But overflows will fail the tests because CPU runs raise # exceptions. -_no_overflow_multiply_gens = [ +_no_overflow_multiply_gens_for_fallback = [ ByteGen(min_val = 1, max_val = 10, special_cases=[]), ShortGen(min_val = 1, max_val = 100, special_cases=[]), IntegerGen(min_val = 1, max_val = 1000, special_cases=[]), LongGen(min_val = 1, max_val = 3000, special_cases=[])] + +_no_overflow_multiply_gens = _no_overflow_multiply_gens_for_fallback + [ + DecimalGen(10, 0), + DecimalGen(19, 0)] + _decimal_gen_7_7 = DecimalGen(precision=7, scale=7) _decimal_gen_18_0 = DecimalGen(precision=18, scale=0) _decimal_gen_18_3 = DecimalGen(precision=18, scale=3) @@ -114,19 +119,24 @@ def test_subtraction_ansi_no_overflow(data_gen): @pytest.mark.parametrize('data_gen', numeric_gens + [ decimal_gen_32bit_neg_scale, decimal_gen_32bit, _decimal_gen_7_7, - DecimalGen(precision=8, scale=8), decimal_gen_64bit, _decimal_gen_18_3], ids=idfn) + DecimalGen(precision=8, scale=8), decimal_gen_64bit, _decimal_gen_18_3, + _decimal_gen_38_10, + _decimal_gen_38_neg10 + ], ids=idfn) def test_multiplication(data_gen): data_type = data_gen.data_type assert_gpu_and_cpu_are_equal_collect( lambda spark : binary_op_df(spark, data_gen).select( + f.col('a'), f.col('b'), f.col('a') * f.lit(100).cast(data_type), f.lit(-12).cast(data_type) * f.col('b'), f.lit(None).cast(data_type) * f.col('a'), f.col('b') * f.lit(None).cast(data_type), - f.col('a') * f.col('b'))) + f.col('a') * f.col('b') + )) @allow_non_gpu('ProjectExec', 'Alias', 'Multiply', 'Cast') -@pytest.mark.parametrize('data_gen', _no_overflow_multiply_gens, ids=idfn) +@pytest.mark.parametrize('data_gen', _no_overflow_multiply_gens_for_fallback, ids=idfn) def test_multiplication_fallback_when_ansi_enabled(data_gen): assert_gpu_fallback_collect( lambda spark : binary_op_df(spark, data_gen).select( @@ -134,7 +144,7 @@ def test_multiplication_fallback_when_ansi_enabled(data_gen): 'Multiply', conf=ansi_enabled_conf) -@pytest.mark.parametrize('data_gen', [float_gen, double_gen, decimal_gen_32bit], ids=idfn) +@pytest.mark.parametrize('data_gen', [float_gen, double_gen, decimal_gen_32bit, DecimalGen(19, 0)], ids=idfn) def test_multiplication_ansi_enabled(data_gen): data_type = data_gen.data_type assert_gpu_and_cpu_are_equal_collect( @@ -143,8 +153,18 @@ def test_multiplication_ansi_enabled(data_gen): f.col('a') * f.col('b')), conf=ansi_enabled_conf) -@pytest.mark.parametrize('lhs', [byte_gen, short_gen, int_gen, long_gen, DecimalGen(6, 5), DecimalGen(6, 4), DecimalGen(5, 4), DecimalGen(5, 3), DecimalGen(4, 2), DecimalGen(3, -2), DecimalGen(16, 7)], ids=idfn) -@pytest.mark.parametrize('rhs', [byte_gen, short_gen, int_gen, long_gen, DecimalGen(6, 3), DecimalGen(10, -2), DecimalGen(15, 3)], ids=idfn) +def test_multiplication_ansi_overflow(): + exception_str = 'ArithmeticException' + assert_gpu_and_cpu_error( + lambda spark : unary_op_df(spark, DecimalGen(38, 0)).selectExpr("a * " + "9"*38 + " as ret").collect(), + ansi_enabled_conf, + exception_str) + +@pytest.mark.parametrize('lhs', [byte_gen, short_gen, int_gen, long_gen, DecimalGen(6, 5), DecimalGen(6, 4), DecimalGen(5, 4), DecimalGen(5, 3), + DecimalGen(4, 2), DecimalGen(3, -2), DecimalGen(16, 7), DecimalGen(19, 0), + DecimalGen(30, 10)], ids=idfn) +@pytest.mark.parametrize('rhs', [byte_gen, short_gen, int_gen, long_gen, DecimalGen(6, 3), DecimalGen(10, -2), DecimalGen(15, 3), + DecimalGen(30, 12), DecimalGen(3, -3), DecimalGen(27, 7), DecimalGen(20, -3)], ids=idfn) def test_multiplication_mixed(lhs, rhs): assert_gpu_and_cpu_are_equal_collect( lambda spark : two_col_df(spark, lhs, rhs).select( @@ -212,22 +232,6 @@ def test_decimal_division_mixed_no_overflow_guarantees(lhs, lhs_type, rhs, rhs_t .select(f.col('lhs'), f.col('rhs'), f.col('lhs') / f.col('rhs')), conf={'spark.rapids.sql.decimalOverflowGuarantees': 'false'}) -@ignore_order -@pytest.mark.parametrize('rhs,rhs_type', [ - (DecimalGen(15, 3), DecimalType(30, 10)), - (DecimalGen(10, 2), DecimalType(28, 9))], ids=idfn) -@pytest.mark.parametrize('lhs,lhs_type', [ - (DecimalGen(10, 3), DecimalType(27, 7)), - (DecimalGen(3, -3), DecimalType(20, -3))], ids=idfn) -def test_decimal_multiplication_mixed_no_overflow_guarantees(lhs, lhs_type, rhs, rhs_type): - assert_gpu_and_cpu_are_equal_collect( - lambda spark : two_col_df(spark, lhs, rhs)\ - .withColumn('lhs', f.col('a').cast(lhs_type))\ - .withColumn('rhs', f.col('b').cast(rhs_type))\ - .repartition(1)\ - .select(f.col('lhs'), f.col('rhs'), f.col('lhs') * f.col('rhs')), - conf={'spark.rapids.sql.decimalOverflowGuarantees': 'false'}) - @pytest.mark.parametrize('data_gen', integral_gens + [ decimal_gen_32bit, decimal_gen_64bit, _decimal_gen_7_7, _decimal_gen_18_3, _decimal_gen_30_2, _decimal_gen_36_5, _decimal_gen_38_0], ids=idfn) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 33fd8cff3d3..0784174dfe2 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -996,6 +996,8 @@ object GpuOverrides extends Logging { // Here we try to strip out the extra casts, etc to get to as close to the original // query as possible. This lets us then calculate what CUDF needs to get the correct // answer, which in some cases is a lot smaller. + // For multiply we can support all of the types. For divide we still have to fall back + // to the CPU in some cases. case _: Divide => val intermediatePrecision = GpuDecimalDivide.nonRoundedIntermediateArgPrecision(lhsDecimalType, @@ -1012,21 +1014,6 @@ object GpuOverrides extends Logging { s"intermediate precision of $intermediatePrecision") } } - case _: Multiply => - val intermediatePrecision = - GpuDecimalMultiply.nonRoundedIntermediatePrecision(lhsDecimalType, - rhsDecimalType, a.dataType) - if (intermediatePrecision > DType.DECIMAL128_MAX_PRECISION) { - if (conf.needDecimalGuarantees) { - binExpr.willNotWorkOnGpu(s"the intermediate precision of " + - s"$intermediatePrecision that is required to guarantee no overflow issues " + - s"for this multiply is too large to be supported on the GPU") - } else { - logWarning("Decimal overflow guarantees disabled for " + - s"${lhs.dataType} * ${rhs.dataType} produces ${a.dataType} with an " + - s"intermediate precision of $intermediatePrecision") - } - } case _ => // NOOP } } @@ -1042,7 +1029,7 @@ object GpuOverrides extends Logging { GpuDecimalMultiply.nonRoundedIntermediatePrecision(lhsDecimalType, rhsDecimalType, a.dataType) GpuDecimalMultiply(lhs.convertToGpu(), rhs.convertToGpu(), wrapped.dataType, - needsExtraOverflowChecks = intermediatePrecision > DType.DECIMAL128_MAX_PRECISION) + useLongMultiply = intermediatePrecision > DType.DECIMAL128_MAX_PRECISION) case _ => GpuCheckOverflow(childExprs.head.convertToGpu(), wrapped.dataType, wrapped.nullOnOverflow) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala index 0411aeafba6..6945296b7bb 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala @@ -394,7 +394,7 @@ case class GpuDecimalMultiply( left: Expression, right: Expression, dataType: DecimalType, - needsExtraOverflowChecks: Boolean = false, + useLongMultiply: Boolean = false, failOnError: Boolean = SQLConf.get.ansiEnabled) extends ShimExpression with GpuExpression { @@ -409,7 +409,7 @@ case class GpuDecimalMultiply( private[this] lazy val intermediateResultType = GpuDecimalMultiply.intermediateResultType(lhsType, rhsType, dataType) - override def columnarEval(batch: ColumnarBatch): Any = { + def regularMultiply(batch: ColumnarBatch): Any = { val castLhs = withResource(GpuExpressionsUtils.columnarEvalToColumn(left, batch)) { lhs => GpuCast.doCast(lhs.getBase, lhs.dataType(), intermediateLhsType, ansiMode = failOnError, legacyCastToString = false, stringToDateAnsiModeEnabled = false) @@ -422,7 +422,7 @@ case class GpuDecimalMultiply( withResource(castRhs) { castRhs => withResource(castLhs.mul(castRhs, GpuColumnVector.getNonNestedRapidsType(intermediateResultType))) { mult => - if (needsExtraOverflowChecks) { + if (useLongMultiply) { withResource(GpuDecimalMultiply.checkForOverflow(castLhs, castRhs)) { wouldOverflow => if (failOnError) { withResource(wouldOverflow.any()) { anyOverflow => @@ -450,6 +450,43 @@ case class GpuDecimalMultiply( } } + def longMultiply(batch: ColumnarBatch): Any = { + val castLhs = withResource(GpuExpressionsUtils.columnarEvalToColumn(left, batch)) { lhs => + lhs.getBase.castTo(DType.create(DType.DTypeEnum.DECIMAL128, lhs.getBase.getType.getScale)) + } + val retTab = withResource(castLhs) { castLhs => + val castRhs = withResource(GpuExpressionsUtils.columnarEvalToColumn(right, batch)) { rhs => + rhs.getBase.castTo(DType.create(DType.DTypeEnum.DECIMAL128, rhs.getBase.getType.getScale)) + } + withResource(castRhs) { castRhs => + com.nvidia.spark.rapids.jni.DecimalUtils.multiply128(castLhs, castRhs, -dataType.scale) + } + } + val retCol = withResource(retTab) { retTab => + if (failOnError) { + withResource(retTab.getColumn(0).any()) { anyOverflow => + if (anyOverflow.isValid && anyOverflow.getBoolean) { + throw new ArithmeticException(GpuCast.INVALID_INPUT_MESSAGE) + } + } + retTab.getColumn(1).incRefCount() + } else { + withResource(GpuScalar.from(null, dataType)) { nullVal => + retTab.getColumn(0).ifElse(nullVal, retTab.getColumn(1)) + } + } + } + GpuColumnVector.from(retCol, dataType) + } + + override def columnarEval(batch: ColumnarBatch): Any = { + if (useLongMultiply) { + longMultiply(batch) + } else { + regularMultiply(batch) + } + } + override def nullable: Boolean = left.nullable || right.nullable override def children: Seq[Expression] = Seq(left, right) From 3367396fe46dffd43b0b11a54b5bf4c62f4c74b4 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Tue, 23 Aug 2022 15:01:41 -0700 Subject: [PATCH 048/190] Support _ in spark conf of integration tests (#6358) Fixes #6351 - Allow escaping _ in Spark conf by duplicating _ to allow running Iceberg/ pytests with xdist #### Additional refactoring and fixes - Remove a stale comment from spark-tests.sh - Allow overriding PYSP_TEST_spark_rapids_memory_gpu_allocSize to be able to run multiple Spark apps with multiple executors per pytest session with TEST_PARALLEL >= 2 on a small GPU - Add support for --packages and --jars in xdist using findspark - Fix xdist failing to init findspark when pyspark already on PYTHONPATH - Enforce findspark usage with xdist - don't start Spark and allocate GPU resources for xdist master - remove direct imports of pyspark from conftest to make sure it happens after session start Will file a follow-up issue to convert non-xdist tests relying on `SPARK_SUBMIT_ARGS` for external sources such as Delta and Iceberg to xdist . The following works locally: ```bash export TEST_PARALLEL=2 export SPARK_HOME=~/dist/spark-3.2.2-bin-hadoop3.2 export PYSP_TEST_spark_master="spark://$(hostname):7077" export PYSP_TEST_spark_executor_cores="2" export PYSP_TEST_spark_cores_max="4" export PYSP_TEST_spark_jars_packages="org.apache.iceberg:iceberg-spark-runtime-3.2_2.12:0.13.2" export PYSP_TEST_spark_sql_extensions="org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions" export PYSP_TEST_spark_sql_catalog_spark__catalog="org.apache.iceberg.spark.SparkSessionCatalog" export PYSP_TEST_spark_sql_catalog_spark__catalog_type="hadoop" export PYSP_TEST_spark_sql_catalog_spark__catalog_warehouse="$(mktemp -d -p /tmp spark-warehouse-XXXXXX)" export PYSP_TEST_spark_rapids_memory_gpu_allocSize='512m' ./integration_tests/run_pyspark_from_build.sh -s -m iceberg --iceberg ``` Signed-off-by: Gera Shegalov --- integration_tests/README.md | 86 +++++++++--------- integration_tests/run_pyspark_from_build.sh | 24 +++-- integration_tests/src/main/python/conftest.py | 17 +++- .../src/main/python/spark_init_internal.py | 90 +++++++++++++++---- .../src/main/python/spark_session.py | 6 +- jenkins/spark-tests.sh | 3 - 6 files changed, 151 insertions(+), 75 deletions(-) diff --git a/integration_tests/README.md b/integration_tests/README.md index c1151a7d02e..7da0ec89ca9 100644 --- a/integration_tests/README.md +++ b/integration_tests/README.md @@ -37,20 +37,20 @@ It is recommended that you use `pyenv` to manage Python installations. - Follow instructions to use the right method of installation described [here](https://github.com/pyenv/pyenv#installation) - Verify that `pyenv` is set correctly - + ```shell script - which pyenv + which pyenv ``` - Using `pyenv` to set Python installation - To check versions to be installed (will return a long list) - + ```shell script ls ~/.pyenv/versions/ ``` - To install a specific version from the available list - + ```shell script pyenv install 3.X.Y ``` @@ -116,7 +116,7 @@ You can install all the dependencies using `pip` by running the following comman ### Installing Spark You need to install spark-3.x and set `$SPARK_HOME/bin` to your `$PATH`, where -`SPARK_HOME` points to the directory of a runnable Spark distribution. +`SPARK_HOME` points to the directory of a runnable Spark distribution. This can be done in the following three steps: 1. Choose the appropriate way to create Spark distribution: @@ -156,10 +156,10 @@ Make sure that you compile the plugin against the same version of Spark that it Tests will run as a part of the maven build if you have the environment variable `SPARK_HOME` set. The suggested way to run these tests is to use the shell-script file located in the - integration_tests folder called [run_pyspark_from_build.sh](run_pyspark_from_build.sh). This script takes -care of some of the flags that are required to run the tests which will have to be set for the -plugin to work. It will be very useful to read the contents of the -[run_pyspark_from_build.sh](run_pyspark_from_build.sh) to get a better insight + integration_tests folder called [run_pyspark_from_build.sh](run_pyspark_from_build.sh). This script takes +care of some of the flags that are required to run the tests which will have to be set for the +plugin to work. It will be very useful to read the contents of the +[run_pyspark_from_build.sh](run_pyspark_from_build.sh) to get a better insight into what is needed as we constantly keep working on to improve and expand the plugin-support. The python tests run with pytest and the script honors pytest parameters. Some handy flags are: @@ -221,16 +221,18 @@ To run the tests separate from the build go to the `integration_tests` directory `runtests.py` through `spark-submit`, but if you want to run the tests in parallel with `pytest-xdist` you will need to submit it as a regular python application and have `findspark` installed. Be sure to include the necessary jars for the RAPIDS plugin either with -`spark-submit` or with the cluster when it is +`spark-submit` or with the cluster when it is [setup](../docs/get-started/getting-started-on-prem.md). -The command line arguments to `runtests.py` are the same as for +The command line arguments to `runtests.py` are the same as for [pytest](https://docs.pytest.org/en/latest/usage.html). The only reason we have a separate script is that `spark-submit` uses python if the file name ends with `.py`. If you want to configure the Spark cluster you may also set environment variables for the tests. The name of the env var should be in the form `"PYSP_TEST_" + conf_key.replace('.', '_')`. Linux -does not allow '.' in the name of an environment variable so we replace it with an underscore. As -Spark configs avoid this character we have no other special processing. +does not allow '.' in the name of an environment variable so we replace it with an underscore. If +the property contains an underscore, substitute '__' for each original '_'. +For example, `spark.sql.catalog.spark_catalog` is represented by the environment variable +`PYSP_TEST_spark_sql_catalog_spark__catalog`. We also have a large number of integration tests that currently run as a part of the unit tests using scala test. Those are in the `src/test/scala` sub-directory and depend on the testing @@ -252,7 +254,7 @@ It is recommended that you use `spark-shell` and the scalatest shell to run each individually, so you don't risk running unit tests along with the integration tests. http://www.scalatest.org/user_guide/using_the_scalatest_shell -```shell +```shell spark-shell --jars rapids-4-spark-tests_2.12-22.10.0-SNAPSHOT-tests.jar,rapids-4-spark-integration-tests_2.12-22.10.0-SNAPSHOT-tests.jar,scalatest_2.12-3.0.5.jar,scalactic_2.12-3.0.5.jar ``` @@ -366,8 +368,8 @@ cudf_udf tests needs a couple of different settings, they may need to run separa To enable cudf_udf tests, need following pre requirements: * Install cuDF Python library on all the nodes running executors. The instruction could be found at [here](https://rapids.ai/start.html). Please follow the steps to choose the version based on your environment and install the cuDF library via Conda or use other ways like building from source. * Disable the GPU exclusive mode on all the nodes running executors. The sample command is `sudo nvidia-smi -c DEFAULT` - -To run cudf_udf tests, need following configuration changes: + +To run cudf_udf tests, need following configuration changes: * Add configurations `--py-files` and `spark.executorEnv.PYTHONPATH` to specify the plugin jar for python modules 'rapids/daemon' 'rapids/worker'. * Decrease `spark.rapids.memory.gpu.allocFraction` to reserve enough GPU memory for Python processes in case of out-of-memory. * Add `spark.rapids.python.concurrentPythonWorkers` and `spark.rapids.python.memory.gpu.allocFraction` to reserve enough GPU memory for Python processes in case of out-of-memory. @@ -380,7 +382,7 @@ $SPARK_HOME/bin/spark-submit --jars "rapids-4-spark_2.12-22.10.0-SNAPSHOT-cuda11 ### Enabling fuzz tests -Fuzz tests are intended to find more corner cases in testing. We disable them by default because they might randomly fail. +Fuzz tests are intended to find more corner cases in testing. We disable them by default because they might randomly fail. The tests can be enabled by appending the option `--fuzz_test` to the command. * `--fuzz_test` (enable the fuzz tests when provided, and remove this option if you want to disable the tests) @@ -459,33 +461,33 @@ When support for a new operator is added to the Rapids Accelerator for Spark, or to support more data types, it is recommended that the following conditions be covered in its corresponding integration tests: ### 1. Cover all supported data types -Ensure that tests cover all data types supported by the added operation. An exhaustive list of data types supported in +Ensure that tests cover all data types supported by the added operation. An exhaustive list of data types supported in Apache Spark is available [here](https://spark.apache.org/docs/latest/sql-ref-datatypes.html). These include: - * Numeric Types - * `ByteType` - * `ShortType` + * Numeric Types + * `ByteType` + * `ShortType` * `IntegerType` * `LongType` * `FloatType` * `DoubleType` * `DecimalType` - * Strings - * `StringType` + * Strings + * `StringType` * `VarcharType` - * Binary (`BinaryType`) + * Binary (`BinaryType`) * Booleans (`BooleanType`) - * Chrono Types - * `TimestampType` + * Chrono Types + * `TimestampType` * `DateType` * `Interval` - * Complex Types + * Complex Types * `ArrayType` * `StructType` * `MapType` `data_gen.py` provides `DataGen` classes that help generate test data in integration tests. -The `assert_gpu_and_cpu_are_equal_collect()` function from `asserts.py` may be used to compare that an operator in +The `assert_gpu_and_cpu_are_equal_collect()` function from `asserts.py` may be used to compare that an operator in the Rapids Accelerator produces the same results as Apache Spark, for a test query. For data types that are not currently supported for an operator in the Rapids Accelerator, @@ -505,17 +507,17 @@ E.g. The `ArrayGen` and `StructGen` classes in `data_gen.py` can be configured to support arbitrary nesting. ### 3. Literal (i.e. Scalar) values -Operators and expressions that support literal operands need to be tested with literal inputs, of all -supported types from 1 and 2, above. +Operators and expressions that support literal operands need to be tested with literal inputs, of all +supported types from 1 and 2, above. For instance, `SUM()` supports numeric columns (e.g. `SUM(a + b)`), or scalars (e.g. `SUM(20)`). Similarly, `COUNT()` supports the following: * Columns: E.g. `COUNT(a)` to count non-null rows for column `a` * Scalars: E.g. `COUNT(1)` to count all rows (including nulls) * `*`: E.g. `COUNT(*)`, functionally equivalent to `COUNT(1)` It is advised that tests be added for all applicable literal types, for an operator. - + Note that for most operations, if all inputs are literal values, the Spark Catalyst optimizer will evaluate -the expression during the logical planning phase of query compilation, via +the expression during the logical planning phase of query compilation, via [Constant Folding](https://jaceklaskowski.gitbooks.io/mastering-spark-sql/content/spark-sql-Optimizer-ConstantFolding.html) E.g. Consider this query: ```sql @@ -529,18 +531,18 @@ need not necessarily add more test coverage. Ensure that the test data accommodates null values for input columns. This includes null values in columns and in literal inputs. -Null values in input columns are a frequent source of bugs in the Rapids Accelerator for Spark, -because of mismatches in null-handling and semantics, between RAPIDS `libcudf` (on which -the Rapids Accelerator relies heavily), and Apache Spark. +Null values in input columns are a frequent source of bugs in the Rapids Accelerator for Spark, +because of mismatches in null-handling and semantics, between RAPIDS `libcudf` (on which +the Rapids Accelerator relies heavily), and Apache Spark. -Tests for aggregations (including group-by, reductions, and window aggregations) should cover cases where +Tests for aggregations (including group-by, reductions, and window aggregations) should cover cases where some rows are null, and where *all* input rows are null. Apart from null rows in columns of primitive types, the following conditions must be covered for nested types: * Null rows at the "top" level for `Array`/`Struct` columns. E.g. `[ [1,2], [3], ∅, [4,5,6] ]`. * Non-null rows containing null elements in the child column. E.g. `[ [1,2], [3,∅], ∅, [4,∅,6] ]`. - * All null rows at a nested level. E.g. + * All null rows at a nested level. E.g. * All null list rows: `[ ∅, ∅, ∅, ∅ ]` * All null elements within list rows: `[ [∅,∅], [∅,∅], [∅,∅], [∅,∅] ]` @@ -579,10 +581,10 @@ describes this with examples. Operations should be tested with multiple bit-repr The `FloatGen` and `DoubleGen` data generators in `integration_tests/src/main/python/data_gen.py` can be configured to generate the special float/double values mentioned above. -For most basic floating-point operations like addition, subtraction, multiplication, and division the plugin will +For most basic floating-point operations like addition, subtraction, multiplication, and division the plugin will produce a bit for bit identical result as Spark does. For some other functions (like `sin`, `cos`, etc.), the output may differ slightly, but remain within the rounding error inherent in floating-point calculations. Certain aggregations -might compound those differences. In those cases, the `@approximate_float` test annotation may be used to mark tests +might compound those differences. In those cases, the `@approximate_float` test annotation may be used to mark tests to use "approximate" comparisons for floating-point values. Refer to the "Floating Point" section of [compatibility.md](../docs/compatibility.md) for details. @@ -590,11 +592,11 @@ Refer to the "Floating Point" section of [compatibility.md](../docs/compatibilit ### 8. Special values in timestamp columns Ensure date/timestamp columns include dates before the [epoch](https://en.wikipedia.org/wiki/Epoch_(computing)). -Apache Spark supports dates/timestamps between `0001-01-01 00:00:00.000000` and `9999-12-31 23:59:59.999999`, but at +Apache Spark supports dates/timestamps between `0001-01-01 00:00:00.000000` and `9999-12-31 23:59:59.999999`, but at values close to the minimum value, the format used in Apache Spark causes rounding errors. To avoid such problems, it is recommended that the minimum value used in a test not actually equal `0001-01-01`. For instance, `0001-01-03` is acceptable. -It is advised that `DateGen` and `TimestampGen` classes from `data_gen.py` be used to generate valid -(proleptic Gregorian calendar) dates when testing operators that work on dates. This data generator respects +It is advised that `DateGen` and `TimestampGen` classes from `data_gen.py` be used to generate valid +(proleptic Gregorian calendar) dates when testing operators that work on dates. This data generator respects the valid boundaries for dates and timestamps. diff --git a/integration_tests/run_pyspark_from_build.sh b/integration_tests/run_pyspark_from_build.sh index 838237c1143..d3b9adacb57 100755 --- a/integration_tests/run_pyspark_from_build.sh +++ b/integration_tests/run_pyspark_from_build.sh @@ -259,7 +259,14 @@ else # If you want to change the amount of GPU memory allocated you have to change it here # and where TEST_PARALLEL is calculated - export PYSP_TEST_spark_rapids_memory_gpu_allocSize='1536m' + if [[ -n "${PYSP_TEST_spark_rapids_memory_gpu_allocSize}" ]]; then + >&2 echo "#### WARNING: using externally set" \ + "PYSP_TEST_spark_rapids_memory_gpu_allocSize" \ + "${PYSP_TEST_spark_rapids_memory_gpu_allocSize}." \ + "If needed permanently in CI please file an issue to accommodate" \ + "for new GPU memory requirements ####" + fi + export PYSP_TEST_spark_rapids_memory_gpu_allocSize=${PYSP_TEST_spark_rapids_memory_gpu_allocSize:-'1536m'} if ((${#TEST_PARALLEL_OPTS[@]} > 0)); then @@ -267,11 +274,17 @@ else else # We set the GPU memory size to be a constant value even if only running with a parallelism of 1 # because it helps us have consistent test runs. + jarOpts=() if [[ -n "$PYSP_TEST_spark_jars" ]]; then - # `spark.jars` is the same as `--jars`, e.g.: --jars a.jar,b.jar... - jarOpts=(--conf spark.jars="${PYSP_TEST_spark_jars}") - elif [[ -n "$PYSP_TEST_spark_driver_extraClassPath" ]]; then - jarOpts=(--driver-class-path "${PYSP_TEST_spark_driver_extraClassPath}") + jarOpts+=(--jars "${PYSP_TEST_spark_jars}") + fi + + if [[ -n "$PYSP_TEST_spark_jars_packages" ]]; then + jarOpts+=(--packages "${PYSP_TEST_spark_jars_packages}") + fi + + if [[ -n "$PYSP_TEST_spark_driver_extraClassPath" ]]; then + jarOpts+=(--driver-class-path "${PYSP_TEST_spark_driver_extraClassPath}") fi driverJavaOpts="$PYSP_TEST_spark_driver_extraJavaOptions" @@ -282,6 +295,7 @@ else unset PYSP_TEST_spark_driver_extraClassPath unset PYSP_TEST_spark_driver_extraJavaOptions unset PYSP_TEST_spark_jars + unset PYSP_TEST_spark_jars_packages unset PYSP_TEST_spark_rapids_memory_gpu_allocSize exec "$SPARK_HOME"/bin/spark-submit "${jarOpts[@]}" \ diff --git a/integration_tests/src/main/python/conftest.py b/integration_tests/src/main/python/conftest.py index 8b880985aa8..97b3be41ba8 100644 --- a/integration_tests/src/main/python/conftest.py +++ b/integration_tests/src/main/python/conftest.py @@ -15,8 +15,15 @@ import os import pytest import random -from spark_init_internal import get_spark_i_know_what_i_am_doing -from pyspark.sql.dataframe import DataFrame + +# TODO redo _spark stuff using fixtures +# +# Don't import pyspark / _spark directly in conftest globally +# import as a plugin to do a lazy per-pytest-session initialization +# +pytest_plugins = [ + 'spark_init_internal' +] _approximate_float_args = None @@ -252,6 +259,7 @@ def get_worker_id(request): @pytest.fixture def spark_tmp_path(request): + from spark_init_internal import get_spark_i_know_what_i_am_doing debug = request.config.getoption('debug_tmp_path') ret = request.config.getoption('tmp_path') if ret is None: @@ -282,6 +290,7 @@ def get(self): @pytest.fixture def spark_tmp_table_factory(request): + from spark_init_internal import get_spark_i_know_what_i_am_doing worker_id = get_worker_id(request) table_id = random.getrandbits(31) base_id = f'tmp_table_{worker_id}_{table_id}' @@ -300,6 +309,7 @@ def _get_jvm(spark): return spark.sparkContext._jvm def spark_jvm(): + from spark_init_internal import get_spark_i_know_what_i_am_doing return _get_jvm(get_spark_i_know_what_i_am_doing()) class MortgageRunner: @@ -309,6 +319,7 @@ def __init__(self, mortgage_format, mortgage_acq_path, mortgage_perf_path): self.mortgage_perf_path = mortgage_perf_path def do_test_query(self, spark): + from pyspark.sql.dataframe import DataFrame jvm_session = _get_jvm_session(spark) jvm = _get_jvm(spark) acq = self.mortgage_acq_path @@ -324,7 +335,7 @@ def do_test_query(self, spark): raise AssertionError('Not Supported Format {}'.format(self.mortgage_format)) return DataFrame(df, spark.getActiveSession()) - + @pytest.fixture(scope="session") def mortgage(request): mortgage_format = request.config.getoption("mortgage_format") diff --git a/integration_tests/src/main/python/spark_init_internal.py b/integration_tests/src/main/python/spark_init_internal.py index e36cc3d282b..3ba6c390c0d 100644 --- a/integration_tests/src/main/python/spark_init_internal.py +++ b/integration_tests/src/main/python/spark_init_internal.py @@ -12,34 +12,89 @@ # See the License for the specific language governing permissions and # limitations under the License. +import logging import os +import re +import sys -try: - import pyspark -except ImportError as error: - import findspark - findspark.init() - import pyspark +logging.basicConfig( + format="%(asctime)s %(levelname)-8s %(message)s", + level=logging.INFO, + datefmt="%Y-%m-%d %H:%M:%S", +) _CONF_ENV_PREFIX = 'PYSP_TEST_' _EXECUTOR_ENV_PREFIX = 'spark_executorEnv_' def env_for_conf(spark_conf_name): - return _CONF_ENV_PREFIX + spark_conf_name.replace('.', '_') + # escape underscores + escaped_conf = spark_conf_name.replace('_', r'__') + return _CONF_ENV_PREFIX + escaped_conf.replace('.', '_') def conf_for_env(env_name): conf_key = env_name[len(_CONF_ENV_PREFIX):] if conf_key.startswith(_EXECUTOR_ENV_PREFIX): res = _EXECUTOR_ENV_PREFIX.replace('_', '.') + conf_key[len(_EXECUTOR_ENV_PREFIX):] else: - res = conf_key.replace('_', '.') + # replace standalone underscores + res1 = re.sub(r'(? Date: Wed, 24 Aug 2022 08:52:47 +0800 Subject: [PATCH 049/190] Rebalance time of parallel stages for pre-merge CI (#6379) * Rebalance time parallel stages of pre-merge CI Signed-off-by: Peixin Li * Clean up comments and add a pytest sanity check for ci_1 * balance some mvn test to ci_2 * also update mvn retry logic of mvn-verify check to resolve ephermeral mvn 502 * add 320 install only to ci_1 * replace test w/ package Signed-off-by: Peixin Li --- .github/workflows/mvn-verify-check.yml | 4 ++-- .../src/main/python/explain_test.py | 3 +++ integration_tests/src/main/python/join_test.py | 3 +-- jenkins/spark-premerge-build.sh | 18 +++++++++--------- 4 files changed, 15 insertions(+), 13 deletions(-) diff --git a/.github/workflows/mvn-verify-check.yml b/.github/workflows/mvn-verify-check.yml index 44038eee414..ee981311582 100644 --- a/.github/workflows/mvn-verify-check.yml +++ b/.github/workflows/mvn-verify-check.yml @@ -68,7 +68,7 @@ jobs: - name: package aggregator check run: > - mvn -B package -pl aggregator -am + mvn -Dmaven.wagon.http.retryHandler.count=3 -B package -pl aggregator -am -P 'individual,pre-merge' -Dbuildver=${{ matrix.spark-version }} -DskipTests @@ -92,7 +92,7 @@ jobs: # includes RAT, code style and doc-gen checks of default shim - name: verify all modules with lowest-supported Spark version run: > - mvn -B verify + mvn -Dmaven.wagon.http.retryHandler.count=3 -B verify -P 'individual,pre-merge' -Dbuildver=${{ needs.get-noSnapshot-versions-from-dist.outputs.sparkHeadVersion }} -DskipTests diff --git a/integration_tests/src/main/python/explain_test.py b/integration_tests/src/main/python/explain_test.py index 53685b5e7c3..b84754a3d3f 100644 --- a/integration_tests/src/main/python/explain_test.py +++ b/integration_tests/src/main/python/explain_test.py @@ -20,6 +20,9 @@ from pyspark.sql.types import * from spark_session import with_cpu_session, with_gpu_session +# mark this test as ci_1 for mvn verify sanity check in pre-merge CI +pytestmark = pytest.mark.premerge_ci_1 + def create_df(spark, data_gen, left_length, right_length): left = binary_op_df(spark, data_gen, length=left_length) right = binary_op_df(spark, data_gen, length=right_length).withColumnRenamed("a", "r_a")\ diff --git a/integration_tests/src/main/python/join_test.py b/integration_tests/src/main/python/join_test.py index 50c1b1ca70e..83d8be09541 100644 --- a/integration_tests/src/main/python/join_test.py +++ b/integration_tests/src/main/python/join_test.py @@ -22,8 +22,7 @@ from marks import ignore_order, allow_non_gpu, incompat, validate_execs_in_gpu_plan from spark_session import with_cpu_session, with_spark_session -# Mark all tests in current file as premerge_ci_1 in order to be run in first k8s pod for parallel build premerge job -pytestmark = [pytest.mark.premerge_ci_1, pytest.mark.nightly_resource_consuming_test] +pytestmark = [pytest.mark.nightly_resource_consuming_test] all_join_types = ['Left', 'Right', 'Inner', 'LeftSemi', 'LeftAnti', 'Cross', 'FullOuter'] diff --git a/jenkins/spark-premerge-build.sh b/jenkins/spark-premerge-build.sh index 256f5fdf7e4..c91dffd2e65 100755 --- a/jenkins/spark-premerge-build.sh +++ b/jenkins/spark-premerge-build.sh @@ -48,9 +48,7 @@ mvn_verify() { $MVN_INSTALL_CMD -DskipTests -Dbuildver=313 [[ $BUILD_MAINTENANCE_VERSION_SNAPSHOTS == "true" ]] && $MVN_INSTALL_CMD -Dbuildver=314 - # don't skip tests - env -u SPARK_HOME $MVN_CMD -U -B $MVN_URM_MIRROR -Dbuildver=320 clean install $MVN_BUILD_ARGS \ - -Dpytest.TEST_TAGS='' -pl '!tools' + $MVN_INSTALL_CMD -DskipTests -Dbuildver=320 # enable UTF-8 for regular expression tests env -u SPARK_HOME LC_ALL="en_US.UTF-8" $MVN_CMD $MVN_URM_MIRROR -Dbuildver=320 test $MVN_BUILD_ARGS \ -Dpytest.TEST_TAGS='' -pl '!tools' \ @@ -130,15 +128,17 @@ ci_2() { $MVN_CMD -U -B $MVN_URM_MIRROR clean package $MVN_BUILD_ARGS -DskipTests=true export TEST_TAGS="not premerge_ci_1" export TEST_TYPE="pre-commit" - export TEST_PARALLEL=4 - # separate process to avoid OOM kill - TEST='conditionals_test or window_function_test' ./integration_tests/run_pyspark_from_build.sh - TEST_PARALLEL=5 TEST='struct_test or time_window_test' ./integration_tests/run_pyspark_from_build.sh - TEST='not conditionals_test and not window_function_test and not struct_test and not time_window_test' \ - ./integration_tests/run_pyspark_from_build.sh + export TEST_PARALLEL=5 + ./integration_tests/run_pyspark_from_build.sh + # enable avro test separately INCLUDE_SPARK_AVRO_JAR=true TEST='avro_test.py' ./integration_tests/run_pyspark_from_build.sh # export 'LC_ALL' to set locale with UTF-8 so regular expressions are enabled LC_ALL="en_US.UTF-8" TEST="regexp_test.py" ./integration_tests/run_pyspark_from_build.sh + + # put some mvn tests here to balance durations of parallel stages + echo "Run mvn package..." + env -u SPARK_HOME $MVN_CMD -U -B $MVN_URM_MIRROR -Dbuildver=320 clean package $MVN_BUILD_ARGS \ + -Dpytest.TEST_TAGS='' -pl '!tools' } From 995cb8a2eee10dd3cfe840c5dd3af53be2c368cf Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Wed, 24 Aug 2022 14:43:01 -0700 Subject: [PATCH 050/190] Remove empty unreferenced file unshimmed-spark311.txt (#6407) Remove empty unreferenced file unshimmed-spark311.txt Signed-off-by: Gera Shegalov --- dist/unshimmed-spark311.txt | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 dist/unshimmed-spark311.txt diff --git a/dist/unshimmed-spark311.txt b/dist/unshimmed-spark311.txt deleted file mode 100644 index e69de29bb2d..00000000000 From 1d1b2b635bb0cf8e9ce79a8cff646e28b99aa2a7 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Wed, 24 Aug 2022 16:57:41 -0500 Subject: [PATCH 051/190] Fix test_dpp_from_swizzled_hash_keys on CDH (#6409) Signed-off-by: Jason Lowe Signed-off-by: Jason Lowe --- integration_tests/src/main/python/dpp_test.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/integration_tests/src/main/python/dpp_test.py b/integration_tests/src/main/python/dpp_test.py index 124b1cd3b7e..ad73358704c 100644 --- a/integration_tests/src/main/python/dpp_test.py +++ b/integration_tests/src/main/python/dpp_test.py @@ -287,7 +287,10 @@ def setup_tables(spark): " PARTITIONED BY (dt date, hr string, mins string) STORED AS PARQUET") spark.sql("INSERT INTO {}(id,dt,hr,mins)".format(fact_table) + " SELECT 'somevalue', to_date('2022-01-01'), '11', '59'") - with_cpu_session(setup_tables) + with_cpu_session(setup_tables, conf={ + "hive.exec.dynamic.partition" : "true", + "hive.exec.dynamic.partition.mode" : "nonstrict" + }) assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.sql("SELECT COUNT(*) AS cnt FROM {} f".format(fact_table) + " LEFT JOIN (SELECT *, " + From 6fe3a7fab906ca5c2ee7786108887ce4f19ac38e Mon Sep 17 00:00:00 2001 From: Tim Liu Date: Thu, 25 Aug 2022 19:52:49 +0800 Subject: [PATCH 052/190] Make the variable "BASE_SPARK_VERSION" consistent [databricks] (#6396) * Make the variable "BASE_SPARK_VERSION" consistent [databricks] Make the variable "BASE_SPARK_VERSION" consistent among build and test scripts Signed-off-by: Tim Liu * copyright Signed-off-by: Tim Liu --- jenkins/databricks/run-tests.py | 4 ++-- jenkins/databricks/test.sh | 10 +++++----- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/jenkins/databricks/run-tests.py b/jenkins/databricks/run-tests.py index 67d07583e28..b7dcd8dfca6 100644 --- a/jenkins/databricks/run-tests.py +++ b/jenkins/databricks/run-tests.py @@ -1,4 +1,4 @@ -# Copyright (c) 2020-2021, NVIDIA CORPORATION. +# Copyright (c) 2020-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -36,7 +36,7 @@ def main(): subprocess.check_call(rsync_command, shell=True) ssh_command = "ssh -o StrictHostKeyChecking=no -o UserKnownHostsFile=/dev/null ubuntu@%s -p 2200 -i %s " \ - "'LOCAL_JAR_PATH=%s SPARK_CONF=%s BASE_SPARK_VER=%s bash %s %s 2>&1 | tee testout; " \ + "'LOCAL_JAR_PATH=%s SPARK_CONF=%s BASE_SPARK_VERSION=%s bash %s %s 2>&1 | tee testout; " \ "if [ ${PIPESTATUS[0]} -ne 0 ]; then false; else true; fi'" % \ (master_addr, params.private_key_file, params.jar_path, params.spark_conf, params.base_spark_pom_version, params.script_dest, ' '.join(params.script_args)) diff --git a/jenkins/databricks/test.sh b/jenkins/databricks/test.sh index ec68bfbcea7..5cb660ad2a0 100755 --- a/jenkins/databricks/test.sh +++ b/jenkins/databricks/test.sh @@ -19,8 +19,8 @@ set -ex LOCAL_JAR_PATH=${LOCAL_JAR_PATH:-''} SPARK_CONF=${SPARK_CONF:-''} -BASE_SPARK_VER=${BASE_SPARK_VER:-'3.1.2'} -[[ -z $SPARK_SHIM_VER ]] && export SPARK_SHIM_VER=spark${BASE_SPARK_VER//.}db +BASE_SPARK_VERSION=${BASE_SPARK_VERSION:-'3.1.2'} +[[ -z $SPARK_SHIM_VER ]] && export SPARK_SHIM_VER=spark${BASE_SPARK_VERSION//.}db # install required packages sudo apt -y install zip unzip @@ -37,7 +37,7 @@ export SPARK_HOME=/databricks/spark # change to not point at databricks confs so we don't conflict with their settings export SPARK_CONF_DIR=$PWD export PYTHONPATH=$SPARK_HOME/python:$SPARK_HOME/python/pyspark/:$SPARK_HOME/python/lib/py4j-0.10.9-src.zip -if [[ $BASE_SPARK_VER == "3.2.1" ]] +if [[ $BASE_SPARK_VERSION == "3.2.1" ]] then # Databricks Koalas can conflict with the actual Pandas version, so put site packages first export PYTHONPATH=/databricks/python3/lib/python3.8/site-packages:$PYTHONPATH @@ -68,7 +68,7 @@ if [ -n "$SPARK_CONF" ]; then fi IS_SPARK_311_OR_LATER=0 -[[ "$(printf '%s\n' "3.1.1" "$BASE_SPARK_VER" | sort -V | head -n1)" = "3.1.1" ]] && IS_SPARK_311_OR_LATER=1 +[[ "$(printf '%s\n' "3.1.1" "$BASE_SPARK_VERSION" | sort -V | head -n1)" = "3.1.1" ]] && IS_SPARK_311_OR_LATER=1 # TEST_MODE @@ -81,7 +81,7 @@ TEST_TYPE="nightly" PCBS_CONF="com.nvidia.spark.ParquetCachedBatchSerializer" ICEBERG_VERSION=${ICEBERG_VERSION:-0.13.2} -ICEBERG_SPARK_VER=$(echo $BASE_SPARK_VER | cut -d. -f1,2) +ICEBERG_SPARK_VER=$(echo $BASE_SPARK_VERSION | cut -d. -f1,2) # Classloader config is here to work around classloader issues with # --packages in distributed setups, should be fixed by # https://github.com/NVIDIA/spark-rapids/pull/5646 From b35311f7c6950fd5d8f7f6ed66aeffa87c480850 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Thu, 25 Aug 2022 08:54:20 -0500 Subject: [PATCH 053/190] Fix DecimalGen to generate full range and fix failing test cases (#6398) Signed-off-by: Jason Lowe --- docs/supported_ops.md | 2 +- .../src/main/python/arithmetic_ops_test.py | 29 +++++++++++++++++-- .../src/main/python/cache_test.py | 6 ++-- .../src/main/python/cast_test.py | 8 ++++- integration_tests/src/main/python/data_gen.py | 18 +++++++++--- .../src/main/python/expand_exec_test.py | 6 ++-- .../src/main/python/hash_aggregate_test.py | 23 +++++++++++++-- .../src/main/python/join_test.py | 5 ++-- integration_tests/src/main/python/orc_test.py | 21 +++++++++----- .../src/main/python/sort_test.py | 12 ++++++-- .../src/main/python/window_function_test.py | 2 +- .../nvidia/spark/rapids/GpuOverrides.scala | 11 ++++++- .../spark/sql/rapids/AggregateFunctions.scala | 19 ++++-------- tools/src/main/resources/supportedExprs.csv | 2 +- 14 files changed, 121 insertions(+), 43 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index ade5980d647..07873043402 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -10126,7 +10126,7 @@ are limited. -S +PS
decimals with precision 38 are not supported
diff --git a/integration_tests/src/main/python/arithmetic_ops_test.py b/integration_tests/src/main/python/arithmetic_ops_test.py index 49852491900..fcb9e7bf4d2 100644 --- a/integration_tests/src/main/python/arithmetic_ops_test.py +++ b/integration_tests/src/main/python/arithmetic_ops_test.py @@ -263,7 +263,14 @@ def test_mod(data_gen): f.col('b') % f.lit(None).cast(data_type), f.col('a') % f.col('b'))) -@pytest.mark.parametrize('data_gen', _arith_data_gens_no_neg_scale, ids=idfn) +# pmod currently falls back for Decimal(precision=38) +# https://github.com/NVIDIA/spark-rapids/issues/6336 +_pmod_gens = numeric_gens + [ decimal_gen_32bit, decimal_gen_64bit, _decimal_gen_18_0, decimal_gen_128bit, + _decimal_gen_30_2, _decimal_gen_36_5, + DecimalGen(precision=37, scale=0), DecimalGen(precision=37, scale=10), + _decimal_gen_7_7] + +@pytest.mark.parametrize('data_gen', _pmod_gens, ids=idfn) def test_pmod(data_gen): string_type = to_cast_string(data_gen.data_type) assert_gpu_and_cpu_are_equal_collect( @@ -274,6 +281,19 @@ def test_pmod(data_gen): 'pmod(b, cast(null as {}))'.format(string_type), 'pmod(a, b)')) +@allow_non_gpu("ProjectExec", "Pmod") +@pytest.mark.parametrize('data_gen', [_decimal_gen_38_0, _decimal_gen_38_10], ids=idfn) +def test_pmod_fallback(data_gen): + string_type = to_cast_string(data_gen.data_type) + assert_gpu_fallback_collect( + lambda spark : binary_op_df(spark, data_gen).selectExpr( + 'pmod(a, cast(100 as {}))'.format(string_type), + 'pmod(cast(-12 as {}), b)'.format(string_type), + 'pmod(cast(null as {}), a)'.format(string_type), + 'pmod(b, cast(null as {}))'.format(string_type), + 'pmod(a, b)'), + "Pmod") + # test pmod(Long.MinValue, -1) = 0 and Long.MinValue % -1 = 0, should not throw def test_mod_pmod_long_min_value(): assert_gpu_and_cpu_are_equal_collect( @@ -282,7 +302,10 @@ def test_mod_pmod_long_min_value(): 'a % -1L'), ansi_enabled_conf) -@pytest.mark.parametrize('data_gen', _arith_data_gens_diff_precision_scale_and_no_neg_scale, ids=idfn) +# pmod currently falls back for Decimal(precision=38) +# https://github.com/NVIDIA/spark-rapids/issues/6336 +@pytest.mark.parametrize('data_gen', [decimal_gen_32bit, decimal_gen_64bit, _decimal_gen_18_0, + decimal_gen_128bit, _decimal_gen_30_2, _decimal_gen_36_5], ids=idfn) @pytest.mark.parametrize('overflow_exp', [ 'pmod(a, cast(0 as {}))', 'pmod(cast(-12 as {}), cast(0 as {}))', @@ -318,7 +341,7 @@ def test_cast_neg_to_decimal_err(): ansi_enabled_conf, exception_type + exception_content) -@pytest.mark.parametrize('data_gen', _arith_data_gens_no_neg_scale, ids=idfn) +@pytest.mark.parametrize('data_gen', _pmod_gens, ids=idfn) def test_mod_pmod_by_zero_not_ansi(data_gen): string_type = to_cast_string(data_gen.data_type) assert_gpu_and_cpu_are_equal_collect( diff --git a/integration_tests/src/main/python/cache_test.py b/integration_tests/src/main/python/cache_test.py index cff9d4e7a07..8e37fb61d1c 100644 --- a/integration_tests/src/main/python/cache_test.py +++ b/integration_tests/src/main/python/cache_test.py @@ -26,7 +26,9 @@ enable_vectorized_confs = [{"spark.sql.inMemoryColumnarStorage.enableVectorizedReader": "true"}, {"spark.sql.inMemoryColumnarStorage.enableVectorizedReader": "false"}] -_cache_decimal_gens = [decimal_gen_32bit, decimal_gen_64bit, decimal_gen_128bit] +# Many tests sort the results, so use a sortable decimal generator as many Spark versions +# fail to sort some large decimals properly. +_cache_decimal_gens = [decimal_gen_32bit, decimal_gen_64bit, orderable_decimal_gen_128bit] _cache_single_array_gens_no_null = [ArrayGen(gen) for gen in all_basic_gens_no_null + _cache_decimal_gens] decimal_struct_gen= StructGen([['child0', sub_gen] for ind, sub_gen in enumerate(_cache_decimal_gens)]) @@ -166,7 +168,7 @@ def n_fold(spark): pytest.param(FloatGen(special_cases=[FLOAT_MIN, FLOAT_MAX, 0.0, 1.0, -1.0]), marks=[incompat]), pytest.param(DoubleGen(special_cases=double_special_cases), marks=[incompat]), BooleanGen(), DateGen(), TimestampGen(), decimal_gen_32bit, decimal_gen_64bit, - decimal_gen_128bit] + _cache_single_array_gens_no_null_no_timestamp, ids=idfn) + orderable_decimal_gen_128bit] + _cache_single_array_gens_no_null_no_timestamp, ids=idfn) @pytest.mark.parametrize('ts_write', ['TIMESTAMP_MICROS', 'TIMESTAMP_MILLIS']) @pytest.mark.parametrize('enable_vectorized', ['true', 'false'], ids=idfn) @ignore_order diff --git a/integration_tests/src/main/python/cast_test.py b/integration_tests/src/main/python/cast_test.py index 2b15e68b2bf..7522592bdc8 100644 --- a/integration_tests/src/main/python/cast_test.py +++ b/integration_tests/src/main/python/cast_test.py @@ -228,9 +228,15 @@ def test_cast_long_to_decimal_overflow(): # casting these types to string should be passed basic_gens_for_cast_to_string = [ByteGen, ShortGen, IntegerGen, LongGen, StringGen, BooleanGen, DateGen, TimestampGen] basic_array_struct_gens_for_cast_to_string = [f() for f in basic_gens_for_cast_to_string] + [null_gen] + decimal_gens + +# We currently do not generate the exact string as Spark for some decimal values of zero +# https://github.com/NVIDIA/spark-rapids/issues/6339 basic_map_gens_for_cast_to_string = [ MapGen(f(nullable=False), f()) for f in basic_gens_for_cast_to_string] + [ - MapGen(DecimalGen(nullable=False), DecimalGen(precision=7, scale=3)), MapGen(DecimalGen(precision=7, scale=7, nullable=False), DecimalGen(precision=12, scale=2))] + MapGen(DecimalGen(nullable=False, special_cases=[]), + DecimalGen(precision=7, scale=3, special_cases=[])), + MapGen(DecimalGen(precision=7, scale=7, nullable=False, special_cases=[]), + DecimalGen(precision=12, scale=2), special_cases=[])] # GPU does not match CPU to casting these types to string, marked as xfail when testing not_matched_gens_for_cast_to_string = [FloatGen, DoubleGen] diff --git a/integration_tests/src/main/python/data_gen.py b/integration_tests/src/main/python/data_gen.py index db4b54b501f..6a8997e7167 100644 --- a/integration_tests/src/main/python/data_gen.py +++ b/integration_tests/src/main/python/data_gen.py @@ -21,7 +21,7 @@ from pyspark.sql.types import * import pyspark.sql.functions as f import random -from spark_session import is_tz_utc +from spark_session import is_tz_utc, is_before_spark_340 import sre_yield import struct from conftest import skip_unless_precommit_tests @@ -215,17 +215,19 @@ def start(self, rand): class DecimalGen(DataGen): """Generate Decimals, with some built in corner cases.""" - def __init__(self, precision=None, scale=None, nullable=True, special_cases=[]): + def __init__(self, precision=None, scale=None, nullable=True, special_cases=None): if precision is None: #Maximum number of decimal digits a Long can represent is 18 precision = 18 scale = 0 DECIMAL_MIN = Decimal('-' + ('9' * precision) + 'e' + str(-scale)) DECIMAL_MAX = Decimal(('9'* precision) + 'e' + str(-scale)) + if (special_cases is None): + special_cases = [DECIMAL_MIN, DECIMAL_MAX, Decimal('0')] super().__init__(DecimalType(precision, scale), nullable=nullable, special_cases=special_cases) self.scale = scale self.precision = precision - pattern = "[0-9]{1,"+ str(precision) + "}e" + str(-scale) + pattern = "-?[0-9]{1,"+ str(precision) + "}e" + str(-scale) self.base_strs = sre_yield.AllStrings(pattern, flags=0, charset=sre_yield.CHARSET, max_count=_MAX_CHOICES) def __repr__(self): @@ -928,10 +930,18 @@ def gen_scalars_for_sql(data_gen, count, seed=0, force_no_nulls=False): all_basic_gens_no_nan = [byte_gen, short_gen, int_gen, long_gen, FloatGen(no_nans=True), DoubleGen(no_nans=True), string_gen, boolean_gen, date_gen, timestamp_gen, null_gen] +# Many Spark versions have issues sorting large decimals, +# see https://issues.apache.org/jira/browse/SPARK-40089. +orderable_decimal_gen_128bit = decimal_gen_128bit +if is_before_spark_340(): + orderable_decimal_gen_128bit = DecimalGen(precision=20, scale=2, special_cases=[]) + +orderable_decimal_gens = [decimal_gen_32bit, decimal_gen_64bit, orderable_decimal_gen_128bit ] + # TODO add in some array generators to this once that is supported for sorting # a selection of generators that should be orderable (sortable and compareable) orderable_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, - string_gen, boolean_gen, date_gen, timestamp_gen, null_gen] + decimal_gens + string_gen, boolean_gen, date_gen, timestamp_gen, null_gen] + orderable_decimal_gens # TODO add in some array generators to this once that is supported for these operations # a selection of generators that can be compared for equality diff --git a/integration_tests/src/main/python/expand_exec_test.py b/integration_tests/src/main/python/expand_exec_test.py index 8974e313f0a..d60b7859095 100644 --- a/integration_tests/src/main/python/expand_exec_test.py +++ b/integration_tests/src/main/python/expand_exec_test.py @@ -1,4 +1,4 @@ -# Copyright (c) 2021, NVIDIA CORPORATION. +# Copyright (c) 2021-2022, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -19,7 +19,9 @@ from marks import ignore_order @pytest.mark.parametrize('data_gen', all_gen, ids=idfn) -@ignore_order +# Many Spark versions have issues sorting large decimals, +# see https://issues.apache.org/jira/browse/SPARK-40089. +@ignore_order(local=True) def test_expand_exec(data_gen): def op_df(spark, length=2048, seed=0): return gen_df(spark, StructGen([ diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index b2c67046e5a..40c35f474d4 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -447,10 +447,27 @@ def test_exceptAll(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : gen_df(spark, data_gen, length=100).exceptAll(gen_df(spark, data_gen, length=100).filter('a != b'))) +# Spark fails to sort some decimal values due to overflow when calculating the sorting prefix. +# See https://issues.apache.org/jira/browse/SPARK-40129 +# Since pivot orders by value, avoid generating these extreme values for this test. +_pivot_gen_128bit = DecimalGen(precision=20, scale=2, special_cases=[]) +_pivot_big_decimals = [ + ('a', RepeatSeqGen(DecimalGen(precision=32, scale=10, nullable=(True, 10.0)), length=50)), + ('b', _pivot_gen_128bit), + ('c', DecimalGen(precision=36, scale=5))] +_pivot_short_big_decimals = [ + ('a', RepeatSeqGen(short_gen, length=50)), + ('b', _pivot_gen_128bit), + ('c', decimal_gen_128bit)] + +_pivot_gens_with_decimals = _init_list_with_nans_and_no_nans + [ + _grpkey_small_decimals, _pivot_big_decimals, _grpkey_short_mid_decimals, + _pivot_short_big_decimals, _grpkey_short_very_big_decimals, + _grpkey_short_very_big_neg_scale_decimals] @approximate_float @ignore_order(local=True) @incompat -@pytest.mark.parametrize('data_gen', _init_list_with_nans_and_no_nans_with_decimalbig, ids=idfn) +@pytest.mark.parametrize('data_gen', _pivot_gens_with_decimals, ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs_with_nans, params_markers_for_confs_nans), ids=idfn) def test_hash_grpby_pivot(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( @@ -1192,7 +1209,9 @@ def test_sorted_groupby_first_last(data_gen): lambda spark: agg_fn(gen_df(spark, gen_fn, num_slices=1)), conf = {'spark.sql.shuffle.partitions': '1'}) -@ignore_order +# Spark has a sorting bug with decimals, see https://issues.apache.org/jira/browse/SPARK-40129. +# Have pytest do the sorting rather than Spark as a workaround. +@ignore_order(local=True) @pytest.mark.parametrize('data_gen', all_gen, ids=idfn) @pytest.mark.parametrize('count_func', [f.count, f.countDistinct]) def test_agg_count(data_gen, count_func): diff --git a/integration_tests/src/main/python/join_test.py b/integration_tests/src/main/python/join_test.py index 83d8be09541..b64eab99d41 100644 --- a/integration_tests/src/main/python/join_test.py +++ b/integration_tests/src/main/python/join_test.py @@ -29,7 +29,7 @@ all_gen = [StringGen(), ByteGen(), ShortGen(), IntegerGen(), LongGen(), BooleanGen(), DateGen(), TimestampGen(), null_gen, pytest.param(FloatGen(), marks=[incompat]), - pytest.param(DoubleGen(), marks=[incompat])] + decimal_gens + pytest.param(DoubleGen(), marks=[incompat])] + orderable_decimal_gens all_gen_no_nulls = [StringGen(nullable=False), ByteGen(nullable=False), ShortGen(nullable=False), IntegerGen(nullable=False), LongGen(nullable=False), @@ -72,7 +72,8 @@ # Types to use when running joins on small batches. Small batch joins can take a long time # to run and are mostly redundant with the normal batch size test, so we only run these on a # set of representative types rather than all types. -join_small_batch_gens = [ StringGen(), IntegerGen(), decimal_gen_128bit ] + +join_small_batch_gens = [ StringGen(), IntegerGen(), orderable_decimal_gen_128bit ] cartesian_join_small_batch_gens = join_small_batch_gens + [basic_struct_gen, ArrayGen(string_gen)] _sortmerge_join_conf = {'spark.sql.autoBroadcastJoinThreshold': '-1', diff --git a/integration_tests/src/main/python/orc_test.py b/integration_tests/src/main/python/orc_test.py index 204b4a127b9..c063dd07dbb 100644 --- a/integration_tests/src/main/python/orc_test.py +++ b/integration_tests/src/main/python/orc_test.py @@ -31,6 +31,14 @@ def read_orc_df(data_path): def read_orc_sql(data_path): return lambda spark : spark.sql('select * from orc.`{}`'.format(data_path)) +# ORC has issues reading timestamps where it is off by 1 second if the timestamp is before +# epoch in 1970 and the microsecond value is between 0 and 1000. +# See https://github.com/rapidsai/cudf/issues/11525. +def get_orc_timestamp_gen(nullable=True): + return TimestampGen(start=datetime(1970, 1, 1, tzinfo=timezone.utc), nullable=nullable) + +orc_timestamp_gen = get_orc_timestamp_gen() + # test with original orc file reader, the multi-file parallel reader for cloud original_orc_file_reader_conf = {'spark.rapids.sql.format.orc.reader.type': 'PERFILE'} multithreaded_orc_file_reader_conf = {'spark.rapids.sql.format.orc.reader.type': 'MULTITHREADED'} @@ -51,7 +59,7 @@ def test_basic_read(std_input_path, name, read_func, v1_enabled_list, orc_impl, conf=all_confs) # ORC does not support negative scale for decimal. So here is "decimal_gens_no_neg". -# Otherwsie it will get the below exception. +# Otherwise it will get the below exception. # ... #E Caused by: java.lang.IllegalArgumentException: Missing integer at # 'struct<`_c0`:decimal(7,^-3),`_c1`:decimal(7,3),`_c2`:decimal(7,7),`_c3`:decimal(12,2)>' @@ -60,8 +68,7 @@ def test_basic_read(std_input_path, name, read_func, v1_enabled_list, orc_impl, # ... orc_basic_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), - TimestampGen(start=datetime(1590, 1, 1, tzinfo=timezone.utc)) - ] + decimal_gens + orc_timestamp_gen] + decimal_gens orc_basic_struct_gen = StructGen([['child'+str(ind), sub_gen] for ind, sub_gen in enumerate(orc_basic_gens)]) @@ -85,7 +92,7 @@ def test_basic_read(std_input_path, name, read_func, v1_enabled_list, orc_impl, orc_basic_map_gens = [simple_string_to_string_map_gen] + [MapGen(f(nullable=False), f()) for f in [ BooleanGen, ByteGen, ShortGen, IntegerGen, LongGen, FloatGen, DoubleGen, - lambda nullable=True: TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc), nullable=nullable), + lambda nullable=True: get_orc_timestamp_gen(nullable), lambda nullable=True: DateGen(start=date(1590, 1, 1), nullable=nullable), lambda nullable=True: DecimalGen(precision=15, scale=1, nullable=nullable), lambda nullable=True: DecimalGen(precision=36, scale=5, nullable=nullable)]] @@ -152,7 +159,7 @@ def test_read_round_trip(spark_tmp_path, orc_gens, read_func, reader_confs, v1_e DateGen(start=date(1590, 1, 1)), # Once https://github.com/NVIDIA/spark-rapids/issues/140 is fixed replace this with # timestamp_gen - TimestampGen(start=datetime(1970, 1, 1, tzinfo=timezone.utc))] + orc_timestamp_gen] @pytest.mark.order(2) @pytest.mark.parametrize('orc_gen', orc_pred_push_gens, ids=idfn) @@ -220,7 +227,7 @@ def test_simple_partitioned_read(spark_tmp_path, v1_enabled_list, reader_confs): # we should go with a more standard set of generators orc_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), - TimestampGen(start=datetime(1590, 1, 1, tzinfo=timezone.utc))] + orc_timestamp_gen] gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gens)] first_data_path = spark_tmp_path + '/ORC_DATA/key=0/key2=20' with_cpu_session( @@ -287,7 +294,7 @@ def test_merge_schema_read(spark_tmp_path, v1_enabled_list, reader_confs): # we should go with a more standard set of generators orc_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), - TimestampGen(start=datetime(1590, 1, 1, tzinfo=timezone.utc))] + orc_timestamp_gen] first_gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gens)] first_data_path = spark_tmp_path + '/ORC_DATA/key=0' with_cpu_session( diff --git a/integration_tests/src/main/python/sort_test.py b/integration_tests/src/main/python/sort_test.py index 4b09db2740b..2f6de5b7f48 100644 --- a/integration_tests/src/main/python/sort_test.py +++ b/integration_tests/src/main/python/sort_test.py @@ -19,12 +19,19 @@ from marks import allow_non_gpu from pyspark.sql.types import * import pyspark.sql.functions as f +from spark_session import is_before_spark_340 + +# Many Spark versions have issues sorting decimals. +# https://issues.apache.org/jira/browse/SPARK-40089 +_orderable_not_null_big_decimal_gen = DecimalGen(precision=20, scale=2, nullable=False) +if is_before_spark_340(): + _orderable_not_null_big_decimal_gen = DecimalGen(precision=20, scale=2, nullable=False, special_cases=[]) orderable_not_null_gen = [ByteGen(nullable=False), ShortGen(nullable=False), IntegerGen(nullable=False), LongGen(nullable=False), FloatGen(nullable=False), DoubleGen(nullable=False), BooleanGen(nullable=False), TimestampGen(nullable=False), DateGen(nullable=False), StringGen(nullable=False), DecimalGen(precision=7, scale=3, nullable=False), DecimalGen(precision=12, scale=2, nullable=False), - DecimalGen(precision=20, scale=2, nullable=False)] + _orderable_not_null_big_decimal_gen] @allow_non_gpu('SortExec', 'ShuffleExchangeExec', 'RangePartitioning', 'SortOrder') @pytest.mark.parametrize('data_gen', [StringGen(nullable=False)], ids=idfn) @@ -164,7 +171,8 @@ def test_single_nested_sort_in_part(data_gen, order, stable_sort): conf=sort_conf) orderable_gens_sort = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, - boolean_gen, timestamp_gen, date_gen, string_gen, null_gen, StructGen([('child0', long_gen)])] + decimal_gens + boolean_gen, timestamp_gen, date_gen, string_gen, null_gen, StructGen([('child0', long_gen)]) + ] + orderable_decimal_gens @pytest.mark.parametrize('data_gen', orderable_gens_sort, ids=idfn) def test_multi_orderby(data_gen): assert_gpu_and_cpu_are_equal_collect( diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py index 7869122fe43..a21feb810f4 100644 --- a/integration_tests/src/main/python/window_function_test.py +++ b/integration_tests/src/main/python/window_function_test.py @@ -441,7 +441,7 @@ def test_running_float_sum_no_part(batch_size): # to allow for duplication in the ordering, because there will be no other columns. This means that if you swtich # rows it does not matter because the only time rows are switched is when the rows are exactly the same. @pytest.mark.parametrize('data_gen', - all_basic_gens_no_nans + [decimal_gen_32bit, decimal_gen_128bit], + all_basic_gens_no_nans + [decimal_gen_32bit, orderable_decimal_gen_128bit], ids=meta_idfn('data:')) def test_window_running_rank_no_part(data_gen): # Keep the batch size small. We have tested these with operators with exact inputs already, this is mostly diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 0784174dfe2..b28c5ce18c0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -1848,9 +1848,18 @@ object GpuOverrides extends Logging { expr[Pmod]( "Pmod", ExprChecks.binaryProject(TypeSig.gpuNumeric, TypeSig.cpuNumeric, - ("lhs", TypeSig.gpuNumeric, TypeSig.cpuNumeric), + ("lhs", TypeSig.gpuNumeric.withPsNote(TypeEnum.DECIMAL, + s"decimals with precision ${DecimalType.MAX_PRECISION} are not supported"), + TypeSig.cpuNumeric), ("rhs", TypeSig.gpuNumeric, TypeSig.cpuNumeric)), (a, conf, p, r) => new BinaryExprMeta[Pmod](a, conf, p, r) { + override def tagExprForGpu(): Unit = { + a.dataType match { + case dt: DecimalType if dt.precision == DecimalType.MAX_PRECISION => + willNotWorkOnGpu("pmod at maximum decimal precision is not supported") + case _ => + } + } override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = GpuPmod(lhs, rhs) }), diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala index c064a41c05c..06c720e63bd 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala @@ -1019,10 +1019,6 @@ abstract class GpuDecimalSum( Seq(updateSum, updateIsEmpty) } - override lazy val postUpdate: Seq[Expression] = { - Seq(GpuCheckOverflow(updateSum.attr, dt, !failOnErrorOverride), updateIsEmpty.attr) - } - // Used for Decimal overflow detection protected lazy val isEmpty: AttributeReference = AttributeReference("isEmpty", BooleanType)() override lazy val aggBufferAttributes: Seq[AttributeReference] = { @@ -1045,10 +1041,7 @@ abstract class GpuDecimalSum( override lazy val postMerge: Seq[Expression] = { Seq( - GpuCheckOverflow(GpuIf(mergeIsOverflow.attr, - GpuLiteral.create(null, dt), - mergeSum.attr), - dt, !failOnErrorOverride), + GpuIf(mergeIsOverflow.attr, GpuLiteral.create(null, dt), mergeSum.attr), mergeIsEmpty.attr) } @@ -1140,8 +1133,9 @@ case class GpuDecimal128Sum( override lazy val updateAggregates: Seq[CudfAggregate] = updateSumChunks :+ updateIsEmpty override lazy val postUpdate: Seq[Expression] = { - val assembleExpr = GpuAssembleSumChunks(updateSumChunks.map(_.attr), dt, !failOnErrorOverride) - Seq(GpuCheckOverflow(assembleExpr, dt, !failOnErrorOverride), updateIsEmpty.attr) + Seq( + GpuAssembleSumChunks(updateSumChunks.map(_.attr), dt, !failOnErrorOverride), + updateIsEmpty.attr) } override lazy val preMerge: Seq[Expression] = { @@ -1165,10 +1159,7 @@ case class GpuDecimal128Sum( override lazy val postMerge: Seq[Expression] = { val assembleExpr = GpuAssembleSumChunks(mergeSumChunks.map(_.attr), dt, !failOnErrorOverride) Seq( - GpuCheckOverflow(GpuIf(mergeIsOverflow.attr, - GpuLiteral.create(null, dt), - assembleExpr), - dt, !failOnErrorOverride), + GpuIf(mergeIsOverflow.attr, GpuLiteral.create(null, dt), assembleExpr), mergeIsEmpty.attr) } diff --git a/tools/src/main/resources/supportedExprs.csv b/tools/src/main/resources/supportedExprs.csv index dac588103e5..b68b46e1f66 100644 --- a/tools/src/main/resources/supportedExprs.csv +++ b/tools/src/main/resources/supportedExprs.csv @@ -360,7 +360,7 @@ Or,S,`or`,None,AST,rhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Or,S,`or`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA PercentRank,S,`percent_rank`,None,window,ordering,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS PercentRank,S,`percent_rank`,None,window,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -Pmod,S,`pmod`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA +Pmod,S,`pmod`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA Pmod,S,`pmod`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA Pmod,S,`pmod`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA PosExplode,S,`posexplode_outer`; `posexplode`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,PS,NA,NA From f0f67f764dd40a4b9619e7c1c4b774db7b5f733f Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Thu, 25 Aug 2022 16:31:33 -0500 Subject: [PATCH 054/190] Fix handling of older array encodings in Parquet (#6410) Signed-off-by: Jason Lowe --- .../nvidia/spark/rapids/GpuParquetScan.scala | 16 ++++++++++---- .../spark/rapids/ParquetSchemaUtils.scala | 21 +++++++++++-------- 2 files changed, 24 insertions(+), 13 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index ac12b021057..a35c5527e9c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -765,10 +765,18 @@ private case class GpuParquetFileFilterHandler(@transient sqlConf: SQLConf) exte } } case array: ArrayType => - val fileChild = fileType.asGroupType().getType(0) - .asGroupType().getType(0) - checkSchemaCompat(fileChild, array.elementType, errorCallback, isCaseSensitive, useFieldId, - rootFileType, rootReadType) + if (fileType.isPrimitive) { + if (fileType.getRepetition == Type.Repetition.REPEATED) { + checkSchemaCompat(fileType, array.elementType, errorCallback, isCaseSensitive, + useFieldId, rootFileType, rootReadType) + } else { + errorCallback(fileType, readType) + } + } else { + val fileChild = fileType.asGroupType().getType(0).asGroupType().getType(0) + checkSchemaCompat(fileChild, array.elementType, errorCallback, isCaseSensitive, + useFieldId, rootFileType, rootReadType) + } case map: MapType => val parquetMap = fileType.asGroupType().getType(0).asGroupType() diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ParquetSchemaUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ParquetSchemaUtils.scala index 00508b9dd49..09d8cf8257b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ParquetSchemaUtils.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ParquetSchemaUtils.scala @@ -345,7 +345,7 @@ object ParquetSchemaUtils extends Arm { sparkType match { case t: ArrayType => // Only clips array types with nested type as element type. - clipSparkArrayType(t, parquetType.asGroupType(), caseSensitive, useFieldId) + clipSparkArrayType(t, parquetType, caseSensitive, useFieldId) case t: MapType => clipSparkMapType(t, parquetType.asGroupType(), caseSensitive, useFieldId) @@ -360,18 +360,20 @@ object ParquetSchemaUtils extends Arm { private def clipSparkArrayType( sparkType: ArrayType, - parquetList: GroupType, + parquetList: Type, caseSensitive: Boolean, useFieldId: Boolean): DataType = { val elementType = sparkType.elementType // Unannotated repeated group should be interpreted as required list of required element, so // list element type is just the group itself. // TODO: When we drop Spark 3.1.x, this should use Parquet's LogicalTypeAnnotation + // Note that the original type is not null for leaf nodes. //if (parquetList.getLogicalTypeAnnotation == null && - if (parquetList.getOriginalType == null && + val newSparkType = if (parquetList.getOriginalType == null && parquetList.isRepetition(Repetition.REPEATED)) { clipSparkType(elementType, parquetList, caseSensitive, useFieldId) } else { + val parquetListGroup = parquetList.asGroupType() assert( // TODO: When we drop Spark 3.1.x, this should use Parquet's LogicalTypeAnnotation //parquetList.getLogicalTypeAnnotation.isInstanceOf[ListLogicalTypeAnnotation], @@ -381,14 +383,15 @@ object ParquetSchemaUtils extends Arm { "ListLogicalTypeAnnotation: " + parquetList.toString) assert( - parquetList.getFieldCount == 1 && parquetList.getType(0).isRepetition(Repetition.REPEATED), + parquetListGroup.getFieldCount == 1 && + parquetListGroup.getType(0).isRepetition(Repetition.REPEATED), "Invalid Parquet schema. " + "LIST-annotated group should only have exactly one repeated field: " + parquetList) - val repeated = parquetList.getType(0) - val newSparkType = if (repeated.isPrimitive) { - clipSparkType(elementType, parquetList.getType(0), caseSensitive, useFieldId) + val repeated = parquetListGroup.getType(0) + if (repeated.isPrimitive) { + clipSparkType(elementType, parquetListGroup.getType(0), caseSensitive, useFieldId) } else { val repeatedGroup = repeated.asGroupType() @@ -408,9 +411,9 @@ object ParquetSchemaUtils extends Arm { } clipSparkType(elementType, parquetElementType, caseSensitive, useFieldId) } - - sparkType.copy(elementType = newSparkType) } + + sparkType.copy(elementType = newSparkType) } private def clipSparkMapType( From b5922d1267d6a56963542452936b76575e48dec4 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 26 Aug 2022 09:48:30 -0500 Subject: [PATCH 055/190] Fix tests for DateTimeInterval that were overflowing on CPU (#6418) Signed-off-by: Robert (Bobby) Evans Signed-off-by: Robert (Bobby) Evans --- integration_tests/src/main/python/arithmetic_ops_test.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/integration_tests/src/main/python/arithmetic_ops_test.py b/integration_tests/src/main/python/arithmetic_ops_test.py index fcb9e7bf4d2..c4b7bff6f46 100644 --- a/integration_tests/src/main/python/arithmetic_ops_test.py +++ b/integration_tests/src/main/python/arithmetic_ops_test.py @@ -1077,16 +1077,15 @@ def test_unary_positive_day_time_interval(): lambda spark: unary_op_df(spark, DayTimeIntervalGen()).selectExpr('+a')) @pytest.mark.skipif(is_before_spark_330(), reason='DayTimeInterval is not supported before Pyspark 3.3.0') -@pytest.mark.parametrize('data_gen', _no_overflow_multiply_gens + [DoubleGen(min_exp=-3, max_exp=5, special_cases=[0.0])], ids=idfn) +@pytest.mark.parametrize('data_gen', _no_overflow_multiply_gens_for_fallback + [DoubleGen(min_exp=-3, max_exp=5, special_cases=[0.0])], ids=idfn) def test_day_time_interval_multiply_number(data_gen): gen_list = [('_c1', DayTimeIntervalGen(min_value=timedelta(seconds=-20 * 86400), max_value=timedelta(seconds=20 * 86400))), ('_c2', data_gen)] assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, gen_list).selectExpr("_c1 * _c2")) - @pytest.mark.skipif(is_before_spark_330(), reason='DayTimeInterval is not supported before Pyspark 3.3.0') -@pytest.mark.parametrize('data_gen', _no_overflow_multiply_gens + [DoubleGen(min_exp=0, max_exp=5, special_cases=[])], ids=idfn) +@pytest.mark.parametrize('data_gen', _no_overflow_multiply_gens_for_fallback + [DoubleGen(min_exp=0, max_exp=5, special_cases=[])], ids=idfn) def test_day_time_interval_division_number_no_overflow1(data_gen): gen_list = [('_c1', DayTimeIntervalGen(min_value=timedelta(seconds=-5000 * 365 * 86400), max_value=timedelta(seconds=5000 * 365 * 86400))), ('_c2', data_gen)] @@ -1095,7 +1094,7 @@ def test_day_time_interval_division_number_no_overflow1(data_gen): lambda spark: gen_df(spark, gen_list).selectExpr("_c1 / case when _c2 = 0 then cast(1 as {}) else _c2 end".format(to_cast_string(data_gen.data_type)))) @pytest.mark.skipif(is_before_spark_330(), reason='DayTimeInterval is not supported before Pyspark 3.3.0') -@pytest.mark.parametrize('data_gen', _no_overflow_multiply_gens + [DoubleGen(min_exp=-5, max_exp=0, special_cases=[])], ids=idfn) +@pytest.mark.parametrize('data_gen', _no_overflow_multiply_gens_for_fallback + [DoubleGen(min_exp=-5, max_exp=0, special_cases=[])], ids=idfn) def test_day_time_interval_division_number_no_overflow2(data_gen): gen_list = [('_c1', DayTimeIntervalGen(min_value=timedelta(seconds=-20 * 86400), max_value=timedelta(seconds=20 * 86400))), ('_c2', data_gen)] From 392900254d4d973c9e1d7b126df6be2292ccf55c Mon Sep 17 00:00:00 2001 From: Nghia Truong Date: Fri, 26 Aug 2022 14:30:21 -0700 Subject: [PATCH 056/190] Support more timestamp format when casting string to timestamp (#6405) * Modify `TIMESTAMP_REGEX_FULL` Signed-off-by: Nghia Truong * Add supported format Signed-off-by: Nghia Truong * Add test Signed-off-by: Nghia Truong * Fix memory leak and update `TIMESTAMP_REGEX_NO_DATE` Signed-off-by: Nghia Truong * Fix input pattern in test Signed-off-by: Nghia Truong * Enforce the second number to be in [0, 59] Signed-off-by: Nghia Truong * Cleanup Signed-off-by: Nghia Truong * Update `TIMESTAMP_REGEX_NO_DATE` Signed-off-by: Nghia Truong * Update docs Signed-off-by: Nghia Truong * Update doc for time zone Signed-off-by: Nghia Truong * Break line Signed-off-by: Nghia Truong * Add more support format Signed-off-by: Nghia Truong * Add test for `T` notion Signed-off-by: Nghia Truong Signed-off-by: Nghia Truong --- docs/compatibility.md | 6 ++++-- integration_tests/src/main/python/cast_test.py | 8 ++++++-- .../scala/com/nvidia/spark/rapids/GpuCast.scala | 17 ++++++++++++++--- 3 files changed, 24 insertions(+), 7 deletions(-) diff --git a/docs/compatibility.md b/docs/compatibility.md index c63bdd47e02..10cfcfd0783 100644 --- a/docs/compatibility.md +++ b/docs/compatibility.md @@ -882,6 +882,8 @@ Casting from string to timestamp currently has the following limitations. | `"yyyy-[M]M "` | Yes | | `"yyyy-[M]M-[d]d"` | Yes | | `"yyyy-[M]M-[d]d "` | Yes | +| `"yyyy-[M]M-[d]dT[h]h:[m]m:[s]s[zone_id]"` | Partial [\[1\]](#Footnote1) | +| `"yyyy-[M]M-[d]d [h]h:[m]m:[s]s[zone_id]"` | Partial [\[1\]](#Footnote1) | | `"yyyy-[M]M-[d]dT[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]"` | Partial [\[1\]](#Footnote1) | | `"yyyy-[M]M-[d]d [h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]"` | Partial [\[1\]](#Footnote1) | | `"[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]"` | Partial [\[1\]](#Footnote1) | @@ -892,8 +894,8 @@ Casting from string to timestamp currently has the following limitations. | `"tomorrow"` | Yes | | `"yesterday"` | Yes | -- [1] The timestamp portion must have 6 digits for milliseconds. - Only timezone 'Z' (UTC) is supported. Casting unsupported formats will result in null values. +- [1] Leap seconds are not supported. If a zone_id is provided then only + timezone 'Z' (UTC) is supported. Casting unsupported formats will result in null values. Spark is very lenient when casting from string to timestamp because all date and time components are optional, meaning that input values such as `T`, `T2`, `:`, `::`, `1:`, `:1`, and `::1` diff --git a/integration_tests/src/main/python/cast_test.py b/integration_tests/src/main/python/cast_test.py index 7522592bdc8..8c5326c44ec 100644 --- a/integration_tests/src/main/python/cast_test.py +++ b/integration_tests/src/main/python/cast_test.py @@ -132,11 +132,15 @@ def test_cast_string_date_non_ansi(): lambda spark: spark.createDataFrame(data_rows, "a string").select(f.col('a').cast(DateType())), conf={'spark.rapids.sql.hasExtendedYearValues': 'false'}) -def test_cast_string_ts_valid_format(): +@pytest.mark.parametrize('data_gen', [StringGen('[0-9]{1,4}-[0-9]{1,2}-[0-9]{1,2}'), + StringGen('[0-9]{1,4}-[0-3][0-9]-[0-5][0-9][ |T][0-3][0-9]:[0-6][0-9]:[0-6][0-9]'), + StringGen('[0-9]{1,4}-[0-3][0-9]-[0-5][0-9][ |T][0-3][0-9]:[0-6][0-9]:[0-6][0-9].[0-9]{0,6}Z?')], + ids=idfn) +def test_cast_string_ts_valid_format(data_gen): # In Spark 3.2.0+ the valid format changed, and we cannot support all of the format. # This provides values that are valid in all of those formats. assert_gpu_and_cpu_are_equal_collect( - lambda spark : unary_op_df(spark, StringGen('[0-9]{1,4}-[0-9]{1,2}-[0-9]{1,2}')).select(f.col('a').cast(TimestampType())), + lambda spark : unary_op_df(spark, data_gen).select(f.col('a').cast(TimestampType())), conf = {'spark.rapids.sql.hasExtendedYearValues': 'false', 'spark.rapids.sql.castStringToTimestamp.enabled': 'true'}) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala index 1e52ff60de2..600669ccf55 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala @@ -169,8 +169,9 @@ object GpuCast extends Arm { private val TIMESTAMP_REGEX_YYYY_MM = "\\A\\d{4}\\-\\d{1,2}[ ]?\\Z" private val TIMESTAMP_REGEX_YYYY = "\\A\\d{4}[ ]?\\Z" private val TIMESTAMP_REGEX_FULL = - "\\A\\d{4}\\-\\d{1,2}\\-\\d{1,2}[ T]?(\\d{1,2}:\\d{1,2}:\\d{1,2}\\.\\d{6}Z)\\Z" - private val TIMESTAMP_REGEX_NO_DATE = "\\A[T]?(\\d{1,2}:\\d{1,2}:\\d{1,2}\\.\\d{6}Z)\\Z" + "\\A\\d{4}\\-\\d{1,2}\\-\\d{1,2}[ T]?(\\d{1,2}:\\d{1,2}:([0-5]\\d|\\d)(\\.\\d{0,6})?Z?)\\Z" + private val TIMESTAMP_REGEX_NO_DATE = + "\\A[T]?(\\d{1,2}:\\d{1,2}:([0-5]\\d|\\d)(\\.\\d{0,6})?Z?)\\Z" private val BIG_DECIMAL_LONG_MIN = BigDecimal(Long.MinValue) private val BIG_DECIMAL_LONG_MAX = BigDecimal(Long.MaxValue) @@ -1314,13 +1315,23 @@ object GpuCast extends Arm { val cudfFormat1 = "%Y-%m-%d %H:%M:%S.%f" val cudfFormat2 = "%Y-%m-%dT%H:%M:%S.%f" + val cudfFormat3 = "%Y-%m-%d %H:%M:%S" + val cudfFormat4 = "%Y-%m-%dT%H:%M:%S" withResource(orElse) { orElse => // valid dates must match the regex and either of the cuDF formats val isCudfMatch = withResource(input.isTimestamp(cudfFormat1)) { isTimestamp1 => withResource(input.isTimestamp(cudfFormat2)) { isTimestamp2 => - isTimestamp1.or(isTimestamp2) + withResource(input.isTimestamp(cudfFormat3)) { isTimestamp3 => + withResource(input.isTimestamp(cudfFormat4)) { isTimestamp4 => + withResource(isTimestamp1.or(isTimestamp2)) { isTimestamp12 => + withResource(isTimestamp12.or(isTimestamp3)) { isTimestamp123 => + isTimestamp123.or(isTimestamp4) + } + } + } + } } } From 8f4b4cb6f55c6f44c56a679578c122f20b15d4a9 Mon Sep 17 00:00:00 2001 From: Remzi Yang <59198230+HaoYang670@users.noreply.github.com> Date: Sat, 27 Aug 2022 07:44:26 +0800 Subject: [PATCH 057/190] support nested keys in GpuMapConcat (#6290) Signed-off-by: remzi <13716567376yh@gmail.com> Signed-off-by: remzi <13716567376yh@gmail.com> --- .../com/nvidia/spark/rapids/GpuOverrides.scala | 11 ----------- .../nvidia/spark/rapids/CollectionOpSuite.scala | 16 ++++++---------- 2 files changed, 6 insertions(+), 21 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index b28c5ce18c0..780c9038b3b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -3202,10 +3202,6 @@ object GpuOverrides extends Logging { }), expr[MapConcat]( "Returns the union of all the given maps", - // Currently, GpuMapConcat supports nested values but not nested keys. - // We will add the nested key support after - // cuDF can fully support nested types in lists::drop_list_duplicates. - // Issue link: https://github.com/rapidsai/cudf/issues/11093 ExprChecks.projectOnly(TypeSig.MAP.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), TypeSig.MAP.nested(TypeSig.all), @@ -3214,13 +3210,6 @@ object GpuOverrides extends Logging { TypeSig.NULL + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), TypeSig.MAP.nested(TypeSig.all)))), (a, conf, p, r) => new ComplexTypeMergingExprMeta[MapConcat](a, conf, p, r) { - override def tagExprForGpu(): Unit = { - a.dataType.keyType match { - case MapType(_,_,_) | ArrayType(_,_) | StructType(_) => willNotWorkOnGpu( - s"GpuMapConcat does not currently support the key type ${a.dataType.keyType}.") - case _ => - } - } override def convertToGpu(child: Seq[Expression]): GpuExpression = GpuMapConcat(child) }), expr[ConcatWs]( diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/CollectionOpSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/CollectionOpSuite.scala index 6d1bd7fc279..7497bc42d7a 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/CollectionOpSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/CollectionOpSuite.scala @@ -19,22 +19,18 @@ package com.nvidia.spark.rapids import org.apache.spark.sql.functions.map_concat class CollectionOpSuite extends SparkQueryCompareTestSuite { - testGpuFallback( - "MapConcat with Array keys fall back", - "ProjectExec", - ArrayKeyMapDF, - execsAllowedNonGpu = Seq("ProjectExec", "ShuffleExchangeExec")) { + testSparkResultsAreEqual( + "MapConcat with Array keys", + ArrayKeyMapDF) { frame => { import frame.sparkSession.implicits._ frame.select(map_concat($"col1", $"col2")) } } - testGpuFallback( - "MapConcat with Struct keys fall back", - "ProjectExec", - StructKeyMapDF, - execsAllowedNonGpu = Seq("ProjectExec", "ShuffleExchangeExec")) { + testSparkResultsAreEqual( + "MapConcat with Struct keys", + StructKeyMapDF) { frame => { import frame.sparkSession.implicits._ frame.select(map_concat($"col1", $"col2")) From 9d38d2ab01b598af0bdfc9ebbdb589d847cfe9ba Mon Sep 17 00:00:00 2001 From: Nghia Truong Date: Fri, 26 Aug 2022 18:00:44 -0700 Subject: [PATCH 058/190] Support simple pass-through for `FromUTCTimestamp` (#6370) * Add `FromUTCTimestamp` Signed-off-by: Nghia Truong * Complete implementation Signed-off-by: Nghia Truong * Remove empty lines Signed-off-by: Nghia Truong * Add more implementation Signed-off-by: Nghia Truong * Reverse to support only `UTC` timezone Signed-off-by: Nghia Truong * Rewrite error messages Signed-off-by: Nghia Truong * Rewrite error messages Signed-off-by: Nghia Truong * Remove try/catch Signed-off-by: Nghia Truong * Cleanup imports Signed-off-by: Nghia Truong * Fix spaces Signed-off-by: Nghia Truong * Add a test Signed-off-by: Nghia Truong * Fix binary projection Signed-off-by: Nghia Truong * Fix type check Signed-off-by: Nghia Truong * Fix input check Signed-off-by: Nghia Truong * Complete test Signed-off-by: Nghia Truong * Reverse changes for `DateFormatClass` override Signed-off-by: Nghia Truong * Handle null case Signed-off-by: Nghia Truong * Add fallback test Signed-off-by: Nghia Truong * Update docs from `mvn verify` Signed-off-by: Nghia Truong * Remove constructor parameter Signed-off-by: Nghia Truong * Rewrite `tagExprForGpu` Signed-off-by: Nghia Truong * Rewrite `tagExprForGpu` Signed-off-by: Nghia Truong * Remove string input type in GpuOverride Signed-off-by: Nghia Truong * Update sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala Co-authored-by: Jason Lowe * Return ColumnVector of all nulls Signed-off-by: Nghia Truong * Update docs Signed-off-by: Nghia Truong * Add more time zones to test Signed-off-by: Nghia Truong * Revert "Add more time zones to test" This reverts commit f547190bacc3a17c83b24af770cf63fdddbc1bc4. * Rewrite `selectExpr` to `select` Signed-off-by: Nghia Truong Signed-off-by: Nghia Truong Co-authored-by: Jason Lowe --- docs/configs.md | 1 + docs/supported_ops.md | 640 ++++++++++-------- .../src/main/python/date_time_test.py | 15 + .../nvidia/spark/rapids/GpuOverrides.scala | 15 +- .../sql/rapids/datetimeExpressions.scala | 68 +- tools/src/main/resources/operatorsScore.csv | 1 + tools/src/main/resources/supportedExprs.csv | 3 + 7 files changed, 453 insertions(+), 290 deletions(-) diff --git a/docs/configs.md b/docs/configs.md index b4598b315a8..edcf1bcc621 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -225,6 +225,7 @@ Name | SQL Function(s) | Description | Default Value | Notes spark.rapids.sql.expression.Explode|`explode`, `explode_outer`|Given an input array produces a sequence of rows for each value in the array|true|None| spark.rapids.sql.expression.Expm1|`expm1`|Euler's number e raised to a power minus 1|true|None| spark.rapids.sql.expression.Floor|`floor`|Floor of a number|true|None| +spark.rapids.sql.expression.FromUTCTimestamp|`from_utc_timestamp`|Render the input UTC timestamp in the input timezone|true|None| spark.rapids.sql.expression.FromUnixTime|`from_unixtime`|Get the string from a unix timestamp|true|None| spark.rapids.sql.expression.GetArrayItem| |Gets the field at `ordinal` in the Array|true|None| spark.rapids.sql.expression.GetArrayStructFields| |Extracts the `ordinal`-th fields of all array elements for the data with the type of array of struct|true|None| diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 07873043402..5981c73de0d 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -6114,6 +6114,74 @@ are limited. +FromUTCTimestamp +`from_utc_timestamp` +Render the input UTC timestamp in the input timezone +None +project +timestamp + + + + + + + + +PS
UTC is only supported TZ for TIMESTAMP
+ + + + + + + + + + + +timezone + + + + + + + + + +PS
Only timezones equivalent to UTC are supported
+ + + + + + + + + + +result + + + + + + + + +PS
UTC is only supported TZ for TIMESTAMP
+ + + + + + + + + + + FromUnixTime `from_unixtime` Get the string from a unix timestamp @@ -6250,6 +6318,32 @@ are limited. NS +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + GetArrayStructFields Extracts the `ordinal`-th fields of all array elements for the data with the type of array of struct @@ -6365,32 +6459,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - GetMapValue Gets Value from a Map based on a key @@ -6706,6 +6774,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + GreaterThanOrEqual `>=` >= operator @@ -6838,32 +6932,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - Greatest `greatest` Returns the greatest value of all parameters, skipping null values @@ -7115,6 +7183,32 @@ are limited. NS +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + In `in` IN operator @@ -7230,32 +7324,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - InitCap `initcap` Returns str with the first letter of each word in uppercase. All other letters are in lowercase @@ -7496,6 +7564,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + IsNotNull `isnotnull` Checks if a value is not null @@ -7637,32 +7731,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - KnownFloatingPointNormalized Tag to prevent redundant normalization @@ -7914,6 +7982,32 @@ are limited. NS +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + LastDay `last_day` Returns the last day of the month which the date belongs to @@ -8050,32 +8144,6 @@ are limited. NS -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - Least `least` Returns the least value of all parameters, skipping null values @@ -8302,6 +8370,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + LessThanOrEqual `<=` <= operator @@ -8434,32 +8528,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - Like `like` Like @@ -8670,6 +8738,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + Log1p `log1p` Natural log 1 + expr @@ -8832,32 +8926,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - Lower `lower`, `lcase` String lowercase operator @@ -9046,6 +9114,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + MapFilter `map_filter` Filters entries in a map using the function @@ -9208,32 +9302,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - Md5 `md5` MD5 hash operator @@ -9533,6 +9601,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + Murmur3Hash `hash` Murmur3 hash operator @@ -9580,32 +9674,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - NaNvl `nanvl` Evaluates to `left` iff left is not NaN, `right` otherwise @@ -9905,6 +9973,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + Or `or` Logical OR @@ -10037,32 +10131,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - PercentRank `percent_rank` Window function that returns the percent rank value within the aggregation window @@ -10357,6 +10425,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + PreciseTimestampConversion Expression used internally to convert the TimestampType to Long and back without losing precision, i.e. in microseconds. Used in time windowing @@ -10404,32 +10498,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - PromotePrecision PromotePrecision before arithmetic operations between DecimalType data diff --git a/integration_tests/src/main/python/date_time_test.py b/integration_tests/src/main/python/date_time_test.py index 722febeb85e..a22ea24ac2b 100644 --- a/integration_tests/src/main/python/date_time_test.py +++ b/integration_tests/src/main/python/date_time_test.py @@ -215,6 +215,21 @@ def test_to_unix_timestamp(data_gen, ansi_enabled): {'spark.sql.ansi.enabled': ansi_enabled}) +@pytest.mark.parametrize('time_zone', ["UTC", "UTC+0", "UTC-0", "GMT", "GMT+0", "GMT-0"], ids=idfn) +@pytest.mark.parametrize('data_gen', [timestamp_gen], ids=idfn) +def test_from_utc_timestamp(data_gen, time_zone): + assert_gpu_and_cpu_are_equal_collect( + lambda spark : unary_op_df(spark, data_gen).select(f.from_utc_timestamp(f.col('a'), time_zone))) + +@allow_non_gpu('ProjectExec, FromUTCTimestamp') +@pytest.mark.parametrize('time_zone', ["PST", "MST", "EST", "VST", "NST", "AST"], ids=idfn) +@pytest.mark.parametrize('data_gen', [timestamp_gen], ids=idfn) +def test_from_utc_timestamp_fallback(data_gen, time_zone): + assert_gpu_fallback_collect( + lambda spark : unary_op_df(spark, data_gen).select(f.from_utc_timestamp(f.col('a'), time_zone)), + 'ProjectExec') + + @pytest.mark.parametrize('invalid,fmt', [ ('2021-01/01', 'yyyy-MM-dd'), ('2021/01-01', 'yyyy/MM/dd'), diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 780c9038b3b..eae4fa99ab0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -1845,6 +1845,15 @@ object GpuOverrides extends Logging { // passing the already converted strf string for a little optimization GpuFromUnixTime(lhs, rhs, strfFormat) }), + expr[FromUTCTimestamp]( + "Render the input UTC timestamp in the input timezone", + ExprChecks.binaryProject(TypeSig.TIMESTAMP, TypeSig.TIMESTAMP, + ("timestamp", TypeSig.TIMESTAMP, TypeSig.TIMESTAMP), + ("timezone", TypeSig.lit(TypeEnum.STRING) + .withPsNote(TypeEnum.STRING, "Only timezones equivalent to UTC are supported"), + TypeSig.lit(TypeEnum.STRING))), + (a, conf, p, r) => new FromUTCTimestampExprMeta(a, conf, p, r) + ), expr[Pmod]( "Pmod", ExprChecks.binaryProject(TypeSig.gpuNumeric, TypeSig.cpuNumeric, @@ -2991,7 +3000,7 @@ object GpuOverrides extends Logging { "the older versions of Spark in this instance and handle NaNs the same as 3.1.3+"), expr[ArraysOverlap]( "Returns true if a1 contains at least a non-null element present also in a2. If the arrays " + - "have no common element and they are both non-empty and either of them contains a null " + + "have no common element and they are both non-empty and either of them contains a null " + "element null is returned, false otherwise.", ExprChecks.binaryProject(TypeSig.BOOLEAN, TypeSig.BOOLEAN, ("array1", @@ -3408,7 +3417,7 @@ object GpuOverrides extends Logging { TypeSig.ARRAY.nested(TypeSig.all), Seq(ParamCheck("input", (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + - TypeSig.NULL + + TypeSig.NULL + TypeSig.STRUCT.withPsNote(TypeEnum.STRUCT, "Support for structs containing " + s"float/double array columns requires ${RapidsConf.HAS_NANS} to be set to false") + TypeSig.ARRAY.withPsNote(TypeEnum.ARRAY, "Support for arrays of arrays of " + @@ -3418,7 +3427,7 @@ object GpuOverrides extends Logging { private def isNestedArrayType(dt: DataType): Boolean = { dt match { - case StructType(fields) => + case StructType(fields) => fields.exists { field => field.dataType match { case sdt: StructType => isNestedArrayType(sdt) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala index 0ec1a295b5c..f1ea25eb1fe 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala @@ -16,6 +16,7 @@ package org.apache.spark.sql.rapids +import java.time.ZoneId import java.util.concurrent.TimeUnit import ai.rapids.cudf.{BinaryOp, ColumnVector, ColumnView, DType, Scalar} @@ -24,7 +25,7 @@ import com.nvidia.spark.rapids.GpuOverrides.{extractStringLit, getTimeParserPoli import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.shims.ShimBinaryExpression -import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, ExpectsInputTypes, Expression, ImplicitCastInputTypes, NullIntolerant, TimeZoneAwareExpression} +import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, ExpectsInputTypes, Expression, FromUTCTimestamp, ImplicitCastInputTypes, NullIntolerant, TimeZoneAwareExpression} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch @@ -877,6 +878,71 @@ case class GpuFromUnixTime( override lazy val resolved: Boolean = childrenResolved && checkInputDataTypes().isSuccess } + +class FromUTCTimestampExprMeta( + expr: FromUTCTimestamp, + override val conf: RapidsConf, + override val parent: Option[RapidsMeta[_, _, _]], + rule: DataFromReplacementRule) + extends BinaryExprMeta[FromUTCTimestamp](expr, conf, parent, rule) { + + override def tagExprForGpu(): Unit = { + extractStringLit(expr.right) match { + case None => + willNotWorkOnGpu("timezone input must be a literal string") + case Some(timezoneShortID) => + if (timezoneShortID != null) { + val utc = ZoneId.of("UTC").normalized + // This is copied from Spark, to convert `(+|-)h:mm` into `(+|-)0h:mm`. + val timezone = ZoneId.of(timezoneShortID.replaceFirst("(\\+|\\-)(\\d):", "$10$2:"), + ZoneId.SHORT_IDS).normalized + + if (timezone != utc) { + willNotWorkOnGpu("only timezones equivalent to UTC are supported") + } + } + } + } + + override def convertToGpu(timestamp: Expression, timezone: Expression): GpuExpression = + GpuFromUTCTimestamp(timestamp, timezone) +} + +case class GpuFromUTCTimestamp(timestamp: Expression, timezone: Expression) + extends GpuBinaryExpression with ImplicitCastInputTypes with NullIntolerant { + + override def left: Expression = timestamp + override def right: Expression = timezone + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, StringType) + override def dataType: DataType = TimestampType + + override def doColumnar(lhs: GpuColumnVector, rhs: GpuColumnVector): ColumnVector = { + throw new IllegalStateException( + "Cannot have time zone given by a column vector in GpuFromUTCTimestamp") + } + + override def doColumnar(lhs: GpuScalar, rhs: GpuColumnVector): ColumnVector = { + throw new IllegalStateException( + "Cannot have time zone given by a column vector in GpuFromUTCTimestamp") + } + + override def doColumnar(lhs: GpuColumnVector, rhs: GpuScalar): ColumnVector = { + if (rhs.getBase.isValid) { + // Just a no-op. + lhs.getBase.incRefCount() + } else { + // All-null output column. + GpuColumnVector.columnVectorFromNull(lhs.getRowCount.toInt, dataType) + } + } + + override def doColumnar(numRows: Int, lhs: GpuScalar, rhs: GpuScalar): ColumnVector = { + withResource(GpuColumnVector.from(lhs, numRows, left.dataType)) { lhsCol => + doColumnar(lhsCol, rhs) + } + } +} + trait GpuDateMathBase extends GpuBinaryExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(DateType, TypeCollection(IntegerType, ShortType, ByteType)) diff --git a/tools/src/main/resources/operatorsScore.csv b/tools/src/main/resources/operatorsScore.csv index aeaf6fd9fb4..ce62bb87fc5 100644 --- a/tools/src/main/resources/operatorsScore.csv +++ b/tools/src/main/resources/operatorsScore.csv @@ -103,6 +103,7 @@ Explode,4 Expm1,4 First,4 Floor,4 +FromUTCTimestamp,4 FromUnixTime,4 GetArrayItem,4 GetArrayStructFields,4 diff --git a/tools/src/main/resources/supportedExprs.csv b/tools/src/main/resources/supportedExprs.csv index b68b46e1f66..df2b31a3848 100644 --- a/tools/src/main/resources/supportedExprs.csv +++ b/tools/src/main/resources/supportedExprs.csv @@ -197,6 +197,9 @@ Expm1,S,`expm1`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Expm1,S,`expm1`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Floor,S,`floor`,None,project,input,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA Floor,S,`floor`,None,project,result,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA +FromUTCTimestamp,S,`from_utc_timestamp`,None,project,timestamp,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA +FromUTCTimestamp,S,`from_utc_timestamp`,None,project,timezone,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA +FromUTCTimestamp,S,`from_utc_timestamp`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA FromUnixTime,S,`from_unixtime`,None,project,sec,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA FromUnixTime,S,`from_unixtime`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA FromUnixTime,S,`from_unixtime`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA From 1e2a074fa681b9f80166f87d209b45b8d138e49e Mon Sep 17 00:00:00 2001 From: sinkinben Date: Mon, 29 Aug 2022 16:02:25 +0800 Subject: [PATCH 059/190] Support bool/int8/int16/int32/int64 castings for ORC reading. (#6273) * feat: implement casting bool/int8/int16/int32/int64 -> {string, float, double(float64), timestamp} In ORC reading, casting from integers to timestamp has different behaviors in different versions of spark. * From spark311 until spark330, it considers integers as milliseconds. * Since spark330, it considers integers as seconds. For more details, see https://github.com/NVIDIA/spark-rapids/pull/6273 Signed-off-by: sinkinben --- .../src/main/python/orc_cast_test.py | 84 +++++++++++++++++++ integration_tests/src/main/python/orc_test.py | 20 ----- .../spark/rapids/shims/OrcCastingShims.scala | 62 ++++++++++++++ .../spark/rapids/shims/OrcCastingShims.scala | 62 ++++++++++++++ .../com/nvidia/spark/rapids/GpuOrcScan.scala | 50 +++++++++-- 5 files changed, 251 insertions(+), 27 deletions(-) create mode 100644 integration_tests/src/main/python/orc_cast_test.py create mode 100644 sql-plugin/src/main/311until330-all/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala create mode 100644 sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala diff --git a/integration_tests/src/main/python/orc_cast_test.py b/integration_tests/src/main/python/orc_cast_test.py new file mode 100644 index 00000000000..98fcbc8b22f --- /dev/null +++ b/integration_tests/src/main/python/orc_cast_test.py @@ -0,0 +1,84 @@ +# Copyright (c) 2020-2022, NVIDIA CORPORATION. +# +# 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. + +import pytest + +from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error +from data_gen import * +from pyspark.sql.types import * +from spark_session import with_cpu_session +from orc_test import reader_opt_confs + + +def create_orc(data_gen_list, data_path): + # generate ORC dataframe, and dump it to local file 'data_path' + with_cpu_session( + lambda spark: gen_df(spark, data_gen_list).write.orc(data_path) + ) + + +int_gens = [boolean_gen] + integral_gens +@pytest.mark.parametrize('offset', list(range(1, len(int_gens))), ids=idfn) +@pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) +@pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) +def test_read_type_casting_integral(spark_tmp_path, offset, reader_confs, v1_enabled_list): + # cast integral types to another integral types + gen_list = [('c' + str(i), gen) for i, gen in enumerate(int_gens)] + data_path = spark_tmp_path + '/ORC_DATA' + create_orc(gen_list, data_path) + + # build the read schema by a left shift of int_gens + shifted_int_gens = int_gens[offset:] + int_gens[:offset] + rs_gen_list = [('c' + str(i), gen) for i, gen in enumerate(shifted_int_gens)] + rs = StructGen(rs_gen_list, nullable=False).data_type + all_confs = copy_and_update(reader_confs, + {'spark.sql.sources.useV1SourceList': v1_enabled_list}) + assert_gpu_and_cpu_are_equal_collect( + lambda spark: spark.read.schema(rs).orc(data_path), + conf=all_confs) + + +@pytest.mark.parametrize('to_type', ['float', 'double', 'string', 'timestamp']) +def test_casting_from_integer(spark_tmp_path, to_type): + orc_path = spark_tmp_path + '/orc_cast_integer' + # The Python 'datetime' module only supports a max-year of 10000, so we set the Long type max + # to '1e11'. If the long-value is out of this range, pytest will throw an exception. + data_gen = [('boolean_col', boolean_gen), ('tinyint_col', byte_gen), + ('smallint_col', ShortGen(min_val=BYTE_MAX + 1)), + ('int_col', IntegerGen(min_val=SHORT_MAX + 1)), + ('bigint_col', LongGen(min_val=INT_MAX + 1, max_val=int(1e11))), + ('negint_col', IntegerGen(max_val=-1))] + create_orc(data_gen, orc_path) + + schema_str = "boolean_col {}, tinyint_col {}, smallint_col {}, int_col {}, bigint_col {}, negint_col {}" + assert_gpu_and_cpu_are_equal_collect( + lambda spark: spark.read.schema( + schema_str.format(*([to_type] * len(data_gen)))).orc(orc_path) + ) + +@pytest.mark.parametrize('overflow_long_gen', [LongGen(min_val=int(1e16)), + LongGen(max_val=int(-1e16))]) +@pytest.mark.parametrize('to_type', ['timestamp']) +def test_casting_from_overflow_long(spark_tmp_path, overflow_long_gen,to_type): + # Timestamp(micro-seconds) is actually type of int64, when casting long(int64) to timestamp, + # we need to multiply 1e6 (or 1e3), and it may cause overflow. This function aims to test + # whether if 'ArithmeticException' is caught. + orc_path = spark_tmp_path + '/orc_cast_overflow_long' + create_orc([('long_column', overflow_long_gen)], orc_path) + schema_str = "long_column {}".format(to_type) + assert_gpu_and_cpu_error( + df_fun=lambda spark: spark.read.schema(schema_str).orc(orc_path).collect(), + conf={}, + error_message="ArithmeticException" + ) diff --git a/integration_tests/src/main/python/orc_test.py b/integration_tests/src/main/python/orc_test.py index c063dd07dbb..7c0c775fc68 100644 --- a/integration_tests/src/main/python/orc_test.py +++ b/integration_tests/src/main/python/orc_test.py @@ -671,26 +671,6 @@ def test_orc_scan_with_aggregate_no_pushdown_on_col_partition(spark_tmp_path, ag conf=_orc_aggregate_pushdown_enabled_conf) -@pytest.mark.parametrize('offset', [1,2,3,4], ids=idfn) -@pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) -@pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) -def test_read_type_casting_integral(spark_tmp_path, offset, reader_confs, v1_enabled_list): - int_gens = [boolean_gen] + integral_gens - gen_list = [('c' + str(i), gen) for i, gen in enumerate(int_gens)] - data_path = spark_tmp_path + '/ORC_DATA' - with_cpu_session( - lambda spark: gen_df(spark, gen_list).write.orc(data_path)) - - # build the read schema by a left shift of int_gens - shifted_int_gens = int_gens[offset:] + int_gens[:offset] - rs_gen_list = [('c' + str(i), gen) for i, gen in enumerate(shifted_int_gens)] - rs = StructGen(rs_gen_list, nullable=False).data_type - all_confs = copy_and_update(reader_confs, - {'spark.sql.sources.useV1SourceList': v1_enabled_list}) - assert_gpu_and_cpu_are_equal_collect( - lambda spark: spark.read.schema(rs).orc(data_path), - conf=all_confs) - def test_orc_read_count(spark_tmp_path): data_path = spark_tmp_path + '/ORC_DATA' orc_gens = [int_gen, string_gen, double_gen] diff --git a/sql-plugin/src/main/311until330-all/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala b/sql-plugin/src/main/311until330-all/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala new file mode 100644 index 00000000000..032ce2d1787 --- /dev/null +++ b/sql-plugin/src/main/311until330-all/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala @@ -0,0 +1,62 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import ai.rapids.cudf.{ColumnView, DType, Scalar} +import com.nvidia.spark.rapids.GpuOrcScan.{testLongMultiplicationOverflow, withResource} + +object OrcCastingShims { + /** + * Cast ColumnView of integer types to timestamp (in milliseconds). + * @param col The column view of integer types. + * @param fromType BOOL8, INT8/16/32/64 + * @return A new timestamp columnar vector. + */ + def castIntegerToTimestamp(col: ColumnView, fromType: DType): ColumnView = { + fromType match { + case DType.BOOL8 | DType.INT8 | DType.INT16 | DType.INT32 => + // From spark311 until spark330 (not include it), spark consider the integers as + // milli-seconds. + // cuDF requires casting to Long first, then we can cast Long to Timestamp(in microseconds) + // In CPU code of ORC casting, its conversion is 'integer -> milliseconds -> microseconds' + withResource(col.castTo(DType.INT64)) { longs => + withResource(Scalar.fromLong(1000L)) { thousand => + withResource(longs.mul(thousand)) { milliSeconds => + milliSeconds.castTo(DType.TIMESTAMP_MICROSECONDS) + } + } + } + case DType.INT64 => + // We need overflow checking here, since max value of INT64 is about 9 * 1e18, and convert + // INT64 to milliseconds(also a INT64 actually), we need multiply 1000, it may cause long + // integer-overflow. + // If these two 'testLongMultiplicationOverflow' throw no exception, it means no + // Long-overflow when casting 'col' to TIMESTAMP_MICROSECONDS. + if (col.max() != null) { + testLongMultiplicationOverflow(col.max().getLong, 1000L) + } + if (col.min() != null) { + testLongMultiplicationOverflow(col.min().getLong, 1000L) + } + withResource(Scalar.fromLong(1000L)) { thousand => + withResource(col.mul(thousand)) { milliSeconds => + milliSeconds.castTo(DType.TIMESTAMP_MICROSECONDS) + } + } + } + } +} diff --git a/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala b/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala new file mode 100644 index 00000000000..484b41b5429 --- /dev/null +++ b/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala @@ -0,0 +1,62 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import ai.rapids.cudf.{ColumnView, DType, Scalar} +import com.nvidia.spark.rapids.GpuOrcScan.{testLongMultiplicationOverflow, withResource} + +object OrcCastingShims { + /** + * Cast ColumnView of integer types to timestamp (in milliseconds). + * @param col The column view of integer types. + * @param fromType BOOL8, INT8/16/32/64 + * @return A new timestamp columnar vector. + */ + def castIntegerToTimestamp(col: ColumnView, fromType: DType): ColumnView = { + fromType match { + case DType.BOOL8 | DType.INT8 | DType.INT16 | DType.INT32 => + // From spark330, spark consider the integers as seconds. + withResource(col.castTo(DType.INT64)) { longs => + // In CPU, ORC assumes the integer value is in seconds, and returns timestamp in + // micro seconds, so we need to multiply 1e6 here. + withResource(Scalar.fromLong(1000000L)) { value => + withResource(longs.mul(value)) { microSeconds => + microSeconds.castTo(DType.TIMESTAMP_MICROSECONDS) + } + } + } + + case DType.INT64 => + // In CPU code of ORC casting, its conversion is 'integer -> milliseconds -> microseconds' + withResource(Scalar.fromLong(1000L)) { thousand => + withResource(col.mul(thousand)) { milliSeconds => + // We need to check long-overflow here. If milliseconds can not convert to + // micorseconds, then testLongMultiplicationOverflow will throw exception. + if (milliSeconds.max() != null) { + testLongMultiplicationOverflow(milliSeconds.max().getLong, 1000L) + } + if (milliSeconds.min() != null) { + testLongMultiplicationOverflow(milliSeconds.min().getLong, 1000L) + } + withResource(milliSeconds.mul(thousand)) { microSeconds => + microSeconds.castTo(DType.TIMESTAMP_MICROSECONDS) + } + } + } + } + } +} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index 4cca3c862b4..fa488debbcb 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -35,7 +35,7 @@ import com.google.protobuf.CodedOutputStream import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.SchemaUtils._ -import com.nvidia.spark.rapids.shims.{OrcReadingShims, OrcShims, ShimFilePartitionReaderFactory} +import com.nvidia.spark.rapids.shims.{OrcCastingShims, OrcReadingShims, OrcShims, ShimFilePartitionReaderFactory} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.common.io.DiskRangeList @@ -211,6 +211,28 @@ object GpuOrcScan extends Arm { } else { downCastAnyInteger(col, toDt) } + + // bool to float, double(float64) + case (DType.BOOL8, DType.FLOAT32 | DType.FLOAT64) => + col.castTo(toDt) + + // bool to string + case (DType.BOOL8, DType.STRING) => + withResource(col.castTo(toDt)) { casted => + // cuDF produces "ture"/"false" while CPU outputs "TRUE"/"FALSE". + casted.upper() + } + + // integer to float, double(float64), string + case (DType.INT8 | DType.INT16 | DType.INT32 | DType.INT64, + DType.FLOAT32 | DType.FLOAT64 | DType.STRING) => + col.castTo(toDt) + + // {bool, integer types} to timestamp(micro seconds) + case (DType.BOOL8 | DType.INT8 | DType.INT16 | DType.INT32 | DType.INT64, + DType.TIMESTAMP_MICROSECONDS) => + OrcCastingShims.castIntegerToTimestamp(col, fromDt) + // TODO more types, tracked in https://github.com/NVIDIA/spark-rapids/issues/5895 case (f, t) => throw new QueryExecutionException(s"Unsupported type casting: $f -> $t") @@ -231,19 +253,33 @@ object GpuOrcScan extends Arm { // Align with what CPU does. return false } + val toType = to.getCategory from.getCategory match { case BOOLEAN | BYTE | SHORT | INT | LONG => - to.getCategory match { - case BOOLEAN | BYTE | SHORT | INT | LONG => true + toType match { + case BOOLEAN | BYTE | SHORT | INT | LONG | FLOAT | DOUBLE | STRING | + TIMESTAMP => true + // BINARY and DATE are not supported by design. + // The 'to' type (aka read schema) is from Spark, and VARCHAR and CHAR will + // be replaced by STRING. Meanwhile, cuDF doesn't support them as output + // types, and also replaces them with STRING. + // TIMESTAMP_INSTANT is not supported by cuDF. case _ => false } case VARCHAR => - to.getCategory == STRING - // TODO more types, tracked in https://github.com/NVIDIA/spark-rapids/issues/5895 - case _ => - false + toType == STRING + case _ => false } } + + /** + * Test whether if a * b will cause Long-overflow. + * In Math.multiplyExact, if there is an integer-overflow, then it will throw an + * ArithmeticException. + */ + def testLongMultiplicationOverflow(a: Long, b: Long) = { + Math.multiplyExact(a, b) + } } /** From 8a93d8a8ea2efe9a82e9d9ae967fb77664d9dde2 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Mon, 29 Aug 2022 13:09:52 -0500 Subject: [PATCH 060/190] Skip decimal gens that overflow on Spark 3.3.0+ (#6441) Signed-off-by: Jason Lowe Signed-off-by: Jason Lowe --- .../src/main/python/arithmetic_ops_test.py | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/integration_tests/src/main/python/arithmetic_ops_test.py b/integration_tests/src/main/python/arithmetic_ops_test.py index c4b7bff6f46..b5e3b4e5a40 100644 --- a/integration_tests/src/main/python/arithmetic_ops_test.py +++ b/integration_tests/src/main/python/arithmetic_ops_test.py @@ -48,11 +48,14 @@ _decimal_gen_38_10 = DecimalGen(precision=38, scale=10) _decimal_gen_38_neg10 = DecimalGen(precision=38, scale=-10) -_arith_data_gens_diff_precision_scale_and_no_neg_scale = [ +_arith_data_gens_diff_precision_scale_and_no_neg_scale_no_38_0 = [ decimal_gen_32bit, decimal_gen_64bit, _decimal_gen_18_0, decimal_gen_128bit, - _decimal_gen_30_2, _decimal_gen_36_5, _decimal_gen_38_0, _decimal_gen_38_10 + _decimal_gen_30_2, _decimal_gen_36_5, _decimal_gen_38_10 ] +_arith_data_gens_diff_precision_scale_and_no_neg_scale = \ + _arith_data_gens_diff_precision_scale_and_no_neg_scale_no_38_0 + [_decimal_gen_38_0] + _arith_decimal_gens_no_neg_scale = _arith_data_gens_diff_precision_scale_and_no_neg_scale + [_decimal_gen_7_7] _arith_decimal_gens = _arith_decimal_gens_no_neg_scale + [ @@ -63,6 +66,12 @@ _arith_data_gens_no_neg_scale = numeric_gens + _arith_decimal_gens_no_neg_scale +_arith_decimal_gens_no_neg_scale_38_0_overflow = \ + _arith_data_gens_diff_precision_scale_and_no_neg_scale_no_38_0 + [ + _decimal_gen_7_7, + pytest.param(_decimal_gen_38_0, marks=pytest.mark.skipif( + is_spark_330_or_later(), reason='This case overflows in Spark 3.3.0+'))] + def _get_overflow_df(spark, data, data_type, expr): return spark.createDataFrame( SparkContext.getOrCreate().parallelize([data]), @@ -458,7 +467,7 @@ def test_floor_scale_zero(data_gen): @pytest.mark.skipif(is_before_spark_330(), reason='scale parameter in Floor function is not supported before Spark 3.3.0') @allow_non_gpu('ProjectExec') -@pytest.mark.parametrize('data_gen', double_n_long_gens + _arith_decimal_gens_no_neg_scale, ids=idfn) +@pytest.mark.parametrize('data_gen', double_n_long_gens + _arith_decimal_gens_no_neg_scale_38_0_overflow, ids=idfn) def test_floor_scale_nonzero(data_gen): assert_gpu_fallback_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr('floor(a, -1)'), 'RoundFloor') @@ -529,7 +538,7 @@ def test_shift_right_unsigned(data_gen): 'shiftrightunsigned(a, cast(null as INT))', 'shiftrightunsigned(a, b)')) -_arith_data_gens_for_round = numeric_gens + _arith_decimal_gens_no_neg_scale + [ +_arith_data_gens_for_round = numeric_gens + _arith_decimal_gens_no_neg_scale_38_0_overflow + [ decimal_gen_32bit_neg_scale, DecimalGen(precision=15, scale=-8), DecimalGen(precision=30, scale=-5), From 3c4af4b45dab44aad89a3808117ba3b1524a02bb Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 29 Aug 2022 19:52:53 -0500 Subject: [PATCH 061/190] Revert patch that caused failing test on databricks 321 [databricks] (#6448) * Revert "Support bool/int8/int16/int32/int64 castings for ORC reading. (#6273)" This reverts commit 1e2a074fa681b9f80166f87d209b45b8d138e49e. * Reverted #6273 because of #6446 Signed-off-by: Robert (Bobby) Evans Signed-off-by: Robert (Bobby) Evans --- .../src/main/python/orc_cast_test.py | 84 ------------------- integration_tests/src/main/python/orc_test.py | 20 +++++ .../spark/rapids/shims/OrcCastingShims.scala | 62 -------------- .../spark/rapids/shims/OrcCastingShims.scala | 62 -------------- .../com/nvidia/spark/rapids/GpuOrcScan.scala | 50 ++--------- 5 files changed, 27 insertions(+), 251 deletions(-) delete mode 100644 integration_tests/src/main/python/orc_cast_test.py delete mode 100644 sql-plugin/src/main/311until330-all/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala delete mode 100644 sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala diff --git a/integration_tests/src/main/python/orc_cast_test.py b/integration_tests/src/main/python/orc_cast_test.py deleted file mode 100644 index 98fcbc8b22f..00000000000 --- a/integration_tests/src/main/python/orc_cast_test.py +++ /dev/null @@ -1,84 +0,0 @@ -# Copyright (c) 2020-2022, NVIDIA CORPORATION. -# -# 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. - -import pytest - -from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error -from data_gen import * -from pyspark.sql.types import * -from spark_session import with_cpu_session -from orc_test import reader_opt_confs - - -def create_orc(data_gen_list, data_path): - # generate ORC dataframe, and dump it to local file 'data_path' - with_cpu_session( - lambda spark: gen_df(spark, data_gen_list).write.orc(data_path) - ) - - -int_gens = [boolean_gen] + integral_gens -@pytest.mark.parametrize('offset', list(range(1, len(int_gens))), ids=idfn) -@pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) -@pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) -def test_read_type_casting_integral(spark_tmp_path, offset, reader_confs, v1_enabled_list): - # cast integral types to another integral types - gen_list = [('c' + str(i), gen) for i, gen in enumerate(int_gens)] - data_path = spark_tmp_path + '/ORC_DATA' - create_orc(gen_list, data_path) - - # build the read schema by a left shift of int_gens - shifted_int_gens = int_gens[offset:] + int_gens[:offset] - rs_gen_list = [('c' + str(i), gen) for i, gen in enumerate(shifted_int_gens)] - rs = StructGen(rs_gen_list, nullable=False).data_type - all_confs = copy_and_update(reader_confs, - {'spark.sql.sources.useV1SourceList': v1_enabled_list}) - assert_gpu_and_cpu_are_equal_collect( - lambda spark: spark.read.schema(rs).orc(data_path), - conf=all_confs) - - -@pytest.mark.parametrize('to_type', ['float', 'double', 'string', 'timestamp']) -def test_casting_from_integer(spark_tmp_path, to_type): - orc_path = spark_tmp_path + '/orc_cast_integer' - # The Python 'datetime' module only supports a max-year of 10000, so we set the Long type max - # to '1e11'. If the long-value is out of this range, pytest will throw an exception. - data_gen = [('boolean_col', boolean_gen), ('tinyint_col', byte_gen), - ('smallint_col', ShortGen(min_val=BYTE_MAX + 1)), - ('int_col', IntegerGen(min_val=SHORT_MAX + 1)), - ('bigint_col', LongGen(min_val=INT_MAX + 1, max_val=int(1e11))), - ('negint_col', IntegerGen(max_val=-1))] - create_orc(data_gen, orc_path) - - schema_str = "boolean_col {}, tinyint_col {}, smallint_col {}, int_col {}, bigint_col {}, negint_col {}" - assert_gpu_and_cpu_are_equal_collect( - lambda spark: spark.read.schema( - schema_str.format(*([to_type] * len(data_gen)))).orc(orc_path) - ) - -@pytest.mark.parametrize('overflow_long_gen', [LongGen(min_val=int(1e16)), - LongGen(max_val=int(-1e16))]) -@pytest.mark.parametrize('to_type', ['timestamp']) -def test_casting_from_overflow_long(spark_tmp_path, overflow_long_gen,to_type): - # Timestamp(micro-seconds) is actually type of int64, when casting long(int64) to timestamp, - # we need to multiply 1e6 (or 1e3), and it may cause overflow. This function aims to test - # whether if 'ArithmeticException' is caught. - orc_path = spark_tmp_path + '/orc_cast_overflow_long' - create_orc([('long_column', overflow_long_gen)], orc_path) - schema_str = "long_column {}".format(to_type) - assert_gpu_and_cpu_error( - df_fun=lambda spark: spark.read.schema(schema_str).orc(orc_path).collect(), - conf={}, - error_message="ArithmeticException" - ) diff --git a/integration_tests/src/main/python/orc_test.py b/integration_tests/src/main/python/orc_test.py index 7c0c775fc68..c063dd07dbb 100644 --- a/integration_tests/src/main/python/orc_test.py +++ b/integration_tests/src/main/python/orc_test.py @@ -671,6 +671,26 @@ def test_orc_scan_with_aggregate_no_pushdown_on_col_partition(spark_tmp_path, ag conf=_orc_aggregate_pushdown_enabled_conf) +@pytest.mark.parametrize('offset', [1,2,3,4], ids=idfn) +@pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) +@pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) +def test_read_type_casting_integral(spark_tmp_path, offset, reader_confs, v1_enabled_list): + int_gens = [boolean_gen] + integral_gens + gen_list = [('c' + str(i), gen) for i, gen in enumerate(int_gens)] + data_path = spark_tmp_path + '/ORC_DATA' + with_cpu_session( + lambda spark: gen_df(spark, gen_list).write.orc(data_path)) + + # build the read schema by a left shift of int_gens + shifted_int_gens = int_gens[offset:] + int_gens[:offset] + rs_gen_list = [('c' + str(i), gen) for i, gen in enumerate(shifted_int_gens)] + rs = StructGen(rs_gen_list, nullable=False).data_type + all_confs = copy_and_update(reader_confs, + {'spark.sql.sources.useV1SourceList': v1_enabled_list}) + assert_gpu_and_cpu_are_equal_collect( + lambda spark: spark.read.schema(rs).orc(data_path), + conf=all_confs) + def test_orc_read_count(spark_tmp_path): data_path = spark_tmp_path + '/ORC_DATA' orc_gens = [int_gen, string_gen, double_gen] diff --git a/sql-plugin/src/main/311until330-all/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala b/sql-plugin/src/main/311until330-all/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala deleted file mode 100644 index 032ce2d1787..00000000000 --- a/sql-plugin/src/main/311until330-all/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Copyright (c) 2022, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids.shims - -import ai.rapids.cudf.{ColumnView, DType, Scalar} -import com.nvidia.spark.rapids.GpuOrcScan.{testLongMultiplicationOverflow, withResource} - -object OrcCastingShims { - /** - * Cast ColumnView of integer types to timestamp (in milliseconds). - * @param col The column view of integer types. - * @param fromType BOOL8, INT8/16/32/64 - * @return A new timestamp columnar vector. - */ - def castIntegerToTimestamp(col: ColumnView, fromType: DType): ColumnView = { - fromType match { - case DType.BOOL8 | DType.INT8 | DType.INT16 | DType.INT32 => - // From spark311 until spark330 (not include it), spark consider the integers as - // milli-seconds. - // cuDF requires casting to Long first, then we can cast Long to Timestamp(in microseconds) - // In CPU code of ORC casting, its conversion is 'integer -> milliseconds -> microseconds' - withResource(col.castTo(DType.INT64)) { longs => - withResource(Scalar.fromLong(1000L)) { thousand => - withResource(longs.mul(thousand)) { milliSeconds => - milliSeconds.castTo(DType.TIMESTAMP_MICROSECONDS) - } - } - } - case DType.INT64 => - // We need overflow checking here, since max value of INT64 is about 9 * 1e18, and convert - // INT64 to milliseconds(also a INT64 actually), we need multiply 1000, it may cause long - // integer-overflow. - // If these two 'testLongMultiplicationOverflow' throw no exception, it means no - // Long-overflow when casting 'col' to TIMESTAMP_MICROSECONDS. - if (col.max() != null) { - testLongMultiplicationOverflow(col.max().getLong, 1000L) - } - if (col.min() != null) { - testLongMultiplicationOverflow(col.min().getLong, 1000L) - } - withResource(Scalar.fromLong(1000L)) { thousand => - withResource(col.mul(thousand)) { milliSeconds => - milliSeconds.castTo(DType.TIMESTAMP_MICROSECONDS) - } - } - } - } -} diff --git a/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala b/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala deleted file mode 100644 index 484b41b5429..00000000000 --- a/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Copyright (c) 2022, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids.shims - -import ai.rapids.cudf.{ColumnView, DType, Scalar} -import com.nvidia.spark.rapids.GpuOrcScan.{testLongMultiplicationOverflow, withResource} - -object OrcCastingShims { - /** - * Cast ColumnView of integer types to timestamp (in milliseconds). - * @param col The column view of integer types. - * @param fromType BOOL8, INT8/16/32/64 - * @return A new timestamp columnar vector. - */ - def castIntegerToTimestamp(col: ColumnView, fromType: DType): ColumnView = { - fromType match { - case DType.BOOL8 | DType.INT8 | DType.INT16 | DType.INT32 => - // From spark330, spark consider the integers as seconds. - withResource(col.castTo(DType.INT64)) { longs => - // In CPU, ORC assumes the integer value is in seconds, and returns timestamp in - // micro seconds, so we need to multiply 1e6 here. - withResource(Scalar.fromLong(1000000L)) { value => - withResource(longs.mul(value)) { microSeconds => - microSeconds.castTo(DType.TIMESTAMP_MICROSECONDS) - } - } - } - - case DType.INT64 => - // In CPU code of ORC casting, its conversion is 'integer -> milliseconds -> microseconds' - withResource(Scalar.fromLong(1000L)) { thousand => - withResource(col.mul(thousand)) { milliSeconds => - // We need to check long-overflow here. If milliseconds can not convert to - // micorseconds, then testLongMultiplicationOverflow will throw exception. - if (milliSeconds.max() != null) { - testLongMultiplicationOverflow(milliSeconds.max().getLong, 1000L) - } - if (milliSeconds.min() != null) { - testLongMultiplicationOverflow(milliSeconds.min().getLong, 1000L) - } - withResource(milliSeconds.mul(thousand)) { microSeconds => - microSeconds.castTo(DType.TIMESTAMP_MICROSECONDS) - } - } - } - } - } -} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index fa488debbcb..4cca3c862b4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -35,7 +35,7 @@ import com.google.protobuf.CodedOutputStream import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.SchemaUtils._ -import com.nvidia.spark.rapids.shims.{OrcCastingShims, OrcReadingShims, OrcShims, ShimFilePartitionReaderFactory} +import com.nvidia.spark.rapids.shims.{OrcReadingShims, OrcShims, ShimFilePartitionReaderFactory} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.common.io.DiskRangeList @@ -211,28 +211,6 @@ object GpuOrcScan extends Arm { } else { downCastAnyInteger(col, toDt) } - - // bool to float, double(float64) - case (DType.BOOL8, DType.FLOAT32 | DType.FLOAT64) => - col.castTo(toDt) - - // bool to string - case (DType.BOOL8, DType.STRING) => - withResource(col.castTo(toDt)) { casted => - // cuDF produces "ture"/"false" while CPU outputs "TRUE"/"FALSE". - casted.upper() - } - - // integer to float, double(float64), string - case (DType.INT8 | DType.INT16 | DType.INT32 | DType.INT64, - DType.FLOAT32 | DType.FLOAT64 | DType.STRING) => - col.castTo(toDt) - - // {bool, integer types} to timestamp(micro seconds) - case (DType.BOOL8 | DType.INT8 | DType.INT16 | DType.INT32 | DType.INT64, - DType.TIMESTAMP_MICROSECONDS) => - OrcCastingShims.castIntegerToTimestamp(col, fromDt) - // TODO more types, tracked in https://github.com/NVIDIA/spark-rapids/issues/5895 case (f, t) => throw new QueryExecutionException(s"Unsupported type casting: $f -> $t") @@ -253,33 +231,19 @@ object GpuOrcScan extends Arm { // Align with what CPU does. return false } - val toType = to.getCategory from.getCategory match { case BOOLEAN | BYTE | SHORT | INT | LONG => - toType match { - case BOOLEAN | BYTE | SHORT | INT | LONG | FLOAT | DOUBLE | STRING | - TIMESTAMP => true - // BINARY and DATE are not supported by design. - // The 'to' type (aka read schema) is from Spark, and VARCHAR and CHAR will - // be replaced by STRING. Meanwhile, cuDF doesn't support them as output - // types, and also replaces them with STRING. - // TIMESTAMP_INSTANT is not supported by cuDF. + to.getCategory match { + case BOOLEAN | BYTE | SHORT | INT | LONG => true case _ => false } case VARCHAR => - toType == STRING - case _ => false + to.getCategory == STRING + // TODO more types, tracked in https://github.com/NVIDIA/spark-rapids/issues/5895 + case _ => + false } } - - /** - * Test whether if a * b will cause Long-overflow. - * In Math.multiplyExact, if there is an integer-overflow, then it will throw an - * ArithmeticException. - */ - def testLongMultiplicationOverflow(a: Long, b: Long) = { - Math.multiplyExact(a, b) - } } /** From 06665892bee733df52732586a0916b163dec32fb Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 29 Aug 2022 23:47:52 -0500 Subject: [PATCH 062/190] Add support for full 128-bit decimal divide (#6414) * Add support for full 128-bit divide Signed-off-by: Robert (Bobby) Evans * Addressed review comments * Attempt to fix issue with databricks Signed-off-by: Robert (Bobby) Evans Signed-off-by: Robert (Bobby) Evans --- docs/supported_ops.md | 2 +- .../src/main/python/arithmetic_ops_test.py | 46 +++++------ .../src/main/python/hash_aggregate_test.py | 7 +- .../spark/rapids/shims/Spark31XShims.scala | 21 +---- .../spark/rapids/shims/Spark31XdbShims.scala | 21 +---- .../rapids/shims/Spark320PlusShims.scala | 21 +---- .../nvidia/spark/rapids/GpuOverrides.scala | 49 +++--------- .../spark/sql/rapids/AggregateFunctions.scala | 17 ++++- .../apache/spark/sql/rapids/arithmetic.scala | 76 ++++++++++++++++++- tools/src/main/resources/supportedExprs.csv | 2 +- 10 files changed, 131 insertions(+), 131 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 5981c73de0d..45a9f48a196 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -5442,7 +5442,7 @@ are limited. -PS
Because of Spark's inner workings the full range of decimal precision (even for 128-bit values) is not supported.
+S diff --git a/integration_tests/src/main/python/arithmetic_ops_test.py b/integration_tests/src/main/python/arithmetic_ops_test.py index b5e3b4e5a40..9e2e3e06a38 100644 --- a/integration_tests/src/main/python/arithmetic_ops_test.py +++ b/integration_tests/src/main/python/arithmetic_ops_test.py @@ -190,7 +190,7 @@ def test_float_multiplication_mixed(lhs, rhs): @pytest.mark.parametrize('data_gen', [double_gen, decimal_gen_32bit_neg_scale, DecimalGen(6, 3), DecimalGen(5, 5), DecimalGen(6, 0), DecimalGen(7, 4), DecimalGen(15, 0), DecimalGen(18, 0), - DecimalGen(17, 2), DecimalGen(16, 4)], ids=idfn) + DecimalGen(17, 2), DecimalGen(16, 4), DecimalGen(38, 21), DecimalGen(21, 17), DecimalGen(3, -2)], ids=idfn) def test_division(data_gen): data_type = data_gen.data_type assert_gpu_and_cpu_are_equal_collect( @@ -201,21 +201,33 @@ def test_division(data_gen): f.col('b') / f.lit(None).cast(data_type), f.col('a') / f.col('b'))) -@allow_non_gpu('ProjectExec', 'Alias', 'Divide', 'Cast', 'PromotePrecision', 'CheckOverflow') -@pytest.mark.parametrize('data_gen', [DecimalGen(38, 21), DecimalGen(21, 17)], ids=idfn) -def test_division_fallback_on_decimal(data_gen): - assert_gpu_fallback_collect( - lambda spark : binary_op_df(spark, data_gen).select( - f.col('a') / f.col('b')), - 'Divide') - @pytest.mark.parametrize('rhs', [byte_gen, short_gen, int_gen, long_gen, DecimalGen(4, 1), DecimalGen(5, 0), DecimalGen(5, 1), DecimalGen(10, 5)], ids=idfn) @pytest.mark.parametrize('lhs', [byte_gen, short_gen, int_gen, long_gen, DecimalGen(5, 3), DecimalGen(4, 2), DecimalGen(1, -2), DecimalGen(16, 1)], ids=idfn) def test_division_mixed(lhs, rhs): assert_gpu_and_cpu_are_equal_collect( lambda spark : two_col_df(spark, lhs, rhs).select( + f.col('a'), f.col('b'), + f.col('a') / f.col('b'))) + +# Spark has some problems with some decimal operations where it can try to generate a type that is invalid (scale > precision) which results in an error +# instead of increasing the precision. So we have a second test that deals with a few of these use cases +@pytest.mark.parametrize('rhs', [DecimalGen(30, 10), DecimalGen(28, 18)], ids=idfn) +@pytest.mark.parametrize('lhs', [DecimalGen(27, 7), DecimalGen(20, -3)], ids=idfn) +def test_division_mixed_larger_dec(lhs, rhs): + assert_gpu_and_cpu_are_equal_collect( + lambda spark : two_col_df(spark, lhs, rhs).select( + f.col('a'), f.col('b'), f.col('a') / f.col('b'))) +def test_special_decimal_division(): + for precision in range(1, 39): + for scale in range(-3, precision + 1): + print("PRECISION " + str(precision) + " SCALE " + str(scale)) + data_gen = DecimalGen(precision, scale) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : two_col_df(spark, data_gen, data_gen).select( + f.col('a') / f.col('b'))) + @approximate_float # we should get the perfectly correct answer for floats except when casting a decimal to a float in some corner cases. @pytest.mark.parametrize('rhs', [float_gen, double_gen], ids=idfn) @pytest.mark.parametrize('lhs', [DecimalGen(5, 3), DecimalGen(4, 2), DecimalGen(1, -2), DecimalGen(16, 1)], ids=idfn) @@ -225,22 +237,6 @@ def test_float_division_mixed(lhs, rhs): f.col('a') / f.col('b')), conf={'spark.rapids.sql.castDecimalToFloat.enabled': 'true'}) -@ignore_order -@pytest.mark.parametrize('rhs,rhs_type', [ - (DecimalGen(15, 3), DecimalType(30, 10)), - (DecimalGen(10, 2), DecimalType(28, 18))], ids=idfn) -@pytest.mark.parametrize('lhs,lhs_type', [ - (DecimalGen(15, 3), DecimalType(27, 7)), - (DecimalGen(3, -3), DecimalType(20, -3))], ids=idfn) -def test_decimal_division_mixed_no_overflow_guarantees(lhs, lhs_type, rhs, rhs_type): - assert_gpu_and_cpu_are_equal_collect( - lambda spark : two_col_df(spark, lhs, rhs)\ - .withColumn('lhs', f.col('a').cast(lhs_type))\ - .withColumn('rhs', f.col('b').cast(rhs_type))\ - .repartition(1)\ - .select(f.col('lhs'), f.col('rhs'), f.col('lhs') / f.col('rhs')), - conf={'spark.rapids.sql.decimalOverflowGuarantees': 'false'}) - @pytest.mark.parametrize('data_gen', integral_gens + [ decimal_gen_32bit, decimal_gen_64bit, _decimal_gen_7_7, _decimal_gen_18_3, _decimal_gen_30_2, _decimal_gen_36_5, _decimal_gen_38_0], ids=idfn) diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index 40c35f474d4..7144ff7822a 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -404,11 +404,12 @@ def test_hash_reduction_sum_full_decimal(data_gen, conf): @approximate_float @ignore_order @incompat -@pytest.mark.parametrize('data_gen', _init_list_with_nans_and_no_nans + [_grpkey_short_mid_decimals, _grpkey_short_big_decimals], ids=idfn) +@pytest.mark.parametrize('data_gen', _init_list_with_nans_and_no_nans + [_grpkey_short_mid_decimals, + _grpkey_short_big_decimals, _grpkey_short_very_big_decimals, _grpkey_short_full_decimals], ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_grpby_avg(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( - lambda spark: gen_df(spark, data_gen, length=100).groupby('a').agg(f.avg('b')), + lambda spark: gen_df(spark, data_gen, length=200).groupby('a').agg(f.avg('b')), conf=conf ) @@ -1858,4 +1859,4 @@ def test_min_max_for_single_level_struct(data_gen): lambda spark : gen_df(spark, df_gen, length=1024), "hash_agg_table", 'select min(a) from hash_agg_table', - _no_nans_float_conf) \ No newline at end of file + _no_nans_float_conf) diff --git a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/Spark31XShims.scala b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/Spark31XShims.scala index 4496ea9c93c..648a498893e 100644 --- a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/Spark31XShims.scala +++ b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/Spark31XShims.scala @@ -262,26 +262,7 @@ abstract class Spark31XShims extends SparkShims with Spark31Xuntil33XShims with TypeSig.cpuNumeric))), (a, conf, p, r) => new AggExprMeta[Average](a, conf, p, r) { override def tagAggForGpu(): Unit = { - // For Decimal Average the SUM adds a precision of 10 to avoid overflowing - // then it divides by the count with an output scale that is 4 more than the input - // scale. With how our divide works to match Spark, this means that we will need a - // precision of 5 more. So 38 - 10 - 5 = 23 - val dataType = a.child.dataType - dataType match { - case dt: DecimalType => - if (dt.precision > 23) { - if (conf.needDecimalGuarantees) { - willNotWorkOnGpu("GpuAverage cannot guarantee proper overflow checks for " + - s"a precision large than 23. The current precision is ${dt.precision}") - } else { - logWarning("Decimal overflow guarantees disabled for " + - s"Average(${a.child.dataType}) produces ${dt} with an " + - s"intermediate precision of ${dt.precision + 15}") - } - } - case _ => // NOOP - } - GpuOverrides.checkAndTagFloatAgg(dataType, conf, this) + GpuOverrides.checkAndTagFloatAgg(a.child.dataType, conf, this) } override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = diff --git a/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/Spark31XdbShims.scala b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/Spark31XdbShims.scala index f09873251ca..b1328b2502b 100644 --- a/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/Spark31XdbShims.scala +++ b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/Spark31XdbShims.scala @@ -133,26 +133,7 @@ abstract class Spark31XdbShims extends Spark31XdbShimsBase with Logging { TypeSig.cpuNumeric))), (a, conf, p, r) => new AggExprMeta[Average](a, conf, p, r) { override def tagAggForGpu(): Unit = { - // For Decimal Average the SUM adds a precision of 10 to avoid overflowing - // then it divides by the count with an output scale that is 4 more than the input - // scale. With how our divide works to match Spark, this means that we will need a - // precision of 5 more. So 38 - 10 - 5 = 23 - val dataType = a.child.dataType - dataType match { - case dt: DecimalType => - if (dt.precision > 23) { - if (conf.needDecimalGuarantees) { - willNotWorkOnGpu("GpuAverage cannot guarantee proper overflow checks for " + - s"a precision large than 23. The current precision is ${dt.precision}") - } else { - logWarning("Decimal overflow guarantees disabled for " + - s"Average(${a.child.dataType}) produces ${dt} with an " + - s"intermediate precision of ${dt.precision + 15}") - } - } - case _ => // NOOP - } - GpuOverrides.checkAndTagFloatAgg(dataType, conf, this) + GpuOverrides.checkAndTagFloatAgg(a.child.dataType, conf, this) } override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = diff --git a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala index ea175b2e6d8..90137bdacf6 100644 --- a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala +++ b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala @@ -141,26 +141,7 @@ trait Spark320PlusShims extends SparkShims with RebaseShims with Logging { TypeSig.numericAndInterval + TypeSig.NULL))), (a, conf, p, r) => new AggExprMeta[Average](a, conf, p, r) { override def tagAggForGpu(): Unit = { - // For Decimal Average the SUM adds a precision of 10 to avoid overflowing - // then it divides by the count with an output scale that is 4 more than the input - // scale. With how our divide works to match Spark, this means that we will need a - // precision of 5 more. So 38 - 10 - 5 = 23 - val dataType = a.child.dataType - dataType match { - case dt: DecimalType => - if (dt.precision > 23) { - if (conf.needDecimalGuarantees) { - willNotWorkOnGpu("GpuAverage cannot guarantee proper overflow checks for " + - s"a precision large than 23. The current precision is ${dt.precision}") - } else { - logWarning("Decimal overflow guarantees disabled for " + - s"Average(${a.child.dataType}) produces $dt with an " + - s"intermediate precision of ${dt.precision + 15}") - } - } - case _ => // NOOP - } - GpuOverrides.checkAndTagFloatAgg(dataType, conf, this) + GpuOverrides.checkAndTagFloatAgg(a.child.dataType, conf, this) } override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index eae4fa99ab0..8ce49f4e4d1 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -985,40 +985,18 @@ object GpuOverrides extends Logging { private[this] lazy val rhsDecimalType = DecimalUtil.asDecimalType(rhs.wrapped.asInstanceOf[Expression].dataType) - override def tagExprForGpu(): Unit = { - a.child match { - // Division and Multiplication of Decimal types is a little odd. Spark will cast the - // inputs to a common wider value where the scale is the max of the two input scales, - // and the precision is max of the two input non-scale portions + the new scale. Then it - // will do the divide or multiply as a BigDecimal value but lie about the return type. - // Finally here in CheckOverflow it will reset the scale and check the precision so that - // Spark knows it fits in the final desired result. - // Here we try to strip out the extra casts, etc to get to as close to the original - // query as possible. This lets us then calculate what CUDF needs to get the correct - // answer, which in some cases is a lot smaller. - // For multiply we can support all of the types. For divide we still have to fall back - // to the CPU in some cases. - case _: Divide => - val intermediatePrecision = - GpuDecimalDivide.nonRoundedIntermediateArgPrecision(lhsDecimalType, - rhsDecimalType, a.dataType) - - if (intermediatePrecision > DType.DECIMAL128_MAX_PRECISION) { - if (conf.needDecimalGuarantees) { - binExpr.willNotWorkOnGpu(s"the intermediate precision of " + - s"$intermediatePrecision that is required to guarantee no overflow issues " + - s"for this divide is too large to be supported on the GPU") - } else { - logWarning("Decimal overflow guarantees disabled for " + - s"${lhs.dataType} / ${rhs.dataType} produces ${a.dataType} with an " + - s"intermediate precision of $intermediatePrecision") - } - } - case _ => // NOOP - } - } - override def convertToGpu(): GpuExpression = { + // Prior to Spark 3.4.0 + // Division and Multiplication of Decimal types is a little odd. Spark will cast the + // inputs to a common wider value where the scale is the max of the two input scales, + // and the precision is max of the two input non-scale portions + the new scale. Then it + // will do the divide or multiply as a BigDecimal value but lie about the return type. + // Finally here in CheckOverflow it will reset the scale and check the precision so that + // Spark knows it fits in the final desired result. + // Here we try to strip out the extra casts, etc to get to as close to the original + // query as possible. This lets us then calculate what CUDF needs to get the correct + // answer, which in some cases is a lot smaller. + a.child match { case _: Divide => // GpuDecimalDivide includes the overflow check in it. @@ -2135,10 +2113,7 @@ object GpuOverrides extends Logging { expr[Divide]( "Division", ExprChecks.binaryProject( - TypeSig.DOUBLE + TypeSig.DECIMAL_128 + - TypeSig.psNote(TypeEnum.DECIMAL, - "Because of Spark's inner workings the full range of decimal precision " + - "(even for 128-bit values) is not supported."), + TypeSig.DOUBLE + TypeSig.DECIMAL_128, TypeSig.DOUBLE + TypeSig.DECIMAL_128, ("lhs", TypeSig.DOUBLE + TypeSig.DECIMAL_128, TypeSig.DOUBLE + TypeSig.DECIMAL_128), diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala index 06c720e63bd..6243f85c59b 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala @@ -1461,11 +1461,20 @@ abstract class GpuDecimalAverage(child: Expression, sumDataType: DecimalType) sumDataType, nullOnOverflow = true), mergeCount.attr) + // This is here to be bug for bug compatible with Spark. They round in the divide and then cast + // the result to the final value. This loses some data in many cases and we need to be able to + // match that. This bug appears to have been fixed in Spark 3.4.0. + lazy val intermediateSparkDivideType = GpuDecimalDivide.calcOrigSparkOutputType(sumDataType, + DecimalType.LongDecimal) + // NOTE: this sets `failOnErrorOverride=false` in `GpuDivide` to force it not to throw // divide-by-zero exceptions, even when ansi mode is enabled in Spark. // This is to conform with Spark's behavior in the Average aggregate function. - override lazy val evaluateExpression: Expression = - GpuDecimalDivide(sum, count, dataType, failOnError = false) + override lazy val evaluateExpression: Expression = { + GpuCast( + GpuDecimalDivide(sum, count, intermediateSparkDivideType, failOnError = false), + dataType) + } // Window // Replace average with SUM/COUNT. This lets us run average in running window mode without @@ -1473,7 +1482,9 @@ abstract class GpuDecimalAverage(child: Expression, sumDataType: DecimalType) override def windowReplacement(spec: GpuWindowSpecDefinition): Expression = { val count = GpuWindowExpression(GpuCount(Seq(child)), spec) val sum = GpuWindowExpression(GpuSum(child, sumDataType, failOnErrorOverride = false), spec) - GpuDecimalDivide(sum, count, dataType, failOnError = false) + GpuCast( + GpuDecimalDivide(sum, count, intermediateSparkDivideType, failOnError = false), + dataType) } override val dataType: DecimalType = child.dataType match { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala index 6945296b7bb..e881e47fa48 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala @@ -815,6 +815,9 @@ case class GpuDecimalDivide( failOnError: Boolean = SQLConf.get.ansiEnabled) extends ShimExpression with GpuExpression { + // For all decimal128 output we will use the long division version. + protected lazy val useLongDivision: Boolean = dataType.precision > Decimal.MAX_LONG_DIGITS + override def toString: String = s"($left / $right)" override def sql: String = s"(${left.sql} / ${right.sql})" @@ -852,7 +855,7 @@ case class GpuDecimalDivide( } } - override def columnarEval(batch: ColumnarBatch): Any = { + def regularDivide(batch: ColumnarBatch): Any = { val castLhs = withResource(GpuExpressionsUtils.columnarEvalToColumn(left, batch)) { lhs => GpuCast.doCast(lhs.getBase, lhs.dataType(), intermediateLhsType, ansiMode = failOnError, legacyCastToString = false, stringToDateAnsiModeEnabled = false) @@ -879,12 +882,83 @@ case class GpuDecimalDivide( } } + def longDivide(batch: ColumnarBatch): Any = { + val castLhs = withResource(GpuExpressionsUtils.columnarEvalToColumn(left, batch)) { lhs => + lhs.getBase.castTo(DType.create(DType.DTypeEnum.DECIMAL128, lhs.getBase.getType.getScale)) + } + val retTab = withResource(castLhs) { castLhs => + val castRhs = withResource(GpuExpressionsUtils.columnarEvalToColumn(right, batch)) { rhs => + withResource(divByZeroFixes(rhs.getBase)) { fixed => + fixed.castTo(DType.create(DType.DTypeEnum.DECIMAL128, fixed.getType.getScale)) + } + } + withResource(castRhs) { castRhs => + com.nvidia.spark.rapids.jni.DecimalUtils.divide128(castLhs, castRhs, -dataType.scale) + } + } + val retCol = withResource(retTab) { retTab => + val overflowed = retTab.getColumn(0) + val quotient = retTab.getColumn(1) + if (failOnError) { + withResource(overflowed.any()) { anyOverflow => + if (anyOverflow.isValid && anyOverflow.getBoolean) { + throw new ArithmeticException(GpuCast.INVALID_INPUT_MESSAGE) + } + } + quotient.incRefCount() + } else { + withResource(GpuScalar.from(null, dataType)) { nullVal => + overflowed.ifElse(nullVal, quotient) + } + } + } + GpuColumnVector.from(retCol, dataType) + } + + override def columnarEval(batch: ColumnarBatch): Any = { + if (useLongDivision) { + longDivide(batch) + } else { + regularDivide(batch) + } + } + override def nullable: Boolean = true override def children: Seq[Expression] = Seq(left, right) } object GpuDecimalDivide { + // This comes from DecimalType.MINIMUM_ADJUSTED_SCALE, but for some reason it is gone + // in databricks so we have it here. + private val MINIMUM_ADJUSTED_SCALE = 6 + + def calcOrigSparkOutputType(lhs: DecimalType, rhs: DecimalType): DecimalType = { + // This comes almost directly from Spark unchanged + val allowPrecisionLoss = SQLConf.get.decimalOperationsAllowPrecisionLoss + val p1 = lhs.precision + val s1 = lhs.scale + val p2 = rhs.precision + val s2 = rhs.scale + if (allowPrecisionLoss) { + // Precision: p1 - s1 + s2 + max(6, s1 + p2 + 1) + // Scale: max(6, s1 + p2 + 1) + val intDig = p1 - s1 + s2 + val scale = math.max(MINIMUM_ADJUSTED_SCALE, s1 + p2 + 1) + val prec = intDig + scale + DecimalType.adjustPrecisionScale(prec, scale) + } else { + var intDig = math.min(DecimalType.MAX_SCALE, p1 - s1 + s2) + var decDig = math.min(DecimalType.MAX_SCALE, math.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) + } + } + // For Spark the final desired output is // new_scale = max(6, lhs.scale + rhs.precision + 1) // new_precision = lhs.precision - lhs.scale + rhs.scale + new_scale diff --git a/tools/src/main/resources/supportedExprs.csv b/tools/src/main/resources/supportedExprs.csv index df2b31a3848..f31dfa1e760 100644 --- a/tools/src/main/resources/supportedExprs.csv +++ b/tools/src/main/resources/supportedExprs.csv @@ -169,7 +169,7 @@ DenseRank,S,`dense_rank`,None,window,ordering,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS, DenseRank,S,`dense_rank`,None,window,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Divide,S,`/`,None,project,lhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA Divide,S,`/`,None,project,rhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA -Divide,S,`/`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA +Divide,S,`/`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA ElementAt,S,`element_at`,None,project,array/map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,PS,NA,NA ElementAt,S,`element_at`,None,project,index/key,PS,PS,PS,S,PS,PS,PS,PS,PS,PS,PS,NS,NS,NS,NS,NS,NS,NS ElementAt,S,`element_at`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS From 0e990a409778d4d1a2c59b03c46bb0ba2f9bb8c0 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Mon, 29 Aug 2022 23:48:11 -0500 Subject: [PATCH 063/190] Remove caching of needles in GpuInSet (#6449) Signed-off-by: Jason Lowe Signed-off-by: Jason Lowe --- .../com/nvidia/spark/rapids/GpuInSet.scala | 19 +++---------------- 1 file changed, 3 insertions(+), 16 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuInSet.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuInSet.scala index 2835b3ef158..d90b8b87c70 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuInSet.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuInSet.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2021, NVIDIA CORPORATION. + * Copyright (c) 2020-2022, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,32 +18,19 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.ColumnVector -import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, Predicate} case class GpuInSet( child: Expression, list: Seq[Any]) extends GpuUnaryExpression with Predicate { - @transient private[this] lazy val _needles: ThreadLocal[ColumnVector] = - new ThreadLocal[ColumnVector] - require(list != null, "list should not be null") override def nullable: Boolean = child.nullable || list.contains(null) override def doColumnar(haystack: GpuColumnVector): ColumnVector = { - val needles = getNeedles - haystack.getBase.contains(needles) - } - - private def getNeedles: ColumnVector = { - var needleVec = _needles.get - if (needleVec == null) { - needleVec = buildNeedles - _needles.set(needleVec) - TaskContext.get.addTaskCompletionListener[Unit](_ => _needles.get.close()) + withResource(buildNeedles) { needles => + haystack.getBase.contains(needles) } - needleVec } private def buildNeedles: ColumnVector = From d542d62546a4853b25bd5c0bfc88768246063337 Mon Sep 17 00:00:00 2001 From: Remzi Yang <59198230+HaoYang670@users.noreply.github.com> Date: Tue, 30 Aug 2022 13:45:04 +0800 Subject: [PATCH 064/190] Add `Nan` handling in the `GpuMin` (#6251) * basic structure, no null handling Signed-off-by: remzi <13716567376yh@gmail.com> * temp save Signed-off-by: remzi <13716567376yh@gmail.com> * wait the replace_nans function Signed-off-by: remzi <13716567376yh@gmail.com> * update GpuMin Signed-off-by: remzi <13716567376yh@gmail.com> * clean the window test Signed-off-by: remzi <13716567376yh@gmail.com> * update agg test Signed-off-by: remzi <13716567376yh@gmail.com> * add docs Signed-off-by: remzi <13716567376yh@gmail.com> * add docs for cudf all Signed-off-by: remzi <13716567376yh@gmail.com> * remove outdate test Signed-off-by: remzi <13716567376yh@gmail.com> * update docs and tests Signed-off-by: remzi <13716567376yh@gmail.com> * update test_min_max_for_single_level_struct Signed-off-by: remzi <13716567376yh@gmail.com> * update test_distinct_count_reductions and test_collect_list_reductions Signed-off-by: remzi <13716567376yh@gmail.com> Signed-off-by: remzi <13716567376yh@gmail.com> --- docs/supported_ops.md | 12 +- .../src/main/python/hash_aggregate_test.py | 22 ++- .../src/main/python/window_function_test.py | 20 +-- .../nvidia/spark/rapids/GpuOverrides.scala | 11 +- .../nvidia/spark/rapids/nullExpressions.scala | 22 ++- .../spark/sql/rapids/AggregateFunctions.scala | 145 +++++++++++++++++- .../spark/rapids/HashAggregatesSuite.scala | 15 -- tools/src/main/resources/supportedExprs.csv | 6 +- 8 files changed, 189 insertions(+), 64 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 45a9f48a196..8d9dc3151cb 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -16554,8 +16554,8 @@ are limited. S S S -PS
Input must not contain NaNs and spark.rapids.sql.hasNans must be false.
-PS
Input must not contain NaNs and spark.rapids.sql.hasNans must be false.
+S +S S PS
UTC is only supported TZ for TIMESTAMP
S @@ -16597,8 +16597,8 @@ are limited. S S S -PS
Input must not contain NaNs and spark.rapids.sql.hasNans must be false.
-PS
Input must not contain NaNs and spark.rapids.sql.hasNans must be false.
+S +S S PS
UTC is only supported TZ for TIMESTAMP
S @@ -16640,8 +16640,8 @@ are limited. S S S -PS
Input must not contain NaNs and spark.rapids.sql.hasNans must be false.
-PS
Input must not contain NaNs and spark.rapids.sql.hasNans must be false.
+S +S S PS
UTC is only supported TZ for TIMESTAMP
S diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index 7144ff7822a..ce2ffcf3094 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -609,6 +609,8 @@ def test_hash_pivot_groupby_duplicates_fallback(data_gen): ('a', RepeatSeqGen(LongGen(), length=20)), ('b', value_gen)] for value_gen in _repeat_agg_column_for_collect_set_op_nested + _array_of_array_gen] +_all_basic_gens_with_all_nans_cases = all_basic_gens + [SetValuesGen(t, [math.nan, None]) for t in [FloatType(), DoubleType()]] + # very simple test for just a count on decimals 128 values until we can support more with them @ignore_order(local=True) @pytest.mark.parametrize('data_gen', [decimal_gen_128bit], ids=idfn) @@ -642,18 +644,12 @@ def test_decimal128_min_max_group_by(data_gen): .agg(f.min('b'), f.max('b'))) @ignore_order(local=True) -@pytest.mark.parametrize('data_gen', [float_gen, double_gen], ids=idfn) -def test_float_max_reduction_with_nan(data_gen): - assert_gpu_and_cpu_are_equal_collect( - lambda spark: unary_op_df(spark, data_gen).selectExpr('max(a)')) - -@ignore_order(local=True) -@pytest.mark.parametrize('data_gen', [float_gen, double_gen], ids=idfn) -def test_float_max_group_by_with_nan(data_gen): +@pytest.mark.parametrize('data_gen', _all_basic_gens_with_all_nans_cases, ids=idfn) +def test_min_max_group_by(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: two_col_df(spark, byte_gen, data_gen) .groupby('a') - .agg(f.max('b'))) + .agg(f.min('b'), f.max('b'))) # to avoid ordering issues with collect_list we do it all in a single task @ignore_order(local=True) @@ -1110,7 +1106,7 @@ def test_first_last_reductions_nested_types(data_gen): lambda spark: unary_op_df(spark, data_gen).coalesce(1).selectExpr( 'first(a)', 'last(a)', 'first(a, true)', 'last(a, true)')) -@pytest.mark.parametrize('data_gen', non_nan_all_basic_gens, ids=idfn) +@pytest.mark.parametrize('data_gen', _all_basic_gens_with_all_nans_cases, ids=idfn) def test_generic_reductions(data_gen): local_conf = copy_and_update(_no_nans_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true'}) assert_gpu_and_cpu_are_equal_collect( @@ -1138,7 +1134,7 @@ def test_count(data_gen): 'count(1)'), conf = {'spark.sql.legacy.allowParameterlessCount': 'true'}) -@pytest.mark.parametrize('data_gen', non_nan_all_basic_gens, ids=idfn) +@pytest.mark.parametrize('data_gen', all_basic_gens, ids=idfn) def test_distinct_count_reductions(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : binary_op_df(spark, data_gen).selectExpr( @@ -1160,7 +1156,7 @@ def test_arithmetic_reductions(data_gen): conf = _no_nans_float_conf) @pytest.mark.parametrize('data_gen', - non_nan_all_basic_gens + decimal_gens + _nested_gens, + all_basic_gens + decimal_gens + _nested_gens, ids=idfn) def test_collect_list_reductions(data_gen): assert_gpu_and_cpu_are_equal_collect( @@ -1822,7 +1818,7 @@ def test_groupby_std_variance_partial_replace_fallback(data_gen, # test min max on single level structure # gens_for_max_min = [byte_gen, short_gen, int_gen, long_gen, - FloatGen(no_nans = True), DoubleGen(no_nans = True), + float_gen, double_gen, string_gen, boolean_gen, date_gen, timestamp_gen, DecimalGen(precision=12, scale=2), diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py index a21feb810f4..fb7f9ce1258 100644 --- a/integration_tests/src/main/python/window_function_test.py +++ b/integration_tests/src/main/python/window_function_test.py @@ -135,22 +135,20 @@ ['child_string', StringGen()] ])] -all_basic_gens_no_nans = [byte_gen, short_gen, int_gen, long_gen, - FloatGen(no_nans=True, special_cases=[]), DoubleGen(no_nans=True, special_cases=[]), - string_gen, boolean_gen, date_gen, timestamp_gen, null_gen] - _no_nans_float_conf = {'spark.rapids.sql.variableFloatAgg.enabled': 'true', 'spark.rapids.sql.hasNans': 'false', 'spark.rapids.sql.castStringToFloat.enabled': 'true' } @ignore_order(local=True) -@pytest.mark.parametrize('data_gen', [float_gen, double_gen], ids=idfn) -def test_float_window_max_with_nan(data_gen): +@pytest.mark.parametrize('data_gen', [SetValuesGen(t, [math.nan, None]) for t in [FloatType(), DoubleType()]], ids=idfn) +def test_float_window_min_max_all_nans(data_gen): w = Window().partitionBy('a') assert_gpu_and_cpu_are_equal_collect( lambda spark: two_col_df(spark, byte_gen, data_gen) - .withColumn("max_b", f.max('a').over(w))) + .withColumn("min_b", f.min('a').over(w)) + .withColumn("max_b", f.max('a').over(w)) + ) @ignore_order @pytest.mark.parametrize('data_gen', [decimal_gen_128bit], ids=idfn) @@ -385,10 +383,9 @@ def test_window_aggs_for_rows(data_gen, batch_size): # specially, but it only works if all of the aggregations can support this. # the order returned should be consistent because the data ends up in a single task (no partitioning) @pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) # set the batch size so we can test multiple stream batches -@pytest.mark.parametrize('b_gen', all_basic_gens_no_nans + [decimal_gen_32bit, decimal_gen_128bit], ids=meta_idfn('data:')) +@pytest.mark.parametrize('b_gen', all_basic_gens + [decimal_gen_32bit, decimal_gen_128bit], ids=meta_idfn('data:')) def test_window_running_no_part(b_gen, batch_size): conf = {'spark.rapids.sql.batchSizeBytes': batch_size, - 'spark.rapids.sql.hasNans': False, 'spark.rapids.sql.castFloatToDecimal.enabled': True} query_parts = ['row_number() over (order by a rows between UNBOUNDED PRECEDING AND CURRENT ROW) as row_num', 'rank() over (order by a rows between UNBOUNDED PRECEDING AND CURRENT ROW) as rank_val', @@ -441,7 +438,7 @@ def test_running_float_sum_no_part(batch_size): # to allow for duplication in the ordering, because there will be no other columns. This means that if you swtich # rows it does not matter because the only time rows are switched is when the rows are exactly the same. @pytest.mark.parametrize('data_gen', - all_basic_gens_no_nans + [decimal_gen_32bit, orderable_decimal_gen_128bit], + all_basic_gens + [decimal_gen_32bit, orderable_decimal_gen_128bit], ids=meta_idfn('data:')) def test_window_running_rank_no_part(data_gen): # Keep the batch size small. We have tested these with operators with exact inputs already, this is mostly @@ -495,10 +492,9 @@ def test_window_running_rank(data_gen): @ignore_order(local=True) @pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) # set the batch size so we can test multiple stream batches @pytest.mark.parametrize('b_gen, c_gen', [(long_gen, x) for x in running_part_and_order_gens] + - [(x, long_gen) for x in all_basic_gens_no_nans + [decimal_gen_32bit]], ids=idfn) + [(x, long_gen) for x in all_basic_gens + [decimal_gen_32bit]], ids=idfn) def test_window_running(b_gen, c_gen, batch_size): conf = {'spark.rapids.sql.batchSizeBytes': batch_size, - 'spark.rapids.sql.hasNans': False, 'spark.rapids.sql.variableFloatAgg.enabled': True, 'spark.rapids.sql.castFloatToDecimal.enabled': True} query_parts = ['b', 'a', 'row_number() over (partition by b order by a rows between UNBOUNDED PRECEDING AND CURRENT ROW) as row_num', diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 8ce49f4e4d1..c7fe112032f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -2293,23 +2293,16 @@ object GpuOverrides extends Logging { TypeSig.orderable, Seq(ParamCheck("input", (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + TypeSig.STRUCT) - .nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL) - .withPsNote(Seq(TypeEnum.DOUBLE, TypeEnum.FLOAT), nanAggPsNote), + .nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL), TypeSig.orderable))).asInstanceOf[ExprChecksImpl].contexts ++ ExprChecks.windowOnly( (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL), TypeSig.orderable, Seq(ParamCheck("input", - (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL) - .withPsNote(Seq(TypeEnum.DOUBLE, TypeEnum.FLOAT), nanAggPsNote), + (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL), TypeSig.orderable))).asInstanceOf[ExprChecksImpl].contexts), (a, conf, p, r) => new AggExprMeta[Min](a, conf, p, r) { - override def tagAggForGpu(): Unit = { - val dataType = a.child.dataType - checkAndTagFloatNanAgg("Min", dataType, conf, this) - } - override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = GpuMin(childExprs.head) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala index 5827c2dfa4b..8d60e36dad8 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala @@ -23,7 +23,7 @@ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.shims.ShimExpression import org.apache.spark.sql.catalyst.expressions.{ComplexTypeMergingExpression, Expression, Predicate} -import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types.{DataType, DoubleType, FloatType} import org.apache.spark.sql.vectorized.ColumnarBatch object GpuNvl extends Arm { @@ -140,6 +140,26 @@ case class GpuIsNan(child: Expression) extends GpuUnaryExpression with Predicate input.getBase.isNan } +/* + * Replace all `Nan`s in child to `null`s. + * The data type of child can only be FloatType or DoubleType. + * + * This class is used in `GpuFloatMin`. + */ +case class GpuNansToNulls(child: Expression) extends GpuUnaryExpression{ + + override def dataType: DataType = child.dataType match { + case FloatType => FloatType + case DoubleType => DoubleType + case t => throw new IllegalStateException(s"child type $t is not FloatType or DoubleType") + } + + override protected def doColumnar(input: GpuColumnVector): ColumnVector = + input.getBase.nansToNulls + + override def nullable = true +} + /** * A GPU accelerated predicate that is evaluated to be true if there are at least `n` non-null * and non-NaN values. diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala index 6243f85c59b..f54e8ea3050 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala @@ -360,6 +360,15 @@ class CudfMin(override val dataType: DataType) extends CudfAggregate { override val name: String = "CudfMin" } +/** + * Check if all values in a boolean column are trues. + * The CUDF all aggregation does not work for reductions or group by aggregations + * so we use Min as a workaround for this. + */ +object CudfAll { + def apply(): CudfAggregate = new CudfMin(BooleanType) +} + class CudfCollectList(override val dataType: DataType) extends CudfAggregate { override lazy val reductionAggregate: cudf.ColumnVector => cudf.Scalar = (col: cudf.ColumnVector) => col.reduce(ReductionAggregation.collectList(), DType.LIST) @@ -470,10 +479,18 @@ class CudfMergeM2 extends CudfAggregate { StructField("m2", DoubleType, nullable = true) :: Nil) } -case class GpuMin(child: Expression) extends GpuAggregateFunction +object GpuMin{ + def apply(child: Expression): GpuMin = child.dataType match { + case FloatType | DoubleType => GpuFloatMin(child) + case _ => GpuBasicMin(child) + } +} + +abstract class GpuMin(child: Expression) extends GpuAggregateFunction with GpuBatchedRunningWindowWithFixer with GpuAggregateWindowFunction - with GpuRunningWindowFunction { + with GpuRunningWindowFunction + with Serializable { override lazy val initialValues: Seq[GpuLiteral] = Seq(GpuLiteral(null, child.dataType)) override lazy val inputProjection: Seq[Expression] = Seq(child) override lazy val updateAggregates: Seq[CudfAggregate] = Seq(new CudfMin(child.dataType)) @@ -522,6 +539,123 @@ case class GpuMin(child: Expression) extends GpuAggregateFunction } } +/** Min aggregation without `Nan` handling */ +case class GpuBasicMin(child: Expression) extends GpuMin(child) + +/** GpuMin for FloatType and DoubleType to handle `Nan`s. + * + * In Spark, `Nan` is the max float value, however in cuDF, the calculation + * involving `Nan` is undefined. + * We design a workaround method here to match the Spark's behaviour. + * The high level idea is: + * if the column contains only `Nan`s or `null`s + * then + if the column contains `Nan` + * then return `Nan` + * else return null + * else + * replace all `Nan`s with nulls; + * use cuDF kernel to find the min value + */ +case class GpuFloatMin(child: Expression) extends GpuMin(child) + with GpuReplaceWindowFunction { + + override val dataType: DataType = child.dataType match { + case FloatType | DoubleType => child.dataType + case t => throw new IllegalStateException(s"child type $t is not FloatType or DoubleType") + } + + protected val nan: Any = child.dataType match { + case FloatType => Float.NaN + case DoubleType => Double.NaN + case t => throw new IllegalStateException(s"child type $t is not FloatType or DoubleType") + } + + protected lazy val updateAllNansOrNulls = CudfAll() + protected lazy val updateHasNan = CudfAny() + protected lazy val updateMinVal = new CudfMin(dataType) + + protected lazy val mergeAllNansOrNulls = CudfAll() + protected lazy val mergeHasNan = CudfAny() + protected lazy val mergeMinVal = new CudfMin(dataType) + + // Project 3 columns: + // 1. A boolean column indicating whether the values in `child` are `Nan`s or `null`s + // 2. A boolean column indicating whether the values in `child` are `Nan`s + // 3. Replace all `Nan`s in the `child` with `null`s + override lazy val inputProjection: Seq[Expression] = Seq( + GpuOr(GpuIsNan(child), GpuIsNull(child)), + GpuIsNan(child), + // We must eliminate all Nans before calling the cuDF min kernel. + // As this expression is only used when `allNansOrNulls` = false, + // and `Nan` is the max value in Spark, the elimination will + // not affect the final result. + GpuNansToNulls(child) + ) + // 1. Check if all values in the `child` are `Nan`s or `null`s + // 2. Check if `child` contains `Nan` + // 3. Calculate the min value on `child` with all `Nan`s has been replaced. + override lazy val updateAggregates: Seq[CudfAggregate] = + Seq(updateAllNansOrNulls, updateHasNan, updateMinVal) + + // If the column only contains `Nan`s or `null`s + // Then + // if the column contains `Nan` + // then return `Nan` + // else return `null` + // Else return the min value + override lazy val postUpdate: Seq[Expression] = Seq( + GpuIf( + updateAllNansOrNulls.attr, + GpuIf( + updateHasNan.attr, GpuLiteral(nan, dataType), GpuLiteral(null, dataType) + ), + updateMinVal.attr + ) + ) + + // Same logic as the `inputProjection` stage. + override lazy val preMerge: Seq[Expression] = Seq ( + GpuOr(GpuIsNan(evaluateExpression), GpuIsNull(evaluateExpression)), + GpuIsNan(evaluateExpression), + GpuNansToNulls(evaluateExpression) + ) + + // Same logic as the `updateAggregates` stage. + override lazy val mergeAggregates: Seq[CudfAggregate] = + Seq(mergeAllNansOrNulls, mergeHasNan, mergeMinVal) + + // Same logic as the `postUpdate` stage. + override lazy val postMerge: Seq[Expression] = Seq( + GpuIf( + mergeAllNansOrNulls.attr, + GpuIf( + mergeHasNan.attr, GpuLiteral(nan, dataType), GpuLiteral(null, dataType) + ), + mergeMinVal.attr + ) + ) + + // We should always override the windowing expression to handle `Nan`. + override def shouldReplaceWindow(spec: GpuWindowSpecDefinition): Boolean = true + + override def windowReplacement(spec: GpuWindowSpecDefinition): Expression = { + // The `GpuBasicMin` here has the same functionality as `CudfAll`, + // as `true > false` in cuDF. + val allNansOrNull = GpuWindowExpression( + GpuBasicMin(GpuOr(GpuIsNan(child), GpuIsNull(child))), spec + ) + val hasNan = GpuWindowExpression(GpuBasicMax(GpuIsNan(child)), spec) + // We use `GpuBasicMin` but not `GpuMin` to avoid self recursion. + val min = GpuWindowExpression(GpuBasicMin(GpuNansToNulls(child)), spec) + GpuIf( + allNansOrNull, + GpuIf(hasNan, GpuLiteral(nan, dataType), GpuLiteral(null, dataType)), + min + ) + } +} + object GpuMax { def apply(child: Expression): GpuMax = { child.dataType match { @@ -584,12 +718,13 @@ abstract class GpuMax(child: Expression) extends GpuAggregateFunction } } -/** Max aggregation without `NaN` handling */ +/** Max aggregation without `Nan` handling */ case class GpuBasicMax(child: Expression) extends GpuMax(child) -/** Max aggregation for FloatType and DoubleType to handle `NaN`s. +/** Max aggregation for FloatType and DoubleType to handle `Nan`s. * - * In Spark, `Nan` is the max float value, however in cuDF, `Infinity` is. + * In Spark, `Nan` is the max float value, however in cuDF, the calculation + * involving `Nan` is undefined. * We design a workaround method here to match the Spark's behaviour. * The high level idea is that, in the projection stage, we create another * column `isNan`. If any value in this column is true, return `Nan`, diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala index ede7fba002a..9e679e61543 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala @@ -1618,21 +1618,6 @@ class HashAggregatesSuite extends SparkQueryCompareTestSuite { frame => frame.groupBy("large_longs").agg(avg("large_longs")) } { (_, gpuPlan) => checkExecPlan(gpuPlan) } - ALLOW_NON_GPU_testSparkResultsAreEqualWithCapture( - "min_with_nans_fall_back", - nanDf, - Seq("HashAggregateExec", "AggregateExpression", - "AttributeReference", "Alias", "Min", "ShuffleExchangeExec"), - conf = enableCsvConf()) { - frame => frame.agg(min("doubles")) - } { (_, gpuPlan) => { - // verify nothing ran on the gpu - if (gpuPlan.conf.getAllConfs(RapidsConf.SQL_ENABLED.key).toBoolean) { - val execNode = gpuPlan.find(_.isInstanceOf[GpuHashAggregateExec]) - assert(execNode.isEmpty) - } - }} - IGNORE_ORDER_testSparkResultsAreEqual( testName = "Test NormalizeNansAndZeros(Float)", floatWithDifferentKindsOfNansAndZeros, diff --git a/tools/src/main/resources/supportedExprs.csv b/tools/src/main/resources/supportedExprs.csv index f31dfa1e760..cd37c792ccc 100644 --- a/tools/src/main/resources/supportedExprs.csv +++ b/tools/src/main/resources/supportedExprs.csv @@ -628,11 +628,11 @@ Max,S,`max`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS Max,S,`max`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS Max,S,`max`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS Max,S,`max`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS -Min,S,`min`,None,aggregation,input,S,S,S,S,S,PS,PS,S,PS,S,S,S,NS,NS,NS,NA,PS,NS +Min,S,`min`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS Min,S,`min`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS -Min,S,`min`,None,reduction,input,S,S,S,S,S,PS,PS,S,PS,S,S,S,NS,NS,NS,NA,PS,NS +Min,S,`min`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS Min,S,`min`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS -Min,S,`min`,None,window,input,S,S,S,S,S,PS,PS,S,PS,S,S,S,NS,NS,NS,NA,NS,NS +Min,S,`min`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS Min,S,`min`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS PivotFirst,S, ,None,aggregation,pivotColumn,S,S,S,S,S,PS,PS,S,PS,S,S,S,NS,NS,NS,NS,NS,NS PivotFirst,S, ,None,aggregation,valueColumn,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS From 7001f30dcea5e37caf2ba97acc03bf86f2b0e357 Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Tue, 30 Aug 2022 10:01:09 -0500 Subject: [PATCH 065/190] Fix leaks in GpuShuffledHashJoinExecSuite (#6439) Signed-off-by: Alessandro Bellina Signed-off-by: Alessandro Bellina --- .../nvidia/spark/rapids/GpuShuffledHashJoinExecSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecSuite.scala index f277921c7d3..2b29555852e 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecSuite.scala @@ -206,7 +206,7 @@ class GpuShuffledHashJoinExecSuite extends FunSuite with Arm with MockitoSugar { test("test a SerializedTableColumn") { TestUtils.withGpuSparkSession(new SparkConf()) { _ => - closeOnExcept(ColumnVector.fromInts(1, 2, 3, 4, 5)) { cudfCol => + withResource(ColumnVector.fromInts(1, 2, 3, 4, 5)) { cudfCol => val cv = GpuColumnVector.from(cudfCol, IntegerType) val batch = new ColumnarBatch(Seq(cv).toArray, 5) withResource(GpuColumnVector.from(batch)) { tbl => @@ -242,7 +242,7 @@ class GpuShuffledHashJoinExecSuite extends FunSuite with Arm with MockitoSugar { test("test two batches, going over the limit") { TestUtils.withGpuSparkSession(new SparkConf()) { _ => - closeOnExcept(ColumnVector.fromInts(1, 2, 3, 4, 5)) { cudfCol => + withResource(ColumnVector.fromInts(1, 2, 3, 4, 5)) { cudfCol => val cv = GpuColumnVector.from(cudfCol, IntegerType) val batch = new ColumnarBatch(Seq(cv).toArray, 5) withResource(GpuColumnVector.from(batch)) { tbl => @@ -281,7 +281,7 @@ class GpuShuffledHashJoinExecSuite extends FunSuite with Arm with MockitoSugar { test("test two batches, stating within the limit") { TestUtils.withGpuSparkSession(new SparkConf()) { _ => - closeOnExcept(ColumnVector.fromInts(1, 2, 3, 4, 5)) { cudfCol => + withResource(ColumnVector.fromInts(1, 2, 3, 4, 5)) { cudfCol => val cv = GpuColumnVector.from(cudfCol, IntegerType) val batch = new ColumnarBatch(Seq(cv).toArray, 5) withResource(GpuColumnVector.from(batch)) { tbl => From 4e20c8f0ef9a86ee9077cd2cc27afb618907789a Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 30 Aug 2022 10:49:14 -0500 Subject: [PATCH 066/190] Reading delta log Table Checkpoint files should fallback the entire plan (#6457) * Fallback to CPU for delta log checkpoint file reads Signed-off-by: Thomas Graves * fix stylie * Handle xxx.checkpoint.yyy.parquet file names * Fix to use or instead of and * cleanup Signed-off-by: Thomas Graves * wrapping Signed-off-by: Thomas Graves --- .../com/nvidia/spark/rapids/GpuOverrides.scala | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index c7fe112032f..30a18b6d77f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -4407,6 +4407,9 @@ case class GpuOverrides() extends Rule[SparkPlan] with Logging { * check for a ScalaUDF using a tahoe.Snapshot function and if we ever see * an AdaptiveSparkPlan on a Spark version we don't expect, fallback to the * CPU for those plans. + * Note that the Delta Lake delta log checkpoint parquet files are just inefficient + * to have to copy the data to GPU and then back off after it does the scan on + * Delta Table Checkpoint, so have the entire plan fallback to CPU at that point. */ def isDeltaLakeMetadataQuery(plan: SparkPlan): Boolean = { val deltaLogScans = PlanUtils.findOperators(plan, { @@ -4416,17 +4419,21 @@ case class GpuOverrides() extends Rule[SparkPlan] with Logging { true case f: FileSourceScanExec => // example filename: "file:/tmp/delta-table/_delta_log/00000000000000000000.json" - val found = f.relation.inputFiles.exists(name => - name.contains("/_delta_log/") && name.endsWith(".json")) + val found = f.relation.inputFiles.exists { name => + name.contains("/_delta_log/") && name.endsWith(".json") + } if (found) { logDebug(s"Fallback for FileSourceScanExec delta log: $f") } found case rdd: RDDScanExec => - // example rdd name: "Delta Table State #1 - file:///tmp/delta-table/_delta_log" + // example rdd name: "Delta Table State #1 - file:///tmp/delta-table/_delta_log" or + // "Scan ExistingRDD Delta Table Checkpoint with Stats #1 - + // file:///tmp/delta-table/_delta_log" val found = rdd.inputRDD != null && rdd.inputRDD.name != null && - rdd.inputRDD.name.startsWith("Delta Table State") && + (rdd.inputRDD.name.startsWith("Delta Table State") + || rdd.inputRDD.name.startsWith("Delta Table Checkpoint")) && rdd.inputRDD.name.endsWith("/_delta_log") if (found) { logDebug(s"Fallback for RDDScanExec delta log: $rdd") From 04ff6b99da3ccce244ae5b6de914d6a776d63f66 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 30 Aug 2022 14:46:17 -0500 Subject: [PATCH 067/190] Change alluxio logs to be debug (#6458) Signed-off-by: Thomas Graves Signed-off-by: Thomas Graves --- .../main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala index ceef7b400dc..33851dffae8 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala @@ -42,7 +42,7 @@ object AlluxioUtils extends Logging { throw new FileNotFoundException( s"Alluxio path $alluxio_path does not exist, maybe forgot to mount it") } - logInfo(s"Alluxio path $alluxio_path is mounted") + logDebug(s"Alluxio path $alluxio_path is mounted") checkedAlluxioPath.add(alluxio_path) } else { logDebug(s"Alluxio path $alluxio_path already mounted") @@ -106,7 +106,7 @@ object AlluxioUtils extends Logging { // record it as a mounted point if (items(0).contains("://")) { mountedBuckets(items(2)) = items(0) - logInfo(s"Found mounted bucket ${items(0)} to ${items(2)}") + logDebug(s"Found mounted bucket ${items(0)} to ${items(2)}") } } } @@ -176,7 +176,7 @@ object AlluxioUtils extends Logging { logInfo(s"Mounted bucket $remote_path to $local_bucket in Alluxio $output") mountedBuckets(local_bucket) = remote_path } else if (mountedBuckets(local_bucket).equals(remote_path)) { - logInfo(s"Already mounted bucket $remote_path to $local_bucket in Alluxio") + logDebug(s"Already mounted bucket $remote_path to $local_bucket in Alluxio") } else { throw new RuntimeException(s"Found a same bucket name in $remote_path " + s"and ${mountedBuckets(local_bucket)}") @@ -242,7 +242,7 @@ object AlluxioUtils extends Logging { // replace s3://foo/.. to alluxio://alluxioMasterHost/foo/... val newPath = new Path(pathStr.replaceFirst( scheme + ":/", "alluxio://" + alluxioMasterHost.get)) - logInfo(s"Replace $pathStr to ${newPath.toString}") + logDebug(s"Replace $pathStr to ${newPath.toString}") newPath } else { f From 99b861349f93ff49b44e84dbaf2073ccb9a31de6 Mon Sep 17 00:00:00 2001 From: YanxuanLiu <104543031+YanxuanLiu@users.noreply.github.com> Date: Wed, 31 Aug 2022 09:03:20 +0800 Subject: [PATCH 068/190] added YanxuanLiu to trigger list (#6450) Signed-off-by: YanxuanLiu Signed-off-by: YanxuanLiu Co-authored-by: YanxuanLiu --- .github/workflows/blossom-ci.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/blossom-ci.yml b/.github/workflows/blossom-ci.yml index e115736460b..43d1b6160c7 100644 --- a/.github/workflows/blossom-ci.yml +++ b/.github/workflows/blossom-ci.yml @@ -67,6 +67,7 @@ jobs: mattahrens,\ sinkinben,\ thirtiseven,\ + YanxuanLiu,\ ', format('{0},', github.actor)) && github.event.comment.body == 'build' steps: - name: Check if comment is issued by authorized person From e2aead3c8b11e96ce232b244b64fe97bfcc520d9 Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Wed, 31 Aug 2022 09:49:48 +0800 Subject: [PATCH 069/190] Iceberg supports coalescing reading for Parquet (#6422) * Iceberg Parquet supports coalescing reading Signed-off-by: Firestarman --- .../src/main/python/iceberg_test.py | 57 ++- .../rapids/iceberg/IcebergProvider.scala | 2 + .../rapids/iceberg/IcebergProviderImpl.scala | 7 + .../spark/source/GpuMultiFileBatchReader.java | 401 +++++++++++++----- .../spark/source/GpuSparkBatchQueryScan.java | 22 +- .../iceberg/spark/source/GpuSparkScan.java | 33 +- .../iceberg/spark/source/SparkBatch.java | 3 +- .../spark/rapids/GpuMultiFileReader.scala | 56 ++- .../com/nvidia/spark/rapids/GpuOrcScan.scala | 5 +- .../nvidia/spark/rapids/GpuParquetScan.scala | 38 +- .../spark/sql/rapids/ExternalSource.scala | 3 +- .../apache/spark/sql/rapids/GpuAvroScan.scala | 6 +- 12 files changed, 457 insertions(+), 176 deletions(-) diff --git a/integration_tests/src/main/python/iceberg_test.py b/integration_tests/src/main/python/iceberg_test.py index 76ee30fcbe8..4b4ccfacc10 100644 --- a/integration_tests/src/main/python/iceberg_test.py +++ b/integration_tests/src/main/python/iceberg_test.py @@ -35,6 +35,9 @@ ArrayGen(StructGen([['child0', string_gen], ['child1', double_gen], ['child2', int_gen]])) ] + iceberg_map_gens + decimal_gens ] +rapids_reader_types = ['PERFILE', 'MULTITHREADED', 'COALESCING'] + + @allow_non_gpu("BatchScanExec") @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering @@ -53,7 +56,7 @@ def setup_iceberg_table(spark): @ignore_order(local=True) @pytest.mark.skipif(is_before_spark_320() or is_databricks_runtime(), reason="AQE+DPP not supported until Spark 3.2.0+ and AQE+DPP not supported on Databricks") -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_aqe_dpp(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -72,7 +75,7 @@ def setup_iceberg_table(spark): @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering @pytest.mark.parametrize("data_gens", iceberg_gens_list, ids=idfn) -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_parquet_read_round_trip(spark_tmp_table_factory, data_gens, reader_type): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(data_gens)] table = spark_tmp_table_factory.get() @@ -89,7 +92,7 @@ def setup_iceberg_table(spark): @iceberg @pytest.mark.parametrize("data_gens", [[long_gen]], ids=idfn) @pytest.mark.parametrize("iceberg_format", ["orc", "avro"], ids=idfn) -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_unsupported_formats(spark_tmp_table_factory, data_gens, iceberg_format, reader_type): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(data_gens)] table = spark_tmp_table_factory.get() @@ -136,7 +139,7 @@ def setup_iceberg_table(spark): marks=pytest.mark.skipif(is_before_spark_320(), reason="Hadoop with Spark 3.1.x does not support lz4 by default")), ("zstd", None)], ids=idfn) -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_read_parquet_compression_codec(spark_tmp_table_factory, codec_info, reader_type): codec, error_msg = codec_info table = spark_tmp_table_factory.get() @@ -160,7 +163,7 @@ def setup_iceberg_table(spark): @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering @pytest.mark.parametrize("key_gen", [int_gen, long_gen, string_gen, boolean_gen, date_gen, timestamp_gen, decimal_gen_64bit], ids=idfn) -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_read_partition_key(spark_tmp_table_factory, key_gen, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -176,7 +179,7 @@ def setup_iceberg_table(spark): @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_input_meta(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -194,7 +197,7 @@ def setup_iceberg_table(spark): @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_disorder_read_schema(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -274,7 +277,7 @@ def setup_iceberg_table(spark): @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering @pytest.mark.skipif(is_before_spark_320(), reason="Spark 3.1.x has a catalog bug precluding scope prefix in table names") -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_read_timetravel(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -298,7 +301,7 @@ def setup_snapshots(spark): @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering @pytest.mark.skipif(is_before_spark_320(), reason="Spark 3.1.x has a catalog bug precluding scope prefix in table names") -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_incremental_read(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -328,7 +331,7 @@ def setup_snapshots(spark): @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_reorder_columns(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -349,7 +352,7 @@ def setup_iceberg_table(spark): @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_rename_column(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -370,7 +373,7 @@ def setup_iceberg_table(spark): @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_column_names_swapped(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -393,7 +396,7 @@ def setup_iceberg_table(spark): @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_alter_column_type(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -416,7 +419,7 @@ def setup_iceberg_table(spark): @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_add_column(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -437,7 +440,7 @@ def setup_iceberg_table(spark): @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_remove_column(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -458,7 +461,7 @@ def setup_iceberg_table(spark): @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_add_partition_field(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -479,7 +482,7 @@ def setup_iceberg_table(spark): @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_drop_partition_field(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -500,7 +503,7 @@ def setup_iceberg_table(spark): @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_v1_delete(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -517,7 +520,7 @@ def setup_iceberg_table(spark): @iceberg @pytest.mark.skipif(is_before_spark_320(), reason="merge-on-read not supported on Spark 3.1.x") -@pytest.mark.parametrize('reader_type', ['PERFILE', 'MULTITHREADED']) +@pytest.mark.parametrize('reader_type', rapids_reader_types) def test_iceberg_v2_delete_unsupported(spark_tmp_table_factory, reader_type): table = spark_tmp_table_factory.get() tmpview = spark_tmp_table_factory.get() @@ -534,3 +537,19 @@ def setup_iceberg_table(spark): lambda spark : spark.sql("SELECT * FROM {}".format(table)).collect(), conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}), "UnsupportedOperationException: Delete filter is not supported") + + +@iceberg +@ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering +@pytest.mark.parametrize('reader_type', rapids_reader_types) +def test_iceberg_parquet_read_with_input_file(spark_tmp_table_factory, reader_type): + table = spark_tmp_table_factory.get() + tmpview = spark_tmp_table_factory.get() + def setup_iceberg_table(spark): + df = binary_op_df(spark, long_gen) + df.createOrReplaceTempView(tmpview) + spark.sql("CREATE TABLE {} USING ICEBERG AS SELECT * FROM {}".format(table, tmpview)) + with_cpu_session(setup_iceberg_table) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT *, input_file_name() FROM {}".format(table)), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}) diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/IcebergProvider.scala b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/IcebergProvider.scala index cd9cc9666c0..b17a70d303f 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/IcebergProvider.scala +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/IcebergProvider.scala @@ -25,6 +25,8 @@ trait IcebergProvider { def isSupportedScan(scan: Scan): Boolean def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] + + def copyScanWithInputFileTrue(scan: Scan): Scan } object IcebergProvider { diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/IcebergProviderImpl.scala b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/IcebergProviderImpl.scala index 9d440885e91..ae855214f2e 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/IcebergProviderImpl.scala +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/IcebergProviderImpl.scala @@ -66,4 +66,11 @@ class IcebergProviderImpl extends IcebergProvider { ClassTag(cpuIcebergScanClass)) ).map(r => (r.getClassFor.asSubclass(classOf[Scan]), r)).toMap } + + override def copyScanWithInputFileTrue(scan: Scan): Scan = scan match { + case icebergBatchScan: GpuSparkBatchQueryScan => + icebergBatchScan.copyWithInputFileTrue(); + case _ => + throw new RuntimeException(s"Unsupported scan type: ${scan.getClass.getSimpleName}") + } } diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java index 9b958cbaa8b..8755fa27289 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java @@ -38,10 +38,12 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.execution.datasources.PartitionedFile; import org.apache.spark.sql.rapids.InputFileUtils; +import org.apache.spark.sql.rapids.execution.TrampolineUtil; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.vectorized.ColumnarBatch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import scala.collection.JavaConverters; import scala.collection.Seq; import scala.Tuple2; @@ -49,15 +51,12 @@ import java.io.UncheckedIOException; import java.net.URI; import java.net.URISyntaxException; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; +import java.util.*; +import java.util.stream.Collectors; -/** The wrapper of the GPU multi-threaded and coalescing(TBD) reader for Iceberg */ +/** The wrapper of the GPU multi-threaded and coalescing reader for Iceberg */ class GpuMultiFileBatchReader extends BaseDataReader { private static final Logger LOG = LoggerFactory.getLogger(GpuMultiFileBatchReader.class); - private final Map, Schema>> constsSchemaMap = - Maps.newConcurrentMap(); private final LinkedHashMap files; private final Schema expectedSchema; private final boolean caseSensitive; @@ -75,7 +74,7 @@ class GpuMultiFileBatchReader extends BaseDataReader { private boolean needNext = true; private boolean isBatchPending; // lazy variables - private FilePartitionReaderBase rapidsReader = null; + private CloseableIterator batchReader = null; GpuMultiFileBatchReader(CombinedScanTask task, Table table, Schema expectedSchema, boolean caseSensitive, Configuration conf, int maxBatchSizeRows, long maxBatchSizeBytes, @@ -104,38 +103,28 @@ class GpuMultiFileBatchReader extends BaseDataReader { @Override public ColumnarBatch get() { - if (rapidsReader == null) { + if (batchReader == null) { // Not initialized, return null to align with PerFile reader. return null; } needNext = true; isBatchPending = false; - // The same post-process with PerFile reader. - try (ColumnarBatch batch = rapidsReader.get()) { - // The Rapids reader should already set the current file. - String curFile = InputFileUtils.getCurInputFilePath(); - Tuple2, Schema> constsSchema = constsSchemaMap.get(curFile); - Map idToConsts = constsSchema._1(); - Schema updatedReadSchema = constsSchema._2(); - return GpuIcebergReader.addUpcastsIfNeeded( - GpuIcebergReader.addConstantColumns(batch, updatedReadSchema, idToConsts), - updatedReadSchema); - } + return batchReader.next(); } @Override public boolean next() throws IOException { - ensureRapidsReader(); + ensureBatchReader(); if (needNext) { needNext = false; - isBatchPending = rapidsReader.next(); + isBatchPending = batchReader.hasNext(); } return isBatchPending; } @Override public void close() throws IOException { - if (rapidsReader != null) rapidsReader.close(); + if (batchReader != null) batchReader.close(); super.close(); } @@ -146,89 +135,23 @@ CloseableIterator open(FileScanTask task) { throw new IllegalStateException(); } - private void ensureRapidsReader() { - if (rapidsReader == null) { - if (FileFormat.PARQUET.equals(fileFormat)) { - if (useMultiThread) { - rapidsReader = createParquetMultiThreadReader(); - } else { - // TODO Support coalescing reading, tracked by - // https://github.com/NVIDIA/spark-rapids/issues/5942 - throw new UnsupportedOperationException( - "Coalescing reading is not supported for Parquet reads yet"); - } + private void ensureBatchReader() { + if (batchReader != null) { + return; + } + if (FileFormat.PARQUET.equals(fileFormat)) { + if (useMultiThread) { + LOG.debug("Using Iceberg Parquet multi-threaded reader, task attempt ID: " + + TaskContext.get().taskAttemptId()); + batchReader = new ParquetMultiThreadBatchReader(); } else { - throw new UnsupportedOperationException( - "Format: " + fileFormat + " is not supported for batched reads"); + LOG.debug("Using Iceberg Parquet coalescing reader, task attempt ID: " + + TaskContext.get().taskAttemptId()); + batchReader = new ParquetCoalescingBatchReader(); } - } - } - - private FilePartitionReaderBase createParquetMultiThreadReader() { - LOG.debug("Using multi-threaded Iceberg Parquet reader, task attempt ID: " + - TaskContext.get().taskAttemptId()); - // Iceberg will handle partition values itself. - StructType emptyPartSchema = new StructType(); - InternalRow emptyPartValue = InternalRow.empty(); - - PartitionedFile[] files = this.files.values().stream() - .map(fst -> PartitionedFileUtils.newPartitionedFile(emptyPartValue, - fst.file().path().toString(), fst.start(), fst.length())) - .toArray(PartitionedFile[]::new); - - return new MultiFileCloudParquetPartitionReader(conf, files, this::filterParquetBlocks, - caseSensitive, parquetDebugDumpPrefix, maxBatchSizeRows, maxBatchSizeBytes, - metrics, emptyPartSchema, numThreads, maxNumFileProcessed, - false, // ignoreMissingFiles - false, // ignoreCorruptFiles - false // useFieldId - ); - } - - /** The filter function for the Parquet multi-file reader */ - private ParquetFileInfoWithBlockMeta filterParquetBlocks(PartitionedFile file) { - FileScanTask fst = this.files.get(file.filePath()); - GpuDeleteFilter deleteFilter = deleteFilter(fst); - if (deleteFilter != null) { - throw new UnsupportedOperationException("Delete filter is not supported"); - } - Schema updatedSchema = requiredSchema(deleteFilter); - Map idToConstant = constantsMap(fst, updatedSchema); - InputFile inFile = getInputFile(fst); - ParquetReadOptions readOptions = - GpuParquet.buildReaderOptions(inFile, fst.start(), fst.length()); - try (ParquetFileReader reader = GpuParquetReader.newReader(inFile, readOptions)) { - MessageType fileSchema = reader.getFileMetaData().getSchema(); - - List filteredRowGroups = GpuParquetReader.filterRowGroups(reader, - nameMapping, updatedSchema, fst.residual(), caseSensitive); - - GpuParquetReader.ReorderColumns reorder = ParquetSchemaUtil.hasIds(fileSchema) ? - new GpuParquetReader.ReorderColumns(idToConstant) : - new GpuParquetReader.ReorderColumnsFallback(idToConstant); - - MessageType fileReadSchema = (MessageType) TypeWithSchemaVisitor.visit( - updatedSchema.asStruct(), fileSchema, reorder); - Seq clippedBlocks = GpuParquetUtils.clipBlocksToSchema( - fileReadSchema, filteredRowGroups, caseSensitive); - StructType partReaderSparkSchema = (StructType) TypeWithSchemaVisitor.visit( - updatedSchema.asStruct(), fileReadSchema, new GpuParquetReader.SparkSchemaConverter()); - - // cache the updated constants - Map updatedConstants = - GpuParquetReader.addNullsForMissingFields(idToConstant, reorder.getMissingFields()); - constsSchemaMap.put(file.filePath(), Tuple2.apply(updatedConstants, updatedSchema)); - - return ParquetFileInfoWithBlockMeta.apply(new Path(new URI(file.filePath())), - clippedBlocks, InternalRow.empty(), fileReadSchema, partReaderSparkSchema, - true, // isCorrectedInt96RebaseMode - true, // isCorrectedRebaseMode - true // hasInt96Timestamps - ); - } catch (IOException e) { - throw new UncheckedIOException("Failed to open file: " + inFile, e); - } catch (URISyntaxException ue) { - throw new IllegalArgumentException("Invalid file path: " + inFile, ue); + } else { + throw new UnsupportedOperationException( + "Format: " + fileFormat + " is not supported for multi-file batched reads"); } } @@ -250,4 +173,276 @@ private Schema requiredSchema(GpuDeleteFilter deleteFilter) { return expectedSchema; } } + + static class FilteredParquetFileInfo { + private final ParquetFileInfoWithBlockMeta parquetBlockMeta; + private final Map idToConstant; + private final Schema expectedSchema; + + FilteredParquetFileInfo(ParquetFileInfoWithBlockMeta parquetBlockMeta, + Map idToConstant, Schema expectedSchema) { + this.parquetBlockMeta = parquetBlockMeta; + this.idToConstant = idToConstant; + this.expectedSchema = expectedSchema; + } + + ParquetFileInfoWithBlockMeta parquetBlockMeta() { + return parquetBlockMeta; + } + + Map idToConstant() { + return idToConstant; + } + + Schema expectedSchema() { + return expectedSchema; + } + } + + static class IcebergParquetExtraInfo extends ParquetExtraInfo { + private final Map idToConstant; + private final Schema expectedSchema; + private final PartitionSpec partitionSpec; + + IcebergParquetExtraInfo(boolean isCorrectedRebaseMode, + boolean isCorrectedInt96RebaseMode, boolean hasInt96Timestamps, + Map idToConstant, Schema expectedSchema, PartitionSpec partitionSpec) { + super(isCorrectedRebaseMode, isCorrectedInt96RebaseMode, hasInt96Timestamps); + this.idToConstant = idToConstant; + this.expectedSchema = expectedSchema; + this.partitionSpec = partitionSpec; + } + + Map idToConstant() { + return idToConstant; + } + + Schema expectedSchema() { + return expectedSchema; + } + + PartitionSpec partitionSpec() { + return partitionSpec; + } + } + + abstract class MultiFileBatchReaderBase implements CloseableIterator { + protected final FilePartitionReaderBase rapidsReader; + + protected MultiFileBatchReaderBase() { + // Iceberg will handle partition values itself. So both + // the partitioned schema and values are empty for the Rapids reader. + final StructType emptyPartSchema = new StructType(); + final InternalRow emptyPartValue = InternalRow.empty(); + PartitionedFile[] pFiles = files.values().stream() + .map(fst -> PartitionedFileUtils.newPartitionedFile(emptyPartValue, + fst.file().path().toString(), fst.start(), fst.length())) + .toArray(PartitionedFile[]::new); + rapidsReader = createRapidsReader(pFiles, emptyPartSchema); + } + + @Override + public void close() throws IOException { + rapidsReader.close(); + } + + @Override + public boolean hasNext() { + try { + return rapidsReader.next(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + protected abstract FilePartitionReaderBase createRapidsReader(PartitionedFile[] pFiles, + StructType partitionSchema); + + /** The filter function for the Parquet multi-file reader */ + protected FilteredParquetFileInfo filterParquetBlocks(FileScanTask fst) { + GpuDeleteFilter deleteFilter = deleteFilter(fst); + if (deleteFilter != null) { + throw new UnsupportedOperationException("Delete filter is not supported"); + } + Schema updatedSchema = requiredSchema(deleteFilter); + Map idToConstant = constantsMap(fst, updatedSchema); + InputFile inFile = getInputFile(fst); + ParquetReadOptions readOptions = + GpuParquet.buildReaderOptions(inFile, fst.start(), fst.length()); + try (ParquetFileReader reader = GpuParquetReader.newReader(inFile, readOptions)) { + MessageType fileSchema = reader.getFileMetaData().getSchema(); + + List filteredRowGroups = GpuParquetReader.filterRowGroups(reader, + nameMapping, updatedSchema, fst.residual(), caseSensitive); + + GpuParquetReader.ReorderColumns reorder = ParquetSchemaUtil.hasIds(fileSchema) ? + new GpuParquetReader.ReorderColumns(idToConstant) : + new GpuParquetReader.ReorderColumnsFallback(idToConstant); + + MessageType fileReadSchema = (MessageType) TypeWithSchemaVisitor.visit( + updatedSchema.asStruct(), fileSchema, reorder); + Seq clippedBlocks = GpuParquetUtils.clipBlocksToSchema( + fileReadSchema, filteredRowGroups, caseSensitive); + StructType partReaderSparkSchema = (StructType) TypeWithSchemaVisitor.visit( + updatedSchema.asStruct(), fileReadSchema, new GpuParquetReader.SparkSchemaConverter()); + + // cache the updated constants + Map updatedConstants = + GpuParquetReader.addNullsForMissingFields(idToConstant, reorder.getMissingFields()); + + ParquetFileInfoWithBlockMeta parquetBlockMeta = ParquetFileInfoWithBlockMeta.apply( + new Path(new URI(fst.file().path().toString())), clippedBlocks, + InternalRow.empty(), fileReadSchema, partReaderSparkSchema, + true, // isCorrectedInt96RebaseMode + true, // isCorrectedRebaseMode + true // hasInt96Timestamps + ); + return new FilteredParquetFileInfo(parquetBlockMeta, updatedConstants, updatedSchema); + } catch (IOException e) { + throw new UncheckedIOException("Failed to open file: " + inFile, e); + } catch (URISyntaxException ue) { + throw new IllegalArgumentException("Invalid file path: " + inFile, ue); + } + } // end of filterParquetBlocks + } + + class ParquetMultiThreadBatchReader extends MultiFileBatchReaderBase { + private final Map, Schema>> constsSchemaMap = + Maps.newConcurrentMap(); + + ParquetMultiThreadBatchReader() { + super(); + } + + @Override + protected FilePartitionReaderBase createRapidsReader(PartitionedFile[] pFiles, + StructType partitionSchema) { + return new MultiFileCloudParquetPartitionReader(conf, pFiles, + this::filterParquetBlocks, caseSensitive, parquetDebugDumpPrefix, + maxBatchSizeRows, maxBatchSizeBytes, metrics, partitionSchema, + numThreads, maxNumFileProcessed, + false, // ignoreMissingFiles + false, // ignoreCorruptFiles + false // useFieldId + ); + } + + private ParquetFileInfoWithBlockMeta filterParquetBlocks(PartitionedFile file) { + FileScanTask fst = files.get(file.filePath()); + FilteredParquetFileInfo filteredInfo = filterParquetBlocks(fst); + constsSchemaMap.put(file.filePath(), + Tuple2.apply(filteredInfo.idToConstant(), filteredInfo.expectedSchema())); + return filteredInfo.parquetBlockMeta(); + } + + @Override + public ColumnarBatch next() { + // The same post-process with PerFile reader. + try (ColumnarBatch batch = rapidsReader.get()) { + // The Rapids reader should already set the current file. + String curFile = InputFileUtils.getCurInputFilePath(); + Tuple2, Schema> constsSchema = constsSchemaMap.get(curFile); + Map idToConsts = constsSchema._1(); + Schema updatedReadSchema = constsSchema._2(); + return GpuIcebergReader.addUpcastsIfNeeded( + GpuIcebergReader.addConstantColumns(batch, updatedReadSchema, idToConsts), + updatedReadSchema); + } + } + } + + class ParquetCoalescingBatchReader extends MultiFileBatchReaderBase { + + ParquetCoalescingBatchReader() { + super(); + } + + @Override + protected FilePartitionReaderBase createRapidsReader(PartitionedFile[] pFiles, + StructType partitionSchema) { + ArrayList clippedBlocks = new ArrayList(); + files.values().forEach(fst -> { + FilteredParquetFileInfo filteredInfo = filterParquetBlocks(fst); + List fileSingleMetas = + JavaConverters.asJavaCollection(filteredInfo.parquetBlockMeta.blocks()).stream() + .map(b -> ParquetSingleDataBlockMeta.apply( + filteredInfo.parquetBlockMeta.filePath(), + ParquetDataBlock.apply(b), + InternalRow.empty(), + ParquetSchemaWrapper.apply(filteredInfo.parquetBlockMeta.schema()), + filteredInfo.parquetBlockMeta.readSchema(), + new IcebergParquetExtraInfo( + filteredInfo.parquetBlockMeta.isCorrectedRebaseMode(), + filteredInfo.parquetBlockMeta.isCorrectedInt96RebaseMode(), + filteredInfo.parquetBlockMeta.hasInt96Timestamps(), + filteredInfo.idToConstant(), + filteredInfo.expectedSchema(), + fst.spec()))) + .collect(Collectors.toList()); + clippedBlocks.addAll(fileSingleMetas); + }); + + return new MultiFileParquetPartitionReader(conf, pFiles, + JavaConverters.asScalaBuffer(clippedBlocks).toSeq(), + caseSensitive, parquetDebugDumpPrefix, maxBatchSizeRows, maxBatchSizeBytes, + metrics, partitionSchema, numThreads, + false, // ignoreMissingFiles + false, // ignoreCorruptFiles + false // useFieldId + ) { + @Override + public boolean checkIfNeedToSplitDataBlock(SingleDataBlockInfo currentBlockInfo, + SingleDataBlockInfo nextBlockInfo) { + // Check the read schema. Because it may differ among files in Iceberg. + if (!TrampolineUtil.sameType(currentBlockInfo.readSchema(), + nextBlockInfo.readSchema())) { + return true; + } + // Now for Iceberg, blocks with different partition schemas or partition values + // do not coalesce. + // Will try to figure out if it is possible to merge and add different + // partition values correctly in the future, to allow coalescing even + // partition values differ but with the same partition schema, + // tracked by https://github.com/NVIDIA/spark-rapids/issues/6423. + IcebergParquetExtraInfo curEInfo = + (IcebergParquetExtraInfo)currentBlockInfo.extraInfo(); + IcebergParquetExtraInfo nextEInfo = + (IcebergParquetExtraInfo)nextBlockInfo.extraInfo(); + if (!samePartitionSpec(curEInfo, nextEInfo)) { + return true; + } + + return super.checkIfNeedToSplitDataBlock(currentBlockInfo, nextBlockInfo); + } + + @Override + public ColumnarBatch finalizeOutputBatch(ColumnarBatch batch, ExtraInfo extraInfo) { + Map idToConsts = ((IcebergParquetExtraInfo)extraInfo).idToConstant(); + Schema expectedSchema = ((IcebergParquetExtraInfo)extraInfo).expectedSchema(); + return GpuIcebergReader.addUpcastsIfNeeded( + GpuIcebergReader.addConstantColumns(batch, expectedSchema, idToConsts), + expectedSchema); + } + + private boolean samePartitionSpec(IcebergParquetExtraInfo curEInfo, + IcebergParquetExtraInfo nextEInfo) { + if (curEInfo.partitionSpec().partitionType() + .equals(nextEInfo.partitionSpec().partitionType())) { + // partition schema is equivalent, check the partition value next. + // Only identity fields were added into constants map. + return curEInfo.partitionSpec().identitySourceIds().stream().allMatch(id -> + Objects.deepEquals( + curEInfo.idToConstant().get(id), + nextEInfo.idToConstant().get(id))); + } + return false; + } + }; // end of "return new MultiFileParquetPartitionReader" + } + + @Override + public ColumnarBatch next() { + return rapidsReader.get(); + } + } } diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkBatchQueryScan.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkBatchQueryScan.java index d1fa44c9469..b89ad62163e 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkBatchQueryScan.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkBatchQueryScan.java @@ -106,7 +106,9 @@ public static GpuSparkBatchQueryScan fromCpu(Scan cpuInstance, RapidsConf rapids // No TableScan instance, so try to build one now scan = buildScan(cpuInstance, table, readConf, expectedSchema, filters); } - return new GpuSparkBatchQueryScan(SparkSession.active(), table, scan, readConf, expectedSchema, filters, rapidsConf); + return new GpuSparkBatchQueryScan(SparkSession.active(), table, scan, readConf, expectedSchema, filters, rapidsConf, + false // queryUsesInputFile + ); } // Try to build an Iceberg TableScan when one was not found in the CPU instance. @@ -165,9 +167,10 @@ private static TableScan buildScan(Scan cpuInstance, } GpuSparkBatchQueryScan(SparkSession spark, Table table, TableScan scan, SparkReadConf readConf, - Schema expectedSchema, List filters, RapidsConf rapidsConf) { + Schema expectedSchema, List filters, RapidsConf rapidsConf, + boolean queryUsesInputFile) { - super(spark, table, readConf, expectedSchema, filters, rapidsConf); + super(spark, table, readConf, expectedSchema, filters, rapidsConf, queryUsesInputFile); this.scan = scan; this.snapshotId = readConf.snapshotId(); @@ -346,14 +349,15 @@ public boolean equals(Object o) { Objects.equals(snapshotId, that.snapshotId) && Objects.equals(startSnapshotId, that.startSnapshotId) && Objects.equals(endSnapshotId, that.endSnapshotId) && - Objects.equals(asOfTimestamp, that.asOfTimestamp); + Objects.equals(asOfTimestamp, that.asOfTimestamp) && + queryUsesInputFile() == that.queryUsesInputFile(); } @Override public int hashCode() { return Objects.hash( table().name(), readSchema(), filterExpressions().toString(), runtimeFilterExpressions.toString(), - snapshotId, startSnapshotId, endSnapshotId, asOfTimestamp); + snapshotId, startSnapshotId, endSnapshotId, asOfTimestamp, queryUsesInputFile()); } @Override @@ -362,4 +366,12 @@ public String toString() { "IcebergScan(table=%s, type=%s, filters=%s, runtimeFilters=%s, caseSensitive=%s)", table(), expectedSchema().asStruct(), filterExpressions(), runtimeFilterExpressions, caseSensitive()); } + + /** Return a copy of "this" but with "queryUsesInputFile = true" */ + public GpuSparkBatchQueryScan copyWithInputFileTrue() { + return new GpuSparkBatchQueryScan(SparkSession.active(), table(), this.scan, readConf(), + expectedSchema(), filterExpressions(), rapidsConf(), + true // queryUsesInputFile + ); + } } diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java index 2b956d33a7e..2c5b670fc3a 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java @@ -79,13 +79,14 @@ abstract class GpuSparkScan extends ScanWithMetricsWrapper private final List filterExpressions; private final boolean readTimestampWithoutZone; private final RapidsConf rapidsConf; + private final boolean queryUsesInputFile; // lazy variables private StructType readSchema = null; GpuSparkScan(SparkSession spark, Table table, SparkReadConf readConf, Schema expectedSchema, List filters, - RapidsConf rapidsConf) { + RapidsConf rapidsConf, boolean queryUsesInputFile) { SparkSchemaUtil.validateMetadataColumnReferences(table.schema(), expectedSchema); @@ -97,12 +98,21 @@ abstract class GpuSparkScan extends ScanWithMetricsWrapper this.filterExpressions = filters != null ? filters : Collections.emptyList(); this.readTimestampWithoutZone = readConf.handleTimestampWithoutZone(); this.rapidsConf = rapidsConf; + this.queryUsesInputFile = queryUsesInputFile; } protected Table table() { return table; } + protected SparkReadConf readConf() { + return readConf; + } + + protected RapidsConf rapidsConf() { + return rapidsConf; + } + protected boolean caseSensitive() { return caseSensitive; } @@ -117,6 +127,10 @@ protected List filterExpressions() { protected abstract List tasks(); + boolean queryUsesInputFile() { + return queryUsesInputFile; + } + @Override public Batch toBatch() { return new SparkBatch(sparkContext, table, readConf, tasks(), expectedSchema, @@ -184,14 +198,19 @@ static class ReaderFactory implements PartitionReaderFactory { private final scala.collection.immutable.Set allCloudSchemes; private final boolean canUseParquetMultiThread; private final boolean canUseParquetCoalescing; + private final boolean isParquetPerFileReadEnabled; public ReaderFactory(scala.collection.immutable.Map metrics, - RapidsConf rapidsConf) { + RapidsConf rapidsConf, boolean queryUsesInputFile) { this.metrics = metrics; this.allCloudSchemes = rapidsConf.getCloudSchemes().toSet(); - // Only multi-threaded Parquet is supported. + this.isParquetPerFileReadEnabled = rapidsConf.isParquetPerFileReadEnabled(); this.canUseParquetMultiThread = rapidsConf.isParquetMultiThreadReadEnabled(); - this.canUseParquetCoalescing = false; + // Here ignores the "ignoreCorruptFiles" comparing to the code in + // "GpuParquetMultiFilePartitionReaderFactory", since "ignoreCorruptFiles" is + // not honored by Iceberg. + this.canUseParquetCoalescing = rapidsConf.isParquetCoalesceFileReadEnabled() && + !queryUsesInputFile; } @Override @@ -203,7 +222,6 @@ public PartitionReader createReader(InputPartition partition) { public PartitionReader createColumnarReader(InputPartition partition) { if (partition instanceof ReadTask) { ReadTask rTask = (ReadTask) partition; - // ret = (canAccelerateRead, isMultiThread, fileFormat) = (_1(), _2(), _3()) scala.Tuple3 ret = multiFileReadCheck(rTask); boolean canAccelerateRead = ret._1(); if (canAccelerateRead) { @@ -233,6 +251,7 @@ public boolean supportColumnarReads(InputPartition partition) { */ private scala.Tuple3 multiFileReadCheck(ReadTask readTask) { Collection scans = readTask.files(); + boolean isSingleFormat = false, isPerFileReadEnabled = false; boolean canUseMultiThread = false, canUseCoalescing = false; FileFormat ff = null; // Require all the files in a partition have the same file format. @@ -240,9 +259,11 @@ private scala.Tuple3 multiFileReadCheck(ReadTask r // Now only Parquet is supported. canUseMultiThread = canUseParquetMultiThread; canUseCoalescing = canUseParquetCoalescing; + isPerFileReadEnabled = isParquetPerFileReadEnabled; + isSingleFormat = true; ff = FileFormat.PARQUET; } - boolean canAccelerateRead = canUseMultiThread || canUseCoalescing; + boolean canAccelerateRead = !isPerFileReadEnabled && isSingleFormat; String[] files = scans.stream().map(f -> f.file().path().toString()) .toArray(String[]::new); // Get the final decision for the subtype of the Rapids reader. diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/SparkBatch.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/SparkBatch.java index 413d07694b0..e10b0883586 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/SparkBatch.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/SparkBatch.java @@ -84,7 +84,8 @@ public InputPartition[] planInputPartitions() { @Override public PartitionReaderFactory createReaderFactory() { - return new GpuSparkScan.ReaderFactory(parentScan.metrics(), rapidsConf); + return new GpuSparkScan.ReaderFactory(parentScan.metrics(), rapidsConf, + parentScan.queryUsesInputFile()); } @Override diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala index b83751ee8fc..9104e01c024 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala @@ -541,6 +541,7 @@ trait SingleDataBlockInfo { def partitionValues: InternalRow // partition value def dataBlock: DataBlockBase // a single block info of a single file def schema: SchemaBase // schema information + def readSchema: StructType // read schema information def extraInfo: ExtraInfo // extra information } @@ -578,8 +579,7 @@ class BatchContext( * * @param conf Configuration * @param clippedBlocks the block metadata from the original file that has been - * clipped to only contain the column chunks to be read - * @param readDataSchema the Spark schema describing what will be read + * clipped to only contain the column chunks to be read * @param partitionSchema schema of partitions * @param maxReadBatchSizeRows soft limit on the maximum number of rows the reader reads per batch * @param maxReadBatchSizeBytes soft limit on the maximum number of bytes the reader reads per batch @@ -589,7 +589,6 @@ class BatchContext( abstract class MultiFileCoalescingPartitionReaderBase( conf: Configuration, clippedBlocks: Seq[SingleDataBlockInfo], - readDataSchema: StructType, partitionSchema: StructType, maxReadBatchSizeRows: Integer, maxReadBatchSizeBytes: Long, @@ -603,6 +602,7 @@ abstract class MultiFileCoalescingPartitionReaderBase( private case class CurrentChunkMeta( clippedSchema: SchemaBase, + readSchema: StructType, currentChunk: Seq[(Path, DataBlockBase)], numTotalRows: Long, rowsPerPartition: Array[Long], @@ -688,11 +688,12 @@ abstract class MultiFileCoalescingPartitionReaderBase( * @param dataBuffer the data which can be decoded in GPU * @param dataSize data size * @param clippedSchema the clipped schema + * @param readSchema the expected schema * @param extraInfo the extra information for specific file format * @return Table */ def readBufferToTable(dataBuffer: HostMemoryBuffer, dataSize: Long, clippedSchema: SchemaBase, - extraInfo: ExtraInfo): Table + readSchema: StructType, extraInfo: ExtraInfo): Table /** * Write a header for a specific file format. If there is no header for the file format, @@ -741,6 +742,21 @@ abstract class MultiFileCoalescingPartitionReaderBase( new BatchContext(chunkedBlocks, clippedSchema) } + /** + * A callback to finalize the output batch. The batch returned will be the final + * output batch of the reader's "get" method. + * + * @param batch the batch after decoding, adding partitioned columns. + * @param extraInfo the corresponding extra information of the input batch. + * @return the finalized columnar batch. + */ + protected def finalizeOutputBatch( + batch: ColumnarBatch, + extraInfo: ExtraInfo): ColumnarBatch = { + // Equivalent to returning the input batch directly. + GpuColumnVector.incRefCounts(batch) + } + override def next(): Boolean = { batch.foreach(_.close()) batch = None @@ -765,7 +781,7 @@ abstract class MultiFileCoalescingPartitionReaderBase( private def readBatch(): Option[ColumnarBatch] = { withResource(new NvtxRange(s"$getFileFormatShortName readBatch", NvtxColor.GREEN)) { _ => val currentChunkMeta = populateCurrentBlockChunk() - if (currentChunkMeta.clippedSchema.fieldNames.isEmpty) { + val retBatch = if (currentChunkMeta.clippedSchema.fieldNames.isEmpty) { // not reading any data, so return a degenerate ColumnarBatch with the row count if (currentChunkMeta.numTotalRows == 0) { None @@ -773,7 +789,7 @@ abstract class MultiFileCoalescingPartitionReaderBase( val rows = currentChunkMeta.numTotalRows.toInt // Someone is going to process this data, even if it is just a row count GpuSemaphore.acquireIfNecessary(TaskContext.get(), metrics(SEMAPHORE_WAIT_TIME)) - val nullColumns = readDataSchema.safeMap(f => + val nullColumns = currentChunkMeta.readSchema.safeMap(f => GpuColumnVector.fromNull(rows, f.dataType).asInstanceOf[SparkVector]) val emptyBatch = new ColumnarBatch(nullColumns.toArray, rows) addAllPartitionValues(Some(emptyBatch), currentChunkMeta.allPartValues, @@ -781,9 +797,9 @@ abstract class MultiFileCoalescingPartitionReaderBase( } } else { val table = readToTable(currentChunkMeta.currentChunk, currentChunkMeta.clippedSchema, - currentChunkMeta.extraInfo) + currentChunkMeta.readSchema, currentChunkMeta.extraInfo) try { - val colTypes = readDataSchema.fields.map(f => f.dataType) + val colTypes = currentChunkMeta.readSchema.fields.map(f => f.dataType) val maybeBatch = table.map(t => GpuColumnVector.from(t, colTypes)) maybeBatch.foreach { batch => logDebug(s"GPU batch size: ${GpuColumnVector.getTotalDeviceMemoryUsed(batch)} bytes") @@ -796,12 +812,16 @@ abstract class MultiFileCoalescingPartitionReaderBase( table.foreach(_.close()) } } + withResource(retBatch) { _ => + retBatch.map(b => finalizeOutputBatch(b, currentChunkMeta.extraInfo)) + } } } private def readToTable( currentChunkedBlocks: Seq[(Path, DataBlockBase)], clippedSchema: SchemaBase, + readDataSchema: StructType, extraInfo: ExtraInfo): Option[Table] = { if (currentChunkedBlocks.isEmpty) { return None @@ -811,7 +831,8 @@ abstract class MultiFileCoalescingPartitionReaderBase( if (dataSize == 0) { None } else { - val table = readBufferToTable(dataBuffer, dataSize, clippedSchema, extraInfo) + val table = readBufferToTable(dataBuffer, dataSize, clippedSchema, readDataSchema, + extraInfo) closeOnExcept(table) { _ => maxDeviceMemory = max(GpuColumnVector.getTotalDeviceMemoryUsed(table), maxDeviceMemory) if (readDataSchema.length < table.getNumberOfColumns) { @@ -890,7 +911,7 @@ abstract class MultiFileCoalescingPartitionReaderBase( // Just ensure to close buffer when there is an exception closeOnExcept(buffer) { _ => logWarning(s"The original estimated size $initTotalSize is too small, " + - s"reallocing and copying data to bigger buffer size: $bufferSize") + s"reallocating and copying data to bigger buffer size: $bufferSize") } // Copy the old buffer to a new allocated bigger buffer and close the old buffer buf = withResource(buffer) { _ => @@ -944,10 +965,11 @@ abstract class MultiFileCoalescingPartitionReaderBase( var currentFile: Path = null var currentPartitionValues: InternalRow = null var currentClippedSchema: SchemaBase = null + var currentReadSchema: StructType = null val rowsPerPartition = new ArrayBuffer[Long]() var lastPartRows: Long = 0 val allPartValues = new ArrayBuffer[InternalRow]() - var currrentDataBlock: SingleDataBlockInfo = null + var currentDataBlock: SingleDataBlockInfo = null var extraInfo: ExtraInfo = null @tailrec @@ -955,11 +977,12 @@ abstract class MultiFileCoalescingPartitionReaderBase( if (blockIterator.hasNext) { if (currentFile == null) { // first time of readNextBatch - currrentDataBlock = blockIterator.head + currentDataBlock = blockIterator.head currentFile = blockIterator.head.filePath currentPartitionValues = blockIterator.head.partitionValues allPartValues += currentPartitionValues currentClippedSchema = blockIterator.head.schema + currentReadSchema = blockIterator.head.readSchema extraInfo = blockIterator.head.extraInfo } @@ -969,12 +992,12 @@ abstract class MultiFileCoalescingPartitionReaderBase( } if (numRows == 0 || numRows + peekedRowCount <= maxReadBatchSizeRows) { - val estimatedBytes = GpuBatchUtils.estimateGpuMemory(readDataSchema, peekedRowCount) + val estimatedBytes = GpuBatchUtils.estimateGpuMemory(currentReadSchema, peekedRowCount) if (numBytes == 0 || numBytes + estimatedBytes <= maxReadBatchSizeBytes) { // only care to check if we are actually adding in the next chunk if (currentFile != blockIterator.head.filePath) { // check if need to split next data block into another ColumnarBatch - if (checkIfNeedToSplitDataBlock(currrentDataBlock, blockIterator.head)) { + if (checkIfNeedToSplitDataBlock(currentDataBlock, blockIterator.head)) { logInfo(s"splitting ${blockIterator.head.filePath} into another batch!") return } @@ -992,7 +1015,8 @@ abstract class MultiFileCoalescingPartitionReaderBase( currentFile = blockIterator.head.filePath currentPartitionValues = blockIterator.head.partitionValues currentClippedSchema = blockIterator.head.schema - currrentDataBlock = blockIterator.head + currentReadSchema = blockIterator.head.readSchema + currentDataBlock = blockIterator.head } val nextBlock = blockIterator.next() @@ -1011,7 +1035,7 @@ abstract class MultiFileCoalescingPartitionReaderBase( logDebug(s"Loaded $numRows rows from ${getFileFormatShortName}. " + s"${getFileFormatShortName} bytes read: $numChunkBytes. Estimated GPU bytes: $numBytes. " + s"Number of different partitions: ${allPartValues.size}") - CurrentChunkMeta(currentClippedSchema, currentChunk, + CurrentChunkMeta(currentClippedSchema, currentReadSchema, currentChunk, numRows, rowsPerPartition.toArray, allPartValues.toArray, extraInfo) } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index 4cca3c862b4..33cdaf82ee9 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -328,6 +328,7 @@ case class GpuOrcMultiFilePartitionReaderFactory( OrcDataStripe(OrcStripeWithMeta(block, orcPartitionReaderContext)), file.partitionValues, OrcSchemaWrapper(orcPartitionReaderContext.updatedReadSchema), + readDataSchema, OrcExtraInfo(orcPartitionReaderContext.requestedMapping))) } metrics.get("scanTime").foreach { @@ -1593,6 +1594,7 @@ private case class OrcSingleStripeMeta( dataBlock: OrcDataStripe, // Orc stripe information with the OrcPartitionReaderContext partitionValues: InternalRow, // partitioned values schema: OrcSchemaWrapper, // Orc schema + readSchema: StructType, // Orc read schema extraInfo: OrcExtraInfo // Orc ExtraInfo containing the requested column ids ) extends SingleDataBlockInfo @@ -1623,7 +1625,7 @@ class MultiFileOrcPartitionReader( partitionSchema: StructType, numThreads: Int, isCaseSensitive: Boolean) - extends MultiFileCoalescingPartitionReaderBase(conf, clippedStripes, readDataSchema, + extends MultiFileCoalescingPartitionReaderBase(conf, clippedStripes, partitionSchema, maxReadBatchSizeRows, maxReadBatchSizeBytes, numThreads, execMetrics) with OrcCommonFunctions { @@ -1865,6 +1867,7 @@ class MultiFileOrcPartitionReader( dataBuffer: HostMemoryBuffer, dataSize: Long, clippedSchema: SchemaBase, + readSchema: StructType, extraInfo: ExtraInfo): Table = decodeToTable(dataBuffer, dataSize, clippedSchema, extraInfo.requestedMapping, isCaseSensitive, files) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index a35c5527e9c..a24472947f9 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -1009,17 +1009,16 @@ case class GpuParquetMultiFilePartitionReaderFactory( ParquetDataBlock(block), file.partitionValues, ParquetSchemaWrapper(singleFileInfo.schema), - ParquetExtraInfo(singleFileInfo.isCorrectedRebaseMode, + singleFileInfo.readSchema, + new ParquetExtraInfo(singleFileInfo.isCorrectedRebaseMode, singleFileInfo.isCorrectedInt96RebaseMode, singleFileInfo.hasInt96Timestamps))) } metrics.get("scanTime").foreach { _ += TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - currentTime) } - new MultiFileParquetPartitionReader(conf, files, clippedBlocks, - isCaseSensitive, readDataSchema, debugDumpPrefix, - maxReadBatchSizeRows, maxReadBatchSizeBytes, metrics, - partitionSchema, numThreads, ignoreMissingFiles, ignoreCorruptFiles, - readUseFieldId) + new MultiFileParquetPartitionReader(conf, files, clippedBlocks, isCaseSensitive, + debugDumpPrefix, maxReadBatchSizeRows, maxReadBatchSizeBytes, metrics, + partitionSchema, numThreads, ignoreMissingFiles, ignoreCorruptFiles, readUseFieldId) } /** @@ -1427,8 +1426,8 @@ private case class ParquetDataBlock(dataBlock: BlockMetaData) extends DataBlockB } /** Parquet extra information containing isCorrectedRebaseMode */ -case class ParquetExtraInfo(isCorrectedRebaseMode: Boolean, - isCorrectedInt96RebaseMode: Boolean, hasInt96Timestamps: Boolean) extends ExtraInfo +class ParquetExtraInfo(val isCorrectedRebaseMode: Boolean, + val isCorrectedInt96RebaseMode: Boolean, val hasInt96Timestamps: Boolean) extends ExtraInfo // contains meta about a single block in a file private case class ParquetSingleDataBlockMeta( @@ -1436,6 +1435,7 @@ private case class ParquetSingleDataBlockMeta( dataBlock: ParquetDataBlock, partitionValues: InternalRow, schema: ParquetSchemaWrapper, + readSchema: StructType, extraInfo: ParquetExtraInfo) extends SingleDataBlockInfo /** @@ -1452,7 +1452,6 @@ private case class ParquetSingleDataBlockMeta( * @param clippedBlocks the block metadata from the original Parquet file that has been clipped * to only contain the column chunks to be read * @param isSchemaCaseSensitive whether schema is case sensitive - * @param readDataSchema the Spark schema describing what will be read * @param debugDumpPrefix a path prefix to use for dumping the fabricated Parquet data or null * @param maxReadBatchSizeRows soft limit on the maximum number of rows the reader reads per batch * @param maxReadBatchSizeBytes soft limit on the maximum number of bytes the reader reads per batch @@ -1467,7 +1466,6 @@ class MultiFileParquetPartitionReader( splits: Array[PartitionedFile], clippedBlocks: Seq[ParquetSingleDataBlockMeta], override val isSchemaCaseSensitive: Boolean, - readDataSchema: StructType, debugDumpPrefix: String, maxReadBatchSizeRows: Integer, maxReadBatchSizeBytes: Long, @@ -1477,7 +1475,7 @@ class MultiFileParquetPartitionReader( ignoreMissingFiles: Boolean, ignoreCorruptFiles: Boolean, useFieldId: Boolean) - extends MultiFileCoalescingPartitionReaderBase(conf, clippedBlocks, readDataSchema, + extends MultiFileCoalescingPartitionReaderBase(conf, clippedBlocks, partitionSchema, maxReadBatchSizeRows, maxReadBatchSizeBytes, numThreads, execMetrics) with ParquetPartitionReaderBase { @@ -1489,7 +1487,7 @@ class MultiFileParquetPartitionReader( block.asInstanceOf[ParquetDataBlock].dataBlock implicit def toDataBlockBase(blocks: Seq[BlockMetaData]): Seq[DataBlockBase] = - blocks.map(ParquetDataBlock(_)) + blocks.map(ParquetDataBlock) implicit def toBlockMetaDataSeq(blocks: Seq[DataBlockBase]): Seq[BlockMetaData] = blocks.map(_.asInstanceOf[ParquetDataBlock].dataBlock) @@ -1585,7 +1583,7 @@ class MultiFileParquetPartitionReader( override final def getFileFormatShortName: String = "Parquet" override def readBufferToTable(dataBuffer: HostMemoryBuffer, dataSize: Long, - clippedSchema: SchemaBase, extraInfo: ExtraInfo): Table = { + clippedSchema: SchemaBase, readDataSchema: StructType, extraInfo: ExtraInfo): Table = { // Dump parquet data into a file dumpDataToFile(dataBuffer, dataSize, splits, Option(debugDumpPrefix), Some("parquet")) @@ -1668,7 +1666,7 @@ class MultiFileParquetPartitionReader( * processed on the GPU. This affects the amount of host memory used. * @param ignoreMissingFiles Whether to ignore missing files * @param ignoreCorruptFiles Whether to ignore corrupt files - * @param useFieldId Whether to ignore corrupt files + * @param useFieldId Whether to use field id for column matching */ class MultiFileCloudParquetPartitionReader( override val conf: Configuration, @@ -1855,8 +1853,7 @@ class MultiFileCloudParquetPartitionReader( val (hostBuffer, size) = memBuffersAndSize.head val nextBatch = readBufferToTable(buffer.isCorrectRebaseMode, buffer.isCorrectInt96RebaseMode, buffer.hasInt96Timestamps, buffer.clippedSchema, - buffer.readSchema, buffer.partitionedFile.partitionValues, - hostBuffer, size, buffer.partitionedFile.filePath) + buffer.readSchema, buffer.partitionedFile, hostBuffer, size) if (memBuffersAndSize.length > 1) { val updatedBuffers = memBuffersAndSize.drop(1) currentFileHostBuffers = Some(buffer.copy(memBuffersAndSizes = updatedBuffers)) @@ -1874,10 +1871,9 @@ class MultiFileCloudParquetPartitionReader( hasInt96Timestamps: Boolean, clippedSchema: MessageType, readDataSchema: StructType, - partValues: InternalRow, + partedFile: PartitionedFile, hostBuffer: HostMemoryBuffer, - dataSize: Long, - fileName: String): Option[ColumnarBatch] = { + dataSize: Long): Option[ColumnarBatch] = { val table = withResource(hostBuffer) { _ => // Dump parquet data into a file @@ -1897,7 +1893,7 @@ class MultiFileCloudParquetPartitionReader( maxDeviceMemory = max(GpuColumnVector.getTotalDeviceMemoryUsed(table), maxDeviceMemory) if (readDataSchema.length < table.getNumberOfColumns) { throw new QueryExecutionException(s"Expected ${readDataSchema.length} columns " + - s"but read ${table.getNumberOfColumns} from $fileName") + s"but read ${table.getNumberOfColumns} from ${partedFile.filePath}") } } metrics(NUM_OUTPUT_BATCHES) += 1 @@ -1912,7 +1908,7 @@ class MultiFileCloudParquetPartitionReader( } // we have to add partition values here for this batch, we already verified that // its not different for all the blocks in this batch - addPartitionValues(maybeBatch, partValues, partitionSchema) + addPartitionValues(maybeBatch, partedFile.partitionValues, partitionSchema) } finally { table.foreach(_.close()) } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExternalSource.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExternalSource.scala index 43ddbf440e6..1367ee95c68 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExternalSource.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExternalSource.scala @@ -137,8 +137,7 @@ object ExternalSource extends Logging { if (hasSparkAvroJar && avroProvider.isSupportedScan(scan)) { avroProvider.copyScanWithInputFileTrue(scan) } else if (hasIcebergJar && icebergProvider.isSupportedScan(scan)) { - // Iceberg does not yet support a coalescing reader, so nothing to change - scan + icebergProvider.copyScanWithInputFileTrue(scan) } else { throw new RuntimeException(s"Unsupported scan type: ${scan.getClass.getSimpleName}") } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala index a5f79b2e12d..f98a8274951 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala @@ -270,6 +270,7 @@ case class GpuAvroMultiFilePartitionReaderFactory( AvroDataBlock(block), file.partitionValues, AvroSchemaWrapper(SchemaConverters.toAvroType(readDataSchema)), + readDataSchema, AvroExtraInfo())) if (singleFileInfo.blocks.nonEmpty) { // No need to check the header since it can not be null when blocks is not empty here. @@ -831,7 +832,7 @@ class GpuMultiFileAvroPartitionReader( override val debugDumpPrefix: Option[String], execMetrics: Map[String, GpuMetric], mapPathHeader: Map[Path, Header]) - extends MultiFileCoalescingPartitionReaderBase(conf, clippedBlocks, readDataSchema, + extends MultiFileCoalescingPartitionReaderBase(conf, clippedBlocks, partitionSchema, maxReadBatchSizeRows, maxReadBatchSizeBytes, numThreads, execMetrics) with GpuAvroReaderBase { @@ -890,7 +891,7 @@ class GpuMultiFileAvroPartitionReader( } override def readBufferToTable(dataBuffer: HostMemoryBuffer, dataSize: Long, - clippedSchema: SchemaBase, extraInfo: ExtraInfo): Table = { + clippedSchema: SchemaBase, readSchema: StructType, extraInfo: ExtraInfo): Table = { sendToGpuUnchecked(dataBuffer, dataSize, splits) } @@ -1017,6 +1018,7 @@ case class AvroSingleDataBlockInfo( dataBlock: AvroDataBlock, partitionValues: InternalRow, schema: AvroSchemaWrapper, + readSchema: StructType, extraInfo: AvroExtraInfo) extends SingleDataBlockInfo case class AvroBatchContext( From 28725a53b97194771585d42404861e16a02aa7a6 Mon Sep 17 00:00:00 2001 From: sinkinben Date: Wed, 31 Aug 2022 10:25:30 +0800 Subject: [PATCH 070/190] Support bool/int8/16/32/64 castings for ORC reading [databricks] (#6454) * feat: implement casting bool/int8/int16/int32/int64 -> {string, float, double(float64), timestamp} In ORC reading, casting from integers to timestamp has different behaviors in different versions of spark. * From 311 <= spark < 320 (including 311, 312, 313, 314), it considers integers as milliseconds. * Since spark >= 320, it considers integers as seconds. Added a shims `OrcCastingShims.scala` to fix this. Signed-off-by: sinkinben --- .../src/main/python/orc_cast_test.py | 83 +++++++++++++++++++ integration_tests/src/main/python/orc_test.py | 20 ----- .../spark/rapids/shims/OrcCastingShims.scala | 62 ++++++++++++++ .../spark/rapids/shims/OrcCastingShims.scala | 62 ++++++++++++++ .../com/nvidia/spark/rapids/GpuOrcScan.scala | 50 +++++++++-- 5 files changed, 250 insertions(+), 27 deletions(-) create mode 100644 integration_tests/src/main/python/orc_cast_test.py create mode 100644 sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala create mode 100644 sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala diff --git a/integration_tests/src/main/python/orc_cast_test.py b/integration_tests/src/main/python/orc_cast_test.py new file mode 100644 index 00000000000..6a84407a632 --- /dev/null +++ b/integration_tests/src/main/python/orc_cast_test.py @@ -0,0 +1,83 @@ +# Copyright (c) 2020-2022, NVIDIA CORPORATION. +# +# 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. + +import pytest + +from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error +from data_gen import * +from pyspark.sql.types import * +from spark_session import with_cpu_session +from orc_test import reader_opt_confs + + +def create_orc(data_gen_list, data_path): + # generate ORC dataframe, and dump it to local file 'data_path' + with_cpu_session( + lambda spark: gen_df(spark, data_gen_list).write.orc(data_path) + ) + + +@pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) +@pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) +@pytest.mark.parametrize('to_type', ['boolean', 'tinyint', 'smallint', 'int', 'bigint']) +def test_casting_among_integer_types(spark_tmp_path, reader_confs, v1_enabled_list, to_type): + # cast integral types to another integral types + int_gens = [boolean_gen] + integral_gens + gen_list = [('c' + str(i), gen) for i, gen in enumerate(int_gens)] + data_path = spark_tmp_path + '/ORC_DATA' + create_orc(gen_list, data_path) + + # generate schema string like "c0 to_type, c1 to_type, ..., c4 to_type" + schema_str = " {}, ".join([x[0] for x in gen_list]) + " {}" + schema_str = schema_str.format(*([to_type] * len(gen_list))) + all_confs = copy_and_update(reader_confs, + {'spark.sql.sources.useV1SourceList': v1_enabled_list}) + assert_gpu_and_cpu_are_equal_collect( + lambda spark: spark.read.schema(schema_str).orc(data_path), + conf=all_confs) + + +@pytest.mark.parametrize('to_type', ['float', 'double', 'string', 'timestamp']) +def test_casting_from_integer(spark_tmp_path, to_type): + orc_path = spark_tmp_path + '/orc_cast_integer' + # The Python 'datetime' module only supports a max-year of 10000, so we set the Long type max + # to '1e11'. If the long-value is out of this range, pytest will throw an exception. + data_gen = [('boolean_col', boolean_gen), ('tinyint_col', byte_gen), + ('smallint_col', ShortGen(min_val=BYTE_MAX + 1)), + ('int_col', IntegerGen(min_val=SHORT_MAX + 1)), + ('bigint_col', LongGen(min_val=INT_MAX + 1, max_val=int(1e11))), + ('negint_col', IntegerGen(max_val=-1))] + create_orc(data_gen, orc_path) + + schema_str = "boolean_col {}, tinyint_col {}, smallint_col {}, int_col {}, bigint_col {}, negint_col {}" + assert_gpu_and_cpu_are_equal_collect( + lambda spark: spark.read.schema( + schema_str.format(*([to_type] * len(data_gen)))).orc(orc_path) + ) + +@pytest.mark.parametrize('overflow_long_gen', [LongGen(min_val=int(1e16)), + LongGen(max_val=int(-1e16))]) +@pytest.mark.parametrize('to_type', ['timestamp']) +def test_casting_from_overflow_long(spark_tmp_path, overflow_long_gen,to_type): + # Timestamp(micro-seconds) is actually type of int64, when casting long(int64) to timestamp, + # we need to multiply 1e6 (or 1e3), and it may cause overflow. This function aims to test + # whether if 'ArithmeticException' is caught. + orc_path = spark_tmp_path + '/orc_cast_overflow_long' + create_orc([('long_column', overflow_long_gen)], orc_path) + schema_str = "long_column {}".format(to_type) + assert_gpu_and_cpu_error( + df_fun=lambda spark: spark.read.schema(schema_str).orc(orc_path).collect(), + conf={}, + error_message="ArithmeticException" + ) diff --git a/integration_tests/src/main/python/orc_test.py b/integration_tests/src/main/python/orc_test.py index c063dd07dbb..7c0c775fc68 100644 --- a/integration_tests/src/main/python/orc_test.py +++ b/integration_tests/src/main/python/orc_test.py @@ -671,26 +671,6 @@ def test_orc_scan_with_aggregate_no_pushdown_on_col_partition(spark_tmp_path, ag conf=_orc_aggregate_pushdown_enabled_conf) -@pytest.mark.parametrize('offset', [1,2,3,4], ids=idfn) -@pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) -@pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) -def test_read_type_casting_integral(spark_tmp_path, offset, reader_confs, v1_enabled_list): - int_gens = [boolean_gen] + integral_gens - gen_list = [('c' + str(i), gen) for i, gen in enumerate(int_gens)] - data_path = spark_tmp_path + '/ORC_DATA' - with_cpu_session( - lambda spark: gen_df(spark, gen_list).write.orc(data_path)) - - # build the read schema by a left shift of int_gens - shifted_int_gens = int_gens[offset:] + int_gens[:offset] - rs_gen_list = [('c' + str(i), gen) for i, gen in enumerate(shifted_int_gens)] - rs = StructGen(rs_gen_list, nullable=False).data_type - all_confs = copy_and_update(reader_confs, - {'spark.sql.sources.useV1SourceList': v1_enabled_list}) - assert_gpu_and_cpu_are_equal_collect( - lambda spark: spark.read.schema(rs).orc(data_path), - conf=all_confs) - def test_orc_read_count(spark_tmp_path): data_path = spark_tmp_path + '/ORC_DATA' orc_gens = [int_gen, string_gen, double_gen] diff --git a/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala b/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala new file mode 100644 index 00000000000..fe914811df2 --- /dev/null +++ b/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala @@ -0,0 +1,62 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import ai.rapids.cudf.{ColumnView, DType, Scalar} +import com.nvidia.spark.rapids.GpuOrcScan.{testLongMultiplicationOverflow, withResource} + +object OrcCastingShims { + /** + * Cast ColumnView of integer types to timestamp (in milliseconds). + * @param col The column view of integer types. + * @param fromType BOOL8, INT8/16/32/64 + * @return A new timestamp columnar vector. + */ + def castIntegerToTimestamp(col: ColumnView, fromType: DType): ColumnView = { + fromType match { + case DType.BOOL8 | DType.INT8 | DType.INT16 | DType.INT32 => + // From spark311 until spark314 (not include it), spark consider the integers as + // milli-seconds. + // cuDF requires casting to Long first, then we can cast Long to Timestamp(in microseconds) + // In CPU code of ORC casting, its conversion is 'integer -> milliseconds -> microseconds' + withResource(col.castTo(DType.INT64)) { longs => + withResource(Scalar.fromLong(1000L)) { thousand => + withResource(longs.mul(thousand)) { milliSeconds => + milliSeconds.castTo(DType.TIMESTAMP_MICROSECONDS) + } + } + } + case DType.INT64 => + // We need overflow checking here, since max value of INT64 is about 9 * 1e18, and convert + // INT64 to milliseconds(also a INT64 actually), we need multiply 1000, it may cause long + // integer-overflow. + // If these two 'testLongMultiplicationOverflow' throw no exception, it means no + // Long-overflow when casting 'col' to TIMESTAMP_MICROSECONDS. + if (col.max() != null) { + testLongMultiplicationOverflow(col.max().getLong, 1000L) + } + if (col.min() != null) { + testLongMultiplicationOverflow(col.min().getLong, 1000L) + } + withResource(Scalar.fromLong(1000L)) { thousand => + withResource(col.mul(thousand)) { milliSeconds => + milliSeconds.castTo(DType.TIMESTAMP_MICROSECONDS) + } + } + } + } +} diff --git a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala new file mode 100644 index 00000000000..b793a683e04 --- /dev/null +++ b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala @@ -0,0 +1,62 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import ai.rapids.cudf.{ColumnView, DType, Scalar} +import com.nvidia.spark.rapids.GpuOrcScan.{testLongMultiplicationOverflow, withResource} + +object OrcCastingShims { + /** + * Cast ColumnView of integer types to timestamp (in milliseconds). + * @param col The column view of integer types. + * @param fromType BOOL8, INT8/16/32/64 + * @return A new timestamp columnar vector. + */ + def castIntegerToTimestamp(col: ColumnView, fromType: DType): ColumnView = { + fromType match { + case DType.BOOL8 | DType.INT8 | DType.INT16 | DType.INT32 => + // From spark320, spark consider the integers as seconds. + withResource(col.castTo(DType.INT64)) { longs => + // In CPU, ORC assumes the integer value is in seconds, and returns timestamp in + // micro seconds, so we need to multiply 1e6 here. + withResource(Scalar.fromLong(1000000L)) { value => + withResource(longs.mul(value)) { microSeconds => + microSeconds.castTo(DType.TIMESTAMP_MICROSECONDS) + } + } + } + + case DType.INT64 => + // In CPU code of ORC casting, its conversion is 'integer -> milliseconds -> microseconds' + withResource(Scalar.fromLong(1000L)) { thousand => + withResource(col.mul(thousand)) { milliSeconds => + // We need to check long-overflow here. If milliseconds can not convert to + // micorseconds, then testLongMultiplicationOverflow will throw exception. + if (milliSeconds.max() != null) { + testLongMultiplicationOverflow(milliSeconds.max().getLong, 1000L) + } + if (milliSeconds.min() != null) { + testLongMultiplicationOverflow(milliSeconds.min().getLong, 1000L) + } + withResource(milliSeconds.mul(thousand)) { microSeconds => + microSeconds.castTo(DType.TIMESTAMP_MICROSECONDS) + } + } + } + } + } +} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index 33cdaf82ee9..fa921e517be 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -35,7 +35,7 @@ import com.google.protobuf.CodedOutputStream import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.SchemaUtils._ -import com.nvidia.spark.rapids.shims.{OrcReadingShims, OrcShims, ShimFilePartitionReaderFactory} +import com.nvidia.spark.rapids.shims.{OrcCastingShims, OrcReadingShims, OrcShims, ShimFilePartitionReaderFactory} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.common.io.DiskRangeList @@ -211,6 +211,28 @@ object GpuOrcScan extends Arm { } else { downCastAnyInteger(col, toDt) } + + // bool to float, double(float64) + case (DType.BOOL8, DType.FLOAT32 | DType.FLOAT64) => + col.castTo(toDt) + + // bool to string + case (DType.BOOL8, DType.STRING) => + withResource(col.castTo(toDt)) { casted => + // cuDF produces "ture"/"false" while CPU outputs "TRUE"/"FALSE". + casted.upper() + } + + // integer to float, double(float64), string + case (DType.INT8 | DType.INT16 | DType.INT32 | DType.INT64, + DType.FLOAT32 | DType.FLOAT64 | DType.STRING) => + col.castTo(toDt) + + // {bool, integer types} to timestamp(micro seconds) + case (DType.BOOL8 | DType.INT8 | DType.INT16 | DType.INT32 | DType.INT64, + DType.TIMESTAMP_MICROSECONDS) => + OrcCastingShims.castIntegerToTimestamp(col, fromDt) + // TODO more types, tracked in https://github.com/NVIDIA/spark-rapids/issues/5895 case (f, t) => throw new QueryExecutionException(s"Unsupported type casting: $f -> $t") @@ -231,19 +253,33 @@ object GpuOrcScan extends Arm { // Align with what CPU does. return false } + val toType = to.getCategory from.getCategory match { case BOOLEAN | BYTE | SHORT | INT | LONG => - to.getCategory match { - case BOOLEAN | BYTE | SHORT | INT | LONG => true + toType match { + case BOOLEAN | BYTE | SHORT | INT | LONG | FLOAT | DOUBLE | STRING | + TIMESTAMP => true + // BINARY and DATE are not supported by design. + // The 'to' type (aka read schema) is from Spark, and VARCHAR and CHAR will + // be replaced by STRING. Meanwhile, cuDF doesn't support them as output + // types, and also replaces them with STRING. + // TIMESTAMP_INSTANT is not supported by cuDF. case _ => false } case VARCHAR => - to.getCategory == STRING - // TODO more types, tracked in https://github.com/NVIDIA/spark-rapids/issues/5895 - case _ => - false + toType == STRING + case _ => false } } + + /** + * Test whether if a * b will cause Long-overflow. + * In Math.multiplyExact, if there is an integer-overflow, then it will throw an + * ArithmeticException. + */ + def testLongMultiplicationOverflow(a: Long, b: Long) = { + Math.multiplyExact(a, b) + } } /** From ecca0032c0fcb4f7dedbd651fb231e305493ff0b Mon Sep 17 00:00:00 2001 From: Peixin Date: Thu, 1 Sep 2022 00:37:05 +0800 Subject: [PATCH 071/190] Increase stability of pre-merge pytest run with PVC storage (#6462) Signed-off-by: Peixin Li Signed-off-by: Peixin Li --- jenkins/spark-premerge-build.sh | 5 ++++- jenkins/spark-tests.sh | 6 +++++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/jenkins/spark-premerge-build.sh b/jenkins/spark-premerge-build.sh index c91dffd2e65..228fb6f51fa 100755 --- a/jenkins/spark-premerge-build.sh +++ b/jenkins/spark-premerge-build.sh @@ -183,7 +183,10 @@ export SPARK_HOME="$ARTF_ROOT/spark-$SPARK_VER-bin-hadoop3.2" export PATH="$SPARK_HOME/bin:$SPARK_HOME/sbin:$PATH" tar zxf $SPARK_HOME.tgz -C $ARTF_ROOT && \ rm -f $SPARK_HOME.tgz -export PYTHONPATH=$SPARK_HOME/python:$SPARK_HOME/python/pyspark/:$SPARK_HOME/python/lib/py4j-0.10.9-src.zip +# copy python path libs to container /tmp instead of workspace to avoid ephemeral PVC issue +TMP_PYTHON=/tmp/$(date +"%Y%m%d") +rm -rf $TMP_PYTHON && cp -r $SPARK_HOME/python $TMP_PYTHON +export PYTHONPATH=$TMP_PYTHON/python:$TMP_PYTHON/python/pyspark/:$TMP_PYTHON/python/lib/py4j-0.10.9-src.zip case $BUILD_TYPE in diff --git a/jenkins/spark-tests.sh b/jenkins/spark-tests.sh index c52c0d3f3a8..ea480d93e47 100755 --- a/jenkins/spark-tests.sh +++ b/jenkins/spark-tests.sh @@ -110,7 +110,11 @@ export SPARK_HOME="$ARTF_ROOT/spark-$SPARK_VER-bin-hadoop3.2" export PATH="$SPARK_HOME/bin:$SPARK_HOME/sbin:$PATH" tar zxf $SPARK_HOME.tgz -C $ARTF_ROOT && \ rm -f $SPARK_HOME.tgz -export PYTHONPATH=$SPARK_HOME/python:$SPARK_HOME/python/pyspark/:$SPARK_HOME/python/lib/py4j-0.10.9-src.zip +# copy python path libs to container /tmp instead of workspace to avoid ephemeral PVC issue +TMP_PYTHON=/tmp/$(date +"%Y%m%d") +rm -rf $TMP_PYTHON && cp -r $SPARK_HOME/python $TMP_PYTHON +export PYTHONPATH=$TMP_PYTHON/python:$TMP_PYTHON/python/pyspark/:$TMP_PYTHON/python/lib/py4j-0.10.9-src.zip + # Extract 'value' from conda config string 'key: value' CONDA_ROOT=`conda config --show root_prefix | cut -d ' ' -f2` PYTHON_VER=`conda config --show default_python | cut -d ' ' -f2` From 14d2d10737cc08c4396fa53f4cb27679a2b5ee61 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Wed, 31 Aug 2022 10:20:56 -0700 Subject: [PATCH 072/190] Install reduced pom for dist module (#6434) Fixes #6433 This PR - disables default install in favor of install-file similar how rapids-4-spark is deploy-file'd to Maven central - disables default deploy in favor of deploy-file similar how rapids-4-spark is deploy-file'd to Maven central in deploy.sh After this, the dependency reduced pom is correctly placed after `mvn deploy -pl dist -PnoSnapshots -Ddist.jar.compress=false` - ~/.m2/repository/com/nvidia/rapids-4-spark_2.12/22.10.0-SNAPSHOT/rapids-4-spark_2.12-22.10.0-SNAPSHOT.pom - /tmp/m2-repo/com/nvidia/rapids-4-spark_2.12/22.10.0-SNAPSHOT/rapids-4-spark_2.12-22.10.0*pom Other fixes: - dist.jar.compress is ignored when updating the dist jar Signed-off-by: Gera Shegalov --- dist/pom.xml | 63 +++++++++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 62 insertions(+), 1 deletion(-) diff --git a/dist/pom.xml b/dist/pom.xml index f2b3d3575ea..facf52d388a 100644 --- a/dist/pom.xml +++ b/dist/pom.xml @@ -58,6 +58,7 @@ 312db, 321db + ${project.build.directory}/${project.build.finalName}-${cuda.version}.jar
@@ -331,7 +332,8 @@ + compress="${dist.jar.compress}" + destfile="${dist.jar.name}"/> @@ -450,6 +452,65 @@ + + org.apache.maven.plugins + maven-install-plugin + 3.0.1 + + + default-install + none + + + install-parallel-worlds-jar + install + + install-file + + + ${dist.jar.name} + ${project.artifactId} + ${cuda.version} + ${project.groupId} + ${project.version} + jar + + + + + + + org.apache.maven.plugins + maven-deploy-plugin + 3.0.0 + + + default-deploy + none + + + deploy-parallel-worlds-jar + deploy + + deploy-file + + + ${dist.jar.name} + file://${java.io.tmpdir}/m2-repo + ${project.artifactId} + ${cuda.version} + ${project.groupId} + jar + + ${project.version} + + + + From 040e5a3988b74391e1128ce89c4d53828f8a3e68 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 31 Aug 2022 22:18:53 -0600 Subject: [PATCH 073/190] Use non-capture groups in LIKE regexp to reduce memory overhead and improve performance (#6468) --- .../scala/org/apache/spark/sql/rapids/stringFunctions.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala index 92d55681f66..1288d69ff03 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala @@ -763,8 +763,8 @@ case class GpuLike(left: Expression, right: Expression, escapeChar: Char) case _ => fail(s"the escape character is not allowed to precede '$c'") } case c if c == escapeChar => fail("it is not allowed to end with the escape character") - case '_' => out ++= "(.|\n)" - case '%' => out ++= "(.|\n)*" + case '_' => out ++= "(?:.|\n)" + case '%' => out ++= "(?:.|\n)*" case c => out ++= cudfQuote(c) } } From 399438f44f3491468769880cef23a9c42adba97c Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Thu, 1 Sep 2022 08:16:50 -0500 Subject: [PATCH 074/190] Check more places for Parquet encryption configs (#6472) * Check more places for Parquet encryption configs Signed-off-by: Jason Lowe * Remove debugging code Signed-off-by: Jason Lowe --- .../src/main/python/parquet_write_test.py | 39 ++++++++++++++++++- .../spark/rapids/GpuParquetFileFormat.scala | 18 +++++++-- 2 files changed, 53 insertions(+), 4 deletions(-) diff --git a/integration_tests/src/main/python/parquet_write_test.py b/integration_tests/src/main/python/parquet_write_test.py index 129036fce54..3e0519cf871 100644 --- a/integration_tests/src/main/python/parquet_write_test.py +++ b/integration_tests/src/main/python/parquet_write_test.py @@ -306,7 +306,7 @@ def test_parquet_write_legacy_fallback(spark_tmp_path, ts_write, ts_rebase, spar @pytest.mark.parametrize('write_options', [{"parquet.encryption.footer.key": "k1"}, {"parquet.encryption.column.keys": "k2:a"}, {"parquet.encryption.footer.key": "k1", "parquet.encryption.column.keys": "k2:a"}]) -def test_parquet_write_encryption_fallback(spark_tmp_path, spark_tmp_table_factory, write_options): +def test_parquet_write_encryption_option_fallback(spark_tmp_path, spark_tmp_table_factory, write_options): def write_func(spark, path): writer = unary_op_df(spark, gen).coalesce(1).write for key in write_options: @@ -320,6 +320,43 @@ def write_func(spark, path): data_path, 'DataWritingCommandExec') +@allow_non_gpu("DataWritingCommandExec") +@pytest.mark.parametrize("write_options", [{"parquet.encryption.footer.key": "k1"}, + {"parquet.encryption.column.keys": "k2:a"}, + {"parquet.encryption.footer.key": "k1", "parquet.encryption.column.keys": "k2:a"}]) +def test_parquet_write_encryption_runtimeconfig_fallback(spark_tmp_path, write_options): + gen = IntegerGen() + data_path = spark_tmp_path + '/PARQUET_DATA' + assert_gpu_fallback_write( + lambda spark, path: unary_op_df(spark, gen).coalesce(1).write.parquet(path), + lambda spark, path: spark.read.parquet(path), + data_path, + "DataWritingCommandExec", + conf=write_options) + +@allow_non_gpu("DataWritingCommandExec") +@pytest.mark.parametrize("write_options", [{"parquet.encryption.footer.key": "k1"}, + {"parquet.encryption.column.keys": "k2:a"}, + {"parquet.encryption.footer.key": "k1", "parquet.encryption.column.keys": "k2:a"}]) +def test_parquet_write_encryption_hadoopconfig_fallback(spark_tmp_path, write_options): + gen = IntegerGen() + data_path = spark_tmp_path + '/PARQUET_DATA' + def setup_hadoop_confs(spark): + for k, v in write_options.items(): + spark.sparkContext._jsc.hadoopConfiguration().set(k, v) + def reset_hadoop_confs(spark): + for k in write_options.keys(): + spark.sparkContext._jsc.hadoopConfiguration().unset(k) + try: + with_cpu_session(setup_hadoop_confs) + assert_gpu_fallback_write( + lambda spark, path: unary_op_df(spark, gen).coalesce(1).write.parquet(path), + lambda spark, path: spark.read.parquet(path), + data_path, + "DataWritingCommandExec") + finally: + with_cpu_session(reset_hadoop_confs) + @allow_non_gpu('DataWritingCommandExec') # note that others should fail as well but requires you to load the libraries for them # 'lzo', 'brotli', 'lz4', 'zstd' should all fallback diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala index 276913e1a11..522cfc3df30 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala @@ -47,10 +47,22 @@ object GpuParquetFileFormat { val parquetOptions = new ParquetOptions(options, sqlConf) - val columnEncryption = options.getOrElse("parquet.encryption.column.keys", "") - val footerEncryption = options.getOrElse("parquet.encryption.footer.key", "") + // lookup encryption keys in the options, then Hadoop conf, then Spark runtime conf + def lookupEncryptionConfig(key: String): String = { + options.getOrElse(key, { + val hadoopConf = spark.sparkContext.hadoopConfiguration.get(key, "") + if (hadoopConf.nonEmpty) { + hadoopConf + } else { + spark.conf.get(key, "") + } + }) + } + + val columnEncryption = lookupEncryptionConfig("parquet.encryption.column.keys") + val footerEncryption = lookupEncryptionConfig("parquet.encryption.footer.key") - if (!columnEncryption.isEmpty || !footerEncryption.isEmpty) { + if (columnEncryption.nonEmpty || footerEncryption.nonEmpty) { meta.willNotWorkOnGpu("Encryption is not yet supported on GPU. If encrypted Parquet " + "writes are not required unset the \"parquet.encryption.column.keys\" and " + "\"parquet.encryption.footer.key\" in Parquet options") From 356d32b033c7637ff87bdbaa771febc20fa66b94 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 1 Sep 2022 12:31:26 -0500 Subject: [PATCH 075/190] Change GpuKryoRegistrator to load the classes we want to register with the ShimLoader (#6475) Signed-off-by: Thomas Graves Signed-off-by: Thomas Graves --- .../com/nvidia/spark/rapids/GpuKryoRegistrator.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuKryoRegistrator.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuKryoRegistrator.scala index ce8bc1e69bf..85ebd084a15 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuKryoRegistrator.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuKryoRegistrator.scala @@ -20,13 +20,14 @@ import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} import org.apache.spark.serializer.KryoRegistrator -import org.apache.spark.sql.rapids.execution.{SerializeBatchDeserializeHostBuffer, SerializeConcatHostBuffersDeserializeBatch} class GpuKryoRegistrator extends KryoRegistrator { override def registerClasses(kryo: Kryo): Unit = { - kryo.register(classOf[SerializeConcatHostBuffersDeserializeBatch], - new KryoJavaSerializer()) - kryo.register(classOf[SerializeBatchDeserializeHostBuffer], - new KryoJavaSerializer()) + val allClassesToRegister = Seq( + "org.apache.spark.sql.rapids.execution.SerializeConcatHostBuffersDeserializeBatch", + "org.apache.spark.sql.rapids.execution.SerializeBatchDeserializeHostBuffer") + allClassesToRegister.foreach { classToRegister => + kryo.register(ShimLoader.loadClass(classToRegister), new KryoJavaSerializer()) + } } } From 122e107b977daa012479456794916e292aa1f6e4 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Thu, 1 Sep 2022 11:31:54 -0700 Subject: [PATCH 076/190] Add support for arrays in hashaggregate [databricks] (#6066) * add support for arrays in hashaggregate Signed-off-by: Raza Jafri * allow shuffle exchange on CPU Signed-off-by: Raza Jafri * Opened up ShuffleExchangeExec for Arrays Signed-off-by: Raza Jafri * Explicitly remove list of structs from allowed list Signed-off-by: Raza Jafri * addressed review comments Signed-off-by: Raza Jafri * added ps note for arrays Signed-off-by: Raza Jafri * simplified GpuOverrides for hash_key Signed-off-by: Raza Jafri * Added a ShuffleExec fallback test Signed-off-by: Raza Jafri * Added a ShuffleExec fallback test Signed-off-by: Raza Jafri * Revert "Added a ShuffleExec fallback test" This reverts commit f26c0ff0c21c4faa3085acb27948067c28c06abb. Signed-off-by: Raza Jafri * removed f.hash from the test Signed-off-by: Raza Jafri * remove test for a single partition Signed-off-by: Raza Jafri * Revert "remove test for a single partition" This reverts commit b6bd34b03c3617aae64916fbe01c15d48cc7a227. Signed-off-by: Raza Jafri * Removed test for a single partition Signed-off-by: Raza Jafri Signed-off-by: Raza Jafri Co-authored-by: Raza Jafri --- docs/supported_ops.md | 72 +++++++++---------- .../src/main/python/hash_aggregate_test.py | 15 +++- .../src/main/python/repart_test.py | 15 +++- .../nvidia/spark/rapids/GpuOverrides.scala | 29 ++++++-- .../com/nvidia/spark/rapids/aggregate.scala | 26 +++++-- tools/src/main/resources/supportedExprs.csv | 4 +- 6 files changed, 108 insertions(+), 53 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 8d9dc3151cb..cdd8f6f1105 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -556,7 +556,7 @@ Accelerator supports are described below. S NS NS -PS
not allowed for grouping expressions;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
not allowed for grouping expressions if containing Struct as child;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
PS
not allowed for grouping expressions;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
PS
not allowed for grouping expressions if containing Array or Map as child;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
NS @@ -724,7 +724,7 @@ Accelerator supports are described below. S S NS -PS
Round-robin partitioning is not supported if spark.sql.execution.sortBeforeRepartition is true;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
PS
Round-robin partitioning is not supported if spark.sql.execution.sortBeforeRepartition is true;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
PS
Round-robin partitioning is not supported for nested structs if spark.sql.execution.sortBeforeRepartition is true;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -7737,45 +7737,45 @@ are limited. None project input - - - - - S S - - - - - - - - - - - +S +S +S +S +S +S +PS
UTC is only supported TZ for TIMESTAMP
+S +S +S +S +S +PS
UTC is only supported TZ for child TIMESTAMP
+PS
UTC is only supported TZ for child TIMESTAMP
+PS
UTC is only supported TZ for child TIMESTAMP
+S result - - - - - S S - - - - - - - - - - - +S +S +S +S +S +S +PS
UTC is only supported TZ for TIMESTAMP
+S +S +S +S +S +PS
UTC is only supported TZ for child TIMESTAMP
+PS
UTC is only supported TZ for child TIMESTAMP
+PS
UTC is only supported TZ for child TIMESTAMP
+S KnownNotNull @@ -18594,9 +18594,9 @@ as `a` don't show up in the table. They are controlled by the rules for S NS NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
NS -NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, ARRAY, MAP, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
NS diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index ce2ffcf3094..752a461f58f 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -128,6 +128,19 @@ ('b', FloatGen(nullable=(True, 10.0), special_cases=[(float('nan'), 10.0)])), ('c', LongGen())] +# grouping single-level lists +_grpkey_list_with_non_nested_children = [[('a', RepeatSeqGen(ArrayGen(data_gen), length=3)), + ('b', IntegerGen())] for data_gen in all_basic_gens + decimal_gens] + +#grouping mutliple-level structs with arrays +_grpkey_nested_structs_with_array_basic_child = [ + ('a', RepeatSeqGen(StructGen([ + ['aa', IntegerGen()], + ['ab', ArrayGen(IntegerGen())]]), + length=20)), + ('b', IntegerGen()), + ('c', NullGen())] + _nan_zero_float_special_cases = [ (float('nan'), 5.0), (NEG_FLOAT_NAN_MIN_VALUE, 5.0), @@ -335,7 +348,7 @@ def test_hash_reduction_decimal_overflow_sum(precision): # some optimizations are conspiring against us. conf = {'spark.rapids.sql.batchSizeBytes': '128m'}) -@pytest.mark.parametrize('data_gen', [_longs_with_nulls], ids=idfn) +@pytest.mark.parametrize('data_gen', [_grpkey_nested_structs_with_array_basic_child, _longs_with_nulls] + _grpkey_list_with_non_nested_children, ids=idfn) def test_hash_grpby_sum_count_action(data_gen): assert_gpu_and_cpu_row_counts_equal( lambda spark: gen_df(spark, data_gen, length=100).groupby('a').agg(f.sum('b')) diff --git a/integration_tests/src/main/python/repart_test.py b/integration_tests/src/main/python/repart_test.py index 7b77b7be426..b12a680d3eb 100644 --- a/integration_tests/src/main/python/repart_test.py +++ b/integration_tests/src/main/python/repart_test.py @@ -214,10 +214,23 @@ def test_round_robin_sort_fallback(data_gen): lambda spark : gen_df(spark, data_gen).withColumn('extra', lit(1)).repartition(13), 'ShuffleExchangeExec') +@allow_non_gpu("ProjectExec", "ShuffleExchangeExec") +@ignore_order(local=True) # To avoid extra data shuffle by 'sort on Spark' for this repartition test. +@pytest.mark.parametrize('num_parts', [2, 10, 17, 19, 32], ids=idfn) +@pytest.mark.parametrize('gen', [([('ag', ArrayGen(StructGen([('b1', long_gen)])))], ['ag'])], ids=idfn) +def test_hash_repartition_exact_fallback(gen, num_parts): + data_gen = gen[0] + part_on = gen[1] + assert_gpu_fallback_collect( + lambda spark : gen_df(spark, data_gen, length=1024) \ + .repartition(num_parts, *part_on) \ + .withColumn('id', f.spark_partition_id()) \ + .selectExpr('*'), "ShuffleExchangeExec") + @ignore_order(local=True) # To avoid extra data shuffle by 'sort on Spark' for this repartition test. @pytest.mark.parametrize('num_parts', [1, 2, 10, 17, 19, 32], ids=idfn) @pytest.mark.parametrize('gen', [ - ([('a', boolean_gen)], ['a']), + ([('a', boolean_gen)], ['a']), ([('a', byte_gen)], ['a']), ([('a', short_gen)], ['a']), ([('a', int_gen)], ['a']), diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 30a18b6d77f..cf33be44905 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -1645,9 +1645,7 @@ object GpuOverrides extends Logging { }), expr[KnownFloatingPointNormalized]( "Tag to prevent redundant normalization", - ExprChecks.unaryProjectInputMatchesOutput( - TypeSig.DOUBLE + TypeSig.FLOAT, - TypeSig.DOUBLE + TypeSig.FLOAT), + ExprChecks.unaryProjectInputMatchesOutput(TypeSig.all, TypeSig.all), (a, conf, p, r) => new UnaryExprMeta[KnownFloatingPointNormalized](a, conf, p, r) { override def convertToGpu(child: Expression): GpuExpression = GpuKnownFloatingPointNormalized(child) @@ -3692,11 +3690,26 @@ object GpuOverrides extends Logging { // This needs to match what murmur3 supports. PartChecks(RepeatingParamCheck("hash_key", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.STRUCT).nested(), TypeSig.all)), + TypeSig.STRUCT + TypeSig.ARRAY).nested(), + TypeSig.all) + ), (hp, conf, p, r) => new PartMeta[HashPartitioning](hp, conf, p, r) { override val childExprs: Seq[BaseExprMeta[_]] = hp.expressions.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + override def tagPartForGpu(): Unit = { + val arrayWithStructsHashing = hp.expressions.exists(e => + TrampolineUtil.dataTypeExistsRecursively(e.dataType, + dt => dt match { + case ArrayType(_: StructType, _) => true + case _ => false + }) + ) + if (arrayWithStructsHashing) { + willNotWorkOnGpu("hashing arrays with structs is not supported") + } + } + override def convertToGpu(): GpuPartitioning = GpuHashPartitioning(childExprs.map(_.convertToGpu()), hp.numPartitions) }), @@ -3912,7 +3925,7 @@ object GpuOverrides extends Logging { .withPsNote(TypeEnum.STRUCT, "Round-robin partitioning is not supported for nested " + s"structs if ${SQLConf.SORT_BEFORE_REPARTITION.key} is true") .withPsNote( - Seq(TypeEnum.ARRAY, TypeEnum.MAP), + Seq(TypeEnum.MAP), "Round-robin partitioning is not supported if " + s"${SQLConf.SORT_BEFORE_REPARTITION.key} is true"), TypeSig.all), @@ -3974,10 +3987,12 @@ object GpuOverrides extends Logging { "The backend for hash based aggregations", ExecChecks( (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.MAP + TypeSig.ARRAY + TypeSig.STRUCT) + TypeSig.MAP + TypeSig.STRUCT + TypeSig.ARRAY) .nested() - .withPsNote(Seq(TypeEnum.ARRAY, TypeEnum.MAP), + .withPsNote(TypeEnum.MAP, "not allowed for grouping expressions") + .withPsNote(TypeEnum.ARRAY, + "not allowed for grouping expressions if containing Struct as child") .withPsNote(TypeEnum.STRUCT, "not allowed for grouping expressions if containing Array or Map as child"), TypeSig.all), diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala index 3a9c9595ab2..6eb222335cd 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala @@ -42,7 +42,7 @@ import org.apache.spark.sql.execution.{ExplainUtils, SortExec, SparkPlan} import org.apache.spark.sql.execution.aggregate.{BaseAggregateExec, HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} import org.apache.spark.sql.rapids.{CpuToGpuAggregateBufferConverter, CudfAggregate, GpuAggregateExpression, GpuToCpuAggregateBufferConverter} import org.apache.spark.sql.rapids.execution.{GpuShuffleMeta, TrampolineUtil} -import org.apache.spark.sql.types.{ArrayType, DataType, MapType} +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch object AggregateUtils { @@ -852,13 +852,27 @@ abstract class GpuBaseAggregateMeta[INPUT <: SparkPlan]( groupingExpressions ++ aggregateExpressions ++ aggregateAttributes ++ resultExpressions override def tagPlanForGpu(): Unit = { - // We don't support Arrays and Maps as GroupBy keys yet, even they are nested in Structs. So, + // We don't support Maps as GroupBy keys yet, even if they are nested in Structs. So, // we need to run recursive type check on the structs. - val arrayOrMapGroupings = agg.groupingExpressions.exists(e => + val mapGroupings = agg.groupingExpressions.exists(e => TrampolineUtil.dataTypeExistsRecursively(e.dataType, - dt => dt.isInstanceOf[ArrayType] || dt.isInstanceOf[MapType])) - if (arrayOrMapGroupings) { - willNotWorkOnGpu("ArrayTypes or MapTypes in grouping expressions are not supported") + dt => dt.isInstanceOf[MapType])) + if (mapGroupings) { + willNotWorkOnGpu("MapTypes in grouping expressions are not supported") + } + + // We support Arrays as grouping expression but not if the child is a struct. So we need to + // run recursive type check on the lists of structs + val arrayWithStructsGroupings = agg.groupingExpressions.exists(e => + TrampolineUtil.dataTypeExistsRecursively(e.dataType, + dt => dt match { + case ArrayType(_: StructType, _) => true + case _ => false + }) + ) + if (arrayWithStructsGroupings) { + willNotWorkOnGpu("ArrayTypes with Struct children in grouping expressions are not " + + "supported") } tagForReplaceMode() diff --git a/tools/src/main/resources/supportedExprs.csv b/tools/src/main/resources/supportedExprs.csv index cd37c792ccc..7b93812944c 100644 --- a/tools/src/main/resources/supportedExprs.csv +++ b/tools/src/main/resources/supportedExprs.csv @@ -263,8 +263,8 @@ IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA JsonToStructs,NS,`from_json`,This is disabled by default because parsing JSON from a column has a large number of issues and should be considered beta quality right now.,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA JsonToStructs,NS,`from_json`,This is disabled by default because parsing JSON from a column has a large number of issues and should be considered beta quality right now.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,NS,NA -KnownFloatingPointNormalized,S, ,None,project,input,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -KnownFloatingPointNormalized,S, ,None,project,result,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +KnownFloatingPointNormalized,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S +KnownFloatingPointNormalized,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S KnownNotNull,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,NS,S,S,PS,PS,PS,NS KnownNotNull,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,NS,S,S,PS,PS,PS,NS Lag,S,`lag`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS From 66450a3549d7cbb23799ec7be2f6f02b253efb85 Mon Sep 17 00:00:00 2001 From: Remzi Yang <59198230+HaoYang670@users.noreply.github.com> Date: Fri, 2 Sep 2022 09:09:12 +0800 Subject: [PATCH 077/190] Normalize nans in GpuSortArray (#6451) * normalize nans in GpuSortArray Signed-off-by: remzi <13716567376yh@gmail.com> * remove duplicated code Signed-off-by: remzi <13716567376yh@gmail.com> * fix nit Signed-off-by: remzi <13716567376yh@gmail.com> * add comments Signed-off-by: remzi <13716567376yh@gmail.com> * test asc and desc Signed-off-by: remzi <13716567376yh@gmail.com> * normalize nans in nested types column Signed-off-by: remzi <13716567376yh@gmail.com> Signed-off-by: remzi <13716567376yh@gmail.com> --- .../src/main/python/collection_ops_test.py | 23 +++++++++++++++++++ .../sql/rapids/collectionOperations.scala | 23 ++++++++++++++++--- 2 files changed, 43 insertions(+), 3 deletions(-) diff --git a/integration_tests/src/main/python/collection_ops_test.py b/integration_tests/src/main/python/collection_ops_test.py index 17b57156bf0..ea32e1eeef8 100644 --- a/integration_tests/src/main/python/collection_ops_test.py +++ b/integration_tests/src/main/python/collection_ops_test.py @@ -149,6 +149,29 @@ def test_sort_array_lit(data_gen): f.sort_array(f.lit(array_lit), True), f.sort_array(f.lit(array_lit), False))) + +def test_sort_array_normalize_nans(): + """ + When the average length of array is > 100, + and there are `-Nan`s in the data, the sorting order + of `Nan` is inconsistent in cuDF (https://github.com/rapidsai/cudf/issues/11630). + `GpuSortArray` fixes the inconsistency by normalizing `Nan`s. + """ + bytes1 = struct.pack('L', 0x7ff83cec2c05b870) + bytes2 = struct.pack('L', 0xfff5101d3f1cd31b) + bytes3 = struct.pack('L', 0x7c22453f18c407a8) + nan1 = struct.unpack('d', bytes1)[0] + nan2 = struct.unpack('d', bytes2)[0] + other = struct.unpack('d', bytes3)[0] + + data1 = [([nan2] + [other for _ in range(256)] + [nan1],)] + # array of struct + data2 = [([(nan2, nan1)] + [(other, nan2) for _ in range(256)] + [(nan1, nan2)],)] + for data in [data1, data2]: + assert_gpu_and_cpu_are_equal_collect( + lambda spark: spark.createDataFrame(data).selectExpr('sort_array(_1, true)', 'sort_array(_1, false)') + ) + # For functionality test, the sequence length in each row should be limited, # to avoid the exception as below, # "Too long sequence: 2147483745. Should be <= 2147483632" diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala index 92cf18a9b79..4a4af59d78b 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala @@ -407,13 +407,30 @@ case class GpuSortArray(base: Expression, ascendingOrder: Expression) override def doColumnar(lhs: GpuColumnVector, rhs: GpuScalar): cudf.ColumnVector = { val isDescending = isDescendingOrder(rhs) - lhs.getBase.listSortRows(isDescending, true) + val base = lhs.getBase() + withResource(base.getChildColumnView(0)) {child => + withResource(child.copyToColumnVector()) {child => + // When the average length of array is > 100 + // and there are `-Nan`s in the data, the sort ordering + // of `Nan`s is inconsistent. So we need to normalize the data + // before sorting. This workaround can be removed after + // solving https://github.com/rapidsai/cudf/issues/11630 + val normalizedChild = ColumnCastUtil.deepTransform(child) { + case cv if cv.getType == cudf.DType.FLOAT32 || cv.getType == cudf.DType.FLOAT64 => + cv.normalizeNANsAndZeros() + } + withResource(normalizedChild) {normalizedChild => + withResource(base.replaceListChild(normalizedChild)) {normalizedList => + normalizedList.listSortRows(isDescending, true) + } + } + } + } } override def doColumnar(numRows: Int, lhs: GpuScalar, rhs: GpuScalar): cudf.ColumnVector = { - val isDescending = isDescendingOrder(rhs) withResource(GpuColumnVector.from(lhs, numRows, left.dataType)) { cv => - cv.getBase.listSortRows(isDescending, true) + doColumnar(cv, rhs) } } From 6bd071c2e55753a604b8aee7f7a8f70f2b784c23 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Fri, 2 Sep 2022 09:51:11 -0500 Subject: [PATCH 078/190] Fix leak in interval divide (#6482) Signed-off-by: Robert (Bobby) Evans --- .../apache/spark/sql/rapids/shims/intervalExpressions.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sql-plugin/src/main/330+/scala/org/apache/spark/sql/rapids/shims/intervalExpressions.scala b/sql-plugin/src/main/330+/scala/org/apache/spark/sql/rapids/shims/intervalExpressions.scala index 785232f4f34..97f3ef3f3ca 100644 --- a/sql-plugin/src/main/330+/scala/org/apache/spark/sql/rapids/shims/intervalExpressions.scala +++ b/sql-plugin/src/main/330+/scala/org/apache/spark/sql/rapids/shims/intervalExpressions.scala @@ -290,7 +290,10 @@ object IntervalUtils extends Arm { case pCv: ColumnVector => pCv.castTo(dT) case pS: Scalar => Scalar.fromDecimal(-1, new BigInteger((getLong(pS) * 10L).toString)) } - withResource(leftDecimal.div(q, dT)) { t => + val t = withResource(leftDecimal) { leftDecimal => + leftDecimal.div(q, dT) + } + withResource(t) { t => t.round(RoundMode.HALF_UP) } } From d413a9b048902bf9d371f197d11eda02c6758282 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Fri, 2 Sep 2022 09:25:56 -0700 Subject: [PATCH 079/190] remove KnownFloatingPointNormalized from allow_non_gpu as it's not needed (#6479) Signed-off-by: Raza Jafri Signed-off-by: Raza Jafri Co-authored-by: Raza Jafri --- integration_tests/src/main/python/hash_aggregate_test.py | 9 ++++----- integration_tests/src/main/python/join_test.py | 4 ++-- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index 752a461f58f..7b4ba86daaa 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -254,7 +254,7 @@ def get_params(init_list, marked_params=[]): 'HashAggregateExec', 'AggregateExpression', 'UnscaledValue', 'MakeDecimal', 'AttributeReference', 'Alias', 'Sum', 'Count', 'Max', 'Min', 'Average', 'Cast', 'StddevPop', 'StddevSamp', 'VariancePop', 'VarianceSamp', - 'KnownFloatingPointNormalized', 'NormalizeNaNAndZero', 'GreaterThan', 'Literal', 'If', + 'NormalizeNaNAndZero', 'GreaterThan', 'Literal', 'If', 'EqualTo', 'First', 'SortAggregateExec', 'Coalesce', 'IsNull', 'EqualNullSafe', 'PivotFirst', 'GetArrayItem', 'ShuffleExchangeExec', 'HashPartitioning') @@ -433,7 +433,7 @@ def test_hash_grpby_avg(data_gen, conf): @pytest.mark.allow_non_gpu( 'HashAggregateExec', 'AggregateExpression', 'AttributeReference', 'Alias', 'Sum', 'Count', 'Max', 'Min', 'Average', 'Cast', - 'KnownFloatingPointNormalized', 'NormalizeNaNAndZero', 'GreaterThan', 'Literal', 'If', + 'NormalizeNaNAndZero', 'GreaterThan', 'Literal', 'If', 'EqualTo', 'First', 'SortAggregateExec') @pytest.mark.parametrize('data_gen', [ StructGen(children=[('a', int_gen), ('b', int_gen)],nullable=False, @@ -546,8 +546,7 @@ def test_hash_reduction_pivot_with_nans(data_gen, conf): @approximate_float @ignore_order(local=True) @allow_non_gpu('HashAggregateExec', 'PivotFirst', 'AggregateExpression', 'Alias', 'GetArrayItem', - 'Literal', 'ShuffleExchangeExec', 'HashPartitioning', 'KnownFloatingPointNormalized', - 'NormalizeNaNAndZero') + 'Literal', 'ShuffleExchangeExec', 'HashPartitioning', 'NormalizeNaNAndZero') @incompat @pytest.mark.parametrize('data_gen', [_grpkey_floats_with_nulls_and_nans], ids=idfn) def test_hash_pivot_groupby_duplicates_fallback(data_gen): @@ -1788,7 +1787,7 @@ def test_groupby_std_variance_nulls(data_gen, conf, ansi_enabled): @ignore_order(local=True) @approximate_float -@allow_non_gpu('KnownFloatingPointNormalized', 'NormalizeNaNAndZero', +@allow_non_gpu('NormalizeNaNAndZero', 'HashAggregateExec', 'SortAggregateExec', 'Cast', 'ShuffleExchangeExec', 'HashPartitioning', 'SortExec', diff --git a/integration_tests/src/main/python/join_test.py b/integration_tests/src/main/python/join_test.py index b64eab99d41..3c7ee538138 100644 --- a/integration_tests/src/main/python/join_test.py +++ b/integration_tests/src/main/python/join_test.py @@ -191,7 +191,7 @@ def do_join(spark): # For floating point values the normalization is done using a higher order function. We could probably work around this # for now it falls back to the CPU -@allow_non_gpu('SortMergeJoinExec', 'SortExec', 'KnownFloatingPointNormalized', 'ArrayTransform', 'LambdaFunction', +@allow_non_gpu('SortMergeJoinExec', 'SortExec', 'ArrayTransform', 'LambdaFunction', 'NamedLambdaVariable', 'NormalizeNaNAndZero', 'ShuffleExchangeExec', 'HashPartitioning') @ignore_order(local=True) @pytest.mark.parametrize('data_gen', single_level_array_gens + [binary_gen], ids=idfn) @@ -728,7 +728,7 @@ def do_join(spark): return left.join(right, left.a == right.r_a, join_type) assert_gpu_and_cpu_are_equal_collect(do_join, conf=_sortmerge_join_conf) -@allow_non_gpu('SortMergeJoinExec', 'SortExec', 'KnownFloatingPointNormalized', 'NormalizeNaNAndZero', 'CreateNamedStruct', +@allow_non_gpu('SortMergeJoinExec', 'SortExec', 'NormalizeNaNAndZero', 'CreateNamedStruct', 'GetStructField', 'Literal', 'If', 'IsNull', 'ShuffleExchangeExec', 'HashPartitioning') @ignore_order(local=True) @pytest.mark.parametrize('data_gen', struct_gens, ids=idfn) From a846fcc17698c5da27f193450ff925fda12669a8 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Fri, 2 Sep 2022 09:50:53 -0700 Subject: [PATCH 080/190] Fix DOP calculation for xdist (#6483) This PR fixes #6467, contributes to #6403 Improve TEST_PARALLEL formula by accounting for - number of executor JVMs - the fact that xdist master no longer starts a dummy Spark app Use UCX shuffle test as a POC Signed-off-by: Gera Shegalov --- integration_tests/run_pyspark_from_build.sh | 20 +++++++++++++------- jenkins/spark-premerge-build.sh | 12 +++++------- 2 files changed, 18 insertions(+), 14 deletions(-) diff --git a/integration_tests/run_pyspark_from_build.sh b/integration_tests/run_pyspark_from_build.sh index d3b9adacb57..b70ea80e463 100755 --- a/integration_tests/run_pyspark_from_build.sh +++ b/integration_tests/run_pyspark_from_build.sh @@ -127,14 +127,20 @@ else HOST_MEM_PARALLEL=`cat /proc/meminfo | grep MemAvailable | awk '{print int($2 / (5 * 1024))}'` TMP_PARALLEL=$(( $GPU_MEM_PARALLEL > $CPU_CORES ? $CPU_CORES : $GPU_MEM_PARALLEL )) TMP_PARALLEL=$(( $TMP_PARALLEL > $HOST_MEM_PARALLEL ? $HOST_MEM_PARALLEL : $TMP_PARALLEL )) - if [[ $TMP_PARALLEL -gt 1 ]]; - then - # We subtract 1 from the parallel number because xdist launches a process to - # control and monitor the other processes. It takes up one available parallel - # slot, even if it is not truly using all of the resources we give it. - TEST_PARALLEL=$(( $TMP_PARALLEL - 1 )) - else + + # Account for intra-Spark parallelism + numGpuJVM=1 + if [[ "$NUM_LOCAL_EXECS" != "" ]]; then + numGpuJVM=$NUM_LOCAL_EXECS + elif [[ "$PYSP_TEST_spark_cores_max" != "" && "$PYSP_TEST_spark_executor_cores" != "" ]]; then + numGpuJVM=$(( $PYSP_TEST_spark_cores_max / $PYSP_TEST_spark_executor_cores )) + fi + TMP_PARALLEL=$(( $TMP_PARALLEL / $numGpuJVM )) + + if (( $TMP_PARALLEL <= 1 )); then TEST_PARALLEL=1 + else + TEST_PARALLEL=$TMP_PARALLEL fi echo "AUTO DETECTED PARALLELISM OF $TEST_PARALLEL" diff --git a/jenkins/spark-premerge-build.sh b/jenkins/spark-premerge-build.sh index 228fb6f51fa..60c645cc232 100755 --- a/jenkins/spark-premerge-build.sh +++ b/jenkins/spark-premerge-build.sh @@ -98,26 +98,24 @@ rapids_shuffle_smoke_test() { $SPARK_HOME/sbin/spark-daemon.sh start org.apache.spark.deploy.worker.Worker 1 $SPARK_MASTER invoke_shuffle_integration_test() { - SPECIFIC_SHUFFLE_FLAGS=$1 PYSP_TEST_spark_master=$SPARK_MASTER \ - TEST_PARALLEL=0 \ PYSP_TEST_spark_cores_max=2 \ PYSP_TEST_spark_executor_cores=1 \ PYSP_TEST_spark_shuffle_manager=com.nvidia.spark.rapids.$SHUFFLE_SPARK_SHIM.RapidsShuffleManager \ PYSP_TEST_spark_rapids_memory_gpu_minAllocFraction=0 \ PYSP_TEST_spark_rapids_memory_gpu_maxAllocFraction=0.1 \ PYSP_TEST_spark_rapids_memory_gpu_allocFraction=0.1 \ - SPARK_SUBMIT_FLAGS=$SPECIFIC_SHUFFLE_FLAGS \ ./integration_tests/run_pyspark_from_build.sh -m shuffle_test } # using UCX shuffle - invoke_shuffle_integration_test "--conf spark.executorEnv.UCX_ERROR_SIGNALS=" + PYSP_TEST_spark_executorEnv_UCX_ERROR_SIGNALS="" \ + invoke_shuffle_integration_test # using MULTITHREADED shuffle - invoke_shuffle_integration_test "\ - --conf spark.rapids.shuffle.mode=MULTITHREADED \ - --conf spark.rapids.shuffle.multiThreaded.writer.threads=2" + PYSP_TEST_spark_rapids_shuffle_mode=MULTITHREADED \ + PYSP_TEST_spark_rapids_shuffle_multiThreaded_writer_threads=2 \ + invoke_shuffle_integration_test $SPARK_HOME/sbin/spark-daemon.sh stop org.apache.spark.deploy.worker.Worker 1 $SPARK_HOME/sbin/stop-master.sh From e8687f4e091e6c1a585a515912a943fe3bac305c Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 2 Sep 2022 14:19:34 -0500 Subject: [PATCH 081/190] Fix intermittent failure on test_cast_float_to_timestamp_side_effect (#6492) * ignore order on test_cast_float_to_timestamp_side_effect Signed-off-by: Thomas Graves * Change to set parallelism to 1 instead of repartition Signed-off-by: Thomas Graves --- integration_tests/src/main/python/cast_test.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/integration_tests/src/main/python/cast_test.py b/integration_tests/src/main/python/cast_test.py index 8c5326c44ec..e5de7acac9f 100644 --- a/integration_tests/src/main/python/cast_test.py +++ b/integration_tests/src/main/python/cast_test.py @@ -398,8 +398,9 @@ def fun(spark): @pytest.mark.skipif(is_before_spark_330(), reason='330+ throws exception in ANSI mode') def test_cast_float_to_timestamp_side_effect(): def getDf(spark): - return spark.createDataFrame([(True, float(LONG_MAX) + 100), (False, float(1))], - "c_b boolean, c_f float").repartition(1) + data = [(True, float(LONG_MAX) + 100), (False, float(1))] + distData = spark.sparkContext.parallelize(data, 1) + return spark.createDataFrame(distData, "c_b boolean, c_f float") assert_gpu_and_cpu_are_equal_collect( lambda spark: getDf(spark).selectExpr("if(c_b, cast(0 as timestamp), cast(c_f as timestamp))"), conf=ansi_enabled_conf) From 0bdaf6a8001f61f2cfb162403d9d1dae9af616bc Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Fri, 2 Sep 2022 17:13:47 -0500 Subject: [PATCH 082/190] Fixes excessive ShuffleBlockId object creation due to missing map index bounds [databricks] (#6484) * Fixes excessive ShuffleBlockId object due to missing map index bounds Signed-off-by: Alessandro Bellina * Removes extra filtering * Fix extra line Signed-off-by: Alessandro Bellina --- .../rapids/shims/ShuffledBatchRDDUtil.scala | 39 +++++++++------ .../rapids/shims/ShuffledBatchRDDUtil.scala | 50 ++++++++++++------- 2 files changed, 56 insertions(+), 33 deletions(-) diff --git a/sql-plugin/src/main/311until320-all/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala b/sql-plugin/src/main/311until320-all/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala index 06aeb631bbc..7934c296cb3 100644 --- a/sql-plugin/src/main/311until320-all/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala +++ b/sql-plugin/src/main/311until320-all/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.execution.{CoalescedPartitionSpec, PartialMapperPart import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter import org.apache.spark.sql.rapids.execution.ShuffledBatchRDDPartition import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.storage.{BlockId, BlockManagerId} /** * Some APIs for the ShuffledBatchRDD are only accessible from org.apache.spark... @@ -49,6 +50,13 @@ object ShuffledBatchRDDUtil { } } + private def getPartitionSize( + blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])]): Long = { + blocksByAddress.flatMap { case (_, blockInfos) => + blockInfos.map { case (_, size, _) => size } + }.sum + } + def getReaderAndPartSize( split: Partition, context: TaskContext, @@ -64,10 +72,12 @@ object ShuffledBatchRDDUtil { context, sqlMetricsReporter) val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId( - dependency.shuffleHandle.shuffleId, 0, Int.MaxValue, startReducerIndex, endReducerIndex) - val partitionSize = blocksByAddress.flatMap(_._2).map(_._2).sum - (reader, partitionSize) - + dependency.shuffleHandle.shuffleId, + 0, + Int.MaxValue, + startReducerIndex, + endReducerIndex) + (reader, getPartitionSize(blocksByAddress)) case PartialReducerPartitionSpec(reducerIndex, startMapIndex, endMapIndex, _) => val reader = SparkEnv.get.shuffleManager.getReader( dependency.shuffleHandle, @@ -78,12 +88,12 @@ object ShuffledBatchRDDUtil { context, sqlMetricsReporter) val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId( - dependency.shuffleHandle.shuffleId, 0, Int.MaxValue, reducerIndex, + dependency.shuffleHandle.shuffleId, + startMapIndex, + endMapIndex, + reducerIndex, reducerIndex + 1) - val partitionSize = blocksByAddress.flatMap(_._2) - .filter(tuple => tuple._3 >= startMapIndex && tuple._3 < endMapIndex) - .map(_._2).sum - (reader, partitionSize) + (reader, getPartitionSize(blocksByAddress)) case PartialMapperPartitionSpec(mapIndex, startReducerIndex, endReducerIndex) => val reader = SparkEnv.get.shuffleManager.getReader( dependency.shuffleHandle, @@ -94,11 +104,12 @@ object ShuffledBatchRDDUtil { context, sqlMetricsReporter) val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId( - dependency.shuffleHandle.shuffleId, 0, Int.MaxValue, startReducerIndex, endReducerIndex) - val partitionSize = blocksByAddress.flatMap(_._2) - .filter(_._3 == mapIndex) - .map(_._2).sum - (reader, partitionSize) + dependency.shuffleHandle.shuffleId, + mapIndex, + mapIndex + 1, + startReducerIndex, + endReducerIndex) + (reader, getPartitionSize(blocksByAddress)) } } } diff --git a/sql-plugin/src/main/320+/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala b/sql-plugin/src/main/320+/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala index e25557bd362..cd895e98120 100644 --- a/sql-plugin/src/main/320+/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala +++ b/sql-plugin/src/main/320+/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.execution.{CoalescedMapperPartitionSpec, CoalescedPa import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter import org.apache.spark.sql.rapids.execution.ShuffledBatchRDDPartition import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.storage.{BlockId, BlockManagerId} /** * Some APIs for the ShuffledBatchRDD are only accessible from org.apache.spark... @@ -52,6 +53,13 @@ object ShuffledBatchRDDUtil { } } + private def getPartitionSize( + blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])]): Long = { + blocksByAddress.flatMap { case (_, blockInfos) => + blockInfos.map { case (_, size, _) => size } + }.sum + } + def getReaderAndPartSize( split: Partition, context: TaskContext, @@ -67,10 +75,12 @@ object ShuffledBatchRDDUtil { context, sqlMetricsReporter) val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId( - dependency.shuffleHandle.shuffleId, 0, Int.MaxValue, startReducerIndex, endReducerIndex) - val partitionSize = blocksByAddress.flatMap(_._2).map(_._2).sum - (reader, partitionSize) - + dependency.shuffleHandle.shuffleId, + 0, + Int.MaxValue, + startReducerIndex, + endReducerIndex) + (reader, getPartitionSize(blocksByAddress)) case PartialReducerPartitionSpec(reducerIndex, startMapIndex, endMapIndex, _) => val reader = SparkEnv.get.shuffleManager.getReader( dependency.shuffleHandle, @@ -81,12 +91,12 @@ object ShuffledBatchRDDUtil { context, sqlMetricsReporter) val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId( - dependency.shuffleHandle.shuffleId, 0, Int.MaxValue, reducerIndex, + dependency.shuffleHandle.shuffleId, + startMapIndex, + endMapIndex, + reducerIndex, reducerIndex + 1) - val partitionSize = blocksByAddress.flatMap(_._2) - .filter(tuple => tuple._3 >= startMapIndex && tuple._3 < endMapIndex) - .map(_._2).sum - (reader, partitionSize) + (reader, getPartitionSize(blocksByAddress)) case PartialMapperPartitionSpec(mapIndex, startReducerIndex, endReducerIndex) => val reader = SparkEnv.get.shuffleManager.getReader( dependency.shuffleHandle, @@ -97,11 +107,12 @@ object ShuffledBatchRDDUtil { context, sqlMetricsReporter) val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId( - dependency.shuffleHandle.shuffleId, 0, Int.MaxValue, startReducerIndex, endReducerIndex) - val partitionSize = blocksByAddress.flatMap(_._2) - .filter(_._3 == mapIndex) - .map(_._2).sum - (reader, partitionSize) + dependency.shuffleHandle.shuffleId, + mapIndex, + mapIndex + 1, + startReducerIndex, + endReducerIndex) + (reader, getPartitionSize(blocksByAddress)) case CoalescedMapperPartitionSpec(startMapIndex, endMapIndex, numReducers) => val reader = SparkEnv.get.shuffleManager.getReader( dependency.shuffleHandle, @@ -112,11 +123,12 @@ object ShuffledBatchRDDUtil { context, sqlMetricsReporter) val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId( - dependency.shuffleHandle.shuffleId, startMapIndex, endMapIndex, 0, numReducers) - val partitionSize = blocksByAddress.flatMap(_._2) - .filter(tuple => tuple._3 >= startMapIndex && tuple._3 < endMapIndex) - .map(_._2).sum - (reader, partitionSize) + dependency.shuffleHandle.shuffleId, + startMapIndex, + endMapIndex, + 0, + numReducers) + (reader, getPartitionSize(blocksByAddress)) } } } From 542ecd15b6fc4f2b7a96745ac6e36e588d91ff64 Mon Sep 17 00:00:00 2001 From: Remzi Yang <59198230+HaoYang670@users.noreply.github.com> Date: Sat, 3 Sep 2022 09:49:22 +0800 Subject: [PATCH 083/190] Remove the `hasNans` config from `GpuCollectSet` (#6486) * remove hasnans config Signed-off-by: remzi <13716567376yh@gmail.com> * upadte test cases that support nan but not cover nan Signed-off-by: remzi <13716567376yh@gmail.com> * update data_gen Signed-off-by: remzi <13716567376yh@gmail.com> Signed-off-by: remzi <13716567376yh@gmail.com> --- docs/supported_ops.md | 12 ++++++------ .../src/main/python/hash_aggregate_test.py | 14 +++++++------- .../com/nvidia/spark/rapids/GpuOverrides.scala | 13 +++---------- 3 files changed, 16 insertions(+), 23 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index cdd8f6f1105..d2c8b5ee370 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -15872,9 +15872,9 @@ are limited. S NS NS -PS
Support for arrays of arrays of floats/doubles requires spark.rapids.sql.hasNans to be set to false;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
NS -PS
Support for structs containing float/double array columns requires spark.rapids.sql.hasNans to be set to false;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
NS @@ -15915,9 +15915,9 @@ are limited. S NS NS -PS
Support for arrays of arrays of floats/doubles requires spark.rapids.sql.hasNans to be set to false;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
NS -PS
Support for structs containing float/double array columns requires spark.rapids.sql.hasNans to be set to false;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
NS @@ -15958,9 +15958,9 @@ are limited. S NS NS -PS
Support for arrays of arrays of floats/doubles requires spark.rapids.sql.hasNans to be set to false;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
NS -PS
Support for structs containing float/double array columns requires spark.rapids.sql.hasNans to be set to false;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
NS diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index 7b4ba86daaa..f8901449642 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -1094,12 +1094,6 @@ def test_count_distinct_with_nan_floats(data_gen): # REDUCTIONS -non_nan_all_basic_gens = [byte_gen, short_gen, int_gen, long_gen, - # nans and -0.0 cannot work because of nan support in min/max, -0.0 == 0.0 in cudf for distinct and - # Spark fixed ordering of 0.0 and -0.0 in Spark 3.1 in the ordering - FloatGen(no_nans=True, special_cases=[]), DoubleGen(no_nans=True, special_cases=[]), - string_gen, boolean_gen, date_gen, timestamp_gen] - _nested_gens = array_gens_sample + struct_gens_sample + map_gens_sample @pytest.mark.parametrize('data_gen', decimal_gens, ids=idfn) @@ -1176,11 +1170,17 @@ def test_collect_list_reductions(data_gen): lambda spark: unary_op_df(spark, data_gen).coalesce(1).selectExpr('collect_list(a)'), conf=_no_nans_float_conf) +_no_neg_zero_all_basic_gens = [byte_gen, short_gen, int_gen, long_gen, + # -0.0 cannot work because of -0.0 == 0.0 in cudf for distinct and + # Spark fixed ordering of 0.0 and -0.0 in Spark 3.1 in the ordering + FloatGen(special_cases=[FLOAT_MIN, FLOAT_MAX, 0.0, 1.0, -1.0]), DoubleGen(special_cases=[]), + string_gen, boolean_gen, date_gen, timestamp_gen] + _struct_only_nested_gens = [all_basic_struct_gen, StructGen([['child0', byte_gen], ['child1', all_basic_struct_gen]]), StructGen([])] @pytest.mark.parametrize('data_gen', - non_nan_all_basic_gens + decimal_gens + _struct_only_nested_gens, + _no_neg_zero_all_basic_gens + decimal_gens + _struct_only_nested_gens, ids=idfn) def test_collect_set_reductions(data_gen): assert_gpu_and_cpu_are_equal_collect( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index cf33be44905..dde10c5834a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -3383,11 +3383,9 @@ object GpuOverrides extends Logging { TypeSig.ARRAY.nested(TypeSig.all), Seq(ParamCheck("input", (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + - TypeSig.NULL + - TypeSig.STRUCT.withPsNote(TypeEnum.STRUCT, "Support for structs containing " + - s"float/double array columns requires ${RapidsConf.HAS_NANS} to be set to false") + - TypeSig.ARRAY.withPsNote(TypeEnum.ARRAY, "Support for arrays of arrays of " + - s"floats/doubles requires ${RapidsConf.HAS_NANS} to be set to false")).nested(), + TypeSig.NULL + + TypeSig.STRUCT + + TypeSig.ARRAY).nested(), TypeSig.all))), (c, conf, p, r) => new TypedImperativeAggExprMeta[CollectSet](c, conf, p, r) { @@ -3406,11 +3404,6 @@ object GpuOverrides extends Logging { } } - override def tagAggForGpu(): Unit = { - if (isNestedArrayType(c.child.dataType)) { - checkAndTagFloatNanAgg("CollectSet", c.child.dataType, conf, this) - } - } override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = GpuCollectSet(childExprs.head, c.mutableAggBufferOffset, c.inputAggBufferOffset) From 9567948365d1219b61ceb01336aa0a4d86da7963 Mon Sep 17 00:00:00 2001 From: sinkinben Date: Tue, 6 Sep 2022 09:29:15 +0800 Subject: [PATCH 084/190] Fix: orc_cast_test fails on CDH [databricks] (#6490) The spark-321cdh assumes the integers in columnar vector as milliseconds, added a new shim to fix it. --- .../spark/rapids/shims/OrcCastingShims.scala | 48 +++----------- .../spark/rapids/shims/OrcCastingShims.scala | 29 +++++++++ .../spark/rapids/shims/OrcCastingShims.scala | 62 ------------------- .../spark/rapids/shims/OrcCastingShims.scala | 29 +++++++++ .../com/nvidia/spark/rapids/GpuOrcScan.scala | 55 +++++++++++++++- 5 files changed, 120 insertions(+), 103 deletions(-) create mode 100644 sql-plugin/src/main/320+-noncdh/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala delete mode 100644 sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala create mode 100644 sql-plugin/src/main/321cdh/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala diff --git a/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala b/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala index fe914811df2..edf8bd12c2c 100644 --- a/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala +++ b/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala @@ -16,47 +16,15 @@ package com.nvidia.spark.rapids.shims -import ai.rapids.cudf.{ColumnView, DType, Scalar} -import com.nvidia.spark.rapids.GpuOrcScan.{testLongMultiplicationOverflow, withResource} +import ai.rapids.cudf.{ColumnView, DType} +import com.nvidia.spark.rapids.GpuOrcScan + object OrcCastingShims { - /** - * Cast ColumnView of integer types to timestamp (in milliseconds). - * @param col The column view of integer types. - * @param fromType BOOL8, INT8/16/32/64 - * @return A new timestamp columnar vector. - */ - def castIntegerToTimestamp(col: ColumnView, fromType: DType): ColumnView = { - fromType match { - case DType.BOOL8 | DType.INT8 | DType.INT16 | DType.INT32 => - // From spark311 until spark314 (not include it), spark consider the integers as - // milli-seconds. - // cuDF requires casting to Long first, then we can cast Long to Timestamp(in microseconds) - // In CPU code of ORC casting, its conversion is 'integer -> milliseconds -> microseconds' - withResource(col.castTo(DType.INT64)) { longs => - withResource(Scalar.fromLong(1000L)) { thousand => - withResource(longs.mul(thousand)) { milliSeconds => - milliSeconds.castTo(DType.TIMESTAMP_MICROSECONDS) - } - } - } - case DType.INT64 => - // We need overflow checking here, since max value of INT64 is about 9 * 1e18, and convert - // INT64 to milliseconds(also a INT64 actually), we need multiply 1000, it may cause long - // integer-overflow. - // If these two 'testLongMultiplicationOverflow' throw no exception, it means no - // Long-overflow when casting 'col' to TIMESTAMP_MICROSECONDS. - if (col.max() != null) { - testLongMultiplicationOverflow(col.max().getLong, 1000L) - } - if (col.min() != null) { - testLongMultiplicationOverflow(col.min().getLong, 1000L) - } - withResource(Scalar.fromLong(1000L)) { thousand => - withResource(col.mul(thousand)) { milliSeconds => - milliSeconds.castTo(DType.TIMESTAMP_MICROSECONDS) - } - } - } + + def castIntegerToTimestamp(col: ColumnView, colType: DType): ColumnView = { + // For 311 <= spark < 320 (including 311, 312, 313, 314), they consider the integer as + // milliseconds. + GpuOrcScan.castIntegersToTimestamp(col, colType, DType.TIMESTAMP_MILLISECONDS) } } diff --git a/sql-plugin/src/main/320+-noncdh/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala b/sql-plugin/src/main/320+-noncdh/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala new file mode 100644 index 00000000000..abc427e4272 --- /dev/null +++ b/sql-plugin/src/main/320+-noncdh/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala @@ -0,0 +1,29 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import ai.rapids.cudf.{ColumnView, DType} +import com.nvidia.spark.rapids.GpuOrcScan + + +object OrcCastingShims { + + def castIntegerToTimestamp(col: ColumnView, colType: DType): ColumnView = { + // For spark >= 320 (except spark-321-cdh), they consider the integers in `col` as seconds + GpuOrcScan.castIntegersToTimestamp(col, colType, DType.TIMESTAMP_SECONDS) + } +} diff --git a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala deleted file mode 100644 index b793a683e04..00000000000 --- a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Copyright (c) 2022, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids.shims - -import ai.rapids.cudf.{ColumnView, DType, Scalar} -import com.nvidia.spark.rapids.GpuOrcScan.{testLongMultiplicationOverflow, withResource} - -object OrcCastingShims { - /** - * Cast ColumnView of integer types to timestamp (in milliseconds). - * @param col The column view of integer types. - * @param fromType BOOL8, INT8/16/32/64 - * @return A new timestamp columnar vector. - */ - def castIntegerToTimestamp(col: ColumnView, fromType: DType): ColumnView = { - fromType match { - case DType.BOOL8 | DType.INT8 | DType.INT16 | DType.INT32 => - // From spark320, spark consider the integers as seconds. - withResource(col.castTo(DType.INT64)) { longs => - // In CPU, ORC assumes the integer value is in seconds, and returns timestamp in - // micro seconds, so we need to multiply 1e6 here. - withResource(Scalar.fromLong(1000000L)) { value => - withResource(longs.mul(value)) { microSeconds => - microSeconds.castTo(DType.TIMESTAMP_MICROSECONDS) - } - } - } - - case DType.INT64 => - // In CPU code of ORC casting, its conversion is 'integer -> milliseconds -> microseconds' - withResource(Scalar.fromLong(1000L)) { thousand => - withResource(col.mul(thousand)) { milliSeconds => - // We need to check long-overflow here. If milliseconds can not convert to - // micorseconds, then testLongMultiplicationOverflow will throw exception. - if (milliSeconds.max() != null) { - testLongMultiplicationOverflow(milliSeconds.max().getLong, 1000L) - } - if (milliSeconds.min() != null) { - testLongMultiplicationOverflow(milliSeconds.min().getLong, 1000L) - } - withResource(milliSeconds.mul(thousand)) { microSeconds => - microSeconds.castTo(DType.TIMESTAMP_MICROSECONDS) - } - } - } - } - } -} diff --git a/sql-plugin/src/main/321cdh/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala b/sql-plugin/src/main/321cdh/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala new file mode 100644 index 00000000000..6b9874750ec --- /dev/null +++ b/sql-plugin/src/main/321cdh/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala @@ -0,0 +1,29 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import ai.rapids.cudf.{ColumnView, DType} +import com.nvidia.spark.rapids.GpuOrcScan + + +object OrcCastingShims { + + def castIntegerToTimestamp(col: ColumnView, colType: DType): ColumnView = { + // For spark-321cdh, it consider the integers in `col` as milliseconds + GpuOrcScan.castIntegersToTimestamp(col, colType, DType.TIMESTAMP_MILLISECONDS) + } +} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index fa921e517be..781009f1121 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -50,7 +50,7 @@ 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.expressions.Expression -import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeConstants} import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.execution.datasources.{PartitionedFile, PartitioningAwareFileIndex} @@ -280,6 +280,59 @@ object GpuOrcScan extends Arm { def testLongMultiplicationOverflow(a: Long, b: Long) = { Math.multiplyExact(a, b) } + + + /** + * Convert the integer vector into timestamp(microseconds) vector. + * @param col The integer columnar vector. + * @param colType Specific integer type, it should be BOOL/INT8/INT16/INT32/INT64. + * @param timeUnit It should be one of {DType.TIMESTAMP_SECONDS, DType.TIMESTAMP_MILLISECONDS}. + * If timeUnit == SECONDS, then we consider the integers as seconds. + * If timeUnit == MILLISECONDS, then we consider the integers as milliseconds. + * This parameter is determined by the shims. + * @return A timestamp vector. + */ + def castIntegersToTimestamp(col: ColumnView, colType: DType, + timeUnit: DType): ColumnVector = { + assert(colType == DType.BOOL8 || colType == DType.INT8 || colType == DType.INT16 + || colType == DType.INT32 || colType == DType.INT64) + assert(timeUnit == DType.TIMESTAMP_SECONDS || timeUnit == DType.TIMESTAMP_MILLISECONDS) + + colType match { + case DType.BOOL8 | DType.INT8 | DType.INT16 | DType.INT32 => + // cuDF requires casting to Long first, then we can cast Long to Timestamp(in microseconds) + withResource(col.castTo(DType.INT64)) { longs => + // bitCastTo will re-interpret the long values as 'timeUnit', and it will zero-copy cast + // between types with the same underlying length. + withResource(longs.bitCastTo(timeUnit)) { timeView => + timeView.castTo(DType.TIMESTAMP_MICROSECONDS) + } + } + case DType.INT64 => + // In CPU code of ORC casting, if the integers are consider as seconds, then the conversion + // is 'integer -> milliseconds -> microseconds', and it checks the long-overflow when + // casting 'milliseconds -> microseconds', here we follow it. + val milliseconds = withResource(col.bitCastTo(timeUnit)) { timeView => + timeView.castTo(DType.TIMESTAMP_MILLISECONDS) + } + withResource(milliseconds) { _ => + // Check long-multiplication overflow + withResource(milliseconds.max()) { maxValue => + // If the elements in 'milliseconds' are all nulls, then 'maxValue' and 'minValue' will + // be null. We should check their validity. + if (maxValue.isValid) { + testLongMultiplicationOverflow(maxValue.getLong, DateTimeConstants.MICROS_PER_MILLIS) + } + } + withResource(milliseconds.min()) { minValue => + if (minValue.isValid) { + testLongMultiplicationOverflow(minValue.getLong, DateTimeConstants.MICROS_PER_MILLIS) + } + } + milliseconds.castTo(DType.TIMESTAMP_MICROSECONDS) + } + } + } } /** From 0cd0c10e95483dee7fe75420cc16738b97f506db Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Tue, 6 Sep 2022 08:33:03 -0500 Subject: [PATCH 085/190] Fix non-deterministic overflows in test_hash_grpby_sum_full_decimal (#6499) Signed-off-by: Jason Lowe Signed-off-by: Jason Lowe --- docs/compatibility.md | 18 +++++------- integration_tests/src/main/python/data_gen.py | 11 +++++--- .../src/main/python/hash_aggregate_test.py | 28 ++++++++++--------- 3 files changed, 29 insertions(+), 28 deletions(-) diff --git a/docs/compatibility.md b/docs/compatibility.md index 10cfcfd0783..f7a0fe21943 100644 --- a/docs/compatibility.md +++ b/docs/compatibility.md @@ -111,15 +111,6 @@ a few operations that we cannot support to the same degree as Spark can on the C ### Decimal Sum Aggregation -A number of fixes for overflow detection went into Spark 3.1.0. Please see -[SPARK-28067](https://issues.apache.org/jira/browse/SPARK-28067) and -[SPARK-32018](https://issues.apache.org/jira/browse/SPARK-32018) for more detailed information. -Some of these fixes we were able to back port, but some of them require Spark 3.1.0 or above to -fully be able to detect overflow in all cases. As such on versions of Spark older than 3.1.0 for -large decimal values there is the possibility of data corruption in some corner cases. -This is true for both the CPU and GPU implementations, but there are fewer of these cases for the -GPU. If this concerns you, you should upgrade to Spark 3.1.0 or above. - When Apache Spark does a sum aggregation on decimal values it will store the result in a value with a precision that is the input precision + 10, but with a maximum precision of 38. For an input precision of 9 and above, Spark will do the aggregations as a Java `BigDecimal` @@ -131,8 +122,13 @@ longer detected. Even then all the values would need to be either the largest or possible to be stored in the type for the overflow to cause data corruption. For the RAPIDS Accelerator we don't have direct access to unlimited precision for our calculations -like the CPU does. For input values with a precision of 8 and below we follow Spark and process the -data the same way, as a 64-bit value. For larger values we will do extra calculations looking at the +like the CPU does, and the aggregations are processed in batches within each task. Therefore it is +possible for the GPU to detect an intermediate overflow after aggregating a batch, e.g.: a sum +aggregation on positive and negative values, where the accumulating sum value temporarily +overflows but returns within bounds before the final cast back into a decimal with precision 38. + +For input values with a precision of 8 and below we follow Spark and process the data the +same way, as a 64-bit value. For larger values we will do extra calculations looking at the higher order digits to be able to detect overflow in all cases. But because of this you may see some performance differences depending on the input precision used. The differences will show up when going from an input precision of 8 to 9 and again when going from an input precision of 28 to 29. diff --git a/integration_tests/src/main/python/data_gen.py b/integration_tests/src/main/python/data_gen.py index 6a8997e7167..100fa105563 100644 --- a/integration_tests/src/main/python/data_gen.py +++ b/integration_tests/src/main/python/data_gen.py @@ -215,19 +215,22 @@ def start(self, rand): class DecimalGen(DataGen): """Generate Decimals, with some built in corner cases.""" - def __init__(self, precision=None, scale=None, nullable=True, special_cases=None): + def __init__(self, precision=None, scale=None, nullable=True, special_cases=None, avoid_positive_values=False): if precision is None: #Maximum number of decimal digits a Long can represent is 18 precision = 18 scale = 0 DECIMAL_MIN = Decimal('-' + ('9' * precision) + 'e' + str(-scale)) DECIMAL_MAX = Decimal(('9'* precision) + 'e' + str(-scale)) - if (special_cases is None): - special_cases = [DECIMAL_MIN, DECIMAL_MAX, Decimal('0')] + if special_cases is None: + special_cases = [DECIMAL_MIN, Decimal('0')] + if not avoid_positive_values: + special_cases.append(DECIMAL_MAX) super().__init__(DecimalType(precision, scale), nullable=nullable, special_cases=special_cases) self.scale = scale self.precision = precision - pattern = "-?[0-9]{1,"+ str(precision) + "}e" + str(-scale) + negative_pattern = "-" if avoid_positive_values else "-?" + pattern = negative_pattern + "[0-9]{1,"+ str(precision) + "}e" + str(-scale) self.base_strs = sre_yield.AllStrings(pattern, flags=0, charset=sre_yield.CHARSET, max_count=_MAX_CHOICES) def __repr__(self): diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index f8901449642..35763262a77 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -218,9 +218,7 @@ _decimal_gen_36_5 = DecimalGen(precision=36, scale=5) _decimal_gen_36_neg5 = DecimalGen(precision=36, scale=-5) -_decimal_gen_38_0 = DecimalGen(precision=38, scale=0) _decimal_gen_38_10 = DecimalGen(precision=38, scale=10) -_decimal_gen_38_neg10 = DecimalGen(precision=38, scale=-10) def get_params(init_list, marked_params=[]): @@ -303,15 +301,22 @@ def get_params(init_list, marked_params=[]): ('b', _decimal_gen_36_neg5), ('c', _decimal_gen_36_neg5)] -_grpkey_short_full_decimals = [ +# Only use negative values to avoid the potential to hover around an overflow +# as values are added and subtracted during the sum. Non-deterministic ordering +# of values from shuffle cannot guarantee overflow calculation is predictable +# when the sum can move in both directions as new partitions are aggregated. +_decimal_gen_sum_38_0 = DecimalGen(precision=38, scale=0, avoid_positive_values=True) +_decimal_gen_sum_38_neg10 = DecimalGen(precision=38, scale=-10, avoid_positive_values=True) + +_grpkey_short_sum_full_decimals = [ ('a', RepeatSeqGen(short_gen, length=50)), - ('b', _decimal_gen_38_0), - ('c', _decimal_gen_38_0)] + ('b', _decimal_gen_sum_38_0), + ('c', _decimal_gen_sum_38_0)] -_grpkey_short_full_neg_scale_decimals = [ +_grpkey_short_sum_full_neg_scale_decimals = [ ('a', RepeatSeqGen(short_gen, length=50)), - ('b', _decimal_gen_38_neg10), - ('c', _decimal_gen_38_neg10)] + ('b', _decimal_gen_sum_38_neg10), + ('c', _decimal_gen_sum_38_neg10)] _init_list_no_nans_with_decimal = _init_list_no_nans + [ @@ -328,9 +333,6 @@ def get_params(init_list, marked_params=[]): _grpkey_short_very_big_neg_scale_decimals] -_init_list_full_decimal = [_grpkey_short_full_decimals, - _grpkey_short_full_neg_scale_decimals] - #Any smaller precision takes way too long to process on the CPU # or results in using too much memory on the GPU @nightly_gpu_mem_consuming_case @@ -386,7 +388,7 @@ def test_hash_grpby_sum(data_gen, conf): @approximate_float @ignore_order @incompat -@pytest.mark.parametrize('data_gen', _init_list_full_decimal, ids=idfn) +@pytest.mark.parametrize('data_gen', [_grpkey_short_sum_full_decimals, _grpkey_short_sum_full_neg_scale_decimals], ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_grpby_sum_full_decimal(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( @@ -418,7 +420,7 @@ def test_hash_reduction_sum_full_decimal(data_gen, conf): @ignore_order @incompat @pytest.mark.parametrize('data_gen', _init_list_with_nans_and_no_nans + [_grpkey_short_mid_decimals, - _grpkey_short_big_decimals, _grpkey_short_very_big_decimals, _grpkey_short_full_decimals], ids=idfn) + _grpkey_short_big_decimals, _grpkey_short_very_big_decimals, _grpkey_short_sum_full_decimals], ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_grpby_avg(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( From c09cf2df60c3395e520abf23f63a49338b98e280 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Tue, 6 Sep 2022 11:33:03 -0500 Subject: [PATCH 086/190] Add GPU acceleration for OptimizedCreateHiveTableAsSelect (#6476) * Add GPU acceleration for OptimizedCreateHiveTableAsSelect Signed-off-by: Jason Lowe * Remove assert_gpu_and_cpu_sql_writes_are_equal_iterator * Move Hive data command overrides to GpuHiveOverrides * Fix test_write_hive_bucketed_table_fallback and add bucketed Hive CTAS test * Fix test skipping Signed-off-by: Jason Lowe --- integration_tests/src/main/python/asserts.py | 44 +++- .../main/python/datasourcev2_write_test.py | 37 ++-- .../src/main/python/hive_write_test.py | 133 +++++++++++ .../src/main/python/spark_session.py | 9 +- .../nvidia/spark/rapids/shims/PlanShims.scala | 23 ++ .../sql/rapids/shims/RapidsErrorUtils.scala | 6 + .../rapids/shims/CharVarcharUtilsShims.scala | 27 +++ .../sql/rapids/shims/RapidsErrorUtils.scala | 6 + .../nvidia/spark/rapids/shims/PlanShims.scala | 26 +++ .../sql/rapids/shims/RapidsErrorUtils.scala | 7 +- .../sql/rapids/shims/RapidsErrorUtils.scala | 7 +- .../rapids/shims/CharVarcharUtilsShims.scala | 27 +++ .../sql/rapids/shims/RapidsErrorUtils.scala | 7 +- .../rapids/GpuDataWritingCommandExec.scala | 21 +- ...imizedCreateHiveTableAsSelectCommand.scala | 208 ++++++++++++++++++ .../nvidia/spark/rapids/GpuOverrides.scala | 6 +- .../nvidia/spark/rapids/HiveProvider.scala | 4 + .../com/nvidia/spark/rapids/Plugin.scala | 3 +- .../sql/hive/rapids/GpuHiveOverrides.scala | 33 ++- .../sql/hive/rapids/HiveProviderImpl.scala | 17 +- 20 files changed, 608 insertions(+), 43 deletions(-) create mode 100644 integration_tests/src/main/python/hive_write_test.py create mode 100644 sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/PlanShims.scala create mode 100644 sql-plugin/src/main/311until330-all/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala create mode 100644 sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/PlanShims.scala create mode 100644 sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOptimizedCreateHiveTableAsSelectCommand.scala diff --git a/integration_tests/src/main/python/asserts.py b/integration_tests/src/main/python/asserts.py index fe15041e9ba..b7d807364d7 100644 --- a/integration_tests/src/main/python/asserts.py +++ b/integration_tests/src/main/python/asserts.py @@ -261,7 +261,7 @@ def _assert_gpu_and_cpu_writes_are_equal( def assert_gpu_and_cpu_writes_are_equal_collect(write_func, read_func, base_path, conf={}): """ Assert when running write_func on both the CPU and the GPU and reading using read_func - ont he CPU that the results are equal. + on the CPU that the results are equal. In this case the data is collected back to the driver and compared here, so be careful about the amount of data returned. """ @@ -270,12 +270,52 @@ def assert_gpu_and_cpu_writes_are_equal_collect(write_func, read_func, base_path def assert_gpu_and_cpu_writes_are_equal_iterator(write_func, read_func, base_path, conf={}): """ Assert when running write_func on both the CPU and the GPU and reading using read_func - ont he CPU that the results are equal. + on the CPU that the results are equal. In this case the data is pulled back to the driver in chunks and compared here so any amount of data can work, just be careful about how long it might take. """ _assert_gpu_and_cpu_writes_are_equal(write_func, read_func, base_path, 'ITERATOR', conf=conf) +def assert_gpu_and_cpu_sql_writes_are_equal_collect(table_name_factory, write_sql_func, conf={}): + """ + Assert when running SQL text from write_sql_func on both the CPU and the GPU and reading + both resulting tables on the CPU that the results are equal. + In this case the data is collected back to the driver and compared here, so be + careful about the amount of data returned. + """ + conf = _prep_incompat_conf(conf) + + print('### CPU RUN ###') + cpu_table = table_name_factory.get() + cpu_start = time.time() + def do_write(spark, table_name): + sql_text = write_sql_func(spark, table_name) + spark.sql(sql_text) + return None + with_cpu_session(lambda spark : do_write(spark, cpu_table), conf=conf) + cpu_end = time.time() + print('### GPU RUN ###') + gpu_start = time.time() + gpu_table = table_name_factory.get() + with_gpu_session(lambda spark : do_write(spark, gpu_table), conf=conf) + gpu_end = time.time() + print('### WRITE: GPU TOOK {} CPU TOOK {} ###'.format( + gpu_end - gpu_start, cpu_end - cpu_start)) + + mode = "COLLECT" + (cpu_bring_back, cpu_collect_type) = _prep_func_for_compare( + lambda spark: spark.sql("SELECT * FROM {}".format(cpu_table)), mode) + (gpu_bring_back, gpu_collect_type) = _prep_func_for_compare( + lambda spark: spark.sql("SELECT * FROM {}".format(gpu_table)), mode) + + from_cpu = with_cpu_session(cpu_bring_back, conf=conf) + from_gpu = with_cpu_session(gpu_bring_back, conf=conf) + if should_sort_locally(): + from_cpu.sort(key=_RowCmp) + from_gpu.sort(key=_RowCmp) + + assert_equal(from_cpu, from_gpu) + def assert_gpu_fallback_write(write_func, read_func, base_path, diff --git a/integration_tests/src/main/python/datasourcev2_write_test.py b/integration_tests/src/main/python/datasourcev2_write_test.py index ac9b7bfeb30..2a2a8f27e0e 100644 --- a/integration_tests/src/main/python/datasourcev2_write_test.py +++ b/integration_tests/src/main/python/datasourcev2_write_test.py @@ -13,39 +13,34 @@ # limitations under the License. import pytest -from asserts import assert_gpu_fallback_write +from asserts import assert_gpu_fallback_collect from data_gen import * from marks import * from pyspark.sql.types import * +from spark_session import is_hive_available, is_spark_330_or_later, with_cpu_session @ignore_order @allow_non_gpu('DataWritingCommandExec') +@pytest.mark.skipif(not (is_hive_available() and is_spark_330_or_later()), reason="Must have Hive on Spark 3.3+") @pytest.mark.parametrize('fileFormat', ['parquet', 'orc']) -def test_write_hive_bucketed_table_fallback(spark_tmp_path, spark_tmp_table_factory, fileFormat): +def test_write_hive_bucketed_table_fallback(spark_tmp_table_factory, fileFormat): """ fallback because GPU does not support Hive hash partition """ - src = spark_tmp_table_factory.get() table = spark_tmp_table_factory.get() - def write_hive_table(spark): - - data = map(lambda i: (i % 13, str(i), i % 5), range(50)) - df = spark.createDataFrame(data, ["i", "j", "k"]) - df.write.mode("overwrite").partitionBy("k").bucketBy(8, "i", "j").format(fileFormat).saveAsTable(src) + def create_hive_table(spark): + spark.sql("""create table {0} (a bigint, b bigint, c bigint) + stored as {1} + clustered by (b) into 3 buckets""".format(table, fileFormat)) + return None - spark.sql(""" - create table if not exists {0} - using hive options(fileFormat \"{1}\") - as select * from {2} - """.format(table, fileFormat, src)) + conf = {"hive.enforce.bucketing": "true", + "hive.exec.dynamic.partition": "true", + "hive.exec.dynamic.partition.mode": "nonstrict"} + with_cpu_session(create_hive_table, conf = conf) - data_path = spark_tmp_path + '/HIVE_DATA' - - assert_gpu_fallback_write( - lambda spark, _: write_hive_table(spark), - lambda spark, _: spark.sql("SELECT * FROM {}".format(table)), - data_path, + assert_gpu_fallback_collect( + lambda spark: spark.sql("insert into {} values (1, 2, 3)".format(table)), 'DataWritingCommandExec', - conf = {"hive.exec.dynamic.partition": "true", - "hive.exec.dynamic.partition.mode": "nonstrict"}) \ No newline at end of file + conf = conf) diff --git a/integration_tests/src/main/python/hive_write_test.py b/integration_tests/src/main/python/hive_write_test.py new file mode 100644 index 00000000000..4df2e4d2852 --- /dev/null +++ b/integration_tests/src/main/python/hive_write_test.py @@ -0,0 +1,133 @@ +# Copyright (c) 2022, NVIDIA CORPORATION. +# +# 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. + +import pytest + +from asserts import assert_gpu_and_cpu_sql_writes_are_equal_collect, assert_gpu_fallback_collect +from data_gen import * +from datetime import date, datetime, timezone +from marks import * +from spark_session import is_hive_available, is_spark_330_or_later, with_cpu_session + +# Using positive timestamps to work around a cudf ORC bug +# https://github.com/rapidsai/cudf/issues/11525 +# Using a limited upper end for timestamps to avoid INT96 overflow on Parquet +def _restricted_timestamp(nullable=True): + return TimestampGen(start=datetime(1970, 1, 1, tzinfo=timezone.utc), + end=datetime(2262, 4, 11, tzinfo=timezone.utc), + nullable=nullable) + +_basic_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, + string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), + _restricted_timestamp() + ] + decimal_gens + +_basic_struct_gen = StructGen([['child'+str(ind), sub_gen] for ind, sub_gen in enumerate(_basic_gens)]) + +_struct_gens = [_basic_struct_gen, + StructGen([['child0', byte_gen], ['child1', _basic_struct_gen]]), + StructGen([['child0', ArrayGen(short_gen)], ['child1', double_gen]])] + +_array_gens = [ArrayGen(sub_gen) for sub_gen in _basic_gens] + [ + ArrayGen(ArrayGen(short_gen, max_length=10), max_length=10), + ArrayGen(ArrayGen(string_gen, max_length=10), max_length=10), + ArrayGen(StructGen([['child0', byte_gen], ['child1', string_gen], ['child2', float_gen]]))] + +_map_gens = [simple_string_to_string_map_gen] + [MapGen(f(nullable=False), f()) for f in [ + BooleanGen, ByteGen, ShortGen, IntegerGen, LongGen, FloatGen, DoubleGen, + lambda nullable=True: _restricted_timestamp(nullable=nullable), + lambda nullable=True: DateGen(start=date(1590, 1, 1), nullable=nullable), + lambda nullable=True: DecimalGen(precision=15, scale=1, nullable=nullable), + lambda nullable=True: DecimalGen(precision=36, scale=5, nullable=nullable)]] + +_write_gens = [_basic_gens, _struct_gens, _array_gens, _map_gens] + +# There appears to be a race when computing tasks for writing, order can be different even on CPU +@ignore_order(local=True) +@pytest.mark.skipif(not is_hive_available(), reason="Hive is missing") +@pytest.mark.parametrize("gens", _write_gens, ids=idfn) +@pytest.mark.parametrize("storage", ["PARQUET", "nativeorc", "hiveorc"]) +def test_optimized_hive_ctas_basic(gens, storage, spark_tmp_table_factory): + data_table = spark_tmp_table_factory.get() + gen_list = [('c' + str(i), gen) for i, gen in enumerate(gens)] + with_cpu_session(lambda spark: gen_df(spark, gen_list).createOrReplaceTempView(data_table)) + def do_write(spark, table_name): + store_name = storage + if storage.endswith("orc"): + store_name = "ORC" + return "CREATE TABLE {} STORED AS {} AS SELECT * FROM {}".format( + table_name, store_name, data_table) + conf = { + "spark.sql.legacy.parquet.datetimeRebaseModeInWrite": "CORRECTED", + "spark.sql.legacy.parquet.int96RebaseModeInWrite": "CORRECTED" + } + if storage == "nativeorc": + conf["spark.sql.orc.impl"] = "native" + elif storage == "hiveorc": + conf["spark.sql.orc.impl"] = "hive" + assert_gpu_and_cpu_sql_writes_are_equal_collect(spark_tmp_table_factory, do_write, conf=conf) + +@allow_non_gpu("DataWritingCommandExec") +@pytest.mark.skipif(not is_hive_available(), reason="Hive is missing") +@pytest.mark.parametrize("gens", [_basic_gens], ids=idfn) +@pytest.mark.parametrize("storage_with_confs", [ + ("PARQUET", {"spark.sql.legacy.parquet.datetimeRebaseModeInWrite": "LEGACY", + "spark.sql.legacy.parquet.int96RebaseModeInWrite": "LEGACY"}), + ("PARQUET", {"parquet.encryption.footer.key": "k1", + "parquet.encryption.column.keys": "k2:a"}), + ("PARQUET", {"spark.sql.parquet.compression.codec": "gzip"}), + ("PARQUET", {"spark.sql.parquet.writeLegacyFormat": "true"}), + ("ORC", {"spark.sql.orc.compression.codec": "zlib"})], ids=idfn) +def test_optimized_hive_ctas_configs_fallback(gens, storage_with_confs, spark_tmp_table_factory): + data_table = spark_tmp_table_factory.get() + gen_list = [('c' + str(i), gen) for i, gen in enumerate(gens)] + with_cpu_session(lambda spark: gen_df(spark, gen_list).createOrReplaceTempView(data_table)) + storage, confs = storage_with_confs + assert_gpu_fallback_collect( + lambda spark: spark.sql("CREATE TABLE {} STORED AS {} AS SELECT * FROM {}".format( + spark_tmp_table_factory.get(), storage, data_table)), + "DataWritingCommandExec", conf=confs) + +@allow_non_gpu("DataWritingCommandExec") +@pytest.mark.skipif(not is_hive_available(), reason="Hive is missing") +@pytest.mark.parametrize("gens", [_basic_gens], ids=idfn) +@pytest.mark.parametrize("storage_with_opts", [ + ("PARQUET", {"parquet.encryption.footer.key": "k1", + "parquet.encryption.column.keys": "k2:a"}), + ("ORC", {"orc.compress": "zlib"})], ids=idfn) +def test_optimized_hive_ctas_options_fallback(gens, storage_with_opts, spark_tmp_table_factory): + data_table = spark_tmp_table_factory.get() + gen_list = [('c' + str(i), gen) for i, gen in enumerate(gens)] + with_cpu_session(lambda spark: gen_df(spark, gen_list).createOrReplaceTempView(data_table)) + storage, opts = storage_with_opts + opts_string = ", ".join(["'{}'='{}'".format(k, v) for k, v in opts.items()]) + assert_gpu_fallback_collect( + lambda spark: spark.sql("CREATE TABLE {} OPTIONS ({}) STORED AS {} AS SELECT * FROM {}".format( + spark_tmp_table_factory.get(), opts_string, storage, data_table)), + "DataWritingCommandExec") + +@allow_non_gpu("DataWritingCommandExec") +@pytest.mark.skipif(not (is_hive_available() and is_spark_330_or_later()), + reason="Requires Hive and Spark 3.3+ to write bucketed Hive tables") +@pytest.mark.parametrize("gens", [_basic_gens], ids=idfn) +@pytest.mark.parametrize("storage", ["PARQUET", "ORC"], ids=idfn) +def test_optimized_hive_bucketed_fallback(gens, storage, spark_tmp_table_factory): + in_table = spark_tmp_table_factory.get() + with_cpu_session(lambda spark: three_col_df(spark, int_gen, int_gen, int_gen).createOrReplaceTempView(in_table)) + assert_gpu_fallback_collect( + lambda spark: spark.sql( + """CREATE TABLE {} STORED AS {} + CLUSTERED BY (b) INTO 3 BUCKETS + AS SELECT * FROM {}""".format(spark_tmp_table_factory.get(), storage, in_table)), + "DataWritingCommandExec") diff --git a/integration_tests/src/main/python/spark_session.py b/integration_tests/src/main/python/spark_session.py index 4175c3c6f8e..4d518f3705e 100644 --- a/integration_tests/src/main/python/spark_session.py +++ b/integration_tests/src/main/python/spark_session.py @@ -13,7 +13,7 @@ # limitations under the License. import os -from conftest import is_allowing_any_non_gpu, get_non_gpu_allowed, get_validate_execs_in_gpu_plan, is_databricks_runtime +from conftest import is_allowing_any_non_gpu, get_non_gpu_allowed, get_validate_execs_in_gpu_plan, is_databricks_runtime, is_at_least_precommit_run from pyspark.sql import DataFrame from spark_init_internal import get_spark_i_know_what_i_am_doing, spark_version @@ -197,3 +197,10 @@ def get_jvm_charset(): def is_jvm_charset_utf8(): return get_jvm_charset() == 'UTF-8' + +def is_hive_available(): + # precommit and nightly runs are supposed to have Hive, + # so tests should fail if Hive is missing in those environments. + if is_at_least_precommit_run(): + return True + return _spark.conf.get("spark.sql.catalogImplementation") == "hive" diff --git a/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/PlanShims.scala b/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/PlanShims.scala new file mode 100644 index 00000000000..d0753ed3efa --- /dev/null +++ b/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/PlanShims.scala @@ -0,0 +1,23 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.execution.SparkPlan + +object PlanShims { + def extractExecutedPlan(plan: SparkPlan): SparkPlan = plan +} diff --git a/sql-plugin/src/main/311until320-nondb/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/311until320-nondb/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index 580ff0a3194..03ab6e2cbc1 100644 --- a/sql-plugin/src/main/311until320-nondb/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/311until320-nondb/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -16,6 +16,8 @@ package org.apache.spark.sql.rapids.shims +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.types.{DataType, Decimal, DecimalType} @@ -70,4 +72,8 @@ object RapidsErrorUtils { new RuntimeException(s"""Found duplicate field(s) "$requiredFieldName": """ + s"$matchedFields in case-insensitive mode") } + + def tableIdentifierExistsError(tableIdentifier: TableIdentifier): Throwable = { + throw new AnalysisException(s"$tableIdentifier already exists.") + } } diff --git a/sql-plugin/src/main/311until330-all/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala b/sql-plugin/src/main/311until330-all/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala new file mode 100644 index 00000000000..70db614c351 --- /dev/null +++ b/sql-plugin/src/main/311until330-all/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala @@ -0,0 +1,27 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.catalyst.util.CharVarcharUtils +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType + +object CharVarcharUtilsShims { + def getRawSchema(schema: StructType, conf: SQLConf): StructType = { + CharVarcharUtils.getRawSchema(schema) + } +} diff --git a/sql-plugin/src/main/31xdb/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/31xdb/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index 580ff0a3194..03ab6e2cbc1 100644 --- a/sql-plugin/src/main/31xdb/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/31xdb/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -16,6 +16,8 @@ package org.apache.spark.sql.rapids.shims +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.types.{DataType, Decimal, DecimalType} @@ -70,4 +72,8 @@ object RapidsErrorUtils { new RuntimeException(s"""Found duplicate field(s) "$requiredFieldName": """ + s"$matchedFields in case-insensitive mode") } + + def tableIdentifierExistsError(tableIdentifier: TableIdentifier): Throwable = { + throw new AnalysisException(s"$tableIdentifier already exists.") + } } diff --git a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/PlanShims.scala b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/PlanShims.scala new file mode 100644 index 00000000000..8e6f4ace219 --- /dev/null +++ b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/PlanShims.scala @@ -0,0 +1,26 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.execution.{CommandResultExec, SparkPlan} + +object PlanShims { + def extractExecutedPlan(plan: SparkPlan): SparkPlan = plan match { + case p: CommandResultExec => p.commandPhysicalPlan + case _ => plan + } +} diff --git a/sql-plugin/src/main/320until330-nondb/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/320until330-nondb/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index a073eb02c07..9520cf206c8 100644 --- a/sql-plugin/src/main/320until330-nondb/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/320until330-nondb/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -16,8 +16,9 @@ package org.apache.spark.sql.rapids.shims +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.trees.Origin -import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types.{DataType, Decimal, DecimalType} object RapidsErrorUtils { @@ -72,4 +73,8 @@ object RapidsErrorUtils { QueryExecutionErrors.foundDuplicateFieldInCaseInsensitiveModeError( requiredFieldName, matchedFields) } + + def tableIdentifierExistsError(tableIdentifier: TableIdentifier): Throwable = { + QueryCompilationErrors.tableIdentifierExistsError(tableIdentifier) + } } diff --git a/sql-plugin/src/main/321db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/321db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index e1ea8d508dc..32acf837710 100644 --- a/sql-plugin/src/main/321db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/321db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -16,8 +16,9 @@ package org.apache.spark.sql.rapids.shims +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.trees.Origin -import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types.{DataType, Decimal, DecimalType} object RapidsErrorUtils { @@ -75,4 +76,8 @@ object RapidsErrorUtils { QueryExecutionErrors.foundDuplicateFieldInCaseInsensitiveModeError( requiredFieldName, matchedFields) } + + def tableIdentifierExistsError(tableIdentifier: TableIdentifier): Throwable = { + QueryCompilationErrors.tableIdentifierExistsError(tableIdentifier) + } } diff --git a/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala b/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala new file mode 100644 index 00000000000..52d66623e41 --- /dev/null +++ b/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala @@ -0,0 +1,27 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.catalyst.util.CharVarcharUtils +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType + +object CharVarcharUtilsShims { + def getRawSchema(schema: StructType, conf: SQLConf): StructType = { + CharVarcharUtils.getRawSchema(schema, conf) + } +} diff --git a/sql-plugin/src/main/330+/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/330+/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index d35068a4c11..ce428a6de92 100644 --- a/sql-plugin/src/main/330+/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/330+/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -16,8 +16,9 @@ package org.apache.spark.sql.rapids.shims +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.trees.Origin -import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types.{DataType, Decimal, DecimalType} object RapidsErrorUtils { @@ -76,4 +77,8 @@ object RapidsErrorUtils { QueryExecutionErrors.foundDuplicateFieldInCaseInsensitiveModeError( requiredFieldName, matchedFields) } + + def tableIdentifierExistsError(tableIdentifier: TableIdentifier): Throwable = { + QueryCompilationErrors.tableIdentifierExistsError(tableIdentifier) + } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala index a2a620a0ba3..6d1389e72a4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala @@ -21,13 +21,14 @@ import java.net.URI import com.nvidia.spark.rapids.shims.{ShimUnaryCommand, ShimUnaryExecNode} import org.apache.hadoop.conf.Configuration +import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.command.DataWritingCommand -import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.GpuWriteJobStatsTracker import org.apache.spark.sql.rapids.execution.TrampolineUtil @@ -87,6 +88,22 @@ object GpuDataWritingCommand { } } } + + /** + * When execute CTAS operators, the write can be delegated to a sub-command + * and we need to propagate the metrics from that sub-command to the + * parent command. + * Derived from Spark's DataWritingCommand.propagateMetrics + */ + def propogateMetrics( + sparkContext: SparkContext, + command: GpuDataWritingCommand, + metrics: Map[String, SQLMetric]): Unit = { + command.metrics.foreach { case (key, metric) => metrics(key).set(metric.value) } + SQLMetrics.postDriverMetricUpdates(sparkContext, + sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY), + metrics.values.toSeq) + } } case class GpuDataWritingCommandExec(cmd: GpuDataWritingCommand, child: SparkPlan) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOptimizedCreateHiveTableAsSelectCommand.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOptimizedCreateHiveTableAsSelectCommand.scala new file mode 100644 index 00000000000..95b13b160de --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOptimizedCreateHiveTableAsSelectCommand.scala @@ -0,0 +1,208 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids + +import java.util.Locale + +import scala.util.control.NonFatal + +import com.nvidia.spark.rapids.shims.CharVarcharUtilsShims + +import org.apache.spark.sql.{SaveMode, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SessionCatalog} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand +import org.apache.spark.sql.hive.execution.OptimizedCreateHiveTableAsSelectCommand +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.rapids.{GpuInsertIntoHadoopFsRelationCommand, GpuOrcFileFormat} +import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils +import org.apache.spark.sql.vectorized.ColumnarBatch + +/** GPU version of Spark's CreateHiveTableAsSelectBase */ +trait GpuCreateHiveTableAsSelectBase extends GpuDataWritingCommand { + val tableDesc: CatalogTable + val query: LogicalPlan + val outputColumnNames: Seq[String] + val mode: SaveMode + + protected val tableIdentifier: TableIdentifier = tableDesc.identifier + + override def runColumnar(sparkSession: SparkSession, child: SparkPlan): Seq[ColumnarBatch] = { + val catalog = sparkSession.sessionState.catalog + val tableExists = catalog.tableExists(tableIdentifier) + + if (tableExists) { + assert(mode != SaveMode.Overwrite, + s"Expect the table $tableIdentifier has been dropped when the save mode is Overwrite") + + if (mode == SaveMode.ErrorIfExists) { + throw RapidsErrorUtils.tableIdentifierExistsError(tableIdentifier) + } + if (mode == SaveMode.Ignore) { + // Since the table already exists and the save mode is Ignore, we will just return. + return Seq.empty + } + + val command = getWritingCommand(catalog, tableDesc, tableExists = true) + command.runColumnar(sparkSession, child) + GpuDataWritingCommand.propogateMetrics(sparkSession.sparkContext, command, metrics) + } else { + tableDesc.storage.locationUri.foreach { p => + GpuDataWritingCommand.assertEmptyRootPath(p, mode, sparkSession.sessionState.newHadoopConf) + } + // TODO ideally, we should get the output data ready first and then + // add the relation into catalog, just in case of failure occurs while data + // processing. + val tableSchema = CharVarcharUtilsShims.getRawSchema( + outputColumns.toStructType, sparkSession.sessionState.conf) + assert(tableDesc.schema.isEmpty) + catalog.createTable( + tableDesc.copy(schema = tableSchema), ignoreIfExists = false) + + try { + // Read back the metadata of the table which was created just now. + val createdTableMeta = catalog.getTableMetadata(tableDesc.identifier) + val command = getWritingCommand(catalog, createdTableMeta, tableExists = false) + command.runColumnar(sparkSession, child) + GpuDataWritingCommand.propogateMetrics(sparkSession.sparkContext, command, metrics) + } catch { + case NonFatal(e) => + // drop the created table. + catalog.dropTable(tableIdentifier, ignoreIfNotExists = true, purge = false) + throw e + } + } + + Seq.empty[ColumnarBatch] + } + + // Returns `GpuDataWritingCommand` which actually writes data into the table. + def getWritingCommand( + catalog: SessionCatalog, + tableDesc: CatalogTable, + tableExists: Boolean): GpuDataWritingCommand + + // A subclass should override this with the Class name of the concrete type expected to be + // returned from `getWritingCommand`. + def writingCommandClassName: String + + override def argString(maxFields: Int): String = { + s"[Database: ${tableDesc.database}, " + + s"TableName: ${tableDesc.identifier.table}, " + + s"$writingCommandClassName]" + } +} + +case class GpuOptimizedCreateHiveTableAsSelectCommand( + tableDesc: CatalogTable, + query: LogicalPlan, + outputColumnNames: Seq[String], + mode: SaveMode, + cpuCmd: OptimizedCreateHiveTableAsSelectCommand) extends GpuCreateHiveTableAsSelectBase { + override def getWritingCommand( + catalog: SessionCatalog, + tableDesc: CatalogTable, + tableExists: Boolean): GpuDataWritingCommand = { + // Leverage the existing support for InsertIntoHadoopFsRelationCommand to do the write + cpuCmd.getWritingCommand(catalog, tableDesc, tableExists) match { + case c: InsertIntoHadoopFsRelationCommand => + val rapidsConf = new RapidsConf(conf) + val rule = GpuOverrides.dataWriteCmds(c.getClass) + val meta = new InsertIntoHadoopFsRelationCommandMeta(c, rapidsConf, None, rule) + meta.tagForGpu() + if (!meta.canThisBeReplaced) { + throw new IllegalStateException("Unable to convert writing command: " + + meta.explain(all = false)) + } + meta.convertToGpu() + case c => + throw new UnsupportedOperationException(s"Unsupported write command: $c") + } + } + + override def writingCommandClassName: String = + TrampolineUtil.getSimpleName(classOf[GpuInsertIntoHadoopFsRelationCommand]) + + // Do not support partitioned or bucketed writes + override def requireSingleBatch: Boolean = false +} + +final class OptimizedCreateHiveTableAsSelectCommandMeta( + cmd: OptimizedCreateHiveTableAsSelectCommand, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: DataFromReplacementRule) + extends DataWritingCommandMeta[OptimizedCreateHiveTableAsSelectCommand]( + cmd, conf, parent, rule) { + + override def tagSelfForGpu(): Unit = { + // It would be cleaner if we could simply call `cmd.getWritingCommand` and let + // InsertIntoHadoopFsRelationCommandMeta tag the result, but calling getWritingCommand + // before the table exists will crash. So this ends up replicating a portion of the logic + // from OptimizedCreateHiveTableAsSelectCommand.getWritingCommand and underlying + // utility methods to be able to tag whether we can support the optimized Hive write. + val spark = SparkSession.active + val tableDesc = cmd.tableDesc + + if (tableDesc.partitionColumnNames.nonEmpty) { + willNotWorkOnGpu("partitioned writes are not supported") + } + + if (tableDesc.bucketSpec.isDefined) { + willNotWorkOnGpu("bucketing is not supported") + } + + val serde = tableDesc.storage.serde.getOrElse("").toLowerCase(Locale.ROOT) + if (serde.contains("parquet")) { + val mergeSchemaConfKey = "spark.sql.hive.convertMetastoreParquet.mergeSchema" + val shouldMergeSchema = SQLConf.get.getConfString(mergeSchemaConfKey, "false").toBoolean + if (shouldMergeSchema) { + willNotWorkOnGpu("Merging Parquet schemas across part files is not supported, " + + s"see $mergeSchemaConfKey") + } + val options = tableDesc.properties.filterKeys(isParquetProperty) ++ + tableDesc.storage.properties + GpuParquetFileFormat.tagGpuSupport(this, spark, options, cmd.query.schema) + } else if (serde.contains("orc")) { + val options = tableDesc.properties.filterKeys(isOrcProperty) ++ + tableDesc.storage.properties + GpuOrcFileFormat.tagGpuSupport(this, spark, options, cmd.query.schema) + } else { + willNotWorkOnGpu(s"unsupported serde detected: $serde") + } + } + + override def convertToGpu(): GpuDataWritingCommand = { + GpuOptimizedCreateHiveTableAsSelectCommand( + wrapped.tableDesc, + wrapped.query, + wrapped.outputColumnNames, + wrapped.mode, + cmd) + } + + // Return true for Apache ORC and Hive ORC-related configuration names. + // Note that Spark doesn't support configurations like `hive.merge.orcfile.stripe.level`. + private def isOrcProperty(key: String) = + key.startsWith("orc.") || key.contains(".orc.") + + private def isParquetProperty(key: String) = + key.startsWith("parquet.") || key.contains(".parquet.") +} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index dde10c5834a..8657a707b35 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -3748,7 +3748,7 @@ object GpuOverrides extends Logging { .map(r => r.wrap(writeCmd, conf, parent, r).asInstanceOf[DataWritingCommandMeta[INPUT]]) .getOrElse(new RuleNotFoundDataWritingCommandMeta(writeCmd, conf, parent)) - val dataWriteCmds: Map[Class[_ <: DataWritingCommand], + val commonDataWriteCmds: Map[Class[_ <: DataWritingCommand], DataWritingCommandRule[_ <: DataWritingCommand]] = Seq( dataWriteCmd[InsertIntoHadoopFsRelationCommand]( "Write to Hadoop filesystem", @@ -3758,6 +3758,10 @@ object GpuOverrides extends Logging { (a, conf, p, r) => new CreateDataSourceTableAsSelectCommandMeta(a, conf, p, r)) ).map(r => (r.getClassFor.asSubclass(classOf[DataWritingCommand]), r)).toMap + val dataWriteCmds: Map[Class[_ <: DataWritingCommand], + DataWritingCommandRule[_ <: DataWritingCommand]] = + commonDataWriteCmds ++ GpuHiveOverrides.dataWriteCmds + def wrapPlan[INPUT <: SparkPlan]( plan: INPUT, conf: RapidsConf, diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HiveProvider.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HiveProvider.scala index 9f2f67b3673..58027aa4210 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HiveProvider.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HiveProvider.scala @@ -17,6 +17,7 @@ package com.nvidia.spark.rapids import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.execution.command.DataWritingCommand /** * The subclass of HiveProvider imports spark-hive classes. This file should not imports @@ -24,5 +25,8 @@ import org.apache.spark.sql.catalyst.expressions.Expression * runtime. Details see: https://github.com/NVIDIA/spark-rapids/issues/5648 */ trait HiveProvider { + def getDataWriteCmds: Map[Class[_ <: DataWritingCommand], + DataWritingCommandRule[_ <: DataWritingCommand]] + def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala index 5daef05221c..6eba5523407 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala @@ -27,6 +27,7 @@ import scala.util.matching.Regex import ai.rapids.cudf.{CudaException, CudaFatalException, CudfException, MemoryCleaner} import com.nvidia.spark.rapids.python.PythonWorkerSemaphore +import com.nvidia.spark.rapids.shims.PlanShims import org.apache.spark.{ExceptionFailure, SparkConf, SparkContext, TaskFailedReason} import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext} @@ -420,7 +421,7 @@ object ExecutionPlanCaptureCallback { def extractExecutedPlan(plan: Option[SparkPlan]): SparkPlan = { plan match { case Some(p: AdaptiveSparkPlanExec) => p.executedPlan - case Some(p) => p + case Some(p) => PlanShims.extractExecutedPlan(p) case _ => throw new IllegalStateException("No execution plan available") } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveOverrides.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveOverrides.scala index 78c243e4836..7adee9c1c30 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveOverrides.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveOverrides.scala @@ -16,12 +16,13 @@ package org.apache.spark.sql.hive.rapids -import com.nvidia.spark.rapids.{ExprRule, ShimLoader} +import com.nvidia.spark.rapids.{DataWritingCommandRule, ExprRule, HiveProvider, ShimLoader} import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.execution.command.DataWritingCommand object GpuHiveOverrides { - def isSparkHiveAvailable: Boolean = { + val isSparkHiveAvailable: Boolean = { try { ShimLoader.loadClass("org.apache.spark.sql.hive.HiveSessionStateBuilder") ShimLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf") @@ -31,15 +32,29 @@ object GpuHiveOverrides { } } - /** - * Builds the rules that are specific to spark-hive Catalyst nodes. This will return an empty - * mapping if spark-hive is unavailable. - */ - def exprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = { + private lazy val hiveProvider: HiveProvider = { if (isSparkHiveAvailable) { - ShimLoader.newHiveProvider().getExprs + ShimLoader.newHiveProvider() } else { - Map.empty + new HiveProvider() { + override def getDataWriteCmds: Map[Class[_ <: DataWritingCommand], + DataWritingCommandRule[_ <: DataWritingCommand]] = Map.empty + override def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Map.empty + } } } + + + /** + * Builds the data writing command rules that are specific to spark-hive Catalyst nodes. + * This will return an empty mapping if spark-hive is unavailable + */ + def dataWriteCmds: Map[Class[_ <: DataWritingCommand], + DataWritingCommandRule[_ <: DataWritingCommand]] = hiveProvider.getDataWriteCmds + + /** + * Builds the expression rules that are specific to spark-hive Catalyst nodes + * This will return an empty mapping if spark-hive is unavailable. + */ + def exprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = hiveProvider.getExprs } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/HiveProviderImpl.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/HiveProviderImpl.scala index 83de910753c..e921d28af23 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/HiveProviderImpl.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/HiveProviderImpl.scala @@ -17,17 +17,28 @@ package org.apache.spark.sql.hive.rapids import com.nvidia.spark.RapidsUDF -import com.nvidia.spark.rapids.{ExprChecks, ExprMeta, ExprRule, GpuExpression, GpuOverrides, HiveProvider, RapidsConf, RepeatingParamCheck, TypeSig} +import com.nvidia.spark.rapids.{DataWritingCommandRule, ExprChecks, ExprMeta, ExprRule, GpuExpression, GpuOverrides, HiveProvider, OptimizedCreateHiveTableAsSelectCommandMeta, RapidsConf, RepeatingParamCheck, TypeSig} import com.nvidia.spark.rapids.GpuUserDefinedFunction.udfTypeSig import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.execution.command.DataWritingCommand import org.apache.spark.sql.hive.{HiveGenericUDF, HiveSimpleUDF} +import org.apache.spark.sql.hive.execution.OptimizedCreateHiveTableAsSelectCommand class HiveProviderImpl extends HiveProvider { /** - * Builds the rules that are specific to spark-hive Catalyst nodes. This will return an empty - * mapping if spark-hive is unavailable. + * Builds the data writing command rules that are specific to spark-hive Catalyst nodes. + */ + override def getDataWriteCmds: Map[Class[_ <: DataWritingCommand], + DataWritingCommandRule[_ <: DataWritingCommand]] = Seq ( + GpuOverrides.dataWriteCmd[OptimizedCreateHiveTableAsSelectCommand]( + "Create a Hive table from a query result using Spark data source APIs", + (a, conf, p, r) => new OptimizedCreateHiveTableAsSelectCommandMeta(a, conf, p, r)) + ).map(r => (r.getClassFor.asSubclass(classOf[DataWritingCommand]), r)).toMap + + /** + * Builds the expression rules that are specific to spark-hive Catalyst nodes. */ override def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = { Seq( From 1b4a488660ff12f20cb65aa9333954b196f549b6 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Wed, 7 Sep 2022 01:54:39 +0800 Subject: [PATCH 087/190] [DOC] Change recommend setting of spark.locality.wait to 3s [skip ci] (#6491) * Change recommend setting of spark.locality.wait to 3s Signed-off-by: Haoyang Li * Removed locality settings in docs and updated tuning guide Signed-off-by: Haoyang Li * Update docs/tuning-guide.md Co-authored-by: Jason Lowe Signed-off-by: Haoyang Li Co-authored-by: Jason Lowe --- docs/demo/AWS-EMR/Mortgage-ETL-GPU-EMR.ipynb | 2 +- docs/demo/GCP/Mortgage-ETL.ipynb | 1 - docs/get-started/getting-started-aws-emr.md | 1 - docs/get-started/getting-started-databricks.md | 1 - docs/get-started/getting-started-kubernetes.md | 4 ---- docs/get-started/getting-started-on-prem.md | 6 ------ docs/get-started/yarn-gpu.md | 1 - docs/tuning-guide.md | 10 +++++----- 8 files changed, 6 insertions(+), 20 deletions(-) diff --git a/docs/demo/AWS-EMR/Mortgage-ETL-GPU-EMR.ipynb b/docs/demo/AWS-EMR/Mortgage-ETL-GPU-EMR.ipynb index 2e7bd934104..b30f3073856 100644 --- a/docs/demo/AWS-EMR/Mortgage-ETL-GPU-EMR.ipynb +++ b/docs/demo/AWS-EMR/Mortgage-ETL-GPU-EMR.ipynb @@ -39,7 +39,7 @@ " \"driverMemory\": \"4000M\",\n", " \"driverCores\": 2,\n", " \"executorMemory\": \"4000M\",\n", - " \"conf\": {\"spark.sql.adaptive.enabled\": \"false\", \"spark.dynamicAllocation.enabled\": \"false\", \"spark.executor.instances\":2, \"spark.executor.cores\":2, \"spark.rapids.sql.explain\":\"ALL\", \"spark.task.cpus\":\"1\", \"spark.rapids.sql.concurrentGpuTasks\":\"2\", \"spark.rapids.memory.pinnedPool.size\":\"2G\", \"spark.executor.memoryOverhead\":\"2G\", \"spark.executor.extraJavaOptions\":\"-Dai.rapids.cudf.prefer-pinned=true\", \"spark.locality.wait\":\"0s\", \"spark.sql.files.maxPartitionBytes\":\"512m\", \"spark.executor.resource.gpu.amount\":\"1\", \"spark.task.resource.gpu.amount\":\"0.5\", \"spark.plugins\":\"com.nvidia.spark.SQLPlugin\", \"spark.rapids.sql.hasNans\":\"false\", \"spark.rapids.sql.batchSizeBytes\":\"512M\", \"spark.rapids.sql.reader.batchSizeBytes\":\"768M\", \"spark.sql.cache.serializer\" : \"com.nvidia.spark.ParquetCachedBatchSerializer\"}\n", + " \"conf\": {\"spark.sql.adaptive.enabled\": \"false\", \"spark.dynamicAllocation.enabled\": \"false\", \"spark.executor.instances\":2, \"spark.executor.cores\":2, \"spark.rapids.sql.explain\":\"ALL\", \"spark.task.cpus\":\"1\", \"spark.rapids.sql.concurrentGpuTasks\":\"2\", \"spark.rapids.memory.pinnedPool.size\":\"2G\", \"spark.executor.memoryOverhead\":\"2G\", \"spark.executor.extraJavaOptions\":\"-Dai.rapids.cudf.prefer-pinned=true\", \"spark.sql.files.maxPartitionBytes\":\"512m\", \"spark.executor.resource.gpu.amount\":\"1\", \"spark.task.resource.gpu.amount\":\"0.5\", \"spark.plugins\":\"com.nvidia.spark.SQLPlugin\", \"spark.rapids.sql.hasNans\":\"false\", \"spark.rapids.sql.batchSizeBytes\":\"512M\", \"spark.rapids.sql.reader.batchSizeBytes\":\"768M\", \"spark.sql.cache.serializer\" : \"com.nvidia.spark.ParquetCachedBatchSerializer\"}\n", "}" ] }, diff --git a/docs/demo/GCP/Mortgage-ETL.ipynb b/docs/demo/GCP/Mortgage-ETL.ipynb index 54460930f69..1a05cbfbdef 100644 --- a/docs/demo/GCP/Mortgage-ETL.ipynb +++ b/docs/demo/GCP/Mortgage-ETL.ipynb @@ -83,7 +83,6 @@ "conf.set('spark.rapids.sql.reader.batchSizeBytes', '768M')\n", "conf.set(\"spark.rapids.memory.pinnedPool.size\", \"2G\")\n", "conf.set(\"spark.executor.memoryOverhead\", \"2G\")\n", - "conf.set(\"spark.locality.wait\", \"0s\")\n", "conf.set(\"spark.sql.broadcastTimeout\", \"700\")\n", "\n", "spark = SparkSession.builder \\\n", diff --git a/docs/get-started/getting-started-aws-emr.md b/docs/get-started/getting-started-aws-emr.md index 77f553b293c..a3a04678bdf 100644 --- a/docs/get-started/getting-started-aws-emr.md +++ b/docs/get-started/getting-started-aws-emr.md @@ -145,7 +145,6 @@ default settings: "spark.task.resource.gpu.amount":"0.125", "spark.rapids.memory.pinnedPool.size":"2G", "spark.executor.memoryOverhead":"2G", - "spark.locality.wait":"0s", "spark.sql.files.maxPartitionBytes":"256m", "spark.sql.adaptive.enabled":"false" } diff --git a/docs/get-started/getting-started-databricks.md b/docs/get-started/getting-started-databricks.md index 1107202cf62..ea40fa7c372 100644 --- a/docs/get-started/getting-started-databricks.md +++ b/docs/get-started/getting-started-databricks.md @@ -140,7 +140,6 @@ cluster. spark.plugins com.nvidia.spark.SQLPlugin spark.task.resource.gpu.amount 0.1 spark.rapids.memory.pinnedPool.size 2G - spark.locality.wait 0s spark.databricks.delta.optimizeWrite.enabled false spark.sql.adaptive.enabled false spark.sql.optimizer.dynamicPartitionPruning.enabled false diff --git a/docs/get-started/getting-started-kubernetes.md b/docs/get-started/getting-started-kubernetes.md index 244fa3ea44b..2f887a01e7e 100644 --- a/docs/get-started/getting-started-kubernetes.md +++ b/docs/get-started/getting-started-kubernetes.md @@ -108,7 +108,6 @@ $SPARK_HOME/bin/spark-submit \ --conf spark.task.resource.gpu.amount=1 \ --conf spark.rapids.memory.pinnedPool.size=2G \ --conf spark.executor.memoryOverhead=3G \ - --conf spark.locality.wait=0s \ --conf spark.sql.files.maxPartitionBytes=512m \ --conf spark.sql.shuffle.partitions=10 \ --conf spark.plugins=com.nvidia.spark.SQLPlugin \ @@ -173,7 +172,6 @@ $SPARK_HOME/bin/spark-shell \ --conf spark.task.resource.gpu.amount=1 \ --conf spark.rapids.memory.pinnedPool.size=2G \ --conf spark.executor.memoryOverhead=3G \ - --conf spark.locality.wait=0s \ --conf spark.sql.files.maxPartitionBytes=512m \ --conf spark.sql.shuffle.partitions=10 \ --conf spark.plugins=com.nvidia.spark.SQLPlugin \ @@ -240,7 +238,6 @@ $SPARK_HOME/bin/spark-submit \ --conf spark.task.resource.gpu.amount=1 \ --conf spark.rapids.memory.pinnedPool.size=2G \ --conf spark.executor.memoryOverhead=3G \ - --conf spark.locality.wait=0s \ --conf spark.sql.files.maxPartitionBytes=512m \ --conf spark.sql.shuffle.partitions=10 \ --conf spark.plugins=com.nvidia.spark.SQLPlugin \ @@ -302,7 +299,6 @@ Using Spark Operator is another way to submit Spark Applications into a Kubernet "spark.executor.memory": "1g" "spark.rapids.memory.pinnedPool.size": "2g" "spark.executor.memoryOverhead": "3g" - "spark.locality.wait": "0s" "spark.sql.files.maxPartitionBytes": "512m" "spark.sql.shuffle.partitions": "10" "spark.plugins": "com.nvidia.spark.SQLPlugin" diff --git a/docs/get-started/getting-started-on-prem.md b/docs/get-started/getting-started-on-prem.md index b83b9a5f169..b4240b6df9f 100644 --- a/docs/get-started/getting-started-on-prem.md +++ b/docs/get-started/getting-started-on-prem.md @@ -91,7 +91,6 @@ $SPARK_HOME/bin/spark-shell \ --conf spark.rapids.sql.concurrentGpuTasks=1 \ --driver-memory 10g \ --conf spark.rapids.memory.pinnedPool.size=2G \ - --conf spark.locality.wait=0s \ --conf spark.sql.files.maxPartitionBytes=512m \ --conf spark.plugins=com.nvidia.spark.SQLPlugin \ --jars ${SPARK_RAPIDS_PLUGIN_JAR} @@ -165,7 +164,6 @@ $SPARK_HOME/bin/spark-shell \ --conf spark.executor.resource.gpu.amount=1 \ --conf spark.task.resource.gpu.amount=0.25 \ --conf spark.rapids.memory.pinnedPool.size=2G \ - --conf spark.locality.wait=0s \ --conf spark.sql.files.maxPartitionBytes=512m \ --conf spark.plugins=com.nvidia.spark.SQLPlugin ``` @@ -219,7 +217,6 @@ $SPARK_HOME/bin/spark-shell \ --conf spark.task.cpus=1 \ --conf spark.task.resource.gpu.amount=0.25 \ --conf spark.rapids.memory.pinnedPool.size=2G \ - --conf spark.locality.wait=0s \ --conf spark.sql.files.maxPartitionBytes=512m \ --conf spark.plugins=com.nvidia.spark.SQLPlugin \ --conf spark.executor.resource.gpu.discoveryScript=./getGpusResources.sh \ @@ -245,7 +242,6 @@ $SPARK_HOME/bin/spark-shell \ --conf spark.task.cpus=1 \ --conf spark.task.resource.gpu.amount=0.25 \ --conf spark.rapids.memory.pinnedPool.size=2G \ - --conf spark.locality.wait=0s \ --conf spark.sql.files.maxPartitionBytes=512m \ --conf spark.plugins=com.nvidia.spark.SQLPlugin \ --conf spark.executor.resource.gpu.amount=1 \ @@ -282,7 +278,6 @@ $SPARK_HOME/bin/spark-shell \ --conf spark.task.cpus=1 \ --conf spark.task.resource.gpu.amount=0.25 \ --conf spark.rapids.memory.pinnedPool.size=2G \ - --conf spark.locality.wait=0s \ --conf spark.sql.files.maxPartitionBytes=512m \ --conf spark.plugins=com.nvidia.spark.SQLPlugin \ --conf spark.resources.discoveryPlugin=com.nvidia.spark.ExclusiveModeGpuDiscoveryPlugin \ @@ -356,7 +351,6 @@ $SPARK_HOME/bin/spark-shell --master yarn \ --executor-memory 20g \ --conf spark.executor.memoryOverhead=10g \ --conf spark.rapids.memory.pinnedPool.size=8G \ - --conf spark.locality.wait=0s \ --conf spark.sql.files.maxPartitionBytes=512m \ --conf spark.executor.resource.gpu.discoveryScript=./getGpusResources.sh \ --conf spark.task.resource.gpu.amount=0.166 \ diff --git a/docs/get-started/yarn-gpu.md b/docs/get-started/yarn-gpu.md index 42b8f64293e..22dbb4aba4e 100644 --- a/docs/get-started/yarn-gpu.md +++ b/docs/get-started/yarn-gpu.md @@ -27,7 +27,6 @@ spark.rapids.memory.pinnedPool.size=2G spark.executor.memoryOverhead=2G spark.plugins=com.nvidia.spark.SQLPlugin spark.executor.extraJavaOptions='-Dai.rapids.cudf.prefer-pinned=true' -spark.locality.wait=0s spark.executor.resource.gpu.discoveryScript=/usr/lib/spark/scripts/gpu/getGpusResources.sh # this match the location of discovery script spark.sql.files.maxPartitionBytes=512m ``` diff --git a/docs/tuning-guide.md b/docs/tuning-guide.md index 1b1471ea6c9..979083ec10c 100644 --- a/docs/tuning-guide.md +++ b/docs/tuning-guide.md @@ -121,11 +121,11 @@ Configuration key: Default value: `3s` -This configuration setting controls how long Spark should wait to obtain better locality for tasks. -When tasks complete quicker than this setting, the Spark scheduler can end up not leveraging all -of the executors in the cluster during a stage. If you see stages in the job where it appears -Spark is running tasks serially through a small subset of executors it is probably due to this -setting. Some queries will see significant performance gains by setting this to `0`. +This configuration setting controls how long Spark should wait to obtain a better locality for tasks. +If your tasks are long and see poor locality, you can increase this value. If the data sets are small +and the cost of waiting will have less impact on the job's overall completion time, you can reduce this +value to get higher parallelization. In a cluster with high I/O bandwidth you can set it to 0 because it +will be faster to not wait when you can get the data across the network fast enough. ## Number of Concurrent Tasks per GPU Configuration key: [`spark.rapids.sql.concurrentGpuTasks`](configs.md#sql.concurrentGpuTasks) From 93387e49909da9e5d3b4c30c8d7930fa6abb073d Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Tue, 6 Sep 2022 15:38:45 -0500 Subject: [PATCH 088/190] Perform columnar-to-row transition in GpuBringBackToHost.doExecute (#6507) Signed-off-by: Jason Lowe Signed-off-by: Jason Lowe --- .../spark/rapids/GpuBringBackToHost.scala | 3 +- .../rapids/GpuBringBackToHostSuite.scala | 41 +++++++++++++++++++ 2 files changed, 43 insertions(+), 1 deletion(-) create mode 100644 tests/src/test/scala/com/nvidia/spark/rapids/GpuBringBackToHostSuite.scala diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBringBackToHost.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBringBackToHost.scala index 50788a95e47..bb3eb30d850 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBringBackToHost.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBringBackToHost.scala @@ -35,7 +35,8 @@ case class GpuBringBackToHost(child: SparkPlan) extends ShimUnaryExecNode with G override def supportsColumnar: Boolean = true override protected def doExecute(): RDD[InternalRow] = { - child.execute() + val columnarToRow = GpuColumnarToRowExec(child) + columnarToRow.execute() } override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuBringBackToHostSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuBringBackToHostSuite.scala new file mode 100644 index 00000000000..94038e90a96 --- /dev/null +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuBringBackToHostSuite.scala @@ -0,0 +1,41 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids + +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.types.LongType + +class GpuBringBackToHostSuite extends SparkQueryCompareTestSuite { + test("doExecute") { + val numRows = 1024 * 1024 + val rows = withGpuSparkSession { _ => + val plan = GpuBringBackToHost( + GpuRangeExec(start=0, end=numRows, step=1, numSlices=1, + output=Seq(AttributeReference("id", LongType)()), + targetSizeBytes=Math.max(numRows / 10, 1))) + plan.executeCollect() + } + + var rowId = 0 + rows.foreach { row => + assertResult(1)(row.numFields) + assertResult(rowId)(row.getLong(0)) + rowId += 1 + } + assertResult(numRows)(rowId) + } +} From ccab5e8aadb3993f4331b46ffcba7193a2da5f44 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 6 Sep 2022 16:00:58 -0500 Subject: [PATCH 089/190] Allow filtering blocks to be done multithreaded in the Parquet coalescing reader (#6498) * Fallback to CPU for delta log checkpoint file reads Signed-off-by: Thomas Graves * fix stylie * Handle xxx.checkpoint.yyy.parquet file names * Fix to use or instead of and * cleanup Signed-off-by: Thomas Graves * wrapping * Change alluxio logs to be debug Signed-off-by: Thomas Graves * Do the parquet filter with the coalescing reader in paralell Signed-off-by: Thomas Graves * fix filter param * Add filter time to datasource v2 * cleanup code and commonize * cleanup * change metric level * removing log warning messages * Add tests and logging * fix syntax * update configs doc * Update integration_tests/src/main/python/parquet_test.py Co-authored-by: Jason Lowe * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala Co-authored-by: Jason Lowe * Rework from review Signed-off-by: Thomas Graves * change filtertime to be optional * fix filtertime being nano * revert multithreaded changes * add filter time to orc and avro * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala Co-authored-by: Jason Lowe * Always make sure FILTER_TIME in metrics Signed-off-by: Thomas Graves Signed-off-by: Thomas Graves Co-authored-by: Jason Lowe --- docs/configs.md | 1 + .../src/main/python/parquet_test.py | 27 ++++- .../rapids/GpuBatchScanExecMetrics.scala | 1 + .../spark/rapids/GpuMultiFileReader.scala | 9 +- .../com/nvidia/spark/rapids/GpuOrcScan.scala | 12 ++- .../nvidia/spark/rapids/GpuParquetScan.scala | 99 +++++++++++++++---- .../com/nvidia/spark/rapids/RapidsConf.scala | 13 +++ .../apache/spark/sql/rapids/GpuAvroScan.scala | 14 ++- .../sql/rapids/GpuFileSourceScanExec.scala | 1 + 9 files changed, 149 insertions(+), 28 deletions(-) diff --git a/docs/configs.md b/docs/configs.md index edcf1bcc621..8b7d834d5de 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -71,6 +71,7 @@ Name | Description | Default Value spark.rapids.sql.castFloatToString.enabled|Casting from floating point types to string on the GPU returns results that have a different precision than the default results of Spark.|true spark.rapids.sql.castStringToFloat.enabled|When set to true, enables casting from strings to float types (float, double) on the GPU. Currently hex values aren't supported on the GPU. Also note that casting from string to float types on the GPU returns incorrect results when the string represents any number "1.7976931348623158E308" <= x < "1.7976931348623159E308" and "-1.7976931348623158E308" >= x > "-1.7976931348623159E308" in both these cases the GPU returns Double.MaxValue while CPU returns "+Infinity" and "-Infinity" respectively|true spark.rapids.sql.castStringToTimestamp.enabled|When set to true, casting from string to timestamp is supported on the GPU. The GPU only supports a subset of formats when casting strings to timestamps. Refer to the CAST documentation for more details.|false +spark.rapids.sql.coalescing.reader.numFilterParallel|This controls the number of files the coalescing reader will run in each thread when it filters blocks for reading. If this value is greater than zero the files will be filtered in a multithreaded manner where each thread filters the number of files set by this config. If this is set to zero the files are filtered serially. This uses the same thread pool as the multithreaded reader, see spark.rapids.sql.multiThreadedRead.numThreads. Note that filtering multithreaded is useful with Alluxio.|0 spark.rapids.sql.concurrentGpuTasks|Set the number of tasks that can execute concurrently per GPU. Tasks may temporarily block when the number of concurrent tasks in the executor exceeds this amount. Allowing too many concurrent tasks on the same GPU may lead to GPU out of memory errors.|1 spark.rapids.sql.csv.read.decimal.enabled|CSV reading is not 100% compatible when reading decimals.|false spark.rapids.sql.csv.read.double.enabled|CSV reading is not 100% compatible when reading doubles.|true diff --git a/integration_tests/src/main/python/parquet_test.py b/integration_tests/src/main/python/parquet_test.py index 99d1a43f391..286b89ac6e0 100644 --- a/integration_tests/src/main/python/parquet_test.py +++ b/integration_tests/src/main/python/parquet_test.py @@ -70,6 +70,8 @@ def read_parquet_sql(data_path): original_parquet_file_reader_conf = {'spark.rapids.sql.format.parquet.reader.type': 'PERFILE'} multithreaded_parquet_file_reader_conf = {'spark.rapids.sql.format.parquet.reader.type': 'MULTITHREADED'} coalesce_parquet_file_reader_conf = {'spark.rapids.sql.format.parquet.reader.type': 'COALESCING'} +coalesce_parquet_file_reader_multithread_filter_conf = {'spark.rapids.sql.format.parquet.reader.type': 'COALESCING', + 'spark.rapids.sql.coalescing.reader.numFilterParallel': '2'} native_parquet_file_reader_conf = {'spark.rapids.sql.format.parquet.reader.type': 'PERFILE', 'spark.rapids.sql.format.parquet.reader.footer.type': 'NATIVE'} native_multithreaded_parquet_file_reader_conf = {'spark.rapids.sql.format.parquet.reader.type': 'MULTITHREADED', @@ -83,10 +85,33 @@ def read_parquet_sql(data_path): native_coalesce_parquet_file_reader_conf] reader_opt_confs_no_native = [original_parquet_file_reader_conf, multithreaded_parquet_file_reader_conf, - coalesce_parquet_file_reader_conf] + coalesce_parquet_file_reader_conf, coalesce_parquet_file_reader_multithread_filter_conf] reader_opt_confs = reader_opt_confs_native + reader_opt_confs_no_native + +@pytest.mark.parametrize('parquet_gens', [[byte_gen, short_gen, int_gen, long_gen]], ids=idfn) +@pytest.mark.parametrize('read_func', [read_parquet_df]) +@pytest.mark.parametrize('reader_confs', [coalesce_parquet_file_reader_multithread_filter_conf]) +@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) +def test_parquet_read_coalescing_multiple_files(spark_tmp_path, parquet_gens, read_func, reader_confs, v1_enabled_list): + gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] + data_path = spark_tmp_path + '/PARQUET_DATA' + with_cpu_session( + # high number of slices so that a single task reads more than 1 file + lambda spark : gen_df(spark, gen_list, num_slices=30).write.parquet(data_path), + conf=rebase_write_corrected_conf) + all_confs = copy_and_update(reader_confs, { + 'spark.sql.sources.useV1SourceList': v1_enabled_list, + # set the int96 rebase mode values because its LEGACY in databricks which will preclude this op from running on GPU + 'spark.sql.legacy.parquet.int96RebaseModeInRead' : 'CORRECTED', + 'spark.sql.legacy.parquet.datetimeRebaseModeInRead': 'CORRECTED'}) + # once https://github.com/NVIDIA/spark-rapids/issues/1126 is in we can remove spark.sql.legacy.parquet.datetimeRebaseModeInRead config which is a workaround + # for nested timestamp/date support + assert_gpu_and_cpu_are_equal_collect(read_func(data_path), + conf=all_confs) + + @pytest.mark.parametrize('parquet_gens', parquet_gens_list, ids=idfn) @pytest.mark.parametrize('read_func', [read_parquet_df, read_parquet_sql]) @pytest.mark.parametrize('reader_confs', reader_opt_confs) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchScanExecMetrics.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchScanExecMetrics.scala index 376a76135c9..0f08ea25a2a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchScanExecMetrics.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchScanExecMetrics.scala @@ -26,6 +26,7 @@ trait GpuBatchScanExecMetrics extends GpuExec { override lazy val additionalMetrics: Map[String, GpuMetric] = Map( GPU_DECODE_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_GPU_DECODE_TIME), BUFFER_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_BUFFER_TIME), + FILTER_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_FILTER_TIME), PEAK_DEVICE_MEMORY -> createSizeMetric(MODERATE_LEVEL, DESCRIPTION_PEAK_DEVICE_MEMORY)) ++ semaphoreMetrics } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala index 9104e01c024..07d2da7d369 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala @@ -27,7 +27,7 @@ import scala.language.implicitConversions import scala.math.max import ai.rapids.cudf.{ColumnVector, HostMemoryBuffer, NvtxColor, NvtxRange, Table} -import com.nvidia.spark.rapids.GpuMetric.{NUM_OUTPUT_BATCHES, PEAK_DEVICE_MEMORY, SEMAPHORE_WAIT_TIME} +import com.nvidia.spark.rapids.GpuMetric.{FILTER_TIME, NUM_OUTPUT_BATCHES, PEAK_DEVICE_MEMORY, SEMAPHORE_WAIT_TIME} import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableProducingSeq import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration @@ -410,7 +410,14 @@ abstract class MultiFileCloudPartitionReaderBase( } } else { if (filesToRead > 0 && !isDone) { + // Filter time here includes the buffer time as well since those + // happen in the same background threads. This is as close to wall + // clock as we can get right now without further work. + val startTime = System.nanoTime() val fileBufsAndMeta = tasks.poll.get() + metrics.get(FILTER_TIME).foreach { + _ += (System.nanoTime() - startTime) + } filesToRead -= 1 TrampolineUtil.incBytesRead(inputMetrics, fileBufsAndMeta.bytesRead) InputFileUtils.setInputFileBlock( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index 781009f1121..b8d5745b6bb 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -405,7 +405,7 @@ case class GpuOrcMultiFilePartitionReaderFactory( // we must split the different compress files into different ColumnarBatch. // So here try the best to group the same compression files together before hand. val compressionAndStripes = LinkedHashMap[CompressionKind, ArrayBuffer[OrcSingleStripeMeta]]() - val currentTime = System.nanoTime() + val startTime = System.nanoTime() files.map { file => val orcPartitionReaderContext = filterHandler.filterStripes(file, dataSchema, readDataSchema, partitionSchema) @@ -420,8 +420,12 @@ case class GpuOrcMultiFilePartitionReaderFactory( readDataSchema, OrcExtraInfo(orcPartitionReaderContext.requestedMapping))) } + val filterTime = System.nanoTime() - startTime + metrics.get(FILTER_TIME).foreach { + _ += filterTime + } metrics.get("scanTime").foreach { - _ += TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - currentTime) + _ += TimeUnit.NANOSECONDS.toMillis(filterTime) } val clippedStripes = compressionAndStripes.values.flatten.toSeq new MultiFileOrcPartitionReader(conf, files, clippedStripes, readDataSchema, debugDumpPrefix, @@ -463,8 +467,12 @@ case class GpuOrcPartitionReaderFactory( } override def buildColumnarReader(partFile: PartitionedFile): PartitionReader[ColumnarBatch] = { + val startTime = System.nanoTime() val ctx = filterHandler.filterStripes(partFile, dataSchema, readDataSchema, partitionSchema) + metrics.get(FILTER_TIME).foreach { + _ += (System.nanoTime() - startTime) + } if (ctx == null) { new EmptyPartitionReader[ColumnarBatch] } else { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index a24472947f9..63c0ffe5345 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -345,6 +345,8 @@ case class ParquetFileInfoWithBlockMeta(filePath: Path, blocks: Seq[BlockMetaDat isCorrectedInt96RebaseMode: Boolean, isCorrectedRebaseMode: Boolean, hasInt96Timestamps: Boolean) +private case class BlockMetaWithPartFile(meta: ParquetFileInfoWithBlockMeta, file: PartitionedFile) + /** * A parquet compatible stream that allows reading from a HostMemoryBuffer to Parquet. * The majority of the code here was copied from Parquet's DelegatingSeekableInputStream with @@ -945,6 +947,7 @@ case class GpuParquetMultiFilePartitionReaderFactory( private val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles private val filterHandler = GpuParquetFileFilterHandler(sqlConf) private val readUseFieldId = ParquetSchemaClipShims.useFieldId(sqlConf) + private val numFilesFilterParallel = rapidsConf.numFilesFilterParallel // we can't use the coalescing files reader when InputFileName, InputFileBlockStart, // or InputFileBlockLength because we are combining all the files into a single buffer @@ -973,6 +976,55 @@ case class GpuParquetMultiFilePartitionReaderFactory( ignoreMissingFiles, ignoreCorruptFiles, readUseFieldId) } + private def filterBlocksForCoalescingReader( + footerReadType: ParquetFooterReaderType.Value, + file: PartitionedFile, + conf: Configuration, + filters: Array[Filter], + readDataSchema: StructType): BlockMetaWithPartFile = { + try { + val meta = filterHandler.filterBlocks(footerReadType, file, conf, filters, + readDataSchema) + BlockMetaWithPartFile(meta, file) + } catch { + case e: FileNotFoundException if ignoreMissingFiles => + logWarning(s"Skipped missing file: ${file.filePath}", e) + val meta = ParquetFileInfoWithBlockMeta(new Path(new URI(file.filePath)), Seq.empty, + file.partitionValues, null, null, false, false, false) + BlockMetaWithPartFile(meta, file) + // Throw FileNotFoundException even if `ignoreCorruptFiles` is true + case e: FileNotFoundException if !ignoreMissingFiles => throw e + // If ignoreMissingFiles=true, this case will never be reached. But it's ok + // to leave this branch here. + case e@(_: RuntimeException | _: IOException) if ignoreCorruptFiles => + logWarning( + s"Skipped the rest of the content in the corrupted file: ${file.filePath}", e) + val meta = ParquetFileInfoWithBlockMeta(new Path(new URI(file.filePath)), Seq.empty, + file.partitionValues, null, null, false, false, false) + BlockMetaWithPartFile(meta, file) + } + } + + private class CoalescingFilterRunner( + footerReadType: ParquetFooterReaderType.Value, + taskContext: TaskContext, + files: Array[PartitionedFile], + conf: Configuration, + filters: Array[Filter], + readDataSchema: StructType) extends Callable[Array[BlockMetaWithPartFile]] with Logging { + + override def call(): Array[BlockMetaWithPartFile] = { + TrampolineUtil.setTaskContext(taskContext) + try { + files.map { file => + filterBlocksForCoalescingReader(footerReadType, file, conf, filters, readDataSchema) + } + } finally { + TrampolineUtil.unsetTaskContext() + } + } + } + /** * Build the PartitionReader for coalescing reading * @@ -984,37 +1036,38 @@ case class GpuParquetMultiFilePartitionReaderFactory( files: Array[PartitionedFile], conf: Configuration): PartitionReader[ColumnarBatch] = { val clippedBlocks = ArrayBuffer[ParquetSingleDataBlockMeta]() - val currentTime = System.nanoTime() - files.map { file => - val singleFileInfo = try { - filterHandler.filterBlocks(footerReadType, file, conf, filters, readDataSchema) - } catch { - case e: FileNotFoundException if ignoreMissingFiles => - logWarning(s"Skipped missing file: ${file.filePath}", e) - ParquetFileInfoWithBlockMeta(new Path(new URI(file.filePath)), Seq.empty, - file.partitionValues, null, null, false, false, false) - // Throw FileNotFoundException even if `ignoreCorruptFiles` is true - case e: FileNotFoundException if !ignoreMissingFiles => throw e - // If ignoreMissingFiles=true, this case will never be reached. But it's ok - // to leave this branch here. - case e@(_: RuntimeException | _: IOException) if ignoreCorruptFiles => - logWarning( - s"Skipped the rest of the content in the corrupted file: ${file.filePath}", e) - ParquetFileInfoWithBlockMeta(new Path(new URI(file.filePath)), Seq.empty, - file.partitionValues, null, null, false, false, false) + val startTime = System.nanoTime() + val metaAndFilesArr = if (numFilesFilterParallel > 0) { + val tc = TaskContext.get() + val threadPool = MultiFileReaderThreadPool.getOrCreateThreadPool(numThreads) + files.grouped(numFilesFilterParallel).map { fileGroup => + threadPool.submit( + new CoalescingFilterRunner(footerReadType, tc, fileGroup, conf, filters, readDataSchema)) + }.toArray.flatMap(_.get()) + } else { + files.map { file => + filterBlocksForCoalescingReader(footerReadType, file, conf, filters, readDataSchema) } + } + metaAndFilesArr.foreach { metaAndFile => + val singleFileInfo = metaAndFile.meta clippedBlocks ++= singleFileInfo.blocks.map(block => ParquetSingleDataBlockMeta( singleFileInfo.filePath, ParquetDataBlock(block), - file.partitionValues, + metaAndFile.file.partitionValues, ParquetSchemaWrapper(singleFileInfo.schema), singleFileInfo.readSchema, new ParquetExtraInfo(singleFileInfo.isCorrectedRebaseMode, - singleFileInfo.isCorrectedInt96RebaseMode, singleFileInfo.hasInt96Timestamps))) + singleFileInfo.isCorrectedInt96RebaseMode, + singleFileInfo.hasInt96Timestamps))) + } + val filterTime = System.nanoTime() - startTime + metrics.get(FILTER_TIME).foreach { + _ += filterTime } metrics.get("scanTime").foreach { - _ += TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - currentTime) + _ += TimeUnit.NANOSECONDS.toMillis(filterTime) } new MultiFileParquetPartitionReader(conf, files, clippedBlocks, isCaseSensitive, debugDumpPrefix, maxReadBatchSizeRows, maxReadBatchSizeBytes, metrics, @@ -1068,8 +1121,12 @@ case class GpuParquetPartitionReaderFactory( private def buildBaseColumnarParquetReader( file: PartitionedFile): PartitionReader[ColumnarBatch] = { val conf = broadcastedConf.value.value + val startTime = System.nanoTime() val singleFileInfo = filterHandler.filterBlocks(footerReadType, file, conf, filters, readDataSchema) + metrics.get(FILTER_TIME).foreach { + _ += (System.nanoTime() - startTime) + } new ParquetPartitionReader(conf, file, singleFileInfo.filePath, singleFileInfo.blocks, singleFileInfo.schema, isCaseSensitive, readDataSchema, debugDumpPrefix, maxReadBatchSizeRows, diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 9c56b96664a..3c8183754d0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -1524,6 +1524,17 @@ object RapidsConf { .booleanConf .createWithDefault(value = true) + val NUM_FILES_FILTER_PARALLEL = conf("spark.rapids.sql.coalescing.reader.numFilterParallel") + .doc("This controls the number of files the coalescing reader will run " + + "in each thread when it filters blocks for reading. If this value is greater than zero " + + "the files will be filtered in a multithreaded manner where each thread filters " + + "the number of files set by this config. If this is set to zero the files are " + + "filtered serially. This uses the same thread pool as the multithreaded reader, " + + s"see $MULTITHREAD_READ_NUM_THREADS. Note that filtering multithreaded " + + "is useful with Alluxio.") + .integerConf + .createWithDefault(value = 0) + private def printSectionHeader(category: String): Unit = println(s"\n### $category") @@ -1817,6 +1828,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { values.max } + lazy val numFilesFilterParallel: Int = get(NUM_FILES_FILTER_PARALLEL) + lazy val isParquetEnabled: Boolean = get(ENABLE_PARQUET) lazy val isParquetInt96WriteEnabled: Boolean = get(ENABLE_PARQUET_INT96_WRITE) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala index f98a8274951..d62e1f488f1 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala @@ -28,7 +28,7 @@ import scala.math.max import ai.rapids.cudf.{AvroOptions => CudfAvroOptions, HostMemoryBuffer, NvtxColor, NvtxRange, Table} import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetric.{GPU_DECODE_TIME, NUM_OUTPUT_BATCHES, PEAK_DEVICE_MEMORY, READ_FS_TIME, SEMAPHORE_WAIT_TIME, WRITE_BUFFER_TIME} +import com.nvidia.spark.rapids.GpuMetric.{FILTER_TIME, GPU_DECODE_TIME, NUM_OUTPUT_BATCHES, PEAK_DEVICE_MEMORY, READ_FS_TIME, SEMAPHORE_WAIT_TIME, WRITE_BUFFER_TIME} import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.shims.ShimFilePartitionReaderFactory import org.apache.avro.Schema @@ -171,7 +171,11 @@ case class GpuAvroPartitionReaderFactory( override def buildColumnarReader(partFile: PartitionedFile): PartitionReader[ColumnarBatch] = { val conf = broadcastedConf.value.value + val startTime = System.nanoTime() val blockMeta = AvroFileFilterHandler(conf, avroOptions).filterBlocks(partFile) + metrics.get(FILTER_TIME).foreach { + _ += (System.nanoTime() - startTime) + } val reader = new PartitionReaderWithBytesRead(new GpuAvroPartitionReader(conf, partFile, blockMeta, readDataSchema, debugDumpPrefix, maxReadBatchSizeRows, maxReadBatchSizeBytes, metrics)) @@ -248,7 +252,7 @@ case class GpuAvroMultiFilePartitionReaderFactory( val clippedBlocks = ArrayBuffer[AvroSingleDataBlockInfo]() val mapPathHeader = LinkedHashMap[Path, Header]() val filterHandler = AvroFileFilterHandler(conf, options) - val currentTime = System.nanoTime() + val startTime = System.nanoTime() files.foreach { file => val singleFileInfo = try { filterHandler.filterBlocks(file) @@ -277,8 +281,12 @@ case class GpuAvroMultiFilePartitionReaderFactory( mapPathHeader.put(fPath, singleFileInfo.header) } } + val filterTime = System.nanoTime() - startTime + metrics.get(FILTER_TIME).foreach { + _ += filterTime + } metrics.get("scanTime").foreach { - _ += TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - currentTime) + _ += TimeUnit.NANOSECONDS.toMillis(filterTime) } new GpuMultiFileAvroPartitionReader(conf, files, clippedBlocks, readDataSchema, partitionSchema, maxReadBatchSizeRows, maxReadBatchSizeBytes, numThreads, diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala index b638749e93b..0c5310f54aa 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala @@ -378,6 +378,7 @@ case class GpuFileSourceScanExec( "filesSize" -> createSizeMetric(ESSENTIAL_LEVEL, "size of files read"), GPU_DECODE_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_GPU_DECODE_TIME), BUFFER_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_BUFFER_TIME), + FILTER_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_FILTER_TIME), PEAK_DEVICE_MEMORY -> createSizeMetric(MODERATE_LEVEL, DESCRIPTION_PEAK_DEVICE_MEMORY) ) ++ { relation.fileFormat match { From 165dac497eff329558b2e9445ffa668dc7bcb201 Mon Sep 17 00:00:00 2001 From: sinkinben Date: Wed, 7 Sep 2022 11:27:38 +0800 Subject: [PATCH 090/190] Support float/double castings for ORC reading [databricks] (#6319) * Implemented casting float/double to {bool, int8/16/32/64, double/float, string, timestamp}. * When casting float/double to string, since the precision of float numbers in GPU differs from CPU (the casting result will differ from result of CPU), we added a config item `spark.rapids.sql.format.orc.floatTypesToString.enable` (default value is true) to control whether if we can cast float/double to string while reading ORC. Signed-off-by: sinkinben --- docs/configs.md | 1 + .../src/main/python/orc_cast_test.py | 52 +++++ .../com/nvidia/spark/rapids/GpuCast.scala | 2 +- .../com/nvidia/spark/rapids/GpuOrcScan.scala | 193 ++++++++++++++++-- .../com/nvidia/spark/rapids/RapidsConf.scala | 15 ++ 5 files changed, 248 insertions(+), 15 deletions(-) diff --git a/docs/configs.md b/docs/configs.md index 8b7d834d5de..18560823614 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -93,6 +93,7 @@ Name | Description | Default Value spark.rapids.sql.format.json.enabled|When set to true enables all json input and output acceleration. (only input is currently supported anyways)|false spark.rapids.sql.format.json.read.enabled|When set to true enables json input acceleration|false spark.rapids.sql.format.orc.enabled|When set to false disables all orc input and output acceleration|true +spark.rapids.sql.format.orc.floatTypesToString.enable|When reading an ORC file, the source data schemas(schemas of ORC file) may differ from the target schemas (schemas of the reader), we need to handle the castings from source type to target type. Since float/double numbers in GPU have different precision with CPU, when casting float/double to string, the result of GPU is different from result of CPU spark. Its default value is `true` (this means the strings result will differ from result of CPU). If it's set `false` explicitly and there exists casting from float/double to string in the job, then such behavior will cause an exception, and the job will fail.|true spark.rapids.sql.format.orc.multiThreadedRead.maxNumFilesParallel|A limit on the maximum number of files per task processed in parallel on the CPU side before the file is sent to the GPU. This affects the amount of host memory used when reading the files in parallel. Used with MULTITHREADED reader, see spark.rapids.sql.format.orc.reader.type.|2147483647 spark.rapids.sql.format.orc.multiThreadedRead.numThreads|The maximum number of threads, on the executor, to use for reading small ORC files in parallel. This can not be changed at runtime after the executor has started. Used with MULTITHREADED reader, see spark.rapids.sql.format.orc.reader.type. DEPRECATED: use spark.rapids.sql.multiThreadedRead.numThreads|None spark.rapids.sql.format.orc.read.enabled|When set to false disables orc input acceleration|true diff --git a/integration_tests/src/main/python/orc_cast_test.py b/integration_tests/src/main/python/orc_cast_test.py index 6a84407a632..6d5966294a8 100644 --- a/integration_tests/src/main/python/orc_cast_test.py +++ b/integration_tests/src/main/python/orc_cast_test.py @@ -66,6 +66,7 @@ def test_casting_from_integer(spark_tmp_path, to_type): schema_str.format(*([to_type] * len(data_gen)))).orc(orc_path) ) + @pytest.mark.parametrize('overflow_long_gen', [LongGen(min_val=int(1e16)), LongGen(max_val=int(-1e16))]) @pytest.mark.parametrize('to_type', ['timestamp']) @@ -81,3 +82,54 @@ def test_casting_from_overflow_long(spark_tmp_path, overflow_long_gen,to_type): conf={}, error_message="ArithmeticException" ) + + +# When casting float/double to double/float, we need to compare values of GPU with CPU +# in an approximate way. +@pytest.mark.approximate_float +@pytest.mark.parametrize('to_type', ['float', 'double', 'boolean', 'tinyint', 'smallint', 'int', 'bigint']) +def test_casting_from_float_and_double(spark_tmp_path, to_type): + orc_path = spark_tmp_path + '/orc_casting_from_float_and_double' + data_gen = [('float_column', float_gen), ('double_column', double_gen)] + create_orc(data_gen, orc_path) + schema_str = "float_column {}, double_column {}".format(to_type, to_type) + assert_gpu_and_cpu_are_equal_collect( + lambda spark: spark.read.schema(schema_str).orc(orc_path) + ) + + +@pytest.mark.parametrize('data_gen', [DoubleGen(max_exp=32, special_cases=None), + DoubleGen(max_exp=32, special_cases=[8.88e9, 9.99e10, 1.314e11])]) +def test_casting_from_double_to_timestamp(spark_tmp_path, data_gen): + # ORC will assume the original double value in seconds, we need to convert them to + # timestamp(INT64 in micro-seconds). + # + # The 'datetime' module in python requires 0 <= year < 10000, and UTC timestamp is start from 1970/1/1. + # That is, the minimum valid negative number is -1970 * 365 * 24 * 3600 = -62125920000 -> 6e10 -> 2^32. + # So we set max_exp = 32 in DoubleGen. + # + # The maximum valid positive number is (10000 - 1970) * 365 * 24 * 3600 = 253234080000 -> 2e11 -> 2^37, + # so we add some special cases from 2^33 - 2^37 (8e9 ~ 1e11). + # + # In DoubleGen, special_case=None will generate some NaN, INF corner cases. + + orc_path = spark_tmp_path + '/orc_casting_from_double_to_timestamp' + with_cpu_session( + lambda spark: unary_op_df(spark, data_gen).write.orc(orc_path) + ) + # the name of unique column is 'a', cast it into timestamp type + assert_gpu_and_cpu_are_equal_collect( + lambda spark: spark.read.schema("a timestamp").orc(orc_path) + ) + + +def test_casting_from_overflow_double_to_timestamp(spark_tmp_path): + orc_path = spark_tmp_path + '/orc_casting_from_overflow_double_to_timestamp' + with_cpu_session( + lambda spark: unary_op_df(spark, DoubleGen(min_exp=38)).write.orc(orc_path) + ) + assert_gpu_and_cpu_error( + df_fun=lambda spark: spark.read.schema("a timestamp").orc(orc_path).collect(), + conf={}, + error_message="ArithmeticException" + ) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala index 600669ccf55..f5e2f4cbac2 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala @@ -989,7 +989,7 @@ object GpuCast extends Arm { } } - private def castFloatingTypeToString(input: ColumnView): ColumnVector = { + private[rapids] def castFloatingTypeToString(input: ColumnView): ColumnVector = { withResource(input.castTo(DType.STRING)) { cudfCast => // replace "e+" with "E" diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index b8d5745b6bb..81b2b2e4655 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -122,6 +122,7 @@ case class GpuOrcScan( } object GpuOrcScan extends Arm { + def tagSupport(scanMeta: ScanMeta[OrcScan]): Unit = { val scan = scanMeta.wrapped val schema = StructType(scan.readDataSchema ++ scan.readPartitionSchema) @@ -186,6 +187,78 @@ object GpuOrcScan extends Arm { } } + /** + * Get the overflow flags in booleans. + * true means no overflow, while false means getting overflow. + * + * @param doubleMillis the input double column + * @param millis the long column casted from the doubleMillis + */ + private def getOverflowFlags(doubleMillis: ColumnView, millis: ColumnView): ColumnView = { + // No overflow when + // doubleMillis <= Long.MAX_VALUE && + // doubleMillis >= Long.MIN_VALUE && + // ((millis >= 0) == (doubleMillis >= 0)) + val rangeCheck = withResource(Scalar.fromLong(Long.MaxValue)) { max => + withResource(doubleMillis.lessOrEqualTo(max)) { upperCheck => + withResource(Scalar.fromLong(Long.MinValue)) { min => + withResource(doubleMillis.greaterOrEqualTo(min)) { lowerCheck => + upperCheck.and(lowerCheck) + } + } + } + } + withResource(rangeCheck) { _ => + val signCheck = withResource(Scalar.fromInt(0)) { zero => + withResource(millis.greaterOrEqualTo(zero)) { longSign => + withResource(doubleMillis.greaterOrEqualTo(zero)) { doubleSign => + longSign.equalTo(doubleSign) + } + } + } + withResource(signCheck) { _ => + rangeCheck.and(signCheck) + } + } + } + + /** + * Borrowed from ORC "ConvertTreeReaderFactory" + * Scala does not support such numeric literal, so parse from string. + */ + private val MIN_LONG_AS_DOUBLE = java.lang.Double.valueOf("-0x1p63") + + /** + * We cannot store Long.MAX_VALUE as a double without losing precision. Instead, we store + * Long.MAX_VALUE + 1 == -Long.MIN_VALUE, and then offset all comparisons by 1. + */ + private val MAX_LONG_AS_DOUBLE_PLUS_ONE = java.lang.Double.valueOf("0x1p63") + + /** + * Return a boolean column indicates whether the rows in col can fix in a long. + * It assumes the input type is float or double. + */ + private def doubleCanFitInLong(col: ColumnView): ColumnVector = { + // It is true when + // (MIN_LONG_AS_DOUBLE - doubleValue < 1.0) && + // (doubleValue < MAX_LONG_AS_DOUBLE_PLUS_ONE) + val lowRet = withResource(Scalar.fromDouble(MIN_LONG_AS_DOUBLE)) { sMin => + withResource(Scalar.fromDouble(1.0)) { sOne => + withResource(sMin.sub(col)) { diff => + diff.lessThan(sOne) + } + } + } + withResource(lowRet) { _ => + withResource(Scalar.fromDouble(MAX_LONG_AS_DOUBLE_PLUS_ONE)) { sMax => + withResource(col.lessThan(sMax)) { highRet => + lowRet.and(highRet) + } + } + } + } + + /** * Cast the column to the target type for ORC schema evolution. * It is designed to support all the cases that `canCast` returns true. @@ -233,6 +306,79 @@ object GpuOrcScan extends Arm { DType.TIMESTAMP_MICROSECONDS) => OrcCastingShims.castIntegerToTimestamp(col, fromDt) + // float to bool/integral + case (DType.FLOAT32 | DType.FLOAT64, DType.BOOL8 | DType.INT8 | DType.INT16 | DType.INT32 + | DType.INT64) => + // Follow the CPU ORC conversion: + // First replace rows that cannot fit in long with nulls, + // next convert to long, + // then down cast long to the target integral type. + val longDoubles = withResource(doubleCanFitInLong(col)) { fitLongs => + col.copyWithBooleanColumnAsValidity(fitLongs) + } + withResource(longDoubles) { _ => + withResource(longDoubles.castTo(DType.INT64)) { longs => + toDt match { + case DType.BOOL8 => longs.castTo(toDt) + case DType.INT64 => longs.incRefCount() + case _ => downCastAnyInteger(longs, toDt) + } + } + } + + // float/double to double/float + case (DType.FLOAT32 | DType.FLOAT64, DType.FLOAT32 | DType.FLOAT64) => + col.castTo(toDt) + + // float/double to string + // cuDF keep 9 decimal numbers after the decimal point, and CPU keeps more than 10. + // So when casting float/double to string, the result of GPU is different from CPU. + // We let a conf 'spark.rapids.sql.format.orc.floatTypesToString.enable' to control it's + // enable or not. + case (DType.FLOAT32 | DType.FLOAT64, DType.STRING) => + GpuCast.castFloatingTypeToString(col) + + // float/double -> timestamp + case (DType.FLOAT32 | DType.FLOAT64, DType.TIMESTAMP_MICROSECONDS) => + // Follow the CPU ORC conversion. + // val doubleMillis = doubleValue * 1000, + // val milliseconds = Math.round(doubleMillis) + // if (noOverflow) { milliseconds } else { null } + val milliseconds = withResource(Scalar.fromDouble(DateTimeConstants.MILLIS_PER_SECOND)) { + thousand => + // ORC assumes value is in seconds + withResource(col.mul(thousand, DType.FLOAT64)) { doubleMillis => + withResource(doubleMillis.round()) { millis => + withResource(getOverflowFlags(doubleMillis, millis)) { overflowFlags => + millis.copyWithBooleanColumnAsValidity(overflowFlags) + } + } + } + } + // Cast milli-seconds to micro-seconds + // We need to pay attention that when convert (milliSeconds * 1000) to INT64, there may be + // INT64-overflow. + // In this step, ORC casting of CPU throw an exception rather than replace such values with + // null. We followed the CPU code here. + withResource(milliseconds) { _ => + // Test whether if there is long-overflow + // If milliSeconds.max() * 1000 > LONG_MAX, then 'Math.multiplyExact' will + // throw an exception (as CPU code does). + withResource(milliseconds.max()) { maxValue => + if (maxValue.isValid) { + testLongMultiplicationOverflow(maxValue.getDouble.toLong, + DateTimeConstants.MICROS_PER_MILLIS) + } + } + withResource(Scalar.fromDouble(DateTimeConstants.MICROS_PER_MILLIS)) { thousand => + withResource(milliseconds.mul(thousand)) { microseconds => + withResource(microseconds.castTo(DType.INT64)) { longVec => + longVec.castTo(DType.TIMESTAMP_MICROSECONDS) + } + } + } + } + // TODO more types, tracked in https://github.com/NVIDIA/spark-rapids/issues/5895 case (f, t) => throw new QueryExecutionException(s"Unsupported type casting: $f -> $t") @@ -246,7 +392,8 @@ object GpuOrcScan extends Arm { * but the ones between GPU supported types. * Each supported casting is implemented in "castColumnTo". */ - def canCast(from: TypeDescription, to: TypeDescription): Boolean = { + def canCast(from: TypeDescription, to: TypeDescription, + isOrcFloatTypesToStringEnable: Boolean): Boolean = { import org.apache.orc.TypeDescription.Category._ if (!to.getCategory.isPrimitive || !from.getCategory.isPrimitive) { // Don't convert from any to complex, or from complex to any. @@ -268,7 +415,16 @@ object GpuOrcScan extends Arm { } case VARCHAR => toType == STRING - case _ => false + + case FLOAT | DOUBLE => + toType match { + case BOOLEAN | BYTE | SHORT | INT | LONG | FLOAT | DOUBLE | TIMESTAMP => true + case STRING => isOrcFloatTypesToStringEnable + case _ => false + } + // TODO more types, tracked in https://github.com/NVIDIA/spark-rapids/issues/5895 + case _ => + false } } @@ -366,7 +522,8 @@ case class GpuOrcMultiFilePartitionReaderFactory( private val debugDumpPrefix = Option(rapidsConf.orcDebugDumpPrefix) private val numThreads = rapidsConf.multiThreadReadNumThreads private val maxNumFileProcessed = rapidsConf.maxNumOrcFilesParallel - private val filterHandler = GpuOrcFileFilterHandler(sqlConf, broadcastedConf, filters) + private val filterHandler = GpuOrcFileFilterHandler(sqlConf, broadcastedConf, filters, + rapidsConf.isOrcFloatTypesToStringEnable) private val ignoreMissingFiles = sqlConf.ignoreMissingFiles private val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles @@ -458,7 +615,8 @@ case class GpuOrcPartitionReaderFactory( private val debugDumpPrefix = Option(rapidsConf.orcDebugDumpPrefix) private val maxReadBatchSizeRows: Integer = rapidsConf.maxReadBatchSizeRows private val maxReadBatchSizeBytes: Long = rapidsConf.maxReadBatchSizeBytes - private val filterHandler = GpuOrcFileFilterHandler(sqlConf, broadcastedConf, pushedFilters) + private val filterHandler = GpuOrcFileFilterHandler(sqlConf, broadcastedConf, pushedFilters, + rapidsConf.isOrcFloatTypesToStringEnable) override def supportColumnarReads(partition: InputPartition): Boolean = true @@ -993,7 +1151,8 @@ private object OrcTools extends Arm { private case class GpuOrcFileFilterHandler( @transient sqlConf: SQLConf, broadcastedConf: Broadcast[SerializableConfiguration], - pushedFilters: Array[Filter]) extends Arm { + pushedFilters: Array[Filter], + isOrcFloatTypesToStringEnable: Boolean) extends Arm { private[rapids] val isCaseSensitive = sqlConf.caseSensitiveAnalysis @@ -1088,7 +1247,7 @@ private case class GpuOrcFileFilterHandler( val isCaseSensitive = readerOpts.getIsSchemaEvolutionCaseAware val (updatedReadSchema, fileIncluded) = checkSchemaCompatibility(orcReader.getSchema, - readerOpts.getSchema, isCaseSensitive) + readerOpts.getSchema, isCaseSensitive, isOrcFloatTypesToStringEnable) // GPU has its own read schema, so unset the reader include to read all the columns // specified by its read schema. readerOpts.include(null) @@ -1268,11 +1427,13 @@ private case class GpuOrcFileFilterHandler( private def checkSchemaCompatibility( fileSchema: TypeDescription, readSchema: TypeDescription, - isCaseAware: Boolean): (TypeDescription, Array[Boolean]) = { + isCaseAware: Boolean, + isOrcFloatTypesToStringEnable: Boolean): (TypeDescription, Array[Boolean]) = { // all default to false val fileIncluded = new Array[Boolean](fileSchema.getMaximumId + 1) val isForcePos = OrcShims.forcePositionalEvolution(conf) - (checkTypeCompatibility(fileSchema, readSchema, isCaseAware, fileIncluded, isForcePos), + (checkTypeCompatibility(fileSchema, readSchema, isCaseAware, fileIncluded, isForcePos, + isOrcFloatTypesToStringEnable), fileIncluded) } @@ -1286,7 +1447,8 @@ private case class GpuOrcFileFilterHandler( readType: TypeDescription, isCaseAware: Boolean, fileIncluded: Array[Boolean], - isForcePos: Boolean): TypeDescription = { + isForcePos: Boolean, + isOrcFloatTypesToStringEnable: Boolean): TypeDescription = { (fileType.getCategory, readType.getCategory) match { case (TypeDescription.Category.STRUCT, TypeDescription.Category.STRUCT) => // Check for the top or nested struct types. @@ -1314,7 +1476,7 @@ private case class GpuOrcFileFilterHandler( .zipWithIndex.foreach { case ((fileFieldName, fType), idx) => getReadFieldType(fileFieldName, idx).foreach { case (rField, rType) => val newChild = checkTypeCompatibility(fType, rType, - isCaseAware, fileIncluded, isForcePos) + isCaseAware, fileIncluded, isForcePos, isOrcFloatTypesToStringEnable) prunedReadSchema.addField(rField, newChild) } } @@ -1324,19 +1486,22 @@ private case class GpuOrcFileFilterHandler( // for struct children. case (TypeDescription.Category.LIST, TypeDescription.Category.LIST) => val newChild = checkTypeCompatibility(fileType.getChildren.get(0), - readType.getChildren.get(0), isCaseAware, fileIncluded, isForcePos) + readType.getChildren.get(0), isCaseAware, fileIncluded, isForcePos, + isOrcFloatTypesToStringEnable) fileIncluded(fileType.getId) = true TypeDescription.createList(newChild) case (TypeDescription.Category.MAP, TypeDescription.Category.MAP) => val newKey = checkTypeCompatibility(fileType.getChildren.get(0), - readType.getChildren.get(0), isCaseAware, fileIncluded, isForcePos) + readType.getChildren.get(0), isCaseAware, fileIncluded, isForcePos, + isOrcFloatTypesToStringEnable) val newValue = checkTypeCompatibility(fileType.getChildren.get(1), - readType.getChildren.get(1), isCaseAware, fileIncluded, isForcePos) + readType.getChildren.get(1), isCaseAware, fileIncluded, isForcePos, + isOrcFloatTypesToStringEnable) fileIncluded(fileType.getId) = true TypeDescription.createMap(newKey, newValue) case (ft, rt) if ft.isPrimitive && rt.isPrimitive => if (OrcShims.typeDescriptionEqual(fileType, readType) || - GpuOrcScan.canCast(fileType, readType)) { + GpuOrcScan.canCast(fileType, readType, isOrcFloatTypesToStringEnable)) { // Since type casting is supported, here should return the file type. fileIncluded(fileType.getId) = true fileType.clone() diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 3c8183754d0..486743180e1 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -864,6 +864,19 @@ object RapidsConf { .booleanConf .createWithDefault(true) + val ENABLE_ORC_FLOAT_TYPES_TO_STRING = + conf("spark.rapids.sql.format.orc.floatTypesToString.enable") + .doc("When reading an ORC file, the source data schemas(schemas of ORC file) may differ " + + "from the target schemas (schemas of the reader), we need to handle the castings from " + + "source type to target type. Since float/double numbers in GPU have different precision " + + "with CPU, when casting float/double to string, the result of GPU is different from " + + "result of CPU spark. Its default value is `true` (this means the strings result will " + + "differ from result of CPU). If it's set `false` explicitly and there exists casting " + + "from float/double to string in the job, then such behavior will cause an exception, " + + "and the job will fail.") + .booleanConf + .createWithDefault(true) + val ORC_READER_TYPE = conf("spark.rapids.sql.format.orc.reader.type") .doc("Sets the ORC reader type. We support different types that are optimized for " + "different environments. The original Spark style reader can be selected by setting this " + @@ -1869,6 +1882,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isOrcWriteEnabled: Boolean = get(ENABLE_ORC_WRITE) + lazy val isOrcFloatTypesToStringEnable: Boolean = get(ENABLE_ORC_FLOAT_TYPES_TO_STRING) + lazy val isOrcPerFileReadEnabled: Boolean = RapidsReaderType.withName(get(ORC_READER_TYPE)) == RapidsReaderType.PERFILE From 1bcef3a58031205cb221027c80e2926cee535c7f Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Wed, 7 Sep 2022 13:47:13 -0500 Subject: [PATCH 091/190] Revert "Added in very specific support for from_json to a Map (#6211)" (#6510) This reverts commit 8b497c5b25ca80d410dee64d45d5592eba1ba307. Signed-off-by: Robert (Bobby) Evans --- docs/configs.md | 1 - docs/supported_ops.md | 47 ----- .../src/main/python/json_test.py | 11 -- .../nvidia/spark/rapids/GpuOverrides.scala | 14 -- .../catalyst/json/rapids/GpuJsonScan.scala | 90 ++++------ .../spark/sql/rapids/GpuJsonToStructs.scala | 161 ------------------ tools/src/main/resources/operatorsScore.csv | 1 - tools/src/main/resources/supportedExprs.csv | 2 - 8 files changed, 36 insertions(+), 291 deletions(-) delete mode 100644 sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala diff --git a/docs/configs.md b/docs/configs.md index 18560823614..032c6552fe1 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -251,7 +251,6 @@ Name | SQL Function(s) | Description | Default Value | Notes spark.rapids.sql.expression.IsNaN|`isnan`|Checks if a value is NaN|true|None| spark.rapids.sql.expression.IsNotNull|`isnotnull`|Checks if a value is not null|true|None| spark.rapids.sql.expression.IsNull|`isnull`|Checks if a value is null|true|None| -spark.rapids.sql.expression.JsonToStructs|`from_json`|Returns a struct value with the given `jsonStr` and `schema`|false|This is disabled by default because parsing JSON from a column has a large number of issues and should be considered beta quality right now.| spark.rapids.sql.expression.KnownFloatingPointNormalized| |Tag to prevent redundant normalization|true|None| spark.rapids.sql.expression.KnownNotNull| |Tag an expression as known to not be null|true|None| spark.rapids.sql.expression.Lag|`lag`|Window function that returns N entries behind this one|true|None| diff --git a/docs/supported_ops.md b/docs/supported_ops.md index d2c8b5ee370..64f12141e2b 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -7684,53 +7684,6 @@ are limited. -JsonToStructs -`from_json` -Returns a struct value with the given `jsonStr` and `schema` -This is disabled by default because parsing JSON from a column has a large number of issues and should be considered beta quality right now. -project -jsonStr - - - - - - - - - -S - - - - - - - - - - -result - - - - - - - - - - - - - - -NS -PS
unsupported child types BOOLEAN, BYTE, SHORT, INT, LONG, FLOAT, DOUBLE, DATE, TIMESTAMP, DECIMAL, NULL, BINARY, CALENDAR, ARRAY, MAP, STRUCT, UDT
-NS - - - KnownFloatingPointNormalized Tag to prevent redundant normalization diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index 18d1eadbd7c..6fe5e800291 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -359,14 +359,3 @@ def test_json_read_count(spark_tmp_path, v1_enabled_list): assert_gpu_and_cpu_row_counts_equal( lambda spark : spark.read.schema(schema).json(data_path), conf=updated_conf) - -def test_from_json_map(): - # The test here is working around some inconsistencies in how the keys are parsed for maps - # on the GPU the keys are dense, but on the CPU they are sparse - json_string_gen = StringGen("{\"a\": \"[0-9]{0,5}\"(, \"b\": \"[A-Z]{0,5}\")?}") - assert_gpu_and_cpu_are_equal_collect( - lambda spark : unary_op_df(spark, json_string_gen)\ - .selectExpr("from_json(a, \"MAP\") as parsed")\ - .selectExpr("parsed[\"a\"] as pa", "parsed[\"b\"] as pb"), - conf={"spark.rapids.sql.expression.JsonToStructs": "true"}) - diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 8657a707b35..402f31af96b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -3532,20 +3532,6 @@ object GpuOverrides extends Logging { GpuGetJsonObject(lhs, rhs) } ), - expr[JsonToStructs]( - "Returns a struct value with the given `jsonStr` and `schema`", - ExprChecks.projectOnly( - TypeSig.MAP.nested(TypeSig.STRING), - (TypeSig.STRUCT + TypeSig.MAP + TypeSig.ARRAY).nested(TypeSig.all), - Seq(ParamCheck("jsonStr", TypeSig.STRING, TypeSig.STRING))), - (a, conf, p, r) => new UnaryExprMeta[JsonToStructs](a, conf, p, r) { - override def tagExprForGpu(): Unit = - GpuJsonScan.tagJsonToStructsSupport(a.options, this) - - override def convertToGpu(child: Expression): GpuExpression = - GpuJsonToStructs(a.schema, a.options, child, a.timeZoneId) - }).disabledByDefault("parsing JSON from a column has a large number of issues and " + - "should be considered beta quality right now."), expr[org.apache.spark.sql.execution.ScalarSubquery]( "Subquery that will return only one row and one column", ExprChecks.projectOnly( diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala index 4dabbe52a9f..41a32ffd246 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala @@ -56,89 +56,71 @@ object GpuJsonScan { scanMeta) } - def tagSupportOptions( - options: JSONOptionsInRead, + def tagSupport( + sparkSession: SparkSession, + dataSchema: StructType, + readSchema: StructType, + options: Map[String, String], meta: RapidsMeta[_, _, _]): Unit = { - if (options.multiLine) { + val parsedOptions = new JSONOptionsInRead( + options, + sparkSession.sessionState.conf.sessionLocalTimeZone, + sparkSession.sessionState.conf.columnNameOfCorruptRecord) + + if (!meta.conf.isJsonEnabled) { + meta.willNotWorkOnGpu("JSON input and output has been disabled. To enable set " + + s"${RapidsConf.ENABLE_JSON} to true") + } + + if (!meta.conf.isJsonReadEnabled) { + meta.willNotWorkOnGpu("JSON input has been disabled. To enable set " + + s"${RapidsConf.ENABLE_JSON_READ} to true. Please note that, currently json reader does " + + s"not support column prune, so user must specify the full schema or just let spark to " + + s"infer the schema") + } + + if (parsedOptions.multiLine) { meta.willNotWorkOnGpu("GpuJsonScan does not support multiLine") } // {"name": /* hello */ "Reynold Xin"} is not supported by CUDF - if (options.allowComments) { + if (parsedOptions.allowComments) { meta.willNotWorkOnGpu("GpuJsonScan does not support allowComments") } // {name: 'Reynold Xin'} is not supported by CUDF - if (options.allowUnquotedFieldNames) { + if (parsedOptions.allowUnquotedFieldNames) { meta.willNotWorkOnGpu("GpuJsonScan does not support allowUnquotedFieldNames") } // {'name': 'Reynold Xin'} is not supported by CUDF - // This is different because the default for this is true, but we don't support it so we lie... - if (options.parameters.get("allowSingleQuotes").map(_.toBoolean).getOrElse(false)) { + if (options.get("allowSingleQuotes").map(_.toBoolean).getOrElse(false)) { meta.willNotWorkOnGpu("GpuJsonScan does not support allowSingleQuotes") } // {"name": "Cazen Lee", "price": "\$10"} is not supported by CUDF - if (options.allowBackslashEscapingAnyCharacter) { + if (parsedOptions.allowBackslashEscapingAnyCharacter) { meta.willNotWorkOnGpu("GpuJsonScan does not support allowBackslashEscapingAnyCharacter") } // {"a":null, "b":1, "c":3.0}, Spark will drop column `a` if dropFieldIfAllNull is enabled. - if (options.dropFieldIfAllNull) { + if (parsedOptions.dropFieldIfAllNull) { meta.willNotWorkOnGpu("GpuJsonScan does not support dropFieldIfAllNull") } - if (options.parseMode != PermissiveMode) { + if (parsedOptions.parseMode != PermissiveMode) { meta.willNotWorkOnGpu("GpuJsonScan only supports Permissive JSON parsing") } - if (options.lineSeparator.getOrElse("\n") != "\n") { + if (parsedOptions.lineSeparator.getOrElse("\n") != "\n") { meta.willNotWorkOnGpu("GpuJsonScan only supports \"\\n\" as a line separator") } - options.encoding.foreach(enc => + parsedOptions.encoding.foreach(enc => if (enc != StandardCharsets.UTF_8.name() && enc != StandardCharsets.US_ASCII.name()) { - meta.willNotWorkOnGpu("GpuJsonScan only supports UTF8 or US-ASCII encoded data") - }) - } - - def tagJsonToStructsSupport(options:Map[String, String], - meta: RapidsMeta[_, _, _]): Unit = { - val parsedOptions = new JSONOptionsInRead( - options, - SQLConf.get.sessionLocalTimeZone, - SQLConf.get.columnNameOfCorruptRecord) - - tagSupportOptions(parsedOptions, meta) - } - - def tagSupport( - sparkSession: SparkSession, - dataSchema: StructType, - readSchema: StructType, - options: Map[String, String], - meta: RapidsMeta[_, _, _]): Unit = { - - val parsedOptions = new JSONOptionsInRead( - options, - sparkSession.sessionState.conf.sessionLocalTimeZone, - sparkSession.sessionState.conf.columnNameOfCorruptRecord) - - if (!meta.conf.isJsonEnabled) { - meta.willNotWorkOnGpu("JSON input and output has been disabled. To enable set " + - s"${RapidsConf.ENABLE_JSON} to true") - } - - if (!meta.conf.isJsonReadEnabled) { - meta.willNotWorkOnGpu("JSON input has been disabled. To enable set " + - s"${RapidsConf.ENABLE_JSON_READ} to true. Please note that, currently json reader does " + - s"not support column prune, so user must specify the full schema or just let spark to " + - s"infer the schema") - } - - tagSupportOptions(parsedOptions, meta) + meta.willNotWorkOnGpu("GpuJsonScan only supports UTF8 or US-ASCII encoded data") + }) val types = readSchema.map(_.dataType) if (types.contains(DateType)) { @@ -154,17 +136,17 @@ object GpuJsonScan { if (!meta.conf.isJsonFloatReadEnabled && types.contains(FloatType)) { meta.willNotWorkOnGpu("JSON reading is not 100% compatible when reading floats. " + - s"To enable it please set ${RapidsConf.ENABLE_READ_JSON_FLOATS} to true.") + s"To enable it please set ${RapidsConf.ENABLE_READ_JSON_FLOATS} to true.") } if (!meta.conf.isJsonDoubleReadEnabled && types.contains(DoubleType)) { meta.willNotWorkOnGpu("JSON reading is not 100% compatible when reading doubles. " + - s"To enable it please set ${RapidsConf.ENABLE_READ_JSON_DOUBLES} to true.") + s"To enable it please set ${RapidsConf.ENABLE_READ_JSON_DOUBLES} to true.") } if (!meta.conf.isJsonDecimalReadEnabled && types.exists(_.isInstanceOf[DecimalType])) { meta.willNotWorkOnGpu("JSON reading is not 100% compatible when reading decimals. " + - s"To enable it please set ${RapidsConf.ENABLE_READ_JSON_DECIMALS} to true.") + s"To enable it please set ${RapidsConf.ENABLE_READ_JSON_DECIMALS} to true.") } dataSchema.getFieldIndex(parsedOptions.columnNameOfCorruptRecord).foreach { corruptFieldIndex => diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala deleted file mode 100644 index d12f8030c38..00000000000 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala +++ /dev/null @@ -1,161 +0,0 @@ -/* - * Copyright (c) 2022, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.rapids - -import ai.rapids.cudf -import com.nvidia.spark.rapids.{GpuColumnVector, GpuScalar, GpuUnaryExpression} -import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableProducingSeq - -import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, NullIntolerant, TimeZoneAwareExpression} -import org.apache.spark.sql.types.{AbstractDataType, DataType, StringType} - -case class GpuJsonToStructs( - schema: DataType, - options: Map[String, String], - child: Expression, - timeZoneId: Option[String] = None) - extends GpuUnaryExpression with TimeZoneAwareExpression with ExpectsInputTypes - with NullIntolerant { - - private def cleanAndConcat(input: cudf.ColumnVector): (cudf.ColumnVector, cudf.ColumnVector) ={ - withResource(cudf.Scalar.fromString("{}")) { emptyRow => - val stripped = withResource(cudf.Scalar.fromString(" ")) { space => - input.strip(space) - } - withResource(stripped) { stripped => - val isNullOrEmptyInput = withResource(input.isNull) { isNull => - val isEmpty = withResource(stripped.getCharLengths) { lengths => - withResource(cudf.Scalar.fromInt(0)) { zero => - lengths.lessOrEqualTo(zero) - } - } - withResource(isEmpty) { isEmpty => - isNull.binaryOp(cudf.BinaryOp.NULL_LOGICAL_OR, isEmpty, cudf.DType.BOOL8) - } - } - closeOnExcept(isNullOrEmptyInput) { _ => - withResource(isNullOrEmptyInput.ifElse(emptyRow, stripped)) { cleaned => - withResource(cudf.Scalar.fromString("\n")) { lineSep => - withResource(cleaned.stringContains(lineSep)) { inputHas => - withResource(inputHas.any()) { anyLineSep => - if (anyLineSep.isValid && anyLineSep.getBoolean) { - throw new IllegalArgumentException("We cannot currently support parsing " + - "JSON that contains a line separator in it") - } - } - } - (isNullOrEmptyInput, cleaned.joinStrings(lineSep, emptyRow)) - } - } - } - } - } - } - - private def castToStrings(rawTable: cudf.Table): Seq[cudf.ColumnVector] = { - (0 until rawTable.getNumberOfColumns).safeMap { i => - val col = rawTable.getColumn(i) - if (!cudf.DType.STRING.equals(col.getType)) { - col.castTo(cudf.DType.STRING) - } else { - col.incRefCount() - } - } - } - - private def makeMap(names: Seq[String], values: Seq[cudf.ColumnVector], - numRows: Int): cudf.ColumnVector = { - val nameCols = names.safeMap { name => - withResource(cudf.Scalar.fromString(name)) { scalarName => - cudf.ColumnVector.fromScalar(scalarName, numRows) - } - } - withResource(nameCols) { nameCols => - val structViews = values.zip(nameCols).safeMap { - case (dataCol, nameCol) => cudf.ColumnView.makeStructView(nameCol, dataCol) - } - withResource(structViews) { structViews => - cudf.ColumnVector.makeList(numRows, cudf.DType.STRUCT, structViews: _*) - } - } - } - - override protected def doColumnar(input: GpuColumnVector): cudf.ColumnVector = { - // We cannot handle all corner cases with this right now. The parser just isn't - // good enough, but we will try to handle a few common ones. - val numRows = input.getRowCount.toInt - - // Step 1: verify and preprocess the data to clean it up and normalize a few things. - // Step 2: Concat the data into a single buffer. - val (isNullOrEmpty, combined) = cleanAndConcat(input.getBase) - withResource(isNullOrEmpty) { isNullOrEmpty => - // Step 3: copy the data back to the host so we can parse it. - val combinedHost = withResource(combined) { combined => - combined.copyToHost() - } - // Step 4: Have cudf parse the JSON data - val (names, rawTable) = withResource(combinedHost) { combinedHost => - val data = combinedHost.getData - val start = combinedHost.getStartListOffset(0) - val end = combinedHost.getEndListOffset(0) - val length = end - start - - withResource(cudf.Table.readJSON(cudf.JSONOptions.DEFAULT, data, start, - length)) { tableWithMeta => - val names = tableWithMeta.getColumnNames - (names, tableWithMeta.releaseTable()) - } - } - - val updatedCols = withResource(rawTable) { rawTable => - // Step 5 verify that the data looks correct. - if (rawTable.getRowCount != numRows) { - throw new IllegalStateException("The input data didn't parse correctly and we read a " + - s"different number of rows than was expected. Expected $numRows, " + - s"but got ${rawTable.getRowCount}") - } - if (names.toSet.size != names.size) { - throw new IllegalStateException("Internal Error: found duplicate key names...") - } - - // Step 6: convert any non-string columns back to strings - castToStrings(rawTable) - } - - // Step 7: turn the data into a Map - val mapData = withResource(updatedCols) { updatedCols => - makeMap(names, updatedCols, numRows) - } - - // Step 8: put nulls back in for nulls and empty strings - withResource(mapData) { mapData => - withResource(GpuScalar.from(null, dataType)) { nullVal => - isNullOrEmpty.ifElse(nullVal, mapData) - } - } - } - } - - override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = - copy(timeZoneId = Option(timeZoneId)) - - override def inputTypes: Seq[AbstractDataType] = StringType :: Nil - - override def dataType: DataType = schema.asNullable - - override def nullable: Boolean = true -} \ No newline at end of file diff --git a/tools/src/main/resources/operatorsScore.csv b/tools/src/main/resources/operatorsScore.csv index ce62bb87fc5..f67f0f396c4 100644 --- a/tools/src/main/resources/operatorsScore.csv +++ b/tools/src/main/resources/operatorsScore.csv @@ -129,7 +129,6 @@ IntegralDivide,4 IsNaN,4 IsNotNull,4 IsNull,4 -JsonToStructs,4 KnownFloatingPointNormalized,4 KnownNotNull,4 Lag,4 diff --git a/tools/src/main/resources/supportedExprs.csv b/tools/src/main/resources/supportedExprs.csv index 7b93812944c..33fb60ec5d7 100644 --- a/tools/src/main/resources/supportedExprs.csv +++ b/tools/src/main/resources/supportedExprs.csv @@ -261,8 +261,6 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -JsonToStructs,NS,`from_json`,This is disabled by default because parsing JSON from a column has a large number of issues and should be considered beta quality right now.,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA -JsonToStructs,NS,`from_json`,This is disabled by default because parsing JSON from a column has a large number of issues and should be considered beta quality right now.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,NS,NA KnownFloatingPointNormalized,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S KnownFloatingPointNormalized,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S KnownNotNull,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,NS,S,S,PS,PS,PS,NS From 87c37d8a98c677b5ffc4e3d825d4fbf515f49b8a Mon Sep 17 00:00:00 2001 From: Jim Brennan Date: Thu, 8 Sep 2022 09:13:37 -0500 Subject: [PATCH 092/190] Use tiered projections for hash aggregates (#6466) Signed-off-by: Jim Brennan --- .../spark/rapids/GpuBoundAttribute.scala | 18 + .../com/nvidia/spark/rapids/RapidsConf.scala | 8 + .../com/nvidia/spark/rapids/aggregate.scala | 48 +- .../spark/rapids/basicPhysicalOperators.scala | 47 ++ .../expressions/GpuEquivalenExpressions.scala | 391 ++++++++++ .../GpuEquivalentExpressionsSuite.scala | 693 ++++++++++++++++++ 6 files changed, 1192 insertions(+), 13 deletions(-) create mode 100644 sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuEquivalenExpressions.scala create mode 100644 tests/src/test/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuEquivalentExpressionsSuite.scala diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBoundAttribute.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBoundAttribute.scala index 44ba329053d..191d6cca3f6 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBoundAttribute.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBoundAttribute.scala @@ -21,6 +21,7 @@ import com.nvidia.spark.rapids.shims.ShimExpression import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSeq, Expression, ExprId, SortOrder} +import org.apache.spark.sql.rapids.catalyst.expressions.GpuEquivalentExpressions import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -126,6 +127,23 @@ object GpuBindReferences extends Logging { // Force list to avoid recursive Java serialization of lazy list Seq implementation expressions.map(GpuBindReferences.bindReference(_, input)).toList } + + /** + * A helper function to bind given expressions to an input schema where the expressions are + * to be processed on the GPU, and the result type indicates this. Common sub-expressions + * bound with their inputs are placed into a sequence of tiers in a GpuTieredProject object. + */ + def bindGpuReferencesTiered[A <: Expression]( + expressions: Seq[A], + input: AttributeSeq): GpuTieredProject = { + + val exprTiers = GpuEquivalentExpressions.getExprTiers(expressions) + val inputTiers = GpuEquivalentExpressions.getInputTiers(exprTiers, input) + GpuTieredProject(exprTiers.zip(inputTiers).map { + case (es:Seq[Expression], is:AttributeSeq) => + es.map(GpuBindReferences.bindGpuReference(_, is)).toList + }) + } } case class GpuBoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 486743180e1..717446d6352 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -724,6 +724,12 @@ object RapidsConf { .booleanConf .createWithDefault(false) + val ENABLE_TIERED_PROJECT = conf("spark.rapids.sql.tiered.project.enabled") + .doc("Enable tiered project for aggregations.") + .internal() + .booleanConf + .createWithDefault(true) + // FILE FORMATS val MULTITHREAD_READ_NUM_THREADS = conf("spark.rapids.sql.multiThreadedRead.numThreads") .doc("The maximum number of threads on each executor to use for reading small " + @@ -1824,6 +1830,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isProjectAstEnabled: Boolean = get(ENABLE_PROJECT_AST) + lazy val isTieredProjectEnabled: Boolean = get(ENABLE_TIERED_PROJECT) + lazy val multiThreadReadNumThreads: Int = { // Use the largest value set among all the options. val deprecatedConfs = Seq( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala index 6eb222335cd..c7754990d68 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala @@ -177,6 +177,7 @@ object AggregateModeInfo { * @param modeInfo identifies which aggregation modes are being used * @param metrics metrics that will be updated during aggregation * @param configuredTargetBatchSize user-specified value for the targeted input batch size + * @param useTieredProject user-specified option to enable tiered projections */ class GpuHashAggregateIterator( cbIter: Iterator[ColumnarBatch], @@ -187,7 +188,8 @@ class GpuHashAggregateIterator( resultExpressions: Seq[NamedExpression], modeInfo: AggregateModeInfo, metrics: GpuHashAggregateMetrics, - configuredTargetBatchSize: Long) + configuredTargetBatchSize: Long, + useTieredProject: Boolean) extends Iterator[ColumnarBatch] with Arm with AutoCloseable with Logging { // Partial mode: @@ -278,7 +280,7 @@ class GpuHashAggregateIterator( /** Aggregate all input batches and place the results in the aggregatedBatches queue. */ private def aggregateInputBatches(): Unit = { - val aggHelper = new AggHelper(forceMerge = false) + val aggHelper = new AggHelper(forceMerge = false, useTieredProject = useTieredProject) while (cbIter.hasNext) { withResource(cbIter.next()) { childBatch => val isLastInputBatch = GpuColumnVector.isTaggedAsFinalBatch(childBatch) @@ -383,7 +385,8 @@ class GpuHashAggregateIterator( wasBatchMerged } - private lazy val concatAndMergeHelper = new AggHelper(forceMerge = true) + private lazy val concatAndMergeHelper = + new AggHelper(forceMerge = true, useTieredProject = useTieredProject) /** * Concatenate batches together and perform a merge aggregation on the result. The input batches @@ -465,7 +468,8 @@ class GpuHashAggregateIterator( new Iterator[ColumnarBatch] { override def hasNext: Boolean = keyBatchingIter.hasNext - private val mergeSortedHelper = new AggHelper(true, isSorted = true) + private val mergeSortedHelper = + new AggHelper(true, isSorted = true, useTieredProject = useTieredProject) override def next(): ColumnarBatch = { // batches coming out of the sort need to be merged @@ -627,8 +631,10 @@ class GpuHashAggregateIterator( * the merge steps for each aggregate function * @param isSorted - if the batch is sorted this is set to true and is passed to cuDF * as an optimization hint + * @param useTieredProject - if true, used tiered project for input projections */ - class AggHelper(forceMerge: Boolean, isSorted: Boolean = false) { + class AggHelper(forceMerge: Boolean, isSorted: Boolean = false, + useTieredProject : Boolean = true) { // `CudfAggregate` instances to apply, either update or merge aggregates private val cudfAggregates = new mutable.ArrayBuffer[CudfAggregate]() @@ -690,10 +696,16 @@ class GpuHashAggregateIterator( } // a bound expression that is applied before the cuDF aggregate - private val preStepBound = if (forceMerge) { - GpuBindReferences.bindGpuReferences(preStep.toList, aggBufferAttributes.toList) + private val preStepAttributes = if (forceMerge) { + aggBufferAttributes } else { - GpuBindReferences.bindGpuReferences(preStep, inputAttributes) + inputAttributes + } + private val (preStepBound, preStepBoundTiered) = if (useTieredProject) { + (None, Some(GpuBindReferences.bindGpuReferencesTiered(preStep.toList, + preStepAttributes.toList))) + } else { + (Some(GpuBindReferences.bindGpuReferences(preStep, preStepAttributes.toList)), None) } // a bound expression that is applied after the cuDF aggregate @@ -708,7 +720,11 @@ class GpuHashAggregateIterator( */ def preProcess(toAggregateBatch: ColumnarBatch): ColumnarBatch = { withResource(new NvtxRange("pre-process", NvtxColor.DARK_GREEN)) { _ => - GpuProjectExec.project(toAggregateBatch, preStepBound) + if (useTieredProject) { + preStepBoundTiered.get.tieredProject(toAggregateBatch) + } else { + GpuProjectExec.project(toAggregateBatch, preStepBound.get) + } } } @@ -990,7 +1006,8 @@ abstract class GpuBaseAggregateMeta[INPUT <: SparkPlan]( aggregateAttributes.map(_.convertToGpu().asInstanceOf[Attribute]), resultExpressions.map(_.convertToGpu().asInstanceOf[NamedExpression]), childPlans.head.convertIfNeeded(), - conf.gpuTargetBatchSizeBytes) + conf.gpuTargetBatchSizeBytes, + conf.isTieredProjectEnabled) } } @@ -1071,7 +1088,8 @@ abstract class GpuTypedImperativeSupportedAggregateExecMeta[INPUT <: BaseAggrega aggAttributes.map(_.convertToGpu().asInstanceOf[Attribute]), retExpressions.map(_.convertToGpu().asInstanceOf[NamedExpression]), childPlans.head.convertIfNeeded(), - conf.gpuTargetBatchSizeBytes) + conf.gpuTargetBatchSizeBytes, + conf.isTieredProjectEnabled) } else { super.convertToGpu() } @@ -1384,6 +1402,7 @@ class GpuObjectHashAggregateExecMeta( * node should project) * @param child incoming plan (where we get input columns from) * @param configuredTargetBatchSize user-configured maximum device memory size of a batch + * @param configuredTieredProjectEnabled configurable optimization to use tiered projections */ case class GpuHashAggregateExec( requiredChildDistributionExpressions: Option[Seq[Expression]], @@ -1392,7 +1411,8 @@ case class GpuHashAggregateExec( aggregateAttributes: Seq[Attribute], resultExpressions: Seq[NamedExpression], child: SparkPlan, - configuredTargetBatchSize: Long) extends ShimUnaryExecNode with GpuExec with Arm { + configuredTargetBatchSize: Long, + configuredTieredProjectEnabled: Boolean) extends ShimUnaryExecNode with GpuExec with Arm { // lifted directly from `BaseAggregateExec.inputAttributes`, edited comment. def inputAttributes: Seq[Attribute] = { @@ -1470,6 +1490,7 @@ case class GpuHashAggregateExec( val resultExprs = resultExpressions val modeInfo = AggregateModeInfo(uniqueModes) val targetBatchSize = configuredTargetBatchSize + val useTieredProject = configuredTieredProjectEnabled val rdd = child.executeColumnar() @@ -1483,7 +1504,8 @@ case class GpuHashAggregateExec( resultExprs, modeInfo, aggMetrics, - targetBatchSize) + targetBatchSize, + useTieredProject) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala index adf2ab6b668..35c6724364c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala @@ -253,6 +253,53 @@ case class GpuProjectAstExec( override def outputBatching: CoalesceGoal = GpuExec.outputBatching(child) } +/** + * Do projections in a tiered fashion, where earlier tiers contain sub-expressions that are + * referenced in later tiers. Each tier adds columns to the original batch corresponding + * to the output of the sub-expressions. + * Example of how this is processed: + * Original projection expressions: + * (((a + b) + c) * e), (((a + b) + d) * f), (a + e), (c + f) + * Input columns for tier 1: a, b, c, d, e, f (original projection inputs) + * Tier 1: (a + b) as ref1 + * Input columns for tier 2: a, b, c, d, e, f, ref1 + * Tier 2: (ref1 + c) as ref2, (ref1 + d) as ref3 + * Input columns for tier 3: a, b, c, d, e, f, ref1, ref2, ref3 + * Tier 3: (ref2 * e), (ref3 * f), (a + e), (c + f) + */ + case class GpuTieredProject(val exprSets: Seq[Seq[GpuExpression]]) extends Arm { + + @tailrec + private def projectTier(boundExprs: Seq[Seq[GpuExpression]], + cb: ColumnarBatch, doClose: Boolean): ColumnarBatch = { + boundExprs match { + case Nil => { + cb + } + case exprSet :: tail => { + val projectCb = withResource(new NvtxRange("project tier", NvtxColor.ORANGE)) { _ => + closeOnExcept(GpuProjectExec.project(cb, exprSet)) { projectResult => + projectResult + } + } + val nextCb = if (tail.isEmpty) { + projectCb + } else { + withResource(projectCb) { newCols => + GpuColumnVector.combineColumns(cb, newCols) + } + } + if (doClose) cb.close() + projectTier(tail, nextCb, true) + } + } + } + + def tieredProject(batch: ColumnarBatch): ColumnarBatch = { + projectTier(exprSets, batch, false) + } +} + /** * Run a filter on a batch. The batch will be consumed. */ diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuEquivalenExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuEquivalenExpressions.scala new file mode 100644 index 00000000000..5321cf881f5 --- /dev/null +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuEquivalenExpressions.scala @@ -0,0 +1,391 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * 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. + */ + +/* Note: This is derived from EquivalentExpressions in Apache Spark + * with changes to adapt it for GPU. + */ +package org.apache.spark.sql.rapids.catalyst.expressions + +import scala.annotation.tailrec +import scala.collection.mutable + +import com.nvidia.spark.rapids.{GpuAlias, GpuCaseWhen, GpuCoalesce, GpuExpression, GpuIf, GpuLeafExpression, GpuUnevaluable} + +import org.apache.spark.TaskContext +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSeq, CaseWhen, Coalesce, Expression, If, LeafExpression, PlanExpression} +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.expressions.objects.LambdaVariable + +/** + * This class is used to compute equality of (sub)expression trees. Expressions can be added + * to this class and they subsequently query for expression equality. Expression trees are + * considered equal if for the same input(s), the same result is produced. + */ +class GpuEquivalentExpressions { + // For each expression, the set of equivalent expressions. + private val equivalenceMap = mutable.HashMap.empty[GpuExpressionEquals, GpuExpressionStats] + + /** + * Adds each expression to this data structure, grouping them with existing equivalent + * expressions. Non-recursive. + * Returns true if there was already a matching expression. + */ + def addExpr(expr: Expression): Boolean = { + addExprToMap(expr, equivalenceMap) + } + + private def addExprToMap( + expr: Expression, map: mutable.HashMap[GpuExpressionEquals, GpuExpressionStats]): Boolean = { + if (expr.deterministic) { + val wrapper = GpuExpressionEquals(expr) + map.get(wrapper) match { + case Some(stats) => + stats.useCount += 1 + true + case _ => + map.put(wrapper, GpuExpressionStats(expr)()) + false + } + } else { + false + } + } + + /** + * Adds only expressions which are common in each of given expressions, in a recursive way. + * For example, given two expressions `(a + (b + (c + 1)))` and `(d + (e + (c + 1)))`, + * the common expression `(c + 1)` will be added into `equivalenceMap`. + * + * Note that as we don't know in advance if any child node of an expression will be common + * across all given expressions, we count all child nodes when looking through the given + * expressions. But when we call `addExprTree` to add common expressions into the map, we + * will add recursively the child nodes. So we need to filter the child expressions first. + * For example, if `((a + b) + c)` and `(a + b)` are common expressions, we only add + * `((a + b) + c)`. + */ + private def addCommonExprs( + exprs: Seq[Expression], + map: mutable.HashMap[GpuExpressionEquals, GpuExpressionStats]): Unit = { + assert(exprs.length > 1) + var localEquivalenceMap = mutable.HashMap.empty[GpuExpressionEquals, GpuExpressionStats] + addExprTree(exprs.head, localEquivalenceMap) + + exprs.tail.foreach { expr => + val otherLocalEquivalenceMap = mutable.HashMap.empty[GpuExpressionEquals, GpuExpressionStats] + addExprTree(expr, otherLocalEquivalenceMap) + localEquivalenceMap = localEquivalenceMap.filter { case (key, _) => + otherLocalEquivalenceMap.contains(key) + } + } + + localEquivalenceMap.foreach { case (commonExpr, state) => + val possibleParents = localEquivalenceMap.filter { case (_, v) => v.height > state.height } + val notChild = possibleParents.forall { case (k, _) => + k == commonExpr || k.e.find(_.semanticEquals(commonExpr.e)).isEmpty + } + if (notChild) { + // If the `commonExpr` already appears in the equivalence map, calling `addExprTree` will + // increase the `useCount` and mark it as a common subexpression. Otherwise, `addExprTree` + // will recursively add `commonExpr` and its descendant to the equivalence map, in case + // they also appear in other places. For example, `If(a + b > 1, a + b + c, a + b + c)`, + // `a + b` also appears in the condition and should be treated as common subexpression. + addExprTree(commonExpr.e, map) + } + } + } + + // There are some special expressions that we should not recurse into all of its children. + // 1. CodegenFallback: it's children will not be used to generate code (call eval() instead) + // 2. If/GpuIf: common subexpressions will always be evaluated at the beginning, but the true + // and false expressions in `If` may not get accessed, according to the predicate + // expression. We should only recurse into the predicate expression. + // 3. CaseWhen/GpuCaseWhen: like `If`, the children of `CaseWhen` only get accessed in a certain + // condition. We should only recurse into the first condition expression as it + // will always get accessed. + // 4. Coalesce/GpuCoalesce: it's also a conditional expression, we should only recurse into the + // first children, because others may not get accessed. + private def childrenToRecurse(expr: Expression): Seq[Expression] = expr match { + case _: CodegenFallback => Nil + case i: If => i.predicate :: Nil + case i: GpuIf => i.predicateExpr :: Nil + case c: CaseWhen => c.children.head :: Nil + case c: GpuCaseWhen => c.children.head :: Nil + case c: Coalesce => c.children.head :: Nil + case c: GpuCoalesce => c.children.head :: Nil + case other => other.children + } + + // For some special expressions we cannot just recurse into all of its children, but we can + // recursively add the common expressions shared between all of its children. + private def commonChildrenToRecurse(expr: Expression): Seq[Seq[Expression]] = expr match { + case _: CodegenFallback => Nil + case i: If => Seq(Seq(i.trueValue, i.falseValue)) + case i: GpuIf => Seq(Seq(i.trueExpr, i.falseExpr)) + case c: CaseWhen => + // We look at subexpressions in conditions and values of `CaseWhen` separately. It is + // because a subexpression in conditions will be run no matter which condition is matched + // if it is shared among conditions, but it doesn't need to be shared in values. Similarly, + // a subexpression among values doesn't need to be in conditions because no matter which + // condition is true, it will be evaluated. + val conditions = if (c.branches.length > 1) { + c.branches.map(_._1) + } else { + // If there is only one branch, the first condition is already covered by + // `childrenToRecurse` and we should exclude it here. + Nil + } + // For an expression to be in all branch values of a CaseWhen statement, it must also be in + // the elseValue. + val values = if (c.elseValue.nonEmpty) { + c.branches.map(_._2) ++ c.elseValue + } else { + Nil + } + Seq(conditions, values) + case c: GpuCaseWhen => + // We look at subexpressions in conditions and values of `CaseWhen` separately. It is + // because a subexpression in conditions will be run no matter which condition is matched + // if it is shared among conditions, but it doesn't need to be shared in values. Similarly, + // a subexpression among values doesn't need to be in conditions because no matter which + // condition is true, it will be evaluated. + val conditions = if (c.branches.length > 1) { + c.branches.map(_._1) + } else { + // If there is only one branch, the first condition is already covered by + // `childrenToRecurse` and we should exclude it here. + Nil + } + // For an expression to be in all branch values of a CaseWhen statement, it must also be in + // the elseValue. + val values = if (c.elseValue.nonEmpty) { + c.branches.map(_._2) ++ c.elseValue + } else { + Nil + } + Seq(conditions, values) + // If there is only one child, the first child is already covered by + // `childrenToRecurse` and we should exclude it here. + case c: Coalesce if c.children.length > 1 => Seq(c.children) + case c: GpuCoalesce if c.children.length > 1 => Seq(c.children) + case _ => Nil + } + + /** + * Adds the expression to this data structure recursively. Stops if a matching expression + * is found. That is, if `expr` has already been added, its children are not added. + */ + def addExprTree( + expr: Expression, + map: mutable.HashMap[GpuExpressionEquals, GpuExpressionStats] = equivalenceMap): Unit = { + val skip = expr.isInstanceOf[LeafExpression] || + expr.isInstanceOf[GpuLeafExpression] || + expr.isInstanceOf[GpuUnevaluable] || + (expr.isInstanceOf[GpuExpression] && expr.asInstanceOf[GpuExpression].hasSideEffects) || + // `LambdaVariable` is usually used as a loop variable, which can't be evaluated ahead of the + // loop. So we can't evaluate sub-expressions containing `LambdaVariable` at the beginning. + expr.find(_.isInstanceOf[LambdaVariable]).isDefined || + // `PlanExpression` wraps query plan. To compare query plans of `PlanExpression` on executor, + // can cause error like NPE. + (expr.find(_.isInstanceOf[PlanExpression[_]]).isDefined && TaskContext.get != null) + + if (!skip && !addExprToMap(expr, map)) { + childrenToRecurse(expr).foreach(addExprTree(_, map)) + commonChildrenToRecurse(expr).filter(_.nonEmpty).foreach(addCommonExprs(_, map)) + } + } + + /** + * Returns the state of the given expression in the `equivalenceMap`. Returns None if there is no + * equivalent expressions. + * Exposed for testing. + */ + private[sql] def getExprState(e: Expression): Option[GpuExpressionStats] = { + equivalenceMap.get(GpuExpressionEquals(e)) + } + + // Exposed for testing. + private[sql] def getAllExprStates(count: Int = 0): Seq[GpuExpressionStats] = { + equivalenceMap.values.filter(_.useCount > count).toSeq.sortBy(_.height) + } + + /** + * Returns a sequence of expressions that more than one equivalent expressions. + */ + def getCommonSubexpressions: Seq[Expression] = { + getAllExprStates(1).map(_.expr) + } + + /** + * Returns the state of the data structure as a string. If `all` is false, skips sets of + * equivalent expressions with cardinality 1. + */ + def debugString(all: Boolean = false): String = { + val sb = new java.lang.StringBuilder() + sb.append("GPU Equivalent expressions:\n") + equivalenceMap.values.filter(stats => all || stats.useCount > 1).foreach { stats => + sb.append(" ").append(s"${stats.expr}: useCount = ${stats.useCount}").append('\n') + } + sb.toString() + } +} + +object GpuEquivalentExpressions { + /** + * Recursively replaces expression with its proxy expression in `substitutionMap`. + */ + private def replaceWithCommonRef( + expr: Expression, + substitutionMap: mutable.HashMap[Expression, Expression]): Expression = { + expr match { + case e: AttributeReference => e + case _ => + substitutionMap.get(expr) match { + case Some(attr) => attr + case None => expr.mapChildren(replaceWithCommonRef(_, substitutionMap)) + } + } + } + + /** + * Recursively calls getCommonSubexpressions to create tiers + * of expressions, where earlier tiers contain subexpressions + * for later tiers. + */ + @tailrec + private def recurseCommonExpressions(exprs: Seq[Expression], + exprTiers: Seq[Seq[Expression]]): Seq[Seq[Expression]] = { + val equivalentExpressions = new GpuEquivalentExpressions + exprs.foreach(equivalentExpressions.addExprTree(_)) + val commonExprs = equivalentExpressions.getCommonSubexpressions + if (commonExprs.isEmpty) { + exprTiers + } else { + recurseCommonExpressions(commonExprs, (Seq(commonExprs) ++ exprTiers)) + } + } + + /** + * Applies substitutions to all expression tiers. + */ + private def doSubstitutions(exprTiers: Seq[Seq[Expression]], currentTier: Seq[Expression], + substitutionMap: mutable.HashMap[Expression, Expression]): Seq[Seq[Expression]] = { + // Make substitutions in given tiers, filtering out matches from original current tier, + // but don't filter the last tier - it needs to match original size + val subTiers = exprTiers.dropRight(1) + val lastTier = exprTiers.last + val updatedSubTiers = subTiers.map { + t => t.filter(e => !currentTier.contains(e)).map(replaceWithCommonRef(_, substitutionMap)) + } + val updatedLastTier = lastTier.map(replaceWithCommonRef(_, substitutionMap)) + updatedSubTiers ++ Seq(updatedLastTier) + } + + /** + * Apply subexpression substitutions to all tiers. + */ + @tailrec + private def recurseUpdateTiers(exprTiers: Seq[Seq[Expression]], + updatedTiers: Seq[Seq[Expression]], + substitutionMap: mutable.HashMap[Expression, Expression], + startIndex: Int):Seq[Seq[Expression]] = { + exprTiers match { + case Nil => updatedTiers + case tier :: tail => { + // Last tier should already be updated. + if (tail.isEmpty) { + updatedTiers ++ Seq(tier) + } else { + // Replace expressions in this tier with GpuAlias + val aliasedTier = tier.zipWithIndex.map { + case (e, i) => + GpuAlias(e, s"tiered_input_${startIndex + i}")() + } + // Add them to the map + tier.zip(aliasedTier).foreach { + case (expr, alias) => { + substitutionMap.get(expr) match { + case None => substitutionMap.put(expr, alias.toAttribute) + case Some(e) => + } + } + } + val newUpdatedTiers = doSubstitutions(tail, tier, substitutionMap) + recurseUpdateTiers(newUpdatedTiers, updatedTiers ++ Seq(aliasedTier), + substitutionMap, startIndex + aliasedTier.size) + } + } + } + } + + def getExprTiers(expressions : Seq[Expression]): Seq[Seq[Expression]] = { + // Get tiers of common expressions + val expressionTiers = recurseCommonExpressions(expressions, Seq(expressions)) + val substitutionMap = mutable.HashMap.empty[Expression, Expression] + // Update expression with common expressions from previous tiers + recurseUpdateTiers(expressionTiers, Seq.empty, substitutionMap, 0) + } + + // Given expression tiers as created by getExprTiers and a set of input attributes, + // return the tiers of input attributes that correspond with the expression tiers. + def getInputTiers(exprTiers: Seq[Seq[Expression]], inputAttrs: AttributeSeq): + Seq[AttributeSeq] = { + def recurse(et: Seq[Seq[Expression]], inputs: AttributeSeq): Seq[AttributeSeq] = { + et match { + case Nil => Nil + case s :: tail => { + val newInputs = if (tail.isEmpty) { + Nil + } else { + s.filter(e => e.isInstanceOf[GpuAlias]).map(_.asInstanceOf[GpuAlias].toAttribute) + } + val attrSeq = AttributeSeq(inputs.attrs ++ newInputs) + val recursionResult = recurse(tail, attrSeq) + Seq(inputs) ++ recursionResult + } + } + } + recurse(exprTiers, inputAttrs) + } +} + +/** + * Wrapper around an Expression that provides semantic equality. + */ +case class GpuExpressionEquals(e: Expression) { + override def equals(o: Any): Boolean = o match { + case other: GpuExpressionEquals => e.semanticEquals(other.e) + case _ => false + } + + override def hashCode: Int = e.semanticHash() +} + +/** + * A wrapper in place of using Seq[Expression] to record a group of equivalent expressions. + * + * This saves a lot of memory when there are a lot of expressions in a same equivalence group. + * Instead of appending to a mutable list/buffer of Expressions, just update the "flattened" + * useCount in this wrapper in-place. + */ +case class GpuExpressionStats(expr: Expression)(var useCount: Int = 1) { + // This is used to do a fast pre-check for child-parent relationship. For example, expr1 can + // only be a parent of expr2 if expr1.height is larger than expr2.height. + lazy val height = getHeight(expr) + + private def getHeight(tree: Expression): Int = { + tree.children.map(getHeight).reduceOption(_ max _).getOrElse(0) + 1 + } +} \ No newline at end of file diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuEquivalentExpressionsSuite.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuEquivalentExpressionsSuite.scala new file mode 100644 index 00000000000..8fd62dec4b0 --- /dev/null +++ b/tests/src/test/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuEquivalentExpressionsSuite.scala @@ -0,0 +1,693 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.rapids.catalyst.expressions + +import com.nvidia.spark.rapids.{GpuAlias, GpuCaseWhen, GpuCast, GpuCoalesce, GpuIf, GpuIsNull, GpuLiteral, GpuMonotonicallyIncreasingID} +import org.scalatest.FunSuite + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSeq, Expression} +import org.apache.spark.sql.rapids.{GpuAbs, GpuAdd, GpuAnd, GpuDecimalMultiply, GpuExtractChunk32, GpuGreaterThan, GpuLessThanOrEqual, GpuMultiply, GpuSqrt, GpuSubtract} +import org.apache.spark.sql.types.{DecimalType, DoubleType, IntegerType, StringType} + +/* + * Many of these tests were derived from SubexpressionEliminationSuite in Apache Spark, + * and changed to use GPU expressions. + */ +class GpuEquivalentExpressionsSuite extends FunSuite with Logging { + + test("Gpu Expression Equivalence - basic") { + val equivalence = new GpuEquivalentExpressions + assert(equivalence.getAllExprStates().isEmpty) + + val oneA = GpuLiteral(1) + val oneB = GpuLiteral(1) + val twoA = GpuLiteral(2) + + assert(equivalence.getExprState(oneA).isEmpty) + assert(equivalence.getExprState(twoA).isEmpty) + + // GpuAdd oneA and test if it is returned. Since it is a group of one, it does not. + assert(!equivalence.addExpr(oneA)) + assert(equivalence.getExprState(oneA).get.useCount == 1) + assert(equivalence.getExprState(twoA).isEmpty) + assert(equivalence.addExpr(oneA)) + assert(equivalence.getExprState(oneA).get.useCount == 2) + + // GpuAdd B and make sure they can see each other. + assert(equivalence.addExpr(oneB)) + // Use exists and reference equality because of how equals is defined. + assert(equivalence.getExprState(oneA).exists(_.expr eq oneA)) + assert(equivalence.getExprState(oneB).exists(_.expr eq oneA)) + assert(equivalence.getExprState(twoA).isEmpty) + assert(equivalence.getAllExprStates().size == 1) + assert(equivalence.getAllExprStates().head.useCount == 3) + assert(equivalence.getAllExprStates().head.expr eq oneA) + + val add1 = GpuAdd(oneA, oneB, false) + val add2 = GpuAdd(oneA, oneB, false) + + equivalence.addExpr(add1) + equivalence.addExpr(add2) + + assert(equivalence.getAllExprStates().size == 2) + assert(equivalence.getExprState(add1).exists(_.expr eq add1)) + assert(equivalence.getExprState(add2).get.useCount == 2) + assert(equivalence.getExprState(add2).exists(_.expr eq add1)) + } + + test("Get Expression Tiers - two the same") { + val oneA = AttributeReference("oneA", IntegerType)() + val oneB = AttributeReference("oneB", IntegerType)() + + val add1 = GpuAdd(oneA, oneB, false) + val add2 = GpuAdd(oneA, oneB, false) + + val initialExprs = Seq(add1, add2) + val inputAttrs = AttributeSeq(Seq(oneA, oneB)) + validateExpressionTiers(initialExprs, inputAttrs, + Seq(add1, add2), // Both are the same, so sub-expression should match both + Seq(), + Seq(add1, add2)) // Both will be updated + } + + test("GpuExpression Equivalence - Trees") { + val one = GpuLiteral(1) + val two = GpuLiteral(2) + + val add = GpuAdd(one, two, false) + val abs = GpuAbs(add, false) + val add2 = GpuAdd(add, add, false) + + var equivalence = new GpuEquivalentExpressions + equivalence.addExprTree(add) + equivalence.addExprTree(abs) + equivalence.addExprTree(add2) + + // Should only have one equivalence for `one + two` + assert(equivalence.getAllExprStates(1).size == 1) + assert(equivalence.getAllExprStates(1).head.useCount == 4) + + // Set up the expressions + // one * two, + // (one * two) * (one * two) + // sqrt( (one * two) * (one * two) ) + // (one * two) + sqrt( (one * two) * (one * two) ) + equivalence = new GpuEquivalentExpressions + val mul = GpuMultiply(one, two) + val mul2 = GpuMultiply(mul, mul) + val sqrt = GpuSqrt(mul2) + val sum = GpuAdd(mul2, sqrt, false) + equivalence.addExprTree(mul) + equivalence.addExprTree(mul2) + equivalence.addExprTree(sqrt) + equivalence.addExprTree(sum) + + // (one * two), (one * two) * (one * two) and sqrt( (one * two) * (one * two) ) should be found + assert(equivalence.getAllExprStates(1).size == 3) + assert(equivalence.getExprState(mul).get.useCount == 3) + assert(equivalence.getExprState(mul2).get.useCount == 3) + assert(equivalence.getExprState(sqrt).get.useCount == 2) + assert(equivalence.getExprState(sum).get.useCount == 1) + } + + test("Get Expression Tiers - Trees") { + // Set up the expressions + // one * two, + // (one * two) * (one * two) + // sqrt( (one * two) * (one * two) ) + // (one * two) + sqrt( (one * two) * (one * two) ) + val one = AttributeReference("one", DoubleType)() + val two = AttributeReference("two", DoubleType)() + val mul = GpuMultiply(one, two) + val mul2 = GpuMultiply(mul, mul) + val sqrt = GpuSqrt(mul2) + val sum = GpuAdd(mul2, sqrt, false) + + // (one * two), (one * two) * (one * two) and sqrt( (one * two) * (one * two) ) are all subs + val initialExprs = Seq(mul, mul2, sqrt, sum) + val inputAttrs = AttributeSeq(Seq(one, two)) + validateExpressionTiers(initialExprs, inputAttrs, + Seq(mul, mul2, sqrt), + Seq(), + Seq(mul, mul2, sqrt, sum)) + } + + test("Get Expression Tiers - simple example") { + // Set up the expressions + // one + two, + // (one + two) + three + // ((one + two) + three) + four + val one = AttributeReference("one", IntegerType)() + val two = AttributeReference("two", IntegerType)() + val three = AttributeReference("three", IntegerType)() + val four = AttributeReference("four", IntegerType)() + val add1 = GpuAdd(one, two, false) + val add2 = GpuAdd(add1, three, false) + val add3 = GpuAdd(add2, four, false) + + // (one + two), ((one + two) + three) are both subs + val initialExprs = Seq(add1, add2, add3) + val inputAttrs = AttributeSeq(Seq(one, two, three, four)) + validateExpressionTiers(initialExprs, inputAttrs, + Seq(add1, add2), //subexpressions + Seq(), // no unchanged + Seq(add1, add2, add3)) // all original expressions are updated + } + + test("Gpu Expression equivalence - non deterministic") { + val sum = GpuAdd(GpuMonotonicallyIncreasingID(), GpuMonotonicallyIncreasingID(), false) + val equivalence = new GpuEquivalentExpressions + equivalence.addExpr(sum) + equivalence.addExpr(sum) + assert(equivalence.getAllExprStates().isEmpty) + } + + test("Get Expression Tiers - non deterministic") { + val sum = GpuAdd(GpuMonotonicallyIncreasingID(), GpuMonotonicallyIncreasingID(), false) + val initialExprs = Seq(sum) + val inputAttrs = AttributeSeq(Seq.empty) + validateExpressionTiers(initialExprs, inputAttrs, + Seq.empty, // No subexpressions + Seq(sum), // Should be unchanged + Seq.empty) // No modified expressions + } + + test("Children of conditional expressions: GpuIf") { + val add = GpuAdd(GpuLiteral(1), GpuLiteral(2), false) + val condition = GpuGreaterThan(add, GpuLiteral(3)) + + val ifExpr1 = GpuIf(condition, add, add) + val equivalence1 = new GpuEquivalentExpressions + equivalence1.addExprTree(ifExpr1) + + // `add` is in both two branches of `If` and predicate. + assert(equivalence1.getAllExprStates().count(_.useCount == 2) == 1) + assert(equivalence1.getAllExprStates().filter(_.useCount == 2).head.expr eq add) + // one-time expressions: only ifExpr and its predicate expression + assert(equivalence1.getAllExprStates().count(_.useCount == 1) == 2) + assert(equivalence1.getAllExprStates().filter(_.useCount == 1).exists(_.expr eq ifExpr1)) + assert(equivalence1.getAllExprStates().filter(_.useCount == 1).exists(_.expr eq condition)) + + // Repeated `add` is only in one branch, so we don't count it. + val ifExpr2 = GpuIf(condition, GpuAdd(GpuLiteral(1), GpuLiteral(3), false), + GpuAdd(add, add, false)) + val equivalence2 = new GpuEquivalentExpressions + equivalence2.addExprTree(ifExpr2) + + assert(equivalence2.getAllExprStates(1).isEmpty) + assert(equivalence2.getAllExprStates().count(_.useCount == 1) == 3) + + val ifExpr3 = GpuIf(condition, ifExpr1, ifExpr1) + val equivalence3 = new GpuEquivalentExpressions + equivalence3.addExprTree(ifExpr3) + + // `add`: 2, `condition`: 2 + assert(equivalence3.getAllExprStates().count(_.useCount == 2) == 2) + assert(equivalence3.getAllExprStates().filter(_.useCount == 2).exists(_.expr eq condition)) + assert(equivalence3.getAllExprStates().filter(_.useCount == 2).exists(_.expr eq add)) + + // `ifExpr1`, `ifExpr3` + assert(equivalence3.getAllExprStates().count(_.useCount == 1) == 2) + assert(equivalence3.getAllExprStates().filter(_.useCount == 1).exists(_.expr eq ifExpr1)) + assert(equivalence3.getAllExprStates().filter(_.useCount == 1).exists(_.expr eq ifExpr3)) + } + + test("Get Expression Tiers GpuIf") { + val one = AttributeReference("one", IntegerType)() + val two = AttributeReference("two", IntegerType)() + val three = AttributeReference("three", IntegerType)() + val add = GpuAdd(one, two, false) + val condition = GpuGreaterThan(add, three) + // if ((one + two) > three) then (one + two) else (one + two) + // `add` is in both branches of `If` and predicate. + val ifExpr1 = GpuIf(condition, add, add) + val initialExprs = Seq(ifExpr1) + val inputAttrs = AttributeSeq(Seq(one, two, three)) + validateExpressionTiers(initialExprs, inputAttrs, + Seq(add), // subexpressions + Seq.empty, // Should be unchanged + Seq(ifExpr1)) // modified expressions + + // if ((one + two) > three) then (one + three) else ((one + two) + (one + two)) + // Repeated `add` is only in one branch, so we don't count it. + val ifExpr2 = GpuIf(condition, GpuAdd(one, three, false), + GpuAdd(add, add, false)) + val initialExprs2 = Seq(ifExpr2) + val inputAttrs2 = AttributeSeq(Seq(one, two, three)) + validateExpressionTiers(initialExprs2, inputAttrs2, + Seq.empty, // subexpressions + Seq(ifExpr2), // Should be unchanged + Seq.empty) // modified expressions + + // if ((one + two) > three) + // if ((one + two) > three) then (one + two) else (one + two) + // else + // if ((one + two) > three) then (one + two) else (one + two) + val ifExpr3 = GpuIf(condition, ifExpr1, ifExpr1) + val initialExprs3 = Seq(ifExpr3) + val inputAttrs3 = AttributeSeq(Seq(one, two, three)) + validateExpressionTiers(initialExprs3, inputAttrs3, + Seq(add, condition), // subexpressions + Seq.empty, // Should be unchanged + Seq(condition, ifExpr1, ifExpr3)) // modified expressions + } + + test("Children of conditional expressions: GpuCaseWhen") { + val add1 = GpuAdd(GpuLiteral(1), GpuLiteral(2), false) + val add2 = GpuAdd(GpuLiteral(2), GpuLiteral(3), false) + val conditions1 = (GpuGreaterThan(add2, GpuLiteral(3)), add1) :: + (GpuGreaterThan(add2, GpuLiteral(4)), add1) :: + (GpuGreaterThan(add2, GpuLiteral(5)), add1) :: Nil + + val caseWhenExpr1 = GpuCaseWhen(conditions1, None) + val equivalence1 = new GpuEquivalentExpressions + equivalence1.addExprTree(caseWhenExpr1) + + // `add2` is repeatedly in all conditions. + assert(equivalence1.getAllExprStates().count(_.useCount == 2) == 1) + assert(equivalence1.getAllExprStates().filter(_.useCount == 2).head.expr eq add2) + + val conditions2 = (GpuGreaterThan(add1, GpuLiteral(3)), add1) :: + (GpuGreaterThan(add2, GpuLiteral(4)), add1) :: + (GpuGreaterThan(add2, GpuLiteral(5)), add1) :: Nil + + val caseWhenExpr2 = GpuCaseWhen(conditions2, Some(add1)) + val equivalence2 = new GpuEquivalentExpressions + equivalence2.addExprTree(caseWhenExpr2) + + // `add1` is repeatedly in all branch values, and first predicate. + assert(equivalence2.getAllExprStates().count(_.useCount == 2) == 1) + assert(equivalence2.getAllExprStates().filter(_.useCount == 2).head.expr eq add1) + + // Negative case. `add1` or `add2` is not commonly used in all predicates/branch values. + val conditions3 = (GpuGreaterThan(add1, GpuLiteral(3)), add2) :: + (GpuGreaterThan(add2, GpuLiteral(4)), add1) :: + (GpuGreaterThan(add2, GpuLiteral(5)), add1) :: Nil + + val caseWhenExpr3 = GpuCaseWhen(conditions3, None) + val equivalence3 = new GpuEquivalentExpressions + equivalence3.addExprTree(caseWhenExpr3) + assert(equivalence3.getAllExprStates().count(_.useCount == 2) == 0) + } + + test("Get Expression Tiers - GpuCaseWhen") { + val one = AttributeReference("one", IntegerType)() + val two = AttributeReference("two", IntegerType)() + val three = AttributeReference("three", IntegerType)() + val four = AttributeReference("four", IntegerType)() + val five = AttributeReference("five", IntegerType)() + + val add1 = GpuAdd(one, two, false) + val add2 = GpuAdd(two, three, false) + val cond1 = GpuGreaterThan(add2, three) + val cond2 = GpuGreaterThan(add2, four) + val cond3 = GpuGreaterThan(add2, five) + val cond4 = GpuGreaterThan(add1, three) + val conditions1 = (cond1, add1) :: (cond2, add1) :: (cond3, add1) :: Nil + val caseWhenExpr1 = GpuCaseWhen(conditions1, None) + val inputAttrs1 = AttributeSeq(Seq(one, two, three, four, five)) + val initialExprs1 = Seq(caseWhenExpr1) + // `add2` is repeatedly in all conditions. + validateExpressionTiers(initialExprs1, inputAttrs1, + Seq(add2), // subexpressions + Seq.empty, // Should be unchanged + Seq(cond1, cond2, cond3, caseWhenExpr1)) // modified expressions + + val conditions2 = (cond4, add1) :: (cond2, add1) :: (cond3, add1) :: Nil + val caseWhenExpr2 = GpuCaseWhen(conditions2, Some(add1)) + val inputAttrs2 = AttributeSeq(Seq(one, two, three, four, five)) + val initialExprs2 = Seq(caseWhenExpr2) + // `add1` is repeatedly in all branch values, and first predicate. + validateExpressionTiers(initialExprs2, inputAttrs2, + Seq(add1), // subexpressions + Seq.empty, // Should be unchanged + Seq(caseWhenExpr2)) // modified expressions + + // Negative case. `add1` or `add2` is not commonly used in all predicates/branch values. + val conditions3 = (cond4, add2) :: (cond2, add1) :: (cond3, add1) :: Nil + val caseWhenExpr3 = GpuCaseWhen(conditions3, None) + val inputAttrs3 = AttributeSeq(Seq(one, two, three, four, five)) + val initialExprs3 = Seq(caseWhenExpr3) + // `add1` is repeatedly in all branch values, and first predicate. + validateExpressionTiers(initialExprs3, inputAttrs3, + Seq.empty, // subexpressions + Seq(caseWhenExpr3), // Should be unchanged + Seq.empty) // modified expressions + } + + test("Children of conditional expressions: GpuCoalesce") { + val add1 = GpuAdd(GpuLiteral(1), GpuLiteral(2), false) + val add2 = GpuAdd(GpuLiteral(2), GpuLiteral(3), false) + val conditions1 = GpuGreaterThan(add2, GpuLiteral(3)) :: + GpuGreaterThan(add2, GpuLiteral(4)) :: + GpuGreaterThan(add2, GpuLiteral(5)) :: Nil + + val coalesceExpr1 = GpuCoalesce(conditions1) + val equivalence1 = new GpuEquivalentExpressions + equivalence1.addExprTree(coalesceExpr1) + + // `add2` is repeatedly in all conditions. + assert(equivalence1.getAllExprStates().count(_.useCount == 2) == 1) + assert(equivalence1.getAllExprStates().filter(_.useCount == 2).head.expr eq add2) + + // Negative case. `add1` and `add2` both are not used in all branches. + val conditions2 = GpuGreaterThan(add1, GpuLiteral(3)) :: + GpuGreaterThan(add2, GpuLiteral(4)) :: + GpuGreaterThan(add2, GpuLiteral(5)) :: Nil + + val coalesceExpr2 = GpuCoalesce(conditions2) + val equivalence2 = new GpuEquivalentExpressions + equivalence2.addExprTree(coalesceExpr2) + + assert(equivalence2.getAllExprStates().count(_.useCount == 2) == 0) + } + + test("Get Expression Tiers: GpuCoalesce") { + val one = AttributeReference("one", IntegerType)() + val two = AttributeReference("two", IntegerType)() + val three = AttributeReference("three", IntegerType)() + val four = AttributeReference("four", IntegerType)() + val five = AttributeReference("five", IntegerType)() + + val add1 = GpuAdd(one, two, false) + val add2 = GpuAdd(two, three, false) + val cond1 = GpuGreaterThan(add2, three) + val cond2 = GpuGreaterThan(add2, four) + val cond3 = GpuGreaterThan(add2, five) + val cond4 = GpuGreaterThan(add1, three) + + val conditions1 = cond1 :: cond2 :: cond3 :: Nil + val coalesceExpr1 = GpuCoalesce(conditions1) + val inputAttrs1 = AttributeSeq(Seq(one, two, three, four, five)) + val initialExprs1 = Seq(coalesceExpr1) + // `add2` is repeatedly in all conditions. + validateExpressionTiers(initialExprs1, inputAttrs1, + Seq(add2), // subexpressions + Seq.empty, // Should be unchanged + Seq(cond1, cond2, cond3, coalesceExpr1)) // modified expressions + + val conditions2 = cond4 :: cond2 :: cond3 :: Nil + val coalesceExpr2 = GpuCoalesce(conditions2) + val inputAttrs2 = AttributeSeq(Seq(one, two, three, four, five)) + val initialExprs2 = Seq(coalesceExpr2) + // Negative case. `add1` and `add2` both are not used in all branches. + validateExpressionTiers(initialExprs2, inputAttrs2, + Seq.empty, // subexpressions + Seq(coalesceExpr2), // Should be unchanged + Seq.empty) // modified expressions + } + + test("SPARK-35410: SubExpr elimination should not include redundant child exprs " + + "for conditional expressions") { + val add1 = GpuAdd(GpuLiteral(1), GpuLiteral(2), false) + val add2 = GpuAdd(GpuLiteral(2), GpuLiteral(3), false) + val add3 = GpuAdd(add1, add2, false) + val condition = (GpuGreaterThan(add3, GpuLiteral(3)), add3) :: Nil + + val caseWhenExpr = GpuCaseWhen(condition, Some(GpuAdd(add3, GpuLiteral(1), false))) + val equivalence = new GpuEquivalentExpressions + equivalence.addExprTree(caseWhenExpr) + + val commonExprs = equivalence.getAllExprStates(1) + assert(commonExprs.size == 1) + assert(commonExprs.head.useCount == 2) + assert(commonExprs.head.expr eq add3) + } + + test("Get Expression Tiers - SPARK-35410: SubExpr elimination should not include " + + "redundant child exprs for conditional expressions") { + val one = AttributeReference("one", IntegerType)() + val two = AttributeReference("two", IntegerType)() + val three = AttributeReference("three", IntegerType)() + + val add1 = GpuAdd(one, two, false) + val add2 = GpuAdd(two, three, false) + val add3 = GpuAdd(add1, add2, false) + val add4 = GpuAdd(add3, one, false) + val condition = (GpuGreaterThan(add3, three), add3) :: Nil + val caseWhenExpr = GpuCaseWhen(condition, Some(add4)) + val inputAttrs = AttributeSeq(Seq(one, two, three)) + val initialExprs = Seq(caseWhenExpr) + validateExpressionTiers(initialExprs, inputAttrs, + Seq(add3), // subexpressions + Seq.empty, // Should be unchanged + Seq(caseWhenExpr)) // modified expressions + } + + test("SPARK-35439: Children subexpr should come first than parent subexpr") { + val add = GpuAdd(GpuLiteral(1), GpuLiteral(2), false) + + val equivalence1 = new GpuEquivalentExpressions + + equivalence1.addExprTree(add) + assert(equivalence1.getAllExprStates().head.expr eq add) + + equivalence1.addExprTree(GpuAdd(GpuLiteral(3), add, false)) + assert(equivalence1.getAllExprStates().map(_.useCount) === Seq(2, 1)) + assert(equivalence1.getAllExprStates().map(_.expr) === + Seq(add, GpuAdd(GpuLiteral(3), add, false))) + + equivalence1.addExprTree(GpuAdd(GpuLiteral(3), add, false)) + assert(equivalence1.getAllExprStates().map(_.useCount) === Seq(2, 2)) + assert(equivalence1.getAllExprStates().map(_.expr) === + Seq(add, GpuAdd(GpuLiteral(3), add, false))) + + val equivalence2 = new GpuEquivalentExpressions + + equivalence2.addExprTree(GpuAdd(GpuLiteral(3), add, false)) + assert(equivalence2.getAllExprStates().map(_.useCount) === Seq(1, 1)) + assert(equivalence2.getAllExprStates().map(_.expr) === + Seq(add, GpuAdd(GpuLiteral(3), add, false))) + + equivalence2.addExprTree(add) + assert(equivalence2.getAllExprStates().map(_.useCount) === Seq(2, 1)) + assert(equivalence2.getAllExprStates().map(_.expr) === + Seq(add, GpuAdd(GpuLiteral(3), add, false))) + + equivalence2.addExprTree(GpuAdd(GpuLiteral(3), add, false)) + assert(equivalence2.getAllExprStates().map(_.useCount) === Seq(2, 2)) + assert(equivalence2.getAllExprStates().map(_.expr) === + Seq(add, GpuAdd(GpuLiteral(3), add, false))) + } + + test("SPARK-35499: Subexpressions should only be extracted from CaseWhen " + + "values with an elseValue") { + val add1 = GpuAdd(GpuLiteral(1), GpuLiteral(2), false) + val add2 = GpuAdd(GpuLiteral(2), GpuLiteral(3), false) + val conditions = (GpuGreaterThan(add1, GpuLiteral(3)), add1) :: + (GpuGreaterThan(add2, GpuLiteral(4)), add1) :: + (GpuGreaterThan(add2, GpuLiteral(5)), add1) :: Nil + + val caseWhenExpr = GpuCaseWhen(conditions, None) + val equivalence = new GpuEquivalentExpressions + equivalence.addExprTree(caseWhenExpr) + + // `add1` is not in the elseValue, so we can't extract it from the branches + assert(equivalence.getAllExprStates().count(_.useCount == 2) == 0) + } + + test("Get Expression Tiers - SPARK-35499: Subexpressions should only be extracted " + + "from CaseWhen values with an elseValue") { + val one = AttributeReference("one", IntegerType)() + val two = AttributeReference("two", IntegerType)() + val three = AttributeReference("three", IntegerType)() + val four = AttributeReference("four", IntegerType)() + val five = AttributeReference("five", IntegerType)() + + val add1 = GpuAdd(one, two, false) + val add2 = GpuAdd(two, three, false) + val cond1 = GpuGreaterThan(add1, three) + val cond2 = GpuGreaterThan(add2, four) + val cond3 = GpuGreaterThan(add2, five) + val conditions = (cond1, add1) :: (cond2, add1) :: (cond3, add1) :: Nil + val caseWhenExpr = GpuCaseWhen(conditions, None) + // `add1` is not in the elseValue, so we can't extract it from the branches + val inputAttrs = AttributeSeq(Seq(one, two, three, four, five)) + val initialExprs = Seq(caseWhenExpr) + // Negative case. `add1` and `add2` both are not used in all branches. + validateExpressionTiers(initialExprs, inputAttrs, + Seq.empty, // subexpressions + Seq(caseWhenExpr), // Should be unchanged + Seq.empty) // modified expressions + } + + test("Get Expression Tiers - Query derived from nds q4") { + val customer: AttributeReference = AttributeReference("customer", IntegerType)() + val quantity: AttributeReference = AttributeReference("quantity", IntegerType)() + val price: AttributeReference = AttributeReference("price", DecimalType(7, 2))() + val inputAttrs = AttributeSeq(Seq(customer, quantity, price)) + + val product = GpuDecimalMultiply( + GpuCast(quantity, DecimalType(10, 0)), price, DecimalType(18,2)) + val nullCheck = GpuIsNull(product) + val castProduct = GpuCast(product, DecimalType(28,2)) + val extract0 = GpuExtractChunk32(castProduct, 0, true) + val extract1 = GpuExtractChunk32(castProduct, 1, true) + val extract2 = GpuExtractChunk32(castProduct, 2, true) + val extract3 = GpuExtractChunk32(castProduct, 3, true) + val initialExprs = Seq(customer, extract0, extract1, extract2, extract3, nullCheck) + val exprTiers = GpuEquivalentExpressions.getExprTiers(initialExprs) + validateExprTiers(exprTiers, initialExprs, + Seq(product, castProduct), // Common sub-expression + Seq(customer), // Unchanged + Seq(extract0, extract1, extract2, extract3, nullCheck)) // updated + validateInputTiers(exprTiers, inputAttrs) + } + + test("Get Expression Tiers - Query derived from nds q62") { + val group: AttributeReference = AttributeReference("group", StringType)() + val smType: AttributeReference = AttributeReference("type", StringType)() + val webName: AttributeReference = AttributeReference("web name", StringType)() + val shipDate: AttributeReference = AttributeReference("ship date", IntegerType)() + val soldDate: AttributeReference = AttributeReference("sold date", IntegerType)() + val inputAttrs = AttributeSeq(Seq(group, smType, webName, shipDate, soldDate)) + + val dateDiff = GpuSubtract(shipDate, soldDate, false) + val caseWhen1 = + GpuCaseWhen( + Seq((GpuLessThanOrEqual(dateDiff, GpuLiteral(30)), GpuLiteral(1))), Some(GpuLiteral(0))) + val caseWhen2 = + GpuCaseWhen( + Seq((GpuAnd( + GpuGreaterThan(dateDiff, GpuLiteral(30)), + GpuLessThanOrEqual(dateDiff, GpuLiteral(60))), GpuLiteral(1))), Some(GpuLiteral(0))) + val caseWhen3 = + GpuCaseWhen( + Seq((GpuAnd( + GpuGreaterThan(dateDiff, GpuLiteral(60)), + GpuLessThanOrEqual(dateDiff, GpuLiteral(90))), GpuLiteral(1))), Some(GpuLiteral(0))) + val caseWhen4 = + GpuCaseWhen(Seq((GpuAnd( + GpuGreaterThan(dateDiff, GpuLiteral(90)), + GpuLessThanOrEqual(dateDiff, GpuLiteral(120))), GpuLiteral(1))), Some(GpuLiteral(0))) + val caseWhen5 = + GpuCaseWhen( + Seq((GpuGreaterThan(dateDiff, GpuLiteral(120)), GpuLiteral(1))), + Some(GpuLiteral(0))) + + val initialExprs = + Seq(group, smType, webName, caseWhen1, caseWhen2, caseWhen3, caseWhen4, caseWhen5) + val exprTiers = GpuEquivalentExpressions.getExprTiers(initialExprs) + validateExprTiers(exprTiers, initialExprs, + Seq(dateDiff), // sub-expressions + Seq(group, smType, webName), // unchanged exprs + Seq(caseWhen1, caseWhen2, caseWhen2, caseWhen3, caseWhen4, caseWhen5)) // updated exprs + validateInputTiers(exprTiers, inputAttrs) + } + + private def realExpr(expr: Expression): Expression = expr match { + case e: GpuAlias => e.child + case _ => expr + } + + private def checkEquals(expr: Expression, other: Expression): Boolean = { + realExpr(expr).semanticEquals(realExpr(other)) + } + + /** + * ValidateExprTiers: run checks on exprTiers vs what is expected + * Args: + * exprTiers - expression tiers we are checking + * initialExprs - original list of expressions + * subExprs - expected lowest level sub-expressions + * unchanged - expressions that are unmodified from the original list + * updated - expressions from the original list that should be updated + */ + private def validateExprTiers(exprTiers: Seq[Seq[Expression]], initialExprs: Seq[Expression], + subExprs: Seq[Expression], unChanged: Seq[Expression], updated: Seq[Expression]): Unit = { + if (subExprs.isEmpty) { + assert(exprTiers.size == 1) + // The one tier should match the initial list + initialExprs.foreach(e => assert(exprTiers(0).contains(e))) + } else { + // Should be more than one tier + assert(exprTiers.size > 1) + } + // Last tier should be same size as initial list + assert(exprTiers.last.size == initialExprs.size) + + // substituted expressions should be in one of the tiers before the last one. + val unSubbed = undoSubstitutions(exprTiers.dropRight(1).flatten) + subExprs.foreach(sub => + assert(unSubbed.exists(e => checkEquals(e, sub)), + "Expected: " + sub.toString() + " not found in: " + unSubbed.toString())) + + // Unchanged expressions should be in the last tier. + unChanged.foreach(expected => + assert(exprTiers.last.contains(expected), + "Expected: " + expected.toString() + " not found in: " + exprTiers.last.toString())) + + // Updated expressions should not match, since they have been updated + updated.foreach(expected => + assert(exprTiers.last.forall(e => !checkEquals(e, expected)), + "Unexpected: " + expected.toString() + " was found in: " + exprTiers.last.toString())) + } + + private def validateInputTiers(exprTiers: Seq[Seq[Expression]], + initialInputs: AttributeSeq): Unit = { + val inputTiers = GpuEquivalentExpressions.getInputTiers(exprTiers, initialInputs) + assert(exprTiers.size == inputTiers.size) + // First tier should have same inputs as original inputs + // Subsequent tiers should add inputs for each expr in previous tier + var expectedNumInputs = initialInputs.attrs.size + var curTier = 0 + while (curTier < inputTiers.size) { + assert(inputTiers(curTier).attrs.size == expectedNumInputs) + expectedNumInputs += exprTiers(curTier).size + curTier += 1 + } + initialInputs.attrs.foreach(a => assert(inputTiers.last.attrs.contains(a))) + } + + /** + * ValidateGetExprTiers: run checks on exprTiers vs what is expected + * Args: + * initialExprs - original list of expressions + * inputAttrs - original list of input attributes + * subExprs - expected lowest level sub-expressions + * unchanged - expressions that are unmodified from the original list + * updated - expressions from the original list that should be updated + */ + private def validateExpressionTiers(initialExprs: Seq[Expression], inputAttrs: AttributeSeq, + subExprs: Seq[Expression], unChanged: Seq[Expression], updated: Seq[Expression]) = { + val exprTiers = GpuEquivalentExpressions.getExprTiers(initialExprs) + validateExprTiers(exprTiers, initialExprs, subExprs, unChanged, updated) + validateInputTiers(exprTiers, inputAttrs) + } + + def restoreOriginalExpr( + expr: Expression, + substitutionMap: Map[Expression, Expression]): Expression = { + val newExpr = substitutionMap.get(expr) match { + case Some(e) => e + case None => expr + } + newExpr.mapChildren(restoreOriginalExpr(_, substitutionMap)) + } + + private def undoSubstitutions(subExprs: Seq[Expression]): Seq[Expression] = { + if (subExprs.isEmpty) { + subExprs + } else { + val subMap = subExprs.filter(p => p.isInstanceOf[GpuAlias]).map { + case e: GpuAlias => (e.toAttribute, e.child) + }.toMap[Expression, Expression] + subExprs.map(restoreOriginalExpr(_, subMap)) + } + } +} From 32c58aebffc4e36917fd7dea53cbb065c88151e5 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Thu, 8 Sep 2022 12:38:17 -0500 Subject: [PATCH 093/190] Use ShimLoader to access PlanShims (#6527) Signed-off-by: Jason Lowe Signed-off-by: Jason Lowe --- dist/unshimmed-from-each-spark3xx.txt | 1 + .../{PlanShims.scala => PlanShimsImpl.scala} | 4 ++- .../{PlanShims.scala => PlanShimsImpl.scala} | 4 ++- .../com/nvidia/spark/rapids/PlanShims.scala | 31 +++++++++++++++++++ .../com/nvidia/spark/rapids/Plugin.scala | 1 - .../com/nvidia/spark/rapids/ShimLoader.scala | 4 +++ 6 files changed, 42 insertions(+), 3 deletions(-) rename sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/{PlanShims.scala => PlanShimsImpl.scala} (90%) rename sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/{PlanShims.scala => PlanShimsImpl.scala} (91%) create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/PlanShims.scala diff --git a/dist/unshimmed-from-each-spark3xx.txt b/dist/unshimmed-from-each-spark3xx.txt index 38f2ffb50ad..fba8a50de01 100644 --- a/dist/unshimmed-from-each-spark3xx.txt +++ b/dist/unshimmed-from-each-spark3xx.txt @@ -2,5 +2,6 @@ com/nvidia/spark/rapids/*/RapidsShuffleManager* com/nvidia/spark/rapids/AvroProvider.class com/nvidia/spark/rapids/HiveProvider.class com/nvidia/spark/rapids/iceberg/IcebergProvider.class +com/nvidia/spark/rapids/PlanShims* org/apache/spark/sql/rapids/shims/*/ProxyRapidsShuffleInternalManager* spark-*-info.properties diff --git a/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/PlanShims.scala b/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala similarity index 90% rename from sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/PlanShims.scala rename to sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala index d0753ed3efa..7687a8663c8 100644 --- a/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/PlanShims.scala +++ b/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala @@ -16,8 +16,10 @@ package com.nvidia.spark.rapids.shims +import com.nvidia.spark.rapids.PlanShims + import org.apache.spark.sql.execution.SparkPlan -object PlanShims { +class PlanShimsImpl extends PlanShims { def extractExecutedPlan(plan: SparkPlan): SparkPlan = plan } diff --git a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/PlanShims.scala b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala similarity index 91% rename from sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/PlanShims.scala rename to sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala index 8e6f4ace219..dd502323485 100644 --- a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/PlanShims.scala +++ b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala @@ -16,9 +16,11 @@ package com.nvidia.spark.rapids.shims +import com.nvidia.spark.rapids.PlanShims + import org.apache.spark.sql.execution.{CommandResultExec, SparkPlan} -object PlanShims { +class PlanShimsImpl extends PlanShims { def extractExecutedPlan(plan: SparkPlan): SparkPlan = plan match { case p: CommandResultExec => p.commandPhysicalPlan case _ => plan diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/PlanShims.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/PlanShims.scala new file mode 100644 index 00000000000..f3ff2640c21 --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/PlanShims.scala @@ -0,0 +1,31 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids + +import org.apache.spark.sql.execution.SparkPlan + +trait PlanShims { + def extractExecutedPlan(plan: SparkPlan): SparkPlan +} + +object PlanShims { + private lazy val shims = ShimLoader.newPlanShims() + + def extractExecutedPlan(plan: SparkPlan): SparkPlan = { + shims.extractExecutedPlan(plan) + } +} \ No newline at end of file diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala index 6eba5523407..e9d85408745 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala @@ -27,7 +27,6 @@ import scala.util.matching.Regex import ai.rapids.cudf.{CudaException, CudaFatalException, CudfException, MemoryCleaner} import com.nvidia.spark.rapids.python.PythonWorkerSemaphore -import com.nvidia.spark.rapids.shims.PlanShims import org.apache.spark.{ExceptionFailure, SparkConf, SparkContext, TaskFailedReason} import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala index 2032cb29a23..3682ccf74d5 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala @@ -409,4 +409,8 @@ object ShimLoader extends Logging { def newIcebergProvider(): IcebergProvider = ShimLoader.newInstanceOf[IcebergProvider]( "com.nvidia.spark.rapids.iceberg.IcebergProviderImpl") + + def newPlanShims(): PlanShims = ShimLoader.newInstanceOf[PlanShims]( + "com.nvidia.spark.rapids.shims.PlanShimsImpl" + ) } From 4cb4e4227d2f7be0e91e6801a56feded661a4565 Mon Sep 17 00:00:00 2001 From: Jim Brennan Date: Thu, 8 Sep 2022 12:49:09 -0500 Subject: [PATCH 094/190] Fix typo in file name (#6525) Signed-off-by: Jim Brennan Signed-off-by: Jim Brennan --- ...uEquivalenExpressions.scala => GpuEquivalentExpressions.scala} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/{GpuEquivalenExpressions.scala => GpuEquivalentExpressions.scala} (100%) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuEquivalenExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuEquivalentExpressions.scala similarity index 100% rename from sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuEquivalenExpressions.scala rename to sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuEquivalentExpressions.scala From 950e2c8082c56397982752ea202d694b3ae3a10f Mon Sep 17 00:00:00 2001 From: zhanga5 Date: Fri, 9 Sep 2022 13:16:44 +0800 Subject: [PATCH 095/190] Include avro test by using '--packages' option [skip ci] (#6505) * Include avro test by using '--packages' option Signed-off-by: zhanga5 * Remove unnecessary 'SPARK_SUBMIT_FLAGS' Signed-off-by: zhanga5 * Drop 'unset TEST_PARALLEL' and backup/restore avro jar file * Revert to previous simple workaround Signed-off-by: zhanga5 --- jenkins/spark-tests.sh | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/jenkins/spark-tests.sh b/jenkins/spark-tests.sh index ea480d93e47..841ea9e912f 100755 --- a/jenkins/spark-tests.sh +++ b/jenkins/spark-tests.sh @@ -211,6 +211,16 @@ run_iceberg_tests() { fi } + # Test spark-avro with documented way of deploying at run time via --packages option from Maven +run_avro_tests() { + export PYSP_TEST_spark_jars_packages="org.apache.spark:spark-avro_2.12:${SPARK_VER}" + + # Workaround to avoid appending avro jar file by '--jars', which would be addressed by https://github.com/NVIDIA/spark-rapids/issues/6532 + rm -vf $LOCAL_JAR_PATH/spark-avro*.jar + + ./run_pyspark_from_build.sh -k avro +} + run_test_not_parallel() { local TEST=${1//\.py/} local LOG_FILE @@ -239,6 +249,10 @@ run_test_not_parallel() { run_iceberg_tests ;; + avro) + run_avro_tests + ;; + *) echo -e "\n\n>>>>> $TEST...\n" LOG_FILE="$TARGET_DIR/$TEST.log" @@ -284,6 +298,7 @@ export -f get_tests_by_tags # - DEFAULT: all tests except cudf_udf tests # - CUDF_UDF_ONLY: cudf_udf tests only, requires extra conda cudf-py lib # - ICEBERG_ONLY: iceberg tests only +# - AVRO_ONLY: avro tests only (with --packages option instead of --jars) # - DELTA_LAKE_ONLY: Delta Lake tests only TEST_MODE=${TEST_MODE:-'DEFAULT'} if [[ $TEST_MODE == "DEFAULT" ]]; then @@ -347,6 +362,11 @@ if [[ "$TEST_MODE" == "DEFAULT" || "$TEST_MODE" == "ICEBERG_ONLY" ]]; then run_test_not_parallel iceberg fi +# avro tests +if [[ "$TEST_MODE" == "DEFAULT" || "$TEST_MODE" == "AVRO_ONLY" ]]; then + run_test_not_parallel avro +fi + popd stop-slave.sh stop-master.sh From 22ee1e7af52d19a8da332c33d0cafdd6dff2d0f7 Mon Sep 17 00:00:00 2001 From: Ryan Lee Date: Fri, 9 Sep 2022 11:03:53 -0700 Subject: [PATCH 096/190] Fix notebook and getting started examples [skip ci] (#6421) * fix notebook, remove end output and fix formatting * fix wording, getting started scripts, and python notebook compatability * add spark.dynamicAllocation.enabled=false to example * remove trailing whitespacing and fix text per line * Sign off commit Signed-off-by: Ryan Lee (SW-GPU) * reformat json Signed-off-by: Ryan Lee (SW-GPU) Co-authored-by: Ryan Lee (SW-GPU) --- docs/demo/GCP/Mortgage-ETL.ipynb | 10966 ++-------------------- docs/get-started/getting-started-gcp.md | 142 +- 2 files changed, 992 insertions(+), 10116 deletions(-) diff --git a/docs/demo/GCP/Mortgage-ETL.ipynb b/docs/demo/GCP/Mortgage-ETL.ipynb index 1a05cbfbdef..1b7d8764b4f 100644 --- a/docs/demo/GCP/Mortgage-ETL.ipynb +++ b/docs/demo/GCP/Mortgage-ETL.ipynb @@ -1,10050 +1,920 @@ { - "cells": [ - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "## Prerequirement\n", - "### 1. Download data\n", - "Dataset is derived from Fannie Mae’s [Single-Family Loan Performance Data](http://www.fanniemae.com/portal/funding-the-market/data/loan-performance-data.html) with all rights reserved by Fannie Mae. Refer to these [instructions](https://github.com/NVIDIA/spark-rapids-examples/blob/branch-22.08/docs/get-started/xgboost-examples/dataset/mortgage.md) to download the dataset.\n", - "\n", - "### 2. Set up Environments\n", - "This notebook runs in a Dataproc cluster with GPU nodes, with [Spark RAPIDS](https://github.com/GoogleCloudDataproc/initialization-actions/tree/master/rapids) set up.\n", - "\n", - "### 3. Start Jupyter Notebook \n", - "```\n", - "$ jupyter notebook --ip=0.0.0.0 --port=8124 --no-browser\n", - "\n", - "```\n" - ] - }, - { - "cell_type": "code", - "execution_count": 1, - "metadata": {}, - "outputs": [], - "source": [ - "import time\n", - "import os\n", - "from pyspark import broadcast\n", - "from pyspark.sql import SparkSession\n", - "from pyspark.sql.functions import *\n", - "from pyspark.sql.types import *\n", - "from pyspark.sql.window import Window\n" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "## Create Spark Session" - ] - }, - { - "cell_type": "code", - "execution_count": 2, - "metadata": {}, - "outputs": [ - { - "name": "stderr", - "output_type": "stream", - "text": [ - "Setting default log level to \"WARN\".\n", - "To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).\n", - "22/07/28 08:53:54 WARN org.apache.spark.resource.ResourceProfile: The executor resource config for resource: gpu was specified but no corresponding task resource request was specified.\n", - "22/07/28 08:53:55 INFO org.apache.spark.SparkEnv: Registering MapOutputTracker\n", - "22/07/28 08:53:55 INFO org.apache.spark.SparkEnv: Registering BlockManagerMaster\n", - "22/07/28 08:53:55 INFO org.apache.spark.SparkEnv: Registering BlockManagerMasterHeartbeat\n", - "22/07/28 08:53:55 INFO org.apache.spark.SparkEnv: Registering OutputCommitCoordinator\n", - "22/07/28 08:53:56 WARN com.nvidia.spark.rapids.RapidsPluginUtils: RAPIDS Accelerator 22.04.0 using cudf 22.04.0.\n", - "22/07/28 08:53:56 WARN com.nvidia.spark.rapids.RapidsPluginUtils: RAPIDS Accelerator is enabled, to disable GPU support set `spark.rapids.sql.enabled` to false.\n", - "22/07/28 08:54:09 WARN com.nvidia.spark.udf.Plugin: Installing rapids UDF compiler extensions to Spark. The compiler is disabled by default. To enable it, set `spark.rapids.sql.udfCompiler.enabled` to true\n" - ] - } - ], - "source": [ - "if \"sc\" in globals():\n", - " sc.stop()\n", - "\n", - "### Configure the parameters based on your dataproc cluster ###\n", - "conf = SparkConf().setAppName(\"MortgageETL\")\n", - "conf.set(\"spark.plugins\", \"com.nvidia.spark.SQLPlugin\")\n", - "conf.set(\"spark.executor.instances\", \"2\")\n", - "conf.set(\"spark.executor.cores\", \"20\")\n", - "conf.set(\"spark.task.resource.gpu.amount\", \"0.05\")\n", - "conf.set(\"spark.task.cpus\", \"1\")\n", - "conf.set(\"spark.rapids.sql.concurrentGpuTasks\", \"2\")\n", - "conf.set(\"spark.executor.memory\", \"20g\")\n", - "conf.set(\"spark.sql.files.maxPartitionBytes\", \"512m\")\n", - "conf.set(\"spark.executor.resource.gpu.amount\", \"1\")\n", - "conf.set(\"spark.rapids.sql.hasNans\", \"false\")\n", - "conf.set(\"spark.rapids.sql.csv.read.double.enabled\", \"true\")\n", - "conf.set('spark.rapids.sql.batchSizeBytes', '512M')\n", - "conf.set('spark.rapids.sql.reader.batchSizeBytes', '768M')\n", - "conf.set(\"spark.rapids.memory.pinnedPool.size\", \"2G\")\n", - "conf.set(\"spark.executor.memoryOverhead\", \"2G\")\n", - "conf.set(\"spark.sql.broadcastTimeout\", \"700\")\n", - "\n", - "spark = SparkSession.builder \\\n", - " .config(conf=conf) \\\n", - " .getOrCreate()" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "## Function Define\n", - "### 1. Define the constants\n", - "\n", - "* Define input file schema (Performance and Acquisition)" - ] - }, - { - "cell_type": "code", - "execution_count": 3, - "metadata": {}, - "outputs": [], - "source": [ - "# File schema\n", - "_csv_raw_schema = StructType([\n", - " StructField(\"reference_pool_id\", StringType()),\n", - " StructField(\"loan_id\", LongType()),\n", - " StructField(\"monthly_reporting_period\", StringType()),\n", - " StructField(\"orig_channel\", StringType()),\n", - " StructField(\"seller_name\", StringType()),\n", - " StructField(\"servicer\", StringType()),\n", - " StructField(\"master_servicer\", StringType()),\n", - " StructField(\"orig_interest_rate\", DoubleType()),\n", - " StructField(\"interest_rate\", DoubleType()),\n", - " StructField(\"orig_upb\", DoubleType()),\n", - " StructField(\"upb_at_issuance\", StringType()),\n", - " StructField(\"current_actual_upb\", DoubleType()),\n", - " StructField(\"orig_loan_term\", IntegerType()),\n", - " StructField(\"orig_date\", StringType()),\n", - " StructField(\"first_pay_date\", StringType()), \n", - " StructField(\"loan_age\", DoubleType()),\n", - " StructField(\"remaining_months_to_legal_maturity\", DoubleType()),\n", - " StructField(\"adj_remaining_months_to_maturity\", DoubleType()),\n", - " StructField(\"maturity_date\", StringType()),\n", - " StructField(\"orig_ltv\", DoubleType()),\n", - " StructField(\"orig_cltv\", DoubleType()),\n", - " StructField(\"num_borrowers\", DoubleType()),\n", - " StructField(\"dti\", DoubleType()),\n", - " StructField(\"borrower_credit_score\", DoubleType()),\n", - " StructField(\"coborrow_credit_score\", DoubleType()),\n", - " StructField(\"first_home_buyer\", StringType()),\n", - " StructField(\"loan_purpose\", StringType()),\n", - " StructField(\"property_type\", StringType()),\n", - " StructField(\"num_units\", IntegerType()),\n", - " StructField(\"occupancy_status\", StringType()),\n", - " StructField(\"property_state\", StringType()),\n", - " StructField(\"msa\", DoubleType()),\n", - " StructField(\"zip\", IntegerType()),\n", - " StructField(\"mortgage_insurance_percent\", DoubleType()),\n", - " StructField(\"product_type\", StringType()),\n", - " StructField(\"prepayment_penalty_indicator\", StringType()),\n", - " StructField(\"interest_only_loan_indicator\", StringType()),\n", - " StructField(\"interest_only_first_principal_and_interest_payment_date\", StringType()),\n", - " StructField(\"months_to_amortization\", StringType()),\n", - " StructField(\"current_loan_delinquency_status\", IntegerType()),\n", - " StructField(\"loan_payment_history\", StringType()),\n", - " StructField(\"mod_flag\", StringType()),\n", - " StructField(\"mortgage_insurance_cancellation_indicator\", StringType()),\n", - " StructField(\"zero_balance_code\", StringType()),\n", - " StructField(\"zero_balance_effective_date\", StringType()),\n", - " StructField(\"upb_at_the_time_of_removal\", StringType()),\n", - " StructField(\"repurchase_date\", StringType()),\n", - " StructField(\"scheduled_principal_current\", StringType()),\n", - " StructField(\"total_principal_current\", StringType()),\n", - " StructField(\"unscheduled_principal_current\", StringType()),\n", - " StructField(\"last_paid_installment_date\", StringType()),\n", - " StructField(\"foreclosed_after\", StringType()),\n", - " StructField(\"disposition_date\", StringType()),\n", - " StructField(\"foreclosure_costs\", DoubleType()),\n", - " StructField(\"prop_preservation_and_repair_costs\", DoubleType()),\n", - " StructField(\"asset_recovery_costs\", DoubleType()),\n", - " StructField(\"misc_holding_expenses\", DoubleType()),\n", - " StructField(\"holding_taxes\", DoubleType()),\n", - " StructField(\"net_sale_proceeds\", DoubleType()),\n", - " StructField(\"credit_enhancement_proceeds\", DoubleType()),\n", - " StructField(\"repurchase_make_whole_proceeds\", StringType()),\n", - " StructField(\"other_foreclosure_proceeds\", DoubleType()),\n", - " StructField(\"non_interest_bearing_upb\", DoubleType()),\n", - " StructField(\"principal_forgiveness_upb\", StringType()),\n", - " StructField(\"original_list_start_date\", StringType()),\n", - " StructField(\"original_list_price\", StringType()),\n", - " StructField(\"current_list_start_date\", StringType()),\n", - " StructField(\"current_list_price\", StringType()),\n", - " StructField(\"borrower_credit_score_at_issuance\", StringType()),\n", - " StructField(\"co-borrower_credit_score_at_issuance\", StringType()),\n", - " StructField(\"borrower_credit_score_current\", StringType()),\n", - " StructField(\"co-Borrower_credit_score_current\", StringType()),\n", - " StructField(\"mortgage_insurance_type\", DoubleType()),\n", - " StructField(\"servicing_activity_indicator\", StringType()),\n", - " StructField(\"current_period_modification_loss_amount\", StringType()),\n", - " StructField(\"cumulative_modification_loss_amount\", StringType()),\n", - " StructField(\"current_period_credit_event_net_gain_or_loss\", StringType()),\n", - " StructField(\"cumulative_credit_event_net_gain_or_loss\", StringType()),\n", - " StructField(\"homeready_program_indicator\", StringType()),\n", - " StructField(\"foreclosure_principal_write_off_amount\", StringType()),\n", - " StructField(\"relocation_mortgage_indicator\", StringType()),\n", - " StructField(\"zero_balance_code_change_date\", StringType()),\n", - " StructField(\"loan_holdback_indicator\", StringType()),\n", - " StructField(\"loan_holdback_effective_date\", StringType()),\n", - " StructField(\"delinquent_accrued_interest\", StringType()),\n", - " StructField(\"property_valuation_method\", StringType()),\n", - " StructField(\"high_balance_loan_indicator\", StringType()),\n", - " StructField(\"arm_initial_fixed-rate_period_lt_5_yr_indicator\", StringType()),\n", - " StructField(\"arm_product_type\", StringType()),\n", - " StructField(\"initial_fixed-rate_period\", StringType()),\n", - " StructField(\"interest_rate_adjustment_frequency\", StringType()),\n", - " StructField(\"next_interest_rate_adjustment_date\", StringType()),\n", - " StructField(\"next_payment_change_date\", StringType()),\n", - " StructField(\"index\", StringType()),\n", - " StructField(\"arm_cap_structure\", StringType()),\n", - " StructField(\"initial_interest_rate_cap_up_percent\", StringType()),\n", - " StructField(\"periodic_interest_rate_cap_up_percent\", StringType()),\n", - " StructField(\"lifetime_interest_rate_cap_up_percent\", StringType()),\n", - " StructField(\"mortgage_margin\", StringType()),\n", - " StructField(\"arm_balloon_indicator\", StringType()),\n", - " StructField(\"arm_plan_number\", StringType()),\n", - " StructField(\"borrower_assistance_plan\", StringType()),\n", - " StructField(\"hltv_refinance_option_indicator\", StringType()),\n", - " StructField(\"deal_name\", StringType()),\n", - " StructField(\"repurchase_make_whole_proceeds_flag\", StringType()),\n", - " StructField(\"alternative_delinquency_resolution\", StringType()),\n", - " StructField(\"alternative_delinquency_resolution_count\", StringType()),\n", - " StructField(\"total_deferral_amount\", StringType())\n", - " ])" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "* Define seller name mapping" - ] - }, - { - "cell_type": "code", - "execution_count": 4, - "metadata": {}, - "outputs": [], - "source": [ - "# name mappings\n", - "_name_mapping = [\n", - " (\"WITMER FUNDING, LLC\", \"Witmer\"),\n", - " (\"WELLS FARGO CREDIT RISK TRANSFER SECURITIES TRUST 2015\", \"Wells Fargo\"),\n", - " (\"WELLS FARGO BANK, NA\" , \"Wells Fargo\"),\n", - " (\"WELLS FARGO BANK, N.A.\" , \"Wells Fargo\"),\n", - " (\"WELLS FARGO BANK, NA\" , \"Wells Fargo\"),\n", - " (\"USAA FEDERAL SAVINGS BANK\" , \"USAA\"),\n", - " (\"UNITED SHORE FINANCIAL SERVICES, LLC D\\\\/B\\\\/A UNITED WHOLESALE MORTGAGE\" , \"United Seq(e\"),\n", - " (\"U.S. BANK N.A.\" , \"US Bank\"),\n", - " (\"SUNTRUST MORTGAGE INC.\" , \"Suntrust\"),\n", - " (\"STONEGATE MORTGAGE CORPORATION\" , \"Stonegate Mortgage\"),\n", - " (\"STEARNS LENDING, LLC\" , \"Stearns Lending\"),\n", - " (\"STEARNS LENDING, INC.\" , \"Stearns Lending\"),\n", - " (\"SIERRA PACIFIC MORTGAGE COMPANY, INC.\" , \"Sierra Pacific Mortgage\"),\n", - " (\"REGIONS BANK\" , \"Regions\"),\n", - " (\"RBC MORTGAGE COMPANY\" , \"RBC\"),\n", - " (\"QUICKEN LOANS INC.\" , \"Quicken Loans\"),\n", - " (\"PULTE MORTGAGE, L.L.C.\" , \"Pulte Mortgage\"),\n", - " (\"PROVIDENT FUNDING ASSOCIATES, L.P.\" , \"Provident Funding\"),\n", - " (\"PROSPECT MORTGAGE, LLC\" , \"Prospect Mortgage\"),\n", - " (\"PRINCIPAL RESIDENTIAL MORTGAGE CAPITAL RESOURCES, LLC\" , \"Principal Residential\"),\n", - " (\"PNC BANK, N.A.\" , \"PNC\"),\n", - " (\"PMT CREDIT RISK TRANSFER TRUST 2015-2\" , \"PennyMac\"),\n", - " (\"PHH MORTGAGE CORPORATION\" , \"PHH Mortgage\"),\n", - " (\"PENNYMAC CORP.\" , \"PennyMac\"),\n", - " (\"PACIFIC UNION FINANCIAL, LLC\" , \"Other\"),\n", - " (\"OTHER\" , \"Other\"),\n", - " (\"NYCB MORTGAGE COMPANY, LLC\" , \"NYCB\"),\n", - " (\"NEW YORK COMMUNITY BANK\" , \"NYCB\"),\n", - " (\"NETBANK FUNDING SERVICES\" , \"Netbank\"),\n", - " (\"NATIONSTAR MORTGAGE, LLC\" , \"Nationstar Mortgage\"),\n", - " (\"METLIFE BANK, NA\" , \"Metlife\"),\n", - " (\"LOANDEPOT.COM, LLC\" , \"LoanDepot.com\"),\n", - " (\"J.P. MORGAN MADISON AVENUE SECURITIES TRUST, SERIES 2015-1\" , \"JP Morgan Chase\"),\n", - " (\"J.P. MORGAN MADISON AVENUE SECURITIES TRUST, SERIES 2014-1\" , \"JP Morgan Chase\"),\n", - " (\"JPMORGAN CHASE BANK, NATIONAL ASSOCIATION\" , \"JP Morgan Chase\"),\n", - " (\"JPMORGAN CHASE BANK, NA\" , \"JP Morgan Chase\"),\n", - " (\"JP MORGAN CHASE BANK, NA\" , \"JP Morgan Chase\"),\n", - " (\"IRWIN MORTGAGE, CORPORATION\" , \"Irwin Mortgage\"),\n", - " (\"IMPAC MORTGAGE CORP.\" , \"Impac Mortgage\"),\n", - " (\"HSBC BANK USA, NATIONAL ASSOCIATION\" , \"HSBC\"),\n", - " (\"HOMEWARD RESIDENTIAL, INC.\" , \"Homeward Mortgage\"),\n", - " (\"HOMESTREET BANK\" , \"Other\"),\n", - " (\"HOMEBRIDGE FINANCIAL SERVICES, INC.\" , \"HomeBridge\"),\n", - " (\"HARWOOD STREET FUNDING I, LLC\" , \"Harwood Mortgage\"),\n", - " (\"GUILD MORTGAGE COMPANY\" , \"Guild Mortgage\"),\n", - " (\"GMAC MORTGAGE, LLC (USAA FEDERAL SAVINGS BANK)\" , \"GMAC\"),\n", - " (\"GMAC MORTGAGE, LLC\" , \"GMAC\"),\n", - " (\"GMAC (USAA)\" , \"GMAC\"),\n", - " (\"FREMONT BANK\" , \"Fremont Bank\"),\n", - " (\"FREEDOM MORTGAGE CORP.\" , \"Freedom Mortgage\"),\n", - " (\"FRANKLIN AMERICAN MORTGAGE COMPANY\" , \"Franklin America\"),\n", - " (\"FLEET NATIONAL BANK\" , \"Fleet National\"),\n", - " (\"FLAGSTAR CAPITAL MARKETS CORPORATION\" , \"Flagstar Bank\"),\n", - " (\"FLAGSTAR BANK, FSB\" , \"Flagstar Bank\"),\n", - " (\"FIRST TENNESSEE BANK NATIONAL ASSOCIATION\" , \"Other\"),\n", - " (\"FIFTH THIRD BANK\" , \"Fifth Third Bank\"),\n", - " (\"FEDERAL HOME LOAN BANK OF CHICAGO\" , \"Fedral Home of Chicago\"),\n", - " (\"FDIC, RECEIVER, INDYMAC FEDERAL BANK FSB\" , \"FDIC\"),\n", - " (\"DOWNEY SAVINGS AND LOAN ASSOCIATION, F.A.\" , \"Downey Mortgage\"),\n", - " (\"DITECH FINANCIAL LLC\" , \"Ditech\"),\n", - " (\"CITIMORTGAGE, INC.\" , \"Citi\"),\n", - " (\"CHICAGO MORTGAGE SOLUTIONS DBA INTERFIRST MORTGAGE COMPANY\" , \"Chicago Mortgage\"),\n", - " (\"CHICAGO MORTGAGE SOLUTIONS DBA INTERBANK MORTGAGE COMPANY\" , \"Chicago Mortgage\"),\n", - " (\"CHASE HOME FINANCE, LLC\" , \"JP Morgan Chase\"),\n", - " (\"CHASE HOME FINANCE FRANKLIN AMERICAN MORTGAGE COMPANY\" , \"JP Morgan Chase\"),\n", - " (\"CHASE HOME FINANCE (CIE 1)\" , \"JP Morgan Chase\"),\n", - " (\"CHASE HOME FINANCE\" , \"JP Morgan Chase\"),\n", - " (\"CASHCALL, INC.\" , \"CashCall\"),\n", - " (\"CAPITAL ONE, NATIONAL ASSOCIATION\" , \"Capital One\"),\n", - " (\"CALIBER HOME LOANS, INC.\" , \"Caliber Funding\"),\n", - " (\"BISHOPS GATE RESIDENTIAL MORTGAGE TRUST\" , \"Bishops Gate Mortgage\"),\n", - " (\"BANK OF AMERICA, N.A.\" , \"Bank of America\"),\n", - " (\"AMTRUST BANK\" , \"AmTrust\"),\n", - " (\"AMERISAVE MORTGAGE CORPORATION\" , \"Amerisave\"),\n", - " (\"AMERIHOME MORTGAGE COMPANY, LLC\" , \"AmeriHome Mortgage\"),\n", - " (\"ALLY BANK\" , \"Ally Bank\"),\n", - " (\"ACADEMY MORTGAGE CORPORATION\" , \"Academy Mortgage\"),\n", - " (\"NO CASH-OUT REFINANCE\" , \"OTHER REFINANCE\"),\n", - " (\"REFINANCE - NOT SPECIFIED\" , \"OTHER REFINANCE\"),\n", - " (\"Other REFINANCE\" , \"OTHER REFINANCE\")]" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "* Define category (string) column and numeric column" - ] - }, - { - "cell_type": "code", - "execution_count": 5, - "metadata": {}, - "outputs": [], - "source": [ - "# String columns\n", - "cate_col_names = [\n", - " \"orig_channel\",\n", - " \"first_home_buyer\",\n", - " \"loan_purpose\",\n", - " \"property_type\",\n", - " \"occupancy_status\",\n", - " \"property_state\",\n", - " \"product_type\",\n", - " \"relocation_mortgage_indicator\",\n", - " \"seller_name\",\n", - " \"mod_flag\"\n", - "]\n", - "# Numberic columns\n", - "label_col_name = \"delinquency_12\"\n", - "numeric_col_names = [\n", - " \"orig_interest_rate\",\n", - " \"orig_upb\",\n", - " \"orig_loan_term\",\n", - " \"orig_ltv\",\n", - " \"orig_cltv\",\n", - " \"num_borrowers\",\n", - " \"dti\",\n", - " \"borrower_credit_score\",\n", - " \"num_units\",\n", - " \"zip\",\n", - " \"mortgage_insurance_percent\",\n", - " \"current_loan_delinquency_status\",\n", - " \"current_actual_upb\",\n", - " \"interest_rate\",\n", - " \"loan_age\",\n", - " \"msa\",\n", - " \"non_interest_bearing_upb\",\n", - " label_col_name\n", - "]\n", - "all_col_names = cate_col_names + numeric_col_names" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "* Functions to extract perf and acq columns from raw schema" - ] - }, - { - "cell_type": "code", - "execution_count": 6, - "metadata": {}, - "outputs": [], - "source": [ - "def extract_perf_columns(rawDf):\n", - " perfDf = rawDf.select(\n", - " col(\"loan_id\"),\n", - " date_format(to_date(col(\"monthly_reporting_period\"),\"MMyyyy\"), \"MM/dd/yyyy\").alias(\"monthly_reporting_period\"),\n", - " upper(col(\"servicer\")).alias(\"servicer\"),\n", - " col(\"interest_rate\"),\n", - " col(\"current_actual_upb\"),\n", - " col(\"loan_age\"),\n", - " col(\"remaining_months_to_legal_maturity\"),\n", - " col(\"adj_remaining_months_to_maturity\"),\n", - " date_format(to_date(col(\"maturity_date\"),\"MMyyyy\"), \"MM/yyyy\").alias(\"maturity_date\"),\n", - " col(\"msa\"),\n", - " col(\"current_loan_delinquency_status\"),\n", - " col(\"mod_flag\"),\n", - " col(\"zero_balance_code\"),\n", - " date_format(to_date(col(\"zero_balance_effective_date\"),\"MMyyyy\"), \"MM/yyyy\").alias(\"zero_balance_effective_date\"),\n", - " date_format(to_date(col(\"last_paid_installment_date\"),\"MMyyyy\"), \"MM/dd/yyyy\").alias(\"last_paid_installment_date\"),\n", - " date_format(to_date(col(\"foreclosed_after\"),\"MMyyyy\"), \"MM/dd/yyyy\").alias(\"foreclosed_after\"),\n", - " date_format(to_date(col(\"disposition_date\"),\"MMyyyy\"), \"MM/dd/yyyy\").alias(\"disposition_date\"),\n", - " col(\"foreclosure_costs\"),\n", - " col(\"prop_preservation_and_repair_costs\"),\n", - " col(\"asset_recovery_costs\"),\n", - " col(\"misc_holding_expenses\"),\n", - " col(\"holding_taxes\"),\n", - " col(\"net_sale_proceeds\"),\n", - " col(\"credit_enhancement_proceeds\"),\n", - " col(\"repurchase_make_whole_proceeds\"),\n", - " col(\"other_foreclosure_proceeds\"),\n", - " col(\"non_interest_bearing_upb\"),\n", - " col(\"principal_forgiveness_upb\"),\n", - " col(\"repurchase_make_whole_proceeds_flag\"),\n", - " col(\"foreclosure_principal_write_off_amount\"),\n", - " col(\"servicing_activity_indicator\"),\n", - " col('quarter')\n", - " )\n", - "\n", - " return perfDf.select(\"*\").filter(\"current_actual_upb != 0.0\")\n", - "\n", - "def extract_acq_columns(rawDf):\n", - " acqDf = rawDf.select(\n", - " col(\"loan_id\"),\n", - " col(\"orig_channel\"),\n", - " upper(col(\"seller_name\")).alias(\"seller_name\"),\n", - " col(\"orig_interest_rate\"),\n", - " col(\"orig_upb\"),\n", - " col(\"orig_loan_term\"),\n", - " date_format(to_date(col(\"orig_date\"),\"MMyyyy\"), \"MM/yyyy\").alias(\"orig_date\"),\n", - " date_format(to_date(col(\"first_pay_date\"),\"MMyyyy\"), \"MM/yyyy\").alias(\"first_pay_date\"),\n", - " col(\"orig_ltv\"),\n", - " col(\"orig_cltv\"),\n", - " col(\"num_borrowers\"),\n", - " col(\"dti\"),\n", - " col(\"borrower_credit_score\"),\n", - " col(\"first_home_buyer\"),\n", - " col(\"loan_purpose\"),\n", - " col(\"property_type\"),\n", - " col(\"num_units\"),\n", - " col(\"occupancy_status\"),\n", - " col(\"property_state\"),\n", - " col(\"zip\"),\n", - " col(\"mortgage_insurance_percent\"),\n", - " col(\"product_type\"),\n", - " col(\"coborrow_credit_score\"),\n", - " col(\"mortgage_insurance_type\"),\n", - " col(\"relocation_mortgage_indicator\"),\n", - " dense_rank().over(Window.partitionBy(\"loan_id\").orderBy(to_date(col(\"monthly_reporting_period\"),\"MMyyyy\"))).alias(\"rank\"),\n", - " col('quarter')\n", - " )\n", - "\n", - " return acqDf.select(\"*\").filter(col(\"rank\")==1)" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "### 2. Define ETL Process\n", - "\n", - "Define the function to do the ETL process\n", - "\n", - "#### 2.1 Define Functions to Read Raw CSV File\n", - "\n", - "* Define function to get quarter from input CSV file name" - ] - }, - { - "cell_type": "code", - "execution_count": 7, - "metadata": {}, - "outputs": [], - "source": [ - "def _get_quarter_from_csv_file_name():\n", - " return substring_index(substring_index(input_file_name(), '.', 1), '/', -1)" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "* Define function to read raw CSV data file" - ] - }, - { - "cell_type": "code", - "execution_count": 8, - "metadata": {}, - "outputs": [], - "source": [ - "def read_raw_csv(spark, path):\n", - " return spark.read.format('csv') \\\n", - " .option('nullValue', '') \\\n", - " .option('header', False) \\\n", - " .option('delimiter', '|') \\\n", - " .schema(_csv_raw_schema) \\\n", - " .load(path) \\\n", - " .withColumn('quarter', _get_quarter_from_csv_file_name())" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "#### 2.2 Define ETL Process\n", - "\n", - "* Define function to parse dates in Performance data" - ] - }, - { - "cell_type": "code", - "execution_count": 9, - "metadata": {}, - "outputs": [], - "source": [ - "def _parse_dates(perf):\n", - " return perf \\\n", - " .withColumn('monthly_reporting_period', to_date(col('monthly_reporting_period'), 'MM/dd/yyyy')) \\\n", - " .withColumn('monthly_reporting_period_month', month(col('monthly_reporting_period'))) \\\n", - " .withColumn('monthly_reporting_period_year', year(col('monthly_reporting_period'))) \\\n", - " .withColumn('monthly_reporting_period_day', dayofmonth(col('monthly_reporting_period'))) \\\n", - " .withColumn('last_paid_installment_date', to_date(col('last_paid_installment_date'), 'MM/dd/yyyy')) \\\n", - " .withColumn('foreclosed_after', to_date(col('foreclosed_after'), 'MM/dd/yyyy')) \\\n", - " .withColumn('disposition_date', to_date(col('disposition_date'), 'MM/dd/yyyy')) \\\n", - " .withColumn('maturity_date', to_date(col('maturity_date'), 'MM/yyyy')) \\\n", - " .withColumn('zero_balance_effective_date', to_date(col('zero_balance_effective_date'), 'MM/yyyy'))" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "* Define function to create deliquency data frame from Performance data" - ] - }, - { - "cell_type": "code", - "execution_count": 10, - "metadata": {}, - "outputs": [], - "source": [ - "def _create_perf_deliquency(spark, perf):\n", - " aggDF = perf.select(\n", - " col(\"quarter\"),\n", - " col(\"loan_id\"),\n", - " col(\"current_loan_delinquency_status\"),\n", - " when(col(\"current_loan_delinquency_status\") >= 1, col(\"monthly_reporting_period\")).alias(\"delinquency_30\"),\n", - " when(col(\"current_loan_delinquency_status\") >= 3, col(\"monthly_reporting_period\")).alias(\"delinquency_90\"),\n", - " when(col(\"current_loan_delinquency_status\") >= 6, col(\"monthly_reporting_period\")).alias(\"delinquency_180\")) \\\n", - " .groupBy(\"quarter\", \"loan_id\") \\\n", - " .agg(\n", - " max(\"current_loan_delinquency_status\").alias(\"delinquency_12\"),\n", - " min(\"delinquency_30\").alias(\"delinquency_30\"),\n", - " min(\"delinquency_90\").alias(\"delinquency_90\"),\n", - " min(\"delinquency_180\").alias(\"delinquency_180\")) \\\n", - " .select(\n", - " col(\"quarter\"),\n", - " col(\"loan_id\"),\n", - " (col(\"delinquency_12\") >= 1).alias(\"ever_30\"),\n", - " (col(\"delinquency_12\") >= 3).alias(\"ever_90\"),\n", - " (col(\"delinquency_12\") >= 6).alias(\"ever_180\"),\n", - " col(\"delinquency_30\"),\n", - " col(\"delinquency_90\"),\n", - " col(\"delinquency_180\"))\n", - " joinedDf = perf \\\n", - " .withColumnRenamed(\"monthly_reporting_period\", \"timestamp\") \\\n", - " .withColumnRenamed(\"monthly_reporting_period_month\", \"timestamp_month\") \\\n", - " .withColumnRenamed(\"monthly_reporting_period_year\", \"timestamp_year\") \\\n", - " .withColumnRenamed(\"current_loan_delinquency_status\", \"delinquency_12\") \\\n", - " .withColumnRenamed(\"current_actual_upb\", \"upb_12\") \\\n", - " .select(\"quarter\", \"loan_id\", \"timestamp\", \"delinquency_12\", \"upb_12\", \"timestamp_month\", \"timestamp_year\") \\\n", - " .join(aggDF, [\"loan_id\", \"quarter\"], \"left_outer\")\n", - "\n", - " # calculate the 12 month delinquency and upb values\n", - " months = 12\n", - " monthArray = [lit(x) for x in range(0, 12)]\n", - " # explode on a small amount of data is actually slightly more efficient than a cross join\n", - " testDf = joinedDf \\\n", - " .withColumn(\"month_y\", explode(array(monthArray))) \\\n", - " .select(\n", - " col(\"quarter\"),\n", - " floor(((col(\"timestamp_year\") * 12 + col(\"timestamp_month\")) - 24000) / months).alias(\"josh_mody\"),\n", - " floor(((col(\"timestamp_year\") * 12 + col(\"timestamp_month\")) - 24000 - col(\"month_y\")) / months).alias(\"josh_mody_n\"),\n", - " col(\"ever_30\"),\n", - " col(\"ever_90\"),\n", - " col(\"ever_180\"),\n", - " col(\"delinquency_30\"),\n", - " col(\"delinquency_90\"),\n", - " col(\"delinquency_180\"),\n", - " col(\"loan_id\"),\n", - " col(\"month_y\"),\n", - " col(\"delinquency_12\"),\n", - " col(\"upb_12\")) \\\n", - " .groupBy(\"quarter\", \"loan_id\", \"josh_mody_n\", \"ever_30\", \"ever_90\", \"ever_180\", \"delinquency_30\", \"delinquency_90\", \"delinquency_180\", \"month_y\") \\\n", - " .agg(max(\"delinquency_12\").alias(\"delinquency_12\"), min(\"upb_12\").alias(\"upb_12\")) \\\n", - " .withColumn(\"timestamp_year\", floor((lit(24000) + (col(\"josh_mody_n\") * lit(months)) + (col(\"month_y\") - 1)) / lit(12))) \\\n", - " .selectExpr('*', 'pmod(24000 + (josh_mody_n * {}) + month_y, 12) as timestamp_month_tmp'.format(months)) \\\n", - " .withColumn(\"timestamp_month\", when(col(\"timestamp_month_tmp\") == lit(0), lit(12)).otherwise(col(\"timestamp_month_tmp\"))) \\\n", - " .withColumn(\"delinquency_12\", ((col(\"delinquency_12\") > 3).cast(\"int\") + (col(\"upb_12\") == 0).cast(\"int\")).alias(\"delinquency_12\")) \\\n", - " .drop(\"timestamp_month_tmp\", \"josh_mody_n\", \"month_y\")\n", - "\n", - " return perf.withColumnRenamed(\"monthly_reporting_period_month\", \"timestamp_month\") \\\n", - " .withColumnRenamed(\"monthly_reporting_period_year\", \"timestamp_year\") \\\n", - " .join(testDf, [\"quarter\", \"loan_id\", \"timestamp_year\", \"timestamp_month\"], \"left\") \\\n", - " .drop(\"timestamp_year\", \"timestamp_month\")" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "* Define function to create acquisition data frame from Acquisition data" - ] - }, - { - "cell_type": "code", - "execution_count": 11, - "metadata": {}, - "outputs": [], - "source": [ - "def _create_acquisition(spark, acq):\n", - " nameMapping = spark.createDataFrame(_name_mapping, [\"from_seller_name\", \"to_seller_name\"])\n", - " return acq.join(nameMapping, col(\"seller_name\") == col(\"from_seller_name\"), \"left\") \\\n", - " .drop(\"from_seller_name\") \\\n", - " .withColumn(\"old_name\", col(\"seller_name\")) \\\n", - " .withColumn(\"seller_name\", coalesce(col(\"to_seller_name\"), col(\"seller_name\"))) \\\n", - " .drop(\"to_seller_name\") \\\n", - " .withColumn(\"orig_date\", to_date(col(\"orig_date\"), \"MM/yyyy\")) \\\n", - " .withColumn(\"first_pay_date\", to_date(col(\"first_pay_date\"), \"MM/yyyy\")) " - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "#### 2.3 Define Casting Process\n", - "This part is casting String column to Numbric. \n", - "Example:\n", - "```\n", - "col_1\n", - " \"a\"\n", - " \"b\"\n", - " \"c\"\n", - " \"a\"\n", - "# After String ====> Numberic\n", - "col_1\n", - " 0\n", - " 1\n", - " 2\n", - " 0\n", - "``` \n", - "
\n", - "\n", - "* Define function to get column dictionary\n", - "\n", - " Example\n", - " ```\n", - " col1 = [row(data=\"a\",id=0), row(data=\"b\",id=1)]\n", - " ```" - ] - }, - { - "cell_type": "code", - "execution_count": 12, - "metadata": {}, - "outputs": [], - "source": [ - "def _gen_dictionary(etl_df, col_names):\n", - " cnt_table = etl_df.select(posexplode(array([col(i) for i in col_names])))\\\n", - " .withColumnRenamed(\"pos\", \"column_id\")\\\n", - " .withColumnRenamed(\"col\", \"data\")\\\n", - " .filter(\"data is not null\")\\\n", - " .groupBy(\"column_id\", \"data\")\\\n", - " .count()\n", - " windowed = Window.partitionBy(\"column_id\").orderBy(desc(\"count\"))\n", - " return cnt_table.withColumn(\"id\", row_number().over(windowed)).drop(\"count\")" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "* Define function to convert string columns to numeric" - ] - }, - { - "cell_type": "code", - "execution_count": 13, - "metadata": {}, - "outputs": [], - "source": [ - "def _cast_string_columns_to_numeric(spark, input_df):\n", - " cached_dict_df = _gen_dictionary(input_df, cate_col_names).cache()\n", - " output_df = input_df\n", - " # Generate the final table with all columns being numeric.\n", - " for col_pos, col_name in enumerate(cate_col_names):\n", - " col_dict_df = cached_dict_df.filter(col(\"column_id\") == col_pos)\\\n", - " .drop(\"column_id\")\\\n", - " .withColumnRenamed(\"data\", col_name)\n", - " \n", - " output_df = output_df.join(broadcast(col_dict_df), col_name, \"left\")\\\n", - " .drop(col_name)\\\n", - " .withColumnRenamed(\"id\", col_name)\n", - " return output_df " - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "#### 2.4 Define Main Function\n", - "In this function:\n", - "1. Parse date in Performance data by calling _parse_dates (parsed_perf)\n", - "2. Create deliqency dataframe(perf_deliqency) form Performance data by calling _create_perf_deliquency\n", - "3. Create cleaned acquisition dataframe(cleaned_acq) from Acquisition data by calling _create_acquisition\n", - "4. Join deliqency dataframe(perf_deliqency) and cleaned acquisition dataframe(cleaned_acq), get clean_df\n", - "5. Cast String column to Numbric in clean_df by calling _cast_string_columns_to_numeric, get casted_clean_df\n", - "6. Return casted_clean_df as final result" - ] - }, - { - "cell_type": "code", - "execution_count": 14, - "metadata": {}, - "outputs": [], - "source": [ - "def run_mortgage(spark, perf, acq):\n", - " parsed_perf = _parse_dates(perf)\n", - " perf_deliqency = _create_perf_deliquency(spark, parsed_perf)\n", - " cleaned_acq = _create_acquisition(spark, acq)\n", - " clean_df = perf_deliqency.join(cleaned_acq, [\"loan_id\", \"quarter\"], \"inner\").drop(\"quarter\")\n", - " casted_clean_df = _cast_string_columns_to_numeric(spark, clean_df)\\\n", - " .select(all_col_names)\\\n", - " .withColumn(label_col_name, when(col(label_col_name) > 0, 1).otherwise(0))\\\n", - " .fillna(float(0))\n", - " return casted_clean_df" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "## Script Settings\n", - "\n", - "### 1. File Path Settings\n", - "* Define input file path" - ] - }, - { - "cell_type": "code", - "execution_count": 15, - "metadata": {}, - "outputs": [], - "source": [ - "# You need to update them to your real paths!\n", - "dataRoot = os.getenv(\"DATA_ROOT\", 'gs://your-bucket/your-paths')\n", - "orig_raw_path = dataRoot + '/'" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "* Define output folder path" - ] - }, - { - "cell_type": "code", - "execution_count": 16, - "metadata": {}, - "outputs": [], - "source": [ - "output_path = dataRoot + '/output/data/'\n", - "output_path_train = dataRoot + '/output/train/'\n", - "output_path_eval = dataRoot + '/output/eval/'\n", - "output_csv2parquet = dataRoot + '/output/csv2parquet/'\n", - "\n", - "save_train_eval_dataset = True" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "### 2. Common Spark Settings" - ] - }, - { - "cell_type": "code", - "execution_count": 17, - "metadata": {}, - "outputs": [], - "source": [ - "spark.conf.set('spark.rapids.sql.explain', 'ALL')\n", - "spark.conf.set('spark.rapids.sql.batchSizeBytes', '512M')\n", - "spark.conf.set('spark.rapids.sql.reader.batchSizeBytes', '768M')" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "## Run Part" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "### Read Raw File" - ] - }, - { - "cell_type": "code", - "execution_count": 18, - "metadata": {}, - "outputs": [], - "source": [ - "rawDf = read_raw_csv(spark, orig_raw_path)\n", - "rawDf.write.parquet(output_csv2parquet, mode='overwrite')\n", - "rawDf = spark.read.parquet(output_csv2parquet)\n", - "\n", - "acq = extract_acq_columns(rawDf)\n", - "perf = extract_perf_columns(rawDf)" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "### Run ETL\n", - "#### 1. Add additional Spark settings" - ] - }, - { - "cell_type": "code", - "execution_count": 19, - "metadata": {}, - "outputs": [], - "source": [ - "# GPU run, set to true\n", - "spark.conf.set('spark.rapids.sql.enabled', 'true')\n", - "# CPU run, set to false\n", - "# spark.conf.set('spark.rapids.sql.enabled', 'false')\n", - "spark.conf.set('spark.sql.files.maxPartitionBytes', '1G')\n", - "spark.conf.set(\"spark.rapids.sql.hasNans\", \"false\")\n", - "# use GPU to read CSV\n", - "spark.conf.set(\"spark.rapids.sql.csv.read.double.enabled\", \"true\")" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "#### 2.Read Parquet File and Run ETL Process, Save the Result" - ] - }, - { - "cell_type": "code", - "execution_count": 20, - "metadata": {}, - "outputs": [ - { - "name": "stderr", - "output_type": "stream", - "text": [ - "22/07/28 08:54:20 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression row_number() windowspecdefinition(column_id#2031, count#2040L DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS id#2045 will run on GPU\n", - " *Expression row_number() windowspecdefinition(column_id#2031, count#2040L DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression row_number() will run on GPU\n", - " *Expression windowspecdefinition(column_id#2031, count#2040L DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression count#2040L DESC NULLS LAST will run on GPU\n", - " *Expression specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$()) will run on GPU\n", - " *Expression unboundedpreceding$() will run on GPU\n", - " *Expression currentrow$() will run on GPU\n", - " *Expression count#2040L DESC NULLS LAST will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression column_id#2031 ASC NULLS FIRST will run on GPU\n", - " *Expression count#2040L DESC NULLS LAST will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression count(1) will run on GPU\n", - " *Expression count(1) will run on GPU\n", - " *Expression count(1)#2039L AS count#2040L will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_count(1) will run on GPU\n", - " *Expression count(1) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression pos#2027 AS column_id#2031 will run on GPU\n", - " *Expression col#2028 AS data#2034 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression isnotnull(col#2028) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression posexplode(array(orig_channel#1783, first_home_buyer#1805, loan_purpose#1806, property_type#1807, occupancy_status#1809, property_state#1810, product_type#1814, relocation_mortgage_indicator#1860, seller_name#1660, mod_flag#41)) will run on GPU\n", - " *Expression array(orig_channel#1783, first_home_buyer#1805, loan_purpose#1806, property_type#1807, occupancy_status#1809, property_state#1810, product_type#1814, relocation_mortgage_indicator#1860, seller_name#1660, mod_flag#41) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression loan_id#1L could run on GPU\n", - " @Expression mod_flag#41 could run on GPU\n", - " @Expression quarter#216 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#1275 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#2 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#1311 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#2 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) AND isnotnull(loan_id#1L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) AND isnotnull(loan_id#1L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#11) will run on GPU\n", - " *Expression NOT (current_actual_upb#11 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#11 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#216 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) AS timestamp_year#1201L will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END AS timestamp_month#1230L will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END)) will run on GPU\n", - " *Expression isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression explode([0,1,2,3,4,5,6,7,8,9,10,11]) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1347 could run on GPU\n", - " @Expression loan_id#1349L could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1359) will run on GPU\n", - " *Expression NOT (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1349L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1347 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) AS ever_30#821 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) AS ever_90#822 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) AS ever_180#823 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) will run on GPU\n", - " *Expression min(delinquency_30#792)#809 AS delinquency_30#810 will run on GPU\n", - " *Expression min(delinquency_90#793)#811 AS delinquency_90#812 will run on GPU\n", - " *Expression min(delinquency_180#794)#813 AS delinquency_180#814 will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression partial_min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression partial_min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression partial_min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1019 could run on GPU\n", - " @Expression loan_id#1021L could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 1) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 1 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 3) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 3 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 6) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 6 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND isnotnull(loan_id#1021L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND isnotnull(loan_id#1021L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1031) will run on GPU\n", - " *Expression NOT (current_actual_upb#1031 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1031 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1021L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1019 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1781L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#1776 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) AS seller_name#1660 will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression seller_name#326 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(rank#331) AND (rank#331 = 1)) will run on GPU\n", - " *Expression isnotnull(rank#331) will run on GPU\n", - " *Expression (rank#331 = 1) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#331 will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression dense_rank(_w1#342) will run on GPU\n", - " *Expression windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Expression specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$()) will run on GPU\n", - " *Expression unboundedpreceding$() will run on GPU\n", - " *Expression currentrow$() will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression loan_id#1781L ASC NULLS FIRST will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression loan_id#1781L could run on GPU\n", - " @Expression orig_channel#1783 could run on GPU\n", - " @Expression upper(seller_name#1784) AS seller_name#326 could run on GPU\n", - " !Expression upper(seller_name#1784) cannot run on GPU because the GPU version of Upper is not 100% compatible with the Spark version. the Unicode version used by cuDF and the JVM may differ, resulting in some corner-case characters not changing case correctly.. To enable this Expression despite the incompatibilities please set the config spark.rapids.sql.expression.Upper to true. You could also set spark.rapids.sql.incompatibleOps.enabled to true to enable all incompatible ops\n", - " @Expression seller_name#1784 could run on GPU\n", - " @Expression first_home_buyer#1805 could run on GPU\n", - " @Expression loan_purpose#1806 could run on GPU\n", - " @Expression property_type#1807 could run on GPU\n", - " @Expression occupancy_status#1809 could run on GPU\n", - " @Expression property_state#1810 could run on GPU\n", - " @Expression product_type#1814 could run on GPU\n", - " @Expression relocation_mortgage_indicator#1860 could run on GPU\n", - " @Expression quarter#1776 could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) as date) AS _w1#342 could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1782 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression isnotnull(loan_id#1781L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1776 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression from_seller_name#1540 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression isnotnull(from_seller_name#1540) will run on GPU\n", - " ! cannot run on GPU because GPU does not currently support the operator class org.apache.spark.sql.execution.RDDScanExec\n", - " @Expression from_seller_name#1540 could run on GPU\n", - " @Expression to_seller_name#1541 could run on GPU\n", - "\n" - ] + "cells": [ + { + "cell_type": "markdown", + "id": "fb2fd14b-95d0-4718-878e-8ca8d32b9ed0", + "metadata": {}, + "source": [ + "## Prerequirement\n", + "### 1. Download data\n", + "Dataset is derived from Fannie Mae’s Single-Family Loan Performance Data with all rights reserved by Fannie Mae. Refer to these instructions to download the dataset.\n", + "\n", + "### 2. Set up Environments\n", + "This notebook runs in a Dataproc cluster with GPU nodes, with Spark RAPIDS set up.\n", + "\n", + "### 3. Start Jupyter Notebook\n", + "```\n", + "$ jupyter notebook --ip=0.0.0.0 --port=8124 --no-browser\n", + "```" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "433086a7-681e-416c-8340-738e8025cede", + "metadata": {}, + "outputs": [], + "source": [ + "import time\n", + "import os\n", + "from pyspark import broadcast, SparkConf\n", + "from pyspark.sql import SparkSession\n", + "from pyspark.sql.functions import *\n", + "from pyspark.sql.types import *\n", + "from pyspark.sql.window import Window" + ] + }, + { + "cell_type": "markdown", + "id": "ca42bc76-6be8-47a0-a92b-c0f67ec5ddfc", + "metadata": {}, + "source": [ + "## Create Spark Session" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "dfd231a5-83ea-4945-b801-fadb417ded9a", + "metadata": {}, + "outputs": [], + "source": [ + "if \"sc\" in globals():\n", + " sc.stop()\n", + "\n", + "### Configure the parameters based on your dataproc cluster ###\n", + "conf = SparkConf().setAppName(\"MortgageETL\")\n", + "conf.set(\"spark.plugins\", \"com.nvidia.spark.SQLPlugin\")\n", + "conf.set(\"spark.executor.instances\", \"2\")\n", + "conf.set(\"spark.executor.cores\", \"20\")\n", + "conf.set(\"spark.task.resource.gpu.amount\", \"0.05\")\n", + "conf.set(\"spark.task.cpus\", \"1\")\n", + "conf.set(\"spark.rapids.sql.concurrentGpuTasks\", \"2\")\n", + "conf.set(\"spark.executor.memory\", \"20g\")\n", + "conf.set(\"spark.sql.files.maxPartitionBytes\", \"512m\")\n", + "conf.set(\"spark.executor.resource.gpu.amount\", \"1\")\n", + "conf.set(\"spark.rapids.sql.hasNans\", \"false\")\n", + "conf.set(\"spark.rapids.sql.csv.read.double.enabled\", \"true\")\n", + "conf.set('spark.rapids.sql.batchSizeBytes', '512M')\n", + "conf.set('spark.rapids.sql.reader.batchSizeBytes', '768M')\n", + "conf.set(\"spark.rapids.memory.pinnedPool.size\", \"2G\")\n", + "conf.set(\"spark.executor.memoryOverhead\", \"2G\")\n", + "conf.set(\"spark.sql.broadcastTimeout\", \"700\")\n", + "\n", + "spark = SparkSession.builder \\\n", + " .config(conf=conf) \\\n", + " .getOrCreate()" + ] + }, + { + "cell_type": "markdown", + "id": "93d81be1-ce5a-4691-b8ac-16aaf35dfd35", + "metadata": {}, + "source": [ + "## Function Define\n", + "### 1. Define the constants\n", + "- Define input file schema (Performance and Acquisition)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "9eb4f94a-68cc-40de-80d8-b80be0e41fd7", + "metadata": {}, + "outputs": [], + "source": [ + "# File schema\n", + "_csv_raw_schema = StructType([\n", + " StructField(\"reference_pool_id\", StringType()),\n", + " StructField(\"loan_id\", LongType()),\n", + " StructField(\"monthly_reporting_period\", StringType()),\n", + " StructField(\"orig_channel\", StringType()),\n", + " StructField(\"seller_name\", StringType()),\n", + " StructField(\"servicer\", StringType()),\n", + " StructField(\"master_servicer\", StringType()),\n", + " StructField(\"orig_interest_rate\", DoubleType()),\n", + " StructField(\"interest_rate\", DoubleType()),\n", + " StructField(\"orig_upb\", DoubleType()),\n", + " StructField(\"upb_at_issuance\", StringType()),\n", + " StructField(\"current_actual_upb\", DoubleType()),\n", + " StructField(\"orig_loan_term\", IntegerType()),\n", + " StructField(\"orig_date\", StringType()),\n", + " StructField(\"first_pay_date\", StringType()), \n", + " StructField(\"loan_age\", DoubleType()),\n", + " StructField(\"remaining_months_to_legal_maturity\", DoubleType()),\n", + " StructField(\"adj_remaining_months_to_maturity\", DoubleType()),\n", + " StructField(\"maturity_date\", StringType()),\n", + " StructField(\"orig_ltv\", DoubleType()),\n", + " StructField(\"orig_cltv\", DoubleType()),\n", + " StructField(\"num_borrowers\", DoubleType()),\n", + " StructField(\"dti\", DoubleType()),\n", + " StructField(\"borrower_credit_score\", DoubleType()),\n", + " StructField(\"coborrow_credit_score\", DoubleType()),\n", + " StructField(\"first_home_buyer\", StringType()),\n", + " StructField(\"loan_purpose\", StringType()),\n", + " StructField(\"property_type\", StringType()),\n", + " StructField(\"num_units\", IntegerType()),\n", + " StructField(\"occupancy_status\", StringType()),\n", + " StructField(\"property_state\", StringType()),\n", + " StructField(\"msa\", DoubleType()),\n", + " StructField(\"zip\", IntegerType()),\n", + " StructField(\"mortgage_insurance_percent\", DoubleType()),\n", + " StructField(\"product_type\", StringType()),\n", + " StructField(\"prepayment_penalty_indicator\", StringType()),\n", + " StructField(\"interest_only_loan_indicator\", StringType()),\n", + " StructField(\"interest_only_first_principal_and_interest_payment_date\", StringType()),\n", + " StructField(\"months_to_amortization\", StringType()),\n", + " StructField(\"current_loan_delinquency_status\", IntegerType()),\n", + " StructField(\"loan_payment_history\", StringType()),\n", + " StructField(\"mod_flag\", StringType()),\n", + " StructField(\"mortgage_insurance_cancellation_indicator\", StringType()),\n", + " StructField(\"zero_balance_code\", StringType()),\n", + " StructField(\"zero_balance_effective_date\", StringType()),\n", + " StructField(\"upb_at_the_time_of_removal\", StringType()),\n", + " StructField(\"repurchase_date\", StringType()),\n", + " StructField(\"scheduled_principal_current\", StringType()),\n", + " StructField(\"total_principal_current\", StringType()),\n", + " StructField(\"unscheduled_principal_current\", StringType()),\n", + " StructField(\"last_paid_installment_date\", StringType()),\n", + " StructField(\"foreclosed_after\", StringType()),\n", + " StructField(\"disposition_date\", StringType()),\n", + " StructField(\"foreclosure_costs\", DoubleType()),\n", + " StructField(\"prop_preservation_and_repair_costs\", DoubleType()),\n", + " StructField(\"asset_recovery_costs\", DoubleType()),\n", + " StructField(\"misc_holding_expenses\", DoubleType()),\n", + " StructField(\"holding_taxes\", DoubleType()),\n", + " StructField(\"net_sale_proceeds\", DoubleType()),\n", + " StructField(\"credit_enhancement_proceeds\", DoubleType()),\n", + " StructField(\"repurchase_make_whole_proceeds\", StringType()),\n", + " StructField(\"other_foreclosure_proceeds\", DoubleType()),\n", + " StructField(\"non_interest_bearing_upb\", DoubleType()),\n", + " StructField(\"principal_forgiveness_upb\", StringType()),\n", + " StructField(\"original_list_start_date\", StringType()),\n", + " StructField(\"original_list_price\", StringType()),\n", + " StructField(\"current_list_start_date\", StringType()),\n", + " StructField(\"current_list_price\", StringType()),\n", + " StructField(\"borrower_credit_score_at_issuance\", StringType()),\n", + " StructField(\"co-borrower_credit_score_at_issuance\", StringType()),\n", + " StructField(\"borrower_credit_score_current\", StringType()),\n", + " StructField(\"co-Borrower_credit_score_current\", StringType()),\n", + " StructField(\"mortgage_insurance_type\", DoubleType()),\n", + " StructField(\"servicing_activity_indicator\", StringType()),\n", + " StructField(\"current_period_modification_loss_amount\", StringType()),\n", + " StructField(\"cumulative_modification_loss_amount\", StringType()),\n", + " StructField(\"current_period_credit_event_net_gain_or_loss\", StringType()),\n", + " StructField(\"cumulative_credit_event_net_gain_or_loss\", StringType()),\n", + " StructField(\"homeready_program_indicator\", StringType()),\n", + " StructField(\"foreclosure_principal_write_off_amount\", StringType()),\n", + " StructField(\"relocation_mortgage_indicator\", StringType()),\n", + " StructField(\"zero_balance_code_change_date\", StringType()),\n", + " StructField(\"loan_holdback_indicator\", StringType()),\n", + " StructField(\"loan_holdback_effective_date\", StringType()),\n", + " StructField(\"delinquent_accrued_interest\", StringType()),\n", + " StructField(\"property_valuation_method\", StringType()),\n", + " StructField(\"high_balance_loan_indicator\", StringType()),\n", + " StructField(\"arm_initial_fixed-rate_period_lt_5_yr_indicator\", StringType()),\n", + " StructField(\"arm_product_type\", StringType()),\n", + " StructField(\"initial_fixed-rate_period\", StringType()),\n", + " StructField(\"interest_rate_adjustment_frequency\", StringType()),\n", + " StructField(\"next_interest_rate_adjustment_date\", StringType()),\n", + " StructField(\"next_payment_change_date\", StringType()),\n", + " StructField(\"index\", StringType()),\n", + " StructField(\"arm_cap_structure\", StringType()),\n", + " StructField(\"initial_interest_rate_cap_up_percent\", StringType()),\n", + " StructField(\"periodic_interest_rate_cap_up_percent\", StringType()),\n", + " StructField(\"lifetime_interest_rate_cap_up_percent\", StringType()),\n", + " StructField(\"mortgage_margin\", StringType()),\n", + " StructField(\"arm_balloon_indicator\", StringType()),\n", + " StructField(\"arm_plan_number\", StringType()),\n", + " StructField(\"borrower_assistance_plan\", StringType()),\n", + " StructField(\"hltv_refinance_option_indicator\", StringType()),\n", + " StructField(\"deal_name\", StringType()),\n", + " StructField(\"repurchase_make_whole_proceeds_flag\", StringType()),\n", + " StructField(\"alternative_delinquency_resolution\", StringType()),\n", + " StructField(\"alternative_delinquency_resolution_count\", StringType()),\n", + " StructField(\"total_deferral_amount\", StringType())\n", + " ])" + ] + }, + { + "cell_type": "markdown", + "id": "815616a0-024b-4bb5-8a51-a048fe97983a", + "metadata": {}, + "source": [ + "- Define seller name mapping" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "01de3a77-cece-45e5-a16d-da24f56f3173", + "metadata": {}, + "outputs": [], + "source": [ + "# name mappings\n", + "_name_mapping = [\n", + " (\"WITMER FUNDING, LLC\", \"Witmer\"),\n", + " (\"WELLS FARGO CREDIT RISK TRANSFER SECURITIES TRUST 2015\", \"Wells Fargo\"),\n", + " (\"WELLS FARGO BANK, NA\" , \"Wells Fargo\"),\n", + " (\"WELLS FARGO BANK, N.A.\" , \"Wells Fargo\"),\n", + " (\"WELLS FARGO BANK, NA\" , \"Wells Fargo\"),\n", + " (\"USAA FEDERAL SAVINGS BANK\" , \"USAA\"),\n", + " (\"UNITED SHORE FINANCIAL SERVICES, LLC D\\\\/B\\\\/A UNITED WHOLESALE MORTGAGE\" , \"United Seq(e\"),\n", + " (\"U.S. BANK N.A.\" , \"US Bank\"),\n", + " (\"SUNTRUST MORTGAGE INC.\" , \"Suntrust\"),\n", + " (\"STONEGATE MORTGAGE CORPORATION\" , \"Stonegate Mortgage\"),\n", + " (\"STEARNS LENDING, LLC\" , \"Stearns Lending\"),\n", + " (\"STEARNS LENDING, INC.\" , \"Stearns Lending\"),\n", + " (\"SIERRA PACIFIC MORTGAGE COMPANY, INC.\" , \"Sierra Pacific Mortgage\"),\n", + " (\"REGIONS BANK\" , \"Regions\"),\n", + " (\"RBC MORTGAGE COMPANY\" , \"RBC\"),\n", + " (\"QUICKEN LOANS INC.\" , \"Quicken Loans\"),\n", + " (\"PULTE MORTGAGE, L.L.C.\" , \"Pulte Mortgage\"),\n", + " (\"PROVIDENT FUNDING ASSOCIATES, L.P.\" , \"Provident Funding\"),\n", + " (\"PROSPECT MORTGAGE, LLC\" , \"Prospect Mortgage\"),\n", + " (\"PRINCIPAL RESIDENTIAL MORTGAGE CAPITAL RESOURCES, LLC\" , \"Principal Residential\"),\n", + " (\"PNC BANK, N.A.\" , \"PNC\"),\n", + " (\"PMT CREDIT RISK TRANSFER TRUST 2015-2\" , \"PennyMac\"),\n", + " (\"PHH MORTGAGE CORPORATION\" , \"PHH Mortgage\"),\n", + " (\"PENNYMAC CORP.\" , \"PennyMac\"),\n", + " (\"PACIFIC UNION FINANCIAL, LLC\" , \"Other\"),\n", + " (\"OTHER\" , \"Other\"),\n", + " (\"NYCB MORTGAGE COMPANY, LLC\" , \"NYCB\"),\n", + " (\"NEW YORK COMMUNITY BANK\" , \"NYCB\"),\n", + " (\"NETBANK FUNDING SERVICES\" , \"Netbank\"),\n", + " (\"NATIONSTAR MORTGAGE, LLC\" , \"Nationstar Mortgage\"),\n", + " (\"METLIFE BANK, NA\" , \"Metlife\"),\n", + " (\"LOANDEPOT.COM, LLC\" , \"LoanDepot.com\"),\n", + " (\"J.P. MORGAN MADISON AVENUE SECURITIES TRUST, SERIES 2015-1\" , \"JP Morgan Chase\"),\n", + " (\"J.P. MORGAN MADISON AVENUE SECURITIES TRUST, SERIES 2014-1\" , \"JP Morgan Chase\"),\n", + " (\"JPMORGAN CHASE BANK, NATIONAL ASSOCIATION\" , \"JP Morgan Chase\"),\n", + " (\"JPMORGAN CHASE BANK, NA\" , \"JP Morgan Chase\"),\n", + " (\"JP MORGAN CHASE BANK, NA\" , \"JP Morgan Chase\"),\n", + " (\"IRWIN MORTGAGE, CORPORATION\" , \"Irwin Mortgage\"),\n", + " (\"IMPAC MORTGAGE CORP.\" , \"Impac Mortgage\"),\n", + " (\"HSBC BANK USA, NATIONAL ASSOCIATION\" , \"HSBC\"),\n", + " (\"HOMEWARD RESIDENTIAL, INC.\" , \"Homeward Mortgage\"),\n", + " (\"HOMESTREET BANK\" , \"Other\"),\n", + " (\"HOMEBRIDGE FINANCIAL SERVICES, INC.\" , \"HomeBridge\"),\n", + " (\"HARWOOD STREET FUNDING I, LLC\" , \"Harwood Mortgage\"),\n", + " (\"GUILD MORTGAGE COMPANY\" , \"Guild Mortgage\"),\n", + " (\"GMAC MORTGAGE, LLC (USAA FEDERAL SAVINGS BANK)\" , \"GMAC\"),\n", + " (\"GMAC MORTGAGE, LLC\" , \"GMAC\"),\n", + " (\"GMAC (USAA)\" , \"GMAC\"),\n", + " (\"FREMONT BANK\" , \"Fremont Bank\"),\n", + " (\"FREEDOM MORTGAGE CORP.\" , \"Freedom Mortgage\"),\n", + " (\"FRANKLIN AMERICAN MORTGAGE COMPANY\" , \"Franklin America\"),\n", + " (\"FLEET NATIONAL BANK\" , \"Fleet National\"),\n", + " (\"FLAGSTAR CAPITAL MARKETS CORPORATION\" , \"Flagstar Bank\"),\n", + " (\"FLAGSTAR BANK, FSB\" , \"Flagstar Bank\"),\n", + " (\"FIRST TENNESSEE BANK NATIONAL ASSOCIATION\" , \"Other\"),\n", + " (\"FIFTH THIRD BANK\" , \"Fifth Third Bank\"),\n", + " (\"FEDERAL HOME LOAN BANK OF CHICAGO\" , \"Fedral Home of Chicago\"),\n", + " (\"FDIC, RECEIVER, INDYMAC FEDERAL BANK FSB\" , \"FDIC\"),\n", + " (\"DOWNEY SAVINGS AND LOAN ASSOCIATION, F.A.\" , \"Downey Mortgage\"),\n", + " (\"DITECH FINANCIAL LLC\" , \"Ditech\"),\n", + " (\"CITIMORTGAGE, INC.\" , \"Citi\"),\n", + " (\"CHICAGO MORTGAGE SOLUTIONS DBA INTERFIRST MORTGAGE COMPANY\" , \"Chicago Mortgage\"),\n", + " (\"CHICAGO MORTGAGE SOLUTIONS DBA INTERBANK MORTGAGE COMPANY\" , \"Chicago Mortgage\"),\n", + " (\"CHASE HOME FINANCE, LLC\" , \"JP Morgan Chase\"),\n", + " (\"CHASE HOME FINANCE FRANKLIN AMERICAN MORTGAGE COMPANY\" , \"JP Morgan Chase\"),\n", + " (\"CHASE HOME FINANCE (CIE 1)\" , \"JP Morgan Chase\"),\n", + " (\"CHASE HOME FINANCE\" , \"JP Morgan Chase\"),\n", + " (\"CASHCALL, INC.\" , \"CashCall\"),\n", + " (\"CAPITAL ONE, NATIONAL ASSOCIATION\" , \"Capital One\"),\n", + " (\"CALIBER HOME LOANS, INC.\" , \"Caliber Funding\"),\n", + " (\"BISHOPS GATE RESIDENTIAL MORTGAGE TRUST\" , \"Bishops Gate Mortgage\"),\n", + " (\"BANK OF AMERICA, N.A.\" , \"Bank of America\"),\n", + " (\"AMTRUST BANK\" , \"AmTrust\"),\n", + " (\"AMERISAVE MORTGAGE CORPORATION\" , \"Amerisave\"),\n", + " (\"AMERIHOME MORTGAGE COMPANY, LLC\" , \"AmeriHome Mortgage\"),\n", + " (\"ALLY BANK\" , \"Ally Bank\"),\n", + " (\"ACADEMY MORTGAGE CORPORATION\" , \"Academy Mortgage\"),\n", + " (\"NO CASH-OUT REFINANCE\" , \"OTHER REFINANCE\"),\n", + " (\"REFINANCE - NOT SPECIFIED\" , \"OTHER REFINANCE\"),\n", + " (\"Other REFINANCE\" , \"OTHER REFINANCE\")]" + ] + }, + { + "cell_type": "markdown", + "id": "244e3222-4b5b-4570-b9e5-e345a3ba1180", + "metadata": {}, + "source": [ + "- Define category (string) column and numeric column" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "c3aa662c-c11c-458d-a20b-cdfe519051fb", + "metadata": {}, + "outputs": [], + "source": [ + "# String columns\n", + "cate_col_names = [\n", + " \"orig_channel\",\n", + " \"first_home_buyer\",\n", + " \"loan_purpose\",\n", + " \"property_type\",\n", + " \"occupancy_status\",\n", + " \"property_state\",\n", + " \"product_type\",\n", + " \"relocation_mortgage_indicator\",\n", + " \"seller_name\",\n", + " \"mod_flag\"\n", + "]\n", + "# Numberic columns\n", + "label_col_name = \"delinquency_12\"\n", + "numeric_col_names = [\n", + " \"orig_interest_rate\",\n", + " \"orig_upb\",\n", + " \"orig_loan_term\",\n", + " \"orig_ltv\",\n", + " \"orig_cltv\",\n", + " \"num_borrowers\",\n", + " \"dti\",\n", + " \"borrower_credit_score\",\n", + " \"num_units\",\n", + " \"zip\",\n", + " \"mortgage_insurance_percent\",\n", + " \"current_loan_delinquency_status\",\n", + " \"current_actual_upb\",\n", + " \"interest_rate\",\n", + " \"loan_age\",\n", + " \"msa\",\n", + " \"non_interest_bearing_upb\",\n", + " label_col_name\n", + "]\n", + "all_col_names = cate_col_names + numeric_col_names" + ] + }, + { + "cell_type": "markdown", + "id": "36c9c8b3-d251-4d09-a9ea-e4c265a39347", + "metadata": {}, + "source": [ + "- Functions to extract perf and acq columns from raw schema" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "a5710fda-c2ae-4742-8f43-d52725143737", + "metadata": {}, + "outputs": [], + "source": [ + "def extract_perf_columns(rawDf):\n", + " perfDf = rawDf.select(\n", + " col(\"loan_id\"),\n", + " date_format(to_date(col(\"monthly_reporting_period\"),\"MMyyyy\"), \"MM/dd/yyyy\").alias(\"monthly_reporting_period\"),\n", + " upper(col(\"servicer\")).alias(\"servicer\"),\n", + " col(\"interest_rate\"),\n", + " col(\"current_actual_upb\"),\n", + " col(\"loan_age\"),\n", + " col(\"remaining_months_to_legal_maturity\"),\n", + " col(\"adj_remaining_months_to_maturity\"),\n", + " date_format(to_date(col(\"maturity_date\"),\"MMyyyy\"), \"MM/yyyy\").alias(\"maturity_date\"),\n", + " col(\"msa\"),\n", + " col(\"current_loan_delinquency_status\"),\n", + " col(\"mod_flag\"),\n", + " col(\"zero_balance_code\"),\n", + " date_format(to_date(col(\"zero_balance_effective_date\"),\"MMyyyy\"), \"MM/yyyy\").alias(\"zero_balance_effective_date\"),\n", + " date_format(to_date(col(\"last_paid_installment_date\"),\"MMyyyy\"), \"MM/dd/yyyy\").alias(\"last_paid_installment_date\"),\n", + " date_format(to_date(col(\"foreclosed_after\"),\"MMyyyy\"), \"MM/dd/yyyy\").alias(\"foreclosed_after\"),\n", + " date_format(to_date(col(\"disposition_date\"),\"MMyyyy\"), \"MM/dd/yyyy\").alias(\"disposition_date\"),\n", + " col(\"foreclosure_costs\"),\n", + " col(\"prop_preservation_and_repair_costs\"),\n", + " col(\"asset_recovery_costs\"),\n", + " col(\"misc_holding_expenses\"),\n", + " col(\"holding_taxes\"),\n", + " col(\"net_sale_proceeds\"),\n", + " col(\"credit_enhancement_proceeds\"),\n", + " col(\"repurchase_make_whole_proceeds\"),\n", + " col(\"other_foreclosure_proceeds\"),\n", + " col(\"non_interest_bearing_upb\"),\n", + " col(\"principal_forgiveness_upb\"),\n", + " col(\"repurchase_make_whole_proceeds_flag\"),\n", + " col(\"foreclosure_principal_write_off_amount\"),\n", + " col(\"servicing_activity_indicator\"),\n", + " col('quarter')\n", + " )\n", + "\n", + " return perfDf.select(\"*\").filter(\"current_actual_upb != 0.0\")\n", + "\n", + "def extract_acq_columns(rawDf):\n", + " acqDf = rawDf.select(\n", + " col(\"loan_id\"),\n", + " col(\"orig_channel\"),\n", + " upper(col(\"seller_name\")).alias(\"seller_name\"),\n", + " col(\"orig_interest_rate\"),\n", + " col(\"orig_upb\"),\n", + " col(\"orig_loan_term\"),\n", + " date_format(to_date(col(\"orig_date\"),\"MMyyyy\"), \"MM/yyyy\").alias(\"orig_date\"),\n", + " date_format(to_date(col(\"first_pay_date\"),\"MMyyyy\"), \"MM/yyyy\").alias(\"first_pay_date\"),\n", + " col(\"orig_ltv\"),\n", + " col(\"orig_cltv\"),\n", + " col(\"num_borrowers\"),\n", + " col(\"dti\"),\n", + " col(\"borrower_credit_score\"),\n", + " col(\"first_home_buyer\"),\n", + " col(\"loan_purpose\"),\n", + " col(\"property_type\"),\n", + " col(\"num_units\"),\n", + " col(\"occupancy_status\"),\n", + " col(\"property_state\"),\n", + " col(\"zip\"),\n", + " col(\"mortgage_insurance_percent\"),\n", + " col(\"product_type\"),\n", + " col(\"coborrow_credit_score\"),\n", + " col(\"mortgage_insurance_type\"),\n", + " col(\"relocation_mortgage_indicator\"),\n", + " dense_rank().over(Window.partitionBy(\"loan_id\").orderBy(to_date(col(\"monthly_reporting_period\"),\"MMyyyy\"))).alias(\"rank\"),\n", + " col('quarter')\n", + " )\n", + "\n", + " return acqDf.select(\"*\").filter(col(\"rank\")==1)" + ] + }, + { + "cell_type": "markdown", + "id": "95104a51-1885-40b0-a6bc-bcef8d7f991d", + "metadata": {}, + "source": [ + "## 2. Define ETL Process\n", + "Define the function to do the ETL process\n", + "\n", + "### 2.1 Define Functions to Read Raw CSV File\n", + "- Define function to get quarter from input CSV file name" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "2b9863f5-9d0f-4154-9daf-82d3dd064961", + "metadata": {}, + "outputs": [], + "source": [ + "def _get_quarter_from_csv_file_name():\n", + " return substring_index(substring_index(input_file_name(), '.', 1), '/', -1)" + ] + }, + { + "cell_type": "markdown", + "id": "b896715b-e802-44c5-9fb8-7540a62d3112", + "metadata": {}, + "source": [ + "- Define function to read raw CSV data file" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "d62ab066-e839-46d4-a967-dfef21229531", + "metadata": {}, + "outputs": [], + "source": [ + "def read_raw_csv(spark, path):\n", + " return spark.read.format('csv') \\\n", + " .option('nullValue', '') \\\n", + " .option('header', False) \\\n", + " .option('delimiter', '|') \\\n", + " .schema(_csv_raw_schema) \\\n", + " .load(path) \\\n", + " .withColumn('quarter', _get_quarter_from_csv_file_name())" + ] + }, + { + "cell_type": "markdown", + "id": "97fecd32-3da2-4ebd-b22e-4a8bed182152", + "metadata": {}, + "source": [ + "### 2.2 Define ETL Process\n", + "- Define function to parse dates in Performance data" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "2c38305b-2c68-4b73-b938-ed8676c819a2", + "metadata": {}, + "outputs": [], + "source": [ + "def _parse_dates(perf):\n", + " return perf \\\n", + " .withColumn('monthly_reporting_period', to_date(col('monthly_reporting_period'), 'MM/dd/yyyy')) \\\n", + " .withColumn('monthly_reporting_period_month', month(col('monthly_reporting_period'))) \\\n", + " .withColumn('monthly_reporting_period_year', year(col('monthly_reporting_period'))) \\\n", + " .withColumn('monthly_reporting_period_day', dayofmonth(col('monthly_reporting_period'))) \\\n", + " .withColumn('last_paid_installment_date', to_date(col('last_paid_installment_date'), 'MM/dd/yyyy')) \\\n", + " .withColumn('foreclosed_after', to_date(col('foreclosed_after'), 'MM/dd/yyyy')) \\\n", + " .withColumn('disposition_date', to_date(col('disposition_date'), 'MM/dd/yyyy')) \\\n", + " .withColumn('maturity_date', to_date(col('maturity_date'), 'MM/yyyy')) \\\n", + " .withColumn('zero_balance_effective_date', to_date(col('zero_balance_effective_date'), 'MM/yyyy'))" + ] + }, + { + "cell_type": "markdown", + "id": "d15cb17d-4bed-4011-b11f-40aa0aafa14a", + "metadata": {}, + "source": [ + "- Define function to create deliquency data frame from Performance data" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "609b852f-86b6-4803-bbb6-dd7b2f69cc83", + "metadata": {}, + "outputs": [], + "source": [ + "def _create_perf_deliquency(spark, perf):\n", + " aggDF = perf.select(\n", + " col(\"quarter\"),\n", + " col(\"loan_id\"),\n", + " col(\"current_loan_delinquency_status\"),\n", + " when(col(\"current_loan_delinquency_status\") >= 1, col(\"monthly_reporting_period\")).alias(\"delinquency_30\"),\n", + " when(col(\"current_loan_delinquency_status\") >= 3, col(\"monthly_reporting_period\")).alias(\"delinquency_90\"),\n", + " when(col(\"current_loan_delinquency_status\") >= 6, col(\"monthly_reporting_period\")).alias(\"delinquency_180\")) \\\n", + " .groupBy(\"quarter\", \"loan_id\") \\\n", + " .agg(\n", + " max(\"current_loan_delinquency_status\").alias(\"delinquency_12\"),\n", + " min(\"delinquency_30\").alias(\"delinquency_30\"),\n", + " min(\"delinquency_90\").alias(\"delinquency_90\"),\n", + " min(\"delinquency_180\").alias(\"delinquency_180\")) \\\n", + " .select(\n", + " col(\"quarter\"),\n", + " col(\"loan_id\"),\n", + " (col(\"delinquency_12\") >= 1).alias(\"ever_30\"),\n", + " (col(\"delinquency_12\") >= 3).alias(\"ever_90\"),\n", + " (col(\"delinquency_12\") >= 6).alias(\"ever_180\"),\n", + " col(\"delinquency_30\"),\n", + " col(\"delinquency_90\"),\n", + " col(\"delinquency_180\"))\n", + " joinedDf = perf \\\n", + " .withColumnRenamed(\"monthly_reporting_period\", \"timestamp\") \\\n", + " .withColumnRenamed(\"monthly_reporting_period_month\", \"timestamp_month\") \\\n", + " .withColumnRenamed(\"monthly_reporting_period_year\", \"timestamp_year\") \\\n", + " .withColumnRenamed(\"current_loan_delinquency_status\", \"delinquency_12\") \\\n", + " .withColumnRenamed(\"current_actual_upb\", \"upb_12\") \\\n", + " .select(\"quarter\", \"loan_id\", \"timestamp\", \"delinquency_12\", \"upb_12\", \"timestamp_month\", \"timestamp_year\") \\\n", + " .join(aggDF, [\"loan_id\", \"quarter\"], \"left_outer\")\n", + "\n", + " # calculate the 12 month delinquency and upb values\n", + " months = 12\n", + " monthArray = [lit(x) for x in range(0, 12)]\n", + " # explode on a small amount of data is actually slightly more efficient than a cross join\n", + " testDf = joinedDf \\\n", + " .withColumn(\"month_y\", explode(array(monthArray))) \\\n", + " .select(\n", + " col(\"quarter\"),\n", + " floor(((col(\"timestamp_year\") * 12 + col(\"timestamp_month\")) - 24000) / months).alias(\"josh_mody\"),\n", + " floor(((col(\"timestamp_year\") * 12 + col(\"timestamp_month\")) - 24000 - col(\"month_y\")) / months).alias(\"josh_mody_n\"),\n", + " col(\"ever_30\"),\n", + " col(\"ever_90\"),\n", + " col(\"ever_180\"),\n", + " col(\"delinquency_30\"),\n", + " col(\"delinquency_90\"),\n", + " col(\"delinquency_180\"),\n", + " col(\"loan_id\"),\n", + " col(\"month_y\"),\n", + " col(\"delinquency_12\"),\n", + " col(\"upb_12\")) \\\n", + " .groupBy(\"quarter\", \"loan_id\", \"josh_mody_n\", \"ever_30\", \"ever_90\", \"ever_180\", \"delinquency_30\", \"delinquency_90\", \"delinquency_180\", \"month_y\") \\\n", + " .agg(max(\"delinquency_12\").alias(\"delinquency_12\"), min(\"upb_12\").alias(\"upb_12\")) \\\n", + " .withColumn(\"timestamp_year\", floor((lit(24000) + (col(\"josh_mody_n\") * lit(months)) + (col(\"month_y\") - 1)) / lit(12))) \\\n", + " .selectExpr('*', 'pmod(24000 + (josh_mody_n * {}) + month_y, 12) as timestamp_month_tmp'.format(months)) \\\n", + " .withColumn(\"timestamp_month\", when(col(\"timestamp_month_tmp\") == lit(0), lit(12)).otherwise(col(\"timestamp_month_tmp\"))) \\\n", + " .withColumn(\"delinquency_12\", ((col(\"delinquency_12\") > 3).cast(\"int\") + (col(\"upb_12\") == 0).cast(\"int\")).alias(\"delinquency_12\")) \\\n", + " .drop(\"timestamp_month_tmp\", \"josh_mody_n\", \"month_y\")\n", + "\n", + " return perf.withColumnRenamed(\"monthly_reporting_period_month\", \"timestamp_month\") \\\n", + " .withColumnRenamed(\"monthly_reporting_period_year\", \"timestamp_year\") \\\n", + " .join(testDf, [\"quarter\", \"loan_id\", \"timestamp_year\", \"timestamp_month\"], \"left\") \\\n", + " .drop(\"timestamp_year\", \"timestamp_month\")" + ] + }, + { + "cell_type": "markdown", + "id": "dbdce531-1e73-4941-aef5-4bfe32b6967d", + "metadata": {}, + "source": [ + "- Define function to create acquisition data frame from Acquisition data" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "54f05b8b-cab8-4199-8232-f7a930c06075", + "metadata": {}, + "outputs": [], + "source": [ + "def _create_acquisition(spark, acq):\n", + " nameMapping = spark.createDataFrame(_name_mapping, [\"from_seller_name\", \"to_seller_name\"])\n", + " return acq.join(nameMapping, col(\"seller_name\") == col(\"from_seller_name\"), \"left\") \\\n", + " .drop(\"from_seller_name\") \\\n", + " .withColumn(\"old_name\", col(\"seller_name\")) \\\n", + " .withColumn(\"seller_name\", coalesce(col(\"to_seller_name\"), col(\"seller_name\"))) \\\n", + " .drop(\"to_seller_name\") \\\n", + " .withColumn(\"orig_date\", to_date(col(\"orig_date\"), \"MM/yyyy\")) \\\n", + " .withColumn(\"first_pay_date\", to_date(col(\"first_pay_date\"), \"MM/yyyy\")) " + ] + }, + { + "cell_type": "markdown", + "id": "646a2621-d6ea-4272-8896-053405fd3cbc", + "metadata": {}, + "source": [ + "#### 2.3 Define Casting Process\\n\n", + "This part is casting String column to Numbric. Example:\n", + "```\n", + "col_1\n", + " \"a\"\n", + " \"b\"\n", + " \"c\"\n", + " \"a\"\n", + "# After String ====> Numberic\n", + "col_1\n", + " 0\n", + " 1\n", + " 2\n", + " 0\n", + "```\n", + "\n", + "- Define function to get column dictionary\n", + "\n", + " Example\n", + "\n", + " col1 = [row(data=\"a\",id=0), row(data=\"b\",id=1)]" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "54cd2769-08a8-43dd-8a15-6f9985688531", + "metadata": {}, + "outputs": [], + "source": [ + "def _gen_dictionary(etl_df, col_names):\n", + " cnt_table = etl_df.select(posexplode(array([col(i) for i in col_names])))\\\n", + " .withColumnRenamed(\"pos\", \"column_id\")\\\n", + " .withColumnRenamed(\"col\", \"data\")\\\n", + " .filter(\"data is not null\")\\\n", + " .groupBy(\"column_id\", \"data\")\\\n", + " .count()\n", + " windowed = Window.partitionBy(\"column_id\").orderBy(desc(\"count\"))\n", + " return cnt_table.withColumn(\"id\", row_number().over(windowed)).drop(\"count\")" + ] + }, + { + "cell_type": "markdown", + "id": "549b6e2e-b5f0-4394-a97a-9bd187feb113", + "metadata": {}, + "source": [ + "- Define function to convert string columns to numeric" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "bcb88949-207c-4482-9920-a4fcfb1c3aa3", + "metadata": {}, + "outputs": [], + "source": [ + "def _cast_string_columns_to_numeric(spark, input_df):\n", + " cached_dict_df = _gen_dictionary(input_df, cate_col_names).cache()\n", + " output_df = input_df\n", + " # Generate the final table with all columns being numeric.\n", + " for col_pos, col_name in enumerate(cate_col_names):\n", + " col_dict_df = cached_dict_df.filter(col(\"column_id\") == col_pos)\\\n", + " .drop(\"column_id\")\\\n", + " .withColumnRenamed(\"data\", col_name)\n", + " \n", + " output_df = output_df.join(broadcast(col_dict_df), col_name, \"left\")\\\n", + " .drop(col_name)\\\n", + " .withColumnRenamed(\"id\", col_name)\n", + " return output_df " + ] + }, + { + "cell_type": "markdown", + "id": "4819571e-72a0-48a2-8a3d-df9038c4fef8", + "metadata": {}, + "source": [ + "## 2.4 Define Main Function\n", + "In this function:\n", + "\n", + "1. Parse date in Performance data by calling _parse_dates (parsed_perf)\n", + "2. Create deliqency dataframe(perf_deliqency) form Performance data by calling _create_perf_deliquency\n", + "3. Create cleaned acquisition dataframe(cleaned_acq) from Acquisition data by calling _create_acquisition\n", + "4. Join deliqency dataframe(perf_deliqency) and cleaned acquisition dataframe(cleaned_acq), get clean_df\n", + "5. Cast String column to Numbric in clean_df by calling _cast_string_columns_to_numeric, get casted_clean_df\n", + "6. Return casted_clean_df as final result" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "fe335409-32fd-46b3-8151-1c2971fd940a", + "metadata": {}, + "outputs": [], + "source": [ + "def run_mortgage(spark, perf, acq):\n", + " parsed_perf = _parse_dates(perf)\n", + " perf_deliqency = _create_perf_deliquency(spark, parsed_perf)\n", + " cleaned_acq = _create_acquisition(spark, acq)\n", + " clean_df = perf_deliqency.join(cleaned_acq, [\"loan_id\", \"quarter\"], \"inner\").drop(\"quarter\")\n", + " casted_clean_df = _cast_string_columns_to_numeric(spark, clean_df)\\\n", + " .select(all_col_names)\\\n", + " .withColumn(label_col_name, when(col(label_col_name) > 0, 1).otherwise(0))\\\n", + " .fillna(float(0))\n", + " return casted_clean_df" + ] + }, + { + "cell_type": "markdown", + "id": "abda38e0-01ab-4453-bf41-5191015161e4", + "metadata": {}, + "source": [ + "## Script Settings\n", + "### 1. File Path Settings\n", + "- Define input file path" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "60edb468-31d5-49bf-9caa-e2e941e68149", + "metadata": {}, + "outputs": [], + "source": [ + "# You need to update these to your real paths!\n", + "dataRoot = os.getenv(\"DATA_ROOT\", 'gs://your-bucket/your-paths')\n", + "orig_raw_path = dataRoot + '/'" + ] + }, + { + "cell_type": "markdown", + "id": "7a875f70-1315-4e2d-8f7b-892f04a2cd26", + "metadata": {}, + "source": [ + "- Define output folder path" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "d856ad98-c8d0-44e9-81ac-331d67cc7aee", + "metadata": {}, + "outputs": [], + "source": [ + "output_path = dataRoot + '/output/data/'\n", + "output_path_train = dataRoot + '/output/train/'\n", + "output_path_eval = dataRoot + '/output/eval/'\n", + "output_csv2parquet = dataRoot + '/output/csv2parquet/'\n", + "\n", + "save_train_eval_dataset = True" + ] + }, + { + "cell_type": "markdown", + "id": "c15fdbf1-bf0c-4dd3-9f1d-3d16be046a54", + "metadata": {}, + "source": [ + "### 2. Common Spark Settings" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "1ed27bcf-76a0-4650-a7c0-959e10e504ae", + "metadata": {}, + "outputs": [], + "source": [ + "spark.conf.set('spark.rapids.sql.explain', 'ALL')\n", + "spark.conf.set('spark.rapids.sql.batchSizeBytes', '512M')\n", + "spark.conf.set('spark.rapids.sql.reader.batchSizeBytes', '768M')" + ] + }, + { + "cell_type": "markdown", + "id": "433c9f34-998c-48f4-8f4a-89f290818a63", + "metadata": {}, + "source": [ + "## Run Part\n", + "### Read Raw File" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "a321cc03-d5bd-4c5b-ab0b-530730d32d91", + "metadata": {}, + "outputs": [], + "source": [ + "rawDf = read_raw_csv(spark, orig_raw_path)\n", + "rawDf.write.parquet(output_csv2parquet, mode='overwrite')\n", + "rawDf = spark.read.parquet(output_csv2parquet)\n", + "\n", + "acq = extract_acq_columns(rawDf)\n", + "perf = extract_perf_columns(rawDf)" + ] + }, + { + "cell_type": "markdown", + "id": "3418557f-fab5-45d2-8f14-4c856fb3cdf3", + "metadata": {}, + "source": [ + "### Run ETL\n", + "#### 1. Add additional Spark settings" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "772afb34-6b96-43e7-a67c-9259a5196285", + "metadata": {}, + "outputs": [], + "source": [ + "# GPU run, set to true\n", + "spark.conf.set('spark.rapids.sql.enabled', 'true')\n", + "# CPU run, set to false\n", + "# spark.conf.set('spark.rapids.sql.enabled', 'false')\n", + "spark.conf.set('spark.sql.files.maxPartitionBytes', '1G')\n", + "spark.conf.set(\"spark.rapids.sql.hasNans\", \"false\")\n", + "# use GPU to read CSV\n", + "spark.conf.set(\"spark.rapids.sql.csv.read.double.enabled\", \"true\")" + ] + }, + { + "cell_type": "markdown", + "id": "ab1f090a-58ba-4df2-904d-4eefa8ca9590", + "metadata": {}, + "source": [ + "#### 2. Read Parquet File and Run ETL Process, Save the Result" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "b17367a9-12b5-480e-8681-6e03367dd729", + "metadata": {}, + "outputs": [], + "source": [ + "start = time.time()\n", + "\n", + "# run main function to process data\n", + "out = run_mortgage(spark, perf, acq)\n", + "\n", + "# save processed data\n", + "out.write.parquet(output_path, mode='overwrite')\n", + "\n", + "# save processed data\n", + "if save_train_eval_dataset:\n", + " etlDf = spark.read.parquet(output_path)\n", + "\n", + " # split 80% for training, 20% for test\n", + " splits = etlDf.randomSplit([0.8, 0.2])\n", + "\n", + " splits[0].write.format('parquet').save(output_path_train, mode=\"overwrite\")\n", + " splits[1].write.format('parquet').save(output_path_eval, mode=\"overwrite\")\n", + "\n", + "# print explain and time\n", + "print(out.explain())\n", + "end = time.time()\n", + "print(end - start)\n", + "spark.stop()" + ] + } + ], + "metadata": { + "kernelspec": { + "display_name": "Python 3", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.8.13" + } }, - { - "name": "stderr", - "output_type": "stream", - "text": [ - "22/07/28 08:54:25 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " !Exec cannot run on GPU because Columnar exchange without columnar children is inefficient\n", - " @Partitioning could run on GPU\n", - " @Expression quarter#216 could run on GPU\n", - " @Expression loan_id#1L could run on GPU\n", - " @Expression cast(timestamp_year#1311 as bigint) could run on GPU\n", - " @Expression timestamp_year#1311 could run on GPU\n", - " @Expression cast(timestamp_month#1275 as bigint) could run on GPU\n", - " @Expression timestamp_month#1275 could run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression loan_id#1L could run on GPU\n", - " @Expression interest_rate#8 could run on GPU\n", - " @Expression current_actual_upb#11 could run on GPU\n", - " @Expression loan_age#15 could run on GPU\n", - " @Expression msa#31 could run on GPU\n", - " @Expression current_loan_delinquency_status#39 could run on GPU\n", - " @Expression mod_flag#41 could run on GPU\n", - " @Expression non_interest_bearing_upb#62 could run on GPU\n", - " @Expression quarter#216 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#1275 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#2 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#1311 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#2 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) AND isnotnull(loan_id#1L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) AND isnotnull(loan_id#1L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#11) will run on GPU\n", - " *Expression NOT (current_actual_upb#11 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#11 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#216 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) AS delinquency_12#1247 will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) will run on GPU\n", - " *Expression cast((max(delinquency_12#940)#1185 > 3) as int) will run on GPU\n", - " *Expression (max(delinquency_12#940)#1185 > 3) will run on GPU\n", - " *Expression cast((min(upb_12#976)#1187 = 0.0) as int) will run on GPU\n", - " *Expression (min(upb_12#976)#1187 = 0.0) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) AS timestamp_year#1201L will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END AS timestamp_month#1230L will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression partial_min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END)) will run on GPU\n", - " *Expression isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression explode([0,1,2,3,4,5,6,7,8,9,10,11]) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1347 could run on GPU\n", - " @Expression loan_id#1349L could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 AS delinquency_12#940 could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 could run on GPU\n", - " @Expression current_actual_upb#1359 AS upb_12#976 could run on GPU\n", - " @Expression current_actual_upb#1359 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1359) will run on GPU\n", - " *Expression NOT (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1349L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1347 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) AS ever_30#821 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) AS ever_90#822 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) AS ever_180#823 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) will run on GPU\n", - " *Expression min(delinquency_30#792)#809 AS delinquency_30#810 will run on GPU\n", - " *Expression min(delinquency_90#793)#811 AS delinquency_90#812 will run on GPU\n", - " *Expression min(delinquency_180#794)#813 AS delinquency_180#814 will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression partial_min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression partial_min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression partial_min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1019 could run on GPU\n", - " @Expression loan_id#1021L could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 1) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 1 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 3) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 3 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 6) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 6 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND isnotnull(loan_id#1021L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND isnotnull(loan_id#1021L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1031) will run on GPU\n", - " *Expression NOT (current_actual_upb#1031 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1031 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1021L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1019 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1781L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#1776 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) AS seller_name#1660 will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression seller_name#326 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(rank#331) AND (rank#331 = 1)) will run on GPU\n", - " *Expression isnotnull(rank#331) will run on GPU\n", - " *Expression (rank#331 = 1) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#331 will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression dense_rank(_w1#342) will run on GPU\n", - " *Expression windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Expression specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$()) will run on GPU\n", - " *Expression unboundedpreceding$() will run on GPU\n", - " *Expression currentrow$() will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression loan_id#1781L ASC NULLS FIRST will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " !Exec cannot run on GPU because Columnar exchange without columnar children is inefficient\n", - " @Partitioning could run on GPU\n", - " @Expression loan_id#1781L could run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression loan_id#1781L could run on GPU\n", - " @Expression orig_channel#1783 could run on GPU\n", - " @Expression upper(seller_name#1784) AS seller_name#326 could run on GPU\n", - " !Expression upper(seller_name#1784) cannot run on GPU because the GPU version of Upper is not 100% compatible with the Spark version. the Unicode version used by cuDF and the JVM may differ, resulting in some corner-case characters not changing case correctly.. To enable this Expression despite the incompatibilities please set the config spark.rapids.sql.expression.Upper to true. You could also set spark.rapids.sql.incompatibleOps.enabled to true to enable all incompatible ops\n", - " @Expression seller_name#1784 could run on GPU\n", - " @Expression orig_interest_rate#1787 could run on GPU\n", - " @Expression orig_upb#1789 could run on GPU\n", - " @Expression orig_loan_term#1792 could run on GPU\n", - " @Expression orig_ltv#1799 could run on GPU\n", - " @Expression orig_cltv#1800 could run on GPU\n", - " @Expression num_borrowers#1801 could run on GPU\n", - " @Expression dti#1802 could run on GPU\n", - " @Expression borrower_credit_score#1803 could run on GPU\n", - " @Expression first_home_buyer#1805 could run on GPU\n", - " @Expression loan_purpose#1806 could run on GPU\n", - " @Expression property_type#1807 could run on GPU\n", - " @Expression num_units#1808 could run on GPU\n", - " @Expression occupancy_status#1809 could run on GPU\n", - " @Expression property_state#1810 could run on GPU\n", - " @Expression zip#1812 could run on GPU\n", - " @Expression mortgage_insurance_percent#1813 could run on GPU\n", - " @Expression product_type#1814 could run on GPU\n", - " @Expression relocation_mortgage_indicator#1860 could run on GPU\n", - " @Expression quarter#1776 could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) as date) AS _w1#342 could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1782 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression isnotnull(loan_id#1781L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1776 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression from_seller_name#1540 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression isnotnull(from_seller_name#1540) will run on GPU\n", - " ! cannot run on GPU because GPU does not currently support the operator class org.apache.spark.sql.execution.RDDScanExec\n", - " @Expression from_seller_name#1540 could run on GPU\n", - " @Expression to_seller_name#1541 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS orig_channel#2109 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 0) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 0) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 0) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 0 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS first_home_buyer#2314 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 1) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 1) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 1) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 1 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS loan_purpose#2519 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 2) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 2) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 2) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 2 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS property_type#2724 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 3) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 3) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 3) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 3 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS occupancy_status#2929 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 4) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 4) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 4) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 4 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS property_state#3134 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 5) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 5) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 5) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 5 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS product_type#3339 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 6) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 6) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 6) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 6 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS relocation_mortgage_indicator#3544 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 7) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 7) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 7) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 7 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS seller_name#3749 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 8) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 8) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 8) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 8 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS mod_flag#3954 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 9) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 9) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 9) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 9 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - "\n", - "22/07/28 08:54:25 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " !Exec cannot run on GPU because Columnar exchange without columnar children is inefficient\n", - " @Partitioning could run on GPU\n", - " @Expression quarter#216 could run on GPU\n", - " @Expression loan_id#1L could run on GPU\n", - " @Expression cast(timestamp_year#1311 as bigint) could run on GPU\n", - " @Expression timestamp_year#1311 could run on GPU\n", - " @Expression cast(timestamp_month#1275 as bigint) could run on GPU\n", - " @Expression timestamp_month#1275 could run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression loan_id#1L could run on GPU\n", - " @Expression interest_rate#8 could run on GPU\n", - " @Expression current_actual_upb#11 could run on GPU\n", - " @Expression loan_age#15 could run on GPU\n", - " @Expression msa#31 could run on GPU\n", - " @Expression current_loan_delinquency_status#39 could run on GPU\n", - " @Expression mod_flag#41 could run on GPU\n", - " @Expression non_interest_bearing_upb#62 could run on GPU\n", - " @Expression quarter#216 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#1275 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#2 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#1311 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#2 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) AND isnotnull(loan_id#1L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) AND isnotnull(loan_id#1L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#11) will run on GPU\n", - " *Expression NOT (current_actual_upb#11 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#11 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#216 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) AS delinquency_12#1247 will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) will run on GPU\n", - " *Expression cast((max(delinquency_12#940)#1185 > 3) as int) will run on GPU\n", - " *Expression (max(delinquency_12#940)#1185 > 3) will run on GPU\n", - " *Expression cast((min(upb_12#976)#1187 = 0.0) as int) will run on GPU\n", - " *Expression (min(upb_12#976)#1187 = 0.0) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) AS timestamp_year#1201L will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END AS timestamp_month#1230L will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression partial_min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END)) will run on GPU\n", - " *Expression isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression explode([0,1,2,3,4,5,6,7,8,9,10,11]) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1347 could run on GPU\n", - " @Expression loan_id#1349L could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 AS delinquency_12#940 could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 could run on GPU\n", - " @Expression current_actual_upb#1359 AS upb_12#976 could run on GPU\n", - " @Expression current_actual_upb#1359 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1359) will run on GPU\n", - " *Expression NOT (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1349L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1347 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) AS ever_30#821 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) AS ever_90#822 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) AS ever_180#823 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) will run on GPU\n", - " *Expression min(delinquency_30#792)#809 AS delinquency_30#810 will run on GPU\n", - " *Expression min(delinquency_90#793)#811 AS delinquency_90#812 will run on GPU\n", - " *Expression min(delinquency_180#794)#813 AS delinquency_180#814 will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression partial_min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression partial_min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression partial_min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1019 could run on GPU\n", - " @Expression loan_id#1021L could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 1) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 1 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 3) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 3 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 6) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 6 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND isnotnull(loan_id#1021L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND isnotnull(loan_id#1021L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1031) will run on GPU\n", - " *Expression NOT (current_actual_upb#1031 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1031 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1021L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1019 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1781L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#1776 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) AS seller_name#1660 will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression seller_name#326 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(rank#331) AND (rank#331 = 1)) will run on GPU\n", - " *Expression isnotnull(rank#331) will run on GPU\n", - " *Expression (rank#331 = 1) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#331 will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression dense_rank(_w1#342) will run on GPU\n", - " *Expression windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Expression specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$()) will run on GPU\n", - " *Expression unboundedpreceding$() will run on GPU\n", - " *Expression currentrow$() will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression loan_id#1781L ASC NULLS FIRST will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " !Exec cannot run on GPU because Columnar exchange without columnar children is inefficient\n", - " @Partitioning could run on GPU\n", - " @Expression loan_id#1781L could run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression loan_id#1781L could run on GPU\n", - " @Expression orig_channel#1783 could run on GPU\n", - " @Expression upper(seller_name#1784) AS seller_name#326 could run on GPU\n", - " !Expression upper(seller_name#1784) cannot run on GPU because the GPU version of Upper is not 100% compatible with the Spark version. the Unicode version used by cuDF and the JVM may differ, resulting in some corner-case characters not changing case correctly.. To enable this Expression despite the incompatibilities please set the config spark.rapids.sql.expression.Upper to true. You could also set spark.rapids.sql.incompatibleOps.enabled to true to enable all incompatible ops\n", - " @Expression seller_name#1784 could run on GPU\n", - " @Expression orig_interest_rate#1787 could run on GPU\n", - " @Expression orig_upb#1789 could run on GPU\n", - " @Expression orig_loan_term#1792 could run on GPU\n", - " @Expression orig_ltv#1799 could run on GPU\n", - " @Expression orig_cltv#1800 could run on GPU\n", - " @Expression num_borrowers#1801 could run on GPU\n", - " @Expression dti#1802 could run on GPU\n", - " @Expression borrower_credit_score#1803 could run on GPU\n", - " @Expression first_home_buyer#1805 could run on GPU\n", - " @Expression loan_purpose#1806 could run on GPU\n", - " @Expression property_type#1807 could run on GPU\n", - " @Expression num_units#1808 could run on GPU\n", - " @Expression occupancy_status#1809 could run on GPU\n", - " @Expression property_state#1810 could run on GPU\n", - " @Expression zip#1812 could run on GPU\n", - " @Expression mortgage_insurance_percent#1813 could run on GPU\n", - " @Expression product_type#1814 could run on GPU\n", - " @Expression relocation_mortgage_indicator#1860 could run on GPU\n", - " @Expression quarter#1776 could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) as date) AS _w1#342 could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1782 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression isnotnull(loan_id#1781L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1776 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression from_seller_name#1540 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression isnotnull(from_seller_name#1540) will run on GPU\n", - " ! cannot run on GPU because GPU does not currently support the operator class org.apache.spark.sql.execution.RDDScanExec\n", - " @Expression from_seller_name#1540 could run on GPU\n", - " @Expression to_seller_name#1541 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS orig_channel#2109 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 0) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 0) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 0) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 0 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS first_home_buyer#2314 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 1) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 1) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 1) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 1 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS loan_purpose#2519 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 2) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 2) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 2) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 2 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS property_type#2724 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 3) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 3) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 3) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 3 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS occupancy_status#2929 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 4) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 4) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 4) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 4 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS property_state#3134 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 5) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 5) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 5) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 5 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS product_type#3339 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 6) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 6) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 6) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 6 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS relocation_mortgage_indicator#3544 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 7) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 7) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 7) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 7 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS seller_name#3749 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 8) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 8) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 8) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 8 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS mod_flag#3954 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 9) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 9) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 9) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 9 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - "\n", - "22/07/28 08:54:25 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Output will run on GPU\n", - "\n", - "22/07/28 08:54:25 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " !Exec cannot run on GPU because Columnar exchange without columnar children is inefficient\n", - " @Partitioning could run on GPU\n", - " @Expression quarter#216 could run on GPU\n", - " @Expression loan_id#1L could run on GPU\n", - " @Expression cast(timestamp_year#1311 as bigint) could run on GPU\n", - " @Expression timestamp_year#1311 could run on GPU\n", - " @Expression cast(timestamp_month#1275 as bigint) could run on GPU\n", - " @Expression timestamp_month#1275 could run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression loan_id#1L could run on GPU\n", - " @Expression interest_rate#8 could run on GPU\n", - " @Expression current_actual_upb#11 could run on GPU\n", - " @Expression loan_age#15 could run on GPU\n", - " @Expression msa#31 could run on GPU\n", - " @Expression current_loan_delinquency_status#39 could run on GPU\n", - " @Expression mod_flag#41 could run on GPU\n", - " @Expression non_interest_bearing_upb#62 could run on GPU\n", - " @Expression quarter#216 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#1275 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#2 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#1311 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#2 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) AND isnotnull(loan_id#1L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) AND isnotnull(loan_id#1L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#11) will run on GPU\n", - " *Expression NOT (current_actual_upb#11 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#11 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#216 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) AS delinquency_12#1247 will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) will run on GPU\n", - " *Expression cast((max(delinquency_12#940)#1185 > 3) as int) will run on GPU\n", - " *Expression (max(delinquency_12#940)#1185 > 3) will run on GPU\n", - " *Expression cast((min(upb_12#976)#1187 = 0.0) as int) will run on GPU\n", - " *Expression (min(upb_12#976)#1187 = 0.0) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) AS timestamp_year#1201L will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END AS timestamp_month#1230L will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression partial_min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END)) will run on GPU\n", - " *Expression isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression explode([0,1,2,3,4,5,6,7,8,9,10,11]) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1347 could run on GPU\n", - " @Expression loan_id#1349L could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 AS delinquency_12#940 could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 could run on GPU\n", - " @Expression current_actual_upb#1359 AS upb_12#976 could run on GPU\n", - " @Expression current_actual_upb#1359 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1359) will run on GPU\n", - " *Expression NOT (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1349L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1347 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) AS ever_30#821 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) AS ever_90#822 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) AS ever_180#823 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) will run on GPU\n", - " *Expression min(delinquency_30#792)#809 AS delinquency_30#810 will run on GPU\n", - " *Expression min(delinquency_90#793)#811 AS delinquency_90#812 will run on GPU\n", - " *Expression min(delinquency_180#794)#813 AS delinquency_180#814 will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression partial_min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression partial_min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression partial_min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1019 could run on GPU\n", - " @Expression loan_id#1021L could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 1) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 1 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 3) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 3 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 6) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 6 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND isnotnull(loan_id#1021L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND isnotnull(loan_id#1021L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1031) will run on GPU\n", - " *Expression NOT (current_actual_upb#1031 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1031 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1021L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1019 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1781L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#1776 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) AS seller_name#1660 will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression seller_name#326 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(rank#331) AND (rank#331 = 1)) will run on GPU\n", - " *Expression isnotnull(rank#331) will run on GPU\n" - ] - }, - { - "name": "stderr", - "output_type": "stream", - "text": [ - " *Expression (rank#331 = 1) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#331 will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression dense_rank(_w1#342) will run on GPU\n", - " *Expression windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Expression specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$()) will run on GPU\n", - " *Expression unboundedpreceding$() will run on GPU\n", - " *Expression currentrow$() will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression loan_id#1781L ASC NULLS FIRST will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " !Exec cannot run on GPU because Columnar exchange without columnar children is inefficient\n", - " @Partitioning could run on GPU\n", - " @Expression loan_id#1781L could run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression loan_id#1781L could run on GPU\n", - " @Expression orig_channel#1783 could run on GPU\n", - " @Expression upper(seller_name#1784) AS seller_name#326 could run on GPU\n", - " !Expression upper(seller_name#1784) cannot run on GPU because the GPU version of Upper is not 100% compatible with the Spark version. the Unicode version used by cuDF and the JVM may differ, resulting in some corner-case characters not changing case correctly.. To enable this Expression despite the incompatibilities please set the config spark.rapids.sql.expression.Upper to true. You could also set spark.rapids.sql.incompatibleOps.enabled to true to enable all incompatible ops\n", - " @Expression seller_name#1784 could run on GPU\n", - " @Expression orig_interest_rate#1787 could run on GPU\n", - " @Expression orig_upb#1789 could run on GPU\n", - " @Expression orig_loan_term#1792 could run on GPU\n", - " @Expression orig_ltv#1799 could run on GPU\n", - " @Expression orig_cltv#1800 could run on GPU\n", - " @Expression num_borrowers#1801 could run on GPU\n", - " @Expression dti#1802 could run on GPU\n", - " @Expression borrower_credit_score#1803 could run on GPU\n", - " @Expression first_home_buyer#1805 could run on GPU\n", - " @Expression loan_purpose#1806 could run on GPU\n", - " @Expression property_type#1807 could run on GPU\n", - " @Expression num_units#1808 could run on GPU\n", - " @Expression occupancy_status#1809 could run on GPU\n", - " @Expression property_state#1810 could run on GPU\n", - " @Expression zip#1812 could run on GPU\n", - " @Expression mortgage_insurance_percent#1813 could run on GPU\n", - " @Expression product_type#1814 could run on GPU\n", - " @Expression relocation_mortgage_indicator#1860 could run on GPU\n", - " @Expression quarter#1776 could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) as date) AS _w1#342 could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1782 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression isnotnull(loan_id#1781L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1776 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression from_seller_name#1540 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression isnotnull(from_seller_name#1540) will run on GPU\n", - " ! cannot run on GPU because GPU does not currently support the operator class org.apache.spark.sql.execution.RDDScanExec\n", - " @Expression from_seller_name#1540 could run on GPU\n", - " @Expression to_seller_name#1541 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS orig_channel#2109 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 0) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 0) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 0) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 0 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS first_home_buyer#2314 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 1) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 1) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 1) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 1 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS loan_purpose#2519 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 2) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 2) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 2) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 2 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS property_type#2724 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 3) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 3) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 3) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 3 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS occupancy_status#2929 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 4) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 4) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 4) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 4 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS property_state#3134 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 5) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 5) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 5) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 5 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS product_type#3339 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 6) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 6) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 6) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 6 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS relocation_mortgage_indicator#3544 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 7) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 7) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 7) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 7 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS seller_name#3749 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 8) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 8) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 8) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 8 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS mod_flag#3954 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 9) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 9) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 9) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 9 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - "\n", - "22/07/28 08:54:25 WARN org.apache.spark.sql.catalyst.util.package: Truncated the string representation of a plan since it was too large. This behavior can be adjusted by setting 'spark.sql.debug.maxToStringFields'.\n" - ] - }, - { - "name": "stderr", - "output_type": "stream", - "text": [ - "22/07/28 08:54:26 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "!Exec cannot run on GPU because Columnar exchange without columnar children is inefficient\n", - " @Partitioning could run on GPU\n", - " @Expression quarter#216 could run on GPU\n", - " @Expression loan_id#1L could run on GPU\n", - " @Expression cast(timestamp_year#1311 as bigint) could run on GPU\n", - " @Expression timestamp_year#1311 could run on GPU\n", - " @Expression cast(timestamp_month#1275 as bigint) could run on GPU\n", - " @Expression timestamp_month#1275 could run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression loan_id#1L could run on GPU\n", - " @Expression interest_rate#8 could run on GPU\n", - " @Expression current_actual_upb#11 could run on GPU\n", - " @Expression loan_age#15 could run on GPU\n", - " @Expression msa#31 could run on GPU\n", - " @Expression current_loan_delinquency_status#39 could run on GPU\n", - " @Expression mod_flag#41 could run on GPU\n", - " @Expression non_interest_bearing_upb#62 could run on GPU\n", - " @Expression quarter#216 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#1275 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#2 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#1311 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#2 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) AND isnotnull(loan_id#1L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) AND isnotnull(loan_id#1L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#11) will run on GPU\n", - " *Expression NOT (current_actual_upb#11 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#11 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#216 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - "\n", - "22/07/28 08:54:26 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression partial_min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression partial_min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression partial_min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1019 could run on GPU\n", - " @Expression loan_id#1021L could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 1) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 1 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 3) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 3 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 6) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 6 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND isnotnull(loan_id#1021L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND isnotnull(loan_id#1021L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1031) will run on GPU\n", - " *Expression NOT (current_actual_upb#1031 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1031 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1021L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1019 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - "\n", - "22/07/28 08:54:26 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "!Exec cannot run on GPU because Columnar exchange without columnar children is inefficient\n", - " @Partitioning could run on GPU\n", - " @Expression loan_id#1781L could run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression loan_id#1781L could run on GPU\n", - " @Expression orig_channel#1783 could run on GPU\n", - " @Expression upper(seller_name#1784) AS seller_name#326 could run on GPU\n", - " !Expression upper(seller_name#1784) cannot run on GPU because the GPU version of Upper is not 100% compatible with the Spark version. the Unicode version used by cuDF and the JVM may differ, resulting in some corner-case characters not changing case correctly.. To enable this Expression despite the incompatibilities please set the config spark.rapids.sql.expression.Upper to true. You could also set spark.rapids.sql.incompatibleOps.enabled to true to enable all incompatible ops\n", - " @Expression seller_name#1784 could run on GPU\n", - " @Expression orig_interest_rate#1787 could run on GPU\n", - " @Expression orig_upb#1789 could run on GPU\n", - " @Expression orig_loan_term#1792 could run on GPU\n", - " @Expression orig_ltv#1799 could run on GPU\n", - " @Expression orig_cltv#1800 could run on GPU\n", - " @Expression num_borrowers#1801 could run on GPU\n", - " @Expression dti#1802 could run on GPU\n", - " @Expression borrower_credit_score#1803 could run on GPU\n", - " @Expression first_home_buyer#1805 could run on GPU\n", - " @Expression loan_purpose#1806 could run on GPU\n", - " @Expression property_type#1807 could run on GPU\n", - " @Expression num_units#1808 could run on GPU\n", - " @Expression occupancy_status#1809 could run on GPU\n", - " @Expression property_state#1810 could run on GPU\n", - " @Expression zip#1812 could run on GPU\n", - " @Expression mortgage_insurance_percent#1813 could run on GPU\n", - " @Expression product_type#1814 could run on GPU\n", - " @Expression relocation_mortgage_indicator#1860 could run on GPU\n", - " @Expression quarter#1776 could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) as date) AS _w1#342 could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1782 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression isnotnull(loan_id#1781L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1776 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - "\n", - "22/07/28 08:54:26 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression isnotnull(from_seller_name#1540) will run on GPU\n", - " ! cannot run on GPU because GPU does not currently support the operator class org.apache.spark.sql.execution.RDDScanExec\n", - " @Expression from_seller_name#1540 could run on GPU\n", - " @Expression to_seller_name#1541 could run on GPU\n", - "\n", - "22/07/28 08:54:26 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS orig_channel#2109 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 0) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 0) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 0) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 0 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - "\n", - "22/07/28 08:54:26 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS first_home_buyer#2314 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 1) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 1) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 1) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 1 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - "\n", - "22/07/28 08:54:26 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS loan_purpose#2519 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 2) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 2) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 2) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 2 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - "\n", - "22/07/28 08:54:26 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS property_type#2724 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 3) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 3) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 3) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 3 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - "\n", - "22/07/28 08:54:26 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS occupancy_status#2929 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 4) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 4) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 4) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 4 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - "\n", - "22/07/28 08:54:26 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS property_state#3134 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 5) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 5) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 5) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 5 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - "\n", - "22/07/28 08:54:26 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS product_type#3339 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 6) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 6) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 6) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 6 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - "\n", - "22/07/28 08:54:26 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS relocation_mortgage_indicator#3544 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 7) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 7) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 7) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 7 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - "\n", - "22/07/28 08:54:26 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS seller_name#3749 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 8) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 8) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 8) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 8 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - "\n", - "22/07/28 08:54:26 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS mod_flag#3954 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 9) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 9) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 9) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 9 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - "\n" - ] - }, - { - "name": "stderr", - "output_type": "stream", - "text": [ - "22/07/28 08:54:37 WARN com.nvidia.spark.rapids.GpuOverrides: > (0 + 5) / 16]6]\n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) AS delinquency_12#1247 will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) will run on GPU\n", - " *Expression cast((max(delinquency_12#940)#1185 > 3) as int) will run on GPU\n", - " *Expression (max(delinquency_12#940)#1185 > 3) will run on GPU\n", - " *Expression cast((min(upb_12#976)#1187 = 0.0) as int) will run on GPU\n", - " *Expression (min(upb_12#976)#1187 = 0.0) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) AS timestamp_year#1201L will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END AS timestamp_month#1230L will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression partial_min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END)) will run on GPU\n", - " *Expression isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression explode([0,1,2,3,4,5,6,7,8,9,10,11]) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1347 could run on GPU\n", - " @Expression loan_id#1349L could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 AS delinquency_12#940 could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 could run on GPU\n", - " @Expression current_actual_upb#1359 AS upb_12#976 could run on GPU\n", - " @Expression current_actual_upb#1359 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1359) will run on GPU\n", - " *Expression NOT (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1349L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1347 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) AS ever_30#821 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) AS ever_90#822 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) AS ever_180#823 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) will run on GPU\n", - " *Expression min(delinquency_30#792)#809 AS delinquency_30#810 will run on GPU\n", - " *Expression min(delinquency_90#793)#811 AS delinquency_90#812 will run on GPU\n", - " *Expression min(delinquency_180#794)#813 AS delinquency_180#814 will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1781L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#1776 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) AS seller_name#1660 will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression seller_name#326 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(rank#331) AND (rank#331 = 1)) will run on GPU\n", - " *Expression isnotnull(rank#331) will run on GPU\n", - " *Expression (rank#331 = 1) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#331 will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression dense_rank(_w1#342) will run on GPU\n", - " *Expression windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Expression specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$()) will run on GPU\n", - " *Expression unboundedpreceding$() will run on GPU\n", - " *Expression currentrow$() will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression loan_id#1781L ASC NULLS FIRST will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression from_seller_name#1540 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - "\n" - ] - }, - { - "name": "stderr", - "output_type": "stream", - "text": [ - "[Stage 1:> (2 + 6) / 16][Stage 2:> (0 + 5) / 16][Stage 4:> (1 + 5) / 16]\r", - "22/07/28 08:54:37 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) AS delinquency_12#1247 will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) will run on GPU\n", - " *Expression cast((max(delinquency_12#940)#1185 > 3) as int) will run on GPU\n", - " *Expression (max(delinquency_12#940)#1185 > 3) will run on GPU\n", - " *Expression cast((min(upb_12#976)#1187 = 0.0) as int) will run on GPU\n", - " *Expression (min(upb_12#976)#1187 = 0.0) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) AS timestamp_year#1201L will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END AS timestamp_month#1230L will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression partial_min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END)) will run on GPU\n", - " *Expression isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression explode([0,1,2,3,4,5,6,7,8,9,10,11]) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1347 could run on GPU\n", - " @Expression loan_id#1349L could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 AS delinquency_12#940 could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 could run on GPU\n", - " @Expression current_actual_upb#1359 AS upb_12#976 could run on GPU\n", - " @Expression current_actual_upb#1359 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1359) will run on GPU\n", - " *Expression NOT (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1349L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1347 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) AS ever_30#821 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) AS ever_90#822 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) AS ever_180#823 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) will run on GPU\n", - " *Expression min(delinquency_30#792)#809 AS delinquency_30#810 will run on GPU\n", - " *Expression min(delinquency_90#793)#811 AS delinquency_90#812 will run on GPU\n", - " *Expression min(delinquency_180#794)#813 AS delinquency_180#814 will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1781L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#1776 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) AS seller_name#1660 will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(rank#331) AND (rank#331 = 1)) will run on GPU\n", - " *Expression isnotnull(rank#331) will run on GPU\n", - " *Expression (rank#331 = 1) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#331 will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression dense_rank(_w1#342) will run on GPU\n", - " *Expression windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Expression specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$()) will run on GPU\n", - " *Expression unboundedpreceding$() will run on GPU\n", - " *Expression currentrow$() will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression loan_id#1781L ASC NULLS FIRST will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - "\n", - "22/07/28 08:54:37 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Exec will run on GPU\n", - "\n" - ] - }, - { - "name": "stderr", - "output_type": "stream", - "text": [ - "22/07/28 08:54:41 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) AS delinquency_12#1247 will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) will run on GPU\n", - " *Expression cast((max(delinquency_12#940)#1185 > 3) as int) will run on GPU\n", - " *Expression (max(delinquency_12#940)#1185 > 3) will run on GPU\n", - " *Expression cast((min(upb_12#976)#1187 = 0.0) as int) will run on GPU\n", - " *Expression (min(upb_12#976)#1187 = 0.0) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) AS timestamp_year#1201L will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END AS timestamp_month#1230L will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression partial_min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END)) will run on GPU\n", - " *Expression isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression explode([0,1,2,3,4,5,6,7,8,9,10,11]) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1347 could run on GPU\n", - " @Expression loan_id#1349L could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 AS delinquency_12#940 could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 could run on GPU\n", - " @Expression current_actual_upb#1359 AS upb_12#976 could run on GPU\n", - " @Expression current_actual_upb#1359 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1359) will run on GPU\n", - " *Expression NOT (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1349L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1347 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) AS ever_30#821 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) AS ever_90#822 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) AS ever_180#823 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) will run on GPU\n", - " *Expression min(delinquency_30#792)#809 AS delinquency_30#810 will run on GPU\n", - " *Expression min(delinquency_90#793)#811 AS delinquency_90#812 will run on GPU\n", - " *Expression min(delinquency_180#794)#813 AS delinquency_180#814 will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1781L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#1776 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) AS seller_name#1660 will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(rank#331) AND (rank#331 = 1)) will run on GPU\n", - " *Expression isnotnull(rank#331) will run on GPU\n", - " *Expression (rank#331 = 1) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#331 will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression dense_rank(_w1#342) will run on GPU\n", - " *Expression windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Expression specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$()) will run on GPU\n", - " *Expression unboundedpreceding$() will run on GPU\n", - " *Expression currentrow$() will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression loan_id#1781L ASC NULLS FIRST will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - "\n" - ] - }, - { - "name": "stderr", - "output_type": "stream", - "text": [ - "22/07/28 08:54:46 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) AS delinquency_12#1247 will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) will run on GPU\n", - " *Expression cast((max(delinquency_12#940)#1185 > 3) as int) will run on GPU\n", - " *Expression (max(delinquency_12#940)#1185 > 3) will run on GPU\n", - " *Expression cast((min(upb_12#976)#1187 = 0.0) as int) will run on GPU\n", - " *Expression (min(upb_12#976)#1187 = 0.0) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) AS timestamp_year#1201L will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END AS timestamp_month#1230L will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression partial_min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END)) will run on GPU\n", - " *Expression isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression explode([0,1,2,3,4,5,6,7,8,9,10,11]) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1347 could run on GPU\n", - " @Expression loan_id#1349L could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 AS delinquency_12#940 could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 could run on GPU\n", - " @Expression current_actual_upb#1359 AS upb_12#976 could run on GPU\n", - " @Expression current_actual_upb#1359 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1359) will run on GPU\n", - " *Expression NOT (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1349L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1347 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) AS ever_30#821 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) AS ever_90#822 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) AS ever_180#823 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) will run on GPU\n", - " *Expression min(delinquency_30#792)#809 AS delinquency_30#810 will run on GPU\n", - " *Expression min(delinquency_90#793)#811 AS delinquency_90#812 will run on GPU\n", - " *Expression min(delinquency_180#794)#813 AS delinquency_180#814 will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1781L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#1776 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) AS seller_name#1660 will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(rank#331) AND (rank#331 = 1)) will run on GPU\n", - " *Expression isnotnull(rank#331) will run on GPU\n", - " *Expression (rank#331 = 1) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#331 will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression dense_rank(_w1#342) will run on GPU\n", - " *Expression windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Expression specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$()) will run on GPU\n", - " *Expression unboundedpreceding$() will run on GPU\n", - " *Expression currentrow$() will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression loan_id#1781L ASC NULLS FIRST will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - "\n", - "22/07/28 08:54:46 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) AS ever_30#821 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) AS ever_90#822 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) AS ever_180#823 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) will run on GPU\n", - " *Expression min(delinquency_30#792)#809 AS delinquency_30#810 will run on GPU\n", - " *Expression min(delinquency_90#793)#811 AS delinquency_90#812 will run on GPU\n", - " *Expression min(delinquency_180#794)#813 AS delinquency_180#814 will run on GPU\n", - " *Exec will run on GPU\n", - "\n" - ] - }, - { - "name": "stderr", - "output_type": "stream", - "text": [ - "22/07/28 08:54:47 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) AS delinquency_12#1247 will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) will run on GPU\n", - " *Expression cast((max(delinquency_12#940)#1185 > 3) as int) will run on GPU\n", - " *Expression (max(delinquency_12#940)#1185 > 3) will run on GPU\n", - " *Expression cast((min(upb_12#976)#1187 = 0.0) as int) will run on GPU\n", - " *Expression (min(upb_12#976)#1187 = 0.0) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) AS timestamp_year#1201L will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END AS timestamp_month#1230L will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression partial_min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END)) will run on GPU\n", - " *Expression isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression explode([0,1,2,3,4,5,6,7,8,9,10,11]) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1347 could run on GPU\n", - " @Expression loan_id#1349L could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 AS delinquency_12#940 could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 could run on GPU\n", - " @Expression current_actual_upb#1359 AS upb_12#976 could run on GPU\n", - " @Expression current_actual_upb#1359 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1359) will run on GPU\n", - " *Expression NOT (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1349L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1347 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1781L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#1776 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) AS seller_name#1660 will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(rank#331) AND (rank#331 = 1)) will run on GPU\n", - " *Expression isnotnull(rank#331) will run on GPU\n", - " *Expression (rank#331 = 1) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#331 will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression dense_rank(_w1#342) will run on GPU\n", - " *Expression windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Expression specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$()) will run on GPU\n", - " *Expression unboundedpreceding$() will run on GPU\n", - " *Expression currentrow$() will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression loan_id#1781L ASC NULLS FIRST will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - "\n", - "22/07/28 08:54:47 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression partial_min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END)) will run on GPU\n", - " *Expression isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression explode([0,1,2,3,4,5,6,7,8,9,10,11]) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1347 could run on GPU\n", - " @Expression loan_id#1349L could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 AS delinquency_12#940 could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 could run on GPU\n", - " @Expression current_actual_upb#1359 AS upb_12#976 could run on GPU\n", - " @Expression current_actual_upb#1359 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1359) will run on GPU\n", - " *Expression NOT (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1349L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1347 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - "\n" - ] - }, - { - "name": "stderr", - "output_type": "stream", - "text": [ - "22/07/28 08:54:48 WARN com.nvidia.spark.rapids.GpuOverrides: :> (0 + 0) / 16]6]\n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) AS delinquency_12#1247 will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) will run on GPU\n", - " *Expression cast((max(delinquency_12#940)#1185 > 3) as int) will run on GPU\n", - " *Expression (max(delinquency_12#940)#1185 > 3) will run on GPU\n", - " *Expression cast((min(upb_12#976)#1187 = 0.0) as int) will run on GPU\n", - " *Expression (min(upb_12#976)#1187 = 0.0) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) AS timestamp_year#1201L will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END AS timestamp_month#1230L will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1781L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#1776 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) AS seller_name#1660 will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(rank#331) AND (rank#331 = 1)) will run on GPU\n", - " *Expression isnotnull(rank#331) will run on GPU\n", - " *Expression (rank#331 = 1) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#331 will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression dense_rank(_w1#342) will run on GPU\n", - " *Expression windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Expression specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$()) will run on GPU\n", - " *Expression unboundedpreceding$() will run on GPU\n", - " *Expression currentrow$() will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression loan_id#1781L ASC NULLS FIRST will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - "\n", - "22/07/28 08:54:48 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) AS seller_name#1660 will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(rank#331) AND (rank#331 = 1)) will run on GPU\n", - " *Expression isnotnull(rank#331) will run on GPU\n", - " *Expression (rank#331 = 1) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#331 will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression dense_rank(_w1#342) will run on GPU\n", - " *Expression windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Expression specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$()) will run on GPU\n", - " *Expression unboundedpreceding$() will run on GPU\n", - " *Expression currentrow$() will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression loan_id#1781L ASC NULLS FIRST will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " !Exec cannot run on GPU because Unable to replace CustomShuffleReader due to child not being columnar\n", - "\n" - ] - }, - { - "name": "stderr", - "output_type": "stream", - "text": [ - "22/07/28 08:54:58 WARN com.nvidia.spark.rapids.GpuOverrides: :=> (7 + 4) / 16]\n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) AS delinquency_12#1247 will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) will run on GPU\n", - " *Expression cast((max(delinquency_12#940)#1185 > 3) as int) will run on GPU\n", - " *Expression (max(delinquency_12#940)#1185 > 3) will run on GPU\n", - " *Expression cast((min(upb_12#976)#1187 = 0.0) as int) will run on GPU\n", - " *Expression (min(upb_12#976)#1187 = 0.0) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) AS timestamp_year#1201L will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END AS timestamp_month#1230L will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - "\n", - "22/07/28 08:54:58 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Exec will run on GPU\n", - "\n" - ] - }, - { - "name": "stderr", - "output_type": "stream", - "text": [ - "22/07/28 08:55:00 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) AS delinquency_12#1247 will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) will run on GPU\n", - " *Expression cast((max(delinquency_12#940)#1185 > 3) as int) will run on GPU\n", - " *Expression (max(delinquency_12#940)#1185 > 3) will run on GPU\n", - " *Expression cast((min(upb_12#976)#1187 = 0.0) as int) will run on GPU\n", - " *Expression (min(upb_12#976)#1187 = 0.0) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) AS timestamp_year#1201L will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END AS timestamp_month#1230L will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - "\n" - ] - }, - { - "name": "stderr", - "output_type": "stream", - "text": [ - "22/07/28 08:55:06 WARN com.nvidia.spark.rapids.GpuOverrides: (140 + 12) / 200]\n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) AS delinquency_12#1247 will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) will run on GPU\n", - " *Expression cast((max(delinquency_12#940)#1185 > 3) as int) will run on GPU\n", - " *Expression (max(delinquency_12#940)#1185 > 3) will run on GPU\n", - " *Expression cast((min(upb_12#976)#1187 = 0.0) as int) will run on GPU\n", - " *Expression (min(upb_12#976)#1187 = 0.0) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) AS timestamp_year#1201L will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END AS timestamp_month#1230L will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - "\n", - "22/07/28 08:55:06 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) AS delinquency_12#1247 will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) will run on GPU\n", - " *Expression cast((max(delinquency_12#940)#1185 > 3) as int) will run on GPU\n", - " *Expression (max(delinquency_12#940)#1185 > 3) will run on GPU\n", - " *Expression cast((min(upb_12#976)#1187 = 0.0) as int) will run on GPU\n", - " *Expression (min(upb_12#976)#1187 = 0.0) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) AS timestamp_year#1201L will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END AS timestamp_month#1230L will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - "\n" - ] - }, - { - "name": "stderr", - "output_type": "stream", - "text": [ - "22/07/28 08:55:07 WARN com.nvidia.spark.rapids.GpuOverrides: (7 + 10) / 17]\n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - "\n" - ] - }, - { - "name": "stderr", - "output_type": "stream", - "text": [ - "22/07/28 08:55:13 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - "\n", - "22/07/28 08:55:14 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - "\n" - ] - }, - { - "name": "stderr", - "output_type": "stream", - "text": [ - "\r", - "[Stage 26:(187 + 8) / 200][Stage 27:(185 + 5) / 200][Stage 28:(186 + 6) / 200]\r", - "\r", - " \r", - "22/07/28 08:55:14 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - "\n", - "22/07/28 08:55:14 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - "\n", - "22/07/28 08:55:14 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " !Exec cannot run on GPU because Unable to replace CustomShuffleReader due to child not being columnar\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - "\n" - ] - }, - { - "name": "stderr", - "output_type": "stream", - "text": [ - "22/07/28 08:55:22 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Output will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression orig_channel#5321 ASC NULLS FIRST will run on GPU\n", - " *Expression first_home_buyer#5322 ASC NULLS FIRST will run on GPU\n", - " *Expression loan_purpose#5323 ASC NULLS FIRST will run on GPU\n", - " *Expression property_type#5324 ASC NULLS FIRST will run on GPU\n", - " *Expression occupancy_status#5325 ASC NULLS FIRST will run on GPU\n", - " *Expression property_state#5326 ASC NULLS FIRST will run on GPU\n", - " *Expression product_type#5327 ASC NULLS FIRST will run on GPU\n", - " *Expression relocation_mortgage_indicator#5328 ASC NULLS FIRST will run on GPU\n", - " *Expression seller_name#5329 ASC NULLS FIRST will run on GPU\n", - " *Expression mod_flag#5330 ASC NULLS FIRST will run on GPU\n", - " *Expression orig_interest_rate#5331 ASC NULLS FIRST will run on GPU\n", - " *Expression orig_upb#5332 ASC NULLS FIRST will run on GPU\n", - " *Expression orig_loan_term#5333 ASC NULLS FIRST will run on GPU\n", - " *Expression orig_ltv#5334 ASC NULLS FIRST will run on GPU\n", - " *Expression orig_cltv#5335 ASC NULLS FIRST will run on GPU\n", - " *Expression num_borrowers#5336 ASC NULLS FIRST will run on GPU\n", - " *Expression dti#5337 ASC NULLS FIRST will run on GPU\n", - " *Expression borrower_credit_score#5338 ASC NULLS FIRST will run on GPU\n", - " *Expression num_units#5339 ASC NULLS FIRST will run on GPU\n", - " *Expression zip#5340 ASC NULLS FIRST will run on GPU\n", - " *Expression mortgage_insurance_percent#5341 ASC NULLS FIRST will run on GPU\n", - " *Expression current_loan_delinquency_status#5342 ASC NULLS FIRST will run on GPU\n", - " *Expression current_actual_upb#5343 ASC NULLS FIRST will run on GPU\n", - " *Expression interest_rate#5344 ASC NULLS FIRST will run on GPU\n", - " *Expression loan_age#5345 ASC NULLS FIRST will run on GPU\n", - " *Expression msa#5346 ASC NULLS FIRST will run on GPU\n", - " *Expression non_interest_bearing_upb#5347 ASC NULLS FIRST will run on GPU\n", - " *Expression delinquency_12#5348 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - "\n", - "22/07/28 08:55:27 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Output will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression orig_channel#5321 ASC NULLS FIRST will run on GPU\n", - " *Expression first_home_buyer#5322 ASC NULLS FIRST will run on GPU\n", - " *Expression loan_purpose#5323 ASC NULLS FIRST will run on GPU\n", - " *Expression property_type#5324 ASC NULLS FIRST will run on GPU\n", - " *Expression occupancy_status#5325 ASC NULLS FIRST will run on GPU\n", - " *Expression property_state#5326 ASC NULLS FIRST will run on GPU\n", - " *Expression product_type#5327 ASC NULLS FIRST will run on GPU\n", - " *Expression relocation_mortgage_indicator#5328 ASC NULLS FIRST will run on GPU\n", - " *Expression seller_name#5329 ASC NULLS FIRST will run on GPU\n", - " *Expression mod_flag#5330 ASC NULLS FIRST will run on GPU\n", - " *Expression orig_interest_rate#5331 ASC NULLS FIRST will run on GPU\n", - " *Expression orig_upb#5332 ASC NULLS FIRST will run on GPU\n", - " *Expression orig_loan_term#5333 ASC NULLS FIRST will run on GPU\n", - " *Expression orig_ltv#5334 ASC NULLS FIRST will run on GPU\n", - " *Expression orig_cltv#5335 ASC NULLS FIRST will run on GPU\n", - " *Expression num_borrowers#5336 ASC NULLS FIRST will run on GPU\n", - " *Expression dti#5337 ASC NULLS FIRST will run on GPU\n", - " *Expression borrower_credit_score#5338 ASC NULLS FIRST will run on GPU\n", - " *Expression num_units#5339 ASC NULLS FIRST will run on GPU\n", - " *Expression zip#5340 ASC NULLS FIRST will run on GPU\n", - " *Expression mortgage_insurance_percent#5341 ASC NULLS FIRST will run on GPU\n", - " *Expression current_loan_delinquency_status#5342 ASC NULLS FIRST will run on GPU\n", - " *Expression current_actual_upb#5343 ASC NULLS FIRST will run on GPU\n", - " *Expression interest_rate#5344 ASC NULLS FIRST will run on GPU\n", - " *Expression loan_age#5345 ASC NULLS FIRST will run on GPU\n", - " *Expression msa#5346 ASC NULLS FIRST will run on GPU\n", - " *Expression non_interest_bearing_upb#5347 ASC NULLS FIRST will run on GPU\n", - " *Expression delinquency_12#5348 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - "\n", - "22/07/28 08:55:32 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " !Exec cannot run on GPU because Columnar exchange without columnar children is inefficient\n", - " @Partitioning could run on GPU\n", - " @Expression quarter#216 could run on GPU\n", - " @Expression loan_id#1L could run on GPU\n", - " @Expression cast(timestamp_year#1311 as bigint) could run on GPU\n", - " @Expression timestamp_year#1311 could run on GPU\n", - " @Expression cast(timestamp_month#1275 as bigint) could run on GPU\n", - " @Expression timestamp_month#1275 could run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression loan_id#1L could run on GPU\n", - " @Expression interest_rate#8 could run on GPU\n", - " @Expression current_actual_upb#11 could run on GPU\n", - " @Expression loan_age#15 could run on GPU\n", - " @Expression msa#31 could run on GPU\n", - " @Expression current_loan_delinquency_status#39 could run on GPU\n", - " @Expression mod_flag#41 could run on GPU\n", - " @Expression non_interest_bearing_upb#62 could run on GPU\n", - " @Expression quarter#216 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#1275 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#2 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#1311 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#2 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) AND isnotnull(loan_id#1L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) AND isnotnull(loan_id#1L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#11) will run on GPU\n", - " *Expression NOT (current_actual_upb#11 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#11 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#216 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) AS delinquency_12#1247 will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) will run on GPU\n", - " *Expression cast((max(delinquency_12#940)#1185 > 3) as int) will run on GPU\n", - " *Expression (max(delinquency_12#940)#1185 > 3) will run on GPU\n", - " *Expression cast((min(upb_12#976)#1187 = 0.0) as int) will run on GPU\n", - " *Expression (min(upb_12#976)#1187 = 0.0) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) AS timestamp_year#1201L will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END AS timestamp_month#1230L will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression partial_min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END)) will run on GPU\n", - " *Expression isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression explode([0,1,2,3,4,5,6,7,8,9,10,11]) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1347 could run on GPU\n", - " @Expression loan_id#1349L could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 AS delinquency_12#940 could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 could run on GPU\n", - " @Expression current_actual_upb#1359 AS upb_12#976 could run on GPU\n", - " @Expression current_actual_upb#1359 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1359) will run on GPU\n", - " *Expression NOT (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1349L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1347 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) AS ever_30#821 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) AS ever_90#822 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) AS ever_180#823 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) will run on GPU\n", - " *Expression min(delinquency_30#792)#809 AS delinquency_30#810 will run on GPU\n", - " *Expression min(delinquency_90#793)#811 AS delinquency_90#812 will run on GPU\n", - " *Expression min(delinquency_180#794)#813 AS delinquency_180#814 will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression partial_min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression partial_min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression partial_min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1019 could run on GPU\n", - " @Expression loan_id#1021L could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 1) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 1 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 3) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 3 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 6) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 6 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND isnotnull(loan_id#1021L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND isnotnull(loan_id#1021L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1031) will run on GPU\n", - " *Expression NOT (current_actual_upb#1031 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1031 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1021L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1019 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1781L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#1776 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) AS seller_name#1660 will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression seller_name#326 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(rank#331) AND (rank#331 = 1)) will run on GPU\n", - " *Expression isnotnull(rank#331) will run on GPU\n", - " *Expression (rank#331 = 1) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#331 will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression dense_rank(_w1#342) will run on GPU\n", - " *Expression windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Expression specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$()) will run on GPU\n", - " *Expression unboundedpreceding$() will run on GPU\n", - " *Expression currentrow$() will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression loan_id#1781L ASC NULLS FIRST will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " !Exec cannot run on GPU because Columnar exchange without columnar children is inefficient\n", - " @Partitioning could run on GPU\n", - " @Expression loan_id#1781L could run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression loan_id#1781L could run on GPU\n", - " @Expression orig_channel#1783 could run on GPU\n", - " @Expression upper(seller_name#1784) AS seller_name#326 could run on GPU\n", - " !Expression upper(seller_name#1784) cannot run on GPU because the GPU version of Upper is not 100% compatible with the Spark version. the Unicode version used by cuDF and the JVM may differ, resulting in some corner-case characters not changing case correctly.. To enable this Expression despite the incompatibilities please set the config spark.rapids.sql.expression.Upper to true. You could also set spark.rapids.sql.incompatibleOps.enabled to true to enable all incompatible ops\n", - " @Expression seller_name#1784 could run on GPU\n", - " @Expression orig_interest_rate#1787 could run on GPU\n", - " @Expression orig_upb#1789 could run on GPU\n", - " @Expression orig_loan_term#1792 could run on GPU\n", - " @Expression orig_ltv#1799 could run on GPU\n", - " @Expression orig_cltv#1800 could run on GPU\n", - " @Expression num_borrowers#1801 could run on GPU\n", - " @Expression dti#1802 could run on GPU\n", - " @Expression borrower_credit_score#1803 could run on GPU\n", - " @Expression first_home_buyer#1805 could run on GPU\n", - " @Expression loan_purpose#1806 could run on GPU\n", - " @Expression property_type#1807 could run on GPU\n", - " @Expression num_units#1808 could run on GPU\n", - " @Expression occupancy_status#1809 could run on GPU\n", - " @Expression property_state#1810 could run on GPU\n", - " @Expression zip#1812 could run on GPU\n", - " @Expression mortgage_insurance_percent#1813 could run on GPU\n", - " @Expression product_type#1814 could run on GPU\n", - " @Expression relocation_mortgage_indicator#1860 could run on GPU\n", - " @Expression quarter#1776 could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) as date) AS _w1#342 could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1782 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression isnotnull(loan_id#1781L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1776 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression from_seller_name#1540 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression isnotnull(from_seller_name#1540) will run on GPU\n", - " ! cannot run on GPU because GPU does not currently support the operator class org.apache.spark.sql.execution.RDDScanExec\n", - " @Expression from_seller_name#1540 could run on GPU\n", - " @Expression to_seller_name#1541 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS orig_channel#2109 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 0) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 0) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 0) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 0 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS first_home_buyer#2314 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 1) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 1) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 1) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 1 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS loan_purpose#2519 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 2) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 2) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 2) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 2 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS property_type#2724 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 3) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 3) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 3) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 3 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS occupancy_status#2929 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 4) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 4) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 4) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 4 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS property_state#3134 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 5) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 5) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 5) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 5 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS product_type#3339 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 6) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 6) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 6) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 6 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS relocation_mortgage_indicator#3544 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 7) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 7) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 7) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 7 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS seller_name#3749 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 8) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 8) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 8) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 8 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS mod_flag#3954 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 9) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 9) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 9) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 9 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - "\n", - "22/07/28 08:55:32 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " !Exec cannot run on GPU because Columnar exchange without columnar children is inefficient\n", - " @Partitioning could run on GPU\n", - " @Expression quarter#216 could run on GPU\n", - " @Expression loan_id#1L could run on GPU\n", - " @Expression cast(timestamp_year#1311 as bigint) could run on GPU\n", - " @Expression timestamp_year#1311 could run on GPU\n", - " @Expression cast(timestamp_month#1275 as bigint) could run on GPU\n", - " @Expression timestamp_month#1275 could run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression loan_id#1L could run on GPU\n", - " @Expression interest_rate#8 could run on GPU\n", - " @Expression current_actual_upb#11 could run on GPU\n", - " @Expression loan_age#15 could run on GPU\n", - " @Expression msa#31 could run on GPU\n", - " @Expression current_loan_delinquency_status#39 could run on GPU\n", - " @Expression mod_flag#41 could run on GPU\n", - " @Expression non_interest_bearing_upb#62 could run on GPU\n", - " @Expression quarter#216 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#1275 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#2 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#1311 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#2 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) AND isnotnull(loan_id#1L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) AND isnotnull(loan_id#1L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#11) will run on GPU\n", - " *Expression NOT (current_actual_upb#11 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#11 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#216 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) AS delinquency_12#1247 will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) will run on GPU\n", - " *Expression cast((max(delinquency_12#940)#1185 > 3) as int) will run on GPU\n", - " *Expression (max(delinquency_12#940)#1185 > 3) will run on GPU\n", - " *Expression cast((min(upb_12#976)#1187 = 0.0) as int) will run on GPU\n", - " *Expression (min(upb_12#976)#1187 = 0.0) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) AS timestamp_year#1201L will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END AS timestamp_month#1230L will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression partial_min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END)) will run on GPU\n", - " *Expression isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression explode([0,1,2,3,4,5,6,7,8,9,10,11]) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1347 could run on GPU\n", - " @Expression loan_id#1349L could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 AS delinquency_12#940 could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 could run on GPU\n", - " @Expression current_actual_upb#1359 AS upb_12#976 could run on GPU\n", - " @Expression current_actual_upb#1359 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1359) will run on GPU\n", - " *Expression NOT (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1349L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1347 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) AS ever_30#821 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) AS ever_90#822 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) AS ever_180#823 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) will run on GPU\n", - " *Expression min(delinquency_30#792)#809 AS delinquency_30#810 will run on GPU\n", - " *Expression min(delinquency_90#793)#811 AS delinquency_90#812 will run on GPU\n", - " *Expression min(delinquency_180#794)#813 AS delinquency_180#814 will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression partial_min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression partial_min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression partial_min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1019 could run on GPU\n", - " @Expression loan_id#1021L could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 1) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 1 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 3) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 3 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 6) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 6 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND isnotnull(loan_id#1021L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND isnotnull(loan_id#1021L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1031) will run on GPU\n", - " *Expression NOT (current_actual_upb#1031 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1031 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1021L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1019 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1781L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#1776 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) AS seller_name#1660 will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression seller_name#326 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(rank#331) AND (rank#331 = 1)) will run on GPU\n", - " *Expression isnotnull(rank#331) will run on GPU\n", - " *Expression (rank#331 = 1) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#331 will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression dense_rank(_w1#342) will run on GPU\n", - " *Expression windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Expression specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$()) will run on GPU\n", - " *Expression unboundedpreceding$() will run on GPU\n", - " *Expression currentrow$() will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression loan_id#1781L ASC NULLS FIRST will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " !Exec cannot run on GPU because Columnar exchange without columnar children is inefficient\n", - " @Partitioning could run on GPU\n", - " @Expression loan_id#1781L could run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression loan_id#1781L could run on GPU\n", - " @Expression orig_channel#1783 could run on GPU\n", - " @Expression upper(seller_name#1784) AS seller_name#326 could run on GPU\n", - " !Expression upper(seller_name#1784) cannot run on GPU because the GPU version of Upper is not 100% compatible with the Spark version. the Unicode version used by cuDF and the JVM may differ, resulting in some corner-case characters not changing case correctly.. To enable this Expression despite the incompatibilities please set the config spark.rapids.sql.expression.Upper to true. You could also set spark.rapids.sql.incompatibleOps.enabled to true to enable all incompatible ops\n", - " @Expression seller_name#1784 could run on GPU\n", - " @Expression orig_interest_rate#1787 could run on GPU\n", - " @Expression orig_upb#1789 could run on GPU\n", - " @Expression orig_loan_term#1792 could run on GPU\n", - " @Expression orig_ltv#1799 could run on GPU\n", - " @Expression orig_cltv#1800 could run on GPU\n", - " @Expression num_borrowers#1801 could run on GPU\n", - " @Expression dti#1802 could run on GPU\n", - " @Expression borrower_credit_score#1803 could run on GPU\n", - " @Expression first_home_buyer#1805 could run on GPU\n", - " @Expression loan_purpose#1806 could run on GPU\n", - " @Expression property_type#1807 could run on GPU\n", - " @Expression num_units#1808 could run on GPU\n", - " @Expression occupancy_status#1809 could run on GPU\n", - " @Expression property_state#1810 could run on GPU\n", - " @Expression zip#1812 could run on GPU\n", - " @Expression mortgage_insurance_percent#1813 could run on GPU\n", - " @Expression product_type#1814 could run on GPU\n", - " @Expression relocation_mortgage_indicator#1860 could run on GPU\n", - " @Expression quarter#1776 could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) as date) AS _w1#342 could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1782 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression isnotnull(loan_id#1781L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1776 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression from_seller_name#1540 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression isnotnull(from_seller_name#1540) will run on GPU\n", - " ! cannot run on GPU because GPU does not currently support the operator class org.apache.spark.sql.execution.RDDScanExec\n", - " @Expression from_seller_name#1540 could run on GPU\n", - " @Expression to_seller_name#1541 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS orig_channel#2109 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 0) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 0) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 0) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 0 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS first_home_buyer#2314 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 1) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 1) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 1) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 1 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS loan_purpose#2519 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 2) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 2) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 2) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 2 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS property_type#2724 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 3) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 3) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 3) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 3 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS occupancy_status#2929 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 4) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 4) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 4) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 4 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS property_state#3134 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 5) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 5) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 5) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 5 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS product_type#3339 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 6) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 6) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 6) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 6 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS relocation_mortgage_indicator#3544 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 7) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 7) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 7) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 7 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS seller_name#3749 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 8) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 8) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 8) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 8 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS mod_flag#3954 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 9) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 9) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 9) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 9 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - "\n", - "22/07/28 08:55:32 WARN com.nvidia.spark.rapids.GpuOverrides: \n", - "*Exec will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) AS orig_channel#4242 will run on GPU\n", - " *Expression coalesce(orig_channel#2245, 0) will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243 will run on GPU\n", - " *Expression coalesce(first_home_buyer#2450, 0) will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) AS loan_purpose#4244 will run on GPU\n", - " *Expression coalesce(loan_purpose#2655, 0) will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) AS property_type#4245 will run on GPU\n", - " *Expression coalesce(property_type#2860, 0) will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) AS occupancy_status#4246 will run on GPU\n", - " *Expression coalesce(occupancy_status#3065, 0) will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) AS property_state#4247 will run on GPU\n", - " *Expression coalesce(property_state#3270, 0) will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) AS product_type#4248 will run on GPU\n", - " *Expression coalesce(product_type#3475, 0) will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249 will run on GPU\n", - " *Expression coalesce(relocation_mortgage_indicator#3680, 0) will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) AS seller_name#4250 will run on GPU\n", - " *Expression coalesce(seller_name#3885, 0) will run on GPU\n", - " *Expression coalesce(id#2045, 0) AS mod_flag#4251 will run on GPU\n", - " *Expression coalesce(id#2045, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252 will run on GPU\n", - " *Expression coalesce(nanvl(orig_interest_rate#1787, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_interest_rate#1787, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253 will run on GPU\n", - " *Expression coalesce(nanvl(orig_upb#1789, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_upb#1789, null) will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254 will run on GPU\n", - " *Expression coalesce(orig_loan_term#1792, 0) will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255 will run on GPU\n", - " *Expression coalesce(nanvl(orig_ltv#1799, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_ltv#1799, null) will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256 will run on GPU\n", - " *Expression coalesce(nanvl(orig_cltv#1800, null), 0.0) will run on GPU\n", - " *Expression nanvl(orig_cltv#1800, null) will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257 will run on GPU\n", - " *Expression coalesce(nanvl(num_borrowers#1801, null), 0.0) will run on GPU\n", - " *Expression nanvl(num_borrowers#1801, null) will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258 will run on GPU\n", - " *Expression coalesce(nanvl(dti#1802, null), 0.0) will run on GPU\n", - " *Expression nanvl(dti#1802, null) will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259 will run on GPU\n", - " *Expression coalesce(nanvl(borrower_credit_score#1803, null), 0.0) will run on GPU\n", - " *Expression nanvl(borrower_credit_score#1803, null) will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) AS num_units#4260 will run on GPU\n", - " *Expression coalesce(num_units#1808, 0) will run on GPU\n", - " *Expression coalesce(zip#1812, 0) AS zip#4261 will run on GPU\n", - " *Expression coalesce(zip#1812, 0) will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262 will run on GPU\n", - " *Expression coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) will run on GPU\n", - " *Expression nanvl(mortgage_insurance_percent#1813, null) will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263 will run on GPU\n", - " *Expression coalesce(current_loan_delinquency_status#39, 0) will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264 will run on GPU\n", - " *Expression coalesce(nanvl(current_actual_upb#11, null), 0.0) will run on GPU\n", - " *Expression nanvl(current_actual_upb#11, null) will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265 will run on GPU\n", - " *Expression coalesce(nanvl(interest_rate#8, null), 0.0) will run on GPU\n", - " *Expression nanvl(interest_rate#8, null) will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) AS loan_age#4266 will run on GPU\n", - " *Expression coalesce(nanvl(loan_age#15, null), 0.0) will run on GPU\n", - " *Expression nanvl(loan_age#15, null) will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) AS msa#4267 will run on GPU\n", - " *Expression coalesce(nanvl(msa#31, null), 0.0) will run on GPU\n", - " *Expression nanvl(msa#31, null) will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) AS non_interest_bearing_upb#4268 will run on GPU\n", - " *Expression coalesce(nanvl(non_interest_bearing_upb#62, null), 0.0) will run on GPU\n", - " *Expression nanvl(non_interest_bearing_upb#62, null) will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) AS delinquency_12#4269 will run on GPU\n", - " *Expression coalesce(CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END, 0) will run on GPU\n", - " *Expression CASE WHEN (delinquency_12#1247 > 0) THEN 1 ELSE 0 END will run on GPU\n", - " *Expression (delinquency_12#1247 > 0) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS seller_name#3885 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS relocation_mortgage_indicator#3680 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS product_type#3475 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_state#3270 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS occupancy_status#3065 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS property_type#2860 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS loan_purpose#2655 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS first_home_buyer#2450 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression id#2045 AS orig_channel#2245 will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression cast(timestamp_year#1311 as bigint) will run on GPU\n", - " *Expression cast(timestamp_month#1275 as bigint) will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#216 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_year#1311 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression cast(timestamp_month#1275 as bigint) could run on GPU but is going to be removed because parent plan is removed\n", - " !Exec cannot run on GPU because Columnar exchange without columnar children is inefficient\n", - " @Partitioning could run on GPU\n", - " @Expression quarter#216 could run on GPU\n", - " @Expression loan_id#1L could run on GPU\n", - " @Expression cast(timestamp_year#1311 as bigint) could run on GPU\n", - " @Expression timestamp_year#1311 could run on GPU\n", - " @Expression cast(timestamp_month#1275 as bigint) could run on GPU\n", - " @Expression timestamp_month#1275 could run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression loan_id#1L could run on GPU\n", - " @Expression interest_rate#8 could run on GPU\n", - " @Expression current_actual_upb#11 could run on GPU\n", - " @Expression loan_age#15 could run on GPU\n", - " @Expression msa#31 could run on GPU\n", - " @Expression current_loan_delinquency_status#39 could run on GPU\n", - " @Expression mod_flag#41 could run on GPU\n", - " @Expression non_interest_bearing_upb#62 could run on GPU\n", - " @Expression quarter#216 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#1275 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#2 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#1311 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#2 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) AND isnotnull(loan_id#1L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) AND isnotnull(loan_id#1L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#11) will run on GPU\n", - " *Expression NOT (current_actual_upb#11 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#11 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#216 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression quarter#1347 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression loan_id#1349L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_year#1201L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression timestamp_month#1230L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) AS delinquency_12#1247 will run on GPU\n", - " *Expression (cast((max(delinquency_12#940)#1185 > 3) as int) + cast((min(upb_12#976)#1187 = 0.0) as int)) will run on GPU\n", - " *Expression cast((max(delinquency_12#940)#1185 > 3) as int) will run on GPU\n", - " *Expression (max(delinquency_12#940)#1185 > 3) will run on GPU\n", - " *Expression cast((min(upb_12#976)#1187 = 0.0) as int) will run on GPU\n", - " *Expression (min(upb_12#976)#1187 = 0.0) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) AS timestamp_year#1201L will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END AS timestamp_month#1230L will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (josh_mody_n#1158L * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (josh_mody_n#1158L * 12)) will run on GPU\n", - " *Expression (josh_mody_n#1158L * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(delinquency_12#940) will run on GPU\n", - " *Expression max(delinquency_12#940) will run on GPU\n", - " *Expression partial_min(upb_12#976) will run on GPU\n", - " *Expression min(upb_12#976) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END)) will run on GPU\n", - " *Expression isnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) will run on GPU\n", - " *Expression FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0) will run on GPU\n", - " *Expression cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast((month_y#1142 - 1) as bigint) will run on GPU\n", - " *Expression (month_y#1142 - 1) will run on GPU\n", - " *Expression isnotnull(CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END) will run on GPU\n", - " *Expression CASE WHEN (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) THEN 12 ELSE pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) END will run on GPU\n", - " *Expression (pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) = 0) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Expression pmod(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)), 12) will run on GPU\n", - " *Expression ((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) will run on GPU\n", - " *Expression (24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) will run on GPU\n", - " *Expression (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12) will run on GPU\n", - " *Expression FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) will run on GPU\n", - " *Expression (cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0) will run on GPU\n", - " *Expression cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) will run on GPU\n", - " *Expression ((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) will run on GPU\n", - " *Expression (((timestamp_year#904 * 12) + timestamp_month#868) - 24000) will run on GPU\n", - " *Expression ((timestamp_year#904 * 12) + timestamp_month#868) will run on GPU\n", - " *Expression (timestamp_year#904 * 12) will run on GPU\n", - " *Expression cast(month_y#1142 as bigint) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression explode([0,1,2,3,4,5,6,7,8,9,10,11]) will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1347 could run on GPU\n", - " @Expression loan_id#1349L could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 AS delinquency_12#940 could run on GPU\n", - " @Expression current_loan_delinquency_status#1387 could run on GPU\n", - " @Expression current_actual_upb#1359 AS upb_12#976 could run on GPU\n", - " @Expression current_actual_upb#1359 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868 could run on GPU\n", - " @Expression month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904 could run on GPU\n", - " @Expression year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1350 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND isnotnull(loan_id#1349L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1359) will run on GPU\n", - " *Expression NOT (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1359 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1349L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1347 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) AS ever_30#821 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 1) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) AS ever_90#822 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 3) will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) AS ever_180#823 will run on GPU\n", - " *Expression (max(current_loan_delinquency_status#1059)#807 >= 6) will run on GPU\n", - " *Expression min(delinquency_30#792)#809 AS delinquency_30#810 will run on GPU\n", - " *Expression min(delinquency_90#793)#811 AS delinquency_90#812 will run on GPU\n", - " *Expression min(delinquency_180#794)#813 AS delinquency_180#814 will run on GPU\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression partial_max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression max(current_loan_delinquency_status#1059) will run on GPU\n", - " *Expression partial_min(delinquency_30#792) will run on GPU\n", - " *Expression min(delinquency_30#792) will run on GPU\n", - " *Expression partial_min(delinquency_90#793) will run on GPU\n", - " *Expression min(delinquency_90#793) will run on GPU\n", - " *Expression partial_min(delinquency_180#794) will run on GPU\n", - " *Expression min(delinquency_180#794) will run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression quarter#1019 could run on GPU\n", - " @Expression loan_id#1021L could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 1) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 1 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 3) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 3 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794 could run on GPU\n", - " @Expression CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END could run on GPU\n", - " @Expression (current_loan_delinquency_status#1059 >= 6) could run on GPU\n", - " @Expression current_loan_delinquency_status#1059 could run on GPU\n", - " @Expression 6 could run on GPU\n", - " @Expression cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " @Expression gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) could run on GPU\n", - " @Expression date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)) could run on GPU\n", - " @Expression cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp) could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1022 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " @Expression MM/dd/yyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (((isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND isnotnull(loan_id#1021L)) AND true) will run on GPU\n", - " *Expression ((isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND isnotnull(loan_id#1021L)) will run on GPU\n", - " *Expression (isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) will run on GPU\n", - " *Expression isnotnull(current_actual_upb#1031) will run on GPU\n", - " *Expression NOT (current_actual_upb#1031 = 0.0) will run on GPU\n", - " *Expression (current_actual_upb#1031 = 0.0) will run on GPU\n", - " *Expression isnotnull(loan_id#1021L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1019 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression loan_id#1781L ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " #Expression quarter#1776 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) AS seller_name#1660 will run on GPU\n", - " *Expression coalesce(to_seller_name#1541, seller_name#326) will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression seller_name#326 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression (isnotnull(rank#331) AND (rank#331 = 1)) will run on GPU\n", - " *Expression isnotnull(rank#331) will run on GPU\n", - " *Expression (rank#331 = 1) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank#331 will run on GPU\n", - " *Expression dense_rank(_w1#342) windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression dense_rank(_w1#342) will run on GPU\n", - " *Expression windowspecdefinition(loan_id#1781L, _w1#342 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Expression specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$()) will run on GPU\n", - " *Expression unboundedpreceding$() will run on GPU\n", - " *Expression currentrow$() will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression loan_id#1781L ASC NULLS FIRST will run on GPU\n", - " *Expression _w1#342 ASC NULLS FIRST will run on GPU\n", - " !Exec cannot run on GPU because Columnar exchange without columnar children is inefficient\n", - " @Partitioning could run on GPU\n", - " @Expression loan_id#1781L could run on GPU\n", - " !Exec cannot run on GPU because not all expressions can be replaced\n", - " @Expression loan_id#1781L could run on GPU\n", - " @Expression orig_channel#1783 could run on GPU\n", - " @Expression upper(seller_name#1784) AS seller_name#326 could run on GPU\n", - " !Expression upper(seller_name#1784) cannot run on GPU because the GPU version of Upper is not 100% compatible with the Spark version. the Unicode version used by cuDF and the JVM may differ, resulting in some corner-case characters not changing case correctly.. To enable this Expression despite the incompatibilities please set the config spark.rapids.sql.expression.Upper to true. You could also set spark.rapids.sql.incompatibleOps.enabled to true to enable all incompatible ops\n", - " @Expression seller_name#1784 could run on GPU\n", - " @Expression orig_interest_rate#1787 could run on GPU\n", - " @Expression orig_upb#1789 could run on GPU\n", - " @Expression orig_loan_term#1792 could run on GPU\n", - " @Expression orig_ltv#1799 could run on GPU\n", - " @Expression orig_cltv#1800 could run on GPU\n", - " @Expression num_borrowers#1801 could run on GPU\n", - " @Expression dti#1802 could run on GPU\n", - " @Expression borrower_credit_score#1803 could run on GPU\n", - " @Expression first_home_buyer#1805 could run on GPU\n", - " @Expression loan_purpose#1806 could run on GPU\n", - " @Expression property_type#1807 could run on GPU\n", - " @Expression num_units#1808 could run on GPU\n", - " @Expression occupancy_status#1809 could run on GPU\n", - " @Expression property_state#1810 could run on GPU\n", - " @Expression zip#1812 could run on GPU\n", - " @Expression mortgage_insurance_percent#1813 could run on GPU\n", - " @Expression product_type#1814 could run on GPU\n", - " @Expression relocation_mortgage_indicator#1860 could run on GPU\n", - " @Expression quarter#1776 could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) as date) AS _w1#342 could run on GPU\n", - " @Expression cast(gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) as date) could run on GPU\n", - " !Expression gettimestamp(monthly_reporting_period#1782, MMyyyy, Some(Etc/UTC), false) cannot run on GPU because CORRECTED format 'MMyyyy' on the GPU is not guaranteed to produce the same results as Spark on CPU. Set spark.rapids.sql.incompatibleDateFormats.enabled=true to force onto GPU.\n", - " @Expression monthly_reporting_period#1782 could run on GPU\n", - " @Expression MMyyyy could run on GPU\n", - " *Exec will run on GPU\n", - " *Expression isnotnull(loan_id#1781L) will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1776 will run on GPU\n", - " *Expression substring_index(substring_index(input_file_name(), ., 1), /, -1) will run on GPU\n", - " *Expression substring_index(input_file_name(), ., 1) will run on GPU\n", - " *Expression input_file_name() will run on GPU\n", - " *Exec will run on GPU\n", - " #Exec could run on GPU but is going to be removed because replacing sortMergeJoin with shuffleHashJoin\n", - " #Expression from_seller_name#1540 ASC NULLS FIRST could run on GPU but is going to be removed because parent plan is removed\n", - " *Exec will run on GPU\n", - " *Partitioning will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression isnotnull(from_seller_name#1540) will run on GPU\n", - " ! cannot run on GPU because GPU does not currently support the operator class org.apache.spark.sql.execution.RDDScanExec\n", - " @Expression from_seller_name#1540 could run on GPU\n", - " @Expression to_seller_name#1541 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS orig_channel#2109 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 0) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 0) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 0) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 0 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS first_home_buyer#2314 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 1) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 1) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 1) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 1 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS loan_purpose#2519 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 2) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 2) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 2) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 2 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS property_type#2724 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 3) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 3) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 3) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 3 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS occupancy_status#2929 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 4) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 4) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 4) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 4 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS property_state#3134 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 5) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 5) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 5) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 5 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS product_type#3339 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 6) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 6) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 6) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 6 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS relocation_mortgage_indicator#3544 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 7) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 7) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 7) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 7 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS seller_name#3749 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 8) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 8) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 8) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 8 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " *Exec will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression data#2034 AS mod_flag#3954 will run on GPU\n", - " *Exec will run on GPU\n", - " *Expression ((column_id#2031 = 9) AND isnotnull(data#2034)) will run on GPU\n", - " *Expression (column_id#2031 = 9) will run on GPU\n", - " *Expression isnotnull(data#2034) will run on GPU\n", - " !Exec cannot run on GPU because ParquetCachedBatchSerializer is not being used\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - " @Expression id#2045 could run on GPU\n", - " @Expression (column_id#2031 = 9) could run on GPU\n", - " @Expression column_id#2031 could run on GPU\n", - " @Expression 9 could run on GPU\n", - " @Expression isnotnull(data#2034) could run on GPU\n", - " @Expression data#2034 could run on GPU\n", - "\n" - ] - }, - { - "name": "stdout", - "output_type": "stream", - "text": [ - "== Physical Plan ==\n", - "AdaptiveSparkPlan isFinalPlan=false\n", - "+- Project [coalesce(orig_channel#2245, 0) AS orig_channel#4242, coalesce(first_home_buyer#2450, 0) AS first_home_buyer#4243, coalesce(loan_purpose#2655, 0) AS loan_purpose#4244, coalesce(property_type#2860, 0) AS property_type#4245, coalesce(occupancy_status#3065, 0) AS occupancy_status#4246, coalesce(property_state#3270, 0) AS property_state#4247, coalesce(product_type#3475, 0) AS product_type#4248, coalesce(relocation_mortgage_indicator#3680, 0) AS relocation_mortgage_indicator#4249, coalesce(seller_name#3885, 0) AS seller_name#4250, coalesce(id#2045, 0) AS mod_flag#4251, coalesce(nanvl(orig_interest_rate#1787, null), 0.0) AS orig_interest_rate#4252, coalesce(nanvl(orig_upb#1789, null), 0.0) AS orig_upb#4253, coalesce(orig_loan_term#1792, 0) AS orig_loan_term#4254, coalesce(nanvl(orig_ltv#1799, null), 0.0) AS orig_ltv#4255, coalesce(nanvl(orig_cltv#1800, null), 0.0) AS orig_cltv#4256, coalesce(nanvl(num_borrowers#1801, null), 0.0) AS num_borrowers#4257, coalesce(nanvl(dti#1802, null), 0.0) AS dti#4258, coalesce(nanvl(borrower_credit_score#1803, null), 0.0) AS borrower_credit_score#4259, coalesce(num_units#1808, 0) AS num_units#4260, coalesce(zip#1812, 0) AS zip#4261, coalesce(nanvl(mortgage_insurance_percent#1813, null), 0.0) AS mortgage_insurance_percent#4262, coalesce(current_loan_delinquency_status#39, 0) AS current_loan_delinquency_status#4263, coalesce(nanvl(current_actual_upb#11, null), 0.0) AS current_actual_upb#4264, coalesce(nanvl(interest_rate#8, null), 0.0) AS interest_rate#4265, ... 4 more fields]\n", - " +- BroadcastHashJoin [mod_flag#41], [mod_flag#3954], LeftOuter, BuildRight, false\n", - " :- Project [interest_rate#8, current_actual_upb#11, loan_age#15, msa#31, current_loan_delinquency_status#39, mod_flag#41, non_interest_bearing_upb#62, delinquency_12#1247, orig_interest_rate#1787, orig_upb#1789, orig_loan_term#1792, orig_ltv#1799, orig_cltv#1800, num_borrowers#1801, dti#1802, borrower_credit_score#1803, num_units#1808, zip#1812, mortgage_insurance_percent#1813, orig_channel#2245, first_home_buyer#2450, loan_purpose#2655, property_type#2860, occupancy_status#3065, ... 4 more fields]\n", - " : +- BroadcastHashJoin [seller_name#1660], [seller_name#3749], LeftOuter, BuildRight, false\n", - " : :- Project [interest_rate#8, current_actual_upb#11, loan_age#15, msa#31, current_loan_delinquency_status#39, mod_flag#41, non_interest_bearing_upb#62, delinquency_12#1247, seller_name#1660, orig_interest_rate#1787, orig_upb#1789, orig_loan_term#1792, orig_ltv#1799, orig_cltv#1800, num_borrowers#1801, dti#1802, borrower_credit_score#1803, num_units#1808, zip#1812, mortgage_insurance_percent#1813, orig_channel#2245, first_home_buyer#2450, loan_purpose#2655, property_type#2860, ... 4 more fields]\n", - " : : +- BroadcastHashJoin [relocation_mortgage_indicator#1860], [relocation_mortgage_indicator#3544], LeftOuter, BuildRight, false\n", - " : : :- Project [interest_rate#8, current_actual_upb#11, loan_age#15, msa#31, current_loan_delinquency_status#39, mod_flag#41, non_interest_bearing_upb#62, delinquency_12#1247, seller_name#1660, orig_interest_rate#1787, orig_upb#1789, orig_loan_term#1792, orig_ltv#1799, orig_cltv#1800, num_borrowers#1801, dti#1802, borrower_credit_score#1803, num_units#1808, zip#1812, mortgage_insurance_percent#1813, relocation_mortgage_indicator#1860, orig_channel#2245, first_home_buyer#2450, loan_purpose#2655, ... 4 more fields]\n", - " : : : +- BroadcastHashJoin [product_type#1814], [product_type#3339], LeftOuter, BuildRight, false\n", - " : : : :- Project [interest_rate#8, current_actual_upb#11, loan_age#15, msa#31, current_loan_delinquency_status#39, mod_flag#41, non_interest_bearing_upb#62, delinquency_12#1247, seller_name#1660, orig_interest_rate#1787, orig_upb#1789, orig_loan_term#1792, orig_ltv#1799, orig_cltv#1800, num_borrowers#1801, dti#1802, borrower_credit_score#1803, num_units#1808, zip#1812, mortgage_insurance_percent#1813, product_type#1814, relocation_mortgage_indicator#1860, orig_channel#2245, first_home_buyer#2450, ... 4 more fields]\n", - " : : : : +- BroadcastHashJoin [property_state#1810], [property_state#3134], LeftOuter, BuildRight, false\n", - " : : : : :- Project [interest_rate#8, current_actual_upb#11, loan_age#15, msa#31, current_loan_delinquency_status#39, mod_flag#41, non_interest_bearing_upb#62, delinquency_12#1247, seller_name#1660, orig_interest_rate#1787, orig_upb#1789, orig_loan_term#1792, orig_ltv#1799, orig_cltv#1800, num_borrowers#1801, dti#1802, borrower_credit_score#1803, num_units#1808, property_state#1810, zip#1812, mortgage_insurance_percent#1813, product_type#1814, relocation_mortgage_indicator#1860, orig_channel#2245, ... 4 more fields]\n", - " : : : : : +- BroadcastHashJoin [occupancy_status#1809], [occupancy_status#2929], LeftOuter, BuildRight, false\n", - " : : : : : :- Project [interest_rate#8, current_actual_upb#11, loan_age#15, msa#31, current_loan_delinquency_status#39, mod_flag#41, non_interest_bearing_upb#62, delinquency_12#1247, seller_name#1660, orig_interest_rate#1787, orig_upb#1789, orig_loan_term#1792, orig_ltv#1799, orig_cltv#1800, num_borrowers#1801, dti#1802, borrower_credit_score#1803, num_units#1808, occupancy_status#1809, property_state#1810, zip#1812, mortgage_insurance_percent#1813, product_type#1814, relocation_mortgage_indicator#1860, ... 4 more fields]\n", - " : : : : : : +- BroadcastHashJoin [property_type#1807], [property_type#2724], LeftOuter, BuildRight, false\n", - " : : : : : : :- Project [interest_rate#8, current_actual_upb#11, loan_age#15, msa#31, current_loan_delinquency_status#39, mod_flag#41, non_interest_bearing_upb#62, delinquency_12#1247, seller_name#1660, orig_interest_rate#1787, orig_upb#1789, orig_loan_term#1792, orig_ltv#1799, orig_cltv#1800, num_borrowers#1801, dti#1802, borrower_credit_score#1803, property_type#1807, num_units#1808, occupancy_status#1809, property_state#1810, zip#1812, mortgage_insurance_percent#1813, product_type#1814, ... 4 more fields]\n", - " : : : : : : : +- BroadcastHashJoin [loan_purpose#1806], [loan_purpose#2519], LeftOuter, BuildRight, false\n", - " : : : : : : : :- Project [interest_rate#8, current_actual_upb#11, loan_age#15, msa#31, current_loan_delinquency_status#39, mod_flag#41, non_interest_bearing_upb#62, delinquency_12#1247, seller_name#1660, orig_interest_rate#1787, orig_upb#1789, orig_loan_term#1792, orig_ltv#1799, orig_cltv#1800, num_borrowers#1801, dti#1802, borrower_credit_score#1803, loan_purpose#1806, property_type#1807, num_units#1808, occupancy_status#1809, property_state#1810, zip#1812, mortgage_insurance_percent#1813, ... 4 more fields]\n", - " : : : : : : : : +- BroadcastHashJoin [first_home_buyer#1805], [first_home_buyer#2314], LeftOuter, BuildRight, false\n", - " : : : : : : : : :- Project [interest_rate#8, current_actual_upb#11, loan_age#15, msa#31, current_loan_delinquency_status#39, mod_flag#41, non_interest_bearing_upb#62, delinquency_12#1247, seller_name#1660, orig_interest_rate#1787, orig_upb#1789, orig_loan_term#1792, orig_ltv#1799, orig_cltv#1800, num_borrowers#1801, dti#1802, borrower_credit_score#1803, first_home_buyer#1805, loan_purpose#1806, property_type#1807, num_units#1808, occupancy_status#1809, property_state#1810, zip#1812, ... 4 more fields]\n", - " : : : : : : : : : +- BroadcastHashJoin [orig_channel#1783], [orig_channel#2109], LeftOuter, BuildRight, false\n", - " : : : : : : : : : :- Project [interest_rate#8, current_actual_upb#11, loan_age#15, msa#31, current_loan_delinquency_status#39, mod_flag#41, non_interest_bearing_upb#62, delinquency_12#1247, orig_channel#1783, seller_name#1660, orig_interest_rate#1787, orig_upb#1789, orig_loan_term#1792, orig_ltv#1799, orig_cltv#1800, num_borrowers#1801, dti#1802, borrower_credit_score#1803, first_home_buyer#1805, loan_purpose#1806, property_type#1807, num_units#1808, occupancy_status#1809, property_state#1810, ... 4 more fields]\n", - " : : : : : : : : : : +- SortMergeJoin [loan_id#1L, quarter#216], [loan_id#1781L, quarter#1776], Inner\n", - " : : : : : : : : : : :- Sort [loan_id#1L ASC NULLS FIRST, quarter#216 ASC NULLS FIRST], false, 0\n", - " : : : : : : : : : : : +- Exchange hashpartitioning(loan_id#1L, quarter#216, 200), ENSURE_REQUIREMENTS, [id=#8836]\n", - " : : : : : : : : : : : +- Project [quarter#216, loan_id#1L, interest_rate#8, current_actual_upb#11, loan_age#15, msa#31, current_loan_delinquency_status#39, mod_flag#41, non_interest_bearing_upb#62, delinquency_12#1247]\n", - " : : : : : : : : : : : +- SortMergeJoin [quarter#216, loan_id#1L, cast(timestamp_year#1311 as bigint), cast(timestamp_month#1275 as bigint)], [quarter#1347, loan_id#1349L, timestamp_year#1201L, timestamp_month#1230L], LeftOuter\n", - " : : : : : : : : : : : :- Sort [quarter#216 ASC NULLS FIRST, loan_id#1L ASC NULLS FIRST, cast(timestamp_year#1311 as bigint) ASC NULLS FIRST, cast(timestamp_month#1275 as bigint) ASC NULLS FIRST], false, 0\n", - " : : : : : : : : : : : : +- Exchange hashpartitioning(quarter#216, loan_id#1L, cast(timestamp_year#1311 as bigint), cast(timestamp_month#1275 as bigint), 200), ENSURE_REQUIREMENTS, [id=#8815]\n", - " : : : : : : : : : : : : +- Project [loan_id#1L, interest_rate#8, current_actual_upb#11, loan_age#15, msa#31, current_loan_delinquency_status#39, mod_flag#41, non_interest_bearing_upb#62, quarter#216, month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#1275, year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#2, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#1311]\n", - " : : : : : : : : : : : : +- Filter (((isnotnull(current_actual_upb#11) AND NOT (current_actual_upb#11 = 0.0)) AND isnotnull(loan_id#1L)) AND true)\n", - " : : : : : : : : : : : : +- Project [loan_id#1L, monthly_reporting_period#2, interest_rate#8, current_actual_upb#11, loan_age#15, msa#31, current_loan_delinquency_status#39, mod_flag#41, non_interest_bearing_upb#62, substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#216]\n", - " : : : : : : : : : : : : +- FileScan csv [loan_id#1L,monthly_reporting_period#2,interest_rate#8,current_actual_upb#11,loan_age#15,msa#31,current_loan_delinquency_status#39,mod_flag#41,non_interest_bearing_upb#62] Batched: false, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792, CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793, CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794]\n", - " : : : : : : : : : : : +- Filter (((isnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND isnotnull(loan_id#1021L)) AND true)\n", - " : : : : : : : : : : : +- Project [loan_id#1021L, monthly_reporting_period#1022, current_actual_upb#1031, current_loan_delinquency_status#1059, substring_index(substring_index(input_file_name(), ., 1), /, -1) AS quarter#1019]\n", - " : : : : : : : : : : : +- FileScan csv [loan_id#1021L,monthly_reporting_period#1022,current_actual_upb#1031,current_loan_delinquency_status#1059] Batched: false, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : : : : : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, timestamp_year#1201L, timestamp_month#1230L, 200), ENSURE_REQUIREMENTS, [id=#895]\n", - " : : : : : : : : : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : : : : : : : : : +- GpuShuffleCoalesce 536870912\n", - " : : : : : : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142, 200), ENSURE_REQUIREMENTS, [id=#892]\n", - " : : : : : : : : : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : : : : : : : : : +- GpuProject [quarter#1347, FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, loan_id#1349L, month_y#1142]\n", - " : : : : : : : : : : +- GpuCoalesceBatches targetsize(536870912)\n", - " : : : : : : : : : : +- GpuFilter (gpuisnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND gpuisnotnull(CASE WHEN ((((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) = 0) THEN 12 ELSE (((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) END)), true\n", - " : : : : : : : : : : +- GpuGenerate gpuexplode([0,1,2,3,4,5,6,7,8,9,10,11]), [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814], false, [month_y#1142]\n", - " : : : : : : : : : : +- GpuProject [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814]\n", - " : : : : : : : : : : +- GpuBroadcastHashJoin [loan_id#1349L, quarter#1347], [loan_id#1021L, quarter#1019], LeftOuter, GpuBuildRight\n", - " : : : : : : : : : : :- GpuRowToColumnar targetsize(536870912)\n", - " : : : : : : : : : : : +- *(2) Project [quarter#1347, loan_id#1349L, month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868, year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904]\n", - " : : : : : : : : : : : +- GpuColumnarToRow false\n", - " : : : : : : : : : : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND gpuisnotnull(loan_id#1349L)) AND true), true\n", - " : : : : : : : : : : : +- GpuProject [loan_id#1349L, monthly_reporting_period#1350, current_actual_upb#1359, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1347]\n", - " : : : : : : : : : : : +- GpuFileGpuScan csv [loan_id#1349L,monthly_reporting_period#1350,current_actual_upb#1359] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : : : : : : : : : +- GpuBroadcastExchange HashedRelationBroadcastMode(List(input[1, bigint, true], input[0, string, false]),false), [id=#884]\n", - " : : : : : : : : : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[gpumax(current_loan_delinquency_status#1059), gpumin(delinquency_30#792), gpumin(delinquency_90#793), gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : : : : : : : : : +- GpuShuffleCoalesce 536870912\n", - " : : : : : : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1019, loan_id#1021L, 200), ENSURE_REQUIREMENTS, [id=#881]\n", - " : : : : : : : : : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[partial_gpumax(current_loan_delinquency_status#1059), partial_gpumin(delinquency_30#792), partial_gpumin(delinquency_90#793), partial_gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : : : : : : : : : +- GpuRowToColumnar targetsize(536870912)\n", - " : : : : : : : : : : +- *(3) Project [quarter#1019, loan_id#1021L, current_loan_delinquency_status#1059, CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792, CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793, CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794]\n", - " : : : : : : : : : : +- GpuColumnarToRow false\n", - " : : : : : : : : : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND gpuisnotnull(loan_id#1021L)) AND true), true\n", - " : : : : : : : : : : +- GpuProject [loan_id#1021L, monthly_reporting_period#1022, current_actual_upb#1031, current_loan_delinquency_status#1059, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1019]\n", - " : : : : : : : : : : +- GpuFileGpuScan csv [loan_id#1021L,monthly_reporting_period#1022,current_actual_upb#1031,current_loan_delinquency_status#1059] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : : : : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, timestamp_year#1201L, timestamp_month#1230L, 200), ENSURE_REQUIREMENTS, [id=#895]\n", - " : : : : : : : : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : : : : : : : : +- GpuShuffleCoalesce 536870912\n", - " : : : : : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142, 200), ENSURE_REQUIREMENTS, [id=#892]\n", - " : : : : : : : : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : : : : : : : : +- GpuProject [quarter#1347, FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, loan_id#1349L, month_y#1142]\n", - " : : : : : : : : : +- GpuCoalesceBatches targetsize(536870912)\n", - " : : : : : : : : : +- GpuFilter (gpuisnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND gpuisnotnull(CASE WHEN ((((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) = 0) THEN 12 ELSE (((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) END)), true\n", - " : : : : : : : : : +- GpuGenerate gpuexplode([0,1,2,3,4,5,6,7,8,9,10,11]), [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814], false, [month_y#1142]\n", - " : : : : : : : : : +- GpuProject [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814]\n", - " : : : : : : : : : +- GpuBroadcastHashJoin [loan_id#1349L, quarter#1347], [loan_id#1021L, quarter#1019], LeftOuter, GpuBuildRight\n", - " : : : : : : : : : :- GpuRowToColumnar targetsize(536870912)\n", - " : : : : : : : : : : +- *(2) Project [quarter#1347, loan_id#1349L, month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868, year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904]\n", - " : : : : : : : : : : +- GpuColumnarToRow false\n", - " : : : : : : : : : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND gpuisnotnull(loan_id#1349L)) AND true), true\n", - " : : : : : : : : : : +- GpuProject [loan_id#1349L, monthly_reporting_period#1350, current_actual_upb#1359, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1347]\n", - " : : : : : : : : : : +- GpuFileGpuScan csv [loan_id#1349L,monthly_reporting_period#1350,current_actual_upb#1359] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : : : : : : : : +- GpuBroadcastExchange HashedRelationBroadcastMode(List(input[1, bigint, true], input[0, string, false]),false), [id=#884]\n", - " : : : : : : : : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[gpumax(current_loan_delinquency_status#1059), gpumin(delinquency_30#792), gpumin(delinquency_90#793), gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : : : : : : : : +- GpuShuffleCoalesce 536870912\n", - " : : : : : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1019, loan_id#1021L, 200), ENSURE_REQUIREMENTS, [id=#881]\n", - " : : : : : : : : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[partial_gpumax(current_loan_delinquency_status#1059), partial_gpumin(delinquency_30#792), partial_gpumin(delinquency_90#793), partial_gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : : : : : : : : +- GpuRowToColumnar targetsize(536870912)\n", - " : : : : : : : : : +- *(3) Project [quarter#1019, loan_id#1021L, current_loan_delinquency_status#1059, CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792, CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793, CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794]\n", - " : : : : : : : : : +- GpuColumnarToRow false\n", - " : : : : : : : : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND gpuisnotnull(loan_id#1021L)) AND true), true\n", - " : : : : : : : : : +- GpuProject [loan_id#1021L, monthly_reporting_period#1022, current_actual_upb#1031, current_loan_delinquency_status#1059, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1019]\n", - " : : : : : : : : : +- GpuFileGpuScan csv [loan_id#1021L,monthly_reporting_period#1022,current_actual_upb#1031,current_loan_delinquency_status#1059] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : : : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, timestamp_year#1201L, timestamp_month#1230L, 200), ENSURE_REQUIREMENTS, [id=#895]\n", - " : : : : : : : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : : : : : : : +- GpuShuffleCoalesce 536870912\n", - " : : : : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142, 200), ENSURE_REQUIREMENTS, [id=#892]\n", - " : : : : : : : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : : : : : : : +- GpuProject [quarter#1347, FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, loan_id#1349L, month_y#1142]\n", - " : : : : : : : : +- GpuCoalesceBatches targetsize(536870912)\n", - " : : : : : : : : +- GpuFilter (gpuisnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND gpuisnotnull(CASE WHEN ((((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) = 0) THEN 12 ELSE (((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) END)), true\n", - " : : : : : : : : +- GpuGenerate gpuexplode([0,1,2,3,4,5,6,7,8,9,10,11]), [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814], false, [month_y#1142]\n", - " : : : : : : : : +- GpuProject [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814]\n", - " : : : : : : : : +- GpuBroadcastHashJoin [loan_id#1349L, quarter#1347], [loan_id#1021L, quarter#1019], LeftOuter, GpuBuildRight\n", - " : : : : : : : : :- GpuRowToColumnar targetsize(536870912)\n", - " : : : : : : : : : +- *(2) Project [quarter#1347, loan_id#1349L, month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868, year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904]\n", - " : : : : : : : : : +- GpuColumnarToRow false\n", - " : : : : : : : : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND gpuisnotnull(loan_id#1349L)) AND true), true\n", - " : : : : : : : : : +- GpuProject [loan_id#1349L, monthly_reporting_period#1350, current_actual_upb#1359, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1347]\n", - " : : : : : : : : : +- GpuFileGpuScan csv [loan_id#1349L,monthly_reporting_period#1350,current_actual_upb#1359] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : : : : : : : +- GpuBroadcastExchange HashedRelationBroadcastMode(List(input[1, bigint, true], input[0, string, false]),false), [id=#884]\n", - " : : : : : : : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[gpumax(current_loan_delinquency_status#1059), gpumin(delinquency_30#792), gpumin(delinquency_90#793), gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : : : : : : : +- GpuShuffleCoalesce 536870912\n", - " : : : : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1019, loan_id#1021L, 200), ENSURE_REQUIREMENTS, [id=#881]\n", - " : : : : : : : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[partial_gpumax(current_loan_delinquency_status#1059), partial_gpumin(delinquency_30#792), partial_gpumin(delinquency_90#793), partial_gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : : : : : : : +- GpuRowToColumnar targetsize(536870912)\n", - " : : : : : : : : +- *(3) Project [quarter#1019, loan_id#1021L, current_loan_delinquency_status#1059, CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792, CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793, CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794]\n", - " : : : : : : : : +- GpuColumnarToRow false\n", - " : : : : : : : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND gpuisnotnull(loan_id#1021L)) AND true), true\n", - " : : : : : : : : +- GpuProject [loan_id#1021L, monthly_reporting_period#1022, current_actual_upb#1031, current_loan_delinquency_status#1059, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1019]\n", - " : : : : : : : : +- GpuFileGpuScan csv [loan_id#1021L,monthly_reporting_period#1022,current_actual_upb#1031,current_loan_delinquency_status#1059] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, timestamp_year#1201L, timestamp_month#1230L, 200), ENSURE_REQUIREMENTS, [id=#895]\n", - " : : : : : : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : : : : : : +- GpuShuffleCoalesce 536870912\n", - " : : : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142, 200), ENSURE_REQUIREMENTS, [id=#892]\n", - " : : : : : : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : : : : : : +- GpuProject [quarter#1347, FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, loan_id#1349L, month_y#1142]\n", - " : : : : : : : +- GpuCoalesceBatches targetsize(536870912)\n", - " : : : : : : : +- GpuFilter (gpuisnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND gpuisnotnull(CASE WHEN ((((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) = 0) THEN 12 ELSE (((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) END)), true\n", - " : : : : : : : +- GpuGenerate gpuexplode([0,1,2,3,4,5,6,7,8,9,10,11]), [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814], false, [month_y#1142]\n", - " : : : : : : : +- GpuProject [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814]\n", - " : : : : : : : +- GpuBroadcastHashJoin [loan_id#1349L, quarter#1347], [loan_id#1021L, quarter#1019], LeftOuter, GpuBuildRight\n", - " : : : : : : : :- GpuRowToColumnar targetsize(536870912)\n", - " : : : : : : : : +- *(2) Project [quarter#1347, loan_id#1349L, month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868, year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904]\n", - " : : : : : : : : +- GpuColumnarToRow false\n", - " : : : : : : : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND gpuisnotnull(loan_id#1349L)) AND true), true\n", - " : : : : : : : : +- GpuProject [loan_id#1349L, monthly_reporting_period#1350, current_actual_upb#1359, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1347]\n", - " : : : : : : : : +- GpuFileGpuScan csv [loan_id#1349L,monthly_reporting_period#1350,current_actual_upb#1359] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : : : : : : +- GpuBroadcastExchange HashedRelationBroadcastMode(List(input[1, bigint, true], input[0, string, false]),false), [id=#884]\n", - " : : : : : : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[gpumax(current_loan_delinquency_status#1059), gpumin(delinquency_30#792), gpumin(delinquency_90#793), gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : : : : : : +- GpuShuffleCoalesce 536870912\n", - " : : : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1019, loan_id#1021L, 200), ENSURE_REQUIREMENTS, [id=#881]\n", - " : : : : : : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[partial_gpumax(current_loan_delinquency_status#1059), partial_gpumin(delinquency_30#792), partial_gpumin(delinquency_90#793), partial_gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : : : : : : +- GpuRowToColumnar targetsize(536870912)\n", - " : : : : : : : +- *(3) Project [quarter#1019, loan_id#1021L, current_loan_delinquency_status#1059, CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792, CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793, CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794]\n", - " : : : : : : : +- GpuColumnarToRow false\n", - " : : : : : : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND gpuisnotnull(loan_id#1021L)) AND true), true\n", - " : : : : : : : +- GpuProject [loan_id#1021L, monthly_reporting_period#1022, current_actual_upb#1031, current_loan_delinquency_status#1059, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1019]\n", - " : : : : : : : +- GpuFileGpuScan csv [loan_id#1021L,monthly_reporting_period#1022,current_actual_upb#1031,current_loan_delinquency_status#1059] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, timestamp_year#1201L, timestamp_month#1230L, 200), ENSURE_REQUIREMENTS, [id=#895]\n", - " : : : : : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : : : : : +- GpuShuffleCoalesce 536870912\n", - " : : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142, 200), ENSURE_REQUIREMENTS, [id=#892]\n", - " : : : : : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : : : : : +- GpuProject [quarter#1347, FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, loan_id#1349L, month_y#1142]\n", - " : : : : : : +- GpuCoalesceBatches targetsize(536870912)\n", - " : : : : : : +- GpuFilter (gpuisnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND gpuisnotnull(CASE WHEN ((((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) = 0) THEN 12 ELSE (((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) END)), true\n", - " : : : : : : +- GpuGenerate gpuexplode([0,1,2,3,4,5,6,7,8,9,10,11]), [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814], false, [month_y#1142]\n", - " : : : : : : +- GpuProject [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814]\n", - " : : : : : : +- GpuBroadcastHashJoin [loan_id#1349L, quarter#1347], [loan_id#1021L, quarter#1019], LeftOuter, GpuBuildRight\n", - " : : : : : : :- GpuRowToColumnar targetsize(536870912)\n", - " : : : : : : : +- *(2) Project [quarter#1347, loan_id#1349L, month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868, year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904]\n", - " : : : : : : : +- GpuColumnarToRow false\n", - " : : : : : : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND gpuisnotnull(loan_id#1349L)) AND true), true\n", - " : : : : : : : +- GpuProject [loan_id#1349L, monthly_reporting_period#1350, current_actual_upb#1359, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1347]\n", - " : : : : : : : +- GpuFileGpuScan csv [loan_id#1349L,monthly_reporting_period#1350,current_actual_upb#1359] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : : : : : +- GpuBroadcastExchange HashedRelationBroadcastMode(List(input[1, bigint, true], input[0, string, false]),false), [id=#884]\n", - " : : : : : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[gpumax(current_loan_delinquency_status#1059), gpumin(delinquency_30#792), gpumin(delinquency_90#793), gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : : : : : +- GpuShuffleCoalesce 536870912\n", - " : : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1019, loan_id#1021L, 200), ENSURE_REQUIREMENTS, [id=#881]\n", - " : : : : : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[partial_gpumax(current_loan_delinquency_status#1059), partial_gpumin(delinquency_30#792), partial_gpumin(delinquency_90#793), partial_gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : : : : : +- GpuRowToColumnar targetsize(536870912)\n", - " : : : : : : +- *(3) Project [quarter#1019, loan_id#1021L, current_loan_delinquency_status#1059, CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792, CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793, CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794]\n", - " : : : : : : +- GpuColumnarToRow false\n", - " : : : : : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND gpuisnotnull(loan_id#1021L)) AND true), true\n", - " : : : : : : +- GpuProject [loan_id#1021L, monthly_reporting_period#1022, current_actual_upb#1031, current_loan_delinquency_status#1059, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1019]\n", - " : : : : : : +- GpuFileGpuScan csv [loan_id#1021L,monthly_reporting_period#1022,current_actual_upb#1031,current_loan_delinquency_status#1059] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, timestamp_year#1201L, timestamp_month#1230L, 200), ENSURE_REQUIREMENTS, [id=#895]\n", - " : : : : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : : : : +- GpuShuffleCoalesce 536870912\n", - " : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142, 200), ENSURE_REQUIREMENTS, [id=#892]\n", - " : : : : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : : : : +- GpuProject [quarter#1347, FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, loan_id#1349L, month_y#1142]\n", - " : : : : : +- GpuCoalesceBatches targetsize(536870912)\n", - " : : : : : +- GpuFilter (gpuisnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND gpuisnotnull(CASE WHEN ((((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) = 0) THEN 12 ELSE (((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) END)), true\n", - " : : : : : +- GpuGenerate gpuexplode([0,1,2,3,4,5,6,7,8,9,10,11]), [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814], false, [month_y#1142]\n", - " : : : : : +- GpuProject [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814]\n", - " : : : : : +- GpuBroadcastHashJoin [loan_id#1349L, quarter#1347], [loan_id#1021L, quarter#1019], LeftOuter, GpuBuildRight\n", - " : : : : : :- GpuRowToColumnar targetsize(536870912)\n", - " : : : : : : +- *(2) Project [quarter#1347, loan_id#1349L, month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868, year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904]\n", - " : : : : : : +- GpuColumnarToRow false\n", - " : : : : : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND gpuisnotnull(loan_id#1349L)) AND true), true\n", - " : : : : : : +- GpuProject [loan_id#1349L, monthly_reporting_period#1350, current_actual_upb#1359, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1347]\n", - " : : : : : : +- GpuFileGpuScan csv [loan_id#1349L,monthly_reporting_period#1350,current_actual_upb#1359] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : : : : +- GpuBroadcastExchange HashedRelationBroadcastMode(List(input[1, bigint, true], input[0, string, false]),false), [id=#884]\n", - " : : : : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[gpumax(current_loan_delinquency_status#1059), gpumin(delinquency_30#792), gpumin(delinquency_90#793), gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : : : : +- GpuShuffleCoalesce 536870912\n", - " : : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1019, loan_id#1021L, 200), ENSURE_REQUIREMENTS, [id=#881]\n", - " : : : : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[partial_gpumax(current_loan_delinquency_status#1059), partial_gpumin(delinquency_30#792), partial_gpumin(delinquency_90#793), partial_gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : : : : +- GpuRowToColumnar targetsize(536870912)\n", - " : : : : : +- *(3) Project [quarter#1019, loan_id#1021L, current_loan_delinquency_status#1059, CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792, CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793, CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794]\n", - " : : : : : +- GpuColumnarToRow false\n", - " : : : : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND gpuisnotnull(loan_id#1021L)) AND true), true\n", - " : : : : : +- GpuProject [loan_id#1021L, monthly_reporting_period#1022, current_actual_upb#1031, current_loan_delinquency_status#1059, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1019]\n", - " : : : : : +- GpuFileGpuScan csv [loan_id#1021L,monthly_reporting_period#1022,current_actual_upb#1031,current_loan_delinquency_status#1059] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, timestamp_year#1201L, timestamp_month#1230L, 200), ENSURE_REQUIREMENTS, [id=#895]\n", - " : : : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : : : +- GpuShuffleCoalesce 536870912\n", - " : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142, 200), ENSURE_REQUIREMENTS, [id=#892]\n", - " : : : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : : : +- GpuProject [quarter#1347, FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, loan_id#1349L, month_y#1142]\n", - " : : : : +- GpuCoalesceBatches targetsize(536870912)\n", - " : : : : +- GpuFilter (gpuisnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND gpuisnotnull(CASE WHEN ((((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) = 0) THEN 12 ELSE (((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) END)), true\n", - " : : : : +- GpuGenerate gpuexplode([0,1,2,3,4,5,6,7,8,9,10,11]), [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814], false, [month_y#1142]\n", - " : : : : +- GpuProject [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814]\n", - " : : : : +- GpuBroadcastHashJoin [loan_id#1349L, quarter#1347], [loan_id#1021L, quarter#1019], LeftOuter, GpuBuildRight\n", - " : : : : :- GpuRowToColumnar targetsize(536870912)\n", - " : : : : : +- *(2) Project [quarter#1347, loan_id#1349L, month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868, year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904]\n", - " : : : : : +- GpuColumnarToRow false\n", - " : : : : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND gpuisnotnull(loan_id#1349L)) AND true), true\n", - " : : : : : +- GpuProject [loan_id#1349L, monthly_reporting_period#1350, current_actual_upb#1359, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1347]\n", - " : : : : : +- GpuFileGpuScan csv [loan_id#1349L,monthly_reporting_period#1350,current_actual_upb#1359] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : : : +- GpuBroadcastExchange HashedRelationBroadcastMode(List(input[1, bigint, true], input[0, string, false]),false), [id=#884]\n", - " : : : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[gpumax(current_loan_delinquency_status#1059), gpumin(delinquency_30#792), gpumin(delinquency_90#793), gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : : : +- GpuShuffleCoalesce 536870912\n", - " : : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1019, loan_id#1021L, 200), ENSURE_REQUIREMENTS, [id=#881]\n", - " : : : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[partial_gpumax(current_loan_delinquency_status#1059), partial_gpumin(delinquency_30#792), partial_gpumin(delinquency_90#793), partial_gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : : : +- GpuRowToColumnar targetsize(536870912)\n", - " : : : : +- *(3) Project [quarter#1019, loan_id#1021L, current_loan_delinquency_status#1059, CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792, CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793, CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794]\n", - " : : : : +- GpuColumnarToRow false\n", - " : : : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND gpuisnotnull(loan_id#1021L)) AND true), true\n", - " : : : : +- GpuProject [loan_id#1021L, monthly_reporting_period#1022, current_actual_upb#1031, current_loan_delinquency_status#1059, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1019]\n", - " : : : : +- GpuFileGpuScan csv [loan_id#1021L,monthly_reporting_period#1022,current_actual_upb#1031,current_loan_delinquency_status#1059] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, timestamp_year#1201L, timestamp_month#1230L, 200), ENSURE_REQUIREMENTS, [id=#895]\n", - " : : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : : +- GpuShuffleCoalesce 536870912\n", - " : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142, 200), ENSURE_REQUIREMENTS, [id=#892]\n", - " : : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : : +- GpuProject [quarter#1347, FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, loan_id#1349L, month_y#1142]\n", - " : : : +- GpuCoalesceBatches targetsize(536870912)\n", - " : : : +- GpuFilter (gpuisnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND gpuisnotnull(CASE WHEN ((((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) = 0) THEN 12 ELSE (((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) END)), true\n", - " : : : +- GpuGenerate gpuexplode([0,1,2,3,4,5,6,7,8,9,10,11]), [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814], false, [month_y#1142]\n", - " : : : +- GpuProject [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814]\n", - " : : : +- GpuBroadcastHashJoin [loan_id#1349L, quarter#1347], [loan_id#1021L, quarter#1019], LeftOuter, GpuBuildRight\n", - " : : : :- GpuRowToColumnar targetsize(536870912)\n", - " : : : : +- *(2) Project [quarter#1347, loan_id#1349L, month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868, year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904]\n", - " : : : : +- GpuColumnarToRow false\n", - " : : : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND gpuisnotnull(loan_id#1349L)) AND true), true\n", - " : : : : +- GpuProject [loan_id#1349L, monthly_reporting_period#1350, current_actual_upb#1359, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1347]\n", - " : : : : +- GpuFileGpuScan csv [loan_id#1349L,monthly_reporting_period#1350,current_actual_upb#1359] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : : +- GpuBroadcastExchange HashedRelationBroadcastMode(List(input[1, bigint, true], input[0, string, false]),false), [id=#884]\n", - " : : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[gpumax(current_loan_delinquency_status#1059), gpumin(delinquency_30#792), gpumin(delinquency_90#793), gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : : +- GpuShuffleCoalesce 536870912\n", - " : : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1019, loan_id#1021L, 200), ENSURE_REQUIREMENTS, [id=#881]\n", - " : : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[partial_gpumax(current_loan_delinquency_status#1059), partial_gpumin(delinquency_30#792), partial_gpumin(delinquency_90#793), partial_gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : : +- GpuRowToColumnar targetsize(536870912)\n", - " : : : +- *(3) Project [quarter#1019, loan_id#1021L, current_loan_delinquency_status#1059, CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792, CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793, CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794]\n", - " : : : +- GpuColumnarToRow false\n", - " : : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND gpuisnotnull(loan_id#1021L)) AND true), true\n", - " : : : +- GpuProject [loan_id#1021L, monthly_reporting_period#1022, current_actual_upb#1031, current_loan_delinquency_status#1059, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1019]\n", - " : : : +- GpuFileGpuScan csv [loan_id#1021L,monthly_reporting_period#1022,current_actual_upb#1031,current_loan_delinquency_status#1059] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, timestamp_year#1201L, timestamp_month#1230L, 200), ENSURE_REQUIREMENTS, [id=#895]\n", - " : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : +- GpuShuffleCoalesce 536870912\n", - " : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142, 200), ENSURE_REQUIREMENTS, [id=#892]\n", - " : : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : : +- GpuProject [quarter#1347, FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, loan_id#1349L, month_y#1142]\n", - " : : +- GpuCoalesceBatches targetsize(536870912)\n", - " : : +- GpuFilter (gpuisnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND gpuisnotnull(CASE WHEN ((((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) = 0) THEN 12 ELSE (((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) END)), true\n", - " : : +- GpuGenerate gpuexplode([0,1,2,3,4,5,6,7,8,9,10,11]), [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814], false, [month_y#1142]\n", - " : : +- GpuProject [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814]\n", - " : : +- GpuBroadcastHashJoin [loan_id#1349L, quarter#1347], [loan_id#1021L, quarter#1019], LeftOuter, GpuBuildRight\n", - " : : :- GpuRowToColumnar targetsize(536870912)\n", - " : : : +- *(2) Project [quarter#1347, loan_id#1349L, month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868, year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904]\n", - " : : : +- GpuColumnarToRow false\n", - " : : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND gpuisnotnull(loan_id#1349L)) AND true), true\n", - " : : : +- GpuProject [loan_id#1349L, monthly_reporting_period#1350, current_actual_upb#1359, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1347]\n", - " : : : +- GpuFileGpuScan csv [loan_id#1349L,monthly_reporting_period#1350,current_actual_upb#1359] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : : +- GpuBroadcastExchange HashedRelationBroadcastMode(List(input[1, bigint, true], input[0, string, false]),false), [id=#884]\n", - " : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[gpumax(current_loan_delinquency_status#1059), gpumin(delinquency_30#792), gpumin(delinquency_90#793), gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : +- GpuShuffleCoalesce 536870912\n", - " : : +- GpuColumnarExchange gpuhashpartitioning(quarter#1019, loan_id#1021L, 200), ENSURE_REQUIREMENTS, [id=#881]\n", - " : : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[partial_gpumax(current_loan_delinquency_status#1059), partial_gpumin(delinquency_30#792), partial_gpumin(delinquency_90#793), partial_gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : : +- GpuRowToColumnar targetsize(536870912)\n", - " : : +- *(3) Project [quarter#1019, loan_id#1021L, current_loan_delinquency_status#1059, CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792, CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793, CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794]\n", - " : : +- GpuColumnarToRow false\n", - " : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND gpuisnotnull(loan_id#1021L)) AND true), true\n", - " : : +- GpuProject [loan_id#1021L, monthly_reporting_period#1022, current_actual_upb#1031, current_loan_delinquency_status#1059, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1019]\n", - " : : +- GpuFileGpuScan csv [loan_id#1021L,monthly_reporting_period#1022,current_actual_upb#1031,current_loan_delinquency_status#1059] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, timestamp_year#1201L, timestamp_month#1230L, 200), ENSURE_REQUIREMENTS, [id=#895]\n", - " : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : +- GpuShuffleCoalesce 536870912\n", - " : +- GpuColumnarExchange gpuhashpartitioning(quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142, 200), ENSURE_REQUIREMENTS, [id=#892]\n", - " : +- GpuHashAggregate(keys=[quarter#1347, loan_id#1349L, josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, month_y#1142], functions=[]), filters=ArrayBuffer())\n", - " : +- GpuProject [quarter#1347, FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) AS josh_mody_n#1158L, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814, loan_id#1349L, month_y#1142]\n", - " : +- GpuCoalesceBatches targetsize(536870912)\n", - " : +- GpuFilter (gpuisnotnull(FLOOR((cast(((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast((month_y#1142 - 1) as bigint)) as double) / 12.0))) AND gpuisnotnull(CASE WHEN ((((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) = 0) THEN 12 ELSE (((24000 + (FLOOR((cast(((((timestamp_year#904 * 12) + timestamp_month#868) - 24000) - month_y#1142) as double) / 12.0)) * 12)) + cast(month_y#1142 as bigint)) pmod 12) END)), true\n", - " : +- GpuGenerate gpuexplode([0,1,2,3,4,5,6,7,8,9,10,11]), [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814], false, [month_y#1142]\n", - " : +- GpuProject [loan_id#1349L, quarter#1347, timestamp_month#868, timestamp_year#904, ever_30#821, ever_90#822, ever_180#823, delinquency_30#810, delinquency_90#812, delinquency_180#814]\n", - " : +- GpuBroadcastHashJoin [loan_id#1349L, quarter#1347], [loan_id#1021L, quarter#1019], LeftOuter, GpuBuildRight\n", - " : :- GpuRowToColumnar targetsize(536870912)\n", - " : : +- *(2) Project [quarter#1347, loan_id#1349L, month(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_month#868, year(cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1350, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date)) AS timestamp_year#904]\n", - " : : +- GpuColumnarToRow false\n", - " : : +- GpuFilter (((gpuisnotnull(current_actual_upb#1359) AND NOT (current_actual_upb#1359 = 0.0)) AND gpuisnotnull(loan_id#1349L)) AND true), true\n", - " : : +- GpuProject [loan_id#1349L, monthly_reporting_period#1350, current_actual_upb#1359, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1347]\n", - " : : +- GpuFileGpuScan csv [loan_id#1349L,monthly_reporting_period#1350,current_actual_upb#1359] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n", - " : +- GpuBroadcastExchange HashedRelationBroadcastMode(List(input[1, bigint, true], input[0, string, false]),false), [id=#884]\n", - " : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[gpumax(current_loan_delinquency_status#1059), gpumin(delinquency_30#792), gpumin(delinquency_90#793), gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : +- GpuShuffleCoalesce 536870912\n", - " : +- GpuColumnarExchange gpuhashpartitioning(quarter#1019, loan_id#1021L, 200), ENSURE_REQUIREMENTS, [id=#881]\n", - " : +- GpuHashAggregate(keys=[quarter#1019, loan_id#1021L], functions=[partial_gpumax(current_loan_delinquency_status#1059), partial_gpumin(delinquency_30#792), partial_gpumin(delinquency_90#793), partial_gpumin(delinquency_180#794)]), filters=ArrayBuffer(None, None, None, None))\n", - " : +- GpuRowToColumnar targetsize(536870912)\n", - " : +- *(3) Project [quarter#1019, loan_id#1021L, current_loan_delinquency_status#1059, CASE WHEN (current_loan_delinquency_status#1059 >= 1) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_30#792, CASE WHEN (current_loan_delinquency_status#1059 >= 3) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_90#793, CASE WHEN (current_loan_delinquency_status#1059 >= 6) THEN cast(gettimestamp(date_format(cast(cast(gettimestamp(monthly_reporting_period#1022, MMyyyy, Some(Etc/UTC), false) as date) as timestamp), MM/dd/yyyy, Some(Etc/UTC)), MM/dd/yyyy, Some(Etc/UTC), false) as date) END AS delinquency_180#794]\n", - " : +- GpuColumnarToRow false\n", - " : +- GpuFilter (((gpuisnotnull(current_actual_upb#1031) AND NOT (current_actual_upb#1031 = 0.0)) AND gpuisnotnull(loan_id#1021L)) AND true), true\n", - " : +- GpuProject [loan_id#1021L, monthly_reporting_period#1022, current_actual_upb#1031, current_loan_delinquency_status#1059, substring_index(substring_index(input_file_name(), \\A((?:[^.]|\r", - "|\n", - ")*), ., 1), ((?:[^/]|\r", - "|\n", - ")*)\\Z, /, -1) AS quarter#1019]\n", - " : +- GpuFileGpuScan csv [loan_id#1021L,monthly_reporting_period#1022,current_actual_upb#1031,current_loan_delinquency_status#1059] Batched: true, DataFilters: [], Format: CSV, Location: InMemoryFileIndex[gs://rapids-test/yuanli-tools-eventlog-temp/mortgage-fannieMae/2000Q1.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct-jar-with-dependencies.jar` under the -`gs://$GCS_BUCKET/scala/` folder by running -`gsutil cp aggregator/target/sample_xgboost_apps--jar-with-dependencies.jar gs://$GCS_BUCKET/scala/`. -To do this you can either drag and drop files from your local machine into the GCP storage browser, -or use the gsutil cp as shown before to do this from a command line. We can thereby submit the jar by: +Follow these +[instructions](https://github.com/NVIDIA/spark-rapids-examples/blob/branch-22.10/docs/get-started/xgboost-examples/building-sample-apps/scala.md) +to Build the +[xgboost-example](https://github.com/NVIDIA/spark-rapids-examples/blob/branch-22.10/docs/get-started/xgboost-examples) +jars. Upload the `sample_xgboost_apps-${VERSION}-SNAPSHOT-jar-with-dependencies.jar` to a GCS +bucket by dragging and dropping the jar file from your local machine into the GCS web console or by running: +``` +gsutil cp aggregator/target/sample_xgboost_apps-${VERSION}-SNAPSHOT-jar-with-dependencies.jar gs://${GCS_BUCKET}/scala/ +``` +Submit the Spark XGBoost application to dataproc using the following command: ```bash export REGION=[Your Preferred GCP Region] export GCS_BUCKET=[Your GCS Bucket] export CLUSTER_NAME=[Your Cluster Name] +export VERSION=[Your jar version] export SPARK_NUM_EXECUTORS=20 export SPARK_EXECUTOR_MEMORY=20G export SPARK_EXECUTOR_MEMORYOVERHEAD=16G @@ -211,11 +215,11 @@ gcloud dataproc jobs submit spark \ --cluster=$CLUSTER_NAME \ --region=$REGION \ --class=com.nvidia.spark.examples.mortgage.GPUMain \ - --jars=gs://${GCS_BUCKET}/scala/sample_xgboost_apps--jar-with-dependencies.jar \ - --properties=spark.executor.cores=${SPARK_NUM_CORES_PER_EXECUTOR},spark.task.cpus=${SPARK_NUM_CORES_PER_EXECUTOR},spark.executor.memory=${SPARK_EXECUTOR_MEMORY},spark.executor.memoryOverhead=${SPARK_EXECUTOR_MEMORYOVERHEAD},spark.executor.resource.gpu.amount=1,spark.task.resource.gpu.amount=1,spark.rapids.sql.hasNans=false,spark.rapids.sql.batchSizeBytes=512M,spark.rapids.sql.reader.batchSizeBytes=768M,spark.rapids.sql.variableFloatAgg.enabled=true,spark.rapids.memory.gpu.pooling.enabled=false \ + --jars=gs://${GCS_BUCKET}/scala/sample_xgboost_apps-${VERSION}-SNAPSHOT-jar-with-dependencies.jar \ + --properties=spark.executor.cores=${SPARK_NUM_CORES_PER_EXECUTOR},spark.task.cpus=${SPARK_NUM_CORES_PER_EXECUTOR},spark.executor.memory=${SPARK_EXECUTOR_MEMORY},spark.executor.memoryOverhead=${SPARK_EXECUTOR_MEMORYOVERHEAD},spark.executor.resource.gpu.amount=1,spark.task.resource.gpu.amount=1,spark.rapids.sql.hasNans=false,spark.rapids.sql.batchSizeBytes=512M,spark.rapids.sql.reader.batchSizeBytes=768M,spark.rapids.sql.variableFloatAgg.enabled=true,spark.rapids.memory.gpu.pooling.enabled=false,spark.dynamicAllocation.enabled=false \ -- \ -dataPath=train::${DATA_PATH}/train \ - -dataPath=trans::${DATA_PATH}/test \ + -dataPath=trans::${DATA_PATH}/eval \ -format=parquet \ -numWorkers=${SPARK_NUM_EXECUTORS} \ -treeMethod=gpu_hist \ @@ -228,32 +232,32 @@ With the integration between AI Platform Notebooks and Dataproc, users can creat notebook](https://cloud.google.com/blog/products/data-analytics/administering-jupyter-notebooks-for-spark-workloads-on-dataproc). The AI platform will connect to a Dataproc cluster through a yaml configuration. -In the future, users will be able to provision a Dataproc cluster through DataprocHub notebook. You +In the future, users will be able to provision a Dataproc cluster through DataprocHub notebook. You can use example [pyspark notebooks](../demo/GCP/Mortgage-ETL.ipynb) to experiment. ## Build custom dataproc image to accelerate cluster init time In order to accelerate cluster init time to 3-4 minutes, we need to build a custom Dataproc image -that already has NVIDIA drivers and CUDA toolkit installed, with RAPIDS deployed. The custom image -could also be used in an air gap environment. In this section, we will be using [these -instructions from GCP](https://cloud.google.com/dataproc/docs/guides/dataproc-images) to create a -custom image. +that already has NVIDIA drivers and CUDA toolkit installed, with RAPIDS deployed. The custom image +could also be used in an air gap environment. In this section, we will be using [these instructions +from GCP](https://cloud.google.com/dataproc/docs/guides/dataproc-images) to create a custom image. -Currently, the [GPU Driver](https://github.com/GoogleCloudDataproc/initialization-actions/tree/master/gpu) +Currently, the [GPU +Driver](https://github.com/GoogleCloudDataproc/initialization-actions/tree/master/gpu) initialization actions: 1. Configure YARN, the YARN node manager, GPU isolation and GPU exclusive mode. 2. Install GPU drivers. -Let's write a script to move as many of those to custom image. -[gpu_dataproc_packages_ubuntu_sample.sh](gpu_dataproc_packages_ubuntu_sample.sh) -in this directory will be used to create the Dataproc image: +Let's write a script to move as many of those to custom image. +[gpu_dataproc_packages_ubuntu_sample.sh](gpu_dataproc_packages_ubuntu_sample.sh) in this directory +will be used to create the Dataproc image: Google provides a `generate_custom_image.py` script that: - Launches a temporary Compute Engine VM instance with the specified Dataproc base image. -- Then runs the customization script inside the VM instance to install custom packages and/or update - configurations. +- Then runs the customization script inside the VM instance to install custom packages and/or +update configurations. - After the customization script finishes, it shuts down the VM instance and creates a Dataproc custom image from the disk of the VM instance. -- The temporary VM is deleted after the custom image is created. +- The temporary VM is deleted after the custom image is created. - The custom image is saved and can be used to create Dataproc clusters. Download `gpu_dataproc_packages_ubuntu_sample.sh` in this repo. The script uses @@ -285,12 +289,14 @@ python generate_custom_image.py \ ``` See [here](https://cloud.google.com/dataproc/docs/guides/dataproc-images#running_the_code) for more -details on `generate_custom_image.py` script arguments and -[here](https://cloud.google.com/dataproc/docs/concepts/versioning/dataproc-versions) for dataproc version description. +details on `generate_custom_image.py` script arguments and +[here](https://cloud.google.com/dataproc/docs/concepts/versioning/dataproc-versions) for dataproc +version description. The image `sample-20-ubuntu18-gpu-t4` is now ready and can be viewed in the GCP console under -`Compute Engine > Storage > Images`. The next step is to launch the cluster using this new image and -new initialization actions (that do not install NVIDIA drivers since we are already past that step). +`Compute Engine > Storage > Images`. The next step is to launch the cluster using this new image +and new initialization actions (that do not install NVIDIA drivers since we are already past that +step). Move this to your own bucket. Let's launch the cluster: From d09bc9a7df31872d253af278388b3469ca772a05 Mon Sep 17 00:00:00 2001 From: Matt Ahrens Date: Fri, 9 Sep 2022 13:57:08 -0500 Subject: [PATCH 097/190] Updating qual tool speedup factors from latest CSP benchmarks (#6538) * Updating qual tool speedup factors from latest CSP benchmarks Signed-off-by: mattahrens * Updating qual tool speedup factors in TypeChecks from latest CSP benchmarks Signed-off-by: mattahrens Signed-off-by: mattahrens --- .../com/nvidia/spark/rapids/TypeChecks.scala | 12 ++++++------ .../com/nvidia/spark/rapids/TypeChecks.scala | 12 ++++++------ tools/src/main/resources/operatorsScore.csv | 12 ++++++------ .../db_sim_test_expectation.csv | 2 +- .../directory_test_expectation.csv | 2 +- .../jdbc_expectation.csv | 2 +- .../nds_q86_fail_test_expectation.csv | 2 +- .../nds_q86_fail_test_expectation_persql.csv | 2 +- .../nds_q86_test_expectation.csv | 2 +- .../nds_q86_test_expectation_persql.csv | 2 +- .../qual_test_missing_sql_end_expectation.csv | 2 +- .../qual_test_simple_expectation.csv | 4 ++-- .../qual_test_simple_expectation_persql.csv | 12 ++++++------ .../read_dsv1_expectation.csv | 2 +- .../read_dsv2_expectation.csv | 2 +- .../spark2_expectation.csv | 2 +- .../truncated_1_end_expectation.csv | 2 +- .../dev/generate-qualification-test-results.sh | 2 +- .../tool/planparser/SqlPlanParserSuite.scala | 16 ++++++++-------- .../qualification/PluginTypeCheckerSuite.scala | 2 +- 20 files changed, 48 insertions(+), 48 deletions(-) diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index 7c68a3d5f3d..f2943ecfc39 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -2251,12 +2251,12 @@ object SupportedOpsForTools { val header = Seq("CPUOperator", "Score") println(header.mkString(",")) val operatorCustomSpeedUp = Map( - ("BroadcastHashJoinExec", "3.0"), - ("ShuffleExchangeExec", "3.1"), - ("FilterExec", "2.4"), - ("HashAggregateExec", "3.4"), - ("SortExec", "5.2"), - ("SortMergeJoinExec", "14.1"), + ("BroadcastHashJoinExec", "5.1"), + ("ShuffleExchangeExec", "4.2"), + ("FilterExec", "2.8"), + ("HashAggregateExec", "4.5"), + ("SortExec", "8.0"), + ("SortMergeJoinExec", "22.7"), ("ArrowEvalPythonExec", "1.2"), ("AggregateInPandasExec", "1.2"), ("FlatMapGroupsInPandasExec", "1.2"), diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index abf9ed64f6d..512cba00d11 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -2265,12 +2265,12 @@ object SupportedOpsForTools { val header = Seq("CPUOperator", "Score") println(header.mkString(",")) val operatorCustomSpeedUp = Map( - ("BroadcastHashJoinExec", "3.0"), - ("ShuffleExchangeExec", "3.1"), - ("FilterExec", "2.4"), - ("HashAggregateExec", "3.4"), - ("SortExec", "5.2"), - ("SortMergeJoinExec", "14.1"), + ("BroadcastHashJoinExec", "5.1"), + ("ShuffleExchangeExec", "4.2"), + ("FilterExec", "2.8"), + ("HashAggregateExec", "4.5"), + ("SortExec", "8.0"), + ("SortMergeJoinExec", "22.7"), ("ArrowEvalPythonExec", "1.2"), ("AggregateInPandasExec", "1.2"), ("FlatMapGroupsInPandasExec", "1.2"), diff --git a/tools/src/main/resources/operatorsScore.csv b/tools/src/main/resources/operatorsScore.csv index f67f0f396c4..edb1599bc0b 100644 --- a/tools/src/main/resources/operatorsScore.csv +++ b/tools/src/main/resources/operatorsScore.csv @@ -3,31 +3,31 @@ CoalesceExec,3.0 CollectLimitExec,3.0 ExpandExec,3.0 FileSourceScanExec,3.0 -FilterExec,2.4 +FilterExec,2.8 GenerateExec,3.0 GlobalLimitExec,3.0 LocalLimitExec,3.0 ProjectExec,3.0 RangeExec,3.0 SampleExec,3.0 -SortExec,5.2 +SortExec,8.0 SubqueryBroadcastExec,3.0 TakeOrderedAndProjectExec,3.0 UnionExec,3.0 CustomShuffleReaderExec,3.0 -HashAggregateExec,3.4 +HashAggregateExec,4.5 ObjectHashAggregateExec,3.0 SortAggregateExec,3.0 InMemoryTableScanExec,3.0 DataWritingCommandExec,3.0 BatchScanExec,3.0 BroadcastExchangeExec,3.0 -ShuffleExchangeExec,3.1 -BroadcastHashJoinExec,3.0 +ShuffleExchangeExec,4.2 +BroadcastHashJoinExec,5.1 BroadcastNestedLoopJoinExec,3.0 CartesianProductExec,3.0 ShuffledHashJoinExec,3.0 -SortMergeJoinExec,14.1 +SortMergeJoinExec,22.7 AggregateInPandasExec,1.2 ArrowEvalPythonExec,1.2 FlatMapGroupsInPandasExec,1.2 diff --git a/tools/src/test/resources/QualificationExpectations/db_sim_test_expectation.csv b/tools/src/test/resources/QualificationExpectations/db_sim_test_expectation.csv index 2dcc0ec8720..ec7b3fb2597 100644 --- a/tools/src/test/resources/QualificationExpectations/db_sim_test_expectation.csv +++ b/tools/src/test/resources/QualificationExpectations/db_sim_test_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,Recommendation,Estimated GPU Speedup,Estimated GPU Duration,Estimated GPU Time Saved,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,Task Speedup Factor,App Duration Estimated -Spark shell,local-1623876083964,Recommended,1.84,72535.16,61321.83,119903,1417661,133857,93094,91.14,"","","","","","",119903,9399,316964,1100697,2.93,false +Spark shell,local-1623876083964,Recommended,2.07,64632.65,69224.34,119903,1417661,133857,93094,91.14,"","","","","","",119903,9399,316964,1100697,3.9,false diff --git a/tools/src/test/resources/QualificationExpectations/directory_test_expectation.csv b/tools/src/test/resources/QualificationExpectations/directory_test_expectation.csv index 2dcc0ec8720..ec7b3fb2597 100644 --- a/tools/src/test/resources/QualificationExpectations/directory_test_expectation.csv +++ b/tools/src/test/resources/QualificationExpectations/directory_test_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,Recommendation,Estimated GPU Speedup,Estimated GPU Duration,Estimated GPU Time Saved,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,Task Speedup Factor,App Duration Estimated -Spark shell,local-1623876083964,Recommended,1.84,72535.16,61321.83,119903,1417661,133857,93094,91.14,"","","","","","",119903,9399,316964,1100697,2.93,false +Spark shell,local-1623876083964,Recommended,2.07,64632.65,69224.34,119903,1417661,133857,93094,91.14,"","","","","","",119903,9399,316964,1100697,3.9,false diff --git a/tools/src/test/resources/QualificationExpectations/jdbc_expectation.csv b/tools/src/test/resources/QualificationExpectations/jdbc_expectation.csv index 5f068f728fa..4dee2701e65 100644 --- a/tools/src/test/resources/QualificationExpectations/jdbc_expectation.csv +++ b/tools/src/test/resources/QualificationExpectations/jdbc_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,Recommendation,Estimated GPU Speedup,Estimated GPU Duration,Estimated GPU Time Saved,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,Task Speedup Factor,App Duration Estimated -Spark shell,app-20211019113801-0001,Not Recommended,1.0,569723.7,2243.29,3627,19894,571967,3503,28.41,"",JDBC[*],"","","","",1812,544575,677,19217,2.78,false +Spark shell,app-20211019113801-0001,Not Recommended,1.0,569455.94,2511.05,3627,19894,571967,3503,28.41,"",JDBC[*],"","","","",1812,544575,677,19217,3.53,false diff --git a/tools/src/test/resources/QualificationExpectations/nds_q86_fail_test_expectation.csv b/tools/src/test/resources/QualificationExpectations/nds_q86_fail_test_expectation.csv index f208fd49381..f5b4a315e0b 100644 --- a/tools/src/test/resources/QualificationExpectations/nds_q86_fail_test_expectation.csv +++ b/tools/src/test/resources/QualificationExpectations/nds_q86_fail_test_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,Recommendation,Estimated GPU Speedup,Estimated GPU Duration,Estimated GPU Time Saved,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,Task Speedup Factor,App Duration Estimated -TPC-DS Like Bench q86,app-20210319163812-1778,Not Applicable,1.32,19708.81,6462.18,9569,4320658,26171,9569,0.0,24,"","","","","",9565,3595714,0,4320658,3.08,false +TPC-DS Like Bench q86,app-20210319163812-1778,Not Applicable,1.35,19289.92,6881.07,9569,4320658,26171,9569,0.0,24,"","","","","",9565,3595714,0,4320658,3.56,false diff --git a/tools/src/test/resources/QualificationExpectations/nds_q86_fail_test_expectation_persql.csv b/tools/src/test/resources/QualificationExpectations/nds_q86_fail_test_expectation_persql.csv index 3b145f3bbd3..581d20e266b 100644 --- a/tools/src/test/resources/QualificationExpectations/nds_q86_fail_test_expectation_persql.csv +++ b/tools/src/test/resources/QualificationExpectations/nds_q86_fail_test_expectation_persql.csv @@ -22,5 +22,5 @@ TPC-DS Like Bench q86,app-20210319163812-1778,3,Register input tables,0,0,0.0,0. TPC-DS Like Bench q86,app-20210319163812-1778,21,Register input tables,1,1,1.0,1.0,0.0,Not Recommended TPC-DS Like Bench q86,app-20210319163812-1778,15,Register input tables,0,0,0.0,0.0,0.0,Not Recommended TPC-DS Like Bench q86,app-20210319163812-1778,6,Register input tables,0,0,0.0,0.0,0.0,Not Recommended -TPC-DS Like Bench q86,app-20210319163812-1778,24,Benchmark Run: query=q86; iteration=0,9565,9565,3105.51,3.08,6459.48,Not Applicable +TPC-DS Like Bench q86,app-20210319163812-1778,24,Benchmark Run: query=q86; iteration=0,9565,9565,2686.79,3.56,6878.2,Not Applicable TPC-DS Like Bench q86,app-20210319163812-1778,0,Register input tables,2,2,2.0,1.0,0.0,Not Recommended diff --git a/tools/src/test/resources/QualificationExpectations/nds_q86_test_expectation.csv b/tools/src/test/resources/QualificationExpectations/nds_q86_test_expectation.csv index 36587fdc8c0..2157202b602 100644 --- a/tools/src/test/resources/QualificationExpectations/nds_q86_test_expectation.csv +++ b/tools/src/test/resources/QualificationExpectations/nds_q86_test_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,Recommendation,Estimated GPU Speedup,Estimated GPU Duration,Estimated GPU Time Saved,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,Task Speedup Factor,App Duration Estimated -TPC-DS Like Bench q86,app-20210319163812-1778,Recommended,1.32,19708.81,6462.18,9569,4320658,26171,9569,35.34,"","","","","","",9565,3595714,0,4320658,3.08,false +TPC-DS Like Bench q86,app-20210319163812-1778,Recommended,1.35,19289.92,6881.07,9569,4320658,26171,9569,35.34,"","","","","","",9565,3595714,0,4320658,3.56,false diff --git a/tools/src/test/resources/QualificationExpectations/nds_q86_test_expectation_persql.csv b/tools/src/test/resources/QualificationExpectations/nds_q86_test_expectation_persql.csv index e320a93b6e6..b40d7003fcb 100644 --- a/tools/src/test/resources/QualificationExpectations/nds_q86_test_expectation_persql.csv +++ b/tools/src/test/resources/QualificationExpectations/nds_q86_test_expectation_persql.csv @@ -22,5 +22,5 @@ TPC-DS Like Bench q86,app-20210319163812-1778,3,Register input tables,0,0,0.0,0. TPC-DS Like Bench q86,app-20210319163812-1778,21,Register input tables,1,1,1.0,1.0,0.0,Not Recommended TPC-DS Like Bench q86,app-20210319163812-1778,15,Register input tables,0,0,0.0,0.0,0.0,Not Recommended TPC-DS Like Bench q86,app-20210319163812-1778,6,Register input tables,0,0,0.0,0.0,0.0,Not Recommended -TPC-DS Like Bench q86,app-20210319163812-1778,24,Benchmark Run: query=q86; iteration=0,9565,9565,3105.51,3.08,6459.48,Strongly Recommended +TPC-DS Like Bench q86,app-20210319163812-1778,24,Benchmark Run: query=q86; iteration=0,9565,9565,2686.79,3.56,6878.2,Strongly Recommended TPC-DS Like Bench q86,app-20210319163812-1778,0,Register input tables,2,2,2.0,1.0,0.0,Not Recommended diff --git a/tools/src/test/resources/QualificationExpectations/qual_test_missing_sql_end_expectation.csv b/tools/src/test/resources/QualificationExpectations/qual_test_missing_sql_end_expectation.csv index 7e4d6530ca4..ac451550bd8 100644 --- a/tools/src/test/resources/QualificationExpectations/qual_test_missing_sql_end_expectation.csv +++ b/tools/src/test/resources/QualificationExpectations/qual_test_missing_sql_end_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,Recommendation,Estimated GPU Speedup,Estimated GPU Duration,Estimated GPU Time Saved,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,Task Speedup Factor,App Duration Estimated -Rapids Spark Profiling Tool Unit Tests,local-1622561780883,Not Recommended,1.0,7673.0,0.0,0,40448,7673,0,55.94,"","","","","","",0,4003,8096,32352,2.93,false +Rapids Spark Profiling Tool Unit Tests,local-1622561780883,Not Recommended,1.0,7673.0,0.0,0,40448,7673,0,55.94,"","","","","","",0,4003,8096,32352,3.9,false diff --git a/tools/src/test/resources/QualificationExpectations/qual_test_simple_expectation.csv b/tools/src/test/resources/QualificationExpectations/qual_test_simple_expectation.csv index e1765ba2b21..21df18f6c7e 100644 --- a/tools/src/test/resources/QualificationExpectations/qual_test_simple_expectation.csv +++ b/tools/src/test/resources/QualificationExpectations/qual_test_simple_expectation.csv @@ -1,5 +1,5 @@ App Name,App ID,Recommendation,Estimated GPU Speedup,Estimated GPU Duration,Estimated GPU Time Saved,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,Task Speedup Factor,App Duration Estimated -Rapids Spark Profiling Tool Unit Tests,local-1622043423018,Recommended,1.71,9497.79,6821.2,12434,132257,16319,10589,37.7,"","",JSON,"","","",7143,4717,19616,112641,2.81,false -Spark shell,local-1651187225439,Not Recommended,1.0,355496.4,140.59,760,180,355637,350,87.88,"",JSON[string:bigint:int],"","","","",498,343411,97,83,1.67,false +Rapids Spark Profiling Tool Unit Tests,local-1622043423018,Recommended,1.89,8591.29,7727.7,12434,132257,16319,10589,37.7,"","",JSON,"","","",7143,4717,19616,112641,3.7,false +Spark shell,local-1651187225439,Not Recommended,1.0,355493.91,143.08,760,180,355637,350,87.88,"",JSON[string:bigint:int],"","","","",498,343411,97,83,1.69,false Spark shell,local-1651188809790,Not Recommended,1.0,166192.46,22.53,911,283,166215,45,81.18,"",JSON[string:bigint:int],"","","",UDF,715,133608,269,14,2.0,false Rapids Spark Profiling Tool Unit Tests,local-1623281204390,Not Recommended,1.0,6240.0,0.0,2032,4666,6240,0,46.27,"",JSON[string:bigint:int],JSON,"","",UDF,1209,5793,4664,2,1.0,false diff --git a/tools/src/test/resources/QualificationExpectations/qual_test_simple_expectation_persql.csv b/tools/src/test/resources/QualificationExpectations/qual_test_simple_expectation_persql.csv index 2c728e8a6bf..80c5eae969b 100644 --- a/tools/src/test/resources/QualificationExpectations/qual_test_simple_expectation_persql.csv +++ b/tools/src/test/resources/QualificationExpectations/qual_test_simple_expectation_persql.csv @@ -1,8 +1,8 @@ App Name,App ID,SQL ID,SQL Description,SQL DF Duration,GPU Opportunity,Estimated GPU Duration,Estimated GPU Speedup,Estimated GPU Time Saved,Recommendation +Spark shell,local-1651187225439,1,show at :26,262,60,245.35,1.06,16.64,Not Recommended +Spark shell,local-1651187225439,0,show at :26,498,249,373.5,1.33,124.5,Recommended Spark shell,local-1651188809790,1,show at :26,196,98,147.0,1.33,49.0,Recommended Spark shell,local-1651188809790,0,show at :26,715,2,713.6,1.0,1.39,Not Recommended -Spark shell,local-1651187225439,1,show at :26,262,60,246.65,1.06,15.34,Not Recommended -Spark shell,local-1651187225439,0,show at :26,498,249,373.5,1.33,124.5,Recommended Rapids Spark Profiling Tool Unit Tests,local-1623281204390,8,json at QualificationInfoUtils.scala:136,127,0,127.0,1.0,0.0,Not Recommended Rapids Spark Profiling Tool Unit Tests,local-1623281204390,2,json at QualificationInfoUtils.scala:136,321,0,321.0,1.0,0.0,Not Recommended Rapids Spark Profiling Tool Unit Tests,local-1623281204390,5,json at QualificationInfoUtils.scala:136,129,0,129.0,1.0,0.0,Not Recommended @@ -12,7 +12,7 @@ Rapids Spark Profiling Tool Unit Tests,local-1623281204390,1,createOrReplaceTemp Rapids Spark Profiling Tool Unit Tests,local-1623281204390,3,json at QualificationInfoUtils.scala:130,108,0,108.0,1.0,0.0,Not Recommended Rapids Spark Profiling Tool Unit Tests,local-1623281204390,6,json at QualificationInfoUtils.scala:130,110,0,110.0,1.0,0.0,Not Recommended Rapids Spark Profiling Tool Unit Tests,local-1623281204390,0,json at QualificationInfoUtils.scala:130,1209,0,1209.0,1.0,0.0,Not Recommended -Rapids Spark Profiling Tool Unit Tests,local-1622043423018,2,count at QualificationInfoUtils.scala:94,1933,1551,911.3,2.12,1021.69,Recommended -Rapids Spark Profiling Tool Unit Tests,local-1622043423018,1,count at QualificationInfoUtils.scala:94,7143,6719,2716.79,2.62,4426.2,Strongly Recommended -Rapids Spark Profiling Tool Unit Tests,local-1622043423018,3,count at QualificationInfoUtils.scala:94,2052,1660,958.28,2.14,1093.71,Recommended -Rapids Spark Profiling Tool Unit Tests,local-1622043423018,0,json at QualificationInfoUtils.scala:76,1306,187,1263.88,1.03,42.11,Not Recommended +Rapids Spark Profiling Tool Unit Tests,local-1622043423018,2,count at QualificationInfoUtils.scala:94,1933,1551,779.63,2.47,1153.36,Recommended +Rapids Spark Profiling Tool Unit Tests,local-1622043423018,1,count at QualificationInfoUtils.scala:94,7143,6719,2146.39,3.32,4996.6,Strongly Recommended +Rapids Spark Profiling Tool Unit Tests,local-1622043423018,3,count at QualificationInfoUtils.scala:94,2052,1660,817.34,2.51,1234.65,Strongly Recommended +Rapids Spark Profiling Tool Unit Tests,local-1622043423018,0,json at QualificationInfoUtils.scala:76,1306,187,1260.58,1.03,45.41,Not Recommended diff --git a/tools/src/test/resources/QualificationExpectations/read_dsv1_expectation.csv b/tools/src/test/resources/QualificationExpectations/read_dsv1_expectation.csv index 3f8dffef027..50cbb3e9efc 100644 --- a/tools/src/test/resources/QualificationExpectations/read_dsv1_expectation.csv +++ b/tools/src/test/resources/QualificationExpectations/read_dsv1_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,Recommendation,Estimated GPU Speedup,Estimated GPU Duration,Estimated GPU Time Saved,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,Task Speedup Factor,App Duration Estimated -Spark shell,local-1624371544219,Not Recommended,1.0,174720.4,572.59,6695,20421,175293,1034,72.15,"",JSON[string:double:date:int:bigint];Text[*],JSON,"","","",1859,175857,17266,3155,2.24,false +Spark shell,local-1624371544219,Not Recommended,1.0,174714.3,578.69,6695,20421,175293,1034,72.15,"",JSON[string:double:date:int:bigint];Text[*],JSON,"","","",1859,175857,17266,3155,2.27,false diff --git a/tools/src/test/resources/QualificationExpectations/read_dsv2_expectation.csv b/tools/src/test/resources/QualificationExpectations/read_dsv2_expectation.csv index a3a6204b18b..385128e69de 100644 --- a/tools/src/test/resources/QualificationExpectations/read_dsv2_expectation.csv +++ b/tools/src/test/resources/QualificationExpectations/read_dsv2_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,Recommendation,Estimated GPU Speedup,Estimated GPU Duration,Estimated GPU Time Saved,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,Task Speedup Factor,App Duration Estimated -Spark shell,local-1624371906627,Not Recommended,1.0,83200.07,537.92,6760,21802,83738,971,71.3,"",Text[*];json[double],JSON,"","","",1984,82505,18668,3134,2.24,false +Spark shell,local-1624371906627,Not Recommended,1.0,83194.34,543.65,6760,21802,83738,971,71.3,"",Text[*];json[double],JSON,"","","",1984,82505,18668,3134,2.27,false diff --git a/tools/src/test/resources/QualificationExpectations/spark2_expectation.csv b/tools/src/test/resources/QualificationExpectations/spark2_expectation.csv index 45f85588060..8936dc2ecb9 100644 --- a/tools/src/test/resources/QualificationExpectations/spark2_expectation.csv +++ b/tools/src/test/resources/QualificationExpectations/spark2_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,Recommendation,Estimated GPU Speedup,Estimated GPU Duration,Estimated GPU Time Saved,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,Task Speedup Factor,App Duration Estimated -Spark shell,local-1634253215009,Not Recommended,1.01,46464.1,598.89,1520,359,47063,1011,67.64,"",Text[*],"","","","",1068,44935,120,239,2.45,false +Spark shell,local-1634253215009,Not Recommended,1.01,46416.39,646.6,1520,359,47063,1011,67.64,"",Text[*],"","","","",1068,44935,120,239,2.77,false diff --git a/tools/src/test/resources/QualificationExpectations/truncated_1_end_expectation.csv b/tools/src/test/resources/QualificationExpectations/truncated_1_end_expectation.csv index 8eb4a33ab2a..c86671da8fc 100644 --- a/tools/src/test/resources/QualificationExpectations/truncated_1_end_expectation.csv +++ b/tools/src/test/resources/QualificationExpectations/truncated_1_end_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,Recommendation,Estimated GPU Speedup,Estimated GPU Duration,Estimated GPU Time Saved,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,Task Speedup Factor,App Duration Estimated -Rapids Spark Profiling Tool Unit Tests,local-1622043423018,Not Recommended,1.07,4520.24,351.75,1306,14353,4872,570,62.67,"","",JSON,"","","",1306,4477,8086,6267,2.61,true +Rapids Spark Profiling Tool Unit Tests,local-1622043423018,Not Recommended,1.09,4468.49,403.5,1306,14353,4872,570,62.67,"","",JSON,"","","",1306,4477,8086,6267,3.42,true diff --git a/tools/src/test/resources/dev/generate-qualification-test-results.sh b/tools/src/test/resources/dev/generate-qualification-test-results.sh index 0bde83721d2..640e6e74d1e 100755 --- a/tools/src/test/resources/dev/generate-qualification-test-results.sh +++ b/tools/src/test/resources/dev/generate-qualification-test-results.sh @@ -174,7 +174,7 @@ print_banner() set_rapids_jars_from_work_dir() { - rapids_tools_jar_file=( "$( find "${MODULE_PATH}" -type f \( -iname "rapids-4-spark-tools_*.jar" ! -iname "*tests.jar" ! -iname "original-rapids-4*.jar" \) )" ) + rapids_tools_jar_file=( "$( find "${MODULE_PATH}" -type f \( -iname "rapids-4-spark-tools_*.jar" ! -iname "*tests.jar" ! -iname "original-rapids-4*.jar" ! -iname "*javadoc.jar" \) )" ) # get the parent directory rapids_jar_home="$(dirname "${rapids_tools_jar_file}")" } diff --git a/tools/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala b/tools/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala index 7ff34192889..6a79ff10584 100644 --- a/tools/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala +++ b/tools/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala @@ -122,13 +122,13 @@ class SQLPlanParserSuite extends FunSuite with BeforeAndAfterEach with Logging { val allChildren = wholeStages.flatMap(_.children).flatten assert(allChildren.size == 10) val filters = allChildren.filter(_.exec == "Filter") - assertSizeAndSupported(2, filters, 2.4) + assertSizeAndSupported(2, filters, 2.8) val projects = allChildren.filter(_.exec == "Project") assertSizeAndSupported(2, projects) val sorts = allChildren.filter(_.exec == "Sort") - assertSizeAndSupported(3, sorts, 5.2) + assertSizeAndSupported(3, sorts, 8.0) val smj = allChildren.filter(_.exec == "SortMergeJoin") - assertSizeAndSupported(1, smj, 14.1) + assertSizeAndSupported(1, smj, 22.7) } } } @@ -154,7 +154,7 @@ class SQLPlanParserSuite extends FunSuite with BeforeAndAfterEach with Logging { assert(wholeStages.forall(_.duration.nonEmpty)) val allChildren = wholeStages.flatMap(_.children).flatten val hashAggregate = allChildren.filter(_.exec == "HashAggregate") - assertSizeAndSupported(2, hashAggregate, 3.4) + assertSizeAndSupported(2, hashAggregate, 4.5) } } } @@ -264,7 +264,7 @@ class SQLPlanParserSuite extends FunSuite with BeforeAndAfterEach with Logging { assert(stats.nonEmpty) val estimatedGpuSpeed = stats.get.estimatedInfo.estimatedGpuSpeedup val recommendation = stats.get.estimatedInfo.recommendation - assert (ToolUtils.truncateDoubleToTwoDecimal(estimatedGpuSpeed) == 1.11) + assert (ToolUtils.truncateDoubleToTwoDecimal(estimatedGpuSpeed) == 1.13) assert(recommendation.equals("Not Applicable")) } @@ -304,7 +304,7 @@ class SQLPlanParserSuite extends FunSuite with BeforeAndAfterEach with Logging { val subqueryBroadcast = allExecInfo.filter(_.exec == "SubqueryBroadcast") assertSizeAndSupported(1, subqueryBroadcast.toSeq, expectedDur = Seq(Some(1175))) val exchanges = allExecInfo.filter(_.exec == "Exchange") - assertSizeAndSupported(2, exchanges.toSeq, 3.1, expectedDur = Seq(Some(15688), Some(8))) + assertSizeAndSupported(2, exchanges.toSeq, 4.2, expectedDur = Seq(Some(15688), Some(8))) } test("CustomShuffleReaderExec") { @@ -422,7 +422,7 @@ class SQLPlanParserSuite extends FunSuite with BeforeAndAfterEach with Logging { } val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val bhj = allExecInfo.filter(_.exec == "BroadcastHashJoin") - assertSizeAndSupported(1, bhj, 3.0) + assertSizeAndSupported(1, bhj, 5.1) val broadcastNestedJoin = allExecInfo.filter(_.exec == "BroadcastNestedLoopJoin") assertSizeAndSupported(1, broadcastNestedJoin) val shj = allExecInfo.filter(_.exec == "ShuffledHashJoin") @@ -607,7 +607,7 @@ class SQLPlanParserSuite extends FunSuite with BeforeAndAfterEach with Logging { val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val sortExec = allExecInfo.filter(_.exec.contains("Sort")) assert(sortExec.size == 3) - assertSizeAndSupported(3, sortExec, 5.2) + assertSizeAndSupported(3, sortExec, 8.0) } } } diff --git a/tools/src/test/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeCheckerSuite.scala b/tools/src/test/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeCheckerSuite.scala index 9cdb15d9edb..1430e4bf981 100644 --- a/tools/src/test/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeCheckerSuite.scala +++ b/tools/src/test/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeCheckerSuite.scala @@ -113,7 +113,7 @@ class PluginTypeCheckerSuite extends FunSuite with Logging { test("supported operator score from default file") { val checker = new PluginTypeChecker - assert(checker.getSpeedupFactor("FilterExec") == 2.4) + assert(checker.getSpeedupFactor("FilterExec") == 2.8) assert(checker.getSpeedupFactor("Ceil") == 4) } From ebee3b413bd45d503ce6994ca73bb2a3e5b523c3 Mon Sep 17 00:00:00 2001 From: Mike Wilson Date: Fri, 9 Sep 2022 15:45:48 -0400 Subject: [PATCH 098/190] Switch to gpu string to integer casts (#6436) * switch to custom kernel for gpu integer casts Signed-off-by: Mike Wilson --- .../com/nvidia/spark/rapids/GpuCast.scala | 95 +------------------ .../nvidia/spark/rapids/AnsiCastOpSuite.scala | 12 ++- .../com/nvidia/spark/rapids/CastOpSuite.scala | 26 +---- 3 files changed, 12 insertions(+), 121 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala index f5e2f4cbac2..f7b16c8ac0f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala @@ -246,68 +246,6 @@ object GpuCast extends Arm { } } - def sanitizeStringToIntegralType(input: ColumnVector, ansiEnabled: Boolean): ColumnVector = { - // Convert any strings containing whitespace to null values. The input is assumed to already - // have been stripped of leading and trailing whitespace - val sanitized = withResource(input.containsRe("\\s")) { hasWhitespace => - withResource(hasWhitespace.any()) { any => - if (any.isValid && any.getBoolean) { - if (ansiEnabled) { - throw new NumberFormatException(GpuCast.INVALID_INPUT_MESSAGE) - } else { - withResource(GpuScalar.from(null, DataTypes.StringType)) { nullVal => - hasWhitespace.ifElse(nullVal, input) - } - } - } else { - input.incRefCount() - } - } - } - - withResource(sanitized) { _ => - if (ansiEnabled) { - // ansi mode only supports simple integers, so no exponents or decimal places - val regex = "^[+\\-]?[0-9]+$" - withResource(sanitized.matchesRe(regex)) { isInt => - withResource(isInt.all()) { allInts => - // Check that all non-null values are valid integers. - if (allInts.isValid && !allInts.getBoolean) { - throw new NumberFormatException(GpuCast.INVALID_INPUT_MESSAGE) - } - } - sanitized.incRefCount() - } - } else { - // truncate strings that represent decimals to just look at the string before the dot - withResource(Scalar.fromString(".")) { dot => - withResource(sanitized.stringContains(dot)) { hasDot => - // only do the decimal sanitization if any strings do contain dot - withResource(hasDot.any(DType.BOOL8)) { anyDot => - if (anyDot.getBoolean) { - // Special handling for strings that have no numeric value before the dot, such - // as "." and ".1" because extractsRe returns null for the capture group - // for these values and it also returns null for invalid inputs so we need this - // explicit check - withResource(sanitized.matchesRe("^[+\\-]?\\.[0-9]*$")) { startsWithDot => - withResource(sanitized.extractRe("^([+\\-]?[0-9]*)\\.[0-9]*$")) { table => - withResource(Scalar.fromString("0")) { zero => - withResource(startsWithDot.ifElse(zero, table.getColumn(0))) { - decimal => hasDot.ifElse(decimal, sanitized) - } - } - } - } - } else { - sanitized.incRefCount() - } - } - } - } - } - } - } - def doCast( input: ColumnView, fromDataType: DataType, @@ -513,8 +451,10 @@ object GpuCast extends Arm { } case (FloatType | DoubleType, StringType) => castFloatingTypeToString(input) - case (StringType, BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType - | DoubleType | DateType | TimestampType) => + case (StringType, ByteType | ShortType | IntegerType | LongType ) => + com.nvidia.spark.rapids.jni.CastStrings.toInteger(input, ansiMode, + GpuColumnVector.getNonNestedRapidsType(toDataType)) + case (StringType, BooleanType | FloatType | DoubleType | DateType | TimestampType) => withResource(input.strip()) { trimmed => toDataType match { case BooleanType => @@ -530,9 +470,6 @@ object GpuCast extends Arm { case FloatType | DoubleType => castStringToFloats(trimmed, ansiMode, GpuColumnVector.getNonNestedRapidsType(toDataType)) - case ByteType | ShortType | IntegerType | LongType => - castStringToInts(trimmed, ansiMode, - GpuColumnVector.getNonNestedRapidsType(toDataType)) } } case (StringType, dt: DecimalType) => @@ -1043,30 +980,6 @@ object GpuCast extends Arm { } } - def castStringToInts( - input: ColumnVector, - ansiEnabled: Boolean, - dType: DType): ColumnVector = { - - withResource(GpuCast.sanitizeStringToIntegralType(input, ansiEnabled)) { sanitized => - withResource(sanitized.isInteger(dType)) { isInt => - if (ansiEnabled) { - withResource(isInt.all()) { allInts => - // Check that all non-null values are valid integers. - if (allInts.isValid && !allInts.getBoolean) { - throw new IllegalStateException(GpuCast.INVALID_INPUT_MESSAGE) - } - } - } - withResource(sanitized.castTo(dType)) { parsedInt => - withResource(Scalar.fromNull(dType)) { nullVal => - isInt.ifElse(parsedInt, nullVal) - } - } - } - } - } - def castStringToDecimal( input: ColumnView, ansiEnabled: Boolean, diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/AnsiCastOpSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/AnsiCastOpSuite.scala index 3134857cb04..a614ff4ac7e 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/AnsiCastOpSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/AnsiCastOpSuite.scala @@ -34,6 +34,8 @@ class AnsiCastOpSuite extends GpuExpressionTestSuite { import CastOpSuite._ + val INVALID_ROW_VALUE_MSG: String = "Error casting data on row" + private val sparkConf = new SparkConf() .set("spark.sql.ansi.enabled", "true") .set("spark.sql.legacy.allowNegativeScaleOfDecimal", "true") @@ -362,28 +364,28 @@ class AnsiCastOpSuite extends GpuExpressionTestSuite { } testCastFailsForBadInputs("ansi_cast string to byte (invalid values)", shortsAsStrings, - sparkConf) { + sparkConf, msg = INVALID_ROW_VALUE_MSG) { frame => testCastTo(DataTypes.ByteType)(frame) } testCastFailsForBadInputs("ansi_cast string to short (invalid values)", intsAsStrings, - sparkConf) { + sparkConf, msg = INVALID_ROW_VALUE_MSG) { frame => testCastTo(DataTypes.ShortType)(frame) } testCastFailsForBadInputs("ansi_cast string to long (invalid decimal values)", longsAsDecimalStrings, - sparkConf) { + sparkConf, msg = INVALID_ROW_VALUE_MSG) { frame => testCastTo(DataTypes.LongType)(frame) } testCastFailsForBadInputs("ansi_cast string to int (invalid values)", longsAsStrings, - sparkConf) { + sparkConf, msg = INVALID_ROW_VALUE_MSG) { frame => testCastTo(DataTypes.IntegerType)(frame) } testCastFailsForBadInputs("ansi_cast string to int (non-numeric values)", testStrings, - sparkConf) { + sparkConf, msg = INVALID_ROW_VALUE_MSG) { frame => testCastTo(DataTypes.IntegerType)(frame) } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala index a8d5d483c6d..bb612aed41f 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala @@ -89,7 +89,7 @@ class CastOpSuite extends GpuExpressionTestSuite { test("Cast from string to int using hand-picked values") { testCastStringTo(DataTypes.IntegerType, Seq(".--e-37602.n", "\r\r\t\n11.12380", "-.2", ".3", - ".", "+1.2", "\n123\n456\n", "1e+4")) + ".", "+1.2", "\n123\n456\n", "1e+4", "0.123", "321.123", ".\r123")) } test("Cast from string to int ANSI mode with mix of valid and invalid values") { @@ -987,30 +987,6 @@ class CastOpSuite extends GpuExpressionTestSuite { } } - test("CAST string to integer - sanitize step") { - val testPairs: Seq[(String, String)] = Seq( - (null, null), - ("1e4", "1e4"), - ("123", "123"), - (".", "0"), - (".2", "0"), - ("-.2", "0"), - ("0.123", "0"), - ("321.123", "321"), - ("0.123\r123", null), - (".\r123", null) - ) - val inputs = testPairs.map(_._1) - val expected = testPairs.map(_._2) - withResource(ColumnVector.fromStrings(inputs: _*)) { v => - withResource(ColumnVector.fromStrings(expected: _*)) { expected => - withResource(GpuCast.sanitizeStringToIntegralType(v, ansiEnabled = false)) { actual => - CudfTestHelper.assertColumnsAreEqual(expected, actual) - } - } - } - } - protected def testCastToDecimal( dataType: DataType, scale: Int, From e62410a74df0cb1b64f34e2a023ba30af09b7f11 Mon Sep 17 00:00:00 2001 From: NVnavkumar <97137715+NVnavkumar@users.noreply.github.com> Date: Mon, 12 Sep 2022 07:37:07 -0700 Subject: [PATCH 099/190] Enabling AQE on [databricks] (#6461) * WIP: first pass at ShimLeafExecNode, need to update indirect inheritors next Signed-off-by: Navin Kumar * Move this to until 3.4.0 for non-databricks spark versions Signed-off-by: Navin Kumar * Set this flag to true in 3.2.1 DB shim Signed-off-by: Navin Kumar * WIP: some test updates with enabling AQE Signed-off-by: Navin Kumar * Move these shim implementations to right place Signed-off-by: Navin Kumar * revert this test change for now, need a better solution * WIP: Re-enable aqe Databricks tests Signed-off-by: Navin Kumar * Unblock these 2 tests on Databricks Signed-off-by: Navin Kumar * WIP: integration tests for AQE * WIP: AQE integration tests Signed-off-by: Navin Kumar * Updated AQE tests to ensure that leafexecnodes are tested for Databricks * Add shim for DatasourceV2ExecBase to implement the equivalent computeStats fix Signed-off-by: Navin Kumar * fix unused import on Spark 3.1.x Signed-off-by: Navin Kumar * Add AQE unit test to handle window aggregate condition Signed-off-by: Navin Kumar * Fix windowexec issue with missing references to child expressions due to AQE optimizations Signed-off-by: Navin Kumar * Fix some style issues Signed-off-by: Navin Kumar * Found a potential union based join unit test that will crash when AQE is enabled on Databricks Signed-off-by: Navin Kumar * Disable GPU shuffle on older Databricks, and switch current Databricks to using original Spark implementation to fix concurrency bug in shim Signed-off-by: Navin Kumar * Refactor unit tests for handling issues with Databricks 9.1 Signed-off-by: Navin Kumar * Address feedback Signed-off-by: Navin Kumar * Update comment Signed-off-by: Navin Kumar * Enable GPU shuffle in AQE on Databricks 9.1, remove unnecessary shim logic Signed-off-by: Navin Kumar * cleanup and add comments to tests Signed-off-by: Navin Kumar * Add cache join test for AQE Signed-off-by: Navin Kumar * remove windowing fix, and move to a separate branch since this is not an AQE-specific bug Signed-off-by: Navin Kumar * This should be allowed not to run on GPU since AQE can push it off Signed-off-by: Navin Kumar * Allow ColumnarToRowExec to not run on GPU because it tends to fallback in CI Signed-off-by: Navin Kumar Signed-off-by: Navin Kumar --- integration_tests/src/main/python/aqe_test.py | 123 ++++++++++++++++++ .../src/main/python/cache_test.py | 14 ++ .../src/main/python/join_test.py | 6 - .../spark/rapids/shims/ShimLeafExecNode.scala | 43 ++++++ .../spark/rapids/shims/ShimLeafExecNode.scala | 24 ++++ .../spark/rapids/shims/GpuBatchScanExec.scala | 3 +- .../nvidia/spark/rapids/shims/AQEUtils.scala | 3 +- .../rapids/shims/GpuShuffleExchangeExec.scala | 3 +- .../spark/rapids/shims/GpuBatchScanExec.scala | 2 +- .../nvidia/spark/rapids/shims/AQEUtils.scala | 3 +- .../rapids/shims/GpuShuffleExchangeExec.scala | 16 +-- .../spark/rapids/shims/GpuBatchScanExec.scala | 2 +- .../spark/rapids/basicPhysicalOperators.scala | 6 +- .../sql/rapids/GpuDataSourceScanExec.scala | 5 +- .../sql/rapids/GpuInMemoryTableScanExec.scala | 5 +- 15 files changed, 223 insertions(+), 35 deletions(-) create mode 100755 integration_tests/src/main/python/aqe_test.py create mode 100644 sql-plugin/src/main/311+-db/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala create mode 100644 sql-plugin/src/main/311+-nondb/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala diff --git a/integration_tests/src/main/python/aqe_test.py b/integration_tests/src/main/python/aqe_test.py new file mode 100755 index 00000000000..8865bf3477a --- /dev/null +++ b/integration_tests/src/main/python/aqe_test.py @@ -0,0 +1,123 @@ +# Copyright (c) 2022, NVIDIA CORPORATION. +# +# 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. + +import pytest +from pyspark.sql.functions import when, col +from pyspark.sql.types import * +from asserts import assert_gpu_and_cpu_are_equal_collect +from data_gen import * +from marks import ignore_order, allow_non_gpu +from spark_session import with_cpu_session + +_adaptive_conf = { "spark.sql.adaptive.enabled": "true" } + +def create_skew_df(spark, length): + root = spark.range(0, length) + mid = length / 2 + left = root.select( + when(col('id') < mid / 2, mid). + otherwise('id').alias("key1"), + col('id').alias("value1") + ) + right = root.select( + when(col('id') < mid, mid). + otherwise('id').alias("key2"), + col('id').alias("value2") + ) + return left, right + + +# This replicates the skew join test from scala tests, and is here to test +# the computeStats(...) implementation in GpuRangeExec +@ignore_order(local=True) +def test_aqe_skew_join(): + def do_join(spark): + left, right = create_skew_df(spark, 500) + left.createOrReplaceTempView("skewData1") + right.createOrReplaceTempView("skewData2") + return spark.sql("SELECT * FROM skewData1 join skewData2 ON key1 = key2") + + assert_gpu_and_cpu_are_equal_collect(do_join, conf=_adaptive_conf) + +# Test the computeStats(...) implementation in GpuDataSourceScanExec +@ignore_order(local=True) +@pytest.mark.parametrize("data_gen", integral_gens, ids=idfn) +def test_aqe_join_parquet(spark_tmp_path, data_gen): + data_path = spark_tmp_path + '/PARQUET_DATA' + with_cpu_session( + lambda spark: unary_op_df(spark, data_gen).orderBy('a').write.parquet(data_path) + ) + + def do_it(spark): + spark.read.parquet(data_path).createOrReplaceTempView('df1') + spark.read.parquet(data_path).createOrReplaceTempView('df2') + return spark.sql("select count(*) from df1,df2 where df1.a = df2.a") + + assert_gpu_and_cpu_are_equal_collect(do_it, conf=_adaptive_conf) + + +# Test the computeStats(...) implementation in GpuBatchScanExec +@ignore_order(local=True) +@pytest.mark.parametrize("data_gen", integral_gens, ids=idfn) +def test_aqe_join_parquet_batch(spark_tmp_path, data_gen): + # force v2 source for parquet to use BatchScanExec + conf = copy_and_update(_adaptive_conf, { + "spark.sql.sources.useV1SourceList": "" + }) + + first_data_path = spark_tmp_path + '/PARQUET_DATA/key=0' + with_cpu_session( + lambda spark : unary_op_df(spark, data_gen).write.parquet(first_data_path)) + second_data_path = spark_tmp_path + '/PARQUET_DATA/key=1' + with_cpu_session( + lambda spark : unary_op_df(spark, data_gen).write.parquet(second_data_path)) + data_path = spark_tmp_path + '/PARQUET_DATA' + + def do_it(spark): + spark.read.parquet(data_path).createOrReplaceTempView('df1') + spark.read.parquet(data_path).createOrReplaceTempView('df2') + return spark.sql("select count(*) from df1,df2 where df1.a = df2.a") + + assert_gpu_and_cpu_are_equal_collect(do_it, conf=conf) + +# Test the map stage submission handling for GpuShuffleExchangeExec +@ignore_order(local=True) +def test_aqe_struct_self_join(spark_tmp_table_factory): + def do_join(spark): + data = [ + (("Adam ", "", "Green"), "1", "M", 1000), + (("Bob ", "Middle", "Green"), "2", "M", 2000), + (("Cathy ", "", "Green"), "3", "F", 3000) + ] + schema = (StructType() + .add("name", StructType() + .add("firstname", StringType()) + .add("middlename", StringType()) + .add("lastname", StringType())) + .add("id", StringType()) + .add("gender", StringType()) + .add("salary", IntegerType())) + df = spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + df_name = spark_tmp_table_factory.get() + df.createOrReplaceTempView(df_name) + resultdf = spark.sql( + "select struct(name, struct(name.firstname, name.lastname) as newname)" + + " as col,name from " + df_name + " union" + + " select struct(name, struct(name.firstname, name.lastname) as newname) as col,name" + + " from " + df_name) + resultdf_name = spark_tmp_table_factory.get() + resultdf.createOrReplaceTempView(resultdf_name) + return spark.sql("select a.* from {} a, {} b where a.name=b.name".format( + resultdf_name, resultdf_name)) + assert_gpu_and_cpu_are_equal_collect(do_join, conf=_adaptive_conf) diff --git a/integration_tests/src/main/python/cache_test.py b/integration_tests/src/main/python/cache_test.py index 8e37fb61d1c..47e25c8b2f2 100644 --- a/integration_tests/src/main/python/cache_test.py +++ b/integration_tests/src/main/python/cache_test.py @@ -333,3 +333,17 @@ def test_func(spark): df.cache().count() return df.selectExpr("b", "a") assert_gpu_and_cpu_are_equal_collect(test_func, enable_vectorized_conf) + +# For AQE, test the computeStats(...) implementation in GpuInMemoryTableScanExec +# NOTE: this test is here because the necessary cache configuration is only +# available when this test file is used +@ignore_order(local=True) +@allow_non_gpu("ShuffleExchangeExec", "ColumnarToRowExec") +@pytest.mark.parametrize("data_gen", integral_gens, ids=idfn) +def test_aqe_cache_join(data_gen): + conf = {'spark.sql.adaptive.enabled': 'true'} + def do_it(spark): + df1 = unary_op_df(spark, data_gen).orderBy('a').cache() + df2 = df1.alias('df2') + return df1.join(df2, df1.a == df2.a, 'Outer') + assert_gpu_and_cpu_are_equal_collect(do_it, conf=conf) \ No newline at end of file diff --git a/integration_tests/src/main/python/join_test.py b/integration_tests/src/main/python/join_test.py index 3c7ee538138..c1d27828667 100644 --- a/integration_tests/src/main/python/join_test.py +++ b/integration_tests/src/main/python/join_test.py @@ -144,9 +144,6 @@ def do_join(spark): assert_gpu_and_cpu_are_equal_collect(do_join) @ignore_order(local=True) -@pytest.mark.skipif(is_databricks_runtime(), - reason="Disabled for databricks because of lack of AQE support, and " - "differences in BroadcastMode.transform") @pytest.mark.parametrize('join_type', ['Left', 'Inner', 'LeftSemi', 'LeftAnti'], ids=idfn) def test_right_broadcast_nested_loop_join_without_condition_empty_small_batch(join_type): def do_join(spark): @@ -155,9 +152,6 @@ def do_join(spark): assert_gpu_and_cpu_are_equal_collect(do_join, conf={'spark.sql.adaptive.enabled': 'true'}) @ignore_order(local=True) -@pytest.mark.skipif(is_databricks_runtime(), - reason="Disabled for databricks because of lack of AQE support, and " - "differences in BroadcastMode.transform") @pytest.mark.parametrize('join_type', ['Left', 'Right', 'Inner', 'LeftSemi', 'LeftAnti'], ids=idfn) def test_empty_broadcast_hash_join(join_type): def do_join(spark): diff --git a/sql-plugin/src/main/311+-db/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala b/sql-plugin/src/main/311+-db/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala new file mode 100644 index 00000000000..877442f6d90 --- /dev/null +++ b/sql-plugin/src/main/311+-db/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala @@ -0,0 +1,43 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.catalyst.plans.logical.Statistics +import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase + +trait ShimLeafExecNode extends LeafExecNode { + // For AQE support in Databricks, all Exec nodes implement computeStats(). This is actually + // a recursive call to traverse the entire physical plan to aggregate this number. For the + // end of the computation, this means that all LeafExecNodes must implement this method to + // avoid a stack overflow. For now, based on feedback from Databricks, Long.MaxValue is + // sufficient to satisfy this computation. + override def computeStats(): Statistics = { + Statistics( + sizeInBytes = Long.MaxValue + ) + } +} + +// DataSourceV2ScanExecBase actually extends LeafExecNode, so we extend that shim as well here. +trait ShimDataSourceV2ScanExecBase extends DataSourceV2ScanExecBase { + override def computeStats(): Statistics = { + Statistics( + sizeInBytes = Long.MaxValue + ) + } +} \ No newline at end of file diff --git a/sql-plugin/src/main/311+-nondb/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala b/sql-plugin/src/main/311+-nondb/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala new file mode 100644 index 00000000000..3b46a1339d7 --- /dev/null +++ b/sql-plugin/src/main/311+-nondb/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala @@ -0,0 +1,24 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase + +trait ShimLeafExecNode extends LeafExecNode + +trait ShimDataSourceV2ScanExecBase extends DataSourceV2ScanExecBase \ No newline at end of file diff --git a/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index 41dbb864155..11b6b6383ac 100644 --- a/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/311until320-all/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -23,11 +23,10 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.connector.read._ -import org.apache.spark.sql.execution.datasources.v2._ case class GpuBatchScanExec( output: Seq[AttributeReference], - @transient scan: Scan) extends DataSourceV2ScanExecBase with GpuBatchScanExecMetrics { + @transient scan: Scan) extends ShimDataSourceV2ScanExecBase with GpuBatchScanExecMetrics { @transient lazy val batch: Batch = scan.toBatch @transient override lazy val partitions: Seq[InputPartition] = batch.planInputPartitions() diff --git a/sql-plugin/src/main/312db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala b/sql-plugin/src/main/312db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala index 2a291852e8a..33302c1d8d0 100644 --- a/sql-plugin/src/main/312db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala +++ b/sql-plugin/src/main/312db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala @@ -28,6 +28,5 @@ object AQEUtils { ShuffleQueryStageExec(sqse.id, reusedExchange, sqse.originalPlan) } - // currently we don't support AQE on Databricks - def isAdaptiveExecutionSupportedInSparkVersion: Boolean = false + def isAdaptiveExecutionSupportedInSparkVersion: Boolean = true } diff --git a/sql-plugin/src/main/31xdb/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala b/sql-plugin/src/main/31xdb/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala index b261e15823e..57aff608c82 100644 --- a/sql-plugin/src/main/31xdb/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala +++ b/sql-plugin/src/main/31xdb/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala @@ -39,7 +39,8 @@ case class GpuShuffleExchangeExec( override val outputPartitioning: Partitioning = cpuOutputPartitioning // 'mapOutputStatisticsFuture' is only needed when enable AQE. - override def doMapOutputStatisticsFuture: Future[MapOutputStatistics] = { + @transient + override lazy val doMapOutputStatisticsFuture: Future[MapOutputStatistics] = { if (inputBatchRDD.getNumPartitions == 0) { Future.successful(null) } else { diff --git a/sql-plugin/src/main/320until330-all/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/320until330-all/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index 6046b24318f..6fbdaa684e5 100644 --- a/sql-plugin/src/main/320until330-all/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/320until330-all/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -34,7 +34,7 @@ case class GpuBatchScanExec( output: Seq[AttributeReference], @transient scan: Scan, runtimeFilters: Seq[Expression] = Seq.empty) - extends DataSourceV2ScanExecBase with GpuBatchScanExecMetrics { + extends ShimDataSourceV2ScanExecBase with GpuBatchScanExecMetrics { @transient lazy val batch: Batch = scan.toBatch // All expressions are filter expressions used on the CPU. diff --git a/sql-plugin/src/main/321db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala b/sql-plugin/src/main/321db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala index 20075a69b03..1830a01e996 100644 --- a/sql-plugin/src/main/321db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala +++ b/sql-plugin/src/main/321db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala @@ -28,6 +28,5 @@ object AQEUtils { ShuffleQueryStageExec(sqse.id, reusedExchange, sqse.originalPlan, sqse.isSparkExchange) } - // currently we don't support AQE on Databricks - def isAdaptiveExecutionSupportedInSparkVersion: Boolean = false + def isAdaptiveExecutionSupportedInSparkVersion: Boolean = true } diff --git a/sql-plugin/src/main/321db/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala b/sql-plugin/src/main/321db/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala index 44d61961b40..26dda745e8b 100644 --- a/sql-plugin/src/main/321db/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala +++ b/sql-plugin/src/main/321db/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala @@ -15,37 +15,27 @@ */ package org.apache.spark.rapids.shims -import scala.concurrent.Future - import com.nvidia.spark.rapids.GpuPartitioning -import org.apache.spark.MapOutputStatistics import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{ShufflePartitionSpec, SparkPlan} import org.apache.spark.sql.execution.exchange.{ShuffleExchangeLike, ShuffleOrigin} -import org.apache.spark.sql.rapids.execution.{GpuShuffleExchangeExecBase, ShuffledBatchRDD} +import org.apache.spark.sql.rapids.execution.{GpuShuffleExchangeExecBaseWithMetrics, ShuffledBatchRDD} case class GpuShuffleExchangeExec( gpuOutputPartitioning: GpuPartitioning, child: SparkPlan, shuffleOrigin: ShuffleOrigin)( cpuOutputPartitioning: Partitioning) - extends GpuShuffleExchangeExecBase(gpuOutputPartitioning, child) with ShuffleExchangeLike { + extends GpuShuffleExchangeExecBaseWithMetrics(gpuOutputPartitioning, child) + with ShuffleExchangeLike { override def otherCopyArgs: Seq[AnyRef] = cpuOutputPartitioning :: Nil override val outputPartitioning: Partitioning = cpuOutputPartitioning - // 'mapOutputStatisticsFuture' is only needed when enable AQE. - override def mapOutputStatisticsFuture: Future[MapOutputStatistics] = - if (inputBatchRDD.getNumPartitions == 0) { - Future.successful(null) - } else { - sparkContext.submitMapStage(shuffleDependencyColumnar) - } - override def numMappers: Int = shuffleDependencyColumnar.rdd.getNumPartitions override def numPartitions: Int = shuffleDependencyColumnar.partitioner.numPartitions diff --git a/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index a80e4b90392..8f334798fae 100644 --- a/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -36,7 +36,7 @@ case class GpuBatchScanExec( @transient scan: Scan, runtimeFilters: Seq[Expression] = Seq.empty, keyGroupedPartitioning: Option[Seq[Expression]] = None) - extends DataSourceV2ScanExecBase with GpuBatchScanExecMetrics { + extends ShimDataSourceV2ScanExecBase with GpuBatchScanExecMetrics { @transient lazy val batch: Batch = scan.toBatch // All expressions are filter expressions used on the CPU. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala index 35c6724364c..b3cbdcea024 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala @@ -23,7 +23,7 @@ import ai.rapids.cudf import ai.rapids.cudf._ import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ -import com.nvidia.spark.rapids.shims.{ShimSparkPlan, ShimUnaryExecNode} +import com.nvidia.spark.rapids.shims.{ShimLeafExecNode, ShimSparkPlan, ShimUnaryExecNode} import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskContext} import org.apache.spark.internal.Logging @@ -31,7 +31,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeReference, Descending, Expression, NamedExpression, NullIntolerant, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, RangePartitioning, SinglePartition, UnknownPartitioning} -import org.apache.spark.sql.execution.{LeafExecNode, ProjectExec, SampleExec, SparkPlan} +import org.apache.spark.sql.execution.{ProjectExec, SampleExec, SparkPlan} import org.apache.spark.sql.rapids.{GpuPartitionwiseSampledRDD, GpuPoissonSampler, GpuPredicateHelper} import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.types.{DataType, LongType} @@ -591,7 +591,7 @@ case class GpuRangeExec( numSlices: Int, output: Seq[Attribute], targetSizeBytes: Long) - extends LeafExecNode with GpuExec { + extends ShimLeafExecNode with GpuExec { val numElements: BigInt = { val safeStart = BigInt(start) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceScanExec.scala index 321f5f34439..be93d979d00 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceScanExec.scala @@ -17,18 +17,19 @@ package org.apache.spark.sql.rapids import com.nvidia.spark.rapids.GpuExec +import com.nvidia.spark.rapids.shims.ShimLeafExecNode import org.apache.commons.lang3.StringUtils import org.apache.hadoop.fs.Path import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.execution.{ExplainUtils, LeafExecNode} +import org.apache.spark.sql.execution.ExplainUtils import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.util.Utils /** GPU implementation of Spark's `DataSourceScanExec` */ -trait GpuDataSourceScanExec extends LeafExecNode with GpuExec { +trait GpuDataSourceScanExec extends ShimLeafExecNode with GpuExec { def relation: BaseRelation def tableIdentifier: Option[TableIdentifier] diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInMemoryTableScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInMemoryTableScanExec.scala index acbb6a3bc5a..8d546d384d9 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInMemoryTableScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInMemoryTableScanExec.scala @@ -18,13 +18,14 @@ package org.apache.spark.sql.rapids import com.nvidia.spark.ParquetCachedBatchSerializer import com.nvidia.spark.rapids.{DataFromReplacementRule, ExecChecks, GpuExec, GpuMetric, RapidsConf, RapidsMeta, SparkPlanMeta} +import com.nvidia.spark.rapids.shims.ShimLeafExecNode import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Expression, SortOrder} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.types.DataType @@ -75,7 +76,7 @@ class InMemoryTableScanMeta( case class GpuInMemoryTableScanExec( attributes: Seq[Attribute], predicates: Seq[Expression], - @transient relation: InMemoryRelation) extends LeafExecNode with GpuExec { + @transient relation: InMemoryRelation) extends ShimLeafExecNode with GpuExec { override val nodeName: String = { relation.cacheBuilder.tableName match { From f83a5dc128b76ce1fbbacbed406cb3b342ae5e11 Mon Sep 17 00:00:00 2001 From: Nghia Truong Date: Mon, 12 Sep 2022 08:18:06 -0700 Subject: [PATCH 100/190] Fix parquet write when the input column is nested type containing timestamp (#6473) * Rewrite `deepTransform` Signed-off-by: Nghia Truong * Implement `deepTransformColumn` using `deepTransform` Signed-off-by: Nghia Truong * Fix memory leak bug Signed-off-by: Nghia Truong * Reverse some code Signed-off-by: Nghia Truong * Change interface Signed-off-by: Nghia Truong * Add a blank line Signed-off-by: Nghia Truong * Change integration test back, adding the removed tests Signed-off-by: Nghia Truong * Cleanup Signed-off-by: Nghia Truong * Cleanup Signed-off-by: Nghia Truong * Remove `TIMESTAMP_DAYS` from matching Signed-off-by: Nghia Truong * Fix integration test for `test_cache_columnar` Signed-off-by: Nghia Truong * Fix compile error Signed-off-by: Nghia Truong * Passing data type for children columns Signed-off-by: Nghia Truong * Fix compile issue due to merge conflict Signed-off-by: Nghia Truong * Rename variable Signed-off-by: Nghia Truong * Handle `BinaryType` Signed-off-by: Nghia Truong * Use `safeMap` to handle exception Signed-off-by: Nghia Truong * Rewrite pattern matching Signed-off-by: Nghia Truong * Fix import Signed-off-by: Nghia Truong * Fix comment space Signed-off-by: Nghia Truong Signed-off-by: Nghia Truong --- .../src/main/python/cache_test.py | 10 +- .../src/main/python/parquet_write_test.py | 23 +--- .../nvidia/spark/rapids/shims/HashUtils.scala | 4 +- .../nvidia/spark/rapids/ColumnCastUtil.scala | 51 ++++++-- .../spark/rapids/GpuParquetFileFormat.scala | 114 +++++++++++------- .../sql/rapids/collectionOperations.scala | 2 +- 6 files changed, 113 insertions(+), 91 deletions(-) diff --git a/integration_tests/src/main/python/cache_test.py b/integration_tests/src/main/python/cache_test.py index 47e25c8b2f2..afba9539333 100644 --- a/integration_tests/src/main/python/cache_test.py +++ b/integration_tests/src/main/python/cache_test.py @@ -152,14 +152,6 @@ def n_fold(spark): # This test doesn't allow negative scale for Decimals as ` df.write.mode('overwrite').parquet(data_path)` # writes parquet which doesn't allow negative decimals -# In addition, `TIMESTAMP_MILLIS` can't be handled correctly when the input is of nested types containing timestamp. -# See issue https://github.com/NVIDIA/spark-rapids/issues/6302. -# Thus, we exclude nested types contaning timestamp from the tests here. -# When the issue is resolved, remove `_cache_single_array_gens_no_null_no_timestamp` and -# use just `_cache_single_array_gens_no_null` for `data_gen` parameter. -_all_basic_gens_no_null_no_timestamp = [gen for gen in all_basic_gens_no_null if gen != timestamp_gen] -_cache_single_array_gens_no_null_no_timestamp = [ArrayGen(gen) for gen in _all_basic_gens_no_null_no_timestamp + - _cache_decimal_gens] @pytest.mark.parametrize('data_gen', [StringGen(), ByteGen(), ShortGen(), IntegerGen(), LongGen(), ArrayGen( StructGen([['child0', StringGen()], @@ -168,7 +160,7 @@ def n_fold(spark): pytest.param(FloatGen(special_cases=[FLOAT_MIN, FLOAT_MAX, 0.0, 1.0, -1.0]), marks=[incompat]), pytest.param(DoubleGen(special_cases=double_special_cases), marks=[incompat]), BooleanGen(), DateGen(), TimestampGen(), decimal_gen_32bit, decimal_gen_64bit, - orderable_decimal_gen_128bit] + _cache_single_array_gens_no_null_no_timestamp, ids=idfn) + orderable_decimal_gen_128bit] + _cache_single_array_gens_no_null, ids=idfn) @pytest.mark.parametrize('ts_write', ['TIMESTAMP_MICROS', 'TIMESTAMP_MILLIS']) @pytest.mark.parametrize('enable_vectorized', ['true', 'false'], ids=idfn) @ignore_order diff --git a/integration_tests/src/main/python/parquet_write_test.py b/integration_tests/src/main/python/parquet_write_test.py index 3e0519cf871..84326957a99 100644 --- a/integration_tests/src/main/python/parquet_write_test.py +++ b/integration_tests/src/main/python/parquet_write_test.py @@ -97,33 +97,12 @@ def test_write_round_trip(spark_tmp_path, parquet_gens): data_path, conf=writer_confs) -# `TIMESTAMP_MILLIS` can't be handled correctly when the input is of nested types containing timestamp. -# See issue https://github.com/NVIDIA/spark-rapids/issues/6302. -# Thus, we exclude `TIMESTAMP_MILLIS` from the tests here. -# When the issue is resolved, unify this test with the test below. -parquet_ts_write_options_no_millis = ['INT96', 'TIMESTAMP_MICROS'] @pytest.mark.parametrize('parquet_gens', [[ limited_timestamp(), ArrayGen(limited_timestamp(), max_length=10), MapGen(limited_timestamp(nullable=False), limited_timestamp())]], ids=idfn) -@pytest.mark.parametrize('ts_type', parquet_ts_write_options_no_millis) -def test_timestamp_write_round_trip_no_millis(spark_tmp_path, parquet_gens, ts_type): - gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] - data_path = spark_tmp_path + '/PARQUET_DATA' - all_confs = copy_and_update(writer_confs, {'spark.sql.parquet.outputTimestampType': ts_type}) - assert_gpu_and_cpu_writes_are_equal_collect( - lambda spark, path: gen_df(spark, gen_list).coalesce(1).write.parquet(path), - lambda spark, path: spark.read.parquet(path), - data_path, - conf=all_confs) - -# `TIMESTAMP_MILLIS` can't be handled correctly when the input is of nested types containing timestamp. -# See issue https://github.com/NVIDIA/spark-rapids/issues/6302. -# Thus, we exclude nested types contaning timestamp from the tests here. -# When the issue is resolved, unify this test with the test above. -@pytest.mark.parametrize('parquet_gens', [[limited_timestamp()]], ids=idfn) @pytest.mark.parametrize('ts_type', parquet_ts_write_options) -def test_timestamp_write_round_trip_no_nested_timestamp(spark_tmp_path, parquet_gens, ts_type): +def test_timestamp_write_round_trip(spark_tmp_path, parquet_gens, ts_type): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] data_path = spark_tmp_path + '/PARQUET_DATA' all_confs = copy_and_update(writer_confs, {'spark.sql.parquet.outputTimestampType': ts_type}) diff --git a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/HashUtils.scala b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/HashUtils.scala index 9590dd909da..bbaaa68c785 100644 --- a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/HashUtils.scala +++ b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/HashUtils.scala @@ -28,13 +28,13 @@ object HashUtils extends Arm { // This looks really stupid, but -0.0 in cudf is equal to 0.0 so we can check if they are // equal and replace it with the same thing to normalize it. ColumnCastUtil.deepTransform(in) { - case cv if cv.getType == cudf.DType.FLOAT32 => + case (cv, _) if cv.getType == cudf.DType.FLOAT32 => withResource(cudf.Scalar.fromFloat(0.0f)) { zero => withResource(cv.equalTo(zero)) { areEqual => areEqual.ifElse(zero, cv) } } - case cv if cv.getType == cudf.DType.FLOAT64 => + case (cv, _) if cv.getType == cudf.DType.FLOAT64 => withResource(cudf.Scalar.fromDouble(0.0)) { zero => withResource(cv.equalTo(zero)) { areEqual => areEqual.ifElse(zero, cv) diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/ColumnCastUtil.scala b/sql-plugin/src/main/java/com/nvidia/spark/rapids/ColumnCastUtil.scala index f2734f509f9..dd3b0f427a2 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/ColumnCastUtil.scala +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/ColumnCastUtil.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.{ArrayBuffer, ArrayBuilder} import ai.rapids.cudf.{ColumnVector, ColumnView, DType} -import org.apache.spark.sql.types.{ArrayType, DataType, StructType, MapType, StructField} +import org.apache.spark.sql.types.{ArrayType, BinaryType, ByteType, DataType, MapType, StructField, StructType} /** * This class casts a column to another column if the predicate passed resolves to true. @@ -43,16 +43,17 @@ object ColumnCastUtil extends Arm { * everything in the returned collection of AutoCloseable values. * * @param cv the view to be updated + * @param dt the Spark's data type of the input view (if applicable) * @param convert the partial function used to convert the data. If this matches and returns * a updated view this function takes ownership of that view. * @return None if there were no changes to the view or the updated view along with anything else * that needs to be closed. */ - def deepTransformView(cv: ColumnView) - (convert: PartialFunction[ColumnView, ColumnView]): + def deepTransformView(cv: ColumnView, dt: Option[DataType] = None) + (convert: PartialFunction[(ColumnView, Option[DataType]), ColumnView]): (Option[ColumnView], ArrayBuffer[AutoCloseable]) = { closeOnExcept(ArrayBuffer.empty[AutoCloseable]) { needsClosing => - val updated = convert.lift(cv) + val updated = convert.lift((cv, dt)) needsClosing ++= updated updated match { @@ -63,12 +64,24 @@ object ColumnCastUtil extends Arm { cv.getType.getTypeId match { case DType.DTypeEnum.STRUCT => withResource(ArrayBuffer.empty[ColumnView]) { tmpNeedsClosed => + val structFields = dt match { + case None => Array.empty[StructField] + case Some(t: StructType) => t.fields + case Some(t) => /* this should never be reach out */ + throw new IllegalStateException("Invalid input DataType: " + + s"Expect StructType but got ${t.toString}") + } var childrenUpdated = false val newChildren = ArrayBuffer.empty[ColumnView] (0 until cv.getNumChildren).foreach { index => val child = cv.getChildColumnView(index) tmpNeedsClosed += child - val (updatedChild, needsClosingChild) = deepTransformView(child)(convert) + val childDt = if (structFields.nonEmpty) { + Some(structFields(index).dataType) + } else { + None + } + val (updatedChild, needsClosingChild) = deepTransformView(child, childDt)(convert) needsClosing ++= needsClosingChild updatedChild match { case Some(newChild) => @@ -89,8 +102,23 @@ object ColumnCastUtil extends Arm { } } case DType.DTypeEnum.LIST => - withResource(cv.getChildColumnView(0)) { dataView => - val (updatedData, needsClosingData) = deepTransformView(dataView)(convert) + withResource(cv.getChildColumnView(0)) { child => + // A ColumnView of LIST type may have data type is ArrayType or MapType in Spark. + // If it is a MapType, its child will be a column of type struct. + // In such cases, we need to generate the corresponding Spark's data type + // for the child column as a StructType. + val childDt = dt match { + case None => None + case Some(t: ArrayType) => Some(t.elementType) + case Some(_: BinaryType) => Some(ByteType) + case Some(t: MapType) => Some(StructType(Array( + StructField("key", t.keyType, nullable = false), + StructField("value", t.valueType, nullable = t.valueContainsNull)))) + case Some(t) => /* this should never be reach out */ + throw new IllegalStateException("Invalid input DataType: " + + s"Expect ArrayType/BinaryType/MapType but got ${t.toString}") + } + val (updatedData, needsClosingData) = deepTransformView(child, childDt)(convert) needsClosing ++= needsClosingData updatedData match { case Some(updated) => @@ -113,13 +141,14 @@ object ColumnCastUtil extends Arm { * A lot of caution needs to be taken when using this method because of ownership of the data. * * @param cv the vector to be updated + * @param dt the Spark's data type of the input vector (if applicable) * @param convert the partial function used to convert the data. If this matches and returns * a updated view this function takes ownership of that view. * @return the updated vector */ - def deepTransform(cv: ColumnVector) - (convert: PartialFunction[ColumnView, ColumnView]): ColumnVector = { - val (retView, needsClosed) = deepTransformView(cv)(convert) + def deepTransform(cv: ColumnVector, dt: Option[DataType] = None) + (convert: PartialFunction[(ColumnView, Option[DataType]), ColumnView]): ColumnVector = { + val (retView, needsClosed) = deepTransformView(cv, dt)(convert) withResource(needsClosed) { _ => retView match { case Some(updated) => @@ -195,7 +224,7 @@ object ColumnCastUtil extends Arm { // map is list of structure val struct = cv.getChildColumnView(0) toClose += struct - + if(cv.getType != DType.LIST || struct.getType != DType.STRUCT) { throw new IllegalStateException("Map should be List(Structure) in column view") } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala index 522cfc3df30..106455616a4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala @@ -18,6 +18,7 @@ package com.nvidia.spark.rapids import ai.rapids.cudf._ import com.nvidia.spark.RebaseHelper +import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableProducingArray import com.nvidia.spark.rapids.shims.{ParquetFieldIdShims, ParquetTimestampNTZShims, SparkShimImpl} import org.apache.hadoop.mapreduce.{Job, OutputCommitter, TaskAttemptContext} import org.apache.parquet.hadoop.{ParquetOutputCommitter, ParquetOutputFormat} @@ -306,41 +307,36 @@ class GpuParquetWriter( } } - /** - * Persists a columnar batch. Invoked on the executor side. When writing to dynamically - * partitioned tables, dynamic partition columns are not included in columns to be written. - * NOTE: It is the writer's responsibility to close the batch. - */ - override def write(batch: ColumnarBatch, - statsTrackers: Seq[ColumnarWriteTaskStatsTracker]): Unit = { - val outputMillis = outputTimestampType == ParquetOutputTimestampType.TIMESTAMP_MILLIS.toString - val newBatch = - new ColumnarBatch(GpuColumnVector.extractColumns(batch).map { - cv => { - cv.dataType() match { - case DataTypes.TimestampType if outputMillis => - new GpuColumnVector(DataTypes.TimestampType, withResource(cv.getBase()) { v => - v.castTo(DType.TIMESTAMP_MILLISECONDS) - }) - case DataTypes.TimestampType - if outputTimestampType == ParquetOutputTimestampType.INT96.toString => - withResource(Scalar.fromLong(Long.MaxValue / 1000)) { upper => - withResource(Scalar.fromLong(Long.MinValue / 1000)) { lower => - withResource(cv.getBase().bitCastTo(DType.INT64)) { int64 => - withResource(int64.greaterOrEqualTo(upper)) { a => - withResource(int64.lessOrEqualTo(lower)) { b => - withResource(a.or(b)) { aOrB => - withResource(aOrB.any()) { any => - if (any.isValid && any.getBoolean) { - // its the writer's responsibility to close the batch - batch.close() - throw new IllegalArgumentException("INT96 column contains one " + - "or more values that can overflow and will result in data " + - "corruption. Please set " + - "`spark.rapids.sql.format.parquet.writer.int96.enabled` to false" + - " so we can fallback on CPU for writing parquet but still take " + - "advantage of parquet read on the GPU.") - } + private def deepTransformColumn(cv: ColumnVector, dt: DataType): ColumnVector = { + ColumnCastUtil.deepTransform(cv, Some(dt)) { + // Timestamp types are checked and transformed for all nested columns. + // Note that cudf's `isTimestampType` returns `true` for `TIMESTAMP_DAYS`, which is not + // included in Spark's `TimestampType`. + case (cv, _) if cv.getType.isTimestampType && cv.getType != DType.TIMESTAMP_DAYS => + val typeMillis = ParquetOutputTimestampType.TIMESTAMP_MILLIS.toString + val typeInt96 = ParquetOutputTimestampType.INT96.toString + + outputTimestampType match { + case `typeMillis` if cv.getType != DType.TIMESTAMP_MILLISECONDS => + cv.castTo(DType.TIMESTAMP_MILLISECONDS) + + case `typeInt96` => + withResource(Scalar.fromLong(Long.MaxValue / 1000)) { upper => + withResource(Scalar.fromLong(Long.MinValue / 1000)) { lower => + withResource(cv.bitCastTo(DType.INT64)) { int64 => + withResource(int64.greaterOrEqualTo(upper)) { a => + withResource(int64.lessOrEqualTo(lower)) { b => + withResource(a.or(b)) { aOrB => + withResource(aOrB.any()) { any => + if (any.isValid && any.getBoolean) { + // Its the writer's responsibility to close the input batch when this + // exception is thrown. + throw new IllegalArgumentException("INT96 column contains one " + + "or more values that can overflow and will result in data " + + "corruption. Please set " + + "`spark.rapids.sql.format.parquet.writer.int96.enabled` to false" + + " so we can fallback on CPU for writing parquet but still take " + + "advantage of parquet read on the GPU.") } } } @@ -348,19 +344,45 @@ class GpuParquetWriter( } } } - cv - case d: DecimalType if d.precision <= Decimal.MAX_INT_DIGITS => - // There is a bug in Spark that causes a problem if we write Decimals with - // precision < 10 as Decimal64. - // https://issues.apache.org/jira/browse/SPARK-34167 - new GpuColumnVector(d, withResource(cv.getBase()) { v => - v.castTo(DType.create(DType.DTypeEnum.DECIMAL32, -d.scale)) - }) - case _ => cv - } + } + cv.copyToColumnVector() /* the input is unchanged */ + + // Here the value of `outputTimestampType` should be `TIMESTAMP_MICROS` + case _ => cv.copyToColumnVector() /* the input is unchanged */ } - }) + // Decimal types are checked and transformed only for the top level column because we don't + // have access to Spark's data type of the nested column. + case (cv, dtOpt) if dtOpt.isDefined && dtOpt.get == DecimalType => + val d = dtOpt.get.asInstanceOf[DecimalType] + // There is a bug in Spark that causes a problem if we write Decimals with + // precision < 10 as Decimal64. + // https://issues.apache.org/jira/browse/SPARK-34167 + if (d.precision <= Decimal.MAX_INT_DIGITS) { + cv.castTo(DType.create(DType.DTypeEnum.DECIMAL32, -d.scale)) + } else if (d.precision <= Decimal.MAX_LONG_DIGITS) { + cv.castTo(DType.create(DType.DTypeEnum.DECIMAL64, -d.scale)) + } else { + // Here, decimal should be in DECIMAL128 so the input will be unchanged. + cv.copyToColumnVector() + } + } + } + + /** + * Persists a columnar batch. Invoked on the executor side. When writing to dynamically + * partitioned tables, dynamic partition columns are not included in columns to be written. + * NOTE: It is the writer's responsibility to close the batch. + */ + override def write(batch: ColumnarBatch, + statsTrackers: Seq[ColumnarWriteTaskStatsTracker]): Unit = { + val newBatch = withResource(batch) { batch => + val transformedCols = GpuColumnVector.extractColumns(batch).safeMap { cv => + new GpuColumnVector(cv.dataType, deepTransformColumn(cv.getBase, cv.dataType)) + .asInstanceOf[org.apache.spark.sql.vectorized.ColumnVector] + } + new ColumnarBatch(transformedCols) + } super.write(newBatch, statsTrackers) } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala index 4a4af59d78b..5aa6818cc44 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala @@ -416,7 +416,7 @@ case class GpuSortArray(base: Expression, ascendingOrder: Expression) // before sorting. This workaround can be removed after // solving https://github.com/rapidsai/cudf/issues/11630 val normalizedChild = ColumnCastUtil.deepTransform(child) { - case cv if cv.getType == cudf.DType.FLOAT32 || cv.getType == cudf.DType.FLOAT64 => + case (cv, _) if cv.getType == cudf.DType.FLOAT32 || cv.getType == cudf.DType.FLOAT64 => cv.normalizeNANsAndZeros() } withResource(normalizedChild) {normalizedChild => From 853d8a818b810083d8f4693b0d9d3cb444ac2f5a Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Mon, 12 Sep 2022 11:18:15 -0500 Subject: [PATCH 101/190] Fix up `buffer time` for multi-file readers (#6429) * Fix up metrics for multi-file readers Signed-off-by: Alessandro Bellina * WIP adding documentation * Use the percentage of time spent in filter and buffer to handle multiple threads. * Avoid NPE in case of exception by not setting metrics in finally * Fix typo in doc * Move doc line * Clarify comments * Address review comments * Address review comments Signed-off-by: Alessandro Bellina --- docs/tuning-guide.md | 58 ++--- .../spark/rapids/GpuMultiFileReader.scala | 24 +- .../com/nvidia/spark/rapids/GpuOrcScan.scala | 66 +++--- .../nvidia/spark/rapids/GpuParquetScan.scala | 70 +++--- .../rapids/GpuTextBasedPartitionReader.scala | 7 +- .../apache/spark/sql/rapids/GpuAvroScan.scala | 214 +++++++++--------- 6 files changed, 244 insertions(+), 195 deletions(-) diff --git a/docs/tuning-guide.md b/docs/tuning-guide.md index 979083ec10c..d0da4695832 100644 --- a/docs/tuning-guide.md +++ b/docs/tuning-guide.md @@ -294,35 +294,35 @@ If your data has float values but doesn't contain NaNs set [`spark.rapids.sql.ha Custom Spark SQL Metrics are available which can help identify performance bottlenecks in a query. -| Key | Name | Description | -|-------------------|------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| bufferTime | buffer time | Time spent buffering input from file data sources. This buffering time happens on the CPU, typically with no GPU semaphore held. | -| readFsTime | time to read fs data | Time spent actually reading the data and writing it to on-heap memory. This is a part of `bufferTime` | -| writeBufferTime | time to write data to buffer | Time spent moving the on-heap buffered data read from the file system to off-heap memory so the GPU can access it. This is a part of `bufferTime` | -| buildDataSize | build side size | Size in bytes of the build-side of a join. | -| buildTime | build time | Time to load the build-side of a join. | -| collectTime | collect time | For a broadcast the amount of time it took to collect the broadcast data back to the driver before broadcasting it back out. | -| computeAggTime | aggregation time | Time computing an aggregation. | -| concatTime | concat batch time | Time to concatenate batches. Runs on CPU. | -| copyBufferTime | copy buffer time | Time spent on copying upstreaming data into Rapids buffers. | -| filterTime | filter time | Time spent applying filters within other operators, such as joins. | -| gpuDecodeTime | GPU decode time | Time spent on GPU decoding encrypted or compressed data. | -| joinOutputRows | join output rows | The number of rows produced by a join before any filter expression is applied. | -| joinTime | join time | Time doing a join operation. | -| numInputBatches | input columnar batches | Number of columnar batches that the operator received from its child operator(s). | -| numInputRows | input rows | Number of rows that the operator received from its child operator(s). | -| numOutputBatches | output columnar batches | Number of columnar batches that the operator outputs. | -| numOutputRows | output rows | Number of rows that the operator outputs. | -| numPartitions | partitions | Number of output partitions from a file scan or shuffle exchange. | -| opTime | op time | Time that an operator takes, exclusive of the time for executing or fetching results from child operators, and typically outside of the time it takes to acquire the GPU semaphore.
Note: Sometimes contains CPU times, e.g.: concatTime | -| partitionSize | partition data size | Total size in bytes of output partitions. | -| peakDevMemory | peak device memory | Peak GPU memory used during execution of an operator. | -| semaphoreWaitTime | GPU semaphore wait time | Time spent waiting for the GPU semaphore. | -| sortTime | sort time | Time spent in sort operations in GpuSortExec and GpuTopN. | -| spillData | bytes spilled from GPU | Total bytes spilled from GPU. | -| spillDisk | bytes spilled to disk | Total bytes spilled from GPU to disk. | -| spillHost | bytes spilled to host | Total bytes spilled from GPU to host memory. | -| streamTime | stream time | Time spent reading data from a child. This generally happens for the stream side of a hash join or for columnar to row and row to columnar operations. | +| Key | Name | Description | +|-------------------|------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| bufferTime | buffer time | Time spent buffering input from file data sources. This buffering time happens on the CPU, typically with no GPU semaphore held. For Multi-threaded readers, `bufferTime` measures the amount of time we are blocked on while the threads that are buffering are busy. | +| readFsTime | time to read fs data | Time spent actually reading the data and writing it to on-heap memory. This is a part of `bufferTime` | +| writeBufferTime | time to write data to buffer | Time spent moving the on-heap buffered data read from the file system to off-heap memory so the GPU can access it. This is a part of `bufferTime` | +| buildDataSize | build side size | Size in bytes of the build-side of a join. | +| buildTime | build time | Time to load the build-side of a join. | +| collectTime | collect time | For a broadcast the amount of time it took to collect the broadcast data back to the driver before broadcasting it back out. | +| computeAggTime | aggregation time | Time computing an aggregation. | +| concatTime | concat batch time | Time to concatenate batches. Runs on CPU. | +| copyBufferTime | copy buffer time | Time spent on copying upstreaming data into Rapids buffers. | +| filterTime | filter time | Time spent applying filters within other operators, such as joins. | +| gpuDecodeTime | GPU decode time | Time spent on GPU decoding encrypted or compressed data. | +| joinOutputRows | join output rows | The number of rows produced by a join before any filter expression is applied. | +| joinTime | join time | Time doing a join operation. | +| numInputBatches | input columnar batches | Number of columnar batches that the operator received from its child operator(s). | +| numInputRows | input rows | Number of rows that the operator received from its child operator(s). | +| numOutputBatches | output columnar batches | Number of columnar batches that the operator outputs. | +| numOutputRows | output rows | Number of rows that the operator outputs. | +| numPartitions | partitions | Number of output partitions from a file scan or shuffle exchange. | +| opTime | op time | Time that an operator takes, exclusive of the time for executing or fetching results from child operators, and typically outside of the time it takes to acquire the GPU semaphore.
Note: Sometimes contains CPU times, e.g.: concatTime | +| partitionSize | partition data size | Total size in bytes of output partitions. | +| peakDevMemory | peak device memory | Peak GPU memory used during execution of an operator. | +| semaphoreWaitTime | GPU semaphore wait time | Time spent waiting for the GPU semaphore. | +| sortTime | sort time | Time spent in sort operations in GpuSortExec and GpuTopN. | +| spillData | bytes spilled from GPU | Total bytes spilled from GPU. | +| spillDisk | bytes spilled to disk | Total bytes spilled from GPU to disk. | +| spillHost | bytes spilled to host | Total bytes spilled from GPU to host memory. | +| streamTime | stream time | Time spent reading data from a child. This generally happens for the stream side of a hash join or for columnar to row and row to columnar operations. | Not all metrics are enabled by default. The configuration setting `spark.rapids.sql.metrics.level` can be set to `DEBUG`, `MODERATE`, or `ESSENTIAL`, with `MODERATE` being the default value. More information about this diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala index 07d2da7d369..57c21a0fb9a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala @@ -27,7 +27,7 @@ import scala.language.implicitConversions import scala.math.max import ai.rapids.cudf.{ColumnVector, HostMemoryBuffer, NvtxColor, NvtxRange, Table} -import com.nvidia.spark.rapids.GpuMetric.{FILTER_TIME, NUM_OUTPUT_BATCHES, PEAK_DEVICE_MEMORY, SEMAPHORE_WAIT_TIME} +import com.nvidia.spark.rapids.GpuMetric.{BUFFER_TIME, FILTER_TIME, NUM_OUTPUT_BATCHES, PEAK_DEVICE_MEMORY, SEMAPHORE_WAIT_TIME} import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableProducingSeq import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration @@ -58,6 +58,21 @@ trait HostMemoryBuffersWithMetaDataBase { def memBuffersAndSizes: Array[(HostMemoryBuffer, Long)] // Total bytes read def bytesRead: Long + // Percentage of time spent on filtering + private var _filterTimePct: Double = 0L + // Percentage of time spent on buffering + private var _bufferTimePct: Double = 0L + + // Called by parquet/orc/avro scanners to set the amount of time (in nanoseconds) + // that filtering and buffering incurred in one of the scan runners. + def setMetrics(filterTime: Long, bufferTime: Long): Unit = { + val totalTime = filterTime + bufferTime + _filterTimePct = filterTime.toDouble / totalTime + _bufferTimePct = bufferTime.toDouble / totalTime + } + + def getBufferTimePct: Double = _bufferTimePct + def getFilterTimePct: Double = _filterTimePct } // This is a common trait for all kind of file formats @@ -415,9 +430,14 @@ abstract class MultiFileCloudPartitionReaderBase( // clock as we can get right now without further work. val startTime = System.nanoTime() val fileBufsAndMeta = tasks.poll.get() + val blockedTime = System.nanoTime() - startTime metrics.get(FILTER_TIME).foreach { - _ += (System.nanoTime() - startTime) + _ += (blockedTime * fileBufsAndMeta.getFilterTimePct).toLong } + metrics.get(BUFFER_TIME).foreach { + _ += (blockedTime * fileBufsAndMeta.getBufferTimePct).toLong + } + filesToRead -= 1 TrampolineUtil.incBytesRead(inputMetrics, fileBufsAndMeta.bytesRead) InputFileUtils.setInputFileBlock( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index 81b2b2e4655..f022100eb2a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -932,8 +932,7 @@ trait OrcPartitionReaderBase extends OrcCommonFunctions with Logging */ protected def readPartFile(ctx: OrcPartitionReaderContext, stripes: Seq[OrcOutputStripe]): (HostMemoryBuffer, Long) = { - withResource(new NvtxWithMetrics("Buffer file split", NvtxColor.YELLOW, - metrics("bufferTime"))) { _ => + withResource(new NvtxRange("Buffer file split", NvtxColor.YELLOW)) { _ => if (stripes.isEmpty) { return (null, 0L) } @@ -1087,7 +1086,9 @@ class GpuOrcPartitionReader( Some(new ColumnarBatch(nullColumns.toArray, numRows)) } } else { - val (dataBuffer, dataSize) = readPartFile(ctx, currentStripes) + val (dataBuffer, dataSize) = metrics(BUFFER_TIME).ns { + readPartFile(ctx, currentStripes) + } decodeToBatch(dataBuffer, dataSize, ctx.updatedReadSchema, ctx.requestedMapping, isCaseSensitive, Array(partFile)) } @@ -1647,41 +1648,47 @@ class MultiFileCloudOrcPartitionReader( val startingBytesRead = fileSystemBytesRead() val hostBuffers = new ArrayBuffer[(HostMemoryBuffer, Long)] + val filterStartTime = System.nanoTime() val ctx = filterHandler.filterStripes(partFile, dataSchema, readDataSchema, partitionSchema) - try { + val filterTime = System.nanoTime() - filterStartTime + val bufferTimeStart = System.nanoTime() + val result = try { if (ctx == null || ctx.blockIterator.isEmpty) { val bytesRead = fileSystemBytesRead() - startingBytesRead // no blocks so return null buffer and size 0 - return HostMemoryEmptyMetaData(partFile, 0, bytesRead, + HostMemoryEmptyMetaData(partFile, 0, bytesRead, ctx.updatedReadSchema, readDataSchema) - } - blockChunkIter = ctx.blockIterator - if (isDone) { - val bytesRead = fileSystemBytesRead() - startingBytesRead - // got close before finishing - HostMemoryEmptyMetaData(partFile, 0, bytesRead, ctx.updatedReadSchema, readDataSchema) } else { - if (ctx.updatedReadSchema.isEmpty) { + blockChunkIter = ctx.blockIterator + if (isDone) { val bytesRead = fileSystemBytesRead() - startingBytesRead - val numRows = ctx.blockIterator.map(_.infoBuilder.getNumberOfRows).sum.toInt - // overload size to be number of rows with null buffer - HostMemoryEmptyMetaData(partFile, numRows, bytesRead, - ctx.updatedReadSchema, readDataSchema) + // got close before finishing + HostMemoryEmptyMetaData( + partFile, 0, bytesRead, ctx.updatedReadSchema, readDataSchema) } else { - while (blockChunkIter.hasNext) { - val blocksToRead = populateCurrentBlockChunk(blockChunkIter, maxReadBatchSizeRows, - maxReadBatchSizeBytes) - hostBuffers += readPartFile(ctx, blocksToRead) - } - val bytesRead = fileSystemBytesRead() - startingBytesRead - if (isDone) { - // got close before finishing - hostBuffers.foreach(_._1.safeClose()) - HostMemoryEmptyMetaData(partFile, 0, bytesRead, ctx.updatedReadSchema, readDataSchema) + if (ctx.updatedReadSchema.isEmpty) { + val bytesRead = fileSystemBytesRead() - startingBytesRead + val numRows = ctx.blockIterator.map(_.infoBuilder.getNumberOfRows).sum.toInt + // overload size to be number of rows with null buffer + HostMemoryEmptyMetaData(partFile, numRows, bytesRead, + ctx.updatedReadSchema, readDataSchema) } else { - HostMemoryBuffersWithMetaData(partFile, hostBuffers.toArray, bytesRead, - ctx.updatedReadSchema, ctx.requestedMapping) + while (blockChunkIter.hasNext) { + val blocksToRead = populateCurrentBlockChunk(blockChunkIter, maxReadBatchSizeRows, + maxReadBatchSizeBytes) + hostBuffers += readPartFile(ctx, blocksToRead) + } + val bytesRead = fileSystemBytesRead() - startingBytesRead + if (isDone) { + // got close before finishing + hostBuffers.foreach(_._1.safeClose()) + HostMemoryEmptyMetaData( + partFile, 0, bytesRead, ctx.updatedReadSchema, readDataSchema) + } else { + HostMemoryBuffersWithMetaData(partFile, hostBuffers.toArray, bytesRead, + ctx.updatedReadSchema, ctx.requestedMapping) + } } } } @@ -1690,6 +1697,9 @@ class MultiFileCloudOrcPartitionReader( hostBuffers.foreach(_._1.safeClose()) throw e } + val bufferTime = System.nanoTime() - bufferTimeStart + result.setMetrics(filterTime, bufferTime) + result } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index 63c0ffe5345..31f1c300391 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -1328,8 +1328,7 @@ trait ParquetPartitionReaderBase extends Logging with Arm with ScanWithMetrics blocks: Seq[BlockMetaData], clippedSchema: MessageType, filePath: Path): (HostMemoryBuffer, Long) = { - withResource(new NvtxWithMetrics("Parquet buffer file split", NvtxColor.YELLOW, - metrics("bufferTime"))) { _ => + withResource(new NvtxRange("Parquet buffer file split", NvtxColor.YELLOW)) { _ => withResource(filePath.getFileSystem(conf).open(filePath)) { in => val estTotalSize = calculateParquetOutputSize(blocks, clippedSchema, false) closeOnExcept(HostMemoryBuffer.allocate(estTotalSize)) { hmb => @@ -1804,48 +1803,54 @@ class MultiFileCloudParquetPartitionReader( private def doRead(): HostMemoryBuffersWithMetaDataBase = { val startingBytesRead = fileSystemBytesRead() val hostBuffers = new ArrayBuffer[(HostMemoryBuffer, Long)] - try { + var filterTime = 0L + var bufferStartTime = 0L + val result = try { + val filterStartTime = System.nanoTime() val fileBlockMeta = filterFunc(file) + filterTime = System.nanoTime() - filterStartTime + bufferStartTime = System.nanoTime() if (fileBlockMeta.blocks.isEmpty) { val bytesRead = fileSystemBytesRead() - startingBytesRead // no blocks so return null buffer and size 0 - return HostMemoryEmptyMetaData(file, 0, bytesRead, - fileBlockMeta.isCorrectedRebaseMode, fileBlockMeta.isCorrectedInt96RebaseMode, - fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema) - } - blockChunkIter = fileBlockMeta.blocks.iterator.buffered - if (isDone) { - val bytesRead = fileSystemBytesRead() - startingBytesRead - // got close before finishing HostMemoryEmptyMetaData(file, 0, bytesRead, fileBlockMeta.isCorrectedRebaseMode, fileBlockMeta.isCorrectedInt96RebaseMode, fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema) } else { - if (fileBlockMeta.schema.getFieldCount == 0) { + blockChunkIter = fileBlockMeta.blocks.iterator.buffered + if (isDone) { val bytesRead = fileSystemBytesRead() - startingBytesRead - val numRows = fileBlockMeta.blocks.map(_.getRowCount).sum.toInt - // overload size to be number of rows with null buffer - HostMemoryEmptyMetaData(file, numRows, bytesRead, + // got close before finishing + HostMemoryEmptyMetaData(file, 0, bytesRead, fileBlockMeta.isCorrectedRebaseMode, fileBlockMeta.isCorrectedInt96RebaseMode, fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema) } else { - val filePath = new Path(new URI(file.filePath)) - while (blockChunkIter.hasNext) { - val blocksToRead = populateCurrentBlockChunk(blockChunkIter, - maxReadBatchSizeRows, maxReadBatchSizeBytes, fileBlockMeta.readSchema) - hostBuffers += readPartFile(blocksToRead, fileBlockMeta.schema, filePath) - } - val bytesRead = fileSystemBytesRead() - startingBytesRead - if (isDone) { - // got close before finishing - hostBuffers.foreach(_._1.safeClose()) - HostMemoryEmptyMetaData(file, 0, bytesRead, + if (fileBlockMeta.schema.getFieldCount == 0) { + val bytesRead = fileSystemBytesRead() - startingBytesRead + val numRows = fileBlockMeta.blocks.map(_.getRowCount).sum.toInt + // overload size to be number of rows with null buffer + HostMemoryEmptyMetaData(file, numRows, bytesRead, fileBlockMeta.isCorrectedRebaseMode, fileBlockMeta.isCorrectedInt96RebaseMode, fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema) } else { - HostMemoryBuffersWithMetaData(file, hostBuffers.toArray, bytesRead, - fileBlockMeta.isCorrectedRebaseMode, fileBlockMeta.isCorrectedInt96RebaseMode, - fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema) + val filePath = new Path(new URI(file.filePath)) + while (blockChunkIter.hasNext) { + val blocksToRead = populateCurrentBlockChunk(blockChunkIter, + maxReadBatchSizeRows, maxReadBatchSizeBytes, fileBlockMeta.readSchema) + hostBuffers += readPartFile(blocksToRead, fileBlockMeta.schema, filePath) + } + val bytesRead = fileSystemBytesRead() - startingBytesRead + if (isDone) { + // got close before finishing + hostBuffers.foreach(_._1.safeClose()) + HostMemoryEmptyMetaData(file, 0, bytesRead, + fileBlockMeta.isCorrectedRebaseMode, fileBlockMeta.isCorrectedInt96RebaseMode, + fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema) + } else { + HostMemoryBuffersWithMetaData(file, hostBuffers.toArray, bytesRead, + fileBlockMeta.isCorrectedRebaseMode, fileBlockMeta.isCorrectedInt96RebaseMode, + fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema) + } } } } @@ -1854,6 +1859,9 @@ class MultiFileCloudParquetPartitionReader( hostBuffers.foreach(_._1.safeClose()) throw e } + val bufferTime = bufferStartTime - System.nanoTime() + result.setMetrics(filterTime, bufferTime) + result } } @@ -2066,7 +2074,9 @@ class ParquetPartitionReader( if (currentChunkedBlocks.isEmpty) { return None } - val (dataBuffer, dataSize) = readPartFile(currentChunkedBlocks, clippedParquetSchema, filePath) + val (dataBuffer, dataSize) = metrics(BUFFER_TIME).ns { + readPartFile(currentChunkedBlocks, clippedParquetSchema, filePath) + } try { if (dataSize == 0) { None diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala index 8f98fef9c8f..64b45bca054 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala @@ -102,8 +102,7 @@ abstract class GpuTextBasedPartitionReader( } private def readPartFile(): (HostMemoryBuffer, Long) = { - withResource(new NvtxWithMetrics("Buffer file split", NvtxColor.YELLOW, - metrics("bufferTime"))) { _ => + withResource(new NvtxRange("Buffer file split", NvtxColor.YELLOW)) { _ => isFirstChunkForIterator = false val separator = lineSeparatorInRead.getOrElse(Array('\n'.toByte)) var succeeded = false @@ -159,7 +158,9 @@ abstract class GpuTextBasedPartitionReader( } private def readToTable(isFirstChunk: Boolean): Option[Table] = { - val (dataBuffer, dataSize) = readPartFile() + val (dataBuffer, dataSize) = metrics(BUFFER_TIME).ns { + readPartFile() + } try { if (dataSize == 0) { None diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala index d62e1f488f1..309ee6ee15e 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala @@ -28,7 +28,7 @@ import scala.math.max import ai.rapids.cudf.{AvroOptions => CudfAvroOptions, HostMemoryBuffer, NvtxColor, NvtxRange, Table} import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetric.{FILTER_TIME, GPU_DECODE_TIME, NUM_OUTPUT_BATCHES, PEAK_DEVICE_MEMORY, READ_FS_TIME, SEMAPHORE_WAIT_TIME, WRITE_BUFFER_TIME} +import com.nvidia.spark.rapids.GpuMetric.{BUFFER_TIME, FILTER_TIME, GPU_DECODE_TIME, NUM_OUTPUT_BATCHES, PEAK_DEVICE_MEMORY, READ_FS_TIME, SEMAPHORE_WAIT_TIME, WRITE_BUFFER_TIME} import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.shims.ShimFilePartitionReaderFactory import org.apache.avro.Schema @@ -408,8 +408,7 @@ trait GpuAvroReaderBase extends Arm with Logging { self: FilePartitionReaderBase blocks: Seq[BlockInfo], headerSize: Long, conf: Configuration): (HostMemoryBuffer, Long) = { - withResource(new NvtxWithMetrics("Avro buffer file split", NvtxColor.YELLOW, - metrics("bufferTime"))) { _ => + withResource(new NvtxRange("Avro buffer file split", NvtxColor.YELLOW)) { _ => if (blocks.isEmpty) { // No need to check the header here since it can not be null when blocks is not empty. return (null, 0L) @@ -568,8 +567,10 @@ class GpuAvroPartitionReader( if (currentChunkedBlocks.isEmpty) { None } else { - val (dataBuffer, dataSize) = readPartFile(partFilePath, currentChunkedBlocks, - blockMeta.headerSize, conf) + val (dataBuffer, dataSize) = metrics(BUFFER_TIME).ns { + readPartFile(partFilePath, currentChunkedBlocks, + blockMeta.headerSize, conf) + } sendToGpu(dataBuffer, dataSize, Array(partFile)) } } @@ -713,111 +714,118 @@ class GpuMultiFileCloudAvroPartitionReader( * - post processing */ private def doRead(): HostMemoryBuffersWithMetaDataBase = { + val bufferStartTime = System.nanoTime() val startingBytesRead = fileSystemBytesRead() - withResource(AvroFileReader.openDataReader(partFile.filePath, config)) { reader => - // Go to the start of the first block after the start position - reader.sync(partFile.start) - if (!reader.hasNextBlock || isDone) { - // no data or got close before finishing, return null buffer and zero size - return createBufferAndMeta(Array((null, 0)), startingBytesRead) - } - val hostBuffers = new ArrayBuffer[(HostMemoryBuffer, Long)] - try { - val headerSize = reader.headerSize - var isBlockSizeEstimated = false - var estBlocksSize = 0L - var totalRowsNum = 0 - var curBlock: MutableBlockInfo = null - while (reader.hasNextBlock && !reader.pastSync(stopPosition)) { - // Get the block metadata first - curBlock = reader.peekBlock(curBlock) - if (!isBlockSizeEstimated) { - // Initialize the estimated block total size. - // The AVRO file has no special section for block metadata, and collecting the - // block meta through the file is quite expensive for files in cloud. So we do - // not know the target buffer size ahead. Then we have to do an estimation. - // "the estimated total block size = partFile.length + additional space" - // Letting "additional space = one block length * 1.2" is because we may - // move the start and stop positions when reading this split to keep the - // integrity of edge blocks. - // One worst case is the stop position is one byte after a block start, - // then we need to read the whole block into the current batch. And this block - // may be larger than the first block. So we preserve an additional space - // whose size is 'one block length * 1.2' to try to avoid reading it to a new - // batch. - estBlocksSize = partFile.length + (curBlock.blockSize * 1.2F).toLong - isBlockSizeEstimated = true - } + val result = + withResource(AvroFileReader.openDataReader(partFile.filePath, config)) { reader => + // Go to the start of the first block after the start position + reader.sync(partFile.start) + if (!reader.hasNextBlock || isDone) { + // no data or got close before finishing, return null buffer and zero size + createBufferAndMeta(Array((null, 0)), startingBytesRead) + } else { + val hostBuffers = new ArrayBuffer[(HostMemoryBuffer, Long)] + try { + val headerSize = reader.headerSize + var isBlockSizeEstimated = false + var estBlocksSize = 0L + var totalRowsNum = 0 + var curBlock: MutableBlockInfo = null + while (reader.hasNextBlock && !reader.pastSync(stopPosition)) { + // Get the block metadata first + curBlock = reader.peekBlock(curBlock) + if (!isBlockSizeEstimated) { + // Initialize the estimated block total size. + // The AVRO file has no special section for block metadata, and collecting the + // block meta through the file is quite expensive for files in cloud. So we do + // not know the target buffer size ahead. Then we have to do an estimation. + // "the estimated total block size = partFile.length + additional space" + // Letting "additional space = one block length * 1.2" is because we may + // move the start and stop positions when reading this split to keep the + // integrity of edge blocks. + // One worst case is the stop position is one byte after a block start, + // then we need to read the whole block into the current batch. And this block + // may be larger than the first block. So we preserve an additional space + // whose size is 'one block length * 1.2' to try to avoid reading it to a new + // batch. + estBlocksSize = partFile.length + (curBlock.blockSize * 1.2F).toLong + isBlockSizeEstimated = true + } - var estSizeToRead = if (estBlocksSize > maxReadBatchSizeBytes) { - maxReadBatchSizeBytes - } else if (estBlocksSize < curBlock.blockSize) { - // This may happen only for the last block. - logInfo("Less buffer is estimated, read the last block into a new batch.") - curBlock.blockSize - } else { - estBlocksSize - } - val optHmb = if (readDataSchema.nonEmpty) { - Some(HostMemoryBuffer.allocate(headerSize + estSizeToRead)) - } else None - // Allocate the buffer for the header and blocks for a batch - closeOnExcept(optHmb) { _ => - val optOut = optHmb.map { hmb => - val out = new HostMemoryOutputStream(hmb) - // Write the header to the output stream - AvroFileWriter(out).writeHeader(reader.header) - out - } - // Read the block data to the output stream - var batchRowsNum: Int = 0 - var batchSize: Long = 0 - var hasNextBlock = true - do { - if (optOut.nonEmpty) { - reader.readNextRawBlock(optOut.get) + var estSizeToRead = if (estBlocksSize > maxReadBatchSizeBytes) { + maxReadBatchSizeBytes + } else if (estBlocksSize < curBlock.blockSize) { + // This may happen only for the last block. + logInfo("Less buffer is estimated, read the last block into a new batch.") + curBlock.blockSize } else { - // skip the current block - reader.skipCurrentBlock() + estBlocksSize } - batchRowsNum += curBlock.count.toInt - estSizeToRead -= curBlock.blockSize - batchSize += curBlock.blockSize - // Continue reading the next block into the current batch when - // - the next block exists, and - // - the remaining buffer is enough to hold the next block, and - // - the batch rows number does not go beyond the upper limit. - hasNextBlock = reader.hasNextBlock && !reader.pastSync(stopPosition) - if (hasNextBlock) { - curBlock = reader.peekBlock(curBlock) + val optHmb = if (readDataSchema.nonEmpty) { + Some(HostMemoryBuffer.allocate(headerSize + estSizeToRead)) + } else None + // Allocate the buffer for the header and blocks for a batch + closeOnExcept(optHmb) { _ => + val optOut = optHmb.map { hmb => + val out = new HostMemoryOutputStream(hmb) + // Write the header to the output stream + AvroFileWriter(out).writeHeader(reader.header) + out + } + // Read the block data to the output stream + var batchRowsNum: Int = 0 + var batchSize: Long = 0 + var hasNextBlock = true + do { + if (optOut.nonEmpty) { + reader.readNextRawBlock(optOut.get) + } else { + // skip the current block + reader.skipCurrentBlock() + } + batchRowsNum += curBlock.count.toInt + estSizeToRead -= curBlock.blockSize + batchSize += curBlock.blockSize + // Continue reading the next block into the current batch when + // - the next block exists, and + // - the remaining buffer is enough to hold the next block, and + // - the batch rows number does not go beyond the upper limit. + hasNextBlock = reader.hasNextBlock && !reader.pastSync(stopPosition) + if (hasNextBlock) { + curBlock = reader.peekBlock(curBlock) + } + } while (hasNextBlock && curBlock.blockSize <= estSizeToRead && + batchRowsNum <= maxReadBatchSizeRows) + + // One batch is done + optOut.foreach(out => hostBuffers += ((optHmb.get, out.getPos))) + totalRowsNum += batchRowsNum + estBlocksSize -= batchSize } - } while (hasNextBlock && curBlock.blockSize <= estSizeToRead && - batchRowsNum <= maxReadBatchSizeRows) - - // One batch is done - optOut.foreach(out => hostBuffers += ((optHmb.get, out.getPos))) - totalRowsNum += batchRowsNum - estBlocksSize -= batchSize + } // end of while + + val bufAndSize: Array[(HostMemoryBuffer, Long)] = if (readDataSchema.isEmpty) { + hostBuffers.foreach(_._1.safeClose(new Exception)) + Array((null, totalRowsNum)) + } else if (isDone) { + // got close before finishing, return null buffer and zero size + hostBuffers.foreach(_._1.safeClose(new Exception)) + Array((null, 0)) + } else { + hostBuffers.toArray + } + createBufferAndMeta(bufAndSize, startingBytesRead) + } catch { + case e: Throwable => + hostBuffers.foreach(_._1.safeClose(e)) + throw e } - } // end of while - - val bufAndSize: Array[(HostMemoryBuffer, Long)] = if (readDataSchema.isEmpty) { - hostBuffers.foreach(_._1.safeClose(new Exception)) - Array((null, totalRowsNum)) - } else if (isDone) { - // got close before finishing, return null buffer and zero size - hostBuffers.foreach(_._1.safeClose(new Exception)) - Array((null, 0)) - } else { - hostBuffers.toArray } - createBufferAndMeta(bufAndSize, startingBytesRead) - } catch { - case e: Throwable => - hostBuffers.foreach(_._1.safeClose(e)) - throw e - } - } // end of withResource(reader) + } // end of withResource(reader) + val bufferTime = System.nanoTime() - bufferStartTime + // multi-file avro scanner does not filter and then buffer, it just buffers + result.setMetrics(0, bufferTime) + result } // end of doRead } // end of Class ReadBatchRunner From 8adf956658e40685c2a96c85fbcf42501cae8721 Mon Sep 17 00:00:00 2001 From: Yuan Jiang <47068112+cindyyuanjiang@users.noreply.github.com> Date: Mon, 12 Sep 2022 18:32:47 -0700 Subject: [PATCH 102/190] Append new authorized user to blossom-ci whitelist [skip ci] (#6547) * Append my id to blossom-ci whitelist * remove username Signed-off-by: cindyyuanjiang * append new authorized to blossom-ci whitelist Signed-off-by: cindyyuanjiang * need to renew sign-off info Signed-off-by: Cindy Jiang * append new authorized to blossom-ci whitelist Signed-off-by: Cindy Jiang Signed-off-by: cindyyuanjiang Signed-off-by: Cindy Jiang Co-authored-by: Cindy Jiang --- .github/workflows/blossom-ci.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/blossom-ci.yml b/.github/workflows/blossom-ci.yml index 43d1b6160c7..fdfea58f4da 100644 --- a/.github/workflows/blossom-ci.yml +++ b/.github/workflows/blossom-ci.yml @@ -68,6 +68,7 @@ jobs: sinkinben,\ thirtiseven,\ YanxuanLiu,\ + cindyyuanjiang,\ ', format('{0},', github.actor)) && github.event.comment.body == 'build' steps: - name: Check if comment is issued by authorized person From fb86a1a8042f241b31d29f2e48ef73820be734d7 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 13 Sep 2022 08:09:57 -0500 Subject: [PATCH 103/190] Add another alluxio path replacement algorithm (#6541) * Fallback to CPU for delta log checkpoint file reads Signed-off-by: Thomas Graves * fix stylie * Handle xxx.checkpoint.yyy.parquet file names * Fix to use or instead of and * cleanup Signed-off-by: Thomas Graves * wrapping * Change alluxio logs to be debug Signed-off-by: Thomas Graves * Do the parquet filter with the coalescing reader in paralell Signed-off-by: Thomas Graves * fix filter param * Add filter time to datasource v2 * cleanup code and commonize * cleanup * change metric level * removing log warning messages * Add tests and logging * fix syntax * STart to change the way alluxio replaces urls Signed-off-by: Thomas Graves * continue adding support * Add in file source and change methods hardcoded * compilation fixes * use runtimeconfig * revert * Add conf for algo * put code back in * add debug logs * Rework to check for the algorithms and add option to get new filestatus * remove reuse option * fix merge conflict * code cleanup * remove debug log * Shim FileStatus * fix style * fix build for databricks * fix relation replacement * add config doc * update comment Signed-off-by: Thomas Graves --- docs/configs.md | 1 + .../spark/rapids/shims/Spark31XShims.scala | 33 +++++-- .../spark/rapids/shims/Spark31XdbShims.scala | 34 +++++-- .../rapids/shims/Spark320PlusNonDBShims.scala | 21 +++- .../rapids/shims/Spark320PlusShims.scala | 15 +-- .../spark/rapids/shims/SparkShims.scala | 34 +++++-- .../nvidia/spark/rapids/AlluxioUtils.scala | 95 ++++++++++++++----- .../com/nvidia/spark/rapids/RapidsConf.scala | 20 ++++ .../com/nvidia/spark/rapids/SparkShims.scala | 7 +- .../sql/rapids/GpuFileSourceScanExec.scala | 26 ++++- 10 files changed, 230 insertions(+), 56 deletions(-) diff --git a/docs/configs.md b/docs/configs.md index 032c6552fe1..8fe82cbb26d 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -33,6 +33,7 @@ Name | Description | Default Value spark.rapids.alluxio.bucket.regex|A regex to decide which bucket should be auto-mounted to Alluxio. E.g. when setting as "^s3://bucket.*", the bucket which starts with "s3://bucket" will be mounted to Alluxio and the path "s3://bucket-foo/a.csv" will be replaced to "alluxio://0.1.2.3:19998/bucket-foo/a.csv". It's only valid when setting spark.rapids.alluxio.automount.enabled=true. The default value matches all the buckets in "s3://" or "s3a://" scheme.|^s3a{0,1}://.* spark.rapids.alluxio.cmd|Provide the Alluxio command, which is used to mount or get information. The default value is "su,ubuntu,-c,/opt/alluxio-2.8.0/bin/alluxio", it means: run Process(Seq("su", "ubuntu", "-c", "/opt/alluxio-2.8.0/bin/alluxio fs mount --readonly /bucket-foo s3://bucket-foo")), to mount s3://bucket-foo to /bucket-foo. the delimiter "," is used to convert to Seq[String] when you need to use a special user to run the mount command.|List(su, ubuntu, -c, /opt/alluxio-2.8.0/bin/alluxio) spark.rapids.alluxio.pathsToReplace|List of paths to be replaced with corresponding Alluxio scheme. E.g. when configure is set to "s3://foo->alluxio://0.1.2.3:19998/foo,gs://bar->alluxio://0.1.2.3:19998/bar", it means: "s3://foo/a.csv" will be replaced to "alluxio://0.1.2.3:19998/foo/a.csv" and "gs://bar/b.csv" will be replaced to "alluxio://0.1.2.3:19998/bar/b.csv". To use this config, you have to mount the buckets to Alluxio by yourself. If you set this config, spark.rapids.alluxio.automount.enabled won't be valid.|None +spark.rapids.alluxio.replacement.algo|The algorithm used when replacing the UFS path with the Alluxio path. CONVERT_TIME and SELECTION_TIME are the valid options. CONVERT_TIME indicates that we do it when we convert it to a GPU file read, this has extra overhead of creating an entirely new file index, which requires listing the files and getting all new file info from Alluxio. SELECTION_TIME indicates we do it when the file reader is selecting the partitions to process and just replaces the path without fetching the file information again, this is faster but doesn't update locality information if that were to work with Alluxio.|SELECTION_TIME spark.rapids.cloudSchemes|Comma separated list of additional URI schemes that are to be considered cloud based filesystems. Schemes already included: abfs, abfss, dbfs, gs, s3, s3a, s3n, wasbs. Cloud based stores generally would be total separate from the executors and likely have a higher I/O read cost. Many times the cloud filesystems also get better throughput when you have multiple readers in parallel. This is used with spark.rapids.sql.format.parquet.reader.type|None spark.rapids.gpu.resourceName|The name of the Spark resource that represents a GPU that you want the plugin to use if using custom resources with Spark.|gpu spark.rapids.memory.gpu.allocFraction|The fraction of available (free) GPU memory that should be allocated for pooled memory. This must be less than or equal to the maximum limit configured via spark.rapids.memory.gpu.maxAllocFraction, and greater than or equal to the minimum limit configured via spark.rapids.memory.gpu.minAllocFraction.|1.0 diff --git a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/Spark31XShims.scala b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/Spark31XShims.scala index 648a498893e..a65074bfb6e 100644 --- a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/Spark31XShims.scala +++ b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/Spark31XShims.scala @@ -19,7 +19,7 @@ package com.nvidia.spark.rapids.shims import scala.collection.mutable.ListBuffer import com.nvidia.spark.rapids._ -import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -71,6 +71,23 @@ abstract class Spark31XShims extends SparkShims with Spark31Xuntil33XShims with fileIndex.allFiles() } + override def alluxioReplacePathsPartitionDirectory( + pd: PartitionDirectory, + replaceFunc: Option[Path => Path]): (Seq[FileStatus], PartitionDirectory) = { + val updatedFileStatus = pd.files.map { f => + val replaced = replaceFunc.get(f.getPath) + // Alluxio caches the entire file, so the size should be the same. + // Just hardcode block replication to 1 since we don't know what it really + // is in Alluxio and its not used by splits. The modification time shouldn't be + // affected by Alluxio. Blocksize is also not used. Note that we will not + // get new block locations with this so if Alluxio would return new ones + // this isn't going to get them. From my current experiments, Alluxio is not + // returning the block locations of the cached blocks anyway. + new FileStatus(f.getLen, f.isDirectory, 1, f.getBlockSize, f.getModificationTime, replaced) + } + (updatedFileStatus, PartitionDirectory(pd.values, updatedFileStatus)) + } + def broadcastModeTransform(mode: BroadcastMode, rows: Array[InternalRow]): Any = mode.transform(rows) @@ -351,11 +368,15 @@ abstract class Spark31XShims extends SparkShims with Spark31Xuntil33XShims with val sparkSession = wrapped.relation.sparkSession val options = wrapped.relation.options - val location = AlluxioUtils.replacePathIfNeeded( - conf, - wrapped.relation, - partitionFilters, - wrapped.dataFilters) + val location = if (conf.isAlluxioReplacementAlgoConvertTime) { + AlluxioUtils.replacePathIfNeeded( + conf, + wrapped.relation, + partitionFilters, + wrapped.dataFilters) + } else { + wrapped.relation.location + } val newRelation = HadoopFsRelation( location, diff --git a/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/Spark31XdbShims.scala b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/Spark31XdbShims.scala index b1328b2502b..ec66bb6f74d 100644 --- a/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/Spark31XdbShims.scala +++ b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/Spark31XdbShims.scala @@ -18,7 +18,7 @@ package com.nvidia.spark.rapids.shims import com.databricks.sql.execution.window.RunningWindowFunctionExec import com.nvidia.spark.rapids._ -import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.TaskContext import org.apache.spark.internal.Logging @@ -246,12 +246,15 @@ abstract class Spark31XdbShims extends Spark31XdbShimsBase with Logging { val sparkSession = wrapped.relation.sparkSession val options = wrapped.relation.options - val location = AlluxioUtils.replacePathIfNeeded( - conf, - wrapped.relation, - partitionFilters, - wrapped.dataFilters) - + val location = if (conf.isAlluxioReplacementAlgoConvertTime) { + AlluxioUtils.replacePathIfNeeded( + conf, + wrapped.relation, + partitionFilters, + wrapped.dataFilters) + } else { + wrapped.relation.location + } val newRelation = HadoopFsRelation( location, wrapped.relation.partitionSchema, @@ -316,6 +319,23 @@ abstract class Spark31XdbShims extends Spark31XdbShimsBase with Logging { fileCatalog.allFiles().map(_.toFileStatus) } + override def alluxioReplacePathsPartitionDirectory( + pd: PartitionDirectory, + replaceFunc: Option[Path => Path]): (Seq[FileStatus], PartitionDirectory) = { + val updatedFileStatus = pd.files.map { f => + val replaced = replaceFunc.get(f.getPath) + // Alluxio caches the entire file, so the size should be the same. + // Just hardcode block replication to 1 since we don't know what it really + // is in Alluxio and its not used by splits. The modification time shouldn't be + // affected by Alluxio. Blocksize is also not used. Note that we will not + // get new block locations with this so if Alluxio would return new ones + // this isn't going to get them. From my current experiments, Alluxio is not + // returning the block locations of the cached blocks anyway. + new FileStatus(f.length, f.isDir, 1, f.blockSize, f.modificationTime, replaced) + } + (updatedFileStatus, PartitionDirectory(pd.values, updatedFileStatus.toArray)) + } + override def isEmptyRelation(relation: Any): Boolean = false override def tryTransformIfEmptyRelation(mode: BroadcastMode): Option[Any] = None diff --git a/sql-plugin/src/main/320+-nondb/scala/com/nvidia/spark/rapids/shims/Spark320PlusNonDBShims.scala b/sql-plugin/src/main/320+-nondb/scala/com/nvidia/spark/rapids/shims/Spark320PlusNonDBShims.scala index 5ce350fa0e8..5691dab603d 100644 --- a/sql-plugin/src/main/320+-nondb/scala/com/nvidia/spark/rapids/shims/Spark320PlusNonDBShims.scala +++ b/sql-plugin/src/main/320+-nondb/scala/com/nvidia/spark/rapids/shims/Spark320PlusNonDBShims.scala @@ -16,14 +16,14 @@ package com.nvidia.spark.rapids.shims import com.nvidia.spark.rapids.SparkShims -import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.NamedExpression import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.adaptive.{BroadcastQueryStageExec, ShuffleQueryStageExec} -import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.{PartitionDirectory, PartitioningAwareFileIndex} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.python.WindowInPandasExec @@ -44,6 +44,23 @@ trait Spark320PlusNonDBShims extends SparkShims { fileIndex.allFiles() } + override def alluxioReplacePathsPartitionDirectory( + pd: PartitionDirectory, + replaceFunc: Option[Path => Path]): (Seq[FileStatus], PartitionDirectory) = { + val updatedFileStatus = pd.files.map { f => + val replaced = replaceFunc.get(f.getPath) + // Alluxio caches the entire file, so the size should be the same. + // Just hardcode block replication to 1 since we don't know what it really + // is in Alluxio and its not used by splits. The modification time shouldn't be + // affected by Alluxio. Blocksize is also not used. Note that we will not + // get new block locations with this so if Alluxio would return new ones + // this isn't going to get them. From my current experiments, Alluxio is not + // returning the block locations of the cached blocks anyway. + new FileStatus(f.getLen, f.isDirectory, 1, f.getBlockSize, f.getModificationTime, replaced) + } + (updatedFileStatus, PartitionDirectory(pd.values, updatedFileStatus)) + } + def getWindowExpressions(winPy: WindowInPandasExec): Seq[NamedExpression] = winPy.windowExpression /** diff --git a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala index 90137bdacf6..e1371d155a1 100644 --- a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala +++ b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala @@ -382,12 +382,15 @@ trait Spark320PlusShims extends SparkShims with RebaseShims with Logging { val sparkSession = wrapped.relation.sparkSession val options = wrapped.relation.options - val location = AlluxioUtils.replacePathIfNeeded( - conf, - wrapped.relation, - partitionFilters, - wrapped.dataFilters) - + val location = if (conf.isAlluxioReplacementAlgoConvertTime) { + AlluxioUtils.replacePathIfNeeded( + conf, + wrapped.relation, + partitionFilters, + wrapped.dataFilters) + } else { + wrapped.relation.location + } val newRelation = HadoopFsRelation( location, wrapped.relation.partitionSchema, diff --git a/sql-plugin/src/main/321db/scala/com/nvidia/spark/rapids/shims/SparkShims.scala b/sql-plugin/src/main/321db/scala/com/nvidia/spark/rapids/shims/SparkShims.scala index 5db7e5617e4..763a6f8ad58 100644 --- a/sql-plugin/src/main/321db/scala/com/nvidia/spark/rapids/shims/SparkShims.scala +++ b/sql-plugin/src/main/321db/scala/com/nvidia/spark/rapids/shims/SparkShims.scala @@ -18,7 +18,7 @@ package com.nvidia.spark.rapids.shims import com.databricks.sql.execution.window.RunningWindowFunctionExec import com.nvidia.spark.rapids._ -import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.memory.TaskMemoryManager @@ -72,6 +72,23 @@ object SparkShimImpl extends Spark321PlusShims with Spark320until340Shims { fileCatalog.allFiles().map(_.toFileStatus) } + override def alluxioReplacePathsPartitionDirectory( + pd: PartitionDirectory, + replaceFunc: Option[Path => Path]): (Seq[FileStatus], PartitionDirectory) = { + val updatedFileStatus = pd.files.map { f => + val replaced = replaceFunc.get(f.getPath) + // Alluxio caches the entire file, so the size should be the same. + // Just hardcode block replication to 1 since we don't know what it really + // is in Alluxio and its not used by splits. The modification time shouldn't be + // affected by Alluxio. Blocksize is also not used. Note that we will not + // get new block locations with this so if Alluxio would return new ones + // this isn't going to get them. From my current experiments, Alluxio is not + // returning the block locations of the cached blocks anyway. + new FileStatus(f.length, f.isDir, 1, f.blockSize, f.modificationTime, replaced) + } + (updatedFileStatus, PartitionDirectory(pd.values, updatedFileStatus.toArray)) + } + override def neverReplaceShowCurrentNamespaceCommand: ExecRule[_ <: SparkPlan] = null override def getWindowExpressions(winPy: WindowInPandasExec): Seq[NamedExpression] = @@ -139,12 +156,15 @@ object SparkShimImpl extends Spark321PlusShims with Spark320until340Shims { val sparkSession = wrapped.relation.sparkSession val options = wrapped.relation.options - val location = AlluxioUtils.replacePathIfNeeded( - conf, - wrapped.relation, - partitionFilters, - wrapped.dataFilters) - + val location = if (conf.isAlluxioReplacementAlgoConvertTime) { + AlluxioUtils.replacePathIfNeeded( + conf, + wrapped.relation, + partitionFilters, + wrapped.dataFilters) + } else { + wrapped.relation.location + } val newRelation = HadoopFsRelation( location, wrapped.relation.partitionSchema, diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala index 33851dffae8..fd268eacd39 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala @@ -22,22 +22,26 @@ import java.util.Properties import scala.io.{BufferedSource, Source} import scala.sys.process.{Process, ProcessLogger} +import com.nvidia.spark.rapids.shims.SparkShimImpl +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.RuntimeConfig import org.apache.spark.sql.catalyst.expressions.{Expression, PlanExpression} -import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, FileIndex, HadoopFsRelation, InMemoryFileIndex, PartitioningAwareFileIndex, PartitionSpec} +import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, FileIndex, HadoopFsRelation, InMemoryFileIndex, PartitionDirectory, PartitioningAwareFileIndex, PartitionSpec} import org.apache.spark.sql.execution.datasources.rapids.GpuPartitioningUtils object AlluxioUtils extends Logging { private val checkedAlluxioPath = scala.collection.mutable.HashSet[String]() - private def checkAlluxioMounted(sparkSession: SparkSession, alluxio_path: String): Unit = { + private def checkAlluxioMounted( + hadoopConfiguration: Configuration, + alluxio_path: String): Unit = { this.synchronized { if (!checkedAlluxioPath.contains(alluxio_path)) { val path = new Path(alluxio_path) - val fs = path.getFileSystem(sparkSession.sparkContext.hadoopConfiguration) + val fs = path.getFileSystem(hadoopConfiguration) if (!fs.exists(path)) { throw new FileNotFoundException( s"Alluxio path $alluxio_path does not exist, maybe forgot to mount it") @@ -151,9 +155,11 @@ object AlluxioUtils extends Logging { // path is like "s3://foo/test...", it mounts bucket "foo" by calling the alluxio CLI // And we'll append --option to set access_key and secret_key if existing. // Suppose the key doesn't exist when using like Databricks's instance profile - private def autoMountBucket(scheme: String, bucket: String, - access_key: Option[String], - secret_key: Option[String]): Unit = { + private def autoMountBucket( + scheme: String, + bucket: String, + access_key: Option[String], + secret_key: Option[String]): Unit = { // to match the output of alluxio fs mount, append / to remote_path // and add / before bucket name for absolute path in Alluxio val remote_path = scheme + "://" + bucket + "/" @@ -186,16 +192,18 @@ object AlluxioUtils extends Logging { // first try to get fs.s3a.access.key from spark config // second try to get from environment variables - private def getKeyAndSecret(relation: HadoopFsRelation) : (Option[String], Option[String]) = { + private def getKeyAndSecret( + hadoopConfiguration: Configuration, + runtimeConf: RuntimeConfig) : (Option[String], Option[String]) = { val hadoopAccessKey = - relation.sparkSession.sparkContext.hadoopConfiguration.get("fs.s3a.access.key") + hadoopConfiguration.get("fs.s3a.access.key") val hadoopSecretKey = - relation.sparkSession.sparkContext.hadoopConfiguration.get("fs.s3a.secret.key") + hadoopConfiguration.get("fs.s3a.secret.key") if (hadoopAccessKey != null && hadoopSecretKey != null) { (Some(hadoopAccessKey), Some(hadoopSecretKey)) } else { - val accessKey = relation.sparkSession.conf.getOption("spark.hadoop.fs.s3a.access.key") - val secretKey = relation.sparkSession.conf.getOption("spark.hadoop.fs.s3a.secret.key") + val accessKey = runtimeConf.getOption("spark.hadoop.fs.s3a.access.key") + val secretKey = runtimeConf.getOption("spark.hadoop.fs.s3a.secret.key") if (accessKey.isDefined && secretKey.isDefined) { (accessKey, secretKey) } else { @@ -206,8 +214,8 @@ object AlluxioUtils extends Logging { } } - private def genFuncForPathReplacement(replaceMapOption: Option[Map[String, String]] - ) : Option[Path => Path] = { + private def genFuncForPathReplacement( + replaceMapOption: Option[Map[String, String]]): Option[Path => Path] = { if (replaceMapOption.isDefined) { Some((f: Path) => { val pathStr = f.toString @@ -228,13 +236,16 @@ object AlluxioUtils extends Logging { } } - private def genFuncForAutoMountReplacement(conf: RapidsConf, relation: HadoopFsRelation, - alluxioBucketRegex: String) : Option[Path => Path] = { + private def genFuncForAutoMountReplacement( + conf: RapidsConf, + runtimeConf: RuntimeConfig, + hadoopConf: Configuration, + alluxioBucketRegex: String) : Option[Path => Path] = { Some((f: Path) => { val pathStr = f.toString if (pathStr.matches(alluxioBucketRegex)) { initAlluxioInfo(conf) - val (access_key, secret_key) = getKeyAndSecret(relation) + val (access_key, secret_key) = getKeyAndSecret(hadoopConf, runtimeConf) val (scheme, bucket) = getSchemeAndBucketFromPath(pathStr) autoMountBucket(scheme, bucket, access_key, secret_key) @@ -250,12 +261,10 @@ object AlluxioUtils extends Logging { }) } - def replacePathIfNeeded( + private def getReplacementOptions( conf: RapidsConf, - relation: HadoopFsRelation, - partitionFilters: Seq[Expression], - dataFilters: Seq[Expression]): FileIndex = { - + runtimeConf: RuntimeConfig, + hadoopConf: Configuration): (Option[Path => Path], Option[Map[String, String]]) = { val alluxioPathsReplace: Option[Seq[String]] = conf.getAlluxioPathsToReplace val alluxioAutoMountEnabled = conf.getAlluxioAutoMountEnabled val alluxioBucketRegex: String = conf.getAlluxioBucketRegex @@ -283,10 +292,48 @@ object AlluxioUtils extends Logging { val replaceFunc = if (replaceMapOption.isDefined) { genFuncForPathReplacement(replaceMapOption) } else if (alluxioAutoMountEnabled) { - genFuncForAutoMountReplacement(conf, relation, alluxioBucketRegex) + genFuncForAutoMountReplacement(conf, runtimeConf, hadoopConf, alluxioBucketRegex) } else { None } + (replaceFunc, replaceMapOption) + } + + // This function just replaces the path in the PartitionDirectory files + // and does not do a new file listing after replacing with alluxio://. + // This saves time but would not update block locations or other metadata. + def replacePathInPDIfNeeded( + conf: RapidsConf, + pd: PartitionDirectory, + hadoopConf: Configuration, + runtimeConf: RuntimeConfig): PartitionDirectory = { + val (replaceFunc, replaceMapOption) = getReplacementOptions(conf, runtimeConf, hadoopConf) + if (replaceFunc.isDefined) { + val (alluxPaths, updatedPD) = + SparkShimImpl.alluxioReplacePathsPartitionDirectory(pd, replaceFunc) + // check the alluxio paths in root paths exist or not + // throw out an exception to stop the job when any of them is not mounted + if (replaceMapOption.isDefined) { + alluxPaths.map(_.getPath).foreach { rootPath => + replaceMapOption.get.values.find(value => rootPath.toString.startsWith(value)). + foreach(matched => + checkAlluxioMounted(hadoopConf, matched)) + } + } + updatedPD + } else { + pd + } + } + + def replacePathIfNeeded( + conf: RapidsConf, + relation: HadoopFsRelation, + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression]): FileIndex = { + val hadoopConf = relation.sparkSession.sparkContext.hadoopConfiguration + val runtimeConf = relation.sparkSession.conf + val (replaceFunc, replaceMapOption) = getReplacementOptions(conf, runtimeConf, hadoopConf) if (replaceFunc.isDefined) { def replacePathsInPartitionSpec(spec: PartitionSpec): PartitionSpec = { @@ -349,7 +396,7 @@ object AlluxioUtils extends Logging { if (replaceMapOption.isDefined) { rootPaths.foreach { rootPath => replaceMapOption.get.values.find(value => rootPath.toString.startsWith(value)). - foreach(matched => checkAlluxioMounted(relation.sparkSession, matched)) + foreach(matched => checkAlluxioMounted(hadoopConf, matched)) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 717446d6352..3983fe1868b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -1377,6 +1377,18 @@ object RapidsConf { .toSequence .createWithDefault(Seq("su", "ubuntu", "-c", "/opt/alluxio-2.8.0/bin/alluxio")) + val ALLUXIO_REPLACEMENT_ALGO = conf("spark.rapids.alluxio.replacement.algo") + .doc("The algorithm used when replacing the UFS path with the Alluxio path. CONVERT_TIME " + + "and SELECTION_TIME are the valid options. CONVERT_TIME indicates that we do it when " + + "we convert it to a GPU file read, this has extra overhead of creating an entirely new " + + "file index, which requires listing the files and getting all new file info from Alluxio. " + + "SELECTION_TIME indicates we do it when the file reader is selecting the partitions " + + "to process and just replaces the path without fetching the file information again, this " + + "is faster but doesn't update locality information if that were to work with Alluxio.") + .stringConf + .checkValues(Set("CONVERT_TIME", "SELECTION_TIME")) + .createWithDefault("SELECTION_TIME") + // USER FACING DEBUG CONFIGS val SHUFFLE_COMPRESSION_MAX_BATCH_MEMORY = @@ -2058,6 +2070,14 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val getAlluxioCmd: Seq[String] = get(ALLUXIO_CMD) + lazy val getAlluxioReplacementAlgo: String = get(ALLUXIO_REPLACEMENT_ALGO) + + lazy val isAlluxioReplacementAlgoSelectTime: Boolean = + get(ALLUXIO_REPLACEMENT_ALGO) == "SELECTION_TIME" + + lazy val isAlluxioReplacementAlgoConvertTime: Boolean = + get(ALLUXIO_REPLACEMENT_ALGO) == "CONVERT_TIME" + lazy val driverTimeZone: Option[String] = get(DRIVER_TIMEZONE) lazy val isRangeWindowByteEnabled: Boolean = get(ENABLE_RANGE_WINDOW_BYTES) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala index 589840d078b..4007d3815ef 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala @@ -16,7 +16,7 @@ package com.nvidia.spark.rapids -import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.parquet.schema.MessageType import org.apache.spark.rdd.RDD @@ -30,7 +30,7 @@ import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, BroadcastQueryStageExec} import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile, PartitioningAwareFileIndex} +import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionDirectory, PartitionedFile, PartitioningAwareFileIndex} import org.apache.spark.sql.execution.datasources.parquet.ParquetFilters import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.internal.SQLConf @@ -114,6 +114,9 @@ trait SparkShims { def filesFromFileIndex(fileCatalog: PartitioningAwareFileIndex): Seq[FileStatus] + def alluxioReplacePathsPartitionDirectory( pd: PartitionDirectory, + replaceFunc: Option[Path => Path]): (Seq[FileStatus], PartitionDirectory) + def isEmptyRelation(relation: Any): Boolean def broadcastModeTransform(mode: BroadcastMode, toArray: Array[InternalRow]): Any diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala index 0c5310f54aa..1f5caf1ccfd 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala @@ -20,7 +20,7 @@ import java.util.concurrent.TimeUnit.NANOSECONDS import scala.collection.mutable.HashMap -import com.nvidia.spark.rapids.{GpuExec, GpuMetric, GpuOrcMultiFilePartitionReaderFactory, GpuParquetMultiFilePartitionReaderFactory, GpuReadCSVFileFormat, GpuReadFileFormatWithMetrics, GpuReadOrcFileFormat, GpuReadParquetFileFormat, RapidsConf, SparkPlanMeta} +import com.nvidia.spark.rapids.{AlluxioUtils, GpuExec, GpuMetric, GpuOrcMultiFilePartitionReaderFactory, GpuParquetMultiFilePartitionReaderFactory, GpuReadCSVFileFormat, GpuReadFileFormatWithMetrics, GpuReadOrcFileFormat, GpuReadParquetFileFormat, RapidsConf, SparkPlanMeta} import com.nvidia.spark.rapids.shims.{GpuDataSourceRDD, SparkShimImpl} import org.apache.hadoop.fs.Path @@ -84,6 +84,15 @@ case class GpuFileSourceScanExec( case _ => true // For others, default to PERFILE reader } + private val isAlluxioAlgoSelectionTime = { + val alluxioPathsReplace = rapidsConf.getAlluxioPathsToReplace + val alluxioAutoMountEnabled = rapidsConf.getAlluxioAutoMountEnabled + // currently only support Alluxio replacement with Parquet files + (alluxioPathsReplace.isDefined || alluxioAutoMountEnabled) && + rapidsConf.isAlluxioReplacementAlgoSelectTime && + relation.fileFormat.isInstanceOf[ParquetFileFormat] + } + override def otherCopyArgs: Seq[AnyRef] = Seq(rapidsConf) // All expressions are filter expressions used on the CPU. @@ -112,9 +121,22 @@ case class GpuFileSourceScanExec( @transient lazy val selectedPartitions: Array[PartitionDirectory] = { val optimizerMetadataTimeNs = relation.location.metadataOpsTimeNs.getOrElse(0L) val startTime = System.nanoTime() - val ret = + val origRet = relation.location.listFiles( partitionFilters.filterNot(isDynamicPruningFilter), dataFilters) + val ret = if (isAlluxioAlgoSelectionTime) { + val res = origRet.map { pd => + AlluxioUtils.replacePathInPDIfNeeded(rapidsConf, pd, + relation.sparkSession.sparkContext.hadoopConfiguration, + relation.sparkSession.conf) + } + res + } else { + origRet + } + logDebug(s"File listing and possibly replace with Alluxio path " + + s"took: ${System.nanoTime() - startTime}") + setFilesNumAndSizeMetric(ret, true) val timeTakenMs = NANOSECONDS.toMillis( (System.nanoTime() - startTime) + optimizerMetadataTimeNs) From b5c2546dced58de2f6cbbf98a878eb433e1389d8 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 13 Sep 2022 10:07:27 -0600 Subject: [PATCH 104/190] Fixes for recent cuDF regexp changes (#6548) * Fixed for recent cuDF regexp changes Signed-off-by: Andy Grove * update tests * update tests * update tests * update tests * rename test Signed-off-by: Andy Grove --- .../com/nvidia/spark/rapids/RegexParser.scala | 23 +++------ .../RegularExpressionTranspilerSuite.scala | 50 +++++++++++-------- 2 files changed, 35 insertions(+), 38 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala index ac9d38dec3e..357522c004d 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala @@ -823,22 +823,20 @@ class CudfRegexTranspiler(mode: RegexMode) { private def negateCharacterClass(components: Seq[RegexCharacterClassComponent]): RegexAST = { // There are differences between cuDF and Java handling of newlines // for negative character matches. The expression `[^a]` will match - // `\r` and `\n` in Java but not in cuDF, so we replace `[^a]` with - // `(?:[\r\n]|[^a])`. We also have to take into account whether any + // `\r` in Java but not in cuDF, so we replace `[^a]` with + // `(?:[\r]|[^a])`. We also have to take into account whether any // newline characters are included in the character range. // // Examples: // - // `[^a]` => `(?:[\r\n]|[^a])` + // `[^a]` => `(?:[\r]|[^a])` // `[^a\r]` => `(?:[\n]|[^a])` // `[^a\n]` => `(?:[\r]|[^a])` // `[^a\r\n]` => `[^a]` - // `[^\r\n]` => `[^\r\n]` val distinctComponents = components.distinct val linefeedCharsInPattern = distinctComponents.flatMap { - case RegexChar(ch) if ch == '\n' || ch == '\r' => Seq(ch) - case RegexEscaped(ch) if ch == 'n' => Seq('\n') + case RegexChar(ch) if ch == '\r' => Seq(ch) case RegexEscaped(ch) if ch == 'r' => Seq('\r') case RegexCharacterRange(startRegex, RegexChar(end)) => val start = startRegex match { @@ -849,11 +847,7 @@ class CudfRegexTranspiler(mode: RegexMode) { s"Unexpected expression at start of character range: ${other.toRegexString}", other.position) } - if (start <= '\n' && end >= '\r') { - Seq('\n', '\r') - } else if (start <= '\n' && end >= '\n') { - Seq('\n') - } else if (start <= '\r' && end >= '\r') { + if (start <= '\r' && end >= '\r') { Seq('\r') } else { Seq.empty @@ -862,12 +856,9 @@ class CudfRegexTranspiler(mode: RegexMode) { } val onlyLinefeedChars = distinctComponents.length == linefeedCharsInPattern.length - val negatedNewlines = Seq('\r', '\n').diff(linefeedCharsInPattern.distinct) + val negatedNewlines = Seq('\r').diff(linefeedCharsInPattern.distinct) - if (onlyLinefeedChars && linefeedCharsInPattern.length == 2) { - // special case for `[^\r\n]` and `[^\\r\\n]` - RegexCharacterClass(negated = true, ListBuffer(distinctComponents: _*)) - } else if (negatedNewlines.isEmpty) { + if (negatedNewlines.isEmpty) { RegexCharacterClass(negated = true, ListBuffer(distinctComponents: _*)) } else { RegexGroup(capture = false, diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala index 4beb454f30b..ea485b5ad65 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala @@ -30,6 +30,23 @@ import org.apache.spark.sql.types.DataTypes class RegularExpressionTranspilerSuite extends FunSuite with Arm { + test("transpiler detects invalid cuDF patterns that cuDF now supports") { + // these patterns compile in cuDF since https://github.com/rapidsai/cudf/pull/11654 was merged + // but we still reject them because we see failures in fuzz testing if we allow them + for (pattern <- Seq("\t+|a", "(\t+|a)Dc$1", "\n[^\r\n]x*|^3x")) { + assertUnsupported(pattern, RegexFindMode, + "cuDF does not support repetition on one side of a choice") + } + for (pattern <- Seq("$|$[^\n]2]}|B")) { + assertUnsupported(pattern, RegexFindMode, + "End of line/string anchor is not supported in this context") + } + for (pattern <- Seq("a^|b", "w$|b", "]*\\wWW$|zb", "(\\A|\\05)?")) { + assertUnsupported(pattern, RegexFindMode, + "cuDF does not support terms ending with line anchors on one side of a choice") + } + } + test("transpiler detects invalid cuDF patterns") { // The purpose of this test is to document some examples of valid Java regular expressions // that fail to compile in cuDF and to check that the transpiler detects these correctly. @@ -37,15 +54,7 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { // random inputs. val cudfInvalidPatterns = Seq( "a*+", - "\t+|a", - "(\t+|a)Dc$1", - "(?d)", - "$|$[^\n]2]}|B", - "a^|b", - "w$|b", - "\n[^\r\n]x*|^3x", - "]*\\wWW$|zb", - "(\\A|\\05)?" + "(?d)" ) // data is not relevant because we are checking for compilation errors val inputs = Seq("a") @@ -87,7 +96,7 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { test("cuDF does not support choice with repetition") { val patterns = Seq("b+|^\t") patterns.foreach(pattern => - assertUnsupported(pattern, RegexFindMode, + assertUnsupported(pattern, RegexFindMode, "cuDF does not support repetition on one side of a choice") ) } @@ -101,20 +110,16 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { } test("cuDF unsupported choice cases") { - val input = Seq("cat", "dog") val patterns = Seq("c*|d*", "c*|dog", "[cat]{3}|dog") patterns.foreach(pattern => { - val e = intercept[CudfException] { - gpuContains(pattern, input) - } - assert(e.getMessage.contains("invalid regex pattern: nothing to repeat")) + assertUnsupported(pattern, RegexFindMode, + "cuDF does not support repetition on one side of a choice") }) } test("sanity check: choice edge case 2") { - assertThrows[CudfException] { - gpuContains("c+|d+", Seq("cat", "dog")) - } + assertUnsupported("c+|d+", RegexFindMode, + "cuDF does not support repetition on one side of a choice") } test("newline before $ in replace mode") { @@ -142,10 +147,11 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { test("cuDF does not support quantifier syntax when not quantifying anything") { // note that we could choose to transpile and escape the '{' and '}' characters - val patterns = Seq("{1,2}", "{1,}", "{1}", "{2,1}") - patterns.foreach(pattern => - assertUnsupported(pattern, RegexFindMode, + val patterns = Seq("{1,2}", "{1,}", "{1}") + patterns.foreach(pattern => { + assertUnsupported(pattern, RegexFindMode, "Token preceding '{' is not quantifiable near index 0") + } ) } @@ -376,7 +382,7 @@ class RegularExpressionTranspilerSuite extends FunSuite with Arm { test("transpile character class unescaped range symbol") { val patterns = Seq("a[-b]", "a[+-]", "a[-+]", "a[-]", "a[^-]") - val expected = Seq(raw"a[\-b]", raw"a[+\-]", raw"a[\-+]", raw"a[\-]", "a(?:[\r\n]|[^\\-])") + val expected = Seq(raw"a[\-b]", raw"a[+\-]", raw"a[\-+]", raw"a[\-]", "a(?:[\r]|[^\\-])") val transpiler = new CudfRegexTranspiler(RegexFindMode) val transpiled = patterns.map(transpiler.transpile(_, None)._1) assert(transpiled === expected) From 1ee9c4a37e46f9e833ecf8697dd199f08a7a814e Mon Sep 17 00:00:00 2001 From: Peixin Date: Wed, 14 Sep 2022 08:50:35 +0800 Subject: [PATCH 105/190] Nightly spark-tests script to follow PYSP_TEST pattern [skip ci] (#6522) * spark-tests script to follow PYSP_TEST pattern Signed-off-by: Peixin Li * include required pytest plugins for xdist run Signed-off-by: Peixin Li * add test mode comment * address comments and updated some typo Signed-off-by: Peixin Li --- integration_tests/run_pyspark_from_build.sh | 4 +- jenkins/Dockerfile-blossom.integration.centos | 4 +- jenkins/Dockerfile-blossom.integration.rocky | 4 +- jenkins/Dockerfile-blossom.integration.ubuntu | 6 +- jenkins/spark-tests.sh | 251 ++++++------------ 5 files changed, 86 insertions(+), 183 deletions(-) diff --git a/integration_tests/run_pyspark_from_build.sh b/integration_tests/run_pyspark_from_build.sh index b70ea80e463..a02c6add20e 100755 --- a/integration_tests/run_pyspark_from_build.sh +++ b/integration_tests/run_pyspark_from_build.sh @@ -168,8 +168,8 @@ else if [[ ${TEST_PARALLEL} -lt 2 ]]; then - # With xdist 0 and 1 are the same parallelsm but - # 0 is more effecient + # With xdist 0 and 1 are the same parallelism but + # 0 is more efficient TEST_PARALLEL_OPTS=() else TEST_PARALLEL_OPTS=("-n" "$TEST_PARALLEL") diff --git a/jenkins/Dockerfile-blossom.integration.centos b/jenkins/Dockerfile-blossom.integration.centos index 44fb9cb7ce5..3ded7ceed03 100644 --- a/jenkins/Dockerfile-blossom.integration.centos +++ b/jenkins/Dockerfile-blossom.integration.centos @@ -39,7 +39,7 @@ RUN if [ "$CENTOS_VER" == "8" ]; then\ # Install jdk-8, jdk-11, maven, docker image RUN yum update -y && \ yum install epel-release -y && \ - yum install -y java-1.8.0-openjdk-devel java-11-openjdk-devel wget expect parallel rsync zip unzip + yum install -y java-1.8.0-openjdk-devel java-11-openjdk-devel wget expect rsync zip unzip # The default mvn verision is 3.0.5 on centos7 docker container. # The plugin: net.alchim31.maven requires a higher mvn version. @@ -60,6 +60,8 @@ RUN export CUDA_VER=`echo ${CUDA_VER} | cut -d '.' -f 1,2` && \ mamba install -y -c anaconda pytest requests && \ mamba install -y -c conda-forge sre_yield && \ conda clean -ay +# install pytest plugins for xdist parallel run +RUN python -m pip install findspark pytest-xdist pytest-order # Set default java as 1.8.0 ENV JAVA_HOME "/usr/lib/jvm/java-1.8.0-openjdk" diff --git a/jenkins/Dockerfile-blossom.integration.rocky b/jenkins/Dockerfile-blossom.integration.rocky index 606d27b80ab..e90f5350946 100644 --- a/jenkins/Dockerfile-blossom.integration.rocky +++ b/jenkins/Dockerfile-blossom.integration.rocky @@ -33,7 +33,7 @@ ARG URM_URL # Install jdk-8, jdk-11, maven, docker image RUN yum update -y && \ yum install epel-release -y && \ - yum install -y java-1.8.0-openjdk-devel java-11-openjdk-devel wget expect parallel rsync zip unzip + yum install -y java-1.8.0-openjdk-devel java-11-openjdk-devel wget expect rsync zip unzip # The plugin: net.alchim31.maven requires a higher mvn version. ENV MAVEN_HOME "/usr/local/apache-maven-3.6.3" @@ -53,6 +53,8 @@ RUN export CUDA_VER=`echo ${CUDA_VER} | cut -d '.' -f 1,2` && \ mamba install -y -c anaconda pytest requests && \ mamba install -y -c conda-forge sre_yield && \ conda clean -ay +# install pytest plugins for xdist parallel run +RUN python -m pip install findspark pytest-xdist pytest-order # Set default java as 1.8.0 ENV JAVA_HOME "/usr/lib/jvm/java-1.8.0-openjdk" diff --git a/jenkins/Dockerfile-blossom.integration.ubuntu b/jenkins/Dockerfile-blossom.integration.ubuntu index 4e54b47ad3d..e36f24ae82f 100644 --- a/jenkins/Dockerfile-blossom.integration.ubuntu +++ b/jenkins/Dockerfile-blossom.integration.ubuntu @@ -1,5 +1,5 @@ # -# Copyright (c) 2020-2021, NVIDIA CORPORATION. All rights reserved. +# Copyright (c) 2020-2022, NVIDIA CORPORATION. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -40,7 +40,7 @@ RUN apt-get update -y && \ RUN add-apt-repository ppa:deadsnakes/ppa && \ apt-get update -y && \ DEBIAN_FRONTEND="noninteractive" apt-get install -y maven \ - openjdk-8-jdk openjdk-11-jdk python3.8 python3.8-distutils python3-setuptools tzdata git wget parallel + openjdk-8-jdk openjdk-11-jdk python3.8 python3.8-distutils python3-setuptools tzdata git wget RUN python3.8 -m easy_install pip # Set default jdk as 1.8.0 @@ -58,6 +58,8 @@ RUN export CUDA_VER=`echo ${CUDA_VER} | cut -d '.' -f 1,2` && \ mamba install -y -c anaconda pytest requests && \ mamba install -y -c conda-forge sre_yield && \ conda clean -ay +# install pytest plugins for xdist parallel run +RUN python -m pip install findspark pytest-xdist pytest-order RUN apt install -y inetutils-ping expect diff --git a/jenkins/spark-tests.sh b/jenkins/spark-tests.sh index 841ea9e912f..2fdb81aa49f 100755 --- a/jenkins/spark-tests.sh +++ b/jenkins/spark-tests.sh @@ -125,13 +125,8 @@ export PYTHONPATH="$CONDA_ROOT/lib/python$PYTHON_VER/site-packages:$PYTHONPATH" echo "----------------------------START TEST------------------------------------" pushd $RAPIDS_INT_TESTS_HOME - -export TEST_PARALLEL=0 # disable spark local parallel in run_pyspark_from_build.sh export TEST_TYPE="nightly" export LOCAL_JAR_PATH=$ARTF_ROOT -# test collect-only in advance to terminate earlier if ENV issue -COLLECT_BASE_SPARK_SUBMIT_ARGS="$BASE_SPARK_SUBMIT_ARGS" # if passed custom params -SPARK_SUBMIT_FLAGS="$COLLECT_BASE_SPARK_SUBMIT_ARGS" ./run_pyspark_from_build.sh --collect-only -qqq export SPARK_TASK_MAXFAILURES=1 export PATH="$SPARK_HOME/bin:$SPARK_HOME/sbin:$PATH" @@ -139,39 +134,47 @@ export PATH="$SPARK_HOME/bin:$SPARK_HOME/sbin:$PATH" # if failed, we abort the test instantly, so the failed executor log should still be left there for debugging export SPARK_WORKER_OPTS="$SPARK_WORKER_OPTS -Dspark.worker.cleanup.enabled=true -Dspark.worker.cleanup.interval=120 -Dspark.worker.cleanup.appDataTtl=60" #stop and restart SPARK ETL -stop-slave.sh +stop-worker.sh stop-master.sh start-master.sh -start-slave.sh spark://$HOSTNAME:7077 +start-worker.sh spark://$HOSTNAME:7077 jps -export BASE_SPARK_SUBMIT_ARGS="$BASE_SPARK_SUBMIT_ARGS \ ---master spark://$HOSTNAME:7077 \ ---conf spark.sql.shuffle.partitions=12 \ ---conf spark.task.maxFailures=$SPARK_TASK_MAXFAILURES \ ---conf spark.dynamicAllocation.enabled=false \ ---conf spark.driver.extraJavaOptions=-Duser.timezone=UTC \ ---conf spark.executor.extraJavaOptions=-Duser.timezone=UTC \ ---conf spark.sql.session.timeZone=UTC" - -export SEQ_CONF="--executor-memory 16G \ ---total-executor-cores 6" - -export PARALLEL_CONF="--executor-memory 4G \ ---total-executor-cores 1 \ ---conf spark.executor.cores=1 \ ---conf spark.task.cpus=1 \ ---conf spark.rapids.sql.concurrentGpuTasks=1 \ ---conf spark.rapids.memory.gpu.minAllocFraction=0" - -export CUDF_UDF_TEST_ARGS="--conf spark.rapids.memory.gpu.allocFraction=0.1 \ ---conf spark.rapids.memory.gpu.minAllocFraction=0 \ ---conf spark.rapids.python.memory.gpu.allocFraction=0.1 \ ---conf spark.rapids.python.concurrentPythonWorkers=2 \ ---conf spark.executorEnv.PYTHONPATH=${RAPIDS_PLUGIN_JAR} \ ---conf spark.pyspark.python=/opt/conda/bin/python \ ---py-files ${RAPIDS_PLUGIN_JAR}" - +# BASE spark test configs +export PYSP_TEST_spark_master=spark://$HOSTNAME:7077 +export PYSP_TEST_spark_sql_shuffle_partitions=12 +export PYSP_TEST_spark_task_maxFailures=$SPARK_TASK_MAXFAILURES +export PYSP_TEST_spark_dynamicAllocation_enabled=false +export PYSP_TEST_spark_driver_extraJavaOptions=-Duser.timezone=UTC +export PYSP_TEST_spark_executor_extraJavaOptions=-Duser.timezone=UTC +export PYSP_TEST_spark_sql_session_timeZone=UTC + +# PARALLEL or non-PARALLEL specific configs +if [[ $PARALLEL_TEST == "true" ]]; then + export PYSP_TEST_spark_cores_max=1 + export PYSP_TEST_spark_executor_memory=4g + export PYSP_TEST_spark_executor_cores=1 + export PYSP_TEST_spark_task_cores=1 + export PYSP_TEST_spark_rapids_sql_concurrentGpuTasks=1 + export PYSP_TEST_spark_rapids_memory_gpu_minAllocFraction=0 + + if [[ "${PARALLELISM}" == "" ]]; then + PARALLELISM=$(nvidia-smi --query-gpu=memory.free --format=csv,noheader | \ + awk '{if (MAX < $1){ MAX = $1}} END {print int(MAX / (2 * 1024))}') + fi + # parallelism > 6 could slow down the whole process, so we have a limitation for it + # this is based on our CI gpu types, so we do not put it into the run_pyspark_from_build.sh + [[ ${PARALLELISM} -gt 6 ]] && PARALLELISM=6 + MEMORY_FRACTION=$(python -c "print(1/($PARALLELISM + 0.1))") + + export TEST_PARALLEL=${PARALLELISM} + export PYSP_TEST_spark_rapids_memory_gpu_allocFraction=${MEMORY_FRACTION} + export PYSP_TEST_spark_rapids_memory_gpu_maxAllocFraction=${MEMORY_FRACTION} +else + export PYSP_TEST_spark_cores_max=6 + export PYSP_TEST_spark_executor_memory=16g + export TEST_PARALLEL=0 +fi export SCRIPT_PATH="$(pwd -P)" export TARGET_DIR="$SCRIPT_PATH/target" @@ -181,11 +184,11 @@ run_delta_lake_tests() { echo "run_delta_lake_tests SPARK_VER = $SPARK_VER" SPARK_321_PATTERN="(3\.2\.[1-9])" DELTA_LAKE_VER="1.2.1" + if [[ $SPARK_VER =~ $SPARK_321_PATTERN ]]; then - SPARK_SUBMIT_FLAGS="$BASE_SPARK_SUBMIT_ARGS $SEQ_CONF \ - --packages io.delta:delta-core_2.12:$DELTA_LAKE_VER \ - --conf spark.sql.extensions=io.delta.sql.DeltaSparkSessionExtension \ - --conf spark.sql.catalog.spark_catalog=org.apache.spark.sql.delta.catalog.DeltaCatalog" \ + PYSP_TEST_spark_jars_packages="io.delta:delta-core_2.12:$DELTA_LAKE_VER" \ + PYSP_TEST_spark_sql_extensions="io.delta.sql.DeltaSparkSessionExtension" \ + PYSP_TEST_spark_sql_catalog_spark__catalog="org.apache.spark.sql.delta.catalog.DeltaCatalog" \ ./run_pyspark_from_build.sh -m delta_lake --delta_lake else echo "Skipping Delta Lake tests. Delta Lake does not support Spark version $SPARK_VER" @@ -199,174 +202,68 @@ run_iceberg_tests() { # Iceberg does not support Spark 3.3+ yet if [[ "$ICEBERG_SPARK_VER" < "3.3" ]]; then - SPARK_SUBMIT_FLAGS="$BASE_SPARK_SUBMIT_ARGS $SEQ_CONF \ - --packages org.apache.iceberg:iceberg-spark-runtime-${ICEBERG_SPARK_VER}_2.12:${ICEBERG_VERSION} \ - --conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions \ - --conf spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog \ - --conf spark.sql.catalog.spark_catalog.type=hadoop \ - --conf spark.sql.catalog.spark_catalog.warehouse=/tmp/spark-warehouse-$$" \ + PYSP_TEST_spark_jars_packages=org.apache.iceberg:iceberg-spark-runtime-${ICEBERG_SPARK_VER}_2.12:${ICEBERG_VERSION} \ + PYSP_TEST_spark_sql_extensions="org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions" \ + PYSP_TEST_spark_sql_catalog_spark__catalog="org.apache.iceberg.spark.SparkSessionCatalog" \ + PYSP_TEST_spark_sql_catalog_spark__catalog_type="hadoop" \ + PYSP_TEST_spark_sql_catalog_spark__catalog_warehouse="/tmp/spark-warehouse-$RANDOM" \ ./run_pyspark_from_build.sh -m iceberg --iceberg else echo "Skipping Iceberg tests. Iceberg does not support Spark $ICEBERG_SPARK_VER" fi } - # Test spark-avro with documented way of deploying at run time via --packages option from Maven run_avro_tests() { - export PYSP_TEST_spark_jars_packages="org.apache.spark:spark-avro_2.12:${SPARK_VER}" - - # Workaround to avoid appending avro jar file by '--jars', which would be addressed by https://github.com/NVIDIA/spark-rapids/issues/6532 + # Workaround to avoid appending avro jar file by '--jars', + # which would be addressed by https://github.com/NVIDIA/spark-rapids/issues/6532 rm -vf $LOCAL_JAR_PATH/spark-avro*.jar - - ./run_pyspark_from_build.sh -k avro -} - -run_test_not_parallel() { - local TEST=${1//\.py/} - local LOG_FILE - case $TEST in - all) - SPARK_SUBMIT_FLAGS="$BASE_SPARK_SUBMIT_ARGS $SEQ_CONF" \ - ./run_pyspark_from_build.sh - ;; - - cudf_udf_test) - SPARK_SUBMIT_FLAGS="$BASE_SPARK_SUBMIT_ARGS $SEQ_CONF $CUDF_UDF_TEST_ARGS" \ - ./run_pyspark_from_build.sh -m cudf_udf --cudf_udf - ;; - - cache_serializer) - SPARK_SUBMIT_FLAGS="$BASE_SPARK_SUBMIT_ARGS $SEQ_CONF \ - --conf spark.sql.cache.serializer=com.nvidia.spark.ParquetCachedBatchSerializer" \ - ./run_pyspark_from_build.sh -k cache_test - ;; - - delta_lake) - run_delta_lake_tests - ;; - - iceberg) - run_iceberg_tests - ;; - - avro) - run_avro_tests - ;; - - *) - echo -e "\n\n>>>>> $TEST...\n" - LOG_FILE="$TARGET_DIR/$TEST.log" - # set dedicated RUN_DIRs here to avoid conflict between parallel tests - RUN_DIR="$TARGET_DIR/run_dir_$TEST" \ - SPARK_SUBMIT_FLAGS="$BASE_SPARK_SUBMIT_ARGS $PARALLEL_CONF $MEMORY_FRACTION_CONF" \ - ./run_pyspark_from_build.sh -k $TEST >"$LOG_FILE" 2>&1 - - CODE="$?" - if [[ $CODE == "0" ]]; then - sed -n -e '/test session starts/,/deselected,/ p' "$LOG_FILE" || true - else - cat "$LOG_FILE" || true - cat /tmp/artifacts-build.info || true - fi - return $CODE - ;; - esac + PYSP_TEST_spark_jars_packages="org.apache.spark:spark-avro_2.12:${SPARK_VER}" \ + ./run_pyspark_from_build.sh -k avro } -export -f run_test_not_parallel - -get_cases_by_tags() { - local cases - local args=${2} - cases=$(TEST_TAGS="${1}" SPARK_SUBMIT_FLAGS="$COLLECT_BASE_SPARK_SUBMIT_ARGS" \ - ./run_pyspark_from_build.sh "${args}" --collect-only -p no:warnings -qq 2>/dev/null \ - | grep -oP '(?<=::).*?(?=\[)' | uniq | xargs) - echo "$cases" -} -export -f get_cases_by_tags - -get_tests_by_tags() { - local tests - local args=${2} - tests=$(TEST_TAGS="${1}" SPARK_SUBMIT_FLAGS="$COLLECT_BASE_SPARK_SUBMIT_ARGS" \ - ./run_pyspark_from_build.sh "${args}" --collect-only -qqq -p no:warnings 2>/dev/null \ - | grep -oP '(?<=python/).*?(?=.py)' | xargs) - echo "$tests" -} -export -f get_tests_by_tags # TEST_MODE # - DEFAULT: all tests except cudf_udf tests -# - CUDF_UDF_ONLY: cudf_udf tests only, requires extra conda cudf-py lib +# - DELTA_LAKE_ONLY: Delta Lake tests only # - ICEBERG_ONLY: iceberg tests only # - AVRO_ONLY: avro tests only (with --packages option instead of --jars) -# - DELTA_LAKE_ONLY: Delta Lake tests only +# - CUDF_UDF_ONLY: cudf_udf tests only, requires extra conda cudf-py lib TEST_MODE=${TEST_MODE:-'DEFAULT'} if [[ $TEST_MODE == "DEFAULT" ]]; then - # integration tests - if [[ $PARALLEL_TEST == "true" ]] && [ -x "$(command -v parallel)" ]; then - # separate run for special cases that require smaller parallelism - special_cases=$(get_cases_by_tags "nightly_resource_consuming_test \ - and (nightly_gpu_mem_consuming_case or nightly_host_mem_consuming_case)") - # hardcode parallelism as 2 for special cases - export MEMORY_FRACTION_CONF="--conf spark.rapids.memory.gpu.allocFraction=0.45 \ - --conf spark.rapids.memory.gpu.maxAllocFraction=0.45" - # --halt "now,fail=1": exit when the first job fail, and kill running jobs. - # we can set it to "never" and print failed ones after finish running all tests if needed - # --group: print stderr after test finished for better readability - parallel --group --halt "now,fail=1" -j2 run_test_not_parallel ::: ${special_cases} - - resource_consuming_cases=$(get_cases_by_tags "nightly_resource_consuming_test \ - and not nightly_gpu_mem_consuming_case \ - and not nightly_host_mem_consuming_case") - other_tests=$(get_tests_by_tags "not nightly_resource_consuming_test") - tests=$(echo "${resource_consuming_cases} ${other_tests}" | tr ' ' '\n' | awk '!x[$0]++' | xargs) - - if [[ "${PARALLELISM}" == "" ]]; then - PARALLELISM=$(nvidia-smi --query-gpu=memory.free --format=csv,noheader | \ - awk '{if (MAX < $1){ MAX = $1}} END {print int(MAX / (2 * 1024))}') - fi - # parallelism > 7 could slow down the whole process, so we have a limitation for it - [[ ${PARALLELISM} -gt 7 ]] && PARALLELISM=7 - MEMORY_FRACTION=$(python -c "print(1/($PARALLELISM + 0.1))") - export MEMORY_FRACTION_CONF="--conf spark.rapids.memory.gpu.allocFraction=${MEMORY_FRACTION} \ - --conf spark.rapids.memory.gpu.maxAllocFraction=${MEMORY_FRACTION}" - parallel --group --halt "now,fail=1" -j"${PARALLELISM}" run_test_not_parallel ::: ${tests} - else - run_test_not_parallel all - fi + ./run_pyspark_from_build.sh - if [[ $PARALLEL_TEST == "true" ]] && [ -x "$(command -v parallel)" ]; then - cache_test_cases=$(get_cases_by_tags "" "-k cache_test") - # hardcode parallelism as 5 - export MEMORY_FRACTION_CONF="--conf spark.rapids.memory.gpu.allocFraction=0.18 \ - --conf spark.rapids.memory.gpu.maxAllocFraction=0.18 \ - --conf spark.sql.cache.serializer=com.nvidia.spark.ParquetCachedBatchSerializer" - parallel --group --halt "now,fail=1" -j5 run_test_not_parallel ::: ${cache_test_cases} - else - run_test_not_parallel cache_serializer - fi -fi - -# cudf_udf_test -if [[ "$TEST_MODE" == "CUDF_UDF_ONLY" ]]; then - run_test_not_parallel cudf_udf_test + # ParquetCachedBatchSerializer cache_test + PYSP_TEST_spark_sql_cache_serializer=com.nvidia.spark.ParquetCachedBatchSerializer \ + ./run_pyspark_from_build.sh -k cache_test fi # Delta Lake tests if [[ "$TEST_MODE" == "DEFAULT" || "$TEST_MODE" == "DELTA_LAKE_ONLY" ]]; then - run_test_not_parallel delta_lake + run_delta_lake_tests fi # Iceberg tests if [[ "$TEST_MODE" == "DEFAULT" || "$TEST_MODE" == "ICEBERG_ONLY" ]]; then - run_test_not_parallel iceberg + run_iceberg_tests fi -# avro tests +# Avro tests if [[ "$TEST_MODE" == "DEFAULT" || "$TEST_MODE" == "AVRO_ONLY" ]]; then - run_test_not_parallel avro + run_avro_tests +fi + +# cudf_udf test: this mostly depends on cudf-py, so we run it into an independent CI +if [[ "$TEST_MODE" == "CUDF_UDF_ONLY" ]]; then + # hardcode config + [[ ${TEST_PARALLEL} -gt 2 ]] && export TEST_PARALLEL=2 + PYSP_TEST_spark_rapids_memory_gpu_allocFraction=0.1 \ + PYSP_TEST_spark_rapids_memory_gpu_minAllocFraction=0 \ + PYSP_TEST_spark_rapids_python_memory_gpu_allocFraction=0.1 \ + PYSP_TEST_spark_rapids_python_concurrentPythonWorkers=2 \ + PYSP_TEST_spark_executorEnv_PYTHONPATH=${RAPIDS_PLUGIN_JAR} \ + PYSP_TEST_spark_python=${CONDA_ROOT}/bin/python \ + ./run_pyspark_from_build.sh -m cudf_udf --cudf_udf fi popd -stop-slave.sh +stop-worker.sh stop-master.sh From ec10a4a99430766e1ddb74f28ecc81a97937f3af Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Wed, 14 Sep 2022 09:51:22 +0800 Subject: [PATCH 106/190] ORC reading supports mergeSchema (#6523) Signed-off-by: Firestarman --- integration_tests/src/main/python/orc_test.py | 1 - .../main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala | 8 -------- .../com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala | 3 --- 3 files changed, 12 deletions(-) diff --git a/integration_tests/src/main/python/orc_test.py b/integration_tests/src/main/python/orc_test.py index 7c0c775fc68..10e2f6ee691 100644 --- a/integration_tests/src/main/python/orc_test.py +++ b/integration_tests/src/main/python/orc_test.py @@ -286,7 +286,6 @@ def test_partitioned_read_just_partitions(spark_tmp_path, v1_enabled_list, reade lambda spark : spark.read.orc(data_path).select("key"), conf=all_confs) -@pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/135') @pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) @pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) def test_merge_schema_read(spark_tmp_path, v1_enabled_list, reader_confs): diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index f022100eb2a..03222aecde5 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -126,9 +126,6 @@ object GpuOrcScan extends Arm { def tagSupport(scanMeta: ScanMeta[OrcScan]): Unit = { val scan = scanMeta.wrapped val schema = StructType(scan.readDataSchema ++ scan.readPartitionSchema) - if (scan.options.getBoolean("mergeSchema", false)) { - scanMeta.willNotWorkOnGpu("mergeSchema and schema evolution is not supported yet") - } tagSupport(scan.sparkSession, schema, scanMeta) } @@ -147,11 +144,6 @@ object GpuOrcScan extends Arm { } FileFormatChecks.tag(meta, schema, OrcFormatType, ReadFileOp) - - if (sparkSession.conf - .getOption("spark.sql.orc.mergeSchema").exists(_.toBoolean)) { - meta.willNotWorkOnGpu("mergeSchema and schema evolution is not supported yet") - } } private lazy val numericLevels = Seq( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala index 13d0c2fd49e..82b4c41a364 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala @@ -61,9 +61,6 @@ class GpuReadOrcFileFormat extends OrcFileFormat with GpuReadFileFormatWithMetri object GpuReadOrcFileFormat { def tagSupport(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { val fsse = meta.wrapped - if (fsse.relation.options.getOrElse("mergeSchema", "false").toBoolean) { - meta.willNotWorkOnGpu("mergeSchema and schema evolution is not supported yet") - } GpuOrcScan.tagSupport( SparkShimImpl.sessionFromPlan(fsse), fsse.requiredSchema, From 370e1a90f27967d35d58750a61b6d31b3323d33c Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 13 Sep 2022 21:08:11 -0500 Subject: [PATCH 107/190] Update SnakeYaml version for bug fixes (#6549) * Update snakeYaml version * sign off Signed-off-by: Thomas Graves * revert accidental removal of discoveryScript.sh Signed-off-by: Thomas Graves Signed-off-by: Thomas Graves --- tools/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/pom.xml b/tools/pom.xml index deea91935e6..df76292f249 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -87,7 +87,7 @@ org.yaml snakeyaml - 1.30 + 1.31 From 771a35dab62e5d62a02144d2462fc5b7a303c3c2 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Wed, 14 Sep 2022 11:58:24 -0700 Subject: [PATCH 108/190] Document a safe unshimming algorithm [skip ci] (#6555) Provides a guide for developers for unshimming a class. It should be further automated as the dist jar verification for correct placement of public classes Current validation for bitwise-identity is just an approximation. Signed-off-by: Gera Shegalov --- docs/dev/shims.md | 217 ++++++++++++++++++++++++++++++++++++++-------- 1 file changed, 179 insertions(+), 38 deletions(-) diff --git a/docs/dev/shims.md b/docs/dev/shims.md index 844ff2cd007..58c846383aa 100644 --- a/docs/dev/shims.md +++ b/docs/dev/shims.md @@ -6,14 +6,14 @@ parent: Developer Overview --- # Shim Development -RAPIDS Accelerator For Apache Spark supports multiple feature version lines of +RAPIDS Accelerator For Apache Spark supports multiple feature version lines of Apache Spark such as 3.1.x, 3.2.x, 3.3.0 and a number of vendor releases that contain -a mix of patches from different upstream releases. These artifacts are generally -incompatible between each other, at both source code level and even more often -at the binary level. The role of the Shim layer is to hide these issues from the -common code, maximize reuse, and minimize logic duplication. +a mix of patches from different upstream releases. These artifacts are generally +incompatible between each other, at both source code level and even more often +at the binary level. The role of the Shim layer is to hide these issues from the +common code, maximize reuse, and minimize logic duplication. -This is achieved by using a ServiceProvider pattern. All Shims implement the same API, +This is achieved by using a ServiceProvider pattern. All Shims implement the same API, the suitable Shim implementation is loaded after detecting the current Spark build version attempting to instantiate our plugin. We use the [ShimLoader](https://github.com/NVIDIA/spark-rapids/blob/main/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala) @@ -21,25 +21,25 @@ class as a tight entry point for interacting with the host Spark runtime. In the following we provide recipes for typical scenarios addressed by the Shim layer. -## Method signature discrepancies +## Method signature discrepancies It's among the easiest issues to resolve. We define a method in SparkShims -trait covering a superset of parameters from all versions and call it +trait covering a superset of parameters from all versions and call it ``` SparkShimImpl.methodWithDiscrepancies(p_1, ..., p_n) ``` -instead of referencing it directly. Shim implementations (SparkShimImpl) are in charge of dispatching it further -to correct version-dependent methods. Moreover, unlike in the below sections -conflicts between versions are easily avoided by using different package or class names -for conflicting Shim implementations. +instead of referencing it directly. Shim implementations (SparkShimImpl) are in charge of dispatching it further +to correct version-dependent methods. Moreover, unlike in the below sections +conflicts between versions are easily avoided by using different package or class names +for conflicting Shim implementations. -## Base Classes/Traits Changes +## Base Classes/Traits Changes ### Compile-time issues Upstream base classes we derive from might be incompatible in the sense that one version requires us to implement/override the method `M` whereas the other prohibits it by marking the base implementation `final`, E.g. `org.apache.spark.sql.catalyst.trees.TreeNode` changes -between Spark 3.1.x and Spark 3.2.x. So instead of deriving from such classes directly we +between Spark 3.1.x and Spark 3.2.x. So instead of deriving from such classes directly we inject an intermediate trait e.g. `com.nvidia.spark.rapids.shims.ShimExpression` that has a varying source code depending on the Spark version we compile against to overcome this issue as you can see e.g., comparing TreeNode: @@ -50,9 +50,9 @@ This resolves compile-time problems, however, now we face the problem at run tim ### Run-time Issues -Plugin has to be able to deterministically load the right class files +Plugin has to be able to deterministically load the right class files for identically named classes depending on the detected -Spark runtime version. This is typically solved by using ASM-based relocation in the bytecode, +Spark runtime version. This is typically solved by using ASM-based relocation in the bytecode, however it does not work easily with [Scala packages](https://contributors.scala-lang.org/t/scala-signature-layout/3327/4) @@ -67,7 +67,7 @@ jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/spark3xx-common/ jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/spark302/ ``` -Spark 3.2.0's URLs : +Spark 3.2.0's URLs : ``` jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/ jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/spark3xx-common/ @@ -76,39 +76,39 @@ jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/spark320/ ### Late Inheritance in Public Classes -Most classes needed by the plugin can be disambiguated using Parallel World locations without -reservations except for documented classes that are exposed to the user that may be loaded before +Most classes needed by the plugin can be disambiguated using Parallel World locations without +reservations except for documented classes that are exposed to the user that may be loaded before the Plugin is even instantiated by the Spark runtime. The most important example of such a class -is a configurable ShuffleManager. `ShuffleManager` has also changed in a backwards incompatible -manner over the span of supported Spark versions. +is a configurable ShuffleManager. `ShuffleManager` has also changed in a backwards incompatible +manner over the span of supported Spark versions. -The first issue with such a class, since it's loaded by Spark directly outside our control we +The first issue with such a class, since it's loaded by Spark directly outside our control we cannot have a single class name for our implementation that would work across versions. This is resolved, by having the documented facade classes with a shim specifier in their package names. -The second issue that every parent class/trait in the inheritance graph is loaded using the classloader outside -Plugin's control. Therefore, all this bytecode must reside in the conventional jar location, and it must +The second issue that every parent class/trait in the inheritance graph is loaded using the classloader outside +Plugin's control. Therefore, all this bytecode must reside in the conventional jar location, and it must be bitwise-identical across *all* shims. The only way to keep the source code for shared functionality unduplicated, -(i.e., in `sql-plugin/src/main/scala` as opposed to be duplicated in `sql-plugin/src/main/3*/scala` source code roots) -is to delay inheriting `ShuffleManager` until as late as possible, as close as possible to the facade class where we -have to split the source code anyway. Use traits as much as possible for flexibility. +(i.e., in `sql-plugin/src/main/scala` as opposed to be duplicated in `sql-plugin/src/main/3*/scala` source code roots) +is to delay inheriting `ShuffleManager` until as late as possible, as close as possible to the facade class where we +have to split the source code anyway. Use traits as much as possible for flexibility. ### Late Initialization of Public Classes' Ancestors - -The third issue may arise from the fact that the shared logic may transitively reference a class that + +The third issue may arise from the fact that the shared logic may transitively reference a class that for one another reason resides in a Parallel World. Untangling this is tedious and may be unnecessary. -The following approach robustly prevents from running into issues related to that. +The following approach robustly prevents from running into issues related to that. -We know that at the time such a class is loaded by Spark it's not strictly needed if the Plugin -has not been loaded yet. More accurately, it may not be strictly needed until later when the first -query can be run when the Spark SQL session and its extensions are initialized. It improves the -user experience if the first query is not penalized beyond necessary though. By design, Plugin guarantees -that the classloader is +We know that at the time such a class is loaded by Spark it's not strictly needed if the Plugin +has not been loaded yet. More accurately, it may not be strictly needed until later when the first +query can be run when the Spark SQL session and its extensions are initialized. It improves the +user experience if the first query is not penalized beyond necessary though. By design, Plugin guarantees +that the classloader is [set up at load time](https://github.com/NVIDIA/spark-rapids/blob/main/sql-plugin/src/main/scala/com/nvidia/spark/SQLPlugin.scala#L29) -before the DriverPlugin and ExecutorPlugin instances are called the `init` method on. +before the DriverPlugin and ExecutorPlugin instances are called the `init` method on. By making a visible class merely a wrapper of the real implementation, extending `scala.Proxy` where `self` is a lazy -val, we prevent classes from Parallel Worlds to be loaded before they can be, and are actually required. +val, we prevent classes from Parallel Worlds to be loaded before they can be, and are actually required. For examples see: 1. `abstract class ProxyRapidsShuffleInternalManagerBase` @@ -118,5 +118,146 @@ Note that we currently have to manually code up the delegation methods to the tu ``` def method(x: SomeThing) = self.method(x) ``` -This could be automatically generated with a simple tool processing the `scalap` output or Scala macros at +This could be automatically generated with a simple tool processing the `scalap` output or Scala macros at build/compile time. Pull requests are welcome. + +## How to externalize an internal class as a compile-time dependency + +At some point you may find it necessary to expose an existing +`class/trait/object A` currently residing in a "hidden" parallel world +as a dependency for Maven modules/projects dependencies depending on the `dist` +module artifact `rapids-4-spark_2.12`. + +This has two pre-requisites: +1. The .class file with the bytecode is bitwise-identical among the currently +supported Spark versions. To verify this you can inspect the dist jar and check +if the class file is under `spark3xx-common` jar entry. If this is not the case then +code should be refactored until all discrepancies are shimmed away. +1. The transitive closure of the classes compile-time-referenced by `A` should +have the property above. + +JDK ships the `jdeps` tool that can help analyze static dependencies of +a class. Unfortunately, it does not compute the transitive closure (recursive) +at the class granularity. Thus you need additional tools such as +the `[Graphviz tool](https://graphviz.org/)` used here. + +To figure out the transitive closure of a class we first need to build +the `dist` module. While iterating on the PR, it should be sufficient +to build against the lowest and highest versions of the supported Spark version +range. As of the time of this writing: + +```Bash + ./build/buildall --parallel=4 --profile=311,330 --module=dist +``` + +However, before submitting the PR execute the full build `--profile=noSnapshots`. + +Then switch to the parallel-world build dir. +``` +cd dist/target/parallel-world/ +``` + +Move the current externalized classes (outside the spark3* parallel worlds) to +a dedicated directory, say `public`. + +``` +mv org com ai public/ +``` + +`jdeps` can now treat public classes as a separate archive +and you will see the dependencies of `public` classes. By design `public` classes +should have only edges only to other `public` classes in the dist jar. + + +Execute `jdeps` against `public`, `spark3xx-common` and an *exactly one* parallel +world such as `spark330` +```Bash +$JAVA_HOME/bin/jdeps -v \ + -dotoutput /tmp/jdeps330 \ + -regex '(com|org)\..*\.rapids\..*' \ + public spark3xx-common spark330 +``` + +This will produce three DOT files for each "archive" with directed edges for +a class in the archive to a class either in this or another archive. + +Looking at an output file, e.g. `/tmp/jdeps330/spark3xx-common.dot`, +unfortunately you see that jdeps does not label the source class node but labels +the target class node of an edge. Thus the graph is incorrect as it breaks paths +if a node has both incoming and outgoing edges. + +```Bash +grep 'com.nvidia.spark.rapids.GpuFilterExec\$' spark3xx-common.dot + "com.nvidia.spark.rapids.GpuFilterExec$" -> "com.nvidia.spark.rapids.GpuFilterExec (spark330)"; + "com.nvidia.spark.rapids.GpuOverrides$$anon$204" -> "com.nvidia.spark.rapids.GpuFilterExec$ (spark3xx-common)"; +``` + +So first create and `cd` to some other directory `/tmp/jdep330.processed` to massage +the original jdeps output for further analysis. + +Decorate source nodes from `.dot` with the `()` label given +that the source nodes are guaranteed to be from the ``. + +```Bash +sed 's/"\([^(]*\)"\(\s*->.*;\)/"\1 (public)"\2/' \ + /tmp/jdeps330/public.dot > public.dot +sed 's/"\([^(]*\)"\(\s*->.*;\)/"\1 (spark3xx-common)"\2/' \ + /tmp/jdeps330/spark3xx-common.dot > spark3xx-common.dot +sed 's/"\([^(]*\)"\(\s*->.*;\)/"\1 (spark330)"\2/' \ + /tmp/jdeps330/spark330.dot > spark330.dot +``` + +Next you need to union edges of all three graphs into a single graph to be able +to analyze cross-archive paths. + +```Bash +cat public.dot spark3xx-common.dot spark330.dot | tr '\n' '\r' | \ + sed 's/}\rdigraph "[^"]*" {\r//g' | \ + tr '\r' '\n' > merged.dot +``` + +Now you can finally examine the classes reachable from the class you are trying +to externalize. Using the `dijkstra` tool you can annotate all nodes with the +shortest path distance from the input node, the class you are trying to externalize + +If you see a reachable node out of the parallel world `spark330` then you need +to refactor until the dependencies paths to it are eliminated. + +Focus on the nodes with lowest distance to eliminate dependency on the shim. + +### Examples + +GpuTypeColumnVector needs refactoring prior externalization as of the time +of this writing: +``` +dijkstra -d -p "com.nvidia.spark.rapids.GpuColumnVector (spark3xx-common)" merged.dot | \ + grep '\[dist=' | grep '(spark330)' + "org.apache.spark.sql.rapids.GpuFileSourceScanExec (spark330)" [dist=5.000, + "com.nvidia.spark.rapids.GpuExec (spark330)" [dist=3.000, +... + +RegexReplace could be externalized safely: +``` +$ dijkstra -d -p "org.apache.spark.sql.rapids.RegexReplace (spark3xx-common)" merged.dot | grep '\[dist=' + "org.apache.spark.sql.rapids.RegexReplace (spark3xx-common)" [dist=0.000]; + "org.apache.spark.sql.rapids.RegexReplace$ (spark3xx-common)" [dist=1.000, +``` +because it is self-contained. + +### Estimating the scope of the task + +Dealing with a single class at a time may quickly turn into a tedious task. +You can look at the bigger picture by generating clusters of the strongly +connected components using `sccmap` + +``` +sccmap -d -s merged.dot +2440 nodes, 11897 edges, 637 strong components +``` + +Review the clusters in the output of `sccmap -d merged.dot`. Find the cluster containing +your class and how it is connected to the rest of the clusters in the definition of the digraph +`scc_map`. + +This mechanism can also be used as a guidance for refactoring the code in a more self-contained +packages. \ No newline at end of file From 364374903273354171502a194e096bab2d39c142 Mon Sep 17 00:00:00 2001 From: Hao Zhu <9665750+viadea@users.noreply.github.com> Date: Wed, 14 Sep 2022 16:43:57 -0700 Subject: [PATCH 109/190] [Doc]Doc update for databricks single node cluster[skip ci] (#6542) * Doc update for databricks single node cluster. Signed-off-by: Hao Zhu <9665750+viadea@users.noreply.github.com> * reword Signed-off-by: Hao Zhu <9665750+viadea@users.noreply.github.com> * Update docs/get-started/getting-started-databricks.md Co-authored-by: Sameer Raheja Signed-off-by: Hao Zhu <9665750+viadea@users.noreply.github.com> Co-authored-by: Sameer Raheja --- docs/get-started/getting-started-databricks.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/get-started/getting-started-databricks.md b/docs/get-started/getting-started-databricks.md index ea40fa7c372..07586e9644b 100644 --- a/docs/get-started/getting-started-databricks.md +++ b/docs/get-started/getting-started-databricks.md @@ -130,6 +130,8 @@ cluster. executor with 1 GPU, which is limiting, especially on the reads and writes from Parquet. Set this to 1/(number of cores per executor) which will allow multiple tasks to run in parallel just like the CPU side. Having the value smaller is fine as well. + Note: Please remove the `spark.task.resource.gpu.amount` config for a single-node Databricks + cluster because Spark local mode does not support GPU scheduling. There is an incompatibility between the Databricks specific implementation of adaptive query execution (AQE) and the spark-rapids plugin. In order to mitigate this, @@ -148,7 +150,7 @@ cluster. ![Spark Config](../img/Databricks/sparkconfig.png) - If running Pandas UDFs with GPU support from the plugin, at least three additional options + If running Pandas UDFs with GPU support from the plugin, at least three additional options as below are required. The `spark.python.daemon.module` option is to choose the right daemon module of python for Databricks. On Databricks, the python runtime requires different parameters than the Spark one, so a dedicated python deamon module `rapids.daemon_databricks` is created and should From 11da318bc1bf0f1c41d54cfa7c99809c3b9f3bef Mon Sep 17 00:00:00 2001 From: Remzi Yang <59198230+HaoYang670@users.noreply.github.com> Date: Thu, 15 Sep 2022 09:17:54 +0800 Subject: [PATCH 110/190] Remove the `hasNans` config and update the doc (#6512) * remove the hasnan conf Signed-off-by: remzi <13716567376yh@gmail.com> * update docs Signed-off-by: remzi <13716567376yh@gmail.com> * update compatibility Signed-off-by: remzi <13716567376yh@gmail.com> * remove hasNan from spark2 and notebook Signed-off-by: remzi <13716567376yh@gmail.com> * remove the hasNan config from IT Signed-off-by: remzi <13716567376yh@gmail.com> Signed-off-by: remzi <13716567376yh@gmail.com> --- docs/compatibility.md | 13 --- docs/configs.md | 1 - docs/demo/AWS-EMR/Mortgage-ETL-GPU-EMR.ipynb | 2 +- docs/demo/Databricks/Mortgage-ETL-db.ipynb | 39 +++++---- docs/demo/GCP/Mortgage-ETL.ipynb | 4 +- .../GCP/mortgage-xgboost4j-gpu-scala.ipynb | 1 - docs/get-started/getting-started-gcp.md | 2 +- docs/spark-profiling-tool.md | 1 - docs/supported_ops.md | 8 +- docs/tuning-guide.md | 4 - .../src/main/python/array_test.py | 3 +- integration_tests/src/main/python/data_gen.py | 1 - .../src/main/python/hash_aggregate_test.py | 83 +++++++------------ .../src/main/python/mortgage_test.py | 3 +- .../src/main/python/qa_nightly_select_test.py | 1 - .../src/main/python/spark_session.py | 1 - .../src/main/python/window_function_test.py | 13 ++- .../tests/mortgage/MortgageSparkSuite.scala | 1 - .../nvidia/spark/rapids/GpuOverrides.scala | 51 ++---------- .../com/nvidia/spark/rapids/RapidsConf.scala | 9 -- .../nvidia/spark/rapids/GpuOverrides.scala | 18 +--- .../com/nvidia/spark/rapids/RapidsConf.scala | 9 -- .../spark/rapids/HashAggregatesSuite.scala | 6 -- tools/src/main/resources/supportedExprs.csv | 4 +- 24 files changed, 75 insertions(+), 203 deletions(-) diff --git a/docs/compatibility.md b/docs/compatibility.md index f7a0fe21943..fc1dc559bb2 100644 --- a/docs/compatibility.md +++ b/docs/compatibility.md @@ -69,19 +69,6 @@ worked with plain Spark. Starting from 22.06 this is behavior is enabled by defa the config [`spark.rapids.sql.variableFloatAgg.enabled`](configs.md#sql.variableFloatAgg.enabled). -Additionally, some aggregations on floating point columns that contain `NaN` can produce results -different from Spark in versions prior to Spark 3.1.0. If it is known with certainty that the -floating point columns do not contain `NaN`, set -[`spark.rapids.sql.hasNans`](configs.md#sql.hasNans) to `false` to run GPU enabled aggregations on -them. - -In the case of a distinct count on `NaN` values, prior to Spark 3.1.0, the issue only shows up if - you have different `NaN` values. There are several different binary values that are all considered - to be `NaN` by floating point. The plugin treats all of these as the same value, where as Spark - treats them all as different values. Because this is considered to be rare we do not disable - distinct count for floating point values even if - [`spark.rapids.sql.hasNans`](configs.md#sql.hasNans) is `true`. - ### `0.0` vs `-0.0` Floating point allows zero to be encoded as `0.0` and `-0.0`, but the IEEE standard says that they diff --git a/docs/configs.md b/docs/configs.md index 8fe82cbb26d..b121a0e2d92 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -109,7 +109,6 @@ Name | Description | Default Value spark.rapids.sql.format.parquet.write.enabled|When set to false disables parquet output acceleration|true spark.rapids.sql.format.parquet.writer.int96.enabled|When set to false, disables accelerated parquet write if the spark.sql.parquet.outputTimestampType is set to INT96|true spark.rapids.sql.hasExtendedYearValues|Spark 3.2.0+ extended parsing of years in dates and timestamps to support the full range of possible values. Prior to this it was limited to a positive 4 digit year. The Accelerator does not support the extended range yet. This config indicates if your data includes this extended range or not, or if you don't care about getting the correct values on values with the extended range.|true -spark.rapids.sql.hasNans|Config to indicate if your data has NaN's. Cudf doesn't currently support NaN's properly so you can get corrupt data if you have NaN's in your data and it runs on the GPU.|true spark.rapids.sql.hashOptimizeSort.enabled|Whether sorts should be inserted after some hashed operations to improve output ordering. This can improve output file sizes when saving to columnar formats.|false spark.rapids.sql.improvedFloatOps.enabled|For some floating point operations spark uses one way to compute the value and the underlying cudf implementation can use an improved algorithm. In some cases this can result in cudf producing an answer when spark overflows.|true spark.rapids.sql.improvedTimeOps.enabled|When set to true, some operators will avoid overflowing by converting epoch days directly to seconds without first converting to microseconds|false diff --git a/docs/demo/AWS-EMR/Mortgage-ETL-GPU-EMR.ipynb b/docs/demo/AWS-EMR/Mortgage-ETL-GPU-EMR.ipynb index b30f3073856..3e24c498f23 100644 --- a/docs/demo/AWS-EMR/Mortgage-ETL-GPU-EMR.ipynb +++ b/docs/demo/AWS-EMR/Mortgage-ETL-GPU-EMR.ipynb @@ -39,7 +39,7 @@ " \"driverMemory\": \"4000M\",\n", " \"driverCores\": 2,\n", " \"executorMemory\": \"4000M\",\n", - " \"conf\": {\"spark.sql.adaptive.enabled\": \"false\", \"spark.dynamicAllocation.enabled\": \"false\", \"spark.executor.instances\":2, \"spark.executor.cores\":2, \"spark.rapids.sql.explain\":\"ALL\", \"spark.task.cpus\":\"1\", \"spark.rapids.sql.concurrentGpuTasks\":\"2\", \"spark.rapids.memory.pinnedPool.size\":\"2G\", \"spark.executor.memoryOverhead\":\"2G\", \"spark.executor.extraJavaOptions\":\"-Dai.rapids.cudf.prefer-pinned=true\", \"spark.sql.files.maxPartitionBytes\":\"512m\", \"spark.executor.resource.gpu.amount\":\"1\", \"spark.task.resource.gpu.amount\":\"0.5\", \"spark.plugins\":\"com.nvidia.spark.SQLPlugin\", \"spark.rapids.sql.hasNans\":\"false\", \"spark.rapids.sql.batchSizeBytes\":\"512M\", \"spark.rapids.sql.reader.batchSizeBytes\":\"768M\", \"spark.sql.cache.serializer\" : \"com.nvidia.spark.ParquetCachedBatchSerializer\"}\n", + " \"conf\": {\"spark.sql.adaptive.enabled\": \"false\", \"spark.dynamicAllocation.enabled\": \"false\", \"spark.executor.instances\":2, \"spark.executor.cores\":2, \"spark.rapids.sql.explain\":\"ALL\", \"spark.task.cpus\":\"1\", \"spark.rapids.sql.concurrentGpuTasks\":\"2\", \"spark.rapids.memory.pinnedPool.size\":\"2G\", \"spark.executor.memoryOverhead\":\"2G\", \"spark.executor.extraJavaOptions\":\"-Dai.rapids.cudf.prefer-pinned=true\", \"spark.sql.files.maxPartitionBytes\":\"512m\", \"spark.executor.resource.gpu.amount\":\"1\", \"spark.task.resource.gpu.amount\":\"0.5\", \"spark.plugins\":\"com.nvidia.spark.SQLPlugin\", \"spark.rapids.sql.batchSizeBytes\":\"512M\", \"spark.rapids.sql.reader.batchSizeBytes\":\"768M\", \"spark.sql.cache.serializer\" : \"com.nvidia.spark.ParquetCachedBatchSerializer\"}\n", "}" ] }, diff --git a/docs/demo/Databricks/Mortgage-ETL-db.ipynb b/docs/demo/Databricks/Mortgage-ETL-db.ipynb index 99591b3e461..3fb281ce593 100644 --- a/docs/demo/Databricks/Mortgage-ETL-db.ipynb +++ b/docs/demo/Databricks/Mortgage-ETL-db.ipynb @@ -17,7 +17,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -90,7 +90,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -264,7 +264,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -409,7 +409,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -508,7 +508,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -647,7 +647,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -712,7 +712,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -785,7 +785,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -859,7 +859,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -986,7 +986,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -1081,7 +1081,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -1153,7 +1153,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -1236,7 +1236,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -1312,7 +1312,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -1378,7 +1378,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -1447,7 +1447,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -1527,7 +1527,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -1597,7 +1597,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, @@ -1647,7 +1647,6 @@ "# CPU run, set to false\n", "# spark.conf.set('spark.rapids.sql.enabled', 'false')\n", "spark.conf.set('spark.sql.files.maxPartitionBytes', '1G')\n", - "spark.conf.set(\"spark.rapids.sql.hasNans\", \"false\")\n", "# use GPU to read CSV\n", "spark.conf.set(\"spark.rapids.sql.csv.read.double.enabled\", \"true\")" ] @@ -1668,7 +1667,7 @@ }, { "cell_type": "code", - "execution_count": 0, + "execution_count": null, "metadata": { "application/vnd.databricks.v1+cell": { "inputWidgets": {}, diff --git a/docs/demo/GCP/Mortgage-ETL.ipynb b/docs/demo/GCP/Mortgage-ETL.ipynb index 1b7d8764b4f..06ea701b777 100644 --- a/docs/demo/GCP/Mortgage-ETL.ipynb +++ b/docs/demo/GCP/Mortgage-ETL.ipynb @@ -63,7 +63,6 @@ "conf.set(\"spark.executor.memory\", \"20g\")\n", "conf.set(\"spark.sql.files.maxPartitionBytes\", \"512m\")\n", "conf.set(\"spark.executor.resource.gpu.amount\", \"1\")\n", - "conf.set(\"spark.rapids.sql.hasNans\", \"false\")\n", "conf.set(\"spark.rapids.sql.csv.read.double.enabled\", \"true\")\n", "conf.set('spark.rapids.sql.batchSizeBytes', '512M')\n", "conf.set('spark.rapids.sql.reader.batchSizeBytes', '768M')\n", @@ -850,7 +849,6 @@ "# CPU run, set to false\n", "# spark.conf.set('spark.rapids.sql.enabled', 'false')\n", "spark.conf.set('spark.sql.files.maxPartitionBytes', '1G')\n", - "spark.conf.set(\"spark.rapids.sql.hasNans\", \"false\")\n", "# use GPU to read CSV\n", "spark.conf.set(\"spark.rapids.sql.csv.read.double.enabled\", \"true\")" ] @@ -917,4 +915,4 @@ }, "nbformat": 4, "nbformat_minor": 5 -} \ No newline at end of file +} diff --git a/docs/demo/GCP/mortgage-xgboost4j-gpu-scala.ipynb b/docs/demo/GCP/mortgage-xgboost4j-gpu-scala.ipynb index f5e80e748e5..511dfd256da 100644 --- a/docs/demo/GCP/mortgage-xgboost4j-gpu-scala.ipynb +++ b/docs/demo/GCP/mortgage-xgboost4j-gpu-scala.ipynb @@ -214,7 +214,6 @@ "source": [ "// Build the spark session and data reader as usual\n", "val sparkSession = SparkSession.builder.appName(\"mortgage-gpu\").getOrCreate\n", - "sparkSession.conf.set(\"spark.rapids.sql.hasNans\", false)\n", "val reader = sparkSession.read" ] }, diff --git a/docs/get-started/getting-started-gcp.md b/docs/get-started/getting-started-gcp.md index 7e7eb0332bc..eadb8283587 100644 --- a/docs/get-started/getting-started-gcp.md +++ b/docs/get-started/getting-started-gcp.md @@ -216,7 +216,7 @@ gcloud dataproc jobs submit spark \ --region=$REGION \ --class=com.nvidia.spark.examples.mortgage.GPUMain \ --jars=gs://${GCS_BUCKET}/scala/sample_xgboost_apps-${VERSION}-SNAPSHOT-jar-with-dependencies.jar \ - --properties=spark.executor.cores=${SPARK_NUM_CORES_PER_EXECUTOR},spark.task.cpus=${SPARK_NUM_CORES_PER_EXECUTOR},spark.executor.memory=${SPARK_EXECUTOR_MEMORY},spark.executor.memoryOverhead=${SPARK_EXECUTOR_MEMORYOVERHEAD},spark.executor.resource.gpu.amount=1,spark.task.resource.gpu.amount=1,spark.rapids.sql.hasNans=false,spark.rapids.sql.batchSizeBytes=512M,spark.rapids.sql.reader.batchSizeBytes=768M,spark.rapids.sql.variableFloatAgg.enabled=true,spark.rapids.memory.gpu.pooling.enabled=false,spark.dynamicAllocation.enabled=false \ + --properties=spark.executor.cores=${SPARK_NUM_CORES_PER_EXECUTOR},spark.task.cpus=${SPARK_NUM_CORES_PER_EXECUTOR},spark.executor.memory=${SPARK_EXECUTOR_MEMORY},spark.executor.memoryOverhead=${SPARK_EXECUTOR_MEMORYOVERHEAD},spark.executor.resource.gpu.amount=1,spark.task.resource.gpu.amount=1,spark.rapids.sql.batchSizeBytes=512M,spark.rapids.sql.reader.batchSizeBytes=768M,spark.rapids.sql.variableFloatAgg.enabled=true,spark.rapids.memory.gpu.pooling.enabled=false,spark.dynamicAllocation.enabled=false \ -- \ -dataPath=train::${DATA_PATH}/train \ -dataPath=trans::${DATA_PATH}/eval \ diff --git a/docs/spark-profiling-tool.md b/docs/spark-profiling-tool.md index 224c79b4fa4..659815e11a2 100644 --- a/docs/spark-profiling-tool.md +++ b/docs/spark-profiling-tool.md @@ -243,7 +243,6 @@ Compare Rapids Properties which are set explicitly: |spark.rapids.sql.concurrentGpuTasks |null |2 | |spark.rapids.sql.enabled |false |true | |spark.rapids.sql.explain |null |NOT_ON_GPU| -|spark.rapids.sql.hasNans |null |FALSE | |spark.rapids.sql.incompatibleOps.enabled |null |true | +-------------------------------------------+----------+----------+ ``` diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 64f12141e2b..396297f2565 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -16666,8 +16666,8 @@ are limited. S S S -PS
Input must not contain NaNs and spark.rapids.sql.hasNans must be false.
-PS
Input must not contain NaNs and spark.rapids.sql.hasNans must be false.
+S +S S PS
UTC is only supported TZ for TIMESTAMP
S @@ -16730,8 +16730,8 @@ are limited. S S S -PS
Input must not contain NaNs and spark.rapids.sql.hasNans must be false.
-PS
Input must not contain NaNs and spark.rapids.sql.hasNans must be false.
+S +S S PS
UTC is only supported TZ for TIMESTAMP
S diff --git a/docs/tuning-guide.md b/docs/tuning-guide.md index d0da4695832..130f1c22d06 100644 --- a/docs/tuning-guide.md +++ b/docs/tuning-guide.md @@ -286,10 +286,6 @@ from the main [columnar batch size](#columnar-batch-size) setting. Some transco load CSV files then write Parquet files) need to lower this setting when using large task input partition sizes to avoid GPU out of memory errors. -## Set hasNans flag to False -If your data has float values but doesn't contain NaNs set [`spark.rapids.sql.hasNans`](configs.md#sql.hasNans) to -`false` to get the benefit of running on the GPU - ## Metrics Custom Spark SQL Metrics are available which can help identify performance bottlenecks in a query. diff --git a/integration_tests/src/main/python/array_test.py b/integration_tests/src/main/python/array_test.py index 58c05f7bf4d..43adbed7f8a 100644 --- a/integration_tests/src/main/python/array_test.py +++ b/integration_tests/src/main/python/array_test.py @@ -332,8 +332,7 @@ def test_array_min_max_all_nulls(data_gen): def test_array_concat_decimal(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : debug_df(unary_op_df(spark, ArrayGen(data_gen)).selectExpr( - 'concat(a, a)')), - conf=no_nans_conf) + 'concat(a, a)'))) @pytest.mark.parametrize('data_gen', orderable_gens + nested_gens_sample, ids=idfn) def test_array_repeat_with_count_column(data_gen): diff --git a/integration_tests/src/main/python/data_gen.py b/integration_tests/src/main/python/data_gen.py index 100fa105563..8e423532d7a 100644 --- a/integration_tests/src/main/python/data_gen.py +++ b/integration_tests/src/main/python/data_gen.py @@ -1018,7 +1018,6 @@ def gen_scalars_for_sql(data_gen, count, seed=0, force_no_nulls=False): nested_gens_sample = array_gens_sample + struct_gens_sample_with_decimal128 + map_gens_sample + decimal_128_map_gens ansi_enabled_conf = {'spark.sql.ansi.enabled': 'true'} -no_nans_conf = {'spark.rapids.sql.hasNans': 'false'} legacy_interval_enabled_conf = {'spark.sql.legacy.interval.enabled': 'true'} def copy_and_update(conf, *more_confs): diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index 35763262a77..672a0ecc60e 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -28,29 +28,17 @@ pytestmark = pytest.mark.nightly_resource_consuming_test -_no_nans_float_conf = {'spark.rapids.sql.variableFloatAgg.enabled': 'true', - 'spark.rapids.sql.hasNans': 'false', +_float_conf = {'spark.rapids.sql.variableFloatAgg.enabled': 'true', 'spark.rapids.sql.castStringToFloat.enabled': 'true' } -_no_nans_float_smallbatch_conf = copy_and_update(_no_nans_float_conf, +_float_smallbatch_conf = copy_and_update(_float_conf, {'spark.rapids.sql.batchSizeBytes' : '250'}) -_no_nans_float_conf_partial = copy_and_update(_no_nans_float_conf, +_float_conf_partial = copy_and_update(_float_conf, {'spark.rapids.sql.hashAgg.replaceMode': 'partial'}) -_no_nans_float_conf_final = copy_and_update(_no_nans_float_conf, - {'spark.rapids.sql.hashAgg.replaceMode': 'final'}) - -_nans_float_conf = {'spark.rapids.sql.variableFloatAgg.enabled': 'true', - 'spark.rapids.sql.hasNans': 'true', - 'spark.rapids.sql.castStringToFloat.enabled': 'true' - } - -_nans_float_conf_partial = copy_and_update(_nans_float_conf, - {'spark.rapids.sql.hashAgg.replaceMode': 'partial'}) - -_nans_float_conf_final = copy_and_update(_nans_float_conf, +_float_conf_final = copy_and_update(_float_conf, {'spark.rapids.sql.hashAgg.replaceMode': 'final'}) # The input lists or schemas that are used by StructGen. @@ -239,10 +227,8 @@ def get_params(init_list, marked_params=[]): return list -# Run these tests with in 3 modes, all on the GPU, only partial aggregates on GPU and -# only final aggregates on the GPU with conf for spark.rapids.sql.hasNans set to false/true -_confs = [_no_nans_float_conf, _no_nans_float_smallbatch_conf, _no_nans_float_conf_final, _no_nans_float_conf_partial] -_confs_with_nans = [_nans_float_conf, _nans_float_conf_partial, _nans_float_conf_final] +# Run these tests with in 4 modes, all on the GPU +_confs = [_float_conf, _float_smallbatch_conf, _float_conf_final, _float_conf_partial] # Pytest marker for list of operators allowed to run on the CPU, # esp. useful in partial and final only modes. @@ -257,14 +243,9 @@ def get_params(init_list, marked_params=[]): 'PivotFirst', 'GetArrayItem', 'ShuffleExchangeExec', 'HashPartitioning') params_markers_for_confs = [ - (_no_nans_float_conf_partial, [_excluded_operators_marker]), - (_no_nans_float_conf_final, [_excluded_operators_marker]) -] - -params_markers_for_confs_nans = [ - (_nans_float_conf_partial, [_excluded_operators_marker]), - (_nans_float_conf_final, [_excluded_operators_marker]), - (_nans_float_conf, [_excluded_operators_marker]) + (_float_conf_partial, [_excluded_operators_marker]), + (_float_conf_final, [_excluded_operators_marker]), + (_float_conf, [_excluded_operators_marker]) ] _grpkey_small_decimals = [ @@ -399,7 +380,7 @@ def test_hash_grpby_sum_full_decimal(data_gen, conf): @ignore_order @incompat @pytest.mark.parametrize('data_gen', numeric_gens + decimal_gens + [DecimalGen(precision=36, scale=5)], ids=idfn) -@pytest.mark.parametrize('conf', get_params(_confs_with_nans, params_markers_for_confs_nans), ids=idfn) +@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_reduction_sum(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen, length=100).selectExpr("SUM(a)"), @@ -410,7 +391,7 @@ def test_hash_reduction_sum(data_gen, conf): @incompat @pytest.mark.parametrize('data_gen', numeric_gens + decimal_gens + [ DecimalGen(precision=38, scale=0), DecimalGen(precision=38, scale=-10)], ids=idfn) -@pytest.mark.parametrize('conf', get_params(_confs_with_nans, params_markers_for_confs_nans), ids=idfn) +@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_reduction_sum_full_decimal(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen, length=100).selectExpr("SUM(a)"), @@ -444,7 +425,7 @@ def test_hash_grpby_avg(data_gen, conf): def test_hash_avg_nulls_partial_only(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, data_gen, length=2).agg(f.avg('b')), - conf=_no_nans_float_conf_partial + conf=_float_conf_partial ) @approximate_float @@ -484,7 +465,7 @@ def test_exceptAll(data_gen): @ignore_order(local=True) @incompat @pytest.mark.parametrize('data_gen', _pivot_gens_with_decimals, ids=idfn) -@pytest.mark.parametrize('conf', get_params(_confs_with_nans, params_markers_for_confs_nans), ids=idfn) +@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_grpby_pivot(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, data_gen, length=100) @@ -497,7 +478,7 @@ def test_hash_grpby_pivot(data_gen, conf): @ignore_order(local=True) @incompat @pytest.mark.parametrize('data_gen', _init_list_no_nans, ids=idfn) -@pytest.mark.parametrize('conf', get_params(_confs_with_nans, params_markers_for_confs_nans), ids=idfn) +@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_grpby_pivot_without_nans(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, data_gen, length=100) @@ -510,7 +491,7 @@ def test_hash_grpby_pivot_without_nans(data_gen, conf): @ignore_order(local=True) @incompat @pytest.mark.parametrize('data_gen', _init_list_with_nans_and_no_nans, ids=idfn) -@pytest.mark.parametrize('conf', get_params(_confs_with_nans, params_markers_for_confs_nans), ids=idfn) +@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_multiple_grpby_pivot(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, data_gen, length=100) @@ -523,7 +504,7 @@ def test_hash_multiple_grpby_pivot(data_gen, conf): @ignore_order(local=True) @incompat @pytest.mark.parametrize('data_gen', _init_list_no_nans, ids=idfn) -@pytest.mark.parametrize('conf', get_params(_confs_with_nans, params_markers_for_confs_nans), ids=idfn) +@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_reduction_pivot_without_nans(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, data_gen, length=100) @@ -536,7 +517,7 @@ def test_hash_reduction_pivot_without_nans(data_gen, conf): @ignore_order(local=True) @incompat @pytest.mark.parametrize('data_gen', _init_list_with_nans_and_no_nans, ids=idfn) -@pytest.mark.parametrize('conf', get_params(_confs_with_nans, params_markers_for_confs_nans), ids=idfn) +@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_reduction_pivot_with_nans(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, data_gen, length=100) @@ -559,7 +540,7 @@ def test_hash_pivot_groupby_duplicates_fallback(data_gen): .pivot('b', ['10.0', '10.0']) .agg(f.sum('c')), "PivotFirst", - conf=_nans_float_conf) + conf=_float_conf) _repeat_agg_column_for_collect_op = [ RepeatSeqGen(BooleanGen(), length=15), @@ -702,7 +683,7 @@ def test_hash_groupby_collect_set_on_nested_type(data_gen): @allow_non_gpu("ProjectExec", "SortArray") @pytest.mark.parametrize('data_gen', _gen_data_for_collect_set_op_nested, ids=idfn) def test_hash_groupby_collect_set_on_nested_array_type(data_gen): - conf = copy_and_update(_no_nans_float_conf, { + conf = copy_and_update(_float_conf, { "spark.rapids.sql.castFloatToString.enabled": "true", "spark.rapids.sql.castDecimalToString.enabled": "true", "spark.rapids.sql.expression.SortArray": "false" @@ -744,7 +725,7 @@ def test_hash_reduction_collect_set_on_nested_type(data_gen): @allow_non_gpu("ProjectExec", "SortArray") @pytest.mark.parametrize('data_gen', _gen_data_for_collect_set_op_nested, ids=idfn) def test_hash_reduction_collect_set_on_nested_array_type(data_gen): - conf = copy_and_update(_no_nans_float_conf, { + conf = copy_and_update(_float_conf, { "spark.rapids.sql.castFloatToString.enabled": "true", "spark.rapids.sql.castDecimalToString.enabled": "true", "spark.rapids.sql.expression.SortArray": "false" @@ -1022,7 +1003,7 @@ def test_hash_multiple_filters(data_gen, conf): @pytest.mark.parametrize('data_gen', [_grpkey_floats_with_nan_zero_grouping_keys, _grpkey_doubles_with_nan_zero_grouping_keys], ids=idfn) def test_hash_agg_with_nan_keys(data_gen): - local_conf = copy_and_update(_no_nans_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true'}) + local_conf = copy_and_update(_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true'}) assert_gpu_and_cpu_are_equal_sql( lambda spark : gen_df(spark, data_gen, length=1024), "hash_agg_table", @@ -1042,7 +1023,7 @@ def test_hash_agg_with_nan_keys(data_gen): @pytest.mark.parametrize('data_gen', [_grpkey_structs_with_non_nested_children, _grpkey_nested_structs], ids=idfn) def test_hash_agg_with_struct_keys(data_gen): - local_conf = copy_and_update(_no_nans_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true'}) + local_conf = copy_and_update(_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true'}) assert_gpu_and_cpu_are_equal_sql( lambda spark : gen_df(spark, data_gen, length=1024), "hash_agg_table", @@ -1064,7 +1045,7 @@ def test_hash_agg_with_struct_keys(data_gen): 'ShuffleExchangeExec', 'HashPartitioning') @pytest.mark.parametrize('data_gen', [_grpkey_nested_structs_with_array_child], ids=idfn) def test_hash_agg_with_struct_of_array_fallback(data_gen): - local_conf = copy_and_update(_no_nans_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true'}) + local_conf = copy_and_update(_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true'}) assert_cpu_and_gpu_are_equal_sql_with_capture( lambda spark : gen_df(spark, data_gen, length=100), 'select a, ' @@ -1090,7 +1071,7 @@ def test_count_distinct_with_nan_floats(data_gen): lambda spark : gen_df(spark, data_gen, length=1024), "hash_agg_table", 'select a, count(distinct b) as count_distinct_bees from hash_agg_table group by a', - _no_nans_float_conf) + _float_conf) # TODO: Literal tests @@ -1116,7 +1097,7 @@ def test_first_last_reductions_nested_types(data_gen): @pytest.mark.parametrize('data_gen', _all_basic_gens_with_all_nans_cases, ids=idfn) def test_generic_reductions(data_gen): - local_conf = copy_and_update(_no_nans_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true'}) + local_conf = copy_and_update(_float_conf, {'spark.sql.legacy.allowParameterlessCount': 'true'}) assert_gpu_and_cpu_are_equal_collect( # Coalesce and sort are to make sure that first and last, which are non-deterministic # become deterministic @@ -1161,7 +1142,7 @@ def test_arithmetic_reductions(data_gen): lambda spark : unary_op_df(spark, data_gen).selectExpr( 'sum(a)', 'avg(a)'), - conf = _no_nans_float_conf) + conf = _float_conf) @pytest.mark.parametrize('data_gen', all_basic_gens + decimal_gens + _nested_gens, @@ -1170,7 +1151,7 @@ def test_collect_list_reductions(data_gen): assert_gpu_and_cpu_are_equal_collect( # coalescing because collect_list is not deterministic lambda spark: unary_op_df(spark, data_gen).coalesce(1).selectExpr('collect_list(a)'), - conf=_no_nans_float_conf) + conf=_float_conf) _no_neg_zero_all_basic_gens = [byte_gen, short_gen, int_gen, long_gen, # -0.0 cannot work because of -0.0 == 0.0 in cudf for distinct and @@ -1187,7 +1168,7 @@ def test_collect_list_reductions(data_gen): def test_collect_set_reductions(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr('sort_array(collect_set(a))'), - conf=_no_nans_float_conf) + conf=_float_conf) def test_collect_empty(): assert_gpu_and_cpu_are_equal_collect( @@ -1852,21 +1833,21 @@ def test_min_max_for_single_level_struct(data_gen): lambda spark : gen_df(spark, df_gen), "hash_agg_table", 'select b, max(a) from hash_agg_table group by b', - _no_nans_float_conf) + _float_conf) assert_gpu_and_cpu_are_equal_sql( lambda spark : gen_df(spark, df_gen), "hash_agg_table", 'select max(a) from hash_agg_table', - _no_nans_float_conf) + _float_conf) # test min assert_gpu_and_cpu_are_equal_sql( lambda spark : gen_df(spark, df_gen, length=1024), "hash_agg_table", 'select b, min(a) from hash_agg_table group by b', - _no_nans_float_conf) + _float_conf) assert_gpu_and_cpu_are_equal_sql( lambda spark : gen_df(spark, df_gen, length=1024), "hash_agg_table", 'select min(a) from hash_agg_table', - _no_nans_float_conf) + _float_conf) diff --git a/integration_tests/src/main/python/mortgage_test.py b/integration_tests/src/main/python/mortgage_test.py index 513585b7f19..aed9aa63c85 100644 --- a/integration_tests/src/main/python/mortgage_test.py +++ b/integration_tests/src/main/python/mortgage_test.py @@ -24,5 +24,4 @@ @allow_non_gpu(any=True) def test_mortgage(mortgage): assert_gpu_and_cpu_are_equal_iterator( - lambda spark : mortgage.do_test_query(spark), - conf={'spark.rapids.sql.hasNans': 'false'}) + lambda spark : mortgage.do_test_query(spark)) diff --git a/integration_tests/src/main/python/qa_nightly_select_test.py b/integration_tests/src/main/python/qa_nightly_select_test.py index 286eb0bfc07..ba3414e51fe 100644 --- a/integration_tests/src/main/python/qa_nightly_select_test.py +++ b/integration_tests/src/main/python/qa_nightly_select_test.py @@ -144,7 +144,6 @@ def idfn(val): _qa_conf = { 'spark.rapids.sql.variableFloatAgg.enabled': 'true', - 'spark.rapids.sql.hasNans': 'false', 'spark.rapids.sql.castStringToFloat.enabled': 'true', 'spark.rapids.sql.castFloatToIntegralTypes.enabled': 'true', 'spark.rapids.sql.castFloatToString.enabled': 'true', diff --git a/integration_tests/src/main/python/spark_session.py b/integration_tests/src/main/python/spark_session.py index 4d518f3705e..4705e9c1e5e 100644 --- a/integration_tests/src/main/python/spark_session.py +++ b/integration_tests/src/main/python/spark_session.py @@ -47,7 +47,6 @@ def _from_scala_map(scala_map): 'spark.rapids.sql.fast.sample': 'false', 'spark.rapids.sql.hasExtendedYearValues': 'true', 'spark.rapids.sql.hashOptimizeSort.enabled': 'false', - 'spark.rapids.sql.hasNans': 'true', 'spark.rapids.sql.improvedFloatOps.enabled': 'false', 'spark.rapids.sql.improvedTimeOps.enabled': 'false', 'spark.rapids.sql.incompatibleDateFormats.enabled': 'false', diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py index fb7f9ce1258..886593f464f 100644 --- a/integration_tests/src/main/python/window_function_test.py +++ b/integration_tests/src/main/python/window_function_test.py @@ -135,8 +135,7 @@ ['child_string', StringGen()] ])] -_no_nans_float_conf = {'spark.rapids.sql.variableFloatAgg.enabled': 'true', - 'spark.rapids.sql.hasNans': 'false', +_float_conf = {'spark.rapids.sql.variableFloatAgg.enabled': 'true', 'spark.rapids.sql.castStringToFloat.enabled': 'true' } @@ -555,8 +554,7 @@ def test_window_running_float_decimal_sum(batch_size): @pytest.mark.parametrize('c_gen', lead_lag_data_gens, ids=idfn) @pytest.mark.parametrize('a_b_gen', part_and_order_gens, ids=meta_idfn('partAndOrderBy:')) def test_multi_types_window_aggs_for_rows_lead_lag(a_b_gen, c_gen, batch_size): - conf = {'spark.rapids.sql.batchSizeBytes': batch_size, - 'spark.rapids.sql.hasNans': False} + conf = {'spark.rapids.sql.batchSizeBytes': batch_size} data_gen = [ ('a', RepeatSeqGen(a_b_gen, length=20)), ('b', a_b_gen), @@ -613,7 +611,6 @@ def do_it(spark): @pytest.mark.parametrize('struct_gen', lead_lag_struct_with_arrays_gen, ids=idfn) @pytest.mark.parametrize('a_b_gen', part_and_order_gens, ids=meta_idfn('partAndOrderBy:')) def test_lead_lag_for_structs_with_arrays(a_b_gen, struct_gen): - conf = {'spark.rapids.sql.hasNans': False} data_gen = [ ('a', RepeatSeqGen(a_b_gen, length=20)), ('b', IntegerGen(nullable=False, special_cases=[])), @@ -631,7 +628,7 @@ def do_it(spark): .withColumn('lead_5_c', f.lead('c', 5).over(base_window_spec)) \ .withColumn('lag_1_c', f.lag('c', 1).over(base_window_spec)) - assert_gpu_and_cpu_are_equal_collect(do_it, conf=conf) + assert_gpu_and_cpu_are_equal_collect(do_it) lead_lag_array_data_gens =\ @@ -699,7 +696,7 @@ def do_it(spark): .withColumn('dense_rank_val', f.dense_rank().over(baseWindowSpec)) \ .withColumn('percent_rank_val', f.percent_rank().over(baseWindowSpec)) \ .withColumn('row_num', f.row_number().over(baseWindowSpec)) - assert_gpu_and_cpu_are_equal_collect(do_it, conf={'spark.rapids.sql.hasNans': 'false'}) + assert_gpu_and_cpu_are_equal_collect(do_it) @pytest.mark.skipif(is_before_spark_320(), reason="Only in Spark 3.2.0 is IGNORE NULLS supported for lead and lag by Spark") @@ -1016,7 +1013,7 @@ def test_window_aggs_for_rows_collect_set(): @ignore_order(local=True) @allow_non_gpu("ProjectExec", "SortArray") def test_window_aggs_for_rows_collect_set_nested_array(): - conf = copy_and_update(_no_nans_float_conf, { + conf = copy_and_update(_float_conf, { "spark.rapids.sql.castFloatToString.enabled": "true", "spark.rapids.sql.castDecimalToString.enabled": "true", "spark.rapids.sql.expression.SortArray": "false" diff --git a/integration_tests/src/test/scala/com/nvidia/spark/rapids/tests/mortgage/MortgageSparkSuite.scala b/integration_tests/src/test/scala/com/nvidia/spark/rapids/tests/mortgage/MortgageSparkSuite.scala index dcae3d58653..246ae5a70b9 100644 --- a/integration_tests/src/test/scala/com/nvidia/spark/rapids/tests/mortgage/MortgageSparkSuite.scala +++ b/integration_tests/src/test/scala/com/nvidia/spark/rapids/tests/mortgage/MortgageSparkSuite.scala @@ -41,7 +41,6 @@ class MortgageSparkSuite extends FunSuite { .config("spark.rapids.sql.explain", "ALL") .config("spark.rapids.sql.test.enabled", false) .config("spark.rapids.sql.incompatibleOps.enabled", true) - .config("spark.rapids.sql.hasNans", false) val rapidsShuffle = ShimLoader.getRapidsShuffleManagerClass val prop = System.getProperty("rapids.shuffle.manager.override", "false") if (prop.equalsIgnoreCase("true")) { diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index bd863816eaf..b474bcd816b 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -603,21 +603,6 @@ object GpuOverrides extends Logging { } } - def checkAndTagFloatNanAgg( - op: String, - dataType: DataType, - conf: RapidsConf, - meta: RapidsMeta[_,_]): Unit = { - if (conf.hasNans && isOrContainsFloatingPoint(dataType)) { - meta.willNotWorkOnGpu(s"$op aggregation on floating point columns that can contain NaNs " + - "will compute incorrect results. If it is known that there are no NaNs, set " + - s" ${RapidsConf.HAS_NANS} to false.") - } - } - - private val nanAggPsNote = "Input must not contain NaNs and" + - s" ${RapidsConf.HAS_NANS} must be false." - def expr[INPUT <: Expression]( desc: String, pluginChecks: ExprChecks, @@ -1777,8 +1762,7 @@ object GpuOverrides extends Logging { TypeSig.all, Seq(ParamCheck( "pivotColumn", - (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128) - .withPsNote(Seq(TypeEnum.DOUBLE, TypeEnum.FLOAT), nanAggPsNote), + (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128), TypeSig.all), ParamCheck("valueColumn", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128, @@ -1815,22 +1799,16 @@ object GpuOverrides extends Logging { TypeSig.orderable, Seq(ParamCheck("input", (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + TypeSig.STRUCT) - .nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL) - .withPsNote(Seq(TypeEnum.DOUBLE, TypeEnum.FLOAT), nanAggPsNote), + .nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL), TypeSig.orderable))).asInstanceOf[ExprChecksImpl].contexts ++ ExprChecks.windowOnly( (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL), TypeSig.orderable, Seq(ParamCheck("input", - (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL) - .withPsNote(Seq(TypeEnum.DOUBLE, TypeEnum.FLOAT), nanAggPsNote), + (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL), TypeSig.orderable))).asInstanceOf[ExprChecksImpl].contexts), (max, conf, p, r) => new AggExprMeta[Max](max, conf, p, r) { - override def tagAggForGpu(): Unit = { - val dataType = max.child.dataType - checkAndTagFloatNanAgg("Max", dataType, conf, this) - } }), expr[Min]( "Min aggregate operator", @@ -1842,22 +1820,16 @@ object GpuOverrides extends Logging { TypeSig.orderable, Seq(ParamCheck("input", (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + TypeSig.STRUCT) - .nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL) - .withPsNote(Seq(TypeEnum.DOUBLE, TypeEnum.FLOAT), nanAggPsNote), + .nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL), TypeSig.orderable))).asInstanceOf[ExprChecksImpl].contexts ++ ExprChecks.windowOnly( (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL), TypeSig.orderable, Seq(ParamCheck("input", - (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL) - .withPsNote(Seq(TypeEnum.DOUBLE, TypeEnum.FLOAT), nanAggPsNote), + (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL), TypeSig.orderable))).asInstanceOf[ExprChecksImpl].contexts), (a, conf, p, r) => new AggExprMeta[Min](a, conf, p, r) { - override def tagAggForGpu(): Unit = { - val dataType = a.child.dataType - checkAndTagFloatNanAgg("Min", dataType, conf, this) - } }), expr[Sum]( "Sum aggregate operator", @@ -2169,27 +2141,18 @@ object GpuOverrides extends Logging { ExprChecks.unaryProject( TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL, TypeSig.orderable, - TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL) - .withPsNote(Seq(TypeEnum.DOUBLE, TypeEnum.FLOAT), nanAggPsNote), + TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL), TypeSig.ARRAY.nested(TypeSig.orderable)), (in, conf, p, r) => new UnaryExprMeta[ArrayMin](in, conf, p, r) { - override def tagExprForGpu(): Unit = { - checkAndTagFloatNanAgg("Min", in.dataType, conf, this) - } }), expr[ArrayMax]( "Returns the maximum value in the array", ExprChecks.unaryProject( TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL, TypeSig.orderable, - TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL) - .withPsNote(Seq(TypeEnum.DOUBLE, TypeEnum.FLOAT), nanAggPsNote), + TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL), TypeSig.ARRAY.nested(TypeSig.orderable)), (in, conf, p, r) => new UnaryExprMeta[ArrayMax](in, conf, p, r) { - override def tagExprForGpu(): Unit = { - checkAndTagFloatNanAgg("Max", in.dataType, conf, this) - } - }), expr[ArrayRepeat]( "Returns the array containing the given input value (left) count (right) times", diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index feaf02aebac..dd7fbaf128d 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -582,13 +582,6 @@ object RapidsConf { .booleanConf .createWithDefault(true) - val HAS_NANS = conf("spark.rapids.sql.hasNans") - .doc("Config to indicate if your data has NaN's. Cudf doesn't " + - "currently support NaN's properly so you can get corrupt data if you have NaN's in your " + - "data and it runs on the GPU.") - .booleanConf - .createWithDefault(true) - val NEED_DECIMAL_OVERFLOW_GUARANTEES = conf("spark.rapids.sql.decimalOverflowGuarantees") .doc("FOR TESTING ONLY. DO NOT USE IN PRODUCTION. Please see the decimal section of " + "the compatibility documents for more information on this config.") @@ -1716,8 +1709,6 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val gdsSpillBatchWriteBufferSize: Long = get(GDS_SPILL_BATCH_WRITE_BUFFER_SIZE) - lazy val hasNans: Boolean = get(HAS_NANS) - lazy val needDecimalGuarantees: Boolean = get(NEED_DECIMAL_OVERFLOW_GUARANTEES) lazy val gpuTargetBatchSizeBytes: Long = get(GPU_BATCH_SIZE_BYTES) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 402f31af96b..84819c92ec6 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -694,21 +694,6 @@ object GpuOverrides extends Logging { } } - def checkAndTagFloatNanAgg( - op: String, - dataType: DataType, - conf: RapidsConf, - meta: RapidsMeta[_,_,_]): Unit = { - if (conf.hasNans && isOrContainsFloatingPoint(dataType)) { - meta.willNotWorkOnGpu(s"$op aggregation on floating point columns that can contain NaNs " + - "will compute incorrect results. If it is known that there are no NaNs, set " + - s" ${RapidsConf.HAS_NANS} to false.") - } - } - - private val nanAggPsNote = "Input must not contain NaNs and" + - s" ${RapidsConf.HAS_NANS} must be false." - /** * Helper function specific to ANSI mode for the aggregate functions that should * fallback, since we don't have the same overflow checks that Spark provides in @@ -2218,8 +2203,7 @@ object GpuOverrides extends Logging { TypeSig.all, Seq(ParamCheck( "pivotColumn", - (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128) - .withPsNote(Seq(TypeEnum.DOUBLE, TypeEnum.FLOAT), nanAggPsNote), + (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128), TypeSig.all), ParamCheck("valueColumn", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128, diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 3983fe1868b..21e2f0760d3 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -584,13 +584,6 @@ object RapidsConf { .booleanConf .createWithDefault(true) - val HAS_NANS = conf("spark.rapids.sql.hasNans") - .doc("Config to indicate if your data has NaN's. Cudf doesn't " + - "currently support NaN's properly so you can get corrupt data if you have NaN's in your " + - "data and it runs on the GPU.") - .booleanConf - .createWithDefault(true) - val NEED_DECIMAL_OVERFLOW_GUARANTEES = conf("spark.rapids.sql.decimalOverflowGuarantees") .doc("FOR TESTING ONLY. DO NOT USE IN PRODUCTION. Please see the decimal section of " + "the compatibility documents for more information on this config.") @@ -1774,8 +1767,6 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val gdsSpillBatchWriteBufferSize: Long = get(GDS_SPILL_BATCH_WRITE_BUFFER_SIZE) - lazy val hasNans: Boolean = get(HAS_NANS) - lazy val needDecimalGuarantees: Boolean = get(NEED_DECIMAL_OVERFLOW_GUARANTEES) lazy val gpuTargetBatchSizeBytes: Long = get(GPU_BATCH_SIZE_BYTES) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala index 9e679e61543..84fb9920fe8 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala @@ -31,7 +31,6 @@ import org.apache.spark.sql.types.{DataType, DataTypes} class HashAggregatesSuite extends SparkQueryCompareTestSuite { private def floatAggConf: SparkConf = enableCsvConf() .set(RapidsConf.ENABLE_FLOAT_AGG.key, "true") - .set(RapidsConf.HAS_NANS.key, "false") def replaceHashAggMode(mode: String, conf: SparkConf = new SparkConf()): SparkConf = { // configures whether Plugin will replace certain aggregate exec nodes @@ -65,7 +64,6 @@ class HashAggregatesSuite extends SparkQueryCompareTestSuite { if (batchSize > 0) { makeBatchedBytes(batchSize, conf) } - conf.set(RapidsConf.HAS_NANS.key, "false") conf.set(RapidsConf.ENABLE_FLOAT_AGG.key, "true") testSparkResultsAreEqual(testName, df, conf = conf, repart = repart, @@ -1109,12 +1107,10 @@ class HashAggregatesSuite extends SparkQueryCompareTestSuite { private val nonFinalOnGpuConf = replaceHashAggMode( "partial|partialMerge|partial&partialMerge", enableCsvConf()) .set(RapidsConf.ENABLE_FLOAT_AGG.key, "true") - .set(RapidsConf.HAS_NANS.key, "false") // GPU -> GPU -> GPU -> CPU private val nonPartialOnGpuConf = replaceHashAggMode( "final|partial&partialMerge|partialMerge", enableCsvConf()) .set(RapidsConf.ENABLE_FLOAT_AGG.key, "true") - .set(RapidsConf.HAS_NANS.key, "false") IGNORE_ORDER_ALLOW_NON_GPU_testSparkResultsAreEqualWithCapture( "PartMerge:countDistinct:sum:partOnly", @@ -1622,7 +1618,6 @@ class HashAggregatesSuite extends SparkQueryCompareTestSuite { testName = "Test NormalizeNansAndZeros(Float)", floatWithDifferentKindsOfNansAndZeros, conf = enableCsvConf() - .set(RapidsConf.HAS_NANS.key, "false") .set(RapidsConf.ENABLE_FLOAT_AGG.key, "true")) { frame => frame.groupBy(col("float")).agg(sum(col("int"))) } @@ -1631,7 +1626,6 @@ class HashAggregatesSuite extends SparkQueryCompareTestSuite { testName = "Test NormalizeNansAndZeros(Double)", doubleWithDifferentKindsOfNansAndZeros, conf = enableCsvConf() - .set(RapidsConf.HAS_NANS.key, "false") .set(RapidsConf.ENABLE_FLOAT_AGG.key, "true")) { frame => frame.groupBy(col("double")).agg(sum(col("int"))) } diff --git a/tools/src/main/resources/supportedExprs.csv b/tools/src/main/resources/supportedExprs.csv index 33fb60ec5d7..c9d622f3499 100644 --- a/tools/src/main/resources/supportedExprs.csv +++ b/tools/src/main/resources/supportedExprs.csv @@ -632,10 +632,10 @@ Min,S,`min`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS Min,S,`min`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS Min,S,`min`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS Min,S,`min`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS -PivotFirst,S, ,None,aggregation,pivotColumn,S,S,S,S,S,PS,PS,S,PS,S,S,S,NS,NS,NS,NS,NS,NS +PivotFirst,S, ,None,aggregation,pivotColumn,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS PivotFirst,S, ,None,aggregation,valueColumn,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS PivotFirst,S, ,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,NS,NS -PivotFirst,S, ,None,reduction,pivotColumn,S,S,S,S,S,PS,PS,S,PS,S,S,S,NS,NS,NS,NS,NS,NS +PivotFirst,S, ,None,reduction,pivotColumn,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS PivotFirst,S, ,None,reduction,valueColumn,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS PivotFirst,S, ,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,NS,NS StddevPop,S,`stddev_pop`,None,reduction,input,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA From 887ce6b0dea6c918d2c0a025bf68afc2add99c49 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Wed, 14 Sep 2022 18:49:35 -0700 Subject: [PATCH 111/190] Add missing closing ``` for a code block [skip ci] (#6563) - Fixes formatting bug from #6555 - Addresses linter issues Signed-off-by: Gera Shegalov --- docs/dev/shims.md | 60 ++++++++++++++++++++++++++++++----------------- 1 file changed, 38 insertions(+), 22 deletions(-) diff --git a/docs/dev/shims.md b/docs/dev/shims.md index 58c846383aa..60081b8690e 100644 --- a/docs/dev/shims.md +++ b/docs/dev/shims.md @@ -4,6 +4,7 @@ title: Shim Development nav_order: 4 parent: Developer Overview --- + # Shim Development RAPIDS Accelerator For Apache Spark supports multiple feature version lines of @@ -25,9 +26,11 @@ In the following we provide recipes for typical scenarios addressed by the Shim It's among the easiest issues to resolve. We define a method in SparkShims trait covering a superset of parameters from all versions and call it -``` + +```Scala SparkShimImpl.methodWithDiscrepancies(p_1, ..., p_n) ``` + instead of referencing it directly. Shim implementations (SparkShimImpl) are in charge of dispatching it further to correct version-dependent methods. Moreover, unlike in the below sections conflicts between versions are easily avoided by using different package or class names @@ -36,6 +39,7 @@ for conflicting Shim implementations. ## Base Classes/Traits Changes ### Compile-time issues + Upstream base classes we derive from might be incompatible in the sense that one version requires us to implement/override the method `M` whereas the other prohibits it by marking the base implementation `final`, E.g. `org.apache.spark.sql.catalyst.trees.TreeNode` changes @@ -43,6 +47,7 @@ between Spark 3.1.x and Spark 3.2.x. So instead of deriving from such classes di inject an intermediate trait e.g. `com.nvidia.spark.rapids.shims.ShimExpression` that has a varying source code depending on the Spark version we compile against to overcome this issue as you can see e.g., comparing TreeNode: + 1. [ShimExpression For 3.1.x](https://github.com/NVIDIA/spark-rapids/blob/main/sql-plugin/src/main/pre320-treenode/scala/com/nvidia/spark/rapids/shims/TreeNode.scala#L23) 2. [ShimExpression For 3.2.x](https://github.com/NVIDIA/spark-rapids/blob/main/sql-plugin/src/main/post320-treenode/scala/com/nvidia/spark/rapids/shims/TreeNode.scala#L23) @@ -61,14 +66,16 @@ Spark runtime uses mutable classloaders we can alter after detecting the runtime Using JarURLConnection URLs we create a Parallel World of the current version within the jar, e.g.: Spark 3.0.2's URLs: -``` + +```text jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/ jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/spark3xx-common/ jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/spark302/ ``` Spark 3.2.0's URLs : -``` + +```text jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/ jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/spark3xx-common/ jar:file:/home/spark/rapids-4-spark_2.12-22.10.0.jar!/spark320/ @@ -115,9 +122,11 @@ For examples see: 2. `class ExclusiveModeGpuDiscoveryPlugin` Note that we currently have to manually code up the delegation methods to the tune of: -``` + +```Scala def method(x: SomeThing) = self.method(x) ``` + This could be automatically generated with a simple tool processing the `scalap` output or Scala macros at build/compile time. Pull requests are welcome. @@ -129,6 +138,7 @@ as a dependency for Maven modules/projects dependencies depending on the `dist` module artifact `rapids-4-spark_2.12`. This has two pre-requisites: + 1. The .class file with the bytecode is bitwise-identical among the currently supported Spark versions. To verify this you can inspect the dist jar and check if the class file is under `spark3xx-common` jar entry. If this is not the case then @@ -146,21 +156,22 @@ the `dist` module. While iterating on the PR, it should be sufficient to build against the lowest and highest versions of the supported Spark version range. As of the time of this writing: -```Bash - ./build/buildall --parallel=4 --profile=311,330 --module=dist +```bash +./build/buildall --parallel=4 --profile=311,330 --module=dist ``` However, before submitting the PR execute the full build `--profile=noSnapshots`. Then switch to the parallel-world build dir. -``` + +```bash cd dist/target/parallel-world/ ``` Move the current externalized classes (outside the spark3* parallel worlds) to a dedicated directory, say `public`. -``` +```bash mv org com ai public/ ``` @@ -168,11 +179,11 @@ mv org com ai public/ and you will see the dependencies of `public` classes. By design `public` classes should have only edges only to other `public` classes in the dist jar. - Execute `jdeps` against `public`, `spark3xx-common` and an *exactly one* parallel world such as `spark330` -```Bash -$JAVA_HOME/bin/jdeps -v \ + +```bash +${JAVA_HOME}/bin/jdeps -v \ -dotoutput /tmp/jdeps330 \ -regex '(com|org)\..*\.rapids\..*' \ public spark3xx-common spark330 @@ -186,8 +197,8 @@ unfortunately you see that jdeps does not label the source class node but labels the target class node of an edge. Thus the graph is incorrect as it breaks paths if a node has both incoming and outgoing edges. -```Bash -grep 'com.nvidia.spark.rapids.GpuFilterExec\$' spark3xx-common.dot +```bash +$ grep 'com.nvidia.spark.rapids.GpuFilterExec\$' spark3xx-common.dot "com.nvidia.spark.rapids.GpuFilterExec$" -> "com.nvidia.spark.rapids.GpuFilterExec (spark330)"; "com.nvidia.spark.rapids.GpuOverrides$$anon$204" -> "com.nvidia.spark.rapids.GpuFilterExec$ (spark3xx-common)"; ``` @@ -198,7 +209,7 @@ the original jdeps output for further analysis. Decorate source nodes from `.dot` with the `()` label given that the source nodes are guaranteed to be from the ``. -```Bash +```bash sed 's/"\([^(]*\)"\(\s*->.*;\)/"\1 (public)"\2/' \ /tmp/jdeps330/public.dot > public.dot sed 's/"\([^(]*\)"\(\s*->.*;\)/"\1 (spark3xx-common)"\2/' \ @@ -210,8 +221,9 @@ sed 's/"\([^(]*\)"\(\s*->.*;\)/"\1 (spark330)"\2/' \ Next you need to union edges of all three graphs into a single graph to be able to analyze cross-archive paths. -```Bash -cat public.dot spark3xx-common.dot spark330.dot | tr '\n' '\r' | \ +```bash +cat public.dot spark3xx-common.dot spark330.dot | \ + tr '\n' '\r' | \ sed 's/}\rdigraph "[^"]*" {\r//g' | \ tr '\r' '\n' > merged.dot ``` @@ -229,19 +241,23 @@ Focus on the nodes with lowest distance to eliminate dependency on the shim. GpuTypeColumnVector needs refactoring prior externalization as of the time of this writing: -``` -dijkstra -d -p "com.nvidia.spark.rapids.GpuColumnVector (spark3xx-common)" merged.dot | \ + +```bash +$ dijkstra -d -p "com.nvidia.spark.rapids.GpuColumnVector (spark3xx-common)" merged.dot | \ grep '\[dist=' | grep '(spark330)' "org.apache.spark.sql.rapids.GpuFileSourceScanExec (spark330)" [dist=5.000, "com.nvidia.spark.rapids.GpuExec (spark330)" [dist=3.000, ... +``` RegexReplace could be externalized safely: -``` + +```bash $ dijkstra -d -p "org.apache.spark.sql.rapids.RegexReplace (spark3xx-common)" merged.dot | grep '\[dist=' "org.apache.spark.sql.rapids.RegexReplace (spark3xx-common)" [dist=0.000]; "org.apache.spark.sql.rapids.RegexReplace$ (spark3xx-common)" [dist=1.000, ``` + because it is self-contained. ### Estimating the scope of the task @@ -250,8 +266,8 @@ Dealing with a single class at a time may quickly turn into a tedious task. You can look at the bigger picture by generating clusters of the strongly connected components using `sccmap` -``` -sccmap -d -s merged.dot +```bash +$ sccmap -d -s merged.dot 2440 nodes, 11897 edges, 637 strong components ``` @@ -260,4 +276,4 @@ your class and how it is connected to the rest of the clusters in the definition `scc_map`. This mechanism can also be used as a guidance for refactoring the code in a more self-contained -packages. \ No newline at end of file +packages. From 805ba1e03a7334e8ee69b2f875dfcf8cf80b53c2 Mon Sep 17 00:00:00 2001 From: Nghia Truong Date: Sun, 18 Sep 2022 17:47:17 -0700 Subject: [PATCH 112/190] Fix map gen for orc_write_test.py (#6566) Signed-off-by: Nghia Truong Signed-off-by: Nghia Truong --- integration_tests/src/main/python/orc_write_test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/integration_tests/src/main/python/orc_write_test.py b/integration_tests/src/main/python/orc_write_test.py index 7afe00076a3..7861f5c58ee 100644 --- a/integration_tests/src/main/python/orc_write_test.py +++ b/integration_tests/src/main/python/orc_write_test.py @@ -41,7 +41,9 @@ orc_write_basic_map_gens = [simple_string_to_string_map_gen] + [MapGen(f(nullable=False), f()) for f in [ BooleanGen, ByteGen, ShortGen, IntegerGen, LongGen, FloatGen, DoubleGen, - lambda nullable=True: TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc), nullable=nullable), + # Using timestamps from 1970 to work around a cudf ORC bug + # https://github.com/NVIDIA/spark-rapids/issues/140. + lambda nullable=True: TimestampGen(start=datetime(1970, 1, 1, tzinfo=timezone.utc), nullable=nullable), lambda nullable=True: DateGen(start=date(1590, 1, 1), nullable=nullable), lambda nullable=True: DecimalGen(precision=15, scale=1, nullable=nullable), lambda nullable=True: DecimalGen(precision=36, scale=5, nullable=nullable)]] From 10d1a733dd80e62d2b159dea7bb3d54dcf6baa14 Mon Sep 17 00:00:00 2001 From: Peixin Date: Mon, 19 Sep 2022 08:47:51 +0800 Subject: [PATCH 113/190] Mitigate non-test failure and remove 21.xx premerge support (#6557) Signed-off-by: Peixin Li Signed-off-by: Peixin Li --- jenkins/Jenkinsfile-blossom.premerge | 102 ++++----------------------- 1 file changed, 13 insertions(+), 89 deletions(-) diff --git a/jenkins/Jenkinsfile-blossom.premerge b/jenkins/Jenkinsfile-blossom.premerge index 272827f95e8..392e5717142 100644 --- a/jenkins/Jenkinsfile-blossom.premerge +++ b/jenkins/Jenkinsfile-blossom.premerge @@ -41,9 +41,6 @@ def PROJECT_VER // project version retrieved from 'version-def.sh' to determine def major_ver // major version extracted from project version def minor_ver // minor version extracted from project version def PREMERGE_CI_2_ARGUMENT // argument for 'spark-premerge-build.sh' running from stage of Premerge CI 2 -def TestResult1 // keep ci_1 pytest result files that's in JUnit-XML style -def TestResult2 // keep ci_2 pytest result files that's in JUnit-XML style - def sourcePattern = 'shuffle-plugin/src/main/scala/,udf-compiler/src/main/scala/,' + 'sql-plugin/src/main/java/,sql-plugin/src/main/scala/' @@ -73,12 +70,6 @@ ID_SPARK = 1 ID_INITSCRIPTS = 2 ID_INSTALL = 3 RUNTIME_MAP = [ - '7.3': [ - '7.3.x-gpu-ml-scala2.12', - '3.0.1', - 'init_cudf_udf.sh,init_cuda11_runtime.sh', - '3.0.1' - ], '9.1': [ '9.1.x-gpu-ml-scala2.12', '3.1.2', @@ -183,14 +174,7 @@ pipeline { major_ver = versions[0].toInteger() minor_ver = versions[1].toInteger() - // TODO: remove major version 21.XX pre-merge support - if (major_ver == 21) { - if (minor_ver == 8) { - PREMERGE_CI_2_ARGUMENT = "unit_test" // for '21.08' version - } else if (minor_ver >= 10) { - PREMERGE_CI_2_ARGUMENT = "ci_2" // for '21.10' or later version - } - } else if (major_ver >= 22) { + if (major_ver >= 22) { PREMERGE_CI_2_ARGUMENT = "ci_2" } else { error("Unsupported major version: $major_ver") @@ -253,7 +237,7 @@ pipeline { when { beforeAgent true expression { - db_build && major_ver >= 21 + db_build } } agent { @@ -319,8 +303,7 @@ pipeline { sourcePattern : sourcePattern ]) } finally { - // Save pytest result and publish to Jenkins at last - TestResult1 = stashPytestResult("testResult1") + common.publishPytestResult(this, "${STAGE_NAME}") } } } @@ -333,7 +316,6 @@ pipeline { beforeAgent true beforeOptions true anyOf { - expression { major_ver >= 21 && minor_ver >= 8 } expression { major_ver >= 22 } } } @@ -359,8 +341,7 @@ pipeline { try { sh "$PREMERGE_SCRIPT $PREMERGE_CI_2_ARGUMENT" } finally { - // Save pytest result and publish to Jenkins at last - TestResult2 = stashPytestResult("testResult2") + common.publishPytestResult(this, "${STAGE_NAME}") } } } @@ -368,45 +349,10 @@ pipeline { } } // end of Unit Test stage - stage('DB runtime 7.3') { - when { - beforeAgent true - anyOf { - expression { db_build && major_ver == 21 } - expression { db_build && major_ver == 22 && minor_ver <= 2 } - } - } - - agent { - kubernetes { - label "premerge-ci-db-7.3-${BUILD_NUMBER}" - cloud 'sc-ipp-blossom-prod' - yaml "${IMAGE_DB}" - } - } - environment { - DB_RUNTIME = '7.3' - DATABRICKS_RUNTIME = "${RUNTIME_MAP["$DB_RUNTIME"][ID_RUNTIME]}" - BASE_SPARK_VERSION = "${RUNTIME_MAP["$DB_RUNTIME"][ID_SPARK]}" - BASE_SPARK_VERSION_TO_INSTALL_DATABRICKS_JARS = "${RUNTIME_MAP["$DB_RUNTIME"][ID_INSTALL]}" - INIT_SCRIPTS = getInitScripts("$INIT_SCRIPTS_DIR", - "${RUNTIME_MAP["$DB_RUNTIME"][ID_INITSCRIPTS]}") - } - steps { - script { - timeout(time: 5, unit: 'HOURS') { - unstash "source_tree" - databricksBuild() - } - } - } - } // end of DB runtime 7.3 - stage('DB runtime 9.1') { when { beforeAgent true anyOf { - expression { db_build && major_ver == 21 && minor_ver >= 12} expression { db_build && major_ver >= 22 } } } @@ -511,21 +457,6 @@ pipeline { deleteDockerTempTag("${PREMERGE_TAG}") // clean premerge temp image } } - - // Collect test results - if (TestResult1) { - unstash "testResult1" - } - if (TestResult2) { - unstash "testResult2" - } - - // Record test results to Jenkins - if (TestResult1 || TestResult2) { - junit skipPublishingChecks: true, testResults: '**/TEST-pytest*.xml' - } else { - echo 'Skip recording test results' - } } } } @@ -577,13 +508,17 @@ void databricksBuild() { stage("Test agaist $SPARK_MAJOR DB") { script { container('cpu') { - withCredentials([file(credentialsId: 'SPARK_DATABRICKS_PRIVKEY', variable: 'DATABRICKS_PRIVKEY')]) { - def TEST_PARAMS = " -w $DATABRICKS_HOST -t $DATABRICKS_TOKEN -c $CLUSTER_ID" + + try { + withCredentials([file(credentialsId: 'SPARK_DATABRICKS_PRIVKEY', variable: 'DATABRICKS_PRIVKEY')]) { + def TEST_PARAMS = " -w $DATABRICKS_HOST -t $DATABRICKS_TOKEN -c $CLUSTER_ID" + " -p $DATABRICKS_PRIVKEY -l ./jenkins/databricks/test.sh -v $BASE_SPARK_VERSION -d /home/ubuntu/test.sh" - if (params.SPARK_CONF) { - TEST_PARAMS += " -f ${params.SPARK_CONF}" + if (params.SPARK_CONF) { + TEST_PARAMS += " -f ${params.SPARK_CONF}" + } + sh "python3 ./jenkins/databricks/run-tests.py $TEST_PARAMS" } - sh "python3 ./jenkins/databricks/run-tests.py $TEST_PARAMS" + } finally { + common.publishPytestResult(this, "${STAGE_NAME}") } } } @@ -637,17 +572,6 @@ void abortDupBuilds() { } } -String stashPytestResult(String name) { - def result = sh(returnStdout: true, script: "find . -name 'TEST-pytest*.xml' -exec ls -l {} \\;") - echo result - - if (result) { - stash(name: name, includes: "**/TEST-pytest*.xml") - } - - return result -} - void checkoutCode(String url, String sha) { checkout( changelog: false, From 226c4802e9efb30975fc3e77e0aa4bbe7357105f Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 19 Sep 2022 08:24:05 -0500 Subject: [PATCH 114/190] Allow percent_rank to not need an entire group in memory [databricks] (#6550) Signed-off-by: Robert (Bobby) Evans --- .../src/main/python/window_function_test.py | 23 + .../nvidia/spark/rapids/GpuWindowExec.scala | 505 +++++++++++++++++- .../spark/rapids/GpuWindowExpression.scala | 206 +++++-- .../spark/sql/rapids/AggregateFunctions.scala | 8 +- 4 files changed, 681 insertions(+), 61 deletions(-) diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py index 886593f464f..c7564670428 100644 --- a/integration_tests/src/main/python/window_function_test.py +++ b/integration_tests/src/main/python/window_function_test.py @@ -699,6 +699,29 @@ def do_it(spark): assert_gpu_and_cpu_are_equal_collect(do_it) +def test_percent_rank_no_part_multiple_batches(): + data_gen = [('a', long_gen)] + # The goal of this is to have multiple batches so we can verify that the code + # is working properly, but not so large that it takes forever to run. + baseWindowSpec = Window.orderBy('a') + + def do_it(spark): + return gen_df(spark, data_gen, length=8000) \ + .withColumn('percent_rank_val', f.percent_rank().over(baseWindowSpec)) + assert_gpu_and_cpu_are_equal_collect(do_it, conf = {'spark.rapids.sql.batchSizeBytes': '100'}) + +def test_percent_rank_single_part_multiple_batches(): + data_gen = [('a', long_gen)] + # The goal of this is to have multiple batches so we can verify that the code + # is working properly, but not so large that it takes forever to run. + baseWindowSpec = Window.partitionBy('b').orderBy('a') + + def do_it(spark): + return gen_df(spark, data_gen, length=8000) \ + .withColumn('b', f.lit(1)) \ + .withColumn('percent_rank_val', f.percent_rank().over(baseWindowSpec)) + assert_gpu_and_cpu_are_equal_collect(do_it, conf = {'spark.rapids.sql.batchSizeBytes': '100'}) + @pytest.mark.skipif(is_before_spark_320(), reason="Only in Spark 3.2.0 is IGNORE NULLS supported for lead and lag by Spark") @allow_non_gpu('WindowExec', 'Alias', 'WindowExpression', 'Lead', 'Literal', 'WindowSpecDefinition', 'SpecifiedWindowFrame') @ignore_order(local=True) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala index 9bf33829881..667ddfcdcfd 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala @@ -29,7 +29,7 @@ import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeReference, AttributeSeq, AttributeSet, CurrentRow, Expression, FrameType, NamedExpression, RangeFrame, RowFrame, SortOrder, UnboundedPreceding} +import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeReference, AttributeSeq, AttributeSet, CurrentRow, Expression, FrameType, NamedExpression, RangeFrame, RowFrame, SortOrder, UnboundedFollowing, UnboundedPreceding} import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.window.WindowExec @@ -132,20 +132,9 @@ abstract class GpuBaseWindowExecMeta[WindowExecType <: SparkPlan] (windowExec: W remappedWindowOps } - // When we support multiple ways to avoid batching the input data like with - // https://github.com/NVIDIA/spark-rapids/issues/1860 we should check if all of - // the operations fit into one of the supported groups and then split them up into - // multiple execs if they do, so that we can avoid batching on all of them. - val allBatchedRunning = fixedUpWindowOps.forall { + val allBatched = fixedUpWindowOps.forall { case GpuAlias(GpuWindowExpression(func, spec), _) => - val isRunningFunc = func match { - case _: GpuBatchedRunningWindowWithFixer => true - case GpuAggregateExpression(_: GpuBatchedRunningWindowWithFixer, _, _, _ , _) => true - case _ => false - } - // Running windows are limited to row based queries with a few changes we could make this - // work for range based queries too https://github.com/NVIDIA/spark-rapids/issues/2708 - isRunningFunc && GpuWindowExec.isRunningWindow(spec) + GpuWindowExec.isBatchedFunc(func, spec) case GpuAlias(_: AttributeReference, _) | _: AttributeReference => // We allow pure result columns for running windows true @@ -162,12 +151,14 @@ abstract class GpuBaseWindowExecMeta[WindowExecType <: SparkPlan] (windowExec: W childPlans.head.convertIfNeeded() } - val windowExpr = if (allBatchedRunning) { - GpuRunningWindowExec( - fixedUpWindowOps, + val windowExpr = if (allBatched) { + val batchedOps = GpuWindowExec.splitBatchedOps(fixedUpWindowOps) + batchedOps.getWindowExec( partitionSpec.map(_.convertToGpu()), orderSpec.map(_.convertToGpu().asInstanceOf[SortOrder]), - input)(getPartitionSpecs, getOrderSpecs) + input, + getPartitionSpecs, + getOrderSpecs) } else { GpuWindowExec( fixedUpWindowOps, @@ -231,10 +222,71 @@ class GpuWindowExecMeta(windowExec: WindowExec, override def getResultColumnsOnly: Boolean = resultColumnsOnly } +case class BatchedOps(running: Seq[NamedExpression], + unboundedToUnbounded: Seq[NamedExpression], + passThrough: Seq[NamedExpression]) { + def getRunningExpressionsWithPassthrough: Seq[NamedExpression] = + passThrough ++ running + + private def getRunningWindowExec( + gpuPartitionSpec: Seq[Expression], + gpuOrderSpec: Seq[SortOrder], + child: SparkPlan, + cpuPartitionSpec: Seq[Expression], + cpuOrderSpec: Seq[SortOrder]): GpuExec = + GpuRunningWindowExec( + getRunningExpressionsWithPassthrough, + gpuPartitionSpec, + gpuOrderSpec, + child)(cpuPartitionSpec, cpuOrderSpec) + + private def getDoublePassWindowExec( + gpuPartitionSpec: Seq[Expression], + gpuOrderSpec: Seq[SortOrder], + child: SparkPlan, + cpuPartitionSpec: Seq[Expression], + cpuOrderSpec: Seq[SortOrder]): GpuExec = + GpuCachedDoublePassWindowExec( + getDoublePassExpressionsWithRunningAsPassthrough, + gpuPartitionSpec, + gpuOrderSpec, + child)(cpuPartitionSpec, cpuOrderSpec) + + def getWindowExec( + gpuPartitionSpec: Seq[Expression], + gpuOrderSpec: Seq[SortOrder], + child: SparkPlan, + cpuPartitionSpec: Seq[Expression], + cpuOrderSpec: Seq[SortOrder]): GpuExec = { + // The order of these matter so we can pass the output of the first through the second one + if (hasRunning) { + val running = getRunningWindowExec(gpuPartitionSpec, gpuOrderSpec, child, + cpuPartitionSpec, cpuOrderSpec) + if (hasDoublePass) { + getDoublePassWindowExec(gpuPartitionSpec, gpuOrderSpec, running, + cpuPartitionSpec, cpuOrderSpec) + } else { + running + } + } else { + getDoublePassWindowExec(gpuPartitionSpec, gpuOrderSpec, child, + cpuPartitionSpec, cpuOrderSpec) + } + } + + def hasRunning: Boolean = running.nonEmpty + + def getDoublePassExpressionsWithRunningAsPassthrough: Seq[NamedExpression] = + passThrough ++ unboundedToUnbounded ++ running.map(_.toAttribute) + + def hasDoublePass: Boolean = unboundedToUnbounded.nonEmpty +} + object GpuWindowExec extends Arm { /** * As a part of `splitAndDedup` the dedup part adds a layer of indirection. This attempts to * remove that layer of indirection. + * * @param windowOps the windowOps output of splitAndDedup * @param post the post output of splitAndDedup * @return a version of windowOps that has removed as many un-needed temp aliases as possible. @@ -381,6 +433,60 @@ object GpuWindowExec extends Arm { GpuSpecialFrameBoundary(UnboundedPreceding), GpuLiteral(value, _))) if value == 0 => true case _ => false } + + def isUnboundedToUnboundedWindow(spec: GpuWindowSpecDefinition): Boolean = spec match { + case GpuWindowSpecDefinition(_, _, GpuSpecifiedWindowFrame(_, + GpuSpecialFrameBoundary(UnboundedPreceding), + GpuSpecialFrameBoundary(UnboundedFollowing))) => true + case _ => false + } + + def isBatchedRunningFunc(func: Expression, spec: GpuWindowSpecDefinition): Boolean = { + val isSpecOkay = isRunningWindow(spec) + val isFuncOkay = func match { + case _: GpuBatchedRunningWindowWithFixer => true + case GpuAggregateExpression(_: GpuBatchedRunningWindowWithFixer, _, _, _ , _) => true + case _ => false + } + isSpecOkay && isFuncOkay + } + + def isBatchedUnboundedToUnboundedFunc(func: Expression, spec: GpuWindowSpecDefinition): Boolean = + func match { + case _: GpuUnboundToUnboundWindowWithFixer + if GpuWindowExec.isUnboundedToUnboundedWindow(spec) => true + case GpuAggregateExpression(_: GpuUnboundToUnboundWindowWithFixer, _, _, _ , _) + if GpuWindowExec.isUnboundedToUnboundedWindow(spec) => true + case _ => false + } + + def isBatchedFunc(func: Expression, spec: GpuWindowSpecDefinition): Boolean = + isBatchedRunningFunc(func, spec) || isBatchedUnboundedToUnboundedFunc(func, spec) + + def splitBatchedOps(windowOps: Seq[NamedExpression]): BatchedOps = { + val running = ArrayBuffer[NamedExpression]() + val doublePass = ArrayBuffer[NamedExpression]() + val passThrough = ArrayBuffer[NamedExpression]() + windowOps.foreach { + case expr@GpuAlias(GpuWindowExpression(func, spec), _) => + if (isBatchedRunningFunc(func, spec)) { + running.append(expr) + } else if (isBatchedUnboundedToUnboundedFunc(func, spec)) { + doublePass.append(expr) + } else { + throw new IllegalArgumentException( + s"Found unexpected expression $expr in window exec ${expr.getClass}") + } + case expr@(GpuAlias(_: AttributeReference, _) | _: AttributeReference) => + passThrough.append(expr) + case other => + // This should only happen if we did something wrong in splitting/deduping + // the window expressions. + throw new IllegalArgumentException( + s"Found unexpected expression $other in window exec ${other.getClass}") + } + BatchedOps(running, doublePass, passThrough) + } } trait GpuWindowBaseExec extends ShimUnaryExecNode with GpuExec { @@ -392,9 +498,17 @@ trait GpuWindowBaseExec extends ShimUnaryExecNode with GpuExec { import GpuMetric._ - override lazy val additionalMetrics: Map[String, GpuMetric] = Map( - OP_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_OP_TIME) - ) + def needsSpillMetrics: Boolean = false + + override lazy val additionalMetrics: Map[String, GpuMetric] = { + val required = Map( + OP_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_OP_TIME)) + if (needsSpillMetrics) { + required ++ spillMetrics + } else { + required + } + } override def output: Seq[Attribute] = windowOps.map(_.toAttribute) @@ -1108,8 +1222,8 @@ class GpuWindowIterator( } } -object GpuRunningWindowIterator extends Arm { - private def cudfAnd(lhs: cudf.ColumnVector, +object GpuBatchedWindowIterator extends Arm { + def cudfAnd(lhs: cudf.ColumnVector, rhs: cudf.ColumnVector): cudf.ColumnVector = { withResource(lhs) { lhs => withResource(rhs) { rhs => @@ -1118,7 +1232,22 @@ object GpuRunningWindowIterator extends Arm { } } - private def arePartsEqual( + def areRowPartsEqual( + scalars: Seq[Scalar], + columns: Seq[cudf.ColumnVector], + indexes: Seq[Int]): Array[Boolean] = { + withResourceIfAllowed(arePartsEqual(scalars, columns)) { + case scala.util.Right(ret) => Seq.fill(indexes.length)(ret).toArray + case scala.util.Left(column) => + indexes.map { index => + withResource(column.getScalarElement(index)) { scalar => + scalar.isValid && scalar.getBoolean + } + }.toArray + } + } + + def arePartsEqual( scalars: Seq[Scalar], columns: Seq[cudf.ColumnVector]): Either[cudf.ColumnVector, Boolean] = { if (scalars.length != columns.length) { @@ -1159,7 +1288,7 @@ object GpuRunningWindowIterator extends Arm { } } - private def areOrdersEqual( + def areOrdersEqual( scalars: Seq[Scalar], columns: Seq[cudf.ColumnVector], partsEqual: Either[cudf.ColumnVector, Boolean]): Either[cudf.ColumnVector, Boolean] = { @@ -1185,7 +1314,7 @@ object GpuRunningWindowIterator extends Arm { } } - private def getScalarRow(index: Int, columns: Seq[cudf.ColumnVector]): Array[Scalar] = + def getScalarRow(index: Int, columns: Seq[cudf.ColumnVector]): Array[Scalar] = columns.map(_.getScalarElement(index)).toArray } @@ -1205,7 +1334,7 @@ class GpuRunningWindowIterator( numOutputBatches: GpuMetric, numOutputRows: GpuMetric, opTime: GpuMetric) extends Iterator[ColumnarBatch] with BasicWindowCalc { - import GpuRunningWindowIterator._ + import GpuBatchedWindowIterator._ TaskContext.get().addTaskCompletionListener[Unit](_ => close()) override def isRunningBatched: Boolean = true @@ -1344,6 +1473,12 @@ class GpuRunningWindowIterator( } } +/** + * This allows for batches of data to be processed without needing them to correspond to + * the partition by boundaries, but only for window operations that are unbounded preceding + * to current row (Running Window). This works because a small amount of data can be saved + * from a previous batch and used to update the current batch. + */ case class GpuRunningWindowExec( windowOps: Seq[NamedExpression], gpuPartitionSpec: Seq[Expression], @@ -1370,6 +1505,322 @@ case class GpuRunningWindowExec( } } +class FixerPair(op: GpuUnboundToUnboundWindowWithFixer) extends AutoCloseable { + var fixing: BatchedUnboundedToUnboundedWindowFixer = op.newUnboundedToUnboundedFixer + var collecting: BatchedUnboundedToUnboundedWindowFixer = op.newUnboundedToUnboundedFixer + + def updateState(scalar: Scalar): Unit = { + collecting.updateState(scalar) + } + + def fixUp(samePartitionMask: Either[cudf.ColumnVector, Boolean], + column: cudf.ColumnVector): cudf.ColumnVector = + fixing.fixUp(samePartitionMask, column) + + def swap(): Unit = { + val tmp = fixing + tmp.reset() + fixing = collecting + collecting = tmp + } + + override def close(): Unit = { + fixing.close() + collecting.close() + } +} + +/** + * An iterator that can do aggregations on window queries that need a small amount of + * information from all of the batches to update the result in a second pass. It does this by + * having the aggregations be instances of GpuUnboundToUnboundWindowWithFixer + * which can fix up the window output for unbounded to unbounded windows. + * Because of this there is no requirement about how the input data is batched, but it must + * be sorted by both partitioning and ordering. + */ +class GpuCachedDoublePassWindowIterator( + input: Iterator[ColumnarBatch], + override val boundWindowOps: Seq[GpuExpression], + override val boundPartitionSpec: Seq[GpuExpression], + override val boundOrderSpec: Seq[SortOrder], + val outputTypes: Array[DataType], + numOutputBatches: GpuMetric, + numOutputRows: GpuMetric, + opTime: GpuMetric, + spillCallback: SpillCallback) extends Iterator[ColumnarBatch] with BasicWindowCalc { + import GpuBatchedWindowIterator._ + TaskContext.get().addTaskCompletionListener[Unit](_ => close()) + + override def isRunningBatched: Boolean = true + + private var readyForPostProcessing = mutable.Queue[SpillableColumnarBatch]() + private var firstPassProcessed = mutable.Queue[SpillableColumnarBatch]() + // This should only ever be cached in between calls to `hasNext` and `next`. + // This is just to let us filter out empty batches. + private var waitingForFirstPass: Option[ColumnarBatch] = None + private var lastPartsCaching: Array[Scalar] = Array.empty + private var lastPartsProcessing: Array[Scalar] = Array.empty + private var isClosed: Boolean = false + + private def saveLastPartsCaching(newLastParts: Array[Scalar]): Unit = { + lastPartsCaching.foreach(_.close()) + lastPartsCaching = newLastParts + } + + def close(): Unit = { + if (!isClosed) { + isClosed = true + + fixerIndexMap.values.foreach(_.close()) + + saveLastPartsCaching(Array.empty) + + lastPartsProcessing.foreach(_.close()) + lastPartsProcessing = Array.empty + + firstPassProcessed.foreach(_.close()) + firstPassProcessed = mutable.Queue[SpillableColumnarBatch]() + + readyForPostProcessing.foreach(_.close()) + readyForPostProcessing = mutable.Queue[SpillableColumnarBatch]() + + waitingForFirstPass.foreach(_.close()) + waitingForFirstPass = None + } + } + + private lazy val fixerIndexMap: Map[Int, FixerPair] = + boundWindowOps.zipWithIndex.flatMap { + case (GpuAlias(GpuWindowExpression(func, _), _), index) => + func match { + case f: GpuUnboundToUnboundWindowWithFixer => + Some((index, new FixerPair(f))) + case GpuAggregateExpression(f: GpuUnboundToUnboundWindowWithFixer, _, _, _, _) => + Some((index, new FixerPair(f))) + case _ => None + } + case _ => None + }.toMap + + // Do any post processing fixup for the batch before it is sent out the door + def postProcess(cb: ColumnarBatch): ColumnarBatch = { + val computedWindows = GpuColumnVector.extractBases(cb) + withResource(GpuProjectExec.project(cb, boundPartitionSpec)) { parts => + val partColumns = GpuColumnVector.extractBases(parts) + withResourceIfAllowed(arePartsEqual(lastPartsProcessing, partColumns)) { samePartitionMask => + withResource(ArrayBuffer[cudf.ColumnVector]()) { newColumns => + boundWindowOps.indices.foreach { idx => + val column = computedWindows(idx) + fixerIndexMap.get(idx) match { + case Some(fixer) => + closeOnExcept(fixer.fixUp(samePartitionMask, column)) { finalOutput => + newColumns += finalOutput + } + case None => + newColumns += column.incRefCount() + } + } + makeBatch(newColumns) + } + } + } + } + + def makeBatch(columns: Seq[cudf.ColumnVector]): ColumnarBatch = { + withResource(new cudf.Table(columns: _*)) { table => + GpuColumnVector.from(table, outputTypes) + } + } + + def swapFirstPassIsReadyForPost(): Unit = { + // Swap the caching so it is ready to be used for updating + fixerIndexMap.values.foreach(_.swap()) + + // Swap the parts so we know what mask to use for updating + lastPartsProcessing.foreach(_.close()) + lastPartsProcessing = lastPartsCaching + lastPartsCaching = Array.empty + + // Swap the queues so we are ready to dequeue the data + // Before we swap this must be empty, or we are dropping data... + assert(readyForPostProcessing.isEmpty) + readyForPostProcessing = firstPassProcessed + firstPassProcessed = mutable.Queue[SpillableColumnarBatch]() + } + + // The last batch was already processed so everything in processed needs to be moved to + // readyForPostProcessing + def lastBatch(): Unit = swapFirstPassIsReadyForPost() + + private def cacheInFixers(computedWindows: Array[cudf.ColumnVector], + fixers: Map[Int, FixerPair], + rowIndex: Int): Unit = + fixers.foreach { + case (columnIndex, fixer) => + val column = computedWindows(columnIndex) + withResource(column.getScalarElement(rowIndex)) { scalar => + fixer.updateState(scalar) + } + } + + def saveBatchForPostProcessing(batch: ColumnarBatch): Unit = { + firstPassProcessed += SpillableColumnarBatch(batch, SpillPriorities.ACTIVE_ON_DECK_PRIORITY, + spillCallback) + } + + def saveBatchForPostProcessing(basic: Array[cudf.ColumnVector]): Unit = { + closeOnExcept(makeBatch(basic)) { batch => + saveBatchForPostProcessing(batch) + } + } + + // Compute the window operation and cache/update caches as needed. + def firstPassComputeAndCache(cb: ColumnarBatch): Unit = { + val fixers = fixerIndexMap + val numRows = cb.numRows() + withResource(computeBasicWindow(cb)) { basic => + withResource(GpuProjectExec.project(cb, boundPartitionSpec)) { parts => + val partColumns = GpuColumnVector.extractBases(parts) + + val firstLastEqual = areRowPartsEqual(lastPartsCaching, partColumns, Seq(0, numRows - 1)) + val firstEqual = firstLastEqual(0) + val lastEqual = firstLastEqual(1) + if (firstEqual) { + // This batch is a continuation of the previous batch so we need to update the + // fixer with info from it. + // This assumes that the window is unbounded to unbounded. We will need to update + // APIs in the future and rename things if we want to support more than this. + cacheInFixers(basic, fixers, 0) + } + + // If the last part entry in this batch does not match the last entry in the previous batch + // then we need to start post-processing the batches. + if (!lastEqual) { + // We swap the fixers and queues so we are ready to start on the next partition by group + swapFirstPassIsReadyForPost() + // Collect/Cache the needed info from the end of this batch + cacheInFixers(basic, fixers, numRows - 1) + saveLastPartsCaching(getScalarRow(numRows - 1, partColumns)) + + if (firstEqual) { + // Process the batch now, but it will only be for the first part of the batch + // the last part may need to be fixed again, so put it into the queue for + // when the next round finishes. + val processedBatch = withResource(makeBatch(basic)) { basicBatch => + postProcess(basicBatch) + } + closeOnExcept(processedBatch) { processedBatch => + saveBatchForPostProcessing(processedBatch) + } + } else { + // We split on a partition boundary, so just need to save it + saveBatchForPostProcessing(basic) + } + } else { + // No need to save the parts, it was equal... + saveBatchForPostProcessing(basic) + } + } + } + } + + private def cacheBatchIfNeeded(): Unit = { + while (waitingForFirstPass.isEmpty && input.hasNext) { + closeOnExcept(input.next()) { cb => + if (cb.numRows() > 0) { + waitingForFirstPass = Some(cb) + } else { + cb.close() + } + } + } + } + + override def hasNext: Boolean = { + if (readyForPostProcessing.nonEmpty || firstPassProcessed.nonEmpty) { + true + } else { + cacheBatchIfNeeded() + waitingForFirstPass.isDefined + } + } + + override def next(): ColumnarBatch = { + if (!hasNext) { + throw new NoSuchElementException() + } + while (readyForPostProcessing.isEmpty) { + // Keep reading and processing data until we have something to output + cacheBatchIfNeeded() + if (waitingForFirstPass.isEmpty) { + lastBatch() + } else { + withResource(waitingForFirstPass.get) { cb => + waitingForFirstPass = None + withResource( + new NvtxWithMetrics("DoubleBatchedWindow_PRE", NvtxColor.CYAN, opTime)) { _ => + firstPassComputeAndCache(cb) + } + } + } + } + val cb = withResource(readyForPostProcessing.dequeue()) { sb => + sb.getColumnarBatch() + } + withResource(cb) { cb => + val ret = withResource( + new NvtxWithMetrics("DoubleBatchedWindow_POST", NvtxColor.BLUE, opTime)) { _ => + postProcess(cb) + } + numOutputBatches += 1 + numOutputRows += ret.numRows() + ret + } + } +} + +/** + * This allows for batches of data to be processed without needing them to correspond to + * the partition by boundaries. This is similar to GpuRunningWindowExec, but for operations + * that need a small amount of information from all of the batches associated with a partition + * instead of just the previous batch. It does this by processing a batch, collecting and + * updating a small cache of information about the last partition in the batch, and then putting + * that batch into a form that would let it be spilled if needed. A batch is released when the + * last partition key in the batch is fully processed. Before it is released it will be updated + * to include any information needed from the cached data. + * + * Currently this only works for unbounded to unbounded windows, but could be extended to more. + */ +case class GpuCachedDoublePassWindowExec( + windowOps: Seq[NamedExpression], + gpuPartitionSpec: Seq[Expression], + gpuOrderSpec: Seq[SortOrder], + child: SparkPlan)( + override val cpuPartitionSpec: Seq[Expression], + override val cpuOrderSpec: Seq[SortOrder]) extends GpuWindowBaseExec { + + override def needsSpillMetrics: Boolean = true + + override def otherCopyArgs: Seq[AnyRef] = cpuPartitionSpec :: cpuOrderSpec :: Nil + + override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { + val numOutputBatches = gpuLongMetric(GpuMetric.NUM_OUTPUT_BATCHES) + val numOutputRows = gpuLongMetric(GpuMetric.NUM_OUTPUT_ROWS) + val opTime = gpuLongMetric(GpuMetric.OP_TIME) + val spillCallback = GpuMetric.makeSpillCallback(allMetrics) + + val boundWindowOps = GpuBindReferences.bindGpuReferences(windowOps, child.output) + val boundPartitionSpec = GpuBindReferences.bindGpuReferences(gpuPartitionSpec, child.output) + val boundOrderSpec = GpuBindReferences.bindReferences(gpuOrderSpec, child.output) + + child.executeColumnar().mapPartitions { iter => + new GpuCachedDoublePassWindowIterator(iter, boundWindowOps, boundPartitionSpec, + boundOrderSpec, output.map(_.dataType).toArray, numOutputBatches, numOutputRows, opTime, + spillCallback) + } + } +} + case class GpuWindowExec( windowOps: Seq[NamedExpression], gpuPartitionSpec: Seq[Expression], diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala index 4930987ef28..7455b6599b7 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala @@ -27,7 +27,8 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.rapids.{GpuAdd, GpuAggregateExpression, GpuCreateNamedStruct} +import org.apache.spark.sql.rapids.{GpuAdd, GpuAggregateExpression, GpuCount, GpuCreateNamedStruct, GpuDivide, GpuSubtract} +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -811,10 +812,73 @@ trait BatchedRunningWindowFixer extends AutoCloseable { def needsOrderMask: Boolean = false - protected def incRef(col: cudf.ColumnView): cudf.ColumnVector = col match { - case cv: cudf.ColumnVector => cv.incRefCount() - case _ => col.copyToColumnVector() - } + protected def incRef(col: cudf.ColumnView): cudf.ColumnVector = col.copyToColumnVector() +} + +/** + * Provides a way to process window operations without needing to buffer and split the + * batches on partition by boundaries. When this happens part of a partition by key set may + * have been processed in the previous batches, and may need to be updated. For example + * if we are doing a min operation with unbounded preceding and unbounded following. + * We may first get in something like + * + * PARTS: 1, 1, 2, 2 + * VALUES: 2, 3, 10, 9 + * + * + * The output of processing this would result in a new column that would look like + * + * MINS: 2, 2, 9, 9 + * + * + * But we don't know if the group with 2 in PARTS is done or not. So the fixer saved + * the last value in MINS, which is a 9, and caches the batch. When the next batch shows up + * + * + * PARTS: 2, 2, 3, 3 + * VALUES: 11, 5, 13, 14 + * + * + * We generate the window result again and get + * + * + * MINS: 5, 5, 13, 13 + * + * + * And now we need to grab the first entry which is a 5 and update the cached data with another min. + * The cached data for PARTS=2 is now 5. We then need to go back and fix up all of the previous + * batches that had something to do with PARTS=2. The first batch will be pulled from the cache + * and updated to look like + * + * + * PARTS: 1, 1, 2, 2 + * VALUES: 2, 3, 10, 9 + * MINS: 2, 2, 5, 5 + * + * which can be output because we were able to fix up all of the PARTS in that batch. + */ +trait BatchedUnboundedToUnboundedWindowFixer extends AutoCloseable { + /** + * Called to fix up a batch. There is no guarantee on the order the batches are fixed. The only + * ordering guarantee is that the state will be updated for all batches before any are "fixed" + * @param samePartitionMask indicates which rows are a part of the same partition. + * @param column the column of data to be fixed. + * @return a column of data that was fixed. + */ + def fixUp(samePartitionMask: Either[ColumnVector, Boolean], column: ColumnVector): ColumnVector + + /** + * Clear any state so that updateState can be called again for a new partition by group. + */ + def reset(): Unit + + /** + * Cache and update any state needed. Because this is specific to unbounded preceding to + * unbounded following the result should be the same for any row within a batch. As such, this is + * only guaranteed to be called once per batch with the value from a row within the batch. + * @param scalar the value to use to update what is cached. + */ + def updateState(scalar: Scalar): Unit } /** @@ -826,11 +890,76 @@ trait BatchedRunningWindowFixer extends AutoCloseable { trait GpuBatchedRunningWindowWithFixer { /** - * Get a new class that can be used to fix up batched RunningWindowOperations. + * Get a new class that can be used to fix up batched running window operations. */ def newFixer(): BatchedRunningWindowFixer } +/** + * For many window operations the results in earlier rows depends on the results from the last + * or later rows. In many of these cases we chunk the data based off of the partition by groups + * and process the data at once. But this can lead to out of memory errors, or hitting the + * row limit on some columns. Doing two passes through the data where the first pass processes + * the data and a second pass fixes up the data can let us keep the data in the original batches + * and reduce total memory usage. But this requires that some of the batches be made spillable + * while we wait for the end of the partition by group. + * + * Right now this is written to be specific to windows that are unbounded preceding to unbounded + * following, but it could be adapted to also work for current row to unbounded following, and + * possibly more situations. + */ +trait GpuUnboundToUnboundWindowWithFixer { + def newUnboundedToUnboundedFixer: BatchedUnboundedToUnboundedWindowFixer +} + +/** + * Fixes up a count operation for unbounded preceding to unbounded following + * @param errorOnOverflow if we need to throw an exception when an overflow happens or not. + */ +class CountUnboundedToUnboundedFixer(errorOnOverflow: Boolean) + extends BatchedUnboundedToUnboundedWindowFixer with Arm { + private var previousValue: Option[Long] = None + + override def reset(): Unit = { + previousValue = None + } + + override def updateState(scalar: Scalar): Unit = { + // It should be impossible for count to produce a null. + // Even if the input was all nulls the count is 0 + assert(scalar.isValid) + if (previousValue.isEmpty) { + previousValue = Some(scalar.getLong) + } else { + val old = previousValue.get + previousValue = Some(old + scalar.getLong) + if (errorOnOverflow && previousValue.get < 0) { + // This matches what would happen in an add operation, which is where the overflow + // in the CPU count would happen + throw RapidsErrorUtils.arithmeticOverflowError( + "One or more rows overflow for Add operation.") + } + } + } + + override def close(): Unit = reset() + + override def fixUp(samePartitionMask: Either[ColumnVector, Boolean], + column: ColumnVector): ColumnVector = { + assert(previousValue.nonEmpty) + withResource(Scalar.fromLong(previousValue.get)) { scalar => + samePartitionMask match { + case scala.Left(cv) => + cv.ifElse(scalar, column) + case scala.Right(true) => + ColumnVector.fromScalar(scalar, column.getRowCount.toInt) + case _ => + column.incRefCount() + } + } + } +} + /** * This class fixes up batched running windows by performing a binary op on the previous value and * those in the the same partition by key group. It does not deal with nulls, so it works for things @@ -1529,36 +1658,47 @@ case class GpuLag(input: Expression, offset: Expression, default: Expression) /** * percent_rank() is a running window function in that it only operates on a window of unbounded - * preceding to current row. But, an entire window has to be in the batch because the rank is - * divided by the number of entries in the window to get the percent rank. We cannot know the number - * of entries in the window without the entire window. This is why it is not a - * `GpuBatchedRunningWindowWithFixer`. + * preceding to current row. But the percent part actually makes it need a full count of the number + * of rows in the window. This is why we rewrite the operator to allow us to compute the result + * in a way that will not overflow memory. */ -case class GpuPercentRank(children: Seq[Expression]) extends GpuRunningWindowFunction { +case class GpuPercentRank(children: Seq[Expression]) extends GpuReplaceWindowFunction { override def nullable: Boolean = false override def dataType: DataType = DoubleType - override def groupByScanInputProjection(isRunningBatched: Boolean): Seq[Expression] = { - val orderedBy = if (children.length == 1) { - children.head - } else { - val childrenWithNames = children.zipWithIndex.flatMap { - case (expr, idx) => Seq(GpuLiteral(idx.toString, StringType), expr) - } - GpuCreateNamedStruct(childrenWithNames) - } - Seq(orderedBy) - } - - override def groupByScanAggregation( - isRunningBatched: Boolean): Seq[AggAndReplace[GroupByScanAggregation]] = { - Seq(AggAndReplace(GroupByScanAggregation.percentRank(), None)) - } - - override def scanInputProjection(isRunningBatched: Boolean): Seq[Expression] = - groupByScanInputProjection(isRunningBatched) - - override def scanAggregation(isRunningBatched: Boolean): Seq[AggAndReplace[ScanAggregation]] = { - Seq(AggAndReplace(ScanAggregation.percentRank(), None)) + override def windowReplacement(spec: GpuWindowSpecDefinition): Expression = { + // Spark writes this as + // If(n > one, (rank - one).cast(DoubleType) / (n - one).cast(DoubleType), 0.0d) + // where n is the count of all values in the window and rank is the rank. + // + // The databricks docs describe it as + // nvl( + // (rank() over (PARTITION BY p ORDER BY o) - 1) / + // (nullif(count(1) over(PARTITION BY p) - 1, 0)), + // 0.0) + // + // We do it slightly differently to try and optimize things for the GPU. + // We ignore ANSI mode because the count agg will take care of overflows already + // and n - 1 cannot overflow. It also cannot be negative because it is COUNT(1) and 1 + // cannot be null. + // A divide by 0 in non-ANSI mode produces a null, which we can use to avoid extra data copies. + // The If/Else from the original Spark expression on the GPU needs to split the input data to + // avoid the ANSI divide throwing an error on the divide by 0 that it is trying to avoid. We + // skip that and just take the null as output, which we can replace with 0.0 afterwards. + // That is the only case when we would get a null as output. + // From this we essentially do + // coalesce(CAST(rank - 1 AS DOUBLE) / CAST(n - 1 AS DOUBLE), 0.0) + val isAnsi = false + val fullUnboundedFrame = GpuSpecifiedWindowFrame(RowFrame, + GpuSpecialFrameBoundary(UnboundedPreceding), + GpuSpecialFrameBoundary(UnboundedFollowing)) + val fullUnboundedSpec = GpuWindowSpecDefinition(spec.partitionSpec, spec.orderSpec, + fullUnboundedFrame) + val count = GpuWindowExpression(GpuCount(Seq(GpuLiteral(1))), fullUnboundedSpec) + val rank = GpuWindowExpression(GpuRank(children), spec) + val rankMinusOne = GpuCast(GpuSubtract(rank, GpuLiteral(1), isAnsi), DoubleType, isAnsi) + val countMinusOne = GpuCast(GpuSubtract(count, GpuLiteral(1L), isAnsi), DoubleType, isAnsi) + val divided = GpuDivide(rankMinusOne, countMinusOne, failOnErrorOverride = isAnsi) + GpuCoalesce(Seq(divided, GpuLiteral(0.0))) } } \ No newline at end of file diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala index f54e8ea3050..a01a649b88f 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala @@ -1415,8 +1415,11 @@ case class GpuPivotFirst( override def children: Seq[Expression] = pivotColumn :: valueColumn :: Nil } -case class GpuCount(children: Seq[Expression]) extends GpuAggregateFunction +case class GpuCount(children: Seq[Expression], + failOnError: Boolean = SQLConf.get.ansiEnabled) + extends GpuAggregateFunction with GpuBatchedRunningWindowWithFixer + with GpuUnboundToUnboundWindowWithFixer with GpuAggregateWindowFunction with GpuRunningWindowFunction { override lazy val initialValues: Seq[GpuLiteral] = Seq(GpuLiteral(0L, LongType)) @@ -1485,6 +1488,9 @@ case class GpuCount(children: Seq[Expression]) extends GpuAggregateFunction override def scanCombine(isRunningBatched: Boolean, cols: Seq[ColumnVector]): ColumnVector = cols.head.castTo(DType.INT64) + + override def newUnboundedToUnboundedFixer: BatchedUnboundedToUnboundedWindowFixer = + new CountUnboundedToUnboundedFixer(failOnError) } object GpuAverage { From a82a3a04054464943bbb705ad3b15c2f598247b3 Mon Sep 17 00:00:00 2001 From: MithunR Date: Mon, 19 Sep 2022 06:47:05 -0700 Subject: [PATCH 115/190] Support Decimal ordering column for `RANGE` window functions (#6554) Signed-off-by: MithunR --- docs/configs.md | 1 + docs/supported_ops.md | 4 +- .../src/main/python/window_function_test.py | 45 ++++++++++++++++ .../rapids/shims/Spark320PlusShims.scala | 6 ++- .../spark/rapids/shims/gpuWindows.scala | 2 +- .../nvidia/spark/rapids/GpuOverrides.scala | 4 +- .../nvidia/spark/rapids/GpuWindowExec.scala | 51 ++++++++++++------- .../spark/rapids/GpuWindowExpression.scala | 9 +++- .../com/nvidia/spark/rapids/RapidsConf.scala | 9 ++++ tools/src/main/resources/supportedExprs.csv | 4 +- 10 files changed, 107 insertions(+), 28 deletions(-) diff --git a/docs/configs.md b/docs/configs.md index b121a0e2d92..627cc6098fa 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -141,6 +141,7 @@ Name | Description | Default Value spark.rapids.sql.udfCompiler.enabled|When set to true, Scala UDFs will be considered for compilation as Catalyst expressions|false spark.rapids.sql.variableFloatAgg.enabled|Spark assumes that all operations produce the exact same result each time. This is not true for some floating point aggregations, which can produce slightly different results on the GPU as the aggregation is done in parallel. This can enable those operations if you know the query is only computing it once.|true spark.rapids.sql.window.range.byte.enabled|When the order-by column of a range based window is byte type and the range boundary calculated for a value has overflow, CPU and GPU will get the different results. When set to false disables the range window acceleration for the byte type order-by column|false +spark.rapids.sql.window.range.decimal.enabled|When set to false, this disables the range window acceleration for the DECIMAL type order-by column|true spark.rapids.sql.window.range.int.enabled|When the order-by column of a range based window is int type and the range boundary calculated for a value has overflow, CPU and GPU will get the different results. When set to false disables the range window acceleration for the int type order-by column|true spark.rapids.sql.window.range.long.enabled|When the order-by column of a range based window is long type and the range boundary calculated for a value has overflow, CPU and GPU will get the different results. When set to false disables the range window acceleration for the long type order-by column|true spark.rapids.sql.window.range.short.enabled|When the order-by column of a range based window is short type and the range boundary calculated for a value has overflow, CPU and GPU will get the different results. When set to false disables the range window acceleration for the short type order-by column|false diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 396297f2565..d2b8603d8ad 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -12465,7 +12465,7 @@ are limited. -NS +S S @@ -12486,7 +12486,7 @@ are limited. -NS +S S diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py index c7564670428..1c86cdcc551 100644 --- a/integration_tests/src/main/python/window_function_test.py +++ b/integration_tests/src/main/python/window_function_test.py @@ -68,6 +68,11 @@ ('b', DecimalGen(precision=23, scale=10, nullable=True)), ('c', DecimalGen(precision=23, scale=10, nullable=True))] +_grpkey_longs_with_nullable_largest_decimals = [ + ('a', RepeatSeqGen(LongGen(nullable=(True, 10.0)), length=20)), + ('b', DecimalGen(precision=38, scale=2, nullable=True)), + ('c', DecimalGen(precision=38, scale=2, nullable=True))] + _grpkey_decimals_with_nulls = [ ('a', RepeatSeqGen(LongGen(nullable=(True, 10.0)), length=20)), ('b', IntegerGen()), @@ -817,6 +822,46 @@ def test_window_aggs_for_ranges_timestamps(data_gen): conf = {'spark.rapids.sql.castFloatToDecimal.enabled': True}) +# In a distributed setup the order of the partitions returned might be different, so we must ignore the order +# but small batch sizes can make sort very slow, so do the final order by locally +@ignore_order(local=True) +@pytest.mark.parametrize('data_gen', [_grpkey_longs_with_nullable_decimals, + _grpkey_longs_with_nullable_larger_decimals, + _grpkey_longs_with_nullable_largest_decimals], + ids=idfn) +def test_window_aggregations_for_decimal_ranges(data_gen): + """ + Tests for range window aggregations, with DECIMAL order by columns. + The table schema used: + a: Group By column + b: Order By column (decimal) + c: Aggregation column (incidentally, also decimal) + + Since this test is for the order-by column type, and not for each specific windowing aggregation, + we use COUNT(1) throughout the test, for different window widths and ordering. + Some other aggregation functions are thrown in for variety. + """ + assert_gpu_and_cpu_are_equal_sql( + lambda spark: gen_df(spark, data_gen, length=2048), + "window_agg_table", + 'SELECT ' + ' COUNT(1) OVER (PARTITION BY a ORDER BY b ASC RANGE BETWEEN 10.2345 PRECEDING AND 6.7890 FOLLOWING), ' + ' COUNT(1) OVER (PARTITION BY a ORDER BY b ASC), ' + ' COUNT(1) OVER (PARTITION BY a ORDER BY b ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), ' + ' COUNT(1) OVER (PARTITION BY a ORDER BY b ASC RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), ' + ' COUNT(1) OVER (PARTITION BY a ORDER BY b DESC RANGE BETWEEN 10.2345 PRECEDING AND 6.7890 FOLLOWING), ' + ' COUNT(1) OVER (PARTITION BY a ORDER BY b DESC), ' + ' COUNT(1) OVER (PARTITION BY a ORDER BY b DESC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), ' + ' COUNT(1) OVER (PARTITION BY a ORDER BY b DESC RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING),' + ' COUNT(c) OVER (PARTITION BY a ORDER BY b RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), ' + ' SUM(c) OVER (PARTITION BY a ORDER BY b RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), ' + ' MIN(c) OVER (PARTITION BY a ORDER BY b RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), ' + ' MAX(c) OVER (PARTITION BY a ORDER BY b RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), ' + ' RANK() OVER (PARTITION BY a ORDER BY b) ' + 'FROM window_agg_table', + conf={}) + + _gen_data_for_collect_list = [ ('a', RepeatSeqGen(LongGen(), length=20)), ('b', LongRangeGen()), diff --git a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala index e1371d155a1..31ceba8da26 100644 --- a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala +++ b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala @@ -209,10 +209,12 @@ trait Spark320PlusShims extends SparkShims with RebaseShims with Logging { TypeSig.numericAndInterval, Seq( ParamCheck("lower", - TypeSig.CALENDAR + TypeSig.NULL + TypeSig.integral + TypeSig.DAYTIME, + TypeSig.CALENDAR + TypeSig.NULL + TypeSig.integral + TypeSig.DAYTIME + + TypeSig.DECIMAL_128, TypeSig.numericAndInterval), ParamCheck("upper", - TypeSig.CALENDAR + TypeSig.NULL + TypeSig.integral + TypeSig.DAYTIME, + TypeSig.CALENDAR + TypeSig.NULL + TypeSig.integral + TypeSig.DAYTIME + + TypeSig.DECIMAL_128, TypeSig.numericAndInterval))), (windowFrame, conf, p, r) => new GpuSpecifiedWindowFrameMeta(windowFrame, conf, p, r)), GpuOverrides.expr[WindowExpression]( diff --git a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala index 2a11109fb84..68230d6fc04 100644 --- a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala +++ b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala @@ -68,7 +68,7 @@ object GpuWindowUtil { case GpuLiteral(value, _: DayTimeIntervalType) => var x = value.asInstanceOf[Long] if (x == Long.MinValue) x = Long.MaxValue - ParsedBoundary(isUnbounded = false, Math.abs(x)) + ParsedBoundary(isUnbounded = false, Right(Math.abs(x))) case anything => throw new UnsupportedOperationException("Unsupported window frame" + s" expression $anything") } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 84819c92ec6..cbdc7fd8ca9 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -1034,10 +1034,10 @@ object GpuOverrides extends Logging { TypeSig.numericAndInterval, Seq( ParamCheck("lower", - TypeSig.CALENDAR + TypeSig.NULL + TypeSig.integral, + TypeSig.CALENDAR + TypeSig.NULL + TypeSig.integral + TypeSig.DECIMAL_128, TypeSig.numericAndInterval), ParamCheck("upper", - TypeSig.CALENDAR + TypeSig.NULL + TypeSig.integral, + TypeSig.CALENDAR + TypeSig.NULL + TypeSig.integral + TypeSig.DECIMAL_128, TypeSig.numericAndInterval))), (windowFrame, conf, p, r) => new GpuSpecifiedWindowFrameMeta(windowFrame, conf, p, r) ), expr[WindowSpecDefinition]( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala index 667ddfcdcfd..2b7a8300081 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala @@ -608,7 +608,7 @@ case class BoundGpuWindowFunction( val dataType: DataType = windowFunc.dataType } -case class ParsedBoundary(isUnbounded: Boolean, valueAsLong: Long) +case class ParsedBoundary(isUnbounded: Boolean, value: Either[BigInt, Long]) object GroupedAggregations extends Arm { /** @@ -643,8 +643,8 @@ object GroupedAggregations extends Arm { val orderType = GpuColumnVector.getNonNestedRapidsType(orderExpr.dataType) val orderByIndex = orderPositions.head - val lower = getRangeBoundaryValue(frame.lower) - val upper = getRangeBoundaryValue(frame.upper) + val lower = getRangeBoundaryValue(frame.lower, orderType) + val upper = getRangeBoundaryValue(frame.upper, orderType) withResource(asScalarRangeBoundary(orderType, lower)) { preceding => withResource(asScalarRangeBoundary(orderType, upper)) { following => @@ -718,47 +718,64 @@ object GroupedAggregations extends Arm { if (bound.isUnbounded) { None } else { - val value = bound.valueAsLong + val valueLong = bound.value.right // Used for all cases except DECIMAL128. val s = orderByType match { - case DType.INT8 => Scalar.fromByte(value.toByte) - case DType.INT16 => Scalar.fromShort(value.toShort) - case DType.INT32 => Scalar.fromInt(value.toInt) - case DType.INT64 => Scalar.fromLong(value) + case DType.INT8 => Scalar.fromByte(valueLong.get.toByte) + case DType.INT16 => Scalar.fromShort(valueLong.get.toShort) + case DType.INT32 => Scalar.fromInt(valueLong.get.toInt) + case DType.INT64 => Scalar.fromLong(valueLong.get) // Interval is not working for DateType - case DType.TIMESTAMP_DAYS => Scalar.durationFromLong(DType.DURATION_DAYS, value) + case DType.TIMESTAMP_DAYS => Scalar.durationFromLong(DType.DURATION_DAYS, valueLong.get) case DType.TIMESTAMP_MICROSECONDS => - Scalar.durationFromLong(DType.DURATION_MICROSECONDS, value) + Scalar.durationFromLong(DType.DURATION_MICROSECONDS, valueLong.get) + case x if x.getTypeId == DType.DTypeEnum.DECIMAL32 => + Scalar.fromDecimal(x.getScale, valueLong.get.toInt) + case x if x.getTypeId == DType.DTypeEnum.DECIMAL64 => + Scalar.fromDecimal(x.getScale, valueLong.get) + case x if x.getTypeId == DType.DTypeEnum.DECIMAL128 => + Scalar.fromDecimal(x.getScale, bound.value.left.get.underlying()) case _ => throw new RuntimeException(s"Not supported order by type, Found $orderByType") } Some(s) } } - private def getRangeBoundaryValue(boundary: Expression): ParsedBoundary = boundary match { + private def getRangeBoundaryValue(boundary: Expression, orderByType: DType): ParsedBoundary = + boundary match { case special: GpuSpecialFrameBoundary => val isUnBounded = special.isUnbounded - ParsedBoundary(isUnBounded, special.value) + val isDecimal128 = orderByType.getTypeId == DType.DTypeEnum.DECIMAL128 + ParsedBoundary(isUnBounded, if (isDecimal128) Left(special.value) else Right(special.value)) case GpuLiteral(ci: CalendarInterval, CalendarIntervalType) => // Get the total microseconds for TIMESTAMP_MICROSECONDS var x = TimeUnit.DAYS.toMicros(ci.days) + ci.microseconds if (x == Long.MinValue) x = Long.MaxValue - ParsedBoundary(isUnbounded = false, Math.abs(x)) + ParsedBoundary(isUnbounded = false, Right(Math.abs(x))) case GpuLiteral(value, ByteType) => var x = value.asInstanceOf[Byte] if (x == Byte.MinValue) x = Byte.MaxValue - ParsedBoundary(isUnbounded = false, Math.abs(x)) + ParsedBoundary(isUnbounded = false, Right(Math.abs(x))) case GpuLiteral(value, ShortType) => var x = value.asInstanceOf[Short] if (x == Short.MinValue) x = Short.MaxValue - ParsedBoundary(isUnbounded = false, Math.abs(x)) + ParsedBoundary(isUnbounded = false, Right(Math.abs(x))) case GpuLiteral(value, IntegerType) => var x = value.asInstanceOf[Int] if (x == Int.MinValue) x = Int.MaxValue - ParsedBoundary(isUnbounded = false, Math.abs(x)) + ParsedBoundary(isUnbounded = false, Right(Math.abs(x))) case GpuLiteral(value, LongType) => var x = value.asInstanceOf[Long] if (x == Long.MinValue) x = Long.MaxValue - ParsedBoundary(isUnbounded = false, Math.abs(x)) + ParsedBoundary(isUnbounded = false, Right(Math.abs(x))) + case GpuLiteral(value: Decimal, DecimalType()) => + orderByType.getTypeId match { + case DType.DTypeEnum.DECIMAL32 | DType.DTypeEnum.DECIMAL64 => + ParsedBoundary(isUnbounded = false, Right(Math.abs(value.toUnscaledLong))) + case DType.DTypeEnum.DECIMAL128 => + ParsedBoundary(isUnbounded = false, Left(value.toJavaBigDecimal.unscaledValue().abs)) + case anythingElse => + throw new UnsupportedOperationException(s"Unexpected Decimal type: $anythingElse") + } case anything => GpuWindowUtil.getRangeBoundaryValue(anything) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala index 7455b6599b7..cd4d5b59090 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala @@ -106,7 +106,7 @@ abstract class GpuWindowExpressionMetaBase( val orderByTypeSupported = orderSpec.forall { so => so.dataType match { case ByteType | ShortType | IntegerType | LongType | - DateType | TimestampType => true + DateType | TimestampType | DecimalType() => true case _ => false } } @@ -133,13 +133,17 @@ abstract class GpuWindowExpressionMetaBase( s"Range window frame is not 100% compatible when the order by type is " + s"long and the range value calculated has overflow. " + s"To enable it please set ${RapidsConf.ENABLE_RANGE_WINDOW_LONG} to true.") + case DecimalType() => if (!conf.isRangeWindowDecimalEnabled) willNotWorkOnGpu( + s"To enable DECIMAL order by columns with Range window frames, " + + s"please set ${RapidsConf.ENABLE_RANGE_WINDOW_DECIMAL} to true.") case _ => // never reach here } } // check whether the boundaries are supported or not. Seq(spec.lower, spec.upper).foreach { - case l @ Literal(_, ByteType | ShortType | IntegerType | LongType) => + case l @ Literal(_, ByteType | ShortType | IntegerType | + LongType | DecimalType()) => checkRangeBoundaryConfig(l.dataType) case Literal(ci: CalendarInterval, CalendarIntervalType) => // interval is only working for TimeStampType @@ -377,6 +381,7 @@ abstract class GpuSpecifiedWindowFrameMetaBase( case Literal(value, ShortType) => value.asInstanceOf[Short].toLong case Literal(value, IntegerType) => value.asInstanceOf[Int].toLong case Literal(value, LongType) => value.asInstanceOf[Long] + case Literal(value: Decimal, DecimalType()) => value.toLong case Literal(ci: CalendarInterval, CalendarIntervalType) => if (ci.months != 0) { willNotWorkOnGpu("interval months isn't supported") diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 21e2f0760d3..5fe9e83ad39 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -1071,6 +1071,13 @@ object RapidsConf { .booleanConf .createWithDefault(true) + val ENABLE_RANGE_WINDOW_DECIMAL: ConfEntryWithDefault[Boolean] = + conf("spark.rapids.sql.window.range.decimal.enabled") + .doc("When set to false, this disables the range window acceleration for the " + + "DECIMAL type order-by column") + .booleanConf + .createWithDefault(true) + val ENABLE_REGEXP = conf("spark.rapids.sql.regexp.enabled") .doc("Specifies whether supported regular expressions will be evaluated on the GPU. " + "Unsupported expressions will fall back to CPU. However, there are some known edge cases " + @@ -2079,6 +2086,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isRangeWindowLongEnabled: Boolean = get(ENABLE_RANGE_WINDOW_LONG) + lazy val isRangeWindowDecimalEnabled: Boolean = get(ENABLE_RANGE_WINDOW_DECIMAL) + lazy val isRegExpEnabled: Boolean = get(ENABLE_REGEXP) lazy val maxRegExpStateMemory: Long = { diff --git a/tools/src/main/resources/supportedExprs.csv b/tools/src/main/resources/supportedExprs.csv index c9d622f3499..25748ccecf4 100644 --- a/tools/src/main/resources/supportedExprs.csv +++ b/tools/src/main/resources/supportedExprs.csv @@ -456,8 +456,8 @@ SortArray,S,`sort_array`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA SortOrder,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS SortOrder,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS SparkPartitionID,S,`spark_partition_id`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -SpecifiedWindowFrame,S, ,None,project,lower,NA,S,S,S,S,NS,NS,NA,NA,NA,NS,NA,NA,S,NA,NA,NA,NA -SpecifiedWindowFrame,S, ,None,project,upper,NA,S,S,S,S,NS,NS,NA,NA,NA,NS,NA,NA,S,NA,NA,NA,NA +SpecifiedWindowFrame,S, ,None,project,lower,NA,S,S,S,S,NS,NS,NA,NA,NA,S,NA,NA,S,NA,NA,NA,NA +SpecifiedWindowFrame,S, ,None,project,upper,NA,S,S,S,S,NS,NS,NA,NA,NA,S,NA,NA,S,NA,NA,NA,NA SpecifiedWindowFrame,S, ,None,project,result,NA,S,S,S,S,NS,NS,NA,NA,NA,NS,NA,NA,S,NA,NA,NA,NA Sqrt,S,`sqrt`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Sqrt,S,`sqrt`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA From 60f8214a7336b960d05cf3be7dea8ce0d23a6ac4 Mon Sep 17 00:00:00 2001 From: liyuan <84758614+nvliyuan@users.noreply.github.com> Date: Mon, 19 Sep 2022 23:25:53 +0800 Subject: [PATCH 116/190] [DOC] Add notes for cgroup permission reverted[skip ci] (#6504) * add notes for cgroup permission reverted Signed-off-by: liyuan * Update docs/get-started/yarn-gpu.md Co-authored-by: Gary Shen * Update docs/get-started/yarn-gpu.md Co-authored-by: Sameer Raheja * Update docs/get-started/yarn-gpu.md Co-authored-by: Sameer Raheja * address comments Signed-off-by: liyuan * update mountCgroup.sh mode Signed-off-by: liyuan * add comments for cgroup Signed-off-by: liyuan * Update docs/get-started/yarn-gpu.md Co-authored-by: Sameer Raheja * Update docs/get-started/yarn-gpu.md Co-authored-by: Sameer Raheja * Update docs/get-started/yarn-gpu.md Co-authored-by: Sameer Raheja Signed-off-by: liyuan Co-authored-by: Gary Shen Co-authored-by: Sameer Raheja --- docs/get-started/yarn-gpu.md | 40 ++++++++++++++++++++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/docs/get-started/yarn-gpu.md b/docs/get-started/yarn-gpu.md index 22dbb4aba4e..07331e1b943 100644 --- a/docs/get-started/yarn-gpu.md +++ b/docs/get-started/yarn-gpu.md @@ -116,3 +116,43 @@ On all masters: ```bash sudo systemctl restart hadoop-yarn-resourcemanager.service ``` + +Note: If `cgroup` is mounted on `tmpfs` and a node is rebooted, +the cgroup directory permission gets reverted. Please check the +cgroup documentation for your platform for more details. +Below is one example of how this can be handled: + +Update the cgroup permissions: +```bash +chmod a+rwx -R /sys/fs/cgroup/cpu,cpuacct +chmod a+rwx -R /sys/fs/cgroup/devices +``` +Or the operation can be added in the systemd scripts: + +Create mountCgroup scripts: +```bash +sudo bash -c "cat >/etc/systemd/system/mountCgroup.service" </etc/mountCgroup.sh" < Date: Mon, 19 Sep 2022 12:23:30 -0500 Subject: [PATCH 117/190] Handle EmptyRelation in GpuSubqueryBroadcastExec (#6562) Signed-off-by: Jason Lowe Signed-off-by: Jason Lowe --- integration_tests/src/main/python/dpp_test.py | 27 +++++++ .../execution/GpuSubqueryBroadcastExec.scala | 72 +++++++++++-------- 2 files changed, 68 insertions(+), 31 deletions(-) diff --git a/integration_tests/src/main/python/dpp_test.py b/integration_tests/src/main/python/dpp_test.py index ad73358704c..ee7b2b43671 100644 --- a/integration_tests/src/main/python/dpp_test.py +++ b/integration_tests/src/main/python/dpp_test.py @@ -300,3 +300,30 @@ def setup_tables(spark): conf=dict(_dpp_conf + [('spark.sql.adaptive.enabled', aqe_enabled), ("spark.rapids.sql.castStringToTimestamp.enabled", "true"), ("spark.rapids.sql.hasExtendedYearValues", "false")])) + +# Test handling DPP subquery that could broadcast EmptyRelation rather than a GPU serialized batch +@pytest.mark.parametrize('aqe_enabled', [ + 'false', + pytest.param('true', marks=pytest.mark.skipif(is_before_spark_320(), + reason='Only in Spark 3.2.0+ AQE and DPP can be both enabled')) +], ids=idfn) +@pytest.mark.skipif(is_databricks_runtime(), reason='DPP is not supported on Databricks runtime') +def test_dpp_empty_relation(spark_tmp_table_factory, aqe_enabled): + dim_table = spark_tmp_table_factory.get() + fact_table = spark_tmp_table_factory.get() + def setup_tables(spark): + spark.sql("CREATE TABLE {}(id string) PARTITIONED BY (dt date, hr string, mins string) STORED AS PARQUET".format(dim_table)) + spark.sql("INSERT INTO {}(id,dt,hr,mins) values ('somevalue', date('2022-01-01'), '11', '59')".format(dim_table)) + spark.sql("CREATE TABLE {}(id string)".format(fact_table) + + " PARTITIONED BY (dt date, hr string, mins string) STORED AS PARQUET") + spark.sql("INSERT INTO {}(id,dt,hr,mins)".format(fact_table) + + " SELECT 'somevalue', to_date('2022-01-01'), '11', '59'") + with_cpu_session(setup_tables, conf={ + "hive.exec.dynamic.partition" : "true", + "hive.exec.dynamic.partition.mode" : "nonstrict" + }) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT COUNT(*) AS cnt FROM {} f".format(fact_table) + + " LEFT JOIN (SELECT * from {}) tmp".format(dim_table) + + " ON f.hr = tmp.hr AND f.dt = tmp.dt WHERE tmp.mins > 60"), + conf=dict(_dpp_conf + [('spark.sql.adaptive.enabled', aqe_enabled)])) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala index a411d0e9d29..e03188fef31 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala @@ -207,44 +207,54 @@ case class GpuSubqueryBroadcastExec( // This will run in another thread. Set the execution id so that we can connect these jobs // with the correct execution. SQLExecution.withExecutionId(sparkSession, executionId) { - val beforeCollect = System.nanoTime() - - val serBatch = child.executeBroadcast[SerializeConcatHostBuffersDeserializeBatch]() - - // Creates projection to extract target field from Row, as what Spark does. - // Note that unlike Spark, the GPU broadcast data has not applied the key expressions from - // the HashedRelation, so that is applied here if necessary to ensure the proper values - // are being extracted. The CPU already has the key projections applied in the broadcast - // data and thus does not have similar logic here. - val broadcastModeProject = modeKeys.map { keyExprs => - val keyExpr = keyExprs(index) - UnsafeProjection.create(keyExpr) + val broadcastBatch = child.executeBroadcast[Any]() + val result: Array[InternalRow] = broadcastBatch.value match { + case b: SerializeConcatHostBuffersDeserializeBatch => projectSerializedBatchToRows(b) + case b if SparkShimImpl.isEmptyRelation(b) => Array.empty + case b => throw new IllegalStateException(s"Unexpected broadcast type: ${b.getClass}") } - // Use the single output of the broadcast mode projection if it exists - val rowProjectIndex = if (broadcastModeProject.isDefined) 0 else index - val rowProject = UnsafeProjection.create( - BoundReference(rowProjectIndex, buildKeys(index).dataType, buildKeys(index).nullable)) - - // Deserializes the batch on the host. Then, transforms it to rows and performs row-wise - // projection. We should NOT run any device operation on the driver node. - val result = withResource(serBatch.value.hostBatches) { hostBatches => - hostBatches.flatMap { cb => - cb.rowIterator().asScala.map { row => - val broadcastRow = broadcastModeProject.map(_(row)).getOrElse(row) - rowProject(broadcastRow).copy().asInstanceOf[InternalRow] - } - } - } - - gpuLongMetric("dataSize") += serBatch.value.dataSize - gpuLongMetric(COLLECT_TIME) += System.nanoTime() - beforeCollect - result } }(GpuSubqueryBroadcastExec.executionContext) } + private def projectSerializedBatchToRows( + serBatch: SerializeConcatHostBuffersDeserializeBatch): Array[InternalRow] = { + val beforeCollect = System.nanoTime() + + // Creates projection to extract target field from Row, as what Spark does. + // Note that unlike Spark, the GPU broadcast data has not applied the key expressions from + // the HashedRelation, so that is applied here if necessary to ensure the proper values + // are being extracted. The CPU already has the key projections applied in the broadcast + // data and thus does not have similar logic here. + val broadcastModeProject = modeKeys.map { keyExprs => + val keyExpr = keyExprs(index) + UnsafeProjection.create(keyExpr) + } + + // Use the single output of the broadcast mode projection if it exists + val rowProjectIndex = if (broadcastModeProject.isDefined) 0 else index + val rowProject = UnsafeProjection.create( + BoundReference(rowProjectIndex, buildKeys(index).dataType, buildKeys(index).nullable)) + + // Deserializes the batch on the host. Then, transforms it to rows and performs row-wise + // projection. We should NOT run any device operation on the driver node. + val result = withResource(serBatch.hostBatches) { hostBatches => + hostBatches.flatMap { cb => + cb.rowIterator().asScala.map { row => + val broadcastRow = broadcastModeProject.map(_(row)).getOrElse(row) + rowProject(broadcastRow).copy().asInstanceOf[InternalRow] + } + } + } + + gpuLongMetric("dataSize") += serBatch.dataSize + gpuLongMetric(COLLECT_TIME) += System.nanoTime() - beforeCollect + + result + } + protected override def doExecute(): RDD[InternalRow] = { throw new UnsupportedOperationException( "GpuSubqueryBroadcastExec does not support the execute() code path.") From 647deb9c47fc01a8643da34a4d3c5332093a41da Mon Sep 17 00:00:00 2001 From: Navin Kumar <97137715+NVnavkumar@users.noreply.github.com> Date: Mon, 19 Sep 2022 13:42:58 -0700 Subject: [PATCH 118/190] Update handling for projectList based WindowExecs to handle window function of window function [databricks] (#6553) * WIP: Windowing fix for Databricks to handle missing child expressions of GpuWindowFunction Signed-off-by: Navin Kumar * Integration test for window of window and non-window child case * revert this fix Signed-off-by: Navin Kumar * WIP: Working DB 10.4 solution for window of window issue Signed-off-by: Navin Kumar * revert test script changes Signed-off-by: Navin Kumar * Remove logging statements, add comments, and update integration test Signed-off-by: Navin Kumar * Cleanup and add more comments Signed-off-by: Navin Kumar * remove this debug line Signed-off-by: Navin Kumar * Cleanup from feedback Signed-off-by: Navin Kumar * remove unused import Signed-off-by: Navin Kumar Signed-off-by: Navin Kumar --- .../src/main/python/window_function_test.py | 26 ++++++++ .../nvidia/spark/rapids/GpuWindowExec.scala | 65 ++++++++++++++----- 2 files changed, 76 insertions(+), 15 deletions(-) diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py index 1c86cdcc551..87411cbe93b 100644 --- a/integration_tests/src/main/python/window_function_test.py +++ b/integration_tests/src/main/python/window_function_test.py @@ -972,6 +972,32 @@ def test_running_window_function_exec_for_all_aggs(): from window_collect_table ''') +# Test the Databricks WindowExec which combines a WindowExec with a ProjectExec and provides the output +# fields that we need to handle with an extra GpuProjectExec and we need the input expressions to compute +# a window function of another window function case +@ignore_order(local=True) +@pytest.mark.parametrize('data_gen', integral_gens, ids=idfn) +def test_join_sum_window_of_window(data_gen): + def do_it(spark): + agg_table = gen_df(spark, StructGen([('a_1', LongRangeGen()), ('c', data_gen)], nullable=False)) + part_table = gen_df(spark, StructGen([('a_2', LongRangeGen()), ('b', byte_gen)], nullable=False)) + agg_table.createOrReplaceTempView("agg") + part_table.createOrReplaceTempView("part") + # Note that if we include `c` in the select clause here (the output projection), the bug described + # in https://github.com/NVIDIA/spark-rapids/issues/6531 does not manifest + return spark.sql(""" + select + b, + sum(c) as sum_c, + sum(c)/sum(sum(c)) over (partition by b) as ratio_sum, + (b + c)/sum(sum(c)) over (partition by b) as ratio_bc + from agg, part + where a_1 = a_2 + group by b, c + order by b, ratio_sum, ratio_bc""") + + assert_gpu_and_cpu_are_equal_collect(do_it) + # Generates some repeated values to test the deduplication of GpuCollectSet. # And GpuCollectSet does not yet support struct type. _gen_data_for_collect_set = [ diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala index 2b7a8300081..0896ba842d6 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala @@ -26,7 +26,6 @@ import ai.rapids.cudf.{AggregationOverWindow, DType, GroupByOptions, GroupByScan import com.nvidia.spark.rapids.shims.{GpuWindowUtil, ShimUnaryExecNode} import org.apache.spark.TaskContext -import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeReference, AttributeSeq, AttributeSet, CurrentRow, Expression, FrameType, NamedExpression, RangeFrame, RowFrame, SortOrder, UnboundedFollowing, UnboundedPreceding} @@ -50,7 +49,7 @@ abstract class GpuBaseWindowExecMeta[WindowExecType <: SparkPlan] (windowExec: W conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], rule: DataFromReplacementRule) - extends SparkPlanMeta[WindowExecType](windowExec, conf, parent, rule) with Logging { + extends SparkPlanMeta[WindowExecType](windowExec, conf, parent, rule) { /** * Extracts window-expression from WindowExecType. @@ -107,14 +106,18 @@ abstract class GpuBaseWindowExecMeta[WindowExecType <: SparkPlan] (windowExec: W } override def convertToGpu(): GpuExec = { + // resultColumnsOnly specifies that we should only return the values of result columns for + // this WindowExec (applies to some Spark distributions that use `projectList` and combine + // ProjectExec with WindowExec) val resultColumnsOnly = getResultColumnsOnly - val gpuWindowExpressions = if (resultColumnsOnly) { - windowExpressions.map(_.convertToGpu().asInstanceOf[NamedExpression]) - } else { - (inputFields ++ windowExpressions).map(_.convertToGpu().asInstanceOf[NamedExpression]) - } - - val (pre, windowOps, post) = GpuWindowExec.splitAndDedup(gpuWindowExpressions) + // Keep the converted input fields and input window expressions separate + // to handle the resultColumnsOnly case in GpuWindowExec.splitAndDedup + val inputFieldExpressions = inputFields.map(_.convertToGpu().asInstanceOf[NamedExpression]) + val gpuWindowExpressions = windowExpressions.map(_.convertToGpu().asInstanceOf[NamedExpression]) + val (pre, windowOps, post) = GpuWindowExec.splitAndDedup( + inputFieldExpressions, + gpuWindowExpressions, + resultColumnsOnly) // Order is not important for pre. It is unbound and we are inserting it in. val isPreNeeded = (AttributeSet(pre.map(_.toAttribute)) -- windowExec.children.head.output).nonEmpty @@ -122,6 +125,10 @@ abstract class GpuBaseWindowExecMeta[WindowExecType <: SparkPlan] (windowExec: W // might not be needed. Here we want to maintain order, just to match Spark as closely // as possible val remappedWindowOps = GpuWindowExec.remapAttributes(windowOps, post) + // isPostNeeded is determined when there is a difference between the output of the WindowExec + // computation and the output ultimately desired by this WindowExec or whether an additional + // post computation is needed. Ultimately this is used to add an additional ProjectExec + // if that is needed to return the correct output val isPostNeeded = remappedWindowOps.length != post.length || remappedWindowOps.zip(post).exists { case (w, p) => w.exprId != p.exprId @@ -357,12 +364,19 @@ object GpuWindowExec extends Arm { * (_tmp1 - _tmp2) as result * * - * This assumes that there is not a window function of another window function, like - * `LAG(SUM(a), 2)` which appears to be something all distros split apart into separate - * window operations, so we are good. - * @param exprs the input expressions to a GpuWindowExec + * To handle special cases (like window function of another window function eg `LAG(SUM(a), 2)`, + * distros should split apart those into separate window operations. However, we will not + * see all of these in just the input window expressions of the WindowExec, so we process + * both the input fields and input window expressions, and handle whether we *only* want result + * columns using the Post Project stage. + * @param inputFieldExprs the input fields converted to input expressions + * @param windowExprs the input window expressions to a GpuWindowExec + * @param resultColumnsOnly whether the output of the window operation only desires result + * columns or the output of all input expressions */ - def splitAndDedup(exprs: Seq[NamedExpression]): + def splitAndDedup(inputFieldExprs: Seq[NamedExpression], + windowExprs: Seq[NamedExpression], + resultColumnsOnly: Boolean): (Seq[NamedExpression], Seq[NamedExpression], Seq[NamedExpression]) = { // This is based off of similar code in Apache Spark's `ExtractWindowExpressions.extract` but // has been highly modified @@ -372,7 +386,28 @@ object GpuWindowExec extends Arm { val windowDedupe = mutable.HashMap[Expression, Attribute]() val postProject = ArrayBuffer[NamedExpression]() - exprs.foreach { expr => + // Process input field expressions first. There are no window functions here, so + // all of these should pass at least to pre and window stages + inputFieldExprs.foreach { expr => + // If the Spark distribution only wants to output result columns (ie, ones that + // use projectList), then pass the input field to pre and window stages, but + // do not pass to the post project stage (as those are specifically given in + // the projectList set) + if (resultColumnsOnly) { + extractAndSave( + extractAndSave(expr, preProject, preDedupe), windowOps, windowDedupe) + .asInstanceOf[NamedExpression] + } else { + // If the WindowExec returns everything, then pass the input fields through all the + // phases (with deduping) + postProject += extractAndSave( + extractAndSave(expr, preProject, preDedupe), windowOps, windowDedupe) + .asInstanceOf[NamedExpression] + } + } + + // Now split and dedup the input window expressions + windowExprs.foreach { expr => if (hasGpuWindowFunction(expr)) { // First pass replace any operations that should be totally replaced. val replacePass = expr.transformDown { From 9d73bfc528877184c490781ca85a96c22807d76d Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Mon, 19 Sep 2022 16:22:31 -0500 Subject: [PATCH 119/190] Fix maxPartitionBytes bounds checking in AutoTuner (#6575) Signed-off-by: Jason Lowe Signed-off-by: Jason Lowe --- .../nvidia/spark/rapids/tool/profiling/AutoTuner.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala index 5f083688839..46641ac0804 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala @@ -129,7 +129,7 @@ class Config { * --conf spark.executor.memoryOverhead=8.38g * --conf spark.rapids.memory.pinnedPool.size=2g * --conf spark.rapids.sql.concurrentGpuTasks=4 - * --conf spark.sql.files.maxPartitionBytes=31.67g + * --conf spark.sql.files.maxPartitionBytes=4g * --conf spark.sql.shuffle.partitions=200 * --conf spark.task.resource.gpu.amount=0.125 * @@ -330,7 +330,7 @@ class AutoTuner(app: ApplicationSummaryInfo, workerInfo: String) extends Logging * MIN_PARTITION_BYTES_RANGE = 128m, MAX_PARTITION_BYTES_RANGE = 256m * (1) Input: maxPartitionBytes = 512m * taskInputSize = 12m - * Output: newMaxPartitionBytes = 512m * (128m/12m) = 5g + * Output: newMaxPartitionBytes = 512m * (128m/12m) = 4g (hit max value) * (2) Input: maxPartitionBytes = 2g * taskInputSize = 512m, * Output: newMaxPartitionBytes = 2g / (512m/128m) = 512m @@ -342,7 +342,7 @@ class AutoTuner(app: ApplicationSummaryInfo, workerInfo: String) extends Logging if (taskInputSize > 0 && taskInputSize < convertFromHumanReadableSize(MIN_PARTITION_BYTES_RANGE)) { // Increase partition size - val calculatedMaxPartitionBytes = Math.max( + val calculatedMaxPartitionBytes = Math.min( maxPartitionBytesNum * (convertFromHumanReadableSize(MIN_PARTITION_BYTES_RANGE) / taskInputSize), convertFromHumanReadableSize(MAX_PARTITION_BYTES_BOUND)) @@ -350,7 +350,7 @@ class AutoTuner(app: ApplicationSummaryInfo, workerInfo: String) extends Logging convertToHumanReadableSize(calculatedMaxPartitionBytes.toLong) } else if (taskInputSize > convertFromHumanReadableSize(MAX_PARTITION_BYTES_RANGE)) { // Decrease partition size - val calculatedMaxPartitionBytes = Math.max( + val calculatedMaxPartitionBytes = Math.min( maxPartitionBytesNum / (taskInputSize / convertFromHumanReadableSize(MAX_PARTITION_BYTES_RANGE)), convertFromHumanReadableSize(MAX_PARTITION_BYTES_BOUND)) From 742c013362957254fdd95688ba1f46193182ea26 Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Tue, 20 Sep 2022 07:03:36 +0800 Subject: [PATCH 120/190] Add GpuColumnVectorUtils to access GpuColumnVector (#6534) * Unshim GpuColumnVector This PR introduces a new UnshimmedGpuColumnVector to access the shimmed GpuColumnVector. Signed-off-by: Bobby Wang * resolve comments * resolve comments Signed-off-by: Bobby Wang --- dist/unshimmed-common-from-spark311.txt | 1 + .../nvidia/spark/rapids/GpuColumnVector.java | 28 +++++++++++++ .../spark/rapids/GpuColumnVectorUtils.scala | 41 +++++++++++++++++++ .../com/nvidia/spark/rapids/ShimLoader.scala | 4 ++ .../InternalColumnarRDDConverterSuite.scala | 20 +++++++-- 5 files changed, 91 insertions(+), 3 deletions(-) create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnVectorUtils.scala diff --git a/dist/unshimmed-common-from-spark311.txt b/dist/unshimmed-common-from-spark311.txt index f9d391bd858..5644bd18ef7 100644 --- a/dist/unshimmed-common-from-spark311.txt +++ b/dist/unshimmed-common-from-spark311.txt @@ -8,6 +8,7 @@ com/nvidia/spark/ParquetCachedBatchSerializer* com/nvidia/spark/RapidsUDF* com/nvidia/spark/SQLPlugin* com/nvidia/spark/rapids/ColumnarRdd* +com/nvidia/spark/rapids/GpuColumnVectorUtils* com/nvidia/spark/rapids/ClouderaShimVersion* com/nvidia/spark/rapids/DatabricksShimVersion* com/nvidia/spark/rapids/ExecutionPlanCaptureCallback* diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuColumnVector.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuColumnVector.java index cf1405ccb89..e6c0c146987 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuColumnVector.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuColumnVector.java @@ -1022,6 +1022,34 @@ public static GpuColumnVector[] extractColumns(Table table, DataType[] colType) } } + /** + * Convert the table into host columns and return them, outside of a ColumnarBatch. + * @param colType the types of the columns. + */ + public static RapidsHostColumnVector[] extractHostColumns(Table table, DataType[] colType) { + try (ColumnarBatch batch = from(table, colType)) { + GpuColumnVector[] gpuCols = extractColumns(batch); + RapidsHostColumnVector[] hostCols = new RapidsHostColumnVector[gpuCols.length]; + try { + for (int i = 0; i < gpuCols.length; i++) { + hostCols[i] = gpuCols[i].copyToHost(); + } + } catch (Exception e) { + for (RapidsHostColumnVector hostCol : hostCols) { + if (hostCol != null) { + try { + hostCol.close(); + } catch (Exception suppressed) { + e.addSuppressed(suppressed); + } + } + } + throw e; + } + return hostCols; + } + } + private final ai.rapids.cudf.ColumnVector cudfCv; /** diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnVectorUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnVectorUtils.scala new file mode 100644 index 00000000000..9fcbba0f4ff --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnVectorUtils.scala @@ -0,0 +1,41 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids + +import java.lang.reflect.Method + +import ai.rapids.cudf.Table + +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.vectorized.ColumnVector + +object GpuColumnVectorUtils { + lazy val extractHostColumnsMethod: Method = ShimLoader.loadGpuColumnVector() + .getDeclaredMethod("extractHostColumns", classOf[Table], classOf[Array[DataType]]) + + /** + * Extract the columns from a table and convert them to RapidsHostColumnVector. + * @param table to be extracted + * @param colType the column types + * @return an array of ColumnVector + */ + def extractHostColumns(table: Table, colType: Array[DataType]): Array[ColumnVector] = { + val columnVectors = extractHostColumnsMethod.invoke(null, table, colType) + columnVectors.asInstanceOf[Array[ColumnVector]] + } + +} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala index 3682ccf74d5..f5b242de7f5 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala @@ -413,4 +413,8 @@ object ShimLoader extends Logging { def newPlanShims(): PlanShims = ShimLoader.newInstanceOf[PlanShims]( "com.nvidia.spark.rapids.shims.PlanShimsImpl" ) + + def loadGpuColumnVector(): Class[_] = { + loadClass("com.nvidia.spark.rapids.GpuColumnVector") + } } diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRDDConverterSuite.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRDDConverterSuite.scala index cfb060a522e..5e8feb27c99 100644 --- a/tests/src/test/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRDDConverterSuite.scala +++ b/tests/src/test/scala/org/apache/spark/sql/rapids/execution/InternalColumnarRDDConverterSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021, NVIDIA CORPORATION. All rights reserved. + * Copyright (c) 2021-2022, NVIDIA CORPORATION. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,7 +18,7 @@ package org.apache.spark.sql.rapids.execution import scala.collection.mutable -import com.nvidia.spark.rapids.{ColumnarToRowIterator, GpuBatchUtilsSuite, NoopMetric, SparkQueryCompareTestSuite, TestResourceFinder} +import com.nvidia.spark.rapids.{ColumnarRdd, ColumnarToRowIterator, GpuBatchUtilsSuite, GpuColumnVectorUtils, NoopMetric, RapidsHostColumnVector, SparkQueryCompareTestSuite, TestResourceFinder} import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder import org.apache.spark.SparkConf @@ -291,5 +291,19 @@ class InternalColumnarRDDConverterSuite extends SparkQueryCompareTestSuite { }, new SparkConf().set("spark.rapids.sql.test.allowedNonGpu", "DeserializeToObjectExec")) } -} + test("Extract RapidsHostColumnVector from GpuColumnVectorUtils") { + withGpuSparkSession(spark => { + val rdd = ColumnarRdd(spark.range(10).toDF()) + val result = rdd.map(table => { + val columns = GpuColumnVectorUtils.extractHostColumns(table, Array(LongType)) + val isRapidsHostColumnVector = columns(0) match { + case _: RapidsHostColumnVector => true + case _ => false + } + isRapidsHostColumnVector + }).collect() + assert(result.forall(_ == true)) + }, new SparkConf().set("spark.rapids.sql.test.allowedNonGpu", "DeserializeToObjectExec")) + } +} From 52bedfb194f9cdbc5e0709b5bafa3b13a03d6c13 Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Tue, 20 Sep 2022 09:30:12 +0800 Subject: [PATCH 121/190] Fix the check of empty batches for partitioning (#6564) Signed-off-by: Firestarman --- .../execution/GpuShuffleExchangeExecBase.scala | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExecBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExecBase.scala index c0f99ca109c..bccbc871326 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExecBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExecBase.scala @@ -300,12 +300,18 @@ object GpuShuffleExchangeExecBase { batch.close() batch = iter.next() } - partitioned = getParts(batch).asInstanceOf[Array[(ColumnarBatch, Int)]] - partitioned.foreach(batches => { - metrics(GpuMetric.NUM_OUTPUT_ROWS) += batches._1.numRows() - }) - metrics(GpuMetric.NUM_OUTPUT_BATCHES) += partitioned.length - at = 0 + // Get a non-empty batch or the last batch. So still need to + // check if it is empty for the later case. + if (batch.numRows > 0) { + partitioned = getParts(batch).asInstanceOf[Array[(ColumnarBatch, Int)]] + partitioned.foreach(batches => { + metrics(GpuMetric.NUM_OUTPUT_ROWS) += batches._1.numRows() + }) + metrics(GpuMetric.NUM_OUTPUT_BATCHES) += partitioned.length + at = 0 + } else { + batch.close() + } } } From 9f49efccf08d20bf1c1b0f66e96e6848cedc92b3 Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Mon, 19 Sep 2022 20:34:10 -0500 Subject: [PATCH 122/190] Disable UCX smoke test temporarily (#6579) Signed-off-by: Alessandro Bellina Signed-off-by: Alessandro Bellina --- jenkins/spark-premerge-build.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/jenkins/spark-premerge-build.sh b/jenkins/spark-premerge-build.sh index 60c645cc232..d01a3a03c3f 100755 --- a/jenkins/spark-premerge-build.sh +++ b/jenkins/spark-premerge-build.sh @@ -109,8 +109,9 @@ rapids_shuffle_smoke_test() { } # using UCX shuffle - PYSP_TEST_spark_executorEnv_UCX_ERROR_SIGNALS="" \ - invoke_shuffle_integration_test + # Disabled temporarily due to: https://github.com/NVIDIA/spark-rapids/issues/6572 + # PYSP_TEST_spark_executorEnv_UCX_ERROR_SIGNALS="" \ + # invoke_shuffle_integration_test # using MULTITHREADED shuffle PYSP_TEST_spark_rapids_shuffle_mode=MULTITHREADED \ From 0c4c5970692712361c993db43c572d995d437743 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Tue, 20 Sep 2022 08:07:21 -0500 Subject: [PATCH 123/190] Work around multiprocess issues with updating Ivy cache (#6577) Signed-off-by: Jason Lowe Signed-off-by: Jason Lowe --- jenkins/spark-tests.sh | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/jenkins/spark-tests.sh b/jenkins/spark-tests.sh index 2fdb81aa49f..37a62d34d1e 100755 --- a/jenkins/spark-tests.sh +++ b/jenkins/spark-tests.sh @@ -186,7 +186,10 @@ run_delta_lake_tests() { DELTA_LAKE_VER="1.2.1" if [[ $SPARK_VER =~ $SPARK_321_PATTERN ]]; then - PYSP_TEST_spark_jars_packages="io.delta:delta-core_2.12:$DELTA_LAKE_VER" \ + # Avoid parallel testing to work around issues with multiple processes trying to + # download and update Ivy cache + TEST_PARALLEL=0 \ + PYSP_TEST_spark_jars_packages="io.delta:delta-core_2.12:$DELTA_LAKE_VER" \ PYSP_TEST_spark_sql_extensions="io.delta.sql.DeltaSparkSessionExtension" \ PYSP_TEST_spark_sql_catalog_spark__catalog="org.apache.spark.sql.delta.catalog.DeltaCatalog" \ ./run_pyspark_from_build.sh -m delta_lake --delta_lake @@ -202,7 +205,10 @@ run_iceberg_tests() { # Iceberg does not support Spark 3.3+ yet if [[ "$ICEBERG_SPARK_VER" < "3.3" ]]; then - PYSP_TEST_spark_jars_packages=org.apache.iceberg:iceberg-spark-runtime-${ICEBERG_SPARK_VER}_2.12:${ICEBERG_VERSION} \ + # Avoid parallel testing to work around issues with multiple processes trying to + # download and update Ivy cache + TEST_PARALLEL=0 \ + PYSP_TEST_spark_jars_packages=org.apache.iceberg:iceberg-spark-runtime-${ICEBERG_SPARK_VER}_2.12:${ICEBERG_VERSION} \ PYSP_TEST_spark_sql_extensions="org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions" \ PYSP_TEST_spark_sql_catalog_spark__catalog="org.apache.iceberg.spark.SparkSessionCatalog" \ PYSP_TEST_spark_sql_catalog_spark__catalog_type="hadoop" \ @@ -217,7 +223,10 @@ run_avro_tests() { # Workaround to avoid appending avro jar file by '--jars', # which would be addressed by https://github.com/NVIDIA/spark-rapids/issues/6532 rm -vf $LOCAL_JAR_PATH/spark-avro*.jar - PYSP_TEST_spark_jars_packages="org.apache.spark:spark-avro_2.12:${SPARK_VER}" \ + # Avoid parallel testing to work around issues with multiple processes trying to + # download and update Ivy cache + TEST_PARALLEL=0 \ + PYSP_TEST_spark_jars_packages="org.apache.spark:spark-avro_2.12:${SPARK_VER}" \ ./run_pyspark_from_build.sh -k avro } From 0443f7db525bf77d9694966cc31b118c5409dcf1 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Tue, 20 Sep 2022 08:36:31 -0500 Subject: [PATCH 124/190] Bump snakeyaml version to 1.32 (#6576) Signed-off-by: Jason Lowe Signed-off-by: Jason Lowe --- tools/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/pom.xml b/tools/pom.xml index df76292f249..06b6f352ea2 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -87,7 +87,7 @@ org.yaml snakeyaml - 1.31 + 1.32 From a5bc7b85ce5579439f143485d79bb84f9313895a Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Tue, 20 Sep 2022 09:53:56 -0500 Subject: [PATCH 125/190] Multi-threaded shuffle reader for RapidsShuffleManager [databricks] (#6511) * Multi-threaded shuffle reader for RapidsShuffleManager Signed-off-by: Alessandro Bellina * OutOfDirectMemoryError -> OutOfMemoryError due to shading in db * Remove debug statement * Add smoke test to jenkins/databricks/test.sh * Revert "Add smoke test to jenkins/databricks/test.sh" This reverts commit 17707e5159fc5d60ce02d506ae158f17b6cadca5. * Documentation fixes * Use withSparkSession rather than creating a context manually in the reader test * Fix bug in the reader suite where it was stopping the SparkContext * Remove spurious import * Add flow control in shuffle reader * Remove debug logging * Fix late introduced bug * Clarify config doc strings * Partly handle review comments * Remove log-then-throw * Keep pending iterators in an array buffer rather than a priority queue * Use a mutable.Queue instead for performance reasons Signed-off-by: Alessandro Bellina --- .../rapids-shuffle.md | 34 +- docs/configs.md | 4 +- jenkins/spark-premerge-build.sh | 1 + .../RapidsShuffleInternalManager.scala | 21 +- .../shims/RapidsShuffleThreadedReader.scala | 60 + .../shims/RapidsShuffleThreadedWriter.scala | 11 +- .../RapidsShuffleBlockFetcherIterator.scala | 1038 +++++++++++ .../RapidsShuffleInternalManager.scala | 22 +- .../RapidsShuffleInternalManager.scala | 20 +- .../RapidsShuffleInternalManager.scala | 21 +- .../RapidsShuffleInternalManager.scala | 22 +- .../shims/RapidsShuffleThreadedWriter.scala | 20 +- .../storage/RapidsPushBasedFetchHelper.scala | 329 ++++ .../RapidsShuffleBlockFetcherIterator.scala | 1624 +++++++++++++++++ .../RapidsShuffleInternalManager.scala | 21 +- .../shims/RapidsShuffleThreadedReader.scala | 66 + .../RapidsShuffleInternalManager.scala | 21 +- .../RapidsShuffleInternalManager.scala | 21 +- .../RapidsShuffleInternalManager.scala | 20 +- .../RapidsShuffleInternalManager.scala | 23 +- .../RapidsShuffleInternalManager.scala | 21 +- .../shims/RapidsShuffleThreadedReader.scala | 66 + .../RapidsShuffleInternalManager.scala | 21 +- .../rapids/GpuColumnarBatchSerializer.scala | 141 +- .../com/nvidia/spark/rapids/RapidsConf.scala | 28 +- .../RapidsShuffleInternalManagerBase.scala | 671 ++++++- .../GpuShuffleExchangeExecBase.scala | 17 +- .../RapidsShuffleThreadedReaderSuite.scala | 175 ++ .../RapidsShuffleThreadedWriterSuite.scala | 61 +- 29 files changed, 4201 insertions(+), 399 deletions(-) create mode 100644 sql-plugin/src/main/311until320-all/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala create mode 100644 sql-plugin/src/main/311until320-all/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala create mode 100644 sql-plugin/src/main/320+/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala create mode 100644 sql-plugin/src/main/320+/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala create mode 100644 sql-plugin/src/main/320until330-all/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala create mode 100644 sql-plugin/src/main/330until340/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala create mode 100644 tests/src/test/320+-noncdh-nondb/scala/org/apache/spark/sql/rapids/RapidsShuffleThreadedReaderSuite.scala diff --git a/docs/additional-functionality/rapids-shuffle.md b/docs/additional-functionality/rapids-shuffle.md index 097711c3188..7690bd08b6f 100644 --- a/docs/additional-functionality/rapids-shuffle.md +++ b/docs/additional-functionality/rapids-shuffle.md @@ -7,10 +7,15 @@ nav_order: 5 # RAPIDS Shuffle Manager The RAPIDS Shuffle Manager is an implementation of the `ShuffleManager` interface in Apache Spark -that allows custom mechanisms to exchange shuffle data. It has two components: a spillable cache, -and a transport that can utilize Remote Direct Memory Access (RDMA) and high-bandwidth transfers -within a node that has multiple GPUs. This is possible because the plugin utilizes -[Unified Communication X (UCX)](https://www.openucx.org/) as its transport. +that allows custom mechanisms to exchange shuffle data. We currently expose two modes of operation: +UCX and Multi Threaded (experimental). + +## UCX Mode + +UCX mode is the default for the RAPIDS Shuffle Manager. It has two components: a spillable cache, +and a transport that can utilize Remote Direct Memory Access (RDMA) and high-bandwidth transfers +within a node that has multiple GPUs. This is possible because the plugin +utilizes [Unified Communication X (UCX)](https://www.openucx.org/) as its transport. - **Spillable cache**: This store keeps GPU data close by where it was produced in device memory, but can spill in the following cases: @@ -426,3 +431,24 @@ for this, other than to trigger a GC cycle on the driver. Spark has a configuration `spark.cleaner.periodicGC.interval` (defaults to 30 minutes), that can be used to periodically cause garbage collection. If you are experiencing OOM situations, or performance degradation with several Spark actions, consider tuning this setting in your jobs. + +## Multi Threaded Mode (experimental) + +This mode is similar to the built-in Spark shuffle, but it attempts to use more CPU threads +for compute-intensive tasks, such as compression and decompression. This mode does not use UCX, and +so it does not require a UCX installation. + +Minimum configuration: + +```shell +--conf spark.shuffle.manager=com.nvidia.spark.rapids.[shim package].RapidsShuffleManager \ +--conf spark.driver.extraClassPath=${SPARK_RAPIDS_PLUGIN_JAR} \ +--conf spark.executor.extraClassPath=${SPARK_RAPIDS_PLUGIN_JAR} \ +--conf spark.rapids.shuffle.mode=MULTITHREADED +``` + +By default, a thread pool of 20 threads is used for shuffle writes and reads. This +configuration can be independently changed for writers and readers using: +`spark.rapids.shuffle.multiThreaded.[writer|reader].threads`. An appropriate value for these +pools is the number of cores in the system divided by the number of executors per machine. + diff --git a/docs/configs.md b/docs/configs.md index 627cc6098fa..9108f125919 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -56,7 +56,9 @@ Name | Description | Default Value spark.rapids.python.memory.gpu.pooling.enabled|Should RMM in Python workers act as a pooling allocator for GPU memory, or should it just pass through to CUDA memory allocation directly. When not specified, It will honor the value of config 'spark.rapids.memory.gpu.pooling.enabled'|None spark.rapids.shuffle.enabled|Enable or disable the RAPIDS Shuffle Manager at runtime. The [RAPIDS Shuffle Manager](additional-functionality/rapids-shuffle.md) must already be configured. When set to `false`, the built-in Spark shuffle will be used. |true spark.rapids.shuffle.mode|RAPIDS Shuffle Manager mode. The default mode is "UCX", which has to be installed in the system. Consider setting to "CACHE_ONLY" if running with a single executor and UCX is not installed, for short-circuit cached shuffle (for testing purposes). Set to "MULTITHREADED" for an experimental mode that uses a thread pool to speed up shuffle writes without needing UCX. Note: Changing this mode dynamically is not supported.|UCX -spark.rapids.shuffle.multiThreaded.writer.threads|The number of threads to use for writing shuffle blocks per executor.|20 +spark.rapids.shuffle.multiThreaded.maxBytesInFlight|The size limit, in bytes, that the RAPIDS shuffle manager configured in "MULTITHREADED" mode will allow to be deserialized concurrently.|2147483647 +spark.rapids.shuffle.multiThreaded.reader.threads|The number of threads to use for reading shuffle blocks per executor in the RAPIDS shuffle manager configured in "MULTITHREADED" mode. There are two special values: 0 = feature is disabled, falls back to Spark built-in shuffle reader; 1 = our implementation of Spark's built-in shuffle reader with extra metrics.|20 +spark.rapids.shuffle.multiThreaded.writer.threads|The number of threads to use for writing shuffle blocks per executor in the RAPIDS shuffle manager configured in "MULTITHREADED" mode. There are two special values: 0 = feature is disabled, falls back to Spark built-in shuffle writer; 1 = our implementation of Spark's built-in shuffle writer with extra metrics.|20 spark.rapids.shuffle.transport.earlyStart|Enable early connection establishment for RAPIDS Shuffle|true spark.rapids.shuffle.transport.earlyStart.heartbeatInterval|Shuffle early start heartbeat interval (milliseconds). Executors will send a heartbeat RPC message to the driver at this interval|5000 spark.rapids.shuffle.transport.earlyStart.heartbeatTimeout|Shuffle early start heartbeat timeout (milliseconds). Executors that don't heartbeat within this timeout will be considered stale. This timeout must be higher than the value for spark.rapids.shuffle.transport.earlyStart.heartbeatInterval|10000 diff --git a/jenkins/spark-premerge-build.sh b/jenkins/spark-premerge-build.sh index d01a3a03c3f..87b54c30145 100755 --- a/jenkins/spark-premerge-build.sh +++ b/jenkins/spark-premerge-build.sh @@ -116,6 +116,7 @@ rapids_shuffle_smoke_test() { # using MULTITHREADED shuffle PYSP_TEST_spark_rapids_shuffle_mode=MULTITHREADED \ PYSP_TEST_spark_rapids_shuffle_multiThreaded_writer_threads=2 \ + PYSP_TEST_spark_rapids_shuffle_multiThreaded_reader_threads=2 \ invoke_shuffle_integration_test $SPARK_HOME/sbin/spark-daemon.sh stop org.apache.spark.deploy.worker.Worker 1 diff --git a/sql-plugin/src/main/311-nondb/scala/org/apache/spark/sql/rapids/shims/spark311/RapidsShuffleInternalManager.scala b/sql-plugin/src/main/311-nondb/scala/org/apache/spark/sql/rapids/shims/spark311/RapidsShuffleInternalManager.scala index b4e11601d0f..30a6782fceb 100644 --- a/sql-plugin/src/main/311-nondb/scala/org/apache/spark/sql/rapids/shims/spark311/RapidsShuffleInternalManager.scala +++ b/sql-plugin/src/main/311-nondb/scala/org/apache/spark/sql/rapids/shims/spark311/RapidsShuffleInternalManager.scala @@ -16,11 +16,9 @@ package org.apache.spark.sql.rapids.shims.spark311 -import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.SparkConf import org.apache.spark.shuffle._ -import org.apache.spark.shuffle.sort.BypassMergeSortShuffleHandle import org.apache.spark.sql.rapids.{ProxyRapidsShuffleInternalManagerBase, RapidsShuffleInternalManagerBase} -import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter /** * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. @@ -28,22 +26,7 @@ import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter * `ShuffleManager` and `SortShuffleManager` classes. */ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) - extends RapidsShuffleInternalManagerBase(conf, isDriver) { - - override def makeBypassMergeSortShuffleWriter[K, V]( - handle: BypassMergeSortShuffleHandle[K, V], - mapId: Long, - context: TaskContext, - metricsReporter: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { - new RapidsShuffleThreadedWriter[K, V]( - blockManager, - handle, - mapId, - conf, - metricsReporter, - execComponents.get) - } -} + extends RapidsShuffleInternalManagerBase(conf, isDriver) class ProxyRapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) extends ProxyRapidsShuffleInternalManagerBase(conf, isDriver) diff --git a/sql-plugin/src/main/311until320-all/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala b/sql-plugin/src/main/311until320-all/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala new file mode 100644 index 00000000000..eae809ae9f0 --- /dev/null +++ b/sql-plugin/src/main/311until320-all/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala @@ -0,0 +1,60 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.rapids.shims + +import ai.rapids.cudf.{NvtxColor, NvtxRange} + +import org.apache.spark.{MapOutputTracker, SparkEnv, TaskContext} +import org.apache.spark.serializer.SerializerManager +import org.apache.spark.shuffle.ShuffleReadMetricsReporter +import org.apache.spark.sql.rapids.{RapidsShuffleThreadedReaderBase, ShuffleHandleWithMetrics} +import org.apache.spark.storage.BlockManager + +class RapidsShuffleThreadedReader[K, C] ( + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + handle: ShuffleHandleWithMetrics[K, C, C], + context: TaskContext, + readMetrics: ShuffleReadMetricsReporter, + maxBytesInFlight: Long, + serializerManager: SerializerManager = SparkEnv.get.serializerManager, + blockManager: BlockManager = SparkEnv.get.blockManager, + mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker, + canUseBatchFetch: Boolean = false, + numReaderThreads: Int = 0) + extends RapidsShuffleThreadedReaderBase[K, C]( + handle, + context, + readMetrics, + maxBytesInFlight, + serializerManager = serializerManager, + blockManager = blockManager, + mapOutputTracker = mapOutputTracker, + canUseBatchFetch = canUseBatchFetch, + numReaderThreads = numReaderThreads) { + + override protected def getMapSizes: GetMapSizesResult = { + val shuffleId = handle.shuffleId + val mapSizes = withResource(new NvtxRange("getMapSizesByExecId", NvtxColor.CYAN)) { _ => + mapOutputTracker.getMapSizesByExecutorId( + shuffleId, startMapIndex, endMapIndex, startPartition, endPartition) + } + GetMapSizesResult(mapSizes, canEnableBatchFetch = true) + } +} diff --git a/sql-plugin/src/main/311until320-all/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala b/sql-plugin/src/main/311until320-all/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala index cd73e60937b..14093c1648a 100644 --- a/sql-plugin/src/main/311until320-all/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala +++ b/sql-plugin/src/main/311until320-all/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala @@ -19,24 +19,25 @@ package org.apache.spark.sql.rapids.shims import org.apache.spark.SparkConf import org.apache.spark.shuffle.ShuffleWriteMetricsReporter import org.apache.spark.shuffle.api.{ShuffleExecutorComponents, ShuffleMapOutputWriter} -import org.apache.spark.shuffle.sort.BypassMergeSortShuffleHandle -import org.apache.spark.sql.rapids.RapidsShuffleThreadedWriterBase +import org.apache.spark.sql.rapids.{RapidsShuffleThreadedWriterBase, ShuffleHandleWithMetrics} import org.apache.spark.storage.BlockManager class RapidsShuffleThreadedWriter[K, V]( blockManager: BlockManager, - handle: BypassMergeSortShuffleHandle[K, V], + handle: ShuffleHandleWithMetrics[K, V, V], mapId: Long, sparkConf: SparkConf, writeMetrics: ShuffleWriteMetricsReporter, - shuffleExecutorComponents: ShuffleExecutorComponents) + shuffleExecutorComponents: ShuffleExecutorComponents, + numWriterThreads: Int) extends RapidsShuffleThreadedWriterBase[K, V]( blockManager, handle, mapId, sparkConf, writeMetrics, - shuffleExecutorComponents) { + shuffleExecutorComponents, + numWriterThreads) { // emptyChecksums: unused in versions of Spark before 3.2.0 override def doCommitAllPartitions( diff --git a/sql-plugin/src/main/311until320-all/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala b/sql-plugin/src/main/311until320-all/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala new file mode 100644 index 00000000000..8133125c32d --- /dev/null +++ b/sql-plugin/src/main/311until320-all/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala @@ -0,0 +1,1038 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.storage + +import java.io.{InputStream, IOException} +import java.nio.channels.ClosedByInterruptException +import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} +import javax.annotation.concurrent.GuardedBy + +import scala.collection.mutable +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, LinkedHashMap, Queue} +import scala.util.{Failure, Success} + +import org.apache.commons.io.IOUtils + +import org.apache.spark.{SparkEnv, SparkException, TaskContext} +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.shuffle._ +import org.apache.spark.network.util.TransportConf +import org.apache.spark.serializer.SerializerManager +import org.apache.spark.shuffle.{FetchFailedException, ShuffleReadMetricsReporter} +import org.apache.spark.storage.RapidsShuffleBlockFetcherIterator.SuccessFetchResult +import org.apache.spark.util.{CompletionIterator, TaskCompletionListener, Utils} + +/** + * Taken mostly verbatim from `ShuffleBlockFetcherIterator` except for + * a change to the ownership of `currentResult` (which contains the netty buffer). + * Instead of this iterator owning the result and clearing it on `next`, the + * `BufferReleasingInputStream` is in charge of that. This allows for multiple threads + * to consume different `BufferReleasingInputStream`s produced from this single iterator. + */ + +/** + * An iterator that fetches multiple blocks. For local blocks, it fetches from the local block + * manager. For remote blocks, it fetches them using the provided BlockTransferService. + * + * This creates an iterator of (BlockID, InputStream) tuples so the caller can handle blocks + * in a pipelined fashion as they are received. + * + * The implementation throttles the remote fetches so they don't exceed maxBytesInFlight to avoid + * using too much memory. + * + * @param context [[TaskContext]], used for metrics update + * @param shuffleClient [[BlockStoreClient]] for fetching remote blocks + * @param blockManager [[BlockManager]] for reading local blocks + * @param blocksByAddress list of blocks to fetch grouped by the [[BlockManagerId]]. + * For each block we also require two info: 1. the size (in bytes as a long + * field) in order to throttle the memory usage; 2. the mapIndex for this + * block, which indicate the index in the map stage. + * Note that zero-sized blocks are already excluded, which happened in + * [[org.apache.spark.MapOutputTracker.convertMapStatuses]]. + * @param streamWrapper A function to wrap the returned input stream. + * @param maxBytesInFlight max size (in bytes) of remote blocks to fetch at any given point. + * @param maxReqsInFlight max number of remote requests to fetch blocks at any given point. + * @param maxBlocksInFlightPerAddress max number of shuffle blocks being fetched at any given point + * for a given remote host:port. + * @param maxReqSizeShuffleToMem max size (in bytes) of a request that can be shuffled to memory. + * @param detectCorrupt whether to detect any corruption in fetched blocks. + * @param shuffleMetrics used to report shuffle metrics. + * @param doBatchFetch fetch continuous shuffle blocks from same executor in batch if the server + * side supports. + */ +private[spark] +final class RapidsShuffleBlockFetcherIterator( + context: TaskContext, + shuffleClient: BlockStoreClient, + blockManager: BlockManager, + blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])], + streamWrapper: (BlockId, InputStream) => InputStream, + maxBytesInFlight: Long, + maxReqsInFlight: Int, + maxBlocksInFlightPerAddress: Int, + maxReqSizeShuffleToMem: Long, + detectCorrupt: Boolean, + detectCorruptUseExtraMemory: Boolean, + shuffleMetrics: ShuffleReadMetricsReporter, + doBatchFetch: Boolean) + extends Iterator[(BlockId, InputStream)] with DownloadFileManager with Logging { + + import RapidsShuffleBlockFetcherIterator._ + + // Make remote requests at most maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + private val targetRemoteRequestSize = math.max(maxBytesInFlight / 5, 1L) + + /** + * Total number of blocks to fetch. + */ + private[this] var numBlocksToFetch = 0 + + /** + * The number of blocks processed by the caller. The iterator is exhausted when + * [[numBlocksProcessed]] == [[numBlocksToFetch]]. + */ + private[this] var numBlocksProcessed = 0 + + private[this] val startTimeNs = System.nanoTime() + + /** Local blocks to fetch, excluding zero-sized blocks. */ + private[this] val localBlocks = scala.collection.mutable.LinkedHashSet[(BlockId, Int)]() + + /** Host local blockIds to fetch by executors, excluding zero-sized blocks. */ + private[this] val hostLocalBlocksByExecutor = + LinkedHashMap[BlockManagerId, Seq[(BlockId, Long, Int)]]() + + /** Host local blocks to fetch, excluding zero-sized blocks. */ + private[this] val hostLocalBlocks = scala.collection.mutable.LinkedHashSet[(BlockId, Int)]() + + /** + * A queue to hold our results. This turns the asynchronous model provided by + * [[org.apache.spark.network.BlockTransferService]] into a synchronous model (iterator). + */ + private[this] val results = new LinkedBlockingQueue[FetchResult] + + /** + * Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that + * the number of bytes in flight is limited to maxBytesInFlight. + */ + private[this] val fetchRequests = new Queue[FetchRequest] + + /** + * Queue of fetch requests which could not be issued the first time they were dequeued. These + * requests are tried again when the fetch constraints are satisfied. + */ + private[this] val deferredFetchRequests = new HashMap[BlockManagerId, Queue[FetchRequest]]() + + /** Current bytes in flight from our requests */ + private[this] var bytesInFlight = 0L + + /** Current number of requests in flight */ + private[this] var reqsInFlight = 0 + + /** Current number of blocks in flight per host:port */ + private[this] val numBlocksInFlightPerAddress = new HashMap[BlockManagerId, Int]() + + /** + * The blocks that can't be decompressed successfully, it is used to guarantee that we retry + * at most once for those corrupted blocks. + */ + private[this] val corruptedBlocks = mutable.HashSet[BlockId]() + + /** + * Whether the iterator is still active. If isZombie is true, the callback interface will no + * longer place fetched blocks into [[results]]. + */ + @GuardedBy("this") + private[this] var isZombie = false + + /** + * A set to store the files used for shuffling remote huge blocks. Files in this set will be + * deleted when cleanup. This is a layer of defensiveness against disk file leaks. + */ + @GuardedBy("this") + private[this] val shuffleFilesSet = mutable.HashSet[DownloadFile]() + + private[this] val onCompleteCallback = new RapidsShuffleFetchCompletionListener(this) + + initialize() + + def resultCount: Int = results.size() + + override def createTempFile(transportConf: TransportConf): DownloadFile = { + // we never need to do any encryption or decryption here, regardless of configs, because that + // is handled at another layer in the code. When encryption is enabled, shuffle data is written + // to disk encrypted in the first place, and sent over the network still encrypted. + new SimpleDownloadFile( + blockManager.diskBlockManager.createTempLocalBlock()._2, transportConf) + } + + override def registerTempFileToClean(file: DownloadFile): Boolean = synchronized { + if (isZombie) { + false + } else { + shuffleFilesSet += file + true + } + } + + /** + * Mark the iterator as zombie, and release all buffers that haven't been deserialized yet. + */ + private[storage] def cleanup(): Unit = { + synchronized { + isZombie = true + } + // Release buffers in the results queue + val iter = results.iterator() + while (iter.hasNext) { + val result = iter.next() + result match { + case SuccessFetchResult(blockId, mapIndex, address, _, buf, _) => + if (address != blockManager.blockManagerId) { + if (hostLocalBlocks.contains(blockId -> mapIndex)) { + shuffleMetrics.incLocalBlocksFetched(1) + shuffleMetrics.incLocalBytesRead(buf.size) + } else { + shuffleMetrics.incRemoteBytesRead(buf.size) + if (buf.isInstanceOf[FileSegmentManagedBuffer]) { + shuffleMetrics.incRemoteBytesReadToDisk(buf.size) + } + shuffleMetrics.incRemoteBlocksFetched(1) + } + } + buf.release() + case _ => + } + } + shuffleFilesSet.foreach { file => + if (!file.delete()) { + logWarning("Failed to cleanup shuffle fetch temp file " + file.path()) + } + } + } + + private[this] def sendRequest(req: FetchRequest): Unit = { + logDebug("Sending request for %d blocks (%s) from %s".format( + req.blocks.size, Utils.bytesToString(req.size), req.address.hostPort)) + bytesInFlight += req.size + reqsInFlight += 1 + + // so we can look up the block info of each blockID + val infoMap = req.blocks.map { + case FetchBlockInfo(blockId, size, mapIndex) => (blockId.toString, (size, mapIndex)) + }.toMap + val remainingBlocks = new HashSet[String]() ++= infoMap.keys + val blockIds = req.blocks.map(_.blockId.toString) + val address = req.address + + val blockFetchingListener = new BlockFetchingListener { + override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = { + // Only add the buffer to results queue if the iterator is not zombie, + // i.e. cleanup() has not been called yet. + RapidsShuffleBlockFetcherIterator.this.synchronized { + if (!isZombie) { + // Increment the ref count because we need to pass this to a different thread. + // This needs to be released after use. + buf.retain() + remainingBlocks -= blockId + results.put(new SuccessFetchResult(BlockId(blockId), infoMap(blockId)._2, + address, infoMap(blockId)._1, buf, remainingBlocks.isEmpty)) + logDebug("remainingBlocks: " + remainingBlocks) + } + } + logTrace(s"Got remote block $blockId after ${Utils.getUsedTimeNs(startTimeNs)}") + } + + override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = { + logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e) + results.put(new FailureFetchResult(BlockId(blockId), infoMap(blockId)._2, address, e)) + } + } + + // Fetch remote shuffle blocks to disk when the request is too large. Since the shuffle data is + // already encrypted and compressed over the wire(w.r.t. the related configs), we can just fetch + // the data and write it to file directly. + if (req.size > maxReqSizeShuffleToMem) { + shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray, + blockFetchingListener, this) + } else { + shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray, + blockFetchingListener, null) + } + } + + private[this] def partitionBlocksByFetchMode(): ArrayBuffer[FetchRequest] = { + logDebug(s"maxBytesInFlight: $maxBytesInFlight, targetRemoteRequestSize: " + + s"$targetRemoteRequestSize, maxBlocksInFlightPerAddress: $maxBlocksInFlightPerAddress") + + // Partition to local, host-local and remote blocks. Remote blocks are further split into + // FetchRequests of size at most maxBytesInFlight in order to limit the amount of data in flight + val collectedRemoteRequests = new ArrayBuffer[FetchRequest] + var localBlockBytes = 0L + var hostLocalBlockBytes = 0L + var remoteBlockBytes = 0L + + val fallback = FallbackStorage.FALLBACK_BLOCK_MANAGER_ID.executorId + for ((address, blockInfos) <- blocksByAddress) { + if (Seq(blockManager.blockManagerId.executorId, fallback).contains(address.executorId)) { + checkBlockSizes(blockInfos) + val mergedBlockInfos = mergeContinuousShuffleBlockIdsIfNeeded( + blockInfos.map(info => FetchBlockInfo(info._1, info._2, info._3)), doBatchFetch) + numBlocksToFetch += mergedBlockInfos.size + localBlocks ++= mergedBlockInfos.map(info => (info.blockId, info.mapIndex)) + localBlockBytes += mergedBlockInfos.map(_.size).sum + } else if (blockManager.hostLocalDirManager.isDefined && + address.host == blockManager.blockManagerId.host) { + checkBlockSizes(blockInfos) + val mergedBlockInfos = mergeContinuousShuffleBlockIdsIfNeeded( + blockInfos.map(info => FetchBlockInfo(info._1, info._2, info._3)), doBatchFetch) + numBlocksToFetch += mergedBlockInfos.size + val blocksForAddress = + mergedBlockInfos.map(info => (info.blockId, info.size, info.mapIndex)) + hostLocalBlocksByExecutor += address -> blocksForAddress + hostLocalBlocks ++= blocksForAddress.map(info => (info._1, info._3)) + hostLocalBlockBytes += mergedBlockInfos.map(_.size).sum + } else { + remoteBlockBytes += blockInfos.map(_._2).sum + val (_, timeCost) = Utils.timeTakenMs[Unit] { + collectFetchRequests(address, blockInfos, collectedRemoteRequests) + } + logDebug(s"Collected remote fetch requests for $address in $timeCost ms") + } + } + val numRemoteBlocks = collectedRemoteRequests.map(_.blocks.size).sum + val totalBytes = localBlockBytes + remoteBlockBytes + hostLocalBlockBytes + assert(numBlocksToFetch == localBlocks.size + hostLocalBlocks.size + numRemoteBlocks, + s"The number of non-empty blocks $numBlocksToFetch doesn't equal to the number of local " + + s"blocks ${localBlocks.size} + the number of host-local blocks ${hostLocalBlocks.size} " + + s"+ the number of remote blocks ${numRemoteBlocks}.") + logInfo(s"Getting $numBlocksToFetch (${Utils.bytesToString(totalBytes)}) non-empty blocks " + + s"including ${localBlocks.size} (${Utils.bytesToString(localBlockBytes)}) local and " + + s"${hostLocalBlocks.size} (${Utils.bytesToString(hostLocalBlockBytes)}) " + + s"host-local and $numRemoteBlocks (${Utils.bytesToString(remoteBlockBytes)}) remote blocks") + collectedRemoteRequests + } + + private def createFetchRequest( + blocks: Seq[FetchBlockInfo], + address: BlockManagerId): FetchRequest = { + logDebug(s"Creating fetch request of ${blocks.map(_.size).sum} at $address " + + s"with ${blocks.size} blocks") + FetchRequest(address, blocks) + } + + private def createFetchRequests( + curBlocks: Seq[FetchBlockInfo], + address: BlockManagerId, + isLast: Boolean, + collectedRemoteRequests: ArrayBuffer[FetchRequest]): ArrayBuffer[FetchBlockInfo] = { + val mergedBlocks = mergeContinuousShuffleBlockIdsIfNeeded(curBlocks, doBatchFetch) + numBlocksToFetch += mergedBlocks.size + val retBlocks = new ArrayBuffer[FetchBlockInfo] + if (mergedBlocks.length <= maxBlocksInFlightPerAddress) { + collectedRemoteRequests += createFetchRequest(mergedBlocks, address) + } else { + mergedBlocks.grouped(maxBlocksInFlightPerAddress).foreach { blocks => + if (blocks.length == maxBlocksInFlightPerAddress || isLast) { + collectedRemoteRequests += createFetchRequest(blocks, address) + } else { + // The last group does not exceed `maxBlocksInFlightPerAddress`. Put it back + // to `curBlocks`. + retBlocks ++= blocks + numBlocksToFetch -= blocks.size + } + } + } + retBlocks + } + + private def collectFetchRequests( + address: BlockManagerId, + blockInfos: Seq[(BlockId, Long, Int)], + collectedRemoteRequests: ArrayBuffer[FetchRequest]): Unit = { + val iterator = blockInfos.iterator + var curRequestSize = 0L + var curBlocks = new ArrayBuffer[FetchBlockInfo]() + + while (iterator.hasNext) { + val (blockId, size, mapIndex) = iterator.next() + assertPositiveBlockSize(blockId, size) + curBlocks += FetchBlockInfo(blockId, size, mapIndex) + curRequestSize += size + // For batch fetch, the actual block in flight should count for merged block. + val mayExceedsMaxBlocks = !doBatchFetch && curBlocks.size >= maxBlocksInFlightPerAddress + if (curRequestSize >= targetRemoteRequestSize || mayExceedsMaxBlocks) { + curBlocks = createFetchRequests(curBlocks.toSeq, address, isLast = false, + collectedRemoteRequests) + curRequestSize = curBlocks.map(_.size).sum + } + } + // Add in the final request + if (curBlocks.nonEmpty) { + createFetchRequests(curBlocks.toSeq, address, isLast = true, collectedRemoteRequests) + } + } + + private def assertPositiveBlockSize(blockId: BlockId, blockSize: Long): Unit = { + if (blockSize < 0) { + throw BlockException(blockId, "Negative block size " + size) + } else if (blockSize == 0) { + throw BlockException(blockId, "Zero-sized blocks should be excluded.") + } + } + + private def checkBlockSizes(blockInfos: Seq[(BlockId, Long, Int)]): Unit = { + blockInfos.foreach { case (blockId, size, _) => assertPositiveBlockSize(blockId, size) } + } + + /** + * Fetch the local blocks while we are fetching remote blocks. This is ok because + * `ManagedBuffer`'s memory is allocated lazily when we create the input stream, so all we + * track in-memory are the ManagedBuffer references themselves. + */ + private[this] def fetchLocalBlocks(): Unit = { + logDebug(s"Start fetching local blocks: ${localBlocks.mkString(", ")}") + val iter = localBlocks.iterator + while (iter.hasNext) { + val (blockId, mapIndex) = iter.next() + try { + val buf = blockManager.getLocalBlockData(blockId) + shuffleMetrics.incLocalBlocksFetched(1) + shuffleMetrics.incLocalBytesRead(buf.size) + buf.retain() + results.put(new SuccessFetchResult(blockId, mapIndex, blockManager.blockManagerId, + buf.size(), buf, false)) + } catch { + // If we see an exception, stop immediately. + case e: Exception => + e match { + // ClosedByInterruptException is an excepted exception when kill task, + // don't log the exception stack trace to avoid confusing users. + // See: SPARK-28340 + case ce: ClosedByInterruptException => + logError("Error occurred while fetching local blocks, " + ce.getMessage) + case ex: Exception => logError("Error occurred while fetching local blocks", ex) + } + results.put(new FailureFetchResult(blockId, mapIndex, blockManager.blockManagerId, e)) + return + } + } + } + + private[this] def fetchHostLocalBlock( + blockId: BlockId, + mapIndex: Int, + localDirs: Array[String], + blockManagerId: BlockManagerId): Boolean = { + try { + val buf = blockManager.getHostLocalShuffleData(blockId, localDirs) + buf.retain() + results.put(SuccessFetchResult(blockId, mapIndex, blockManagerId, buf.size(), buf, + isNetworkReqDone = false)) + true + } catch { + case e: Exception => + // If we see an exception, stop immediately. + logError(s"Error occurred while fetching local blocks", e) + results.put(FailureFetchResult(blockId, mapIndex, blockManagerId, e)) + false + } + } + + /** + * Fetch the host-local blocks while we are fetching remote blocks. This is ok because + * `ManagedBuffer`'s memory is allocated lazily when we create the input stream, so all we + * track in-memory are the ManagedBuffer references themselves. + */ + private[this] def fetchHostLocalBlocks(hostLocalDirManager: HostLocalDirManager): Unit = { + val cachedDirsByExec = hostLocalDirManager.getCachedHostLocalDirs + val (hostLocalBlocksWithCachedDirs, hostLocalBlocksWithMissingDirs) = { + val (hasCache, noCache) = hostLocalBlocksByExecutor.partition { case (hostLocalBmId, _) => + cachedDirsByExec.contains(hostLocalBmId.executorId) + } + (hasCache.toMap, noCache.toMap) + } + + if (hostLocalBlocksWithMissingDirs.nonEmpty) { + logDebug(s"Asynchronous fetching host-local blocks without cached executors' dir: " + + s"${hostLocalBlocksWithMissingDirs.mkString(", ")}") + + // If the external shuffle service is enabled, we'll fetch the local directories for + // multiple executors from the external shuffle service, which located at the same host + // with the executors, in once. Otherwise, we'll fetch the local directories from those + // executors directly one by one. The fetch requests won't be too much since one host is + // almost impossible to have many executors at the same time practically. + val dirFetchRequests = if (blockManager.externalShuffleServiceEnabled) { + val host = blockManager.blockManagerId.host + val port = blockManager.externalShuffleServicePort + Seq((host, port, hostLocalBlocksWithMissingDirs.keys.toArray)) + } else { + hostLocalBlocksWithMissingDirs.keys.map(bmId => (bmId.host, bmId.port, Array(bmId))).toSeq + } + + dirFetchRequests.foreach { case (host, port, bmIds) => + hostLocalDirManager.getHostLocalDirs(host, port, bmIds.map(_.executorId)) { + case Success(dirsByExecId) => + fetchMultipleHostLocalBlocks( + hostLocalBlocksWithMissingDirs.filterKeys(bmIds.contains).toMap, + dirsByExecId, + cached = false) + + case Failure(throwable) => + logError("Error occurred while fetching host local blocks", throwable) + val bmId = bmIds.head + val blockInfoSeq = hostLocalBlocksWithMissingDirs(bmId) + val (blockId, _, mapIndex) = blockInfoSeq.head + results.put(FailureFetchResult(blockId, mapIndex, bmId, throwable)) + } + } + } + + if (hostLocalBlocksWithCachedDirs.nonEmpty) { + logDebug(s"Synchronous fetching host-local blocks with cached executors' dir: " + + s"${hostLocalBlocksWithCachedDirs.mkString(", ")}") + fetchMultipleHostLocalBlocks(hostLocalBlocksWithCachedDirs, cachedDirsByExec, cached = true) + } + } + + private def fetchMultipleHostLocalBlocks( + bmIdToBlocks: Map[BlockManagerId, Seq[(BlockId, Long, Int)]], + localDirsByExecId: Map[String, Array[String]], + cached: Boolean): Unit = { + // We use `forall` because once there's a failed block fetch, `fetchHostLocalBlock` will put + // a `FailureFetchResult` immediately to the `results`. So there's no reason to fetch the + // remaining blocks. + val allFetchSucceeded = bmIdToBlocks.forall { case (bmId, blockInfos) => + blockInfos.forall { case (blockId, _, mapIndex) => + fetchHostLocalBlock(blockId, mapIndex, localDirsByExecId(bmId.executorId), bmId) + } + } + if (allFetchSucceeded) { + logDebug(s"Got host-local blocks from ${bmIdToBlocks.keys.mkString(", ")} " + + s"(${if (cached) "with" else "without"} cached executors' dir) " + + s"in ${Utils.getUsedTimeNs(startTimeNs)}") + } + } + + private[this] def initialize(): Unit = { + // Add a task completion callback (called in both success case and failure case) to cleanup. + context.addTaskCompletionListener(onCompleteCallback) + + // Partition blocks by the different fetch modes: local, host-local and remote blocks. + val remoteRequests = partitionBlocksByFetchMode() + // Add the remote requests into our queue in a random order + fetchRequests ++= Utils.randomize(remoteRequests) + assert ((0 == reqsInFlight) == (0 == bytesInFlight), + "expected reqsInFlight = 0 but found reqsInFlight = " + reqsInFlight + + ", expected bytesInFlight = 0 but found bytesInFlight = " + bytesInFlight) + + // Send out initial requests for blocks, up to our maxBytesInFlight + fetchUpToMaxBytes() + + val numFetches = remoteRequests.size - fetchRequests.size + logInfo(s"Started $numFetches remote fetches in ${Utils.getUsedTimeNs(startTimeNs)}") + + // Get Local Blocks + fetchLocalBlocks() + logDebug(s"Got local blocks in ${Utils.getUsedTimeNs(startTimeNs)}") + + if (hostLocalBlocks.nonEmpty) { + blockManager.hostLocalDirManager.foreach(fetchHostLocalBlocks) + } + } + + override def hasNext: Boolean = numBlocksProcessed < numBlocksToFetch + + /** + * Fetches the next (BlockId, InputStream). If a task fails, the ManagedBuffers + * underlying each InputStream will be freed by the cleanup() method registered with the + * TaskCompletionListener. However, callers should close() these InputStreams + * as soon as they are no longer needed, in order to release memory as early as possible. + * + * Throws a FetchFailedException if the next block could not be fetched. + */ + override def next(): (BlockId, InputStream) = { + if (!hasNext) { + throw new NoSuchElementException() + } + + numBlocksProcessed += 1 + + var result: FetchResult = null + var input: InputStream = null + var streamCompressedOrEncrypted: Boolean = false + // Take the next fetched result and try to decompress it to detect data corruption, + // then fetch it one more time if it's corrupt, throw FailureFetchResult if the second fetch + // is also corrupt, so the previous stage could be retried. + // For local shuffle block, throw FailureFetchResult for the first IOException. + while (result == null) { + val startFetchWait = System.nanoTime() + result = results.take() + val fetchWaitTime = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startFetchWait) + shuffleMetrics.incFetchWaitTime(fetchWaitTime) + + result match { + case r @ SuccessFetchResult(blockId, mapIndex, address, size, buf, isNetworkReqDone) => + if (address != blockManager.blockManagerId) { + if (hostLocalBlocks.contains(blockId -> mapIndex)) { + shuffleMetrics.incLocalBlocksFetched(1) + shuffleMetrics.incLocalBytesRead(buf.size) + } else { + numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1 + shuffleMetrics.incRemoteBytesRead(buf.size) + if (buf.isInstanceOf[FileSegmentManagedBuffer]) { + shuffleMetrics.incRemoteBytesReadToDisk(buf.size) + } + shuffleMetrics.incRemoteBlocksFetched(1) + bytesInFlight -= size + } + } + if (isNetworkReqDone) { + reqsInFlight -= 1 + logDebug("Number of requests in flight " + reqsInFlight) + } + + if (buf.size == 0) { + // We will never legitimately receive a zero-size block. All blocks with zero records + // have zero size and all zero-size blocks have no records (and hence should never + // have been requested in the first place). This statement relies on behaviors of the + // shuffle writers, which are guaranteed by the following test cases: + // + // - BypassMergeSortShuffleWriterSuite: "write with some empty partitions" + // - UnsafeShuffleWriterSuite: "writeEmptyIterator" + // - DiskBlockObjectWriterSuite: "commit() and close() without ever opening or writing" + // + // There is not an explicit test for SortShuffleWriter but the underlying APIs that + // uses are shared by the UnsafeShuffleWriter (both writers use DiskBlockObjectWriter + // which returns a zero-size from commitAndGet() in case no records were written + // since the last call. + val msg = s"Received a zero-size buffer for block $blockId from $address " + + s"(expectedApproxSize = $size, isNetworkReqDone=$isNetworkReqDone)" + throwFetchFailedException(blockId, mapIndex, address, new IOException(msg)) + } + + val in = try { + buf.createInputStream() + } catch { + // The exception could only be throwed by local shuffle block + case e: IOException => + assert(buf.isInstanceOf[FileSegmentManagedBuffer]) + e match { + case ce: ClosedByInterruptException => + logError("Failed to create input stream from local block, " + + ce.getMessage) + case e: IOException => logError("Failed to create input stream from local block", e) + } + buf.release() + throwFetchFailedException(blockId, mapIndex, address, e) + } + try { + input = streamWrapper(blockId, in) + // If the stream is compressed or wrapped, then we optionally decompress/unwrap the + // first maxBytesInFlight/3 bytes into memory, to check for corruption in that portion + // of the data. But even if 'detectCorruptUseExtraMemory' configuration is off, or if + // the corruption is later, we'll still detect the corruption later in the stream. + streamCompressedOrEncrypted = !input.eq(in) + if (streamCompressedOrEncrypted && detectCorruptUseExtraMemory) { + // TODO: manage the memory used here, and spill it into disk in case of OOM. + input = Utils.copyStreamUpTo(input, maxBytesInFlight / 3) + } + } catch { + case e: IOException => + buf.release() + if (buf.isInstanceOf[FileSegmentManagedBuffer] + || corruptedBlocks.contains(blockId)) { + throwFetchFailedException(blockId, mapIndex, address, e) + } else { + logWarning(s"got an corrupted block $blockId from $address, fetch again", e) + corruptedBlocks += blockId + fetchRequests += FetchRequest( + address, Array(FetchBlockInfo(blockId, size, mapIndex))) + result = null + } + } finally { + // TODO: release the buf here to free memory earlier + if (input == null) { + // Close the underlying stream if there was an issue in wrapping the stream using + // streamWrapper + in.close() + } + } + + case FailureFetchResult(blockId, mapIndex, address, e) => + throwFetchFailedException(blockId, mapIndex, address, e) + } + + // Send fetch requests up to maxBytesInFlight + fetchUpToMaxBytes() + } + + val currentResult = result.asInstanceOf[SuccessFetchResult] + (currentResult.blockId, + new RapidsBufferReleasingInputStream( + input, + this, + currentResult, + currentResult.blockId, + currentResult.mapIndex, + currentResult.address, + detectCorrupt && streamCompressedOrEncrypted)) + } + + def toCompletionIterator: Iterator[(BlockId, InputStream)] = { + CompletionIterator[(BlockId, InputStream), this.type](this, + onCompleteCallback.onComplete(context)) + } + + private def fetchUpToMaxBytes(): Unit = { + // Send fetch requests up to maxBytesInFlight. If you cannot fetch from a remote host + // immediately, defer the request until the next time it can be processed. + + // Process any outstanding deferred fetch requests if possible. + if (deferredFetchRequests.nonEmpty) { + for ((remoteAddress, defReqQueue) <- deferredFetchRequests) { + while (isRemoteBlockFetchable(defReqQueue) && + !isRemoteAddressMaxedOut(remoteAddress, defReqQueue.front)) { + val request = defReqQueue.dequeue() + logDebug(s"Processing deferred fetch request for $remoteAddress with " + + s"${request.blocks.length} blocks") + send(remoteAddress, request) + if (defReqQueue.isEmpty) { + deferredFetchRequests -= remoteAddress + } + } + } + } + + // Process any regular fetch requests if possible. + while (isRemoteBlockFetchable(fetchRequests)) { + val request = fetchRequests.dequeue() + val remoteAddress = request.address + if (isRemoteAddressMaxedOut(remoteAddress, request)) { + logDebug(s"Deferring fetch request for $remoteAddress with ${request.blocks.size} blocks") + val defReqQueue = deferredFetchRequests.getOrElse(remoteAddress, new Queue[FetchRequest]()) + defReqQueue.enqueue(request) + deferredFetchRequests(remoteAddress) = defReqQueue + } else { + send(remoteAddress, request) + } + } + + def send(remoteAddress: BlockManagerId, request: FetchRequest): Unit = { + sendRequest(request) + numBlocksInFlightPerAddress(remoteAddress) = + numBlocksInFlightPerAddress.getOrElse(remoteAddress, 0) + request.blocks.size + } + + def isRemoteBlockFetchable(fetchReqQueue: Queue[FetchRequest]): Boolean = { + fetchReqQueue.nonEmpty && + (bytesInFlight == 0 || + (reqsInFlight + 1 <= maxReqsInFlight && + bytesInFlight + fetchReqQueue.front.size <= maxBytesInFlight)) + } + + // Checks if sending a new fetch request will exceed the max no. of blocks being fetched from a + // given remote address. + def isRemoteAddressMaxedOut(remoteAddress: BlockManagerId, request: FetchRequest): Boolean = { + numBlocksInFlightPerAddress.getOrElse(remoteAddress, 0) + request.blocks.size > + maxBlocksInFlightPerAddress + } + } + + private[storage] def throwFetchFailedException( + blockId: BlockId, + mapIndex: Int, + address: BlockManagerId, + e: Throwable) = { + blockId match { + case ShuffleBlockId(shufId, mapId, reduceId) => + throw new FetchFailedException(address, shufId, mapId, mapIndex, reduceId, e) + case ShuffleBlockBatchId(shuffleId, mapId, startReduceId, _) => + throw new FetchFailedException(address, shuffleId, mapId, mapIndex, startReduceId, e) + case _ => + throw new SparkException( + "Failed to get block " + blockId + ", which is not a shuffle block", e) + } + } +} + +/** + * Helper class that ensures a ManagedBuffer is released upon InputStream.close() and + * also detects stream corruption if streamCompressedOrEncrypted is true + */ +private class RapidsBufferReleasingInputStream( + // This is visible for testing + private[storage] val delegate: InputStream, + private val iterator: RapidsShuffleBlockFetcherIterator, + private val currentResult: SuccessFetchResult, + private val blockId: BlockId, + private val mapIndex: Int, + private val address: BlockManagerId, + private val detectCorruption: Boolean) + extends InputStream { + private[this] var closed = false + + override def read(): Int = { + try { + delegate.read() + } catch { + case e: IOException if detectCorruption => + IOUtils.closeQuietly(this) + iterator.throwFetchFailedException(blockId, mapIndex, address, e) + } + } + + override def close(): Unit = { + if (!closed) { + delegate.close() + currentResult.buf.release() + closed = true + } + } + + override def available(): Int = delegate.available() + + override def mark(readlimit: Int): Unit = delegate.mark(readlimit) + + override def skip(n: Long): Long = { + try { + delegate.skip(n) + } catch { + case e: IOException if detectCorruption => + IOUtils.closeQuietly(this) + iterator.throwFetchFailedException(blockId, mapIndex, address, e) + } + } + + override def markSupported(): Boolean = delegate.markSupported() + + override def read(b: Array[Byte]): Int = { + try { + delegate.read(b) + } catch { + case e: IOException if detectCorruption => + IOUtils.closeQuietly(this) + iterator.throwFetchFailedException(blockId, mapIndex, address, e) + } + } + + override def read(b: Array[Byte], off: Int, len: Int): Int = { + try { + delegate.read(b, off, len) + } catch { + case e: IOException if detectCorruption => + IOUtils.closeQuietly(this) + iterator.throwFetchFailedException(blockId, mapIndex, address, e) + } + } + + override def reset(): Unit = delegate.reset() +} + +/** + * A listener to be called at the completion of the ShuffleBlockFetcherIterator + * @param data the ShuffleBlockFetcherIterator to process + */ +private class RapidsShuffleFetchCompletionListener(var data: RapidsShuffleBlockFetcherIterator) + extends TaskCompletionListener { + + override def onTaskCompletion(context: TaskContext): Unit = { + if (data != null) { + data.cleanup() + // Null out the referent here to make sure we don't keep a reference to this + // ShuffleBlockFetcherIterator, after we're done reading from it, to let it be + // collected during GC. Otherwise we can hold metadata on block locations(blocksByAddress) + data = null + } + } + + // Just an alias for onTaskCompletion to avoid confusing + def onComplete(context: TaskContext): Unit = this.onTaskCompletion(context) +} + +object RapidsShuffleBlockFetcherIterator { + /** + * This function is used to merged blocks when doBatchFetch is true. Blocks which have the + * same `mapId` can be merged into one block batch. The block batch is specified by a range + * of reduceId, which implies the continuous shuffle blocks that we can fetch in a batch. + * For example, input blocks like (shuffle_0_0_0, shuffle_0_0_1, shuffle_0_1_0) can be + * merged into (shuffle_0_0_0_2, shuffle_0_1_0_1), and input blocks like (shuffle_0_0_0_2, + * shuffle_0_0_2, shuffle_0_0_3) can be merged into (shuffle_0_0_0_4). + * + * @param blocks blocks to be merged if possible. May contains already merged blocks. + * @param doBatchFetch whether to merge blocks. + * @return the input blocks if doBatchFetch=false, or the merged blocks if doBatchFetch=true. + */ + def mergeContinuousShuffleBlockIdsIfNeeded( + blocks: Seq[FetchBlockInfo], + doBatchFetch: Boolean): Seq[FetchBlockInfo] = { + val result = if (doBatchFetch) { + val curBlocks = new ArrayBuffer[FetchBlockInfo] + val mergedBlockInfo = new ArrayBuffer[FetchBlockInfo] + + def mergeFetchBlockInfo(toBeMerged: ArrayBuffer[FetchBlockInfo]): FetchBlockInfo = { + val startBlockId = toBeMerged.head.blockId.asInstanceOf[ShuffleBlockId] + + // The last merged block may comes from the input, and we can merge more blocks + // into it, if the map id is the same. + def shouldMergeIntoPreviousBatchBlockId = + mergedBlockInfo.last.blockId.asInstanceOf[ShuffleBlockBatchId].mapId == startBlockId.mapId + + val (startReduceId, size) = + if (mergedBlockInfo.nonEmpty && shouldMergeIntoPreviousBatchBlockId) { + // Remove the previous batch block id as we will add a new one to replace it. + val removed = mergedBlockInfo.remove(mergedBlockInfo.length - 1) + (removed.blockId.asInstanceOf[ShuffleBlockBatchId].startReduceId, + removed.size + toBeMerged.map(_.size).sum) + } else { + (startBlockId.reduceId, toBeMerged.map(_.size).sum) + } + + FetchBlockInfo( + ShuffleBlockBatchId( + startBlockId.shuffleId, + startBlockId.mapId, + startReduceId, + toBeMerged.last.blockId.asInstanceOf[ShuffleBlockId].reduceId + 1), + size, + toBeMerged.head.mapIndex) + } + + val iter = blocks.iterator + while (iter.hasNext) { + val info = iter.next() + // It's possible that the input block id is already a batch ID. For example, we merge some + // blocks, and then make fetch requests with the merged blocks according to "max blocks per + // request". The last fetch request may be too small, and we give up and put the remaining + // merged blocks back to the input list. + if (info.blockId.isInstanceOf[ShuffleBlockBatchId]) { + mergedBlockInfo += info + } else { + if (curBlocks.isEmpty) { + curBlocks += info + } else { + val curBlockId = info.blockId.asInstanceOf[ShuffleBlockId] + val currentMapId = curBlocks.head.blockId.asInstanceOf[ShuffleBlockId].mapId + if (curBlockId.mapId != currentMapId) { + mergedBlockInfo += mergeFetchBlockInfo(curBlocks) + curBlocks.clear() + } + curBlocks += info + } + } + } + if (curBlocks.nonEmpty) { + mergedBlockInfo += mergeFetchBlockInfo(curBlocks) + } + mergedBlockInfo + } else { + blocks + } + result.toSeq + } + + /** + * The block information to fetch used in FetchRequest. + * @param blockId block id + * @param size estimated size of the block. Note that this is NOT the exact bytes. + * Size of remote block is used to calculate bytesInFlight. + * @param mapIndex the mapIndex for this block, which indicate the index in the map stage. + */ + private[storage] case class FetchBlockInfo( + blockId: BlockId, + size: Long, + mapIndex: Int) + + /** + * A request to fetch blocks from a remote BlockManager. + * @param address remote BlockManager to fetch from. + * @param blocks Sequence of the information for blocks to fetch from the same address. + */ + case class FetchRequest(address: BlockManagerId, blocks: Seq[FetchBlockInfo]) { + val size = blocks.map(_.size).sum + } + + /** + * Result of a fetch from a remote block. + */ + private[storage] sealed trait FetchResult { + val blockId: BlockId + val address: BlockManagerId + } + + /** + * Result of a fetch from a remote block successfully. + * @param blockId block id + * @param mapIndex the mapIndex for this block, which indicate the index in the map stage. + * @param address BlockManager that the block was fetched from. + * @param size estimated size of the block. Note that this is NOT the exact bytes. + * Size of remote block is used to calculate bytesInFlight. + * @param buf `ManagedBuffer` for the content. + * @param isNetworkReqDone Is this the last network request for this host in this fetch request. + */ + private[storage] case class SuccessFetchResult( + blockId: BlockId, + mapIndex: Int, + address: BlockManagerId, + size: Long, + buf: ManagedBuffer, + isNetworkReqDone: Boolean) extends FetchResult { + require(buf != null) + require(size >= 0) + } + + /** + * Result of a fetch from a remote block unsuccessfully. + * @param blockId block id + * @param mapIndex the mapIndex for this block, which indicate the index in the map stage + * @param address BlockManager that the block was attempted to be fetched from + * @param e the failure exception + */ + private[storage] case class FailureFetchResult( + blockId: BlockId, + mapIndex: Int, + address: BlockManagerId, + e: Throwable) + extends FetchResult + + def makeIterator( + context: TaskContext, + blockManager: BlockManager, + sparkEnv: SparkEnv, + blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])], + serializerManager: SerializerManager, + readMetrics: ShuffleReadMetricsReporter, + fetchContinuousBlocksInBatch: Boolean): RapidsShuffleBlockFetcherIterator = { + new RapidsShuffleBlockFetcherIterator( + context, + blockManager.blockStoreClient, + blockManager, + blocksByAddress, + serializerManager.wrapStream, + // Note: we use getSizeAsMb when no suffix is provided for backwards compatibility + sparkEnv.conf.get(config.REDUCER_MAX_SIZE_IN_FLIGHT) * 1024 * 1024, // 48mb default per task + sparkEnv.conf.get(config.REDUCER_MAX_REQS_IN_FLIGHT), //Int.MaxValue by default + sparkEnv.conf.get(config.REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS), + sparkEnv.conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM), + sparkEnv.conf.get(config.SHUFFLE_DETECT_CORRUPT), + sparkEnv.conf.get(config.SHUFFLE_DETECT_CORRUPT_MEMORY), + readMetrics, + fetchContinuousBlocksInBatch) + } +} \ No newline at end of file diff --git a/sql-plugin/src/main/312-nondb/scala/org/apache/spark/sql/rapids/shims/spark312/RapidsShuffleInternalManager.scala b/sql-plugin/src/main/312-nondb/scala/org/apache/spark/sql/rapids/shims/spark312/RapidsShuffleInternalManager.scala index 2e201f146ec..59f45d754a9 100644 --- a/sql-plugin/src/main/312-nondb/scala/org/apache/spark/sql/rapids/shims/spark312/RapidsShuffleInternalManager.scala +++ b/sql-plugin/src/main/312-nondb/scala/org/apache/spark/sql/rapids/shims/spark312/RapidsShuffleInternalManager.scala @@ -16,11 +16,9 @@ package org.apache.spark.sql.rapids.shims.spark312 -import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.SparkConf import org.apache.spark.shuffle._ -import org.apache.spark.shuffle.sort.BypassMergeSortShuffleHandle import org.apache.spark.sql.rapids.{ProxyRapidsShuffleInternalManagerBase, RapidsShuffleInternalManagerBase} -import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter /** * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. @@ -28,23 +26,7 @@ import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter * `ShuffleManager` and `SortShuffleManager` classes. */ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) - extends RapidsShuffleInternalManagerBase(conf, isDriver) { - - override def makeBypassMergeSortShuffleWriter[K, V]( - handle: BypassMergeSortShuffleHandle[K, V], - mapId: Long, - context: TaskContext, - metricsReporter: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { - new RapidsShuffleThreadedWriter[K, V]( - blockManager, - handle, - mapId, - conf, - metricsReporter, - execComponents.get) - } -} - + extends RapidsShuffleInternalManagerBase(conf, isDriver) class ProxyRapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) extends ProxyRapidsShuffleInternalManagerBase(conf, isDriver) diff --git a/sql-plugin/src/main/312db/scala/org/apache/spark/sql/rapids/shims/spark312db/RapidsShuffleInternalManager.scala b/sql-plugin/src/main/312db/scala/org/apache/spark/sql/rapids/shims/spark312db/RapidsShuffleInternalManager.scala index cf607410882..b63b142b801 100644 --- a/sql-plugin/src/main/312db/scala/org/apache/spark/sql/rapids/shims/spark312db/RapidsShuffleInternalManager.scala +++ b/sql-plugin/src/main/312db/scala/org/apache/spark/sql/rapids/shims/spark312db/RapidsShuffleInternalManager.scala @@ -16,11 +16,9 @@ package org.apache.spark.sql.rapids.shims.spark312db -import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.SparkConf import org.apache.spark.shuffle._ -import org.apache.spark.shuffle.sort.BypassMergeSortShuffleHandle import org.apache.spark.sql.rapids.{ProxyRapidsShuffleInternalManagerBase, RapidsShuffleInternalManagerBase} -import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter /** * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. @@ -28,21 +26,7 @@ import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter * `ShuffleManager` and `SortShuffleManager` classes. */ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) - extends RapidsShuffleInternalManagerBase(conf, isDriver) { - override def makeBypassMergeSortShuffleWriter[K, V]( - handle: BypassMergeSortShuffleHandle[K, V], - mapId: Long, - context: TaskContext, - metricsReporter: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { - new RapidsShuffleThreadedWriter[K, V]( - blockManager, - handle, - mapId, - conf, - metricsReporter, - execComponents.get) - } -} + extends RapidsShuffleInternalManagerBase(conf, isDriver) class ProxyRapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) extends ProxyRapidsShuffleInternalManagerBase(conf, isDriver) diff --git a/sql-plugin/src/main/313/scala/org/apache/spark/sql/rapids/shims/spark313/RapidsShuffleInternalManager.scala b/sql-plugin/src/main/313/scala/org/apache/spark/sql/rapids/shims/spark313/RapidsShuffleInternalManager.scala index 1c78b626010..22c1fe7940a 100644 --- a/sql-plugin/src/main/313/scala/org/apache/spark/sql/rapids/shims/spark313/RapidsShuffleInternalManager.scala +++ b/sql-plugin/src/main/313/scala/org/apache/spark/sql/rapids/shims/spark313/RapidsShuffleInternalManager.scala @@ -16,11 +16,9 @@ package org.apache.spark.sql.rapids.shims.spark313 -import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.SparkConf import org.apache.spark.shuffle._ -import org.apache.spark.shuffle.sort.BypassMergeSortShuffleHandle import org.apache.spark.sql.rapids.{ProxyRapidsShuffleInternalManagerBase, RapidsShuffleInternalManagerBase} -import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter /** * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. @@ -28,22 +26,7 @@ import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter * `ShuffleManager` and `SortShuffleManager` classes. */ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) - extends RapidsShuffleInternalManagerBase(conf, isDriver) { - - override def makeBypassMergeSortShuffleWriter[K, V]( - handle: BypassMergeSortShuffleHandle[K, V], - mapId: Long, - context: TaskContext, - metricsReporter: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { - new RapidsShuffleThreadedWriter[K, V]( - blockManager, - handle, - mapId, - conf, - metricsReporter, - execComponents.get) - } -} + extends RapidsShuffleInternalManagerBase(conf, isDriver) class ProxyRapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) extends ProxyRapidsShuffleInternalManagerBase(conf, isDriver) diff --git a/sql-plugin/src/main/314/scala/org/apache/spark/sql/rapids/shims/spark314/RapidsShuffleInternalManager.scala b/sql-plugin/src/main/314/scala/org/apache/spark/sql/rapids/shims/spark314/RapidsShuffleInternalManager.scala index 761dca47eb1..9c8ca8c60fb 100644 --- a/sql-plugin/src/main/314/scala/org/apache/spark/sql/rapids/shims/spark314/RapidsShuffleInternalManager.scala +++ b/sql-plugin/src/main/314/scala/org/apache/spark/sql/rapids/shims/spark314/RapidsShuffleInternalManager.scala @@ -16,34 +16,16 @@ package org.apache.spark.sql.rapids.shims.spark314 -import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.SparkConf import org.apache.spark.shuffle._ -import org.apache.spark.shuffle.sort.BypassMergeSortShuffleHandle import org.apache.spark.sql.rapids.{ProxyRapidsShuffleInternalManagerBase, RapidsShuffleInternalManagerBase} -import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter - /** * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. * @note This is an internal class to obtain access to the private * `ShuffleManager` and `SortShuffleManager` classes. */ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) - extends RapidsShuffleInternalManagerBase(conf, isDriver) { - override def makeBypassMergeSortShuffleWriter[K, V]( - handle: BypassMergeSortShuffleHandle[K, V], - mapId: Long, - context: TaskContext, - metricsReporter: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { - new RapidsShuffleThreadedWriter[K, V]( - blockManager, - handle, - mapId, - conf, - metricsReporter, - execComponents.get) - } -} - + extends RapidsShuffleInternalManagerBase(conf, isDriver) class ProxyRapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) extends ProxyRapidsShuffleInternalManagerBase(conf, isDriver) diff --git a/sql-plugin/src/main/320+/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala b/sql-plugin/src/main/320+/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala index 034ed1392a8..121d9b6f04c 100644 --- a/sql-plugin/src/main/320+/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala +++ b/sql-plugin/src/main/320+/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala @@ -19,8 +19,7 @@ package org.apache.spark.sql.rapids.shims import org.apache.spark.SparkConf import org.apache.spark.shuffle.ShuffleWriteMetricsReporter import org.apache.spark.shuffle.api.{ShuffleExecutorComponents, ShuffleMapOutputWriter} -import org.apache.spark.shuffle.sort.BypassMergeSortShuffleHandle -import org.apache.spark.sql.rapids.RapidsShuffleThreadedWriterBase +import org.apache.spark.sql.rapids.{RapidsShuffleThreadedWriterBase, ShuffleHandleWithMetrics} import org.apache.spark.storage.{BlockManager, DiskBlockObjectWriter} object RapidsShuffleThreadedWriter { @@ -31,14 +30,21 @@ object RapidsShuffleThreadedWriter { class RapidsShuffleThreadedWriter[K, V]( blockManager: BlockManager, - handle: BypassMergeSortShuffleHandle[K, V], + handle: ShuffleHandleWithMetrics[K, V, V], mapId: Long, sparkConf: SparkConf, writeMetrics: ShuffleWriteMetricsReporter, - shuffleExecutorComponents: ShuffleExecutorComponents) - extends RapidsShuffleThreadedWriterBase[K, V](blockManager, handle, mapId, sparkConf, - writeMetrics, shuffleExecutorComponents) - with org.apache.spark.shuffle.checksum.ShuffleChecksumSupport { + shuffleExecutorComponents: ShuffleExecutorComponents, + numWriterThreads: Int) + extends RapidsShuffleThreadedWriterBase[K, V]( + blockManager, + handle, + mapId, + sparkConf, + writeMetrics, + shuffleExecutorComponents, + numWriterThreads) + with org.apache.spark.shuffle.checksum.ShuffleChecksumSupport { // Spark 3.2.0+ computes checksums per map partition as it writes the // temporary files to disk. They are stored in a Checksum array. diff --git a/sql-plugin/src/main/320+/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala b/sql-plugin/src/main/320+/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala new file mode 100644 index 00000000000..e1b1c3b90ea --- /dev/null +++ b/sql-plugin/src/main/320+/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala @@ -0,0 +1,329 @@ +/* + * 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.storage + +import java.util.concurrent.TimeUnit + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.util.{Failure, Success} + +import org.roaringbitmap.RoaringBitmap + +import org.apache.spark.MapOutputTracker +import org.apache.spark.MapOutputTracker.SHUFFLE_PUSH_MAP_ID +import org.apache.spark.internal.Logging +import org.apache.spark.network.shuffle.{BlockStoreClient, MergedBlockMeta, MergedBlocksMetaListener} +import org.apache.spark.storage.BlockManagerId.SHUFFLE_MERGER_IDENTIFIER +import org.apache.spark.storage.RapidsShuffleBlockFetcherIterator._ + +/** + * Helper class for [[ShuffleBlockFetcherIterator]] that encapsulates all the push-based + * functionality to fetch push-merged block meta and shuffle chunks. + * A push-merged block contains multiple shuffle chunks where each shuffle chunk contains multiple + * shuffle blocks that belong to the common reduce partition and were merged by the + * external shuffle service to that chunk. + */ +private class RapidsPushBasedFetchHelper( + private val iterator: RapidsShuffleBlockFetcherIterator, + private val shuffleClient: BlockStoreClient, + private val blockManager: BlockManager, + private val mapOutputTracker: MapOutputTracker) extends Logging { + + private[this] val startTimeNs = System.nanoTime() + + private[storage] val localShuffleMergerBlockMgrId = BlockManagerId( + SHUFFLE_MERGER_IDENTIFIER, blockManager.blockManagerId.host, + blockManager.blockManagerId.port, blockManager.blockManagerId.topologyInfo) + + /** + * A map for storing shuffle chunk bitmap. + */ + private[this] val chunksMetaMap = new mutable.HashMap[ShuffleBlockChunkId, RoaringBitmap]() + + /** + * Returns true if the address is for a push-merged block. + */ + def isPushMergedShuffleBlockAddress(address: BlockManagerId): Boolean = { + SHUFFLE_MERGER_IDENTIFIER == address.executorId + } + + /** + * Returns true if the address is of a remote push-merged block. false otherwise. + */ + def isRemotePushMergedBlockAddress(address: BlockManagerId): Boolean = { + isPushMergedShuffleBlockAddress(address) && address.host != blockManager.blockManagerId.host + } + + /** + * Returns true if the address is of a push-merged-local block. false otherwise. + */ + def isLocalPushMergedBlockAddress(address: BlockManagerId): Boolean = { + isPushMergedShuffleBlockAddress(address) && address.host == blockManager.blockManagerId.host + } + + /** + * This is executed by the task thread when the `iterator.next()` is invoked and the iterator + * processes a response of type [[ShuffleBlockFetcherIterator.SuccessFetchResult]]. + * + * @param blockId shuffle chunk id. + */ + def removeChunk(blockId: ShuffleBlockChunkId): Unit = { + chunksMetaMap.remove(blockId) + } + + /** + * This is executed by the task thread when the `iterator.next()` is invoked and the iterator + * processes a response of type [[ShuffleBlockFetcherIterator.PushMergedLocalMetaFetchResult]]. + * + * @param blockId shuffle chunk id. + */ + def addChunk(blockId: ShuffleBlockChunkId, chunkMeta: RoaringBitmap): Unit = { + chunksMetaMap(blockId) = chunkMeta + } + + /** + * This is executed by the task thread when the `iterator.next()` is invoked and the iterator + * processes a response of type [[ShuffleBlockFetcherIterator.PushMergedRemoteMetaFetchResult]]. + * + * @param shuffleId shuffle id. + * @param reduceId reduce id. + * @param blockSize size of the push-merged block. + * @param bitmaps chunk bitmaps, where each bitmap contains all the mapIds that were merged + * to that chunk. + * @return shuffle chunks to fetch. + */ + def createChunkBlockInfosFromMetaResponse( + shuffleId: Int, + shuffleMergeId: Int, + reduceId: Int, + blockSize: Long, + bitmaps: Array[RoaringBitmap]): ArrayBuffer[(BlockId, Long, Int)] = { + val approxChunkSize = blockSize / bitmaps.length + val blocksToFetch = new ArrayBuffer[(BlockId, Long, Int)]() + for (i <- bitmaps.indices) { + val blockChunkId = ShuffleBlockChunkId(shuffleId, shuffleMergeId, reduceId, i) + chunksMetaMap.put(blockChunkId, bitmaps(i)) + logDebug(s"adding block chunk $blockChunkId of size $approxChunkSize") + blocksToFetch += ((blockChunkId, approxChunkSize, SHUFFLE_PUSH_MAP_ID)) + } + blocksToFetch + } + + /** + * This is executed by the task thread when the iterator is initialized and only if it has + * push-merged blocks for which it needs to fetch the metadata. + * + * @param req [[ShuffleBlockFetcherIterator.FetchRequest]] that only contains requests to fetch + * metadata of push-merged blocks. + */ + def sendFetchMergedStatusRequest(req: FetchRequest): Unit = { + val sizeMap = req.blocks.map { + case FetchBlockInfo(blockId, size, _) => + val shuffleBlockId = blockId.asInstanceOf[ShuffleMergedBlockId] + ((shuffleBlockId.shuffleId, shuffleBlockId.reduceId), size) + }.toMap + val address = req.address + val mergedBlocksMetaListener = new MergedBlocksMetaListener { + override def onSuccess(shuffleId: Int, shuffleMergeId: Int, reduceId: Int, + meta: MergedBlockMeta): Unit = { + logDebug(s"Received the meta of push-merged block for ($shuffleId, $shuffleMergeId," + + s" $reduceId) from ${req.address.host}:${req.address.port}") + try { + iterator.addToResultsQueue(PushMergedRemoteMetaFetchResult(shuffleId, shuffleMergeId, + reduceId, sizeMap((shuffleId, reduceId)), meta.readChunkBitmaps(), address)) + } catch { + case exception: Exception => + logError(s"Failed to parse the meta of push-merged block for ($shuffleId, " + + s"$shuffleMergeId, $reduceId) from" + + s" ${req.address.host}:${req.address.port}", exception) + iterator.addToResultsQueue( + PushMergedRemoteMetaFailedFetchResult(shuffleId, shuffleMergeId, reduceId, + address)) + } + } + + override def onFailure(shuffleId: Int, shuffleMergeId: Int, reduceId: Int, + exception: Throwable): Unit = { + logError(s"Failed to get the meta of push-merged block for ($shuffleId, $reduceId) " + + s"from ${req.address.host}:${req.address.port}", exception) + iterator.addToResultsQueue( + PushMergedRemoteMetaFailedFetchResult(shuffleId, shuffleMergeId, reduceId, address)) + } + } + req.blocks.foreach { block => + val shuffleBlockId = block.blockId.asInstanceOf[ShuffleMergedBlockId] + shuffleClient.getMergedBlockMeta(address.host, address.port, shuffleBlockId.shuffleId, + shuffleBlockId.shuffleMergeId, shuffleBlockId.reduceId, mergedBlocksMetaListener) + } + } + + /** + * This is executed by the task thread when the iterator is initialized. It fetches all the + * outstanding push-merged local blocks. + * @param pushMergedLocalBlocks set of identified merged local blocks and their sizes. + */ + def fetchAllPushMergedLocalBlocks( + pushMergedLocalBlocks: mutable.LinkedHashSet[BlockId]): Unit = { + if (pushMergedLocalBlocks.nonEmpty) { + blockManager.hostLocalDirManager.foreach(fetchPushMergedLocalBlocks(_, pushMergedLocalBlocks)) + } + } + + /** + * Fetch the push-merged blocks dirs if they are not in the cache and eventually fetch push-merged + * local blocks. + */ + private def fetchPushMergedLocalBlocks( + hostLocalDirManager: HostLocalDirManager, + pushMergedLocalBlocks: mutable.LinkedHashSet[BlockId]): Unit = { + val cachedPushedMergedDirs = hostLocalDirManager.getCachedHostLocalDirsFor( + SHUFFLE_MERGER_IDENTIFIER) + if (cachedPushedMergedDirs.isDefined) { + logDebug(s"Fetch the push-merged-local blocks with cached merged dirs: " + + s"${cachedPushedMergedDirs.get.mkString(", ")}") + pushMergedLocalBlocks.foreach { blockId => + fetchPushMergedLocalBlock(blockId, cachedPushedMergedDirs.get, + localShuffleMergerBlockMgrId) + } + } else { + // Push-based shuffle is only enabled when the external shuffle service is enabled. If the + // external shuffle service is not enabled, then there will not be any push-merged blocks + // for the iterator to fetch. + logDebug(s"Asynchronous fetch the push-merged-local blocks without cached merged " + + s"dirs from the external shuffle service") + hostLocalDirManager.getHostLocalDirs(blockManager.blockManagerId.host, + blockManager.externalShuffleServicePort, Array(SHUFFLE_MERGER_IDENTIFIER)) { + case Success(dirs) => + logDebug(s"Fetched merged dirs in " + + s"${TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs)} ms") + pushMergedLocalBlocks.foreach { + blockId => + logDebug(s"Successfully fetched local dirs: " + + s"${dirs.get(SHUFFLE_MERGER_IDENTIFIER).mkString(", ")}") + fetchPushMergedLocalBlock(blockId, dirs(SHUFFLE_MERGER_IDENTIFIER), + localShuffleMergerBlockMgrId) + } + case Failure(throwable) => + // If we see an exception with getting the local dirs for push-merged-local blocks, + // we fallback to fetch the original blocks. We do not report block fetch failure. + logWarning(s"Error while fetching the merged dirs for push-merged-local " + + s"blocks: ${pushMergedLocalBlocks.mkString(", ")}. Fetch the original blocks instead", + throwable) + pushMergedLocalBlocks.foreach { + blockId => + iterator.addToResultsQueue(FallbackOnPushMergedFailureResult( + blockId, localShuffleMergerBlockMgrId, 0, isNetworkReqDone = false)) + } + } + } + } + + /** + * Fetch a single push-merged-local block generated. This can also be executed by the task thread + * as well as the netty thread. + * @param blockId ShuffleBlockId to be fetched + * @param localDirs Local directories where the push-merged shuffle files are stored + * @param blockManagerId BlockManagerId + */ + private[this] def fetchPushMergedLocalBlock( + blockId: BlockId, + localDirs: Array[String], + blockManagerId: BlockManagerId): Unit = { + try { + val shuffleBlockId = blockId.asInstanceOf[ShuffleMergedBlockId] + val chunksMeta = blockManager.getLocalMergedBlockMeta(shuffleBlockId, localDirs) + iterator.addToResultsQueue(PushMergedLocalMetaFetchResult( + shuffleBlockId.shuffleId, shuffleBlockId.shuffleMergeId, + shuffleBlockId.reduceId, chunksMeta.readChunkBitmaps(), localDirs)) + } catch { + case e: Exception => + // If we see an exception with reading a push-merged-local meta, we fallback to + // fetch the original blocks. We do not report block fetch failure + // and will continue with the remaining local block read. + logWarning(s"Error occurred while fetching push-merged-local meta, " + + s"prepare to fetch the original blocks", e) + iterator.addToResultsQueue( + FallbackOnPushMergedFailureResult(blockId, blockManagerId, 0, isNetworkReqDone = false)) + } + } + + /** + * This is executed by the task thread when the `iterator.next()` is invoked and the iterator + * processes a response of type: + * 1) [[ShuffleBlockFetcherIterator.SuccessFetchResult]] + * 2) [[ShuffleBlockFetcherIterator.FallbackOnPushMergedFailureResult]] + * 3) [[ShuffleBlockFetcherIterator.PushMergedRemoteMetaFailedFetchResult]] + * + * This initiates fetching fallback blocks for a push-merged block or a shuffle chunk that + * failed to fetch. + * It makes a call to the map output tracker to get the list of original blocks for the + * given push-merged block/shuffle chunk, split them into remote and local blocks, and process + * them accordingly. + * It also updates the numberOfBlocksToFetch in the iterator as it processes failed response and + * finds more push-merged requests to remote and again updates it with additional requests for + * original blocks. + * The fallback happens when: + * 1. There is an exception while creating shuffle chunks from push-merged-local shuffle block. + * See fetchLocalBlock. + * 2. There is a failure when fetching remote shuffle chunks. + * 3. There is a failure when processing SuccessFetchResult which is for a shuffle chunk + * (local or remote). + */ + def initiateFallbackFetchForPushMergedBlock( + blockId: BlockId, + address: BlockManagerId): Unit = { + assert(blockId.isInstanceOf[ShuffleMergedBlockId] || blockId.isInstanceOf[ShuffleBlockChunkId]) + logWarning(s"Falling back to fetch the original blocks for push-merged block $blockId") + // Increase the blocks processed since we will process another block in the next iteration of + // the while loop in ShuffleBlockFetcherIterator.next(). + val fallbackBlocksByAddr: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = + blockId match { + case shuffleBlockId: ShuffleMergedBlockId => + iterator.decreaseNumBlocksToFetch(1) + mapOutputTracker.getMapSizesForMergeResult( + shuffleBlockId.shuffleId, shuffleBlockId.reduceId) + case _ => + val shuffleChunkId = blockId.asInstanceOf[ShuffleBlockChunkId] + val chunkBitmap: RoaringBitmap = chunksMetaMap.remove(shuffleChunkId).get + var blocksProcessed = 1 + // When there is a failure to fetch a remote shuffle chunk, then we try to + // fallback not only for that particular remote shuffle chunk but also for all the + // pending chunks that belong to the same host. The reason for doing so is that it + // is very likely that the subsequent requests for shuffle chunks from this host will + // fail as well. Since, push-based shuffle is best effort and we try not to increase the + // delay of the fetches, we immediately fallback for all the pending shuffle chunks in the + // fetchRequests queue. + if (isRemotePushMergedBlockAddress(address)) { + // Fallback for all the pending fetch requests + val pendingShuffleChunks = iterator.removePendingChunks(shuffleChunkId, address) + pendingShuffleChunks.foreach { pendingBlockId => + logInfo(s"Falling back immediately for shuffle chunk $pendingBlockId") + val bitmapOfPendingChunk: RoaringBitmap = chunksMetaMap.remove(pendingBlockId).get + chunkBitmap.or(bitmapOfPendingChunk) + } + // These blocks were added to numBlocksToFetch so we increment numBlocksProcessed + blocksProcessed += pendingShuffleChunks.size + } + iterator.decreaseNumBlocksToFetch(blocksProcessed) + mapOutputTracker.getMapSizesForMergeResult( + shuffleChunkId.shuffleId, shuffleChunkId.reduceId, chunkBitmap) + } + iterator.fallbackFetch(fallbackBlocksByAddr) + } +} diff --git a/sql-plugin/src/main/320+/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala b/sql-plugin/src/main/320+/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala new file mode 100644 index 00000000000..45272d0f9d9 --- /dev/null +++ b/sql-plugin/src/main/320+/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala @@ -0,0 +1,1624 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import java.io.{InputStream, IOException} +import java.nio.channels.ClosedByInterruptException +import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} +import java.util.concurrent.atomic.AtomicBoolean +import java.util.zip.CheckedInputStream +import javax.annotation.concurrent.GuardedBy + +import scala.collection.mutable +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} +import scala.util.{Failure, Success} + +import org.roaringbitmap.RoaringBitmap + +import org.apache.spark.{MapOutputTracker, SparkEnv, TaskContext} +import org.apache.spark.MapOutputTracker.SHUFFLE_PUSH_MAP_ID +import org.apache.spark.SparkException +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.shuffle._ +import org.apache.spark.network.shuffle.checksum.{Cause, ShuffleChecksumHelper} +import org.apache.spark.network.util.{NettyUtils, TransportConf} +import org.apache.spark.serializer.SerializerManager +import org.apache.spark.shuffle.{FetchFailedException, ShuffleReadMetricsReporter} +import org.apache.spark.util.{CompletionIterator, TaskCompletionListener, Utils} + +/** + * Taken mostly verbatim from `ShuffleBlockFetcherIterator` except for + * a change to the ownership of `currentResult` (which contains the netty buffer). + * Instead of this iterator owning the result and clearing it on `next`, the + * `BufferReleasingInputStream` is in charge of that. This allows for multiple threads + * to consume different `BufferReleasingInputStream`s produced from this single iterator. + * + * Reverts usage of `SparkCoreErrors` to just use the original exception here, + * as this was done in Spark 3.3.0 and is rather minor. + */ + +/** + * An iterator that fetches multiple blocks. For local blocks, it fetches from the local block + * manager. For remote blocks, it fetches them using the provided BlockTransferService. + * + * This creates an iterator of (BlockID, InputStream) tuples so the caller can handle blocks + * in a pipelined fashion as they are received. + * + * The implementation throttles the remote fetches so they don't exceed maxBytesInFlight to avoid + * using too much memory. + * + * @param context [[TaskContext]], used for metrics update + * @param shuffleClient [[BlockStoreClient]] for fetching remote blocks + * @param blockManager [[BlockManager]] for reading local blocks + * @param blocksByAddress list of blocks to fetch grouped by the [[BlockManagerId]]. + * For each block we also require two info: 1. the size (in bytes as a long + * field) in order to throttle the memory usage; 2. the mapIndex for this + * block, which indicate the index in the map stage. + * Note that zero-sized blocks are already excluded, which happened in + * [[org.apache.spark.MapOutputTracker.convertMapStatuses]]. + * @param mapOutputTracker [[MapOutputTracker]] for falling back to fetching the original blocks if + * we fail to fetch shuffle chunks when push based shuffle is enabled. + * @param streamWrapper A function to wrap the returned input stream. + * @param maxBytesInFlight max size (in bytes) of remote blocks to fetch at any given point. + * @param maxReqsInFlight max number of remote requests to fetch blocks at any given point. + * @param maxBlocksInFlightPerAddress max number of shuffle blocks being fetched at any given point + * for a given remote host:port. + * @param maxReqSizeShuffleToMem max size (in bytes) of a request that can be shuffled to memory. + * @param maxAttemptsOnNettyOOM The max number of a block could retry due to Netty OOM before + * throwing the fetch failure. + * @param detectCorrupt whether to detect any corruption in fetched blocks. + * @param checksumEnabled whether the shuffle checksum is enabled. When enabled, Spark will try to + * diagnose the cause of the block corruption. + * @param checksumAlgorithm the checksum algorithm that is used when calculating the checksum value + * for the block data. + * @param shuffleMetrics used to report shuffle metrics. + * @param doBatchFetch fetch continuous shuffle blocks from same executor in batch if the server + * side supports. + */ +private[spark] +final class RapidsShuffleBlockFetcherIterator( + context: TaskContext, + shuffleClient: BlockStoreClient, + blockManager: BlockManager, + mapOutputTracker: MapOutputTracker, + blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])], + streamWrapper: (BlockId, InputStream) => InputStream, + maxBytesInFlight: Long, + maxReqsInFlight: Int, + maxBlocksInFlightPerAddress: Int, + val maxReqSizeShuffleToMem: Long, + maxAttemptsOnNettyOOM: Int, + detectCorrupt: Boolean, + detectCorruptUseExtraMemory: Boolean, + checksumEnabled: Boolean, + checksumAlgorithm: String, + shuffleMetrics: ShuffleReadMetricsReporter, + doBatchFetch: Boolean) + extends Iterator[(BlockId, InputStream)] with DownloadFileManager with Logging { + + import RapidsShuffleBlockFetcherIterator._ + + // Make remote requests at most maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + private val targetRemoteRequestSize = math.max(maxBytesInFlight / 5, 1L) + + /** + * Total number of blocks to fetch. + */ + private[this] var numBlocksToFetch = 0 + + /** + * The number of blocks processed by the caller. The iterator is exhausted when + * [[numBlocksProcessed]] == [[numBlocksToFetch]]. + */ + private[this] var numBlocksProcessed = 0 + + private[this] val startTimeNs = System.nanoTime() + + /** Host local blocks to fetch, excluding zero-sized blocks. */ + private[this] val hostLocalBlocks = scala.collection.mutable.LinkedHashSet[(BlockId, Int)]() + + /** + * A queue to hold our results. This turns the asynchronous model provided by + * [[org.apache.spark.network.BlockTransferService]] into a synchronous model (iterator). + */ + private[this] val results = new LinkedBlockingQueue[FetchResult] + + /** + * Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that + * the number of bytes in flight is limited to maxBytesInFlight. + */ + private[this] val fetchRequests = new Queue[FetchRequest] + + /** + * Queue of fetch requests which could not be issued the first time they were dequeued. These + * requests are tried again when the fetch constraints are satisfied. + */ + private[this] val deferredFetchRequests = new HashMap[BlockManagerId, Queue[FetchRequest]]() + + /** Current bytes in flight from our requests */ + private[this] var bytesInFlight = 0L + + /** Current number of requests in flight */ + private[this] var reqsInFlight = 0 + + /** Current number of blocks in flight per host:port */ + private[this] val numBlocksInFlightPerAddress = new HashMap[BlockManagerId, Int]() + + /** + * Count the retry times for the blocks due to Netty OOM. The block will stop retry if + * retry times has exceeded the [[maxAttemptsOnNettyOOM]]. + */ + private[this] val blockOOMRetryCounts = new HashMap[String, Int] + + /** + * The blocks that can't be decompressed successfully, it is used to guarantee that we retry + * at most once for those corrupted blocks. + */ + private[this] val corruptedBlocks = mutable.HashSet[BlockId]() + + /** + * Whether the iterator is still active. If isZombie is true, the callback interface will no + * longer place fetched blocks into [[results]]. + */ + @GuardedBy("this") + private[this] var isZombie = false + + /** + * A set to store the files used for shuffling remote huge blocks. Files in this set will be + * deleted when cleanup. This is a layer of defensiveness against disk file leaks. + */ + @GuardedBy("this") + private[this] val shuffleFilesSet = mutable.HashSet[DownloadFile]() + + private[this] val onCompleteCallback = new RapidsShuffleFetchCompletionListener(this) + + private[this] val pushBasedFetchHelper = new RapidsPushBasedFetchHelper( + this, shuffleClient, blockManager, mapOutputTracker) + + initialize() + + override def createTempFile(transportConf: TransportConf): DownloadFile = { + // we never need to do any encryption or decryption here, regardless of configs, because that + // is handled at another layer in the code. When encryption is enabled, shuffle data is written + // to disk encrypted in the first place, and sent over the network still encrypted. + new SimpleDownloadFile( + blockManager.diskBlockManager.createTempLocalBlock()._2, transportConf) + } + + override def registerTempFileToClean(file: DownloadFile): Boolean = synchronized { + if (isZombie) { + false + } else { + shuffleFilesSet += file + true + } + } + + /** + * Mark the iterator as zombie, and release all buffers that haven't been deserialized yet. + */ + private[storage] def cleanup(): Unit = { + synchronized { + isZombie = true + } + // Release buffers in the results queue + val iter = results.iterator() + while (iter.hasNext) { + val result = iter.next() + result match { + case SuccessFetchResult(blockId, mapIndex, address, _, buf, _) => + if (address != blockManager.blockManagerId) { + if (hostLocalBlocks.contains(blockId -> mapIndex)) { + shuffleMetrics.incLocalBlocksFetched(1) + shuffleMetrics.incLocalBytesRead(buf.size) + } else { + shuffleMetrics.incRemoteBytesRead(buf.size) + if (buf.isInstanceOf[FileSegmentManagedBuffer]) { + shuffleMetrics.incRemoteBytesReadToDisk(buf.size) + } + shuffleMetrics.incRemoteBlocksFetched(1) + } + } + buf.release() + case _ => + } + } + shuffleFilesSet.foreach { file => + if (!file.delete()) { + logWarning("Failed to cleanup shuffle fetch temp file " + file.path()) + } + } + } + + private[this] def sendRequest(req: FetchRequest): Unit = { + logDebug("Sending request for %d blocks (%s) from %s".format( + req.blocks.size, Utils.bytesToString(req.size), req.address.hostPort)) + bytesInFlight += req.size + reqsInFlight += 1 + + // so we can look up the block info of each blockID + val infoMap = req.blocks.map { + case FetchBlockInfo(blockId, size, mapIndex) => (blockId.toString, (size, mapIndex)) + }.toMap + val remainingBlocks = new HashSet[String]() ++= infoMap.keys + val deferredBlocks = new ArrayBuffer[String]() + val blockIds = req.blocks.map(_.blockId.toString) + val address = req.address + + @inline def enqueueDeferredFetchRequestIfNecessary(): Unit = { + if (remainingBlocks.isEmpty && deferredBlocks.nonEmpty) { + val blocks = deferredBlocks.map { blockId => + val (size, mapIndex) = infoMap(blockId) + FetchBlockInfo(BlockId(blockId), size, mapIndex) + } + results.put(DeferFetchRequestResult(FetchRequest(address, blocks.toSeq))) + deferredBlocks.clear() + } + } + + val blockFetchingListener = new BlockFetchingListener { + override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = { + // Only add the buffer to results queue if the iterator is not zombie, + // i.e. cleanup() has not been called yet. + RapidsShuffleBlockFetcherIterator.this.synchronized { + if (!isZombie) { + // Increment the ref count because we need to pass this to a different thread. + // This needs to be released after use. + buf.retain() + remainingBlocks -= blockId + blockOOMRetryCounts.remove(blockId) + results.put(new SuccessFetchResult(BlockId(blockId), infoMap(blockId)._2, + address, infoMap(blockId)._1, buf, remainingBlocks.isEmpty)) + logDebug("remainingBlocks: " + remainingBlocks) + enqueueDeferredFetchRequestIfNecessary() + } + } + logTrace(s"Got remote block $blockId after ${Utils.getUsedTimeNs(startTimeNs)}") + } + + override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = { + RapidsShuffleBlockFetcherIterator.this.synchronized { + logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e) + e match { + // SPARK-27991: Catch the Netty OOM and set the flag `isNettyOOMOnShuffle` (shared among + // tasks) to true as early as possible. The pending fetch requests won't be sent + // afterwards until the flag is set to false on: + // 1) the Netty free memory >= maxReqSizeShuffleToMem + // - we'll check this whenever there's a fetch request succeeds. + // 2) the number of in-flight requests becomes 0 + // - we'll check this in `fetchUpToMaxBytes` whenever it's invoked. + // Although Netty memory is shared across multiple modules, e.g., shuffle, rpc, the flag + // only takes effect for the shuffle due to the implementation simplicity concern. + // And we'll buffer the consecutive block failures caused by the OOM error until there's + // no remaining blocks in the current request. Then, we'll package these blocks into + // a same fetch request for the retry later. In this way, instead of creating the fetch + // request per block, it would help reduce the concurrent connections and data loads + // pressure at remote server. + // Note that catching OOM and do something based on it is only a workaround for + // handling the Netty OOM issue, which is not the best way towards memory management. + // We can get rid of it when we find a way to manage Netty's memory precisely. + // NOTE: using OutOfMemoryError instead of OutOfDirectMemoryError due to the later being + // shaded in databricks. + case _: OutOfMemoryError + if blockOOMRetryCounts.getOrElseUpdate(blockId, 0) < maxAttemptsOnNettyOOM => + if (!isZombie) { + val failureTimes = blockOOMRetryCounts(blockId) + blockOOMRetryCounts(blockId) += 1 + if (isNettyOOMOnShuffle.compareAndSet(false, true)) { + // The fetcher can fail remaining blocks in batch for the same error. So we only + // log the warning once to avoid flooding the logs. + logInfo(s"Block $blockId has failed $failureTimes times " + + s"due to Netty OOM, will retry") + } + remainingBlocks -= blockId + deferredBlocks += blockId + enqueueDeferredFetchRequestIfNecessary() + } + + case _ => + val block = BlockId(blockId) + if (block.isShuffleChunk) { + remainingBlocks -= blockId + results.put(FallbackOnPushMergedFailureResult( + block, address, infoMap(blockId)._1, remainingBlocks.isEmpty)) + } else { + results.put(FailureFetchResult(block, infoMap(blockId)._2, address, e)) + } + } + } + } + } + + // Fetch remote shuffle blocks to disk when the request is too large. Since the shuffle data is + // already encrypted and compressed over the wire(w.r.t. the related configs), we can just fetch + // the data and write it to file directly. + if (req.size > maxReqSizeShuffleToMem) { + shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray, + blockFetchingListener, this) + } else { + shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray, + blockFetchingListener, null) + } + } + + /** + * This is called from initialize and also from the fallback which is triggered from + * [[PushBasedFetchHelper]]. + */ + private[this] def partitionBlocksByFetchMode( + blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])], + localBlocks: mutable.LinkedHashSet[(BlockId, Int)], + hostLocalBlocksByExecutor: mutable.LinkedHashMap[BlockManagerId, Seq[(BlockId, Long, Int)]], + pushMergedLocalBlocks: mutable.LinkedHashSet[BlockId]): ArrayBuffer[FetchRequest] = { + logDebug(s"maxBytesInFlight: $maxBytesInFlight, targetRemoteRequestSize: " + + s"$targetRemoteRequestSize, maxBlocksInFlightPerAddress: $maxBlocksInFlightPerAddress") + + // Partition to local, host-local, push-merged-local, remote (includes push-merged-remote) + // blocks.Remote blocks are further split into FetchRequests of size at most maxBytesInFlight + // in order to limit the amount of data in flight + val collectedRemoteRequests = new ArrayBuffer[FetchRequest] + var localBlockBytes = 0L + var hostLocalBlockBytes = 0L + var numHostLocalBlocks = 0 + var pushMergedLocalBlockBytes = 0L + val prevNumBlocksToFetch = numBlocksToFetch + + val fallback = FallbackStorage.FALLBACK_BLOCK_MANAGER_ID.executorId + val localExecIds = Set(blockManager.blockManagerId.executorId, fallback) + for ((address, blockInfos) <- blocksByAddress) { + checkBlockSizes(blockInfos) + if (pushBasedFetchHelper.isPushMergedShuffleBlockAddress(address)) { + // These are push-merged blocks or shuffle chunks of these blocks. + if (address.host == blockManager.blockManagerId.host) { + numBlocksToFetch += blockInfos.size + pushMergedLocalBlocks ++= blockInfos.map(_._1) + pushMergedLocalBlockBytes += blockInfos.map(_._2).sum + } else { + collectFetchRequests(address, blockInfos, collectedRemoteRequests) + } + } else if (localExecIds.contains(address.executorId)) { + val mergedBlockInfos = mergeContinuousShuffleBlockIdsIfNeeded( + blockInfos.map(info => FetchBlockInfo(info._1, info._2, info._3)), doBatchFetch) + numBlocksToFetch += mergedBlockInfos.size + localBlocks ++= mergedBlockInfos.map(info => (info.blockId, info.mapIndex)) + localBlockBytes += mergedBlockInfos.map(_.size).sum + } else if (blockManager.hostLocalDirManager.isDefined && + address.host == blockManager.blockManagerId.host) { + val mergedBlockInfos = mergeContinuousShuffleBlockIdsIfNeeded( + blockInfos.map(info => FetchBlockInfo(info._1, info._2, info._3)), doBatchFetch) + numBlocksToFetch += mergedBlockInfos.size + val blocksForAddress = + mergedBlockInfos.map(info => (info.blockId, info.size, info.mapIndex)) + hostLocalBlocksByExecutor += address -> blocksForAddress + numHostLocalBlocks += blocksForAddress.size + hostLocalBlockBytes += mergedBlockInfos.map(_.size).sum + } else { + val (_, timeCost) = Utils.timeTakenMs[Unit] { + collectFetchRequests(address, blockInfos, collectedRemoteRequests) + } + logDebug(s"Collected remote fetch requests for $address in $timeCost ms") + } + } + val (remoteBlockBytes, numRemoteBlocks) = + collectedRemoteRequests.foldLeft((0L, 0))((x, y) => (x._1 + y.size, x._2 + y.blocks.size)) + val totalBytes = localBlockBytes + remoteBlockBytes + hostLocalBlockBytes + + pushMergedLocalBlockBytes + val blocksToFetchCurrentIteration = numBlocksToFetch - prevNumBlocksToFetch + assert(blocksToFetchCurrentIteration == localBlocks.size + + numHostLocalBlocks + numRemoteBlocks + pushMergedLocalBlocks.size, + s"The number of non-empty blocks $blocksToFetchCurrentIteration doesn't equal to the sum " + + s"of the number of local blocks ${localBlocks.size} + " + + s"the number of host-local blocks ${numHostLocalBlocks} " + + s"the number of push-merged-local blocks ${pushMergedLocalBlocks.size} " + + s"+ the number of remote blocks ${numRemoteBlocks} ") + logInfo(s"Getting $blocksToFetchCurrentIteration " + + s"(${Utils.bytesToString(totalBytes)}) non-empty blocks including " + + s"${localBlocks.size} (${Utils.bytesToString(localBlockBytes)}) local and " + + s"${numHostLocalBlocks} (${Utils.bytesToString(hostLocalBlockBytes)}) " + + s"host-local and ${pushMergedLocalBlocks.size} " + + s"(${Utils.bytesToString(pushMergedLocalBlockBytes)}) " + + s"push-merged-local and $numRemoteBlocks (${Utils.bytesToString(remoteBlockBytes)}) " + + s"remote blocks") + this.hostLocalBlocks ++= hostLocalBlocksByExecutor.values + .flatMap { infos => infos.map(info => (info._1, info._3)) } + collectedRemoteRequests + } + + private def createFetchRequest( + blocks: Seq[FetchBlockInfo], + address: BlockManagerId, + forMergedMetas: Boolean): FetchRequest = { + logDebug(s"Creating fetch request of ${blocks.map(_.size).sum} at $address " + + s"with ${blocks.size} blocks") + FetchRequest(address, blocks, forMergedMetas) + } + + private def createFetchRequests( + curBlocks: Seq[FetchBlockInfo], + address: BlockManagerId, + isLast: Boolean, + collectedRemoteRequests: ArrayBuffer[FetchRequest], + enableBatchFetch: Boolean, + forMergedMetas: Boolean = false): ArrayBuffer[FetchBlockInfo] = { + val mergedBlocks = mergeContinuousShuffleBlockIdsIfNeeded(curBlocks, enableBatchFetch) + numBlocksToFetch += mergedBlocks.size + val retBlocks = new ArrayBuffer[FetchBlockInfo] + if (mergedBlocks.length <= maxBlocksInFlightPerAddress) { + collectedRemoteRequests += createFetchRequest(mergedBlocks, address, forMergedMetas) + } else { + mergedBlocks.grouped(maxBlocksInFlightPerAddress).foreach { blocks => + if (blocks.length == maxBlocksInFlightPerAddress || isLast) { + collectedRemoteRequests += createFetchRequest(blocks, address, forMergedMetas) + } else { + // The last group does not exceed `maxBlocksInFlightPerAddress`. Put it back + // to `curBlocks`. + retBlocks ++= blocks + numBlocksToFetch -= blocks.size + } + } + } + retBlocks + } + + private def collectFetchRequests( + address: BlockManagerId, + blockInfos: Seq[(BlockId, Long, Int)], + collectedRemoteRequests: ArrayBuffer[FetchRequest]): Unit = { + val iterator = blockInfos.iterator + var curRequestSize = 0L + var curBlocks = new ArrayBuffer[FetchBlockInfo]() + + while (iterator.hasNext) { + val (blockId, size, mapIndex) = iterator.next() + curBlocks += FetchBlockInfo(blockId, size, mapIndex) + curRequestSize += size + blockId match { + // Either all blocks are push-merged blocks, shuffle chunks, or original blocks. + // Based on these types, we decide to do batch fetch and create FetchRequests with + // forMergedMetas set. + case ShuffleBlockChunkId(_, _, _, _) => + if (curRequestSize >= targetRemoteRequestSize || + curBlocks.size >= maxBlocksInFlightPerAddress) { + curBlocks = createFetchRequests(curBlocks.toSeq, address, isLast = false, + collectedRemoteRequests, enableBatchFetch = false) + curRequestSize = curBlocks.map(_.size).sum + } + case ShuffleMergedBlockId(_, _, _) => + if (curBlocks.size >= maxBlocksInFlightPerAddress) { + curBlocks = createFetchRequests(curBlocks.toSeq, address, isLast = false, + collectedRemoteRequests, enableBatchFetch = false, forMergedMetas = true) + } + case _ => + // For batch fetch, the actual block in flight should count for merged block. + val mayExceedsMaxBlocks = !doBatchFetch && curBlocks.size >= maxBlocksInFlightPerAddress + if (curRequestSize >= targetRemoteRequestSize || mayExceedsMaxBlocks) { + curBlocks = createFetchRequests(curBlocks.toSeq, address, isLast = false, + collectedRemoteRequests, doBatchFetch) + curRequestSize = curBlocks.map(_.size).sum + } + } + } + // Add in the final request + if (curBlocks.nonEmpty) { + val (enableBatchFetch, forMergedMetas) = { + curBlocks.head.blockId match { + case ShuffleBlockChunkId(_, _, _, _) => (false, false) + case ShuffleMergedBlockId(_, _, _) => (false, true) + case _ => (doBatchFetch, false) + } + } + createFetchRequests(curBlocks.toSeq, address, isLast = true, collectedRemoteRequests, + enableBatchFetch = enableBatchFetch, forMergedMetas = forMergedMetas) + } + } + + private def assertPositiveBlockSize(blockId: BlockId, blockSize: Long): Unit = { + if (blockSize < 0) { + throw BlockException(blockId, "Negative block size " + size) + } else if (blockSize == 0) { + throw BlockException(blockId, "Zero-sized blocks should be excluded.") + } + } + + private def checkBlockSizes(blockInfos: Seq[(BlockId, Long, Int)]): Unit = { + blockInfos.foreach { case (blockId, size, _) => assertPositiveBlockSize(blockId, size) } + } + + /** + * Fetch the local blocks while we are fetching remote blocks. This is ok because + * `ManagedBuffer`'s memory is allocated lazily when we create the input stream, so all we + * track in-memory are the ManagedBuffer references themselves. + */ + private[this] def fetchLocalBlocks( + localBlocks: mutable.LinkedHashSet[(BlockId, Int)]): Unit = { + logDebug(s"Start fetching local blocks: ${localBlocks.mkString(", ")}") + val iter = localBlocks.iterator + while (iter.hasNext) { + val (blockId, mapIndex) = iter.next() + try { + val buf = blockManager.getLocalBlockData(blockId) + shuffleMetrics.incLocalBlocksFetched(1) + shuffleMetrics.incLocalBytesRead(buf.size) + buf.retain() + results.put(new SuccessFetchResult(blockId, mapIndex, blockManager.blockManagerId, + buf.size(), buf, false)) + } catch { + // If we see an exception, stop immediately. + case e: Exception => + e match { + // ClosedByInterruptException is an excepted exception when kill task, + // don't log the exception stack trace to avoid confusing users. + // See: SPARK-28340 + case ce: ClosedByInterruptException => + logError("Error occurred while fetching local blocks, " + ce.getMessage) + case ex: Exception => logError("Error occurred while fetching local blocks", ex) + } + results.put(new FailureFetchResult(blockId, mapIndex, blockManager.blockManagerId, e)) + return + } + } + } + + private[this] def fetchHostLocalBlock( + blockId: BlockId, + mapIndex: Int, + localDirs: Array[String], + blockManagerId: BlockManagerId): Boolean = { + try { + val buf = blockManager.getHostLocalShuffleData(blockId, localDirs) + buf.retain() + results.put(SuccessFetchResult(blockId, mapIndex, blockManagerId, buf.size(), buf, + isNetworkReqDone = false)) + true + } catch { + case e: Exception => + // If we see an exception, stop immediately. + logError(s"Error occurred while fetching local blocks", e) + results.put(FailureFetchResult(blockId, mapIndex, blockManagerId, e)) + false + } + } + + /** + * Fetch the host-local blocks while we are fetching remote blocks. This is ok because + * `ManagedBuffer`'s memory is allocated lazily when we create the input stream, so all we + * track in-memory are the ManagedBuffer references themselves. + */ + private[this] def fetchHostLocalBlocks( + hostLocalDirManager: HostLocalDirManager, + hostLocalBlocksByExecutor: mutable.LinkedHashMap[BlockManagerId, Seq[(BlockId, Long, Int)]]): + Unit = { + val cachedDirsByExec = hostLocalDirManager.getCachedHostLocalDirs + val (hostLocalBlocksWithCachedDirs, hostLocalBlocksWithMissingDirs) = { + val (hasCache, noCache) = hostLocalBlocksByExecutor.partition { case (hostLocalBmId, _) => + cachedDirsByExec.contains(hostLocalBmId.executorId) + } + (hasCache.toMap, noCache.toMap) + } + + if (hostLocalBlocksWithMissingDirs.nonEmpty) { + logDebug(s"Asynchronous fetching host-local blocks without cached executors' dir: " + + s"${hostLocalBlocksWithMissingDirs.mkString(", ")}") + + // If the external shuffle service is enabled, we'll fetch the local directories for + // multiple executors from the external shuffle service, which located at the same host + // with the executors, in once. Otherwise, we'll fetch the local directories from those + // executors directly one by one. The fetch requests won't be too much since one host is + // almost impossible to have many executors at the same time practically. + val dirFetchRequests = if (blockManager.externalShuffleServiceEnabled) { + val host = blockManager.blockManagerId.host + val port = blockManager.externalShuffleServicePort + Seq((host, port, hostLocalBlocksWithMissingDirs.keys.toArray)) + } else { + hostLocalBlocksWithMissingDirs.keys.map(bmId => (bmId.host, bmId.port, Array(bmId))).toSeq + } + + dirFetchRequests.foreach { case (host, port, bmIds) => + hostLocalDirManager.getHostLocalDirs(host, port, bmIds.map(_.executorId)) { + case Success(dirsByExecId) => + fetchMultipleHostLocalBlocks( + hostLocalBlocksWithMissingDirs.filterKeys(bmIds.contains).toMap, + dirsByExecId, + cached = false) + + case Failure(throwable) => + logError("Error occurred while fetching host local blocks", throwable) + val bmId = bmIds.head + val blockInfoSeq = hostLocalBlocksWithMissingDirs(bmId) + val (blockId, _, mapIndex) = blockInfoSeq.head + results.put(FailureFetchResult(blockId, mapIndex, bmId, throwable)) + } + } + } + + if (hostLocalBlocksWithCachedDirs.nonEmpty) { + logDebug(s"Synchronous fetching host-local blocks with cached executors' dir: " + + s"${hostLocalBlocksWithCachedDirs.mkString(", ")}") + fetchMultipleHostLocalBlocks(hostLocalBlocksWithCachedDirs, cachedDirsByExec, cached = true) + } + } + + private def fetchMultipleHostLocalBlocks( + bmIdToBlocks: Map[BlockManagerId, Seq[(BlockId, Long, Int)]], + localDirsByExecId: Map[String, Array[String]], + cached: Boolean): Unit = { + // We use `forall` because once there's a failed block fetch, `fetchHostLocalBlock` will put + // a `FailureFetchResult` immediately to the `results`. So there's no reason to fetch the + // remaining blocks. + val allFetchSucceeded = bmIdToBlocks.forall { case (bmId, blockInfos) => + blockInfos.forall { case (blockId, _, mapIndex) => + fetchHostLocalBlock(blockId, mapIndex, localDirsByExecId(bmId.executorId), bmId) + } + } + if (allFetchSucceeded) { + logDebug(s"Got host-local blocks from ${bmIdToBlocks.keys.mkString(", ")} " + + s"(${if (cached) "with" else "without"} cached executors' dir) " + + s"in ${Utils.getUsedTimeNs(startTimeNs)}") + } + } + + private[this] def initialize(): Unit = { + // Add a task completion callback (called in both success case and failure case) to cleanup. + context.addTaskCompletionListener(onCompleteCallback) + // Local blocks to fetch, excluding zero-sized blocks. + val localBlocks = mutable.LinkedHashSet[(BlockId, Int)]() + val hostLocalBlocksByExecutor = + mutable.LinkedHashMap[BlockManagerId, Seq[(BlockId, Long, Int)]]() + val pushMergedLocalBlocks = mutable.LinkedHashSet[BlockId]() + // Partition blocks by the different fetch modes: local, host-local, push-merged-local and + // remote blocks. + val remoteRequests = partitionBlocksByFetchMode( + blocksByAddress, localBlocks, hostLocalBlocksByExecutor, pushMergedLocalBlocks) + // Add the remote requests into our queue in a random order + fetchRequests ++= Utils.randomize(remoteRequests) + assert ((0 == reqsInFlight) == (0 == bytesInFlight), + "expected reqsInFlight = 0 but found reqsInFlight = " + reqsInFlight + + ", expected bytesInFlight = 0 but found bytesInFlight = " + bytesInFlight) + + // Send out initial requests for blocks, up to our maxBytesInFlight + fetchUpToMaxBytes() + + val numDeferredRequest = deferredFetchRequests.values.map(_.size).sum + val numFetches = remoteRequests.size - fetchRequests.size - numDeferredRequest + logInfo(s"Started $numFetches remote fetches in ${Utils.getUsedTimeNs(startTimeNs)}" + + (if (numDeferredRequest > 0 ) s", deferred $numDeferredRequest requests" else "")) + + // Get Local Blocks + fetchLocalBlocks(localBlocks) + logDebug(s"Got local blocks in ${Utils.getUsedTimeNs(startTimeNs)}") + // Get host local blocks if any + fetchAllHostLocalBlocks(hostLocalBlocksByExecutor) + pushBasedFetchHelper.fetchAllPushMergedLocalBlocks(pushMergedLocalBlocks) + } + + private def fetchAllHostLocalBlocks( + hostLocalBlocksByExecutor: mutable.LinkedHashMap[BlockManagerId, Seq[(BlockId, Long, Int)]]): + Unit = { + if (hostLocalBlocksByExecutor.nonEmpty) { + blockManager.hostLocalDirManager.foreach(fetchHostLocalBlocks(_, hostLocalBlocksByExecutor)) + } + } + + override def hasNext: Boolean = numBlocksProcessed < numBlocksToFetch + + /** + * Fetches the next (BlockId, InputStream). If a task fails, the ManagedBuffers + * underlying each InputStream will be freed by the cleanup() method registered with the + * TaskCompletionListener. However, callers should close() these InputStreams + * as soon as they are no longer needed, in order to release memory as early as possible. + * + * Throws a FetchFailedException if the next block could not be fetched. + */ + override def next(): (BlockId, InputStream) = { + if (!hasNext) { + // from SparkCoreErrors + throw new NoSuchElementException() + } + + numBlocksProcessed += 1 + + var result: FetchResult = null + var input: InputStream = null + // This's only initialized when shuffle checksum is enabled. + var checkedIn: CheckedInputStream = null + var streamCompressedOrEncrypted: Boolean = false + // Take the next fetched result and try to decompress it to detect data corruption, + // then fetch it one more time if it's corrupt, throw FailureFetchResult if the second fetch + // is also corrupt, so the previous stage could be retried. + // For local shuffle block, throw FailureFetchResult for the first IOException. + while (result == null) { + val startFetchWait = System.nanoTime() + result = results.take() + val fetchWaitTime = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startFetchWait) + shuffleMetrics.incFetchWaitTime(fetchWaitTime) + + result match { + case r @ SuccessFetchResult(blockId, mapIndex, address, size, buf, isNetworkReqDone) => + if (address != blockManager.blockManagerId) { + if (hostLocalBlocks.contains(blockId -> mapIndex) || + pushBasedFetchHelper.isLocalPushMergedBlockAddress(address)) { + // It is a host local block or a local shuffle chunk + shuffleMetrics.incLocalBlocksFetched(1) + shuffleMetrics.incLocalBytesRead(buf.size) + } else { + numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1 + shuffleMetrics.incRemoteBytesRead(buf.size) + if (buf.isInstanceOf[FileSegmentManagedBuffer]) { + shuffleMetrics.incRemoteBytesReadToDisk(buf.size) + } + shuffleMetrics.incRemoteBlocksFetched(1) + bytesInFlight -= size + } + } + if (isNetworkReqDone) { + reqsInFlight -= 1 + resetNettyOOMFlagIfPossible(maxReqSizeShuffleToMem) + logDebug("Number of requests in flight " + reqsInFlight) + } + + if (buf.size == 0) { + // We will never legitimately receive a zero-size block. All blocks with zero records + // have zero size and all zero-size blocks have no records (and hence should never + // have been requested in the first place). This statement relies on behaviors of the + // shuffle writers, which are guaranteed by the following test cases: + // + // - BypassMergeSortShuffleWriterSuite: "write with some empty partitions" + // - UnsafeShuffleWriterSuite: "writeEmptyIterator" + // - DiskBlockObjectWriterSuite: "commit() and close() without ever opening or writing" + // + // There is not an explicit test for SortShuffleWriter but the underlying APIs that + // uses are shared by the UnsafeShuffleWriter (both writers use DiskBlockObjectWriter + // which returns a zero-size from commitAndGet() in case no records were written + // since the last call. + val msg = s"Received a zero-size buffer for block $blockId from $address " + + s"(expectedApproxSize = $size, isNetworkReqDone=$isNetworkReqDone)" + throwFetchFailedException(blockId, mapIndex, address, new IOException(msg)) + } + + val in = try { + val bufIn = buf.createInputStream() + if (checksumEnabled) { + val checksum = ShuffleChecksumHelper.getChecksumByAlgorithm(checksumAlgorithm) + checkedIn = new CheckedInputStream(bufIn, checksum) + checkedIn + } else { + bufIn + } + } catch { + // The exception could only be throwed by local shuffle block + case e: IOException => + assert(buf.isInstanceOf[FileSegmentManagedBuffer]) + e match { + case ce: ClosedByInterruptException => + logError("Failed to create input stream from local block, " + + ce.getMessage) + case e: IOException => logError("Failed to create input stream from local block", e) + } + buf.release() + if (blockId.isShuffleChunk) { + pushBasedFetchHelper.initiateFallbackFetchForPushMergedBlock(blockId, address) + // Set result to null to trigger another iteration of the while loop to get either. + result = null + null + } else { + throwFetchFailedException(blockId, mapIndex, address, e) + } + } + if (in != null) { + try { + input = streamWrapper(blockId, in) + // If the stream is compressed or wrapped, then we optionally decompress/unwrap the + // first maxBytesInFlight/3 bytes into memory, to check for corruption in that portion + // of the data. But even if 'detectCorruptUseExtraMemory' configuration is off, or if + // the corruption is later, we'll still detect the corruption later in the stream. + streamCompressedOrEncrypted = !input.eq(in) + if (streamCompressedOrEncrypted && detectCorruptUseExtraMemory) { + // TODO: manage the memory used here, and spill it into disk in case of OOM. + input = Utils.copyStreamUpTo(input, maxBytesInFlight / 3) + } + } catch { + case e: IOException => + // When shuffle checksum is enabled, for a block that is corrupted twice, + // we'd calculate the checksum of the block by consuming the remaining data + // in the buf. So, we should release the buf later. + if (!(checksumEnabled && corruptedBlocks.contains(blockId))) { + buf.release() + } + + if (blockId.isShuffleChunk) { + // TODO (SPARK-36284): Add shuffle checksum support for push-based shuffle + // Retrying a corrupt block may result again in a corrupt block. For shuffle + // chunks, we opt to fallback on the original shuffle blocks that belong to that + // corrupt shuffle chunk immediately instead of retrying to fetch the corrupt + // chunk. This also makes the code simpler because the chunkMeta corresponding to + // a shuffle chunk is always removed from chunksMetaMap whenever a shuffle chunk + // gets processed. If we try to re-fetch a corrupt shuffle chunk, then it has to + // be added back to the chunksMetaMap. + pushBasedFetchHelper.initiateFallbackFetchForPushMergedBlock(blockId, address) + // Set result to null to trigger another iteration of the while loop. + result = null + } else if (buf.isInstanceOf[FileSegmentManagedBuffer]) { + throwFetchFailedException(blockId, mapIndex, address, e) + } else if (corruptedBlocks.contains(blockId)) { + // It's the second time this block is detected corrupted + if (checksumEnabled) { + // Diagnose the cause of data corruption if shuffle checksum is enabled + val diagnosisResponse = diagnoseCorruption(checkedIn, address, blockId) + buf.release() + logError(diagnosisResponse) + throwFetchFailedException( + blockId, mapIndex, address, e, Some(diagnosisResponse)) + } else { + throwFetchFailedException(blockId, mapIndex, address, e) + } + } else { + // It's the first time this block is detected corrupted + logWarning(s"got an corrupted block $blockId from $address, fetch again", e) + corruptedBlocks += blockId + fetchRequests += FetchRequest( + address, Array(FetchBlockInfo(blockId, size, mapIndex))) + result = null + } + } finally { + if (blockId.isShuffleChunk) { + pushBasedFetchHelper.removeChunk(blockId.asInstanceOf[ShuffleBlockChunkId]) + } + // TODO: release the buf here to free memory earlier + if (input == null) { + // Close the underlying stream if there was an issue in wrapping the stream using + // streamWrapper + in.close() + } + } + } + + case FailureFetchResult(blockId, mapIndex, address, e) => + var errorMsg: String = null + // NOTE: using OutOfMemoryError instead of OutOfDirectMemoryError due to the later being + // shaded in databricks. + if (e.isInstanceOf[OutOfMemoryError]) { + errorMsg = s"Block $blockId fetch failed after $maxAttemptsOnNettyOOM " + + s"retries due to Netty OOM" + logError(errorMsg) + } + throwFetchFailedException(blockId, mapIndex, address, e, Some(errorMsg)) + + case DeferFetchRequestResult(request) => + val address = request.address + numBlocksInFlightPerAddress(address) = + numBlocksInFlightPerAddress(address) - request.blocks.size + bytesInFlight -= request.size + reqsInFlight -= 1 + logDebug("Number of requests in flight " + reqsInFlight) + val defReqQueue = + deferredFetchRequests.getOrElseUpdate(address, new Queue[FetchRequest]()) + defReqQueue.enqueue(request) + result = null + + case FallbackOnPushMergedFailureResult(blockId, address, size, isNetworkReqDone) => + // We get this result in 3 cases: + // 1. Failure to fetch the data of a remote shuffle chunk. In this case, the + // blockId is a ShuffleBlockChunkId. + // 2. Failure to read the push-merged-local meta. In this case, the blockId is + // ShuffleBlockId. + // 3. Failure to get the push-merged-local directories from the external shuffle service. + // In this case, the blockId is ShuffleBlockId. + if (pushBasedFetchHelper.isRemotePushMergedBlockAddress(address)) { + numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1 + bytesInFlight -= size + } + if (isNetworkReqDone) { + reqsInFlight -= 1 + logDebug("Number of requests in flight " + reqsInFlight) + } + pushBasedFetchHelper.initiateFallbackFetchForPushMergedBlock(blockId, address) + // Set result to null to trigger another iteration of the while loop to get either + // a SuccessFetchResult or a FailureFetchResult. + result = null + + case PushMergedLocalMetaFetchResult( + shuffleId, shuffleMergeId, reduceId, bitmaps, localDirs) => + // Fetch push-merged-local shuffle block data as multiple shuffle chunks + val shuffleBlockId = ShuffleMergedBlockId(shuffleId, shuffleMergeId, reduceId) + try { + val bufs: Seq[ManagedBuffer] = blockManager.getLocalMergedBlockData(shuffleBlockId, + localDirs) + // Since the request for local block meta completed successfully, numBlocksToFetch + // is decremented. + numBlocksToFetch -= 1 + // Update total number of blocks to fetch, reflecting the multiple local shuffle + // chunks. + numBlocksToFetch += bufs.size + bufs.zipWithIndex.foreach { case (buf, chunkId) => + buf.retain() + val shuffleChunkId = ShuffleBlockChunkId(shuffleId, shuffleMergeId, reduceId, + chunkId) + pushBasedFetchHelper.addChunk(shuffleChunkId, bitmaps(chunkId)) + results.put(SuccessFetchResult(shuffleChunkId, SHUFFLE_PUSH_MAP_ID, + pushBasedFetchHelper.localShuffleMergerBlockMgrId, buf.size(), buf, + isNetworkReqDone = false)) + } + } catch { + case e: Exception => + // If we see an exception with reading push-merged-local index file, we fallback + // to fetch the original blocks. We do not report block fetch failure + // and will continue with the remaining local block read. + logWarning(s"Error occurred while reading push-merged-local index, " + + s"prepare to fetch the original blocks", e) + pushBasedFetchHelper.initiateFallbackFetchForPushMergedBlock( + shuffleBlockId, pushBasedFetchHelper.localShuffleMergerBlockMgrId) + } + result = null + + case PushMergedRemoteMetaFetchResult( + shuffleId, shuffleMergeId, reduceId, blockSize, bitmaps, address) => + // The original meta request is processed so we decrease numBlocksToFetch and + // numBlocksInFlightPerAddress by 1. We will collect new shuffle chunks request and the + // count of this is added to numBlocksToFetch in collectFetchReqsFromMergedBlocks. + numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1 + numBlocksToFetch -= 1 + val blocksToFetch = pushBasedFetchHelper.createChunkBlockInfosFromMetaResponse( + shuffleId, shuffleMergeId, reduceId, blockSize, bitmaps) + val additionalRemoteReqs = new ArrayBuffer[FetchRequest] + collectFetchRequests(address, blocksToFetch.toSeq, additionalRemoteReqs) + fetchRequests ++= additionalRemoteReqs + // Set result to null to force another iteration. + result = null + + case PushMergedRemoteMetaFailedFetchResult( + shuffleId, shuffleMergeId, reduceId, address) => + // The original meta request failed so we decrease numBlocksInFlightPerAddress by 1. + numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1 + // If we fail to fetch the meta of a push-merged block, we fall back to fetching the + // original blocks. + pushBasedFetchHelper.initiateFallbackFetchForPushMergedBlock( + ShuffleMergedBlockId(shuffleId, shuffleMergeId, reduceId), address) + // Set result to null to force another iteration. + result = null + } + + // Send fetch requests up to maxBytesInFlight + fetchUpToMaxBytes() + } + + val currentResult = result.asInstanceOf[SuccessFetchResult] + (currentResult.blockId, + new RapidsBufferReleasingInputStream( + input, + this, + currentResult, + currentResult.blockId, + currentResult.mapIndex, + currentResult.address, + detectCorrupt && streamCompressedOrEncrypted, + currentResult.isNetworkReqDone, + Option(checkedIn))) + } + + /** + * Get the suspect corruption cause for the corrupted block. It should be only invoked + * when checksum is enabled and corruption was detected at least once. + * + * This will firstly consume the rest of stream of the corrupted block to calculate the + * checksum of the block. Then, it will raise a synchronized RPC call along with the + * checksum to ask the server(where the corrupted block is fetched from) to diagnose the + * cause of corruption and return it. + * + * Any exception raised during the process will result in the [[Cause.UNKNOWN_ISSUE]] of the + * corruption cause since corruption diagnosis is only a best effort. + * + * @param checkedIn the [[CheckedInputStream]] which is used to calculate the checksum. + * @param address the address where the corrupted block is fetched from. + * @param blockId the blockId of the corrupted block. + * @return The corruption diagnosis response for different causes. + */ + private[storage] def diagnoseCorruption( + checkedIn: CheckedInputStream, + address: BlockManagerId, + blockId: BlockId): String = { + logInfo("Start corruption diagnosis.") + blockId match { + case shuffleBlock: ShuffleBlockId => + val startTimeNs = System.nanoTime() + val buffer = new Array[Byte](ShuffleChecksumHelper.CHECKSUM_CALCULATION_BUFFER) + // consume the remaining data to calculate the checksum + var cause: Cause = null + try { + while (checkedIn.read(buffer) != -1) {} + val checksum = checkedIn.getChecksum.getValue + cause = shuffleClient.diagnoseCorruption(address.host, address.port, address.executorId, + shuffleBlock.shuffleId, shuffleBlock.mapId, shuffleBlock.reduceId, checksum, + checksumAlgorithm) + } catch { + case e: Exception => + logWarning("Unable to diagnose the corruption cause of the corrupted block", e) + cause = Cause.UNKNOWN_ISSUE + } + val duration = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs) + val diagnosisResponse = cause match { + case Cause.UNSUPPORTED_CHECKSUM_ALGORITHM => + s"Block $blockId is corrupted but corruption diagnosis failed due to " + + s"unsupported checksum algorithm: $checksumAlgorithm" + + case Cause.CHECKSUM_VERIFY_PASS => + s"Block $blockId is corrupted but checksum verification passed" + + case Cause.UNKNOWN_ISSUE => + s"Block $blockId is corrupted but the cause is unknown" + + case otherCause => + s"Block $blockId is corrupted due to $otherCause" + } + logInfo(s"Finished corruption diagnosis in $duration ms. $diagnosisResponse") + diagnosisResponse + case shuffleBlockChunk: ShuffleBlockChunkId => + // TODO SPARK-36284 Add shuffle checksum support for push-based shuffle + val diagnosisResponse = s"BlockChunk $shuffleBlockChunk is corrupted but corruption " + + s"diagnosis is skipped due to lack of shuffle checksum support for push-based shuffle." + logWarning(diagnosisResponse) + diagnosisResponse + case unexpected: BlockId => + throw new IllegalArgumentException(s"Unexpected type of BlockId, $unexpected") + } + } + + def toCompletionIterator: Iterator[(BlockId, InputStream)] = { + CompletionIterator[(BlockId, InputStream), this.type](this, + onCompleteCallback.onComplete(context)) + } + + private def fetchUpToMaxBytes(): Unit = { + if (isNettyOOMOnShuffle.get()) { + if (reqsInFlight > 0) { + // Return immediately if Netty is still OOMed and there're ongoing fetch requests + return + } else { + resetNettyOOMFlagIfPossible(0) + } + } + + // Send fetch requests up to maxBytesInFlight. If you cannot fetch from a remote host + // immediately, defer the request until the next time it can be processed. + + // Process any outstanding deferred fetch requests if possible. + if (deferredFetchRequests.nonEmpty) { + for ((remoteAddress, defReqQueue) <- deferredFetchRequests) { + while (isRemoteBlockFetchable(defReqQueue) && + !isRemoteAddressMaxedOut(remoteAddress, defReqQueue.front)) { + val request = defReqQueue.dequeue() + logDebug(s"Processing deferred fetch request for $remoteAddress with " + + s"${request.blocks.length} blocks") + send(remoteAddress, request) + if (defReqQueue.isEmpty) { + deferredFetchRequests -= remoteAddress + } + } + } + } + + // Process any regular fetch requests if possible. + while (isRemoteBlockFetchable(fetchRequests)) { + val request = fetchRequests.dequeue() + val remoteAddress = request.address + if (isRemoteAddressMaxedOut(remoteAddress, request)) { + logDebug(s"Deferring fetch request for $remoteAddress with ${request.blocks.size} blocks") + val defReqQueue = deferredFetchRequests.getOrElse(remoteAddress, new Queue[FetchRequest]()) + defReqQueue.enqueue(request) + deferredFetchRequests(remoteAddress) = defReqQueue + } else { + send(remoteAddress, request) + } + } + + def send(remoteAddress: BlockManagerId, request: FetchRequest): Unit = { + if (request.forMergedMetas) { + pushBasedFetchHelper.sendFetchMergedStatusRequest(request) + } else { + sendRequest(request) + } + numBlocksInFlightPerAddress(remoteAddress) = + numBlocksInFlightPerAddress.getOrElse(remoteAddress, 0) + request.blocks.size + } + + def isRemoteBlockFetchable(fetchReqQueue: Queue[FetchRequest]): Boolean = { + fetchReqQueue.nonEmpty && + (bytesInFlight == 0 || + (reqsInFlight + 1 <= maxReqsInFlight && + bytesInFlight + fetchReqQueue.front.size <= maxBytesInFlight)) + } + + // Checks if sending a new fetch request will exceed the max no. of blocks being fetched from a + // given remote address. + def isRemoteAddressMaxedOut(remoteAddress: BlockManagerId, request: FetchRequest): Boolean = { + numBlocksInFlightPerAddress.getOrElse(remoteAddress, 0) + request.blocks.size > + maxBlocksInFlightPerAddress + } + } + + private[storage] def throwFetchFailedException( + blockId: BlockId, + mapIndex: Int, + address: BlockManagerId, + e: Throwable, + message: Option[String] = None) = { + val msg = message.getOrElse(e.getMessage) + blockId match { + case ShuffleBlockId(shufId, mapId, reduceId) => + // from SparkCoreErrors + throw new FetchFailedException( + address, shufId, mapId, mapIndex, reduceId, msg, e) + case ShuffleBlockBatchId(shuffleId, mapId, startReduceId, _) => + // from SparkCoreErrors + throw new FetchFailedException( + address, shuffleId, mapId, mapIndex, startReduceId, msg, e) + case _ => + // from SparkCoreErrors + throw new SparkException(s"Failed to get block $blockId, which is not a shuffle block", e) + } + } + + /** + * All the below methods are used by [[PushBasedFetchHelper]] to communicate with the iterator + */ + private[storage] def addToResultsQueue(result: FetchResult): Unit = { + results.put(result) + } + + private[storage] def decreaseNumBlocksToFetch(blocksFetched: Int): Unit = { + numBlocksToFetch -= blocksFetched + } + + /** + * Currently used by [[PushBasedFetchHelper]] to fetch fallback blocks when there is a fetch + * failure related to a push-merged block or shuffle chunk. + * This is executed by the task thread when the `iterator.next()` is invoked and if that initiates + * fallback. + */ + private[storage] def fallbackFetch( + originalBlocksByAddr: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])]): Unit = { + val originalLocalBlocks = mutable.LinkedHashSet[(BlockId, Int)]() + val originalHostLocalBlocksByExecutor = + mutable.LinkedHashMap[BlockManagerId, Seq[(BlockId, Long, Int)]]() + val originalMergedLocalBlocks = mutable.LinkedHashSet[BlockId]() + val originalRemoteReqs = partitionBlocksByFetchMode(originalBlocksByAddr, + originalLocalBlocks, originalHostLocalBlocksByExecutor, originalMergedLocalBlocks) + // Add the remote requests into our queue in a random order + fetchRequests ++= Utils.randomize(originalRemoteReqs) + logInfo(s"Created ${originalRemoteReqs.size} fallback remote requests for push-merged") + // fetch all the fallback blocks that are local. + fetchLocalBlocks(originalLocalBlocks) + // Merged local blocks should be empty during fallback + assert(originalMergedLocalBlocks.isEmpty, + "There should be zero push-merged blocks during fallback") + // Some of the fallback local blocks could be host local blocks + fetchAllHostLocalBlocks(originalHostLocalBlocksByExecutor) + } + + /** + * Removes all the pending shuffle chunks that are on the same host and have the same reduceId as + * the current chunk that had a fetch failure. + * This is executed by the task thread when the `iterator.next()` is invoked and if that initiates + * fallback. + * + * @return set of all the removed shuffle chunk Ids. + */ + private[storage] def removePendingChunks( + failedBlockId: ShuffleBlockChunkId, + address: BlockManagerId): mutable.HashSet[ShuffleBlockChunkId] = { + val removedChunkIds = new mutable.HashSet[ShuffleBlockChunkId]() + + def sameShuffleReducePartition(block: BlockId): Boolean = { + val chunkId = block.asInstanceOf[ShuffleBlockChunkId] + chunkId.shuffleId == failedBlockId.shuffleId && chunkId.reduceId == failedBlockId.reduceId + } + + def filterRequests(queue: mutable.Queue[FetchRequest]): Unit = { + val fetchRequestsToRemove = new mutable.Queue[FetchRequest]() + fetchRequestsToRemove ++= queue.dequeueAll { req => + val firstBlock = req.blocks.head + firstBlock.blockId.isShuffleChunk && req.address.equals(address) && + sameShuffleReducePartition(firstBlock.blockId) + } + fetchRequestsToRemove.foreach { _ => + removedChunkIds ++= + fetchRequestsToRemove.flatMap(_.blocks.map(_.blockId.asInstanceOf[ShuffleBlockChunkId])) + } + } + + filterRequests(fetchRequests) + deferredFetchRequests.get(address).foreach { defRequests => + filterRequests(defRequests) + if (defRequests.isEmpty) deferredFetchRequests.remove(address) + } + removedChunkIds + } + + def resultCount: Int = results.size() +} + +/** + * Helper class that ensures a ManagedBuffer is released upon InputStream.close() and + * also detects stream corruption if streamCompressedOrEncrypted is true + */ +private class RapidsBufferReleasingInputStream( + // This is visible for testing + private[storage] val delegate: InputStream, + private val iterator: RapidsShuffleBlockFetcherIterator, + private val currentResult: RapidsShuffleBlockFetcherIterator.SuccessFetchResult, + private val blockId: BlockId, + private val mapIndex: Int, + private val address: BlockManagerId, + private val detectCorruption: Boolean, + private val isNetworkReqDone: Boolean, + private val checkedInOpt: Option[CheckedInputStream]) + extends InputStream { + private[this] var closed = false + + override def read(): Int = + tryOrFetchFailedException(delegate.read()) + + override def close(): Unit = { + if (!closed) { + try { + delegate.close() + currentResult.buf.release() + } finally { + // Unset the flag when a remote request finished and free memory is fairly enough. + if (isNetworkReqDone) { + RapidsShuffleBlockFetcherIterator. + resetNettyOOMFlagIfPossible(iterator.maxReqSizeShuffleToMem) + } + closed = true + } + } + } + + override def available(): Int = delegate.available() + + override def mark(readlimit: Int): Unit = delegate.mark(readlimit) + + override def skip(n: Long): Long = + tryOrFetchFailedException(delegate.skip(n)) + + override def markSupported(): Boolean = delegate.markSupported() + + override def read(b: Array[Byte]): Int = + tryOrFetchFailedException(delegate.read(b)) + + override def read(b: Array[Byte], off: Int, len: Int): Int = + tryOrFetchFailedException(delegate.read(b, off, len)) + + override def reset(): Unit = delegate.reset() + + /** + * Execute a block of code that returns a value, close this stream quietly and re-throwing + * IOException as FetchFailedException when detectCorruption is true. This method is only + * used by the `read` and `skip` methods inside `BufferReleasingInputStream` currently. + */ + private def tryOrFetchFailedException[T](block: => T): T = { + try { + block + } catch { + case e: IOException if detectCorruption => + val diagnosisResponse = checkedInOpt.map { checkedIn => + iterator.diagnoseCorruption(checkedIn, address, blockId) + } + try { + close() + } catch { + case _: Throwable => + // like: IOUtils.closeQuietly(this) + // adding like this since it is deprecated + } + // We'd never retry the block whatever the cause is since the block has been + // partially consumed by downstream RDDs. + iterator.throwFetchFailedException(blockId, mapIndex, address, e, diagnosisResponse) + } + } + +} + +/** + * A listener to be called at the completion of the ShuffleBlockFetcherIterator + * @param data the ShuffleBlockFetcherIterator to process + */ +private class RapidsShuffleFetchCompletionListener(var data: RapidsShuffleBlockFetcherIterator) + extends TaskCompletionListener { + + override def onTaskCompletion(context: TaskContext): Unit = { + if (data != null) { + data.cleanup() + // Null out the referent here to make sure we don't keep a reference to this + // ShuffleBlockFetcherIterator, after we're done reading from it, to let it be + // collected during GC. Otherwise we can hold metadata on block locations(blocksByAddress) + data = null + } + } + + // Just an alias for onTaskCompletion to avoid confusing + def onComplete(context: TaskContext): Unit = this.onTaskCompletion(context) +} + +object RapidsShuffleBlockFetcherIterator { + + /** + * A flag which indicates whether the Netty OOM error has raised during shuffle. + * If true, unless there's no in-flight fetch requests, all the pending shuffle + * fetch requests will be deferred until the flag is unset (whenever there's a + * complete fetch request). + */ + val isNettyOOMOnShuffle = new AtomicBoolean(false) + + def resetNettyOOMFlagIfPossible(freeMemoryLowerBound: Long): Unit = { + if (isNettyOOMOnShuffle.get() && NettyUtils.freeDirectMemory() >= freeMemoryLowerBound) { + isNettyOOMOnShuffle.compareAndSet(true, false) + } + } + + /** + * This function is used to merged blocks when doBatchFetch is true. Blocks which have the + * same `mapId` can be merged into one block batch. The block batch is specified by a range + * of reduceId, which implies the continuous shuffle blocks that we can fetch in a batch. + * For example, input blocks like (shuffle_0_0_0, shuffle_0_0_1, shuffle_0_1_0) can be + * merged into (shuffle_0_0_0_2, shuffle_0_1_0_1), and input blocks like (shuffle_0_0_0_2, + * shuffle_0_0_2, shuffle_0_0_3) can be merged into (shuffle_0_0_0_4). + * + * @param blocks blocks to be merged if possible. May contains already merged blocks. + * @param doBatchFetch whether to merge blocks. + * @return the input blocks if doBatchFetch=false, or the merged blocks if doBatchFetch=true. + */ + def mergeContinuousShuffleBlockIdsIfNeeded( + blocks: Seq[FetchBlockInfo], + doBatchFetch: Boolean): Seq[FetchBlockInfo] = { + val result = if (doBatchFetch) { + val curBlocks = new ArrayBuffer[FetchBlockInfo] + val mergedBlockInfo = new ArrayBuffer[FetchBlockInfo] + + def mergeFetchBlockInfo(toBeMerged: ArrayBuffer[FetchBlockInfo]): FetchBlockInfo = { + val startBlockId = toBeMerged.head.blockId.asInstanceOf[ShuffleBlockId] + + // The last merged block may comes from the input, and we can merge more blocks + // into it, if the map id is the same. + def shouldMergeIntoPreviousBatchBlockId = + mergedBlockInfo.last.blockId.asInstanceOf[ShuffleBlockBatchId].mapId == startBlockId.mapId + + val (startReduceId, size) = + if (mergedBlockInfo.nonEmpty && shouldMergeIntoPreviousBatchBlockId) { + // Remove the previous batch block id as we will add a new one to replace it. + val removed = mergedBlockInfo.remove(mergedBlockInfo.length - 1) + (removed.blockId.asInstanceOf[ShuffleBlockBatchId].startReduceId, + removed.size + toBeMerged.map(_.size).sum) + } else { + (startBlockId.reduceId, toBeMerged.map(_.size).sum) + } + + FetchBlockInfo( + ShuffleBlockBatchId( + startBlockId.shuffleId, + startBlockId.mapId, + startReduceId, + toBeMerged.last.blockId.asInstanceOf[ShuffleBlockId].reduceId + 1), + size, + toBeMerged.head.mapIndex) + } + + val iter = blocks.iterator + while (iter.hasNext) { + val info = iter.next() + // It's possible that the input block id is already a batch ID. For example, we merge some + // blocks, and then make fetch requests with the merged blocks according to "max blocks per + // request". The last fetch request may be too small, and we give up and put the remaining + // merged blocks back to the input list. + if (info.blockId.isInstanceOf[ShuffleBlockBatchId]) { + mergedBlockInfo += info + } else { + if (curBlocks.isEmpty) { + curBlocks += info + } else { + val curBlockId = info.blockId.asInstanceOf[ShuffleBlockId] + val currentMapId = curBlocks.head.blockId.asInstanceOf[ShuffleBlockId].mapId + if (curBlockId.mapId != currentMapId) { + mergedBlockInfo += mergeFetchBlockInfo(curBlocks) + curBlocks.clear() + } + curBlocks += info + } + } + } + if (curBlocks.nonEmpty) { + mergedBlockInfo += mergeFetchBlockInfo(curBlocks) + } + mergedBlockInfo + } else { + blocks + } + result.toSeq + } + + /** + * The block information to fetch used in FetchRequest. + * @param blockId block id + * @param size estimated size of the block. Note that this is NOT the exact bytes. + * Size of remote block is used to calculate bytesInFlight. + * @param mapIndex the mapIndex for this block, which indicate the index in the map stage. + */ + private[storage] case class FetchBlockInfo( + blockId: BlockId, + size: Long, + mapIndex: Int) + + /** + * A request to fetch blocks from a remote BlockManager. + * @param address remote BlockManager to fetch from. + * @param blocks Sequence of the information for blocks to fetch from the same address. + * @param forMergedMetas true if this request is for requesting push-merged meta information; + * false if it is for regular or shuffle chunks. + */ + case class FetchRequest( + address: BlockManagerId, + blocks: Seq[FetchBlockInfo], + forMergedMetas: Boolean = false) { + val size = blocks.map(_.size).sum + } + + /** + * Result of a fetch from a remote block. + */ + private[storage] sealed trait FetchResult + + /** + * Result of a fetch from a remote block successfully. + * @param blockId block id + * @param mapIndex the mapIndex for this block, which indicate the index in the map stage. + * @param address BlockManager that the block was fetched from. + * @param size estimated size of the block. Note that this is NOT the exact bytes. + * Size of remote block is used to calculate bytesInFlight. + * @param buf `ManagedBuffer` for the content. + * @param isNetworkReqDone Is this the last network request for this host in this fetch request. + */ + private[storage] case class SuccessFetchResult( + blockId: BlockId, + mapIndex: Int, + address: BlockManagerId, + size: Long, + buf: ManagedBuffer, + isNetworkReqDone: Boolean) extends FetchResult { + require(buf != null) + require(size >= 0) + } + + /** + * Result of a fetch from a remote block unsuccessfully. + * @param blockId block id + * @param mapIndex the mapIndex for this block, which indicate the index in the map stage + * @param address BlockManager that the block was attempted to be fetched from + * @param e the failure exception + */ + private[storage] case class FailureFetchResult( + blockId: BlockId, + mapIndex: Int, + address: BlockManagerId, + e: Throwable) + extends FetchResult + + /** + * Result of a fetch request that should be deferred for some reasons, e.g., Netty OOM + */ + private[storage] + case class DeferFetchRequestResult(fetchRequest: FetchRequest) extends FetchResult + + /** + * Result of an un-successful fetch of either of these: + * 1) Remote shuffle chunk. + * 2) Local push-merged block. + * + * Instead of treating this as a [[FailureFetchResult]], we fallback to fetch the original blocks. + * + * @param blockId block id + * @param address BlockManager that the push-merged block was attempted to be fetched from + * @param size size of the block, used to update bytesInFlight. + * @param isNetworkReqDone Is this the last network request for this host in this fetch + * request. Used to update reqsInFlight. + */ + private[storage] case class FallbackOnPushMergedFailureResult(blockId: BlockId, + address: BlockManagerId, + size: Long, + isNetworkReqDone: Boolean) extends FetchResult + + /** + * Result of a successful fetch of meta information for a remote push-merged block. + * + * @param shuffleId shuffle id. + * @param shuffleMergeId shuffleMergeId is used to uniquely identify merging process + * of shuffle by an indeterminate stage attempt. + * @param reduceId reduce id. + * @param blockSize size of each push-merged block. + * @param bitmaps bitmaps for every chunk. + * @param address BlockManager that the meta was fetched from. + */ + private[storage] case class PushMergedRemoteMetaFetchResult( + shuffleId: Int, + shuffleMergeId: Int, + reduceId: Int, + blockSize: Long, + bitmaps: Array[RoaringBitmap], + address: BlockManagerId) extends FetchResult + + /** + * Result of a failure while fetching the meta information for a remote push-merged block. + * + * @param shuffleId shuffle id. + * @param shuffleMergeId shuffleMergeId is used to uniquely identify merging process + * of shuffle by an indeterminate stage attempt. + * @param reduceId reduce id. + * @param address BlockManager that the meta was fetched from. + */ + private[storage] case class PushMergedRemoteMetaFailedFetchResult( + shuffleId: Int, + shuffleMergeId: Int, + reduceId: Int, + address: BlockManagerId) extends FetchResult + + /** + * Result of a successful fetch of meta information for a push-merged-local block. + * + * @param shuffleId shuffle id. + * @param shuffleMergeId shuffleMergeId is used to uniquely identify merging process + * of shuffle by an indeterminate stage attempt. + * @param reduceId reduce id. + * @param bitmaps bitmaps for every chunk. + * @param localDirs local directories where the push-merged shuffle files are storedl + */ + private[storage] case class PushMergedLocalMetaFetchResult( + shuffleId: Int, + shuffleMergeId: Int, + reduceId: Int, + bitmaps: Array[RoaringBitmap], + localDirs: Array[String]) extends FetchResult + + def makeIterator( + context: TaskContext, + blockManager: BlockManager, + sparkEnv: SparkEnv, + blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])], + serializerManager: SerializerManager, + readMetrics: ShuffleReadMetricsReporter, + fetchContinuousBlocksInBatch: Boolean): RapidsShuffleBlockFetcherIterator = { + new RapidsShuffleBlockFetcherIterator( + context, + blockManager.blockStoreClient, + blockManager, + sparkEnv.mapOutputTracker, + blocksByAddress, + serializerManager.wrapStream, + // Note: we use getSizeAsMb when no suffix is provided for backwards compatibility + sparkEnv.conf.get(config.REDUCER_MAX_SIZE_IN_FLIGHT) * 1024 * 1024, // 48mb default per task + sparkEnv.conf.get(config.REDUCER_MAX_REQS_IN_FLIGHT), //Int.MaxValue by default + sparkEnv.conf.get(config.REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS), + sparkEnv.conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM), + sparkEnv.conf.get(config.SHUFFLE_MAX_ATTEMPTS_ON_NETTY_OOM), + sparkEnv.conf.get(config.SHUFFLE_DETECT_CORRUPT), + sparkEnv.conf.get(config.SHUFFLE_DETECT_CORRUPT_MEMORY), + sparkEnv.conf.get(config.SHUFFLE_CHECKSUM_ENABLED), + sparkEnv.conf.get(config.SHUFFLE_CHECKSUM_ALGORITHM), + readMetrics, + fetchContinuousBlocksInBatch) + } +} diff --git a/sql-plugin/src/main/320/scala/org/apache/spark/sql/rapids/shims/spark320/RapidsShuffleInternalManager.scala b/sql-plugin/src/main/320/scala/org/apache/spark/sql/rapids/shims/spark320/RapidsShuffleInternalManager.scala index b7393fd6bf6..110770f3a48 100644 --- a/sql-plugin/src/main/320/scala/org/apache/spark/sql/rapids/shims/spark320/RapidsShuffleInternalManager.scala +++ b/sql-plugin/src/main/320/scala/org/apache/spark/sql/rapids/shims/spark320/RapidsShuffleInternalManager.scala @@ -16,11 +16,9 @@ package org.apache.spark.sql.rapids.shims.spark320 -import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.SparkConf import org.apache.spark.shuffle._ -import org.apache.spark.shuffle.sort.BypassMergeSortShuffleHandle import org.apache.spark.sql.rapids.{ProxyRapidsShuffleInternalManagerBase, RapidsShuffleInternalManagerBase} -import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter /** * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. @@ -28,22 +26,7 @@ import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter * `ShuffleManager` and `SortShuffleManager` classes. */ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) - extends RapidsShuffleInternalManagerBase(conf, isDriver) { - - override def makeBypassMergeSortShuffleWriter[K, V]( - handle: BypassMergeSortShuffleHandle[K, V], - mapId: Long, - context: TaskContext, - metricsReporter: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { - new RapidsShuffleThreadedWriter[K, V]( - blockManager, - handle, - mapId, - conf, - metricsReporter, - execComponents.get) - } -} + extends RapidsShuffleInternalManagerBase(conf, isDriver) class ProxyRapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) extends ProxyRapidsShuffleInternalManagerBase(conf, isDriver) diff --git a/sql-plugin/src/main/320until330-all/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala b/sql-plugin/src/main/320until330-all/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala new file mode 100644 index 00000000000..180365f6055 --- /dev/null +++ b/sql-plugin/src/main/320until330-all/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala @@ -0,0 +1,66 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.rapids.shims + +import ai.rapids.cudf.{NvtxColor, NvtxRange} + +import org.apache.spark.{MapOutputTracker, SparkEnv, TaskContext} +import org.apache.spark.serializer.SerializerManager +import org.apache.spark.shuffle.ShuffleReadMetricsReporter +import org.apache.spark.sql.rapids.{RapidsShuffleThreadedReaderBase, ShuffleHandleWithMetrics} +import org.apache.spark.storage.BlockManager + +class RapidsShuffleThreadedReader[K, C] ( + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + handle: ShuffleHandleWithMetrics[K, C, C], + context: TaskContext, + readMetrics: ShuffleReadMetricsReporter, + maxBytesInFlight: Long, + serializerManager: SerializerManager = SparkEnv.get.serializerManager, + blockManager: BlockManager = SparkEnv.get.blockManager, + mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker, + canUseBatchFetch: Boolean = false, + numReaderThreads: Int = 0) + extends RapidsShuffleThreadedReaderBase[K, C]( + handle, + context, + readMetrics, + maxBytesInFlight: Long, + serializerManager = serializerManager, + blockManager = blockManager, + mapOutputTracker = mapOutputTracker, + canUseBatchFetch = canUseBatchFetch, + numReaderThreads = numReaderThreads) { + + override protected def getMapSizes: GetMapSizesResult = { + val shuffleId = handle.shuffleId + withResource(new NvtxRange("getMapSizesByExecId", NvtxColor.CYAN)) { _ => + if (handle.dependency.shuffleMergeEnabled) { + val res = mapOutputTracker.getPushBasedShuffleMapSizesByExecutorId( + handle.shuffleId, startMapIndex, endMapIndex, startPartition, endPartition) + GetMapSizesResult(res.iter, canEnableBatchFetch = res.enableBatchFetch) + } else { + val address = mapOutputTracker.getMapSizesByExecutorId( + handle.shuffleId, startMapIndex, endMapIndex, startPartition, endPartition) + GetMapSizesResult(address, canEnableBatchFetch = true) + } + } + } +} diff --git a/sql-plugin/src/main/321/scala/org/apache/spark/sql/rapids/shims/spark321/RapidsShuffleInternalManager.scala b/sql-plugin/src/main/321/scala/org/apache/spark/sql/rapids/shims/spark321/RapidsShuffleInternalManager.scala index a14339daa36..f064aef811e 100644 --- a/sql-plugin/src/main/321/scala/org/apache/spark/sql/rapids/shims/spark321/RapidsShuffleInternalManager.scala +++ b/sql-plugin/src/main/321/scala/org/apache/spark/sql/rapids/shims/spark321/RapidsShuffleInternalManager.scala @@ -16,11 +16,9 @@ package org.apache.spark.sql.rapids.shims.spark321 -import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.SparkConf import org.apache.spark.shuffle._ -import org.apache.spark.shuffle.sort.BypassMergeSortShuffleHandle import org.apache.spark.sql.rapids.{ProxyRapidsShuffleInternalManagerBase, RapidsShuffleInternalManagerBase} -import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter /** * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. @@ -28,22 +26,7 @@ import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter * `ShuffleManager` and `SortShuffleManager` classes. */ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) - extends RapidsShuffleInternalManagerBase(conf, isDriver) { - - override def makeBypassMergeSortShuffleWriter[K, V]( - handle: BypassMergeSortShuffleHandle[K, V], - mapId: Long, - context: TaskContext, - metricsReporter: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { - new RapidsShuffleThreadedWriter[K, V]( - blockManager, - handle, - mapId, - conf, - metricsReporter, - execComponents.get) - } -} + extends RapidsShuffleInternalManagerBase(conf, isDriver) class ProxyRapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) extends ProxyRapidsShuffleInternalManagerBase(conf, isDriver) diff --git a/sql-plugin/src/main/321cdh/scala/org/apache/spark/sql/rapids/shims/spark321cdh/RapidsShuffleInternalManager.scala b/sql-plugin/src/main/321cdh/scala/org/apache/spark/sql/rapids/shims/spark321cdh/RapidsShuffleInternalManager.scala index bbb4586e803..061729ae2f2 100644 --- a/sql-plugin/src/main/321cdh/scala/org/apache/spark/sql/rapids/shims/spark321cdh/RapidsShuffleInternalManager.scala +++ b/sql-plugin/src/main/321cdh/scala/org/apache/spark/sql/rapids/shims/spark321cdh/RapidsShuffleInternalManager.scala @@ -16,11 +16,9 @@ package org.apache.spark.sql.rapids.shims.spark321cdh -import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.SparkConf import org.apache.spark.shuffle._ -import org.apache.spark.shuffle.sort.BypassMergeSortShuffleHandle import org.apache.spark.sql.rapids.{ProxyRapidsShuffleInternalManagerBase, RapidsShuffleInternalManagerBase} -import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter /** * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. @@ -28,22 +26,7 @@ import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter * `ShuffleManager` and `SortShuffleManager` classes. */ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) - extends RapidsShuffleInternalManagerBase(conf, isDriver) { - - override def makeBypassMergeSortShuffleWriter[K, V]( - handle: BypassMergeSortShuffleHandle[K, V], - mapId: Long, - context: TaskContext, - metricsReporter: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { - new RapidsShuffleThreadedWriter[K, V]( - blockManager, - handle, - mapId, - conf, - metricsReporter, - execComponents.get) - } -} + extends RapidsShuffleInternalManagerBase(conf, isDriver) class ProxyRapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) extends ProxyRapidsShuffleInternalManagerBase(conf, isDriver) diff --git a/sql-plugin/src/main/321db/scala/org/apache/spark/sql/rapids/shims/spark321db/RapidsShuffleInternalManager.scala b/sql-plugin/src/main/321db/scala/org/apache/spark/sql/rapids/shims/spark321db/RapidsShuffleInternalManager.scala index 1a3341a10a6..ab5f608cb97 100644 --- a/sql-plugin/src/main/321db/scala/org/apache/spark/sql/rapids/shims/spark321db/RapidsShuffleInternalManager.scala +++ b/sql-plugin/src/main/321db/scala/org/apache/spark/sql/rapids/shims/spark321db/RapidsShuffleInternalManager.scala @@ -16,11 +16,9 @@ package org.apache.spark.sql.rapids.shims.spark321db -import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.SparkConf import org.apache.spark.shuffle._ -import org.apache.spark.shuffle.sort.BypassMergeSortShuffleHandle import org.apache.spark.sql.rapids.{ProxyRapidsShuffleInternalManagerBase, RapidsShuffleInternalManagerBase} -import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter /** * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. @@ -28,21 +26,7 @@ import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter * `ShuffleManager` and `SortShuffleManager` classes. */ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) - extends RapidsShuffleInternalManagerBase(conf, isDriver) { - override def makeBypassMergeSortShuffleWriter[K, V]( - handle: BypassMergeSortShuffleHandle[K, V], - mapId: Long, - context: TaskContext, - metricsReporter: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { - new RapidsShuffleThreadedWriter[K, V]( - blockManager, - handle, - mapId, - conf, - metricsReporter, - execComponents.get) - } -} + extends RapidsShuffleInternalManagerBase(conf, isDriver) class ProxyRapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) extends ProxyRapidsShuffleInternalManagerBase(conf, isDriver) diff --git a/sql-plugin/src/main/322/scala/org/apache/spark/sql/rapids/shims/spark322/RapidsShuffleInternalManager.scala b/sql-plugin/src/main/322/scala/org/apache/spark/sql/rapids/shims/spark322/RapidsShuffleInternalManager.scala index c74f2bc45d7..b9599045fdd 100644 --- a/sql-plugin/src/main/322/scala/org/apache/spark/sql/rapids/shims/spark322/RapidsShuffleInternalManager.scala +++ b/sql-plugin/src/main/322/scala/org/apache/spark/sql/rapids/shims/spark322/RapidsShuffleInternalManager.scala @@ -16,11 +16,9 @@ package org.apache.spark.sql.rapids.shims.spark322 -import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.SparkConf import org.apache.spark.shuffle._ -import org.apache.spark.shuffle.sort.BypassMergeSortShuffleHandle import org.apache.spark.sql.rapids.{ProxyRapidsShuffleInternalManagerBase, RapidsShuffleInternalManagerBase} -import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter /** * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. @@ -28,22 +26,7 @@ import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter * `ShuffleManager` and `SortShuffleManager` classes. */ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) - extends RapidsShuffleInternalManagerBase(conf, isDriver) { - - override def makeBypassMergeSortShuffleWriter[K, V]( - handle: BypassMergeSortShuffleHandle[K, V], - mapId: Long, - context: TaskContext, - metricsReporter: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { - new RapidsShuffleThreadedWriter[K, V]( - blockManager, - handle, - mapId, - conf, - metricsReporter, - execComponents.get) - } -} + extends RapidsShuffleInternalManagerBase(conf, isDriver) class ProxyRapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) - extends ProxyRapidsShuffleInternalManagerBase(conf, isDriver) with ShuffleManager + extends ProxyRapidsShuffleInternalManagerBase(conf, isDriver) with ShuffleManager diff --git a/sql-plugin/src/main/330/scala/org/apache/spark/sql/rapids/shims/spark330/RapidsShuffleInternalManager.scala b/sql-plugin/src/main/330/scala/org/apache/spark/sql/rapids/shims/spark330/RapidsShuffleInternalManager.scala index abaa7cc9f8a..f42020d535e 100644 --- a/sql-plugin/src/main/330/scala/org/apache/spark/sql/rapids/shims/spark330/RapidsShuffleInternalManager.scala +++ b/sql-plugin/src/main/330/scala/org/apache/spark/sql/rapids/shims/spark330/RapidsShuffleInternalManager.scala @@ -16,11 +16,9 @@ package org.apache.spark.sql.rapids.shims.spark330 -import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.SparkConf import org.apache.spark.shuffle._ -import org.apache.spark.shuffle.sort.BypassMergeSortShuffleHandle import org.apache.spark.sql.rapids.{ProxyRapidsShuffleInternalManagerBase, RapidsShuffleInternalManagerBase} -import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter /** * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. @@ -28,22 +26,7 @@ import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter * `ShuffleManager` and `SortShuffleManager` classes. */ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) - extends RapidsShuffleInternalManagerBase(conf, isDriver) { - - override def makeBypassMergeSortShuffleWriter[K, V]( - handle: BypassMergeSortShuffleHandle[K, V], - mapId: Long, - context: TaskContext, - metricsReporter: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { - new RapidsShuffleThreadedWriter[K, V]( - blockManager, - handle, - mapId, - conf, - metricsReporter, - execComponents.get) - } -} + extends RapidsShuffleInternalManagerBase(conf, isDriver) class ProxyRapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) extends ProxyRapidsShuffleInternalManagerBase(conf, isDriver) diff --git a/sql-plugin/src/main/330until340/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala b/sql-plugin/src/main/330until340/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala new file mode 100644 index 00000000000..dde2aa4e8a9 --- /dev/null +++ b/sql-plugin/src/main/330until340/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala @@ -0,0 +1,66 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.rapids.shims + +import ai.rapids.cudf.{NvtxColor, NvtxRange} + +import org.apache.spark.{MapOutputTracker, SparkEnv, TaskContext} +import org.apache.spark.serializer.SerializerManager +import org.apache.spark.shuffle.ShuffleReadMetricsReporter +import org.apache.spark.sql.rapids.{RapidsShuffleThreadedReaderBase, ShuffleHandleWithMetrics} +import org.apache.spark.storage.BlockManager + +class RapidsShuffleThreadedReader[K, C] ( + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + handle: ShuffleHandleWithMetrics[K, C, C], + context: TaskContext, + readMetrics: ShuffleReadMetricsReporter, + maxBytesInFlight: Long, + serializerManager: SerializerManager = SparkEnv.get.serializerManager, + blockManager: BlockManager = SparkEnv.get.blockManager, + mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker, + canUseBatchFetch: Boolean = false, + numReaderThreads: Int = 0) + extends RapidsShuffleThreadedReaderBase[K, C]( + handle, + context, + readMetrics, + maxBytesInFlight, + serializerManager = serializerManager, + blockManager = blockManager, + mapOutputTracker = mapOutputTracker, + canUseBatchFetch = canUseBatchFetch, + numReaderThreads = numReaderThreads) { + + override protected def getMapSizes: GetMapSizesResult = { + val shuffleId = handle.shuffleId + withResource(new NvtxRange("getMapSizesByExecId", NvtxColor.CYAN)) { _ => + if (handle.dependency.isShuffleMergeFinalizedMarked) { + val res = mapOutputTracker.getPushBasedShuffleMapSizesByExecutorId( + handle.shuffleId, startMapIndex, endMapIndex, startPartition, endPartition) + GetMapSizesResult(res.iter, canEnableBatchFetch = res.enableBatchFetch) + } else { + val address = mapOutputTracker.getMapSizesByExecutorId( + handle.shuffleId, startMapIndex, endMapIndex, startPartition, endPartition) + GetMapSizesResult(address, canEnableBatchFetch = true) + } + } + } +} diff --git a/sql-plugin/src/main/331/scala/org/apache/spark/sql/rapids/shims/spark331/RapidsShuffleInternalManager.scala b/sql-plugin/src/main/331/scala/org/apache/spark/sql/rapids/shims/spark331/RapidsShuffleInternalManager.scala index cc9ba2dc4a9..ed0bbb3dfa3 100644 --- a/sql-plugin/src/main/331/scala/org/apache/spark/sql/rapids/shims/spark331/RapidsShuffleInternalManager.scala +++ b/sql-plugin/src/main/331/scala/org/apache/spark/sql/rapids/shims/spark331/RapidsShuffleInternalManager.scala @@ -16,11 +16,9 @@ package org.apache.spark.sql.rapids.shims.spark331 -import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.SparkConf import org.apache.spark.shuffle._ -import org.apache.spark.shuffle.sort.BypassMergeSortShuffleHandle import org.apache.spark.sql.rapids.{ProxyRapidsShuffleInternalManagerBase, RapidsShuffleInternalManagerBase} -import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter /** * A shuffle manager optimized for the RAPIDS Plugin For Apache Spark. @@ -28,22 +26,7 @@ import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter * `ShuffleManager` and `SortShuffleManager` classes. */ class RapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) - extends RapidsShuffleInternalManagerBase(conf, isDriver) { - - override def makeBypassMergeSortShuffleWriter[K, V]( - handle: BypassMergeSortShuffleHandle[K, V], - mapId: Long, - context: TaskContext, - metricsReporter: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { - new RapidsShuffleThreadedWriter[K, V]( - blockManager, - handle, - mapId, - conf, - metricsReporter, - execComponents.get) - } -} + extends RapidsShuffleInternalManagerBase(conf, isDriver) class ProxyRapidsShuffleInternalManager(conf: SparkConf, isDriver: Boolean) extends ProxyRapidsShuffleInternalManagerBase(conf, isDriver) with ShuffleManager diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala index 5fa3b6aabde..024178bb9eb 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala @@ -31,6 +31,80 @@ import org.apache.spark.serializer.{DeserializationStream, SerializationStream, import org.apache.spark.sql.types.NullType import org.apache.spark.sql.vectorized.ColumnarBatch +class SerializedBatchIterator(dIn: DataInputStream) + extends Iterator[(Int, ColumnarBatch)] with Arm { + private[this] var nextHeader: Option[SerializedTableHeader] = None + private[this] var toBeReturned: Option[ColumnarBatch] = None + private[this] var streamClosed: Boolean = false + + Option(TaskContext.get()).foreach { + _.addTaskCompletionListener[Unit]((_: TaskContext) => { + toBeReturned.foreach(_.close()) + toBeReturned = None + dIn.close() + }) + } + + def tryReadNextHeader(): Option[Long] = { + if (streamClosed){ + None + } else { + if (nextHeader.isEmpty) { + withResource(new NvtxRange("Read Header", NvtxColor.YELLOW)) { _ => + val header = new SerializedTableHeader(dIn) + if (header.wasInitialized) { + nextHeader = Some(header) + } else { + dIn.close() + streamClosed = true + nextHeader = None + } + } + } + nextHeader.map(_.getDataLen) + } + } + + def tryReadNext(): Option[ColumnarBatch] = { + if (nextHeader.isEmpty) { + None + } else { + withResource(new NvtxRange("Read Batch", NvtxColor.YELLOW)) { _ => + val header = nextHeader.get + if (header.getNumColumns > 0) { + // This buffer will later be concatenated into another host buffer before being + // sent to the GPU, so no need to use pinned memory for these buffers. + closeOnExcept( + HostMemoryBuffer.allocate(header.getDataLen, false)) { hostBuffer => + JCudfSerialization.readTableIntoBuffer(dIn, header, hostBuffer) + Some(SerializedTableColumn.from(header, hostBuffer)) + } + } else { + Some(SerializedTableColumn.from(header)) + } + } + } + } + + override def hasNext: Boolean = { + tryReadNextHeader() + nextHeader.isDefined + } + + override def next(): (Int, ColumnarBatch) = { + if (toBeReturned.isEmpty) { + tryReadNextHeader() + toBeReturned = tryReadNext() + if (nextHeader.isEmpty || toBeReturned.isEmpty) { + throw new NoSuchElementException("Walked off of the end...") + } + } + val ret = toBeReturned.get + toBeReturned = None + nextHeader = None + (0, ret) + } +} /** * Serializer for serializing `ColumnarBatch`s for use during normal shuffle. * @@ -137,62 +211,13 @@ private class GpuColumnarBatchSerializerInstance(dataSize: GpuMetric) extends Se } } + override def deserializeStream(in: InputStream): DeserializationStream = { new DeserializationStream { private[this] val dIn: DataInputStream = new DataInputStream(new BufferedInputStream(in)) override def asKeyValueIterator: Iterator[(Int, ColumnarBatch)] = { - new Iterator[(Int, ColumnarBatch)] with Arm { - var toBeReturned: Option[ColumnarBatch] = None - - TaskContext.get().addTaskCompletionListener[Unit]((_: TaskContext) => { - toBeReturned.foreach(_.close()) - toBeReturned = None - dIn.close() - }) - - def tryReadNext(): Option[ColumnarBatch] = { - withResource(new NvtxRange("Read Batch", NvtxColor.YELLOW)) { _ => - val header = new SerializedTableHeader(dIn) - if (header.wasInitialized) { - if (header.getNumColumns > 0) { - // This buffer will later be concatenated into another host buffer before being - // sent to the GPU, so no need to use pinned memory for these buffers. - closeOnExcept(HostMemoryBuffer.allocate(header.getDataLen, false)) { hostBuffer => - JCudfSerialization.readTableIntoBuffer(dIn, header, hostBuffer) - Some(SerializedTableColumn.from(header, hostBuffer)) - } - } else { - Some(SerializedTableColumn.from(header)) - } - } else { - // at EOF - dIn.close() - None - } - } - } - - override def hasNext: Boolean = { - if (toBeReturned.isEmpty) { - toBeReturned = tryReadNext() - } - - toBeReturned.isDefined - } - - override def next(): (Int, ColumnarBatch) = { - if (toBeReturned.isEmpty) { - toBeReturned = tryReadNext() - if (toBeReturned.isEmpty) { - throw new NoSuchElementException("Walked off of the end...") - } - } - val ret = toBeReturned.get - toBeReturned = None - (0, ret) - } - } + new SerializedBatchIterator(dIn) } override def asIterator: Iterator[Any] = { @@ -264,4 +289,18 @@ object SerializedTableColumn { val column = new SerializedTableColumn(header, hostBuffer) new ColumnarBatch(Array(column), header.getNumRows) } + + def getMemoryUsed(batch: ColumnarBatch): Long = { + var sum: Long = 0 + if (batch.numCols == 1) { + val cv = batch.column(0) + cv match { + case serializedTableColumn: SerializedTableColumn + if serializedTableColumn.hostBuffer != null => + sum += serializedTableColumn.hostBuffer.getLength + case _ => + } + } + sum + } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 5fe9e83ad39..4c732f51be2 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -1318,12 +1318,33 @@ object RapidsConf { .bytesConf(ByteUnit.BYTE) .createWithDefault(64 * 1024) + val SHUFFLE_MULTITHREADED_MAX_BYTES_IN_FLIGHT = + conf("spark.rapids.shuffle.multiThreaded.maxBytesInFlight") + .doc("The size limit, in bytes, that the RAPIDS shuffle manager configured in " + + "\"MULTITHREADED\" mode will allow to be deserialized concurrently.") + .bytesConf(ByteUnit.BYTE) + .createWithDefault(Integer.MAX_VALUE) + val SHUFFLE_MULTITHREADED_WRITER_THREADS = conf("spark.rapids.shuffle.multiThreaded.writer.threads") - .doc("The number of threads to use for writing shuffle blocks per executor.") + .doc("The number of threads to use for writing shuffle blocks per executor in the " + + "RAPIDS shuffle manager configured in \"MULTITHREADED\" mode. " + + "There are two special values: " + + "0 = feature is disabled, falls back to Spark built-in shuffle writer; " + + "1 = our implementation of Spark's built-in shuffle writer with extra metrics.") .integerConf .createWithDefault(20) + val SHUFFLE_MULTITHREADED_READER_THREADS = + conf("spark.rapids.shuffle.multiThreaded.reader.threads") + .doc("The number of threads to use for reading shuffle blocks per executor in the " + + "RAPIDS shuffle manager configured in \"MULTITHREADED\" mode. " + + "There are two special values: " + + "0 = feature is disabled, falls back to Spark built-in shuffle reader; " + + "1 = our implementation of Spark's built-in shuffle reader with extra metrics.") + .integerConf + .createWithDefault(20) + // ALLUXIO CONFIGS val ALLUXIO_PATHS_REPLACE = conf("spark.rapids.alluxio.pathsToReplace") @@ -2007,8 +2028,13 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val shuffleCompressionMaxBatchMemory: Long = get(SHUFFLE_COMPRESSION_MAX_BATCH_MEMORY) + lazy val shuffleMultiThreadedMaxBytesInFlight: Long = + get(SHUFFLE_MULTITHREADED_MAX_BYTES_IN_FLIGHT) + lazy val shuffleMultiThreadedWriterThreads: Int = get(SHUFFLE_MULTITHREADED_WRITER_THREADS) + lazy val shuffleMultiThreadedReaderThreads: Int = get(SHUFFLE_MULTITHREADED_READER_THREADS) + def isUCXShuffleManagerMode: Boolean = RapidsShuffleManagerMode .withName(get(SHUFFLE_MANAGER_MODE)) == RapidsShuffleManagerMode.UCX diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala index 6daf570617f..ad8b792c01b 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala @@ -18,8 +18,8 @@ package org.apache.spark.sql.rapids import java.io.{File, FileInputStream} import java.util.Optional -import java.util.concurrent.{Callable, ExecutionException, Executors, Future} -import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.{Callable, ExecutionException, Executors, Future, LinkedBlockingQueue} +import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, ListBuffer} @@ -29,19 +29,22 @@ import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.format.TableMeta import com.nvidia.spark.rapids.shuffle.{RapidsShuffleRequestHandler, RapidsShuffleServer, RapidsShuffleTransport} -import org.apache.spark.{ShuffleDependency, SparkConf, SparkEnv, TaskContext} +import org.apache.spark.{InterruptibleIterator, MapOutputTracker, ShuffleDependency, SparkConf, SparkEnv, TaskContext} import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.internal.{config, Logging} +import org.apache.spark.io.CompressionCodec import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.scheduler.MapStatus +import org.apache.spark.serializer.SerializerManager import org.apache.spark.shuffle.{ShuffleWriter, _} import org.apache.spark.shuffle.api._ import org.apache.spark.shuffle.sort.{BypassMergeSortShuffleHandle, SortShuffleManager} import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.rapids.shims.GpuShuffleBlockResolver +import org.apache.spark.sql.rapids.shims.{GpuShuffleBlockResolver, RapidsShuffleThreadedReader, RapidsShuffleThreadedWriter} import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.storage._ -import org.apache.spark.util.Utils +import org.apache.spark.storage.{RapidsShuffleBlockFetcherIterator, _} +import org.apache.spark.util.{CompletionIterator, Utils} +import org.apache.spark.util.collection.ExternalSorter class GpuShuffleHandle[K, V]( val wrapped: ShuffleHandle, @@ -51,6 +54,13 @@ class GpuShuffleHandle[K, V]( override def toString: String = s"GPU SHUFFLE HANDLE $shuffleId" } +class ShuffleHandleWithMetrics[K, V, C]( + shuffleId: Int, + val metrics: Map[String, SQLMetric], + override val dependency: GpuShuffleDependency[K, V, C]) + extends BaseShuffleHandle(shuffleId, dependency) { +} + abstract class GpuShuffleBlockResolverBase( protected val wrapped: ShuffleBlockResolver, catalog: ShuffleBufferCatalog) @@ -79,8 +89,13 @@ abstract class GpuShuffleBlockResolverBase( * by multiple threads. * @param wrapped */ -class ThreadSafeShuffleWriteMetricsReporter(wrapped: ShuffleWriteMetricsReporter) +class ThreadSafeShuffleWriteMetricsReporter(val wrapped: ShuffleWriteMetricsReporter) extends ShuffleWriteMetrics { + + def getWriteTime: Long = synchronized { + TaskContext.get.taskMetrics().shuffleWriteMetrics.writeTime + } + override private[spark] def incBytesWritten(v: Long): Unit = synchronized { wrapped.incBytesWritten(v) } @@ -107,15 +122,16 @@ object RapidsShuffleInternalManagerBase extends Logging { /** * "slots" are a thread + queue thin wrapper that is used * to execute tasks that need to be done in sequentially. - * This is done such that the threaded shuffle writer posts - * tasks that are for writer_i, and that writer is guaranteed - * to be written to sequentially, but writer_j may end up - * in a different slot, and could perform its work in parallel. + * This is done such that the threaded shuffle posts + * tasks that are for writer_i, or reader_i, which are + * guaranteed to be processed sequentially for that writer or reader. + * Writers/readers that land in a different slot are working independently + * and could perform their work in parallel. * @param slotNum this slot's unique number only used to name its executor */ - private class Slot(slotNum: Int) { + private class Slot(slotNum: Int, slotType: String) { private val p = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder() - .setNameFormat(s"rapids-shuffle-writer-$slotNum") + .setNameFormat(s"rapids-shuffle-$slotType-$slotNum") .setDaemon(true) .build()) @@ -127,40 +143,67 @@ object RapidsShuffleInternalManagerBase extends Logging { } // this is set by the executor on startup, when the MULTITHREADED - // shuffle mode is utilized, as per this config: + // shuffle mode is utilized, as per these configs: // spark.rapids.shuffle.multiThreaded.writer.threads - private var numSlots: Int = 0 - private lazy val slots = new mutable.HashMap[Int, Slot]() + // spark.rapids.shuffle.multiThreaded.reader.threads + private var numWriterSlots: Int = 0 + private var numReaderSlots: Int = 0 + private lazy val writerSlots = new mutable.HashMap[Int, Slot]() + private lazy val readerSlots = new mutable.HashMap[Int, Slot]() // used by callers to obtain a unique slot - private val slotNumber = new AtomicInteger(0) + private val writerSlotNumber = new AtomicInteger(0) + private val readerSlotNumber= new AtomicInteger(0) + + /** + * Send a task to a specific write slot. + * @param slotNum the slot to submit to + * @param task a task to execute + * @note there must not be an uncaught exception while calling + * `task`. + */ + def queueWriteTask[T](slotNum: Int, task: Callable[T]): Future[T] = { + writerSlots(slotNum % numWriterSlots).offer(task) + } /** - * Send a task to a specific slot. + * Send a task to a specific read slot. * @param slotNum the slot to submit to * @param task a task to execute * @note there must not be an uncaught exception while calling * `task`. */ - def queueTask[T](slotNum: Int, task: Callable[T]): Future[T] = { - slots(slotNum % numSlots).offer(task) + def queueReadTask[T](slotNum: Int, task: Callable[T]): Future[T] = { + readerSlots(slotNum % numReaderSlots).offer(task) } - def startThreadPoolIfNeeded(numConfiguredThreads: Int): Unit = synchronized { - if (slots.isEmpty) { - numSlots = numConfiguredThreads - (0 until numSlots).foreach { slotNum => - slots.put(slotNum, new Slot(slotNum)) + def startThreadPoolIfNeeded( + numWriterThreads: Int, + numReaderThreads: Int): Unit = synchronized { + numWriterSlots = numWriterThreads + numReaderSlots = numReaderThreads + if (writerSlots.isEmpty) { + (0 until numWriterSlots).foreach { slotNum => + writerSlots.put(slotNum, new Slot(slotNum, "writer")) + } + } + if (readerSlots.isEmpty) { + (0 until numReaderSlots).foreach { slotNum => + readerSlots.put(slotNum, new Slot(slotNum, "reader")) } } } def stopThreadPool(): Unit = synchronized { - slots.values.foreach(_.shutdownNow()) - slots.clear() + writerSlots.values.foreach(_.shutdownNow()) + writerSlots.clear() + + readerSlots.values.foreach(_.shutdownNow()) + readerSlots.clear() } - def getNextSlot: Int = Math.abs(slotNumber.incrementAndGet()) + def getNextWriterSlot: Int = Math.abs(writerSlotNumber.incrementAndGet()) + def getNextReaderSlot: Int = Math.abs(readerSlotNumber.incrementAndGet()) } trait RapidsShuffleWriterShimHelper { @@ -184,18 +227,26 @@ trait RapidsShuffleWriterShimHelper { abstract class RapidsShuffleThreadedWriterBase[K, V]( blockManager: BlockManager, - handle: BypassMergeSortShuffleHandle[K, V], + handle: ShuffleHandleWithMetrics[K, V, V], mapId: Long, sparkConf: SparkConf, writeMetrics: ShuffleWriteMetricsReporter, - shuffleExecutorComponents: ShuffleExecutorComponents) + shuffleExecutorComponents: ShuffleExecutorComponents, + numWriterThreads: Int) extends ShuffleWriter[K, V] with RapidsShuffleWriterShimHelper with Arm with Logging { - private var myMapStatus: Option[MapStatus] = None - + private val metrics = handle.metrics + private val serializationTimeMetric = + metrics.get("rapidsShuffleSerializationTime") + private val shuffleWriteTimeMetric = + metrics.get("rapidsShuffleWriteTime") + private val shuffleCombineTimeMetric = + metrics.get("rapidsShuffleCombineTime") + private val ioTimeMetric = + metrics.get("rapidsShuffleWriteIoTime") private val dep: ShuffleDependency[K, V, V] = handle.dependency private val shuffleId = dep.shuffleId private val partitioner = dep.partitioner @@ -215,12 +266,12 @@ abstract class RapidsShuffleThreadedWriterBase[K, V]( override def write(records: Iterator[Product2[K, V]]): Unit = { withResource(new NvtxRange("ThreadedWriter.write", NvtxColor.RED)) { _ => withResource(new NvtxRange("compute", NvtxColor.GREEN)) { _ => - var errorOcurred: Throwable = null val mapOutputWriter = shuffleExecutorComponents.createMapOutputWriter( shuffleId, mapId, numPartitions) try { + var openTimeNs = 0L val partLengths = if (!records.hasNext) { commitAllPartitions(mapOutputWriter, true /*empty checksum*/) } else { @@ -236,33 +287,47 @@ abstract class RapidsShuffleThreadedWriterBase[K, V]( // Places writer objects at round robin slot numbers apriori // this choice is for simplicity but likely needs to change so that // we can handle skew better - val slotNum = RapidsShuffleInternalManagerBase.getNextSlot + val slotNum = RapidsShuffleInternalManagerBase.getNextWriterSlot diskBlockObjectWriters.put(i, (slotNum, writer)) } - writeMetrics.incWriteTime(System.nanoTime() - openStartTime); + openTimeNs = System.nanoTime() - openStartTime // we call write on every writer for every record in parallel val writeFutures = new mutable.Queue[Future[Unit]] - records.foreach { record => + val writeTimeStart: Long = System.nanoTime() + val recordWriteTime: AtomicLong = new AtomicLong(0L) + var computeTime: Long = 0L + while (records.hasNext) { + // get the record + val computeStartTime = System.nanoTime() + val record = records.next() + computeTime += System.nanoTime() - computeStartTime val key = record._1 val value = record._2 val reducePartitionId: Int = partitioner.getPartition(key) val (slotNum, myWriter) = diskBlockObjectWriters(reducePartitionId) - // we close batches actively in the `records` iterator as we get the next batch - // this makes sure it is kept alive while a task is able to handle it. - val cb = value match { - case columnarBatch: ColumnarBatch => - SlicedGpuColumnVector.incRefCount(columnarBatch) - case _ => - null - } - - writeFutures += RapidsShuffleInternalManagerBase.queueTask(slotNum, () => { - withResource(cb) { _ => - myWriter.write(key, value) + if (numWriterThreads == 1) { + val recordWriteTimeStart = System.nanoTime() + myWriter.write(key, value) + recordWriteTime.getAndAdd(System.nanoTime() - recordWriteTimeStart) + } else { + // we close batches actively in the `records` iterator as we get the next batch + // this makes sure it is kept alive while a task is able to handle it. + val cb = value match { + case columnarBatch: ColumnarBatch => + SlicedGpuColumnVector.incRefCount(columnarBatch) + case _ => + null } - }) + writeFutures += RapidsShuffleInternalManagerBase.queueWriteTask(slotNum, () => { + withResource(cb) { _ => + val recordWriteTimeStart = System.nanoTime() + myWriter.write(key, value) + recordWriteTime.getAndAdd(System.nanoTime() - recordWriteTimeStart) + } + }) + } } withResource(new NvtxRange("WaitingForWrites", NvtxColor.PURPLE)) { _ => @@ -283,7 +348,41 @@ abstract class RapidsShuffleThreadedWriterBase[K, V]( writeFutures.foreach(_.cancel(true /*ok to interrupt*/)) } } - writePartitionedData(mapOutputWriter) + // writeTime is the amount of time it took to push bytes through the stream + // minus the amount of time it took to get the batch from the upstream execs + val writeTimeNs = (System.nanoTime() - writeTimeStart) - computeTime + + val combineTimeStart = System.nanoTime() + val pl = writePartitionedData(mapOutputWriter) + val combineTimeNs = System.nanoTime() - combineTimeStart + + // add openTime which is also done by Spark, and we are counting + // in the ioTime later + writeMetrics.incWriteTime(openTimeNs) + + // At this point, Spark has timed the amount of time it took to write + // to disk (the IO, per write). But note that when we look at the + // multi threaded case, this metric is now no longer task-time. + // Users need to look at "rs. shuffle write time" (shuffleWriteTimeMetric), + // which does its own calculation at the task-thread level. + // We use ioTimeNs, however, to get an approximation of serialization time. + val ioTimeNs = + writeMetrics.asInstanceOf[ThreadSafeShuffleWriteMetricsReporter].getWriteTime + + // serializationTime is the time spent compressing/encoding batches that wasn't + // counted in the ioTime + val totalPerRecordWriteTime = recordWriteTime.get() + ioTimeNs + val ioRatio = (ioTimeNs.toDouble/totalPerRecordWriteTime) + val serializationRatio = 1.0 - ioRatio + + // update metrics, note that we expect them to be relative to the task + ioTimeMetric.foreach(_ += (ioRatio * writeTimeNs).toLong) + serializationTimeMetric.foreach(_ += (serializationRatio * writeTimeNs).toLong) + // we add all three here because this metric is meant to show the time + // we are blocked on writes + shuffleWriteTimeMetric.foreach(_ += (openTimeNs + writeTimeNs + combineTimeNs)) + shuffleCombineTimeMetric.foreach(_ += combineTimeNs) + pl } myMapStatus = Some(MapStatus(blockManager.shuffleServerId, partLengths, mapId)) } catch { @@ -402,6 +501,382 @@ abstract class RapidsShuffleThreadedWriterBase[K, V]( } } +abstract class RapidsShuffleThreadedReaderBase[K, C]( + handle: ShuffleHandleWithMetrics[K, C, C], + context: TaskContext, + readMetrics: ShuffleReadMetricsReporter, + maxBytesInFlight: Long, + serializerManager: SerializerManager = SparkEnv.get.serializerManager, + blockManager: BlockManager = SparkEnv.get.blockManager, + mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker, + canUseBatchFetch: Boolean = false, + numReaderThreads: Int = 0) + extends ShuffleReader[K, C] with Logging with Arm { + + case class GetMapSizesResult( + blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])], + canEnableBatchFetch: Boolean) + + protected def getMapSizes: GetMapSizesResult + + private val GetMapSizesResult(blocksByAddress, canEnableBatchFetch) = getMapSizes + + // For spark versions 3.2.0+ `canEnableBatchFetch` can be false given merged + // map output + private val shouldBatchFetch = canUseBatchFetch && canEnableBatchFetch + + private val sqlMetrics = handle.metrics + private val dep = handle.dependency + private val deserializationTimeNs = sqlMetrics.get("rapidsShuffleDeserializationTime") + private val shuffleReadTimeNs = sqlMetrics.get("rapidsShuffleReadTime") + private val dataReadSize = sqlMetrics.get("dataReadSize") + + private var shuffleReadRange: NvtxRange = + new NvtxRange("ThreadedReader.read", NvtxColor.PURPLE) + + private def closeShuffleReadRange(): Unit = { + if (shuffleReadRange != null) { + shuffleReadRange.close() + shuffleReadRange = null + } + } + + Option(TaskContext.get()).foreach {_.addTaskCompletionListener[Unit]( _ => { + // should not be needed, but just in case + closeShuffleReadRange() + })} + + private def fetchContinuousBlocksInBatch: Boolean = { + val conf = SparkEnv.get.conf + val serializerRelocatable = dep.serializer.supportsRelocationOfSerializedObjects + val compressed = conf.get(config.SHUFFLE_COMPRESS) + val codecConcatenation = if (compressed) { + CompressionCodec.supportsConcatenationOfSerializedStreams(CompressionCodec.createCodec(conf)) + } else { + true + } + val useOldFetchProtocol = conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) + // SPARK-34790: Fetching continuous blocks in batch is incompatible with io encryption. + val ioEncryption = conf.get(config.IO_ENCRYPTION_ENABLED) + + val doBatchFetch = shouldBatchFetch && serializerRelocatable && + (!compressed || codecConcatenation) && !useOldFetchProtocol && !ioEncryption + if (shouldBatchFetch && !doBatchFetch) { + logDebug("The feature tag of continuous shuffle block fetching is set to true, but " + + "we can not enable the feature because other conditions are not satisfied. " + + s"Shuffle compress: $compressed, serializer relocatable: $serializerRelocatable, " + + s"codec concatenation: $codecConcatenation, use old shuffle fetch protocol: " + + s"$useOldFetchProtocol, io encryption: $ioEncryption.") + } + doBatchFetch + } + + class BytesInFlightLimiter(maxBytesInFlight: Long) { + private var inFlight: Long = 0L + + def acquire(sz: Long): Boolean = { + if (sz == 0) { + true + } else { + synchronized { + if (inFlight == 0 || sz + inFlight < maxBytesInFlight) { + inFlight += sz + true + } else { + false + } + } + } + } + + def release(sz: Long): Unit = synchronized { + inFlight -= sz + } + } + + class RapidsShuffleThreadedBlockIterator( + fetcherIterator: RapidsShuffleBlockFetcherIterator, + serializer: GpuColumnarBatchSerializer) + extends Iterator[(Any, Any)] with Arm { + private val queued = new LinkedBlockingQueue[(Any, Any)] + private val futures = new mutable.Queue[Future[Option[BlockState]]]() + private val serializerInstance = serializer.newInstance() + private var readBlockedTime: Long = 0L + private var fetchTime: Long = 0L + private var waitTime: Long = 0L + private val limiter = new BytesInFlightLimiter(maxBytesInFlight) + private val fallbackIter: Iterator[(Any, Any)] = if (numReaderThreads == 1) { + // this is the non-optimized case, where we add metrics to capture the blocked + // time and the deserialization time as part of the shuffle read time. + new Iterator[(Any, Any)]() { + private var currentIter: Iterator[(Any, Any)] = _ + override def hasNext: Boolean = fetcherIterator.hasNext || ( + currentIter != null && currentIter.hasNext) + + override def next(): (Any, Any) = { + val fetchTimeStart = System.nanoTime() + readBlockedTime = 0 + if (currentIter == null || !currentIter.hasNext) { + val readBlockedStart = System.nanoTime() + val (_, stream) = fetcherIterator.next() + readBlockedTime = System.nanoTime() - readBlockedStart + currentIter = serializerInstance.deserializeStream(stream).asKeyValueIterator + } + val res = currentIter.next() + fetchTime = System.nanoTime() - fetchTimeStart + res + } + } + } else { + null + } + + override def hasNext: Boolean = { + if (fallbackIter != null) { + fallbackIter.hasNext + } else { + pendingIts.nonEmpty || + fetcherIterator.hasNext || futures.nonEmpty || queued.size() > 0 + } + } + + case class BlockState(blockId: BlockId, batchIter: SerializedBatchIterator) + extends Iterator[(Any, Any)] { + private var nextBatchSize = batchIter.tryReadNextHeader().getOrElse(0L) + + def getNextBatchSize: Long = nextBatchSize + + override def hasNext: Boolean = batchIter.hasNext + + override def next(): (Any, Any) = { + val nextBatch = batchIter.next() + nextBatchSize = batchIter.tryReadNextHeader().getOrElse(0L) + nextBatch + } + } + + private val pendingIts = new mutable.Queue[BlockState]() + + override def next(): (Any, Any) = { + require(hasNext, "called next on an empty iterator") + withResource(new NvtxRange("ParallelDeserializerIterator.next", NvtxColor.CYAN)) { _ => + val result = if (fallbackIter != null) { + fallbackIter.next() + } else { + var waitTimeStart: Long = 0L + popFetchedIfAvailable() + waitTime = 0L + if (futures.nonEmpty) { + withResource(new NvtxRange("BatchWait", NvtxColor.CYAN)) { _ => + waitTimeStart = System.nanoTime() + val pending = futures.dequeue().get // wait for one future + waitTime += System.nanoTime() - waitTimeStart + + // if the future returned a block state, we have more work to do + pending match { + case Some(leftOver@BlockState(_, _)) => + pendingIts.enqueue(leftOver) + case _ => // done + } + } + } + + if (pendingIts.nonEmpty) { + // if we had pending iterators, we should try to see if now one can be handled + popFetchedIfAvailable() + } + + // We either have added futures and so will have items queued + // or we already exhausted the fetchIterator and are just waiting + // for our futures to finish. Either way, it's safe to block + // here while we wait. + waitTimeStart = System.nanoTime() + val res = queued.take() + waitTime += System.nanoTime() - waitTimeStart + res + } + + val uncompressedSize = result match { + case (_, cb: ColumnarBatch) => SerializedTableColumn.getMemoryUsed(cb) + case _ => 0 // TODO: do we need to handle other types here? + } + + // the deserialization time is approximated by subtracting + // time waiting for the fetch (`readBlockedTime`) from `fetchTime` + // and adding any time that was waited on while deserialization + // tasks finished. + val deserTime = (fetchTime - readBlockedTime) + waitTime + + // the amount of time blocked on shuffle reads is the fetch time + // + any time spent waiting for deserialization tasks + val shuffleReadTime = fetchTime + waitTime + + deserializationTimeNs.foreach(_ += deserTime) + shuffleReadTimeNs.foreach(_ += shuffleReadTime) + dataReadSize.foreach(_ += uncompressedSize) + + // if this is the last call, close our range + if (!hasNext) { + closeShuffleReadRange() + } + result + } + } + + private def deserializeTask(blockState: BlockState): Unit = { + val slot = RapidsShuffleInternalManagerBase.getNextReaderSlot + futures += RapidsShuffleInternalManagerBase.queueReadTask(slot, () => { + var currentBatchSize = blockState.getNextBatchSize + var didFit = true + while (blockState.hasNext && didFit) { + val batch = blockState.next() + queued.offer(batch) + limiter.release(currentBatchSize) + // peek at the next batch + currentBatchSize = blockState.getNextBatchSize + didFit = limiter.acquire(currentBatchSize) + } + if (!didFit) { + Some(blockState) + } else { + None // no further batches + } + }) + } + + private def popFetchedIfAvailable(): Unit = { + // If fetcherIterator is not exhausted, we try and get as many + // ready results. + if (pendingIts.nonEmpty) { + var continue = true + while(pendingIts.nonEmpty && continue) { + val blockState = pendingIts.head + // check if we can handle the head batch now + if (limiter.acquire(blockState.getNextBatchSize)) { + // kick off deserialization task + pendingIts.dequeue() + deserializeTask(blockState) + } else { + continue = false + } + } + } else { + if (fetcherIterator.hasNext) { + withResource(new NvtxRange("queueFetched", NvtxColor.YELLOW)) { _ => + // `resultCount` is exposed from the fetcher iterator and if non-zero, + // it means that there are pending results that need to be handled. + // We max with 1, because there could be a race condition where + // we are trying to get a batch and we haven't received any results + // yet, we need to block on the fetch for this case so we have + // something to return. + var amountToDrain = Math.max(fetcherIterator.resultCount, 1) + val fetchTimeStart = System.nanoTime() + + // We drain fetched results. That is, we push decode tasks + // onto our queue until the results in the fetcher iterator + // are all dequeued (the ones that were completed up until now). + readBlockedTime = 0 + var didFit = true + while (amountToDrain > 0 && fetcherIterator.hasNext && didFit) { + amountToDrain -= 1 + // fetch block time accounts for time spent waiting for streams.next() + val readBlockedStart = System.nanoTime() + val (blockId: BlockId, inputStream) = fetcherIterator.next() + readBlockedTime += System.nanoTime() - readBlockedStart + + val deserStream = serializerInstance.deserializeStream(inputStream) + val batchIter = deserStream.asKeyValueIterator.asInstanceOf[SerializedBatchIterator] + val blockState = BlockState(blockId, batchIter) + // get the next known batch size (there could be multiple batches) + if (limiter.acquire(blockState.getNextBatchSize)) { + // we can fit at least the first batch in this block + // kick off a deserialization task + deserializeTask(blockState) + } else { + // first batch didn't fit, put iterator aside and stop asking for results + // from the fetcher + pendingIts.enqueue(blockState) + didFit = false + } + } + // keep track of the overall metric which includes blocked time + fetchTime = System.nanoTime() - fetchTimeStart + } + } + } + } + } + + /** Read the combined key-values for this reduce task */ + override def read(): Iterator[Product2[K, C]] = { + val wrappedStreams = RapidsShuffleBlockFetcherIterator.makeIterator( + context, + blockManager, + SparkEnv.get, + blocksByAddress, + serializerManager, + readMetrics, + fetchContinuousBlocksInBatch) + + val recordIter = new RapidsShuffleThreadedBlockIterator( + wrappedStreams, + dep.serializer.asInstanceOf[GpuColumnarBatchSerializer]) + + // Update the context task metrics for each record read. + val metricIter = CompletionIterator[(Any, Any), Iterator[(Any, Any)]]( + recordIter.map { record => + readMetrics.incRecordsRead(1) + record + }, context.taskMetrics().mergeShuffleReadMetrics()) + + // An interruptible iterator must be used here in order to support task cancellation + val interruptibleIter = new InterruptibleIterator[(Any, Any)](context, metricIter) + + val aggregatedIter: Iterator[Product2[K, C]] = if (dep.aggregator.isDefined) { + if (dep.mapSideCombine) { + // We are reading values that are already combined + val combinedKeyValuesIterator = interruptibleIter.asInstanceOf[Iterator[(K, C)]] + dep.aggregator.get.combineCombinersByKey(combinedKeyValuesIterator, context) + } else { + // We don't know the value type, but also don't care -- the dependency *should* + // have made sure its compatible w/ this aggregator, which will convert the value + // type to the combined type C + val keyValuesIterator = interruptibleIter.asInstanceOf[Iterator[(K, Nothing)]] + dep.aggregator.get.combineValuesByKey(keyValuesIterator, context) + } + } else { + interruptibleIter.asInstanceOf[Iterator[Product2[K, C]]] + } + + // Sort the output if there is a sort ordering defined. + val resultIter = dep.keyOrdering match { + case Some(keyOrd: Ordering[K]) => + // Create an ExternalSorter to sort the data. + val sorter = + new ExternalSorter[K, C, C](context, ordering = Some(keyOrd), serializer = dep.serializer) + sorter.insertAll(aggregatedIter) + context.taskMetrics().incMemoryBytesSpilled(sorter.memoryBytesSpilled) + context.taskMetrics().incDiskBytesSpilled(sorter.diskBytesSpilled) + context.taskMetrics().incPeakExecutionMemory(sorter.peakMemoryUsedBytes) + // Use completion callback to stop sorter if task was finished/cancelled. + context.addTaskCompletionListener[Unit](_ => { + sorter.stop() + }) + CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]](sorter.iterator, sorter.stop()) + case None => + aggregatedIter + } + + resultIter match { + case _: InterruptibleIterator[Product2[K, C]] => resultIter + case _ => + // Use another interruptible iterator here to support task cancellation as aggregator + // or(and) sorter may have consumed previous interruptible iterator. + new InterruptibleIterator[Product2[K, C]](context, resultIter) + } + } +} + class RapidsCachingWriter[K, V]( blockManager: BlockManager, // Never keep a reference to the ShuffleHandle in the cache as it being GCed triggers @@ -413,7 +888,6 @@ class RapidsCachingWriter[K, V]( shuffleStorage: RapidsDeviceMemoryStore, rapidsShuffleServer: Option[RapidsShuffleServer], metrics: Map[String, SQLMetric]) extends ShuffleWriter[K, V] with Logging { - private val numParts = handle.dependency.partitioner.numPartitions private val sizes = new Array[Long](numParts) private val writtenBufferIds = new ArrayBuffer[ShuffleBufferId](numParts) @@ -539,8 +1013,8 @@ class RapidsCachingWriter[K, V]( * `ShuffleManager` and `SortShuffleManager` classes. When configuring * Apache Spark to use the RAPIDS shuffle manager, */ -abstract class RapidsShuffleInternalManagerBase(conf: SparkConf, val isDriver: Boolean) - extends ShuffleManager with RapidsShuffleHeartbeatHandler with Logging { +class RapidsShuffleInternalManagerBase(conf: SparkConf, val isDriver: Boolean) + extends ShuffleManager with Arm with RapidsShuffleHeartbeatHandler with Logging { def getServerId: BlockManagerId = server.fold(blockManager.blockManagerId)(_.getId) @@ -568,7 +1042,8 @@ abstract class RapidsShuffleInternalManagerBase(conf: SparkConf, val isDriver: B if (!isDriver && rapidsConf.isMultiThreadedShuffleManagerMode) { RapidsShuffleInternalManagerBase.startThreadPoolIfNeeded( - rapidsConf.shuffleMultiThreadedWriterThreads) + rapidsConf.shuffleMultiThreadedWriterThreads, + rapidsConf.shuffleMultiThreadedReaderThreads) } protected val wrapped = new SortShuffleManager(conf) @@ -578,7 +1053,7 @@ abstract class RapidsShuffleInternalManagerBase(conf: SparkConf, val isDriver: B if (rapidsConf.isCacheOnlyShuffleManagerMode) { "Transport disabled (local cached blocks only)" } else { - "Experimental threaded shuffle writer mode" + "Experimental threaded shuffle mode" } } else { s"Transport enabled (remote fetches will use ${rapidsConf.shuffleTransportClassName}" @@ -689,18 +1164,8 @@ abstract class RapidsShuffleInternalManagerBase(conf: SparkConf, val isDriver: B Some(executorComponents) } - /** - * Creates a BypassMergeSortShuffleWriter specific to a spark version. - * Implemented in subclasses of `RapidsShuffleInternalManagerBase` per shim. - */ - def makeBypassMergeSortShuffleWriter[K, V]( - handle: BypassMergeSortShuffleHandle[K, V], - mapId: Long, - context: TaskContext, - metricsReporter: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] - override def getWriter[K, V]( - handle: ShuffleHandle, mapId: Long, context: TaskContext, + handle: ShuffleHandle, mapId: Long, context: TaskContext, metricsReporter: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { handle match { case gpu: GpuShuffleHandle[_, _] => @@ -716,11 +1181,18 @@ abstract class RapidsShuffleInternalManagerBase(conf: SparkConf, val isDriver: B gpu.dependency.metrics) case bmssh: BypassMergeSortShuffleHandle[_, _] => bmssh.dependency match { - case g: GpuShuffleDependency[_, _, _] if g.useMultiThreadedShuffle => - makeBypassMergeSortShuffleWriter( - handle.asInstanceOf[BypassMergeSortShuffleHandle[K, V]], - mapId, context, - new ThreadSafeShuffleWriteMetricsReporter(metricsReporter)) + case gpuDep: GpuShuffleDependency[_, _, _] if gpuDep.useMultiThreadedShuffle => + // cast the handle with specific generic types due to type-erasure + val handleWithMetrics = new ShuffleHandleWithMetrics( + bmssh.shuffleId, gpuDep.metrics, gpuDep.asInstanceOf[GpuShuffleDependency[K, V, V]]) + new RapidsShuffleThreadedWriter[K, V]( + blockManager, + handleWithMetrics, + mapId, + conf, + new ThreadSafeShuffleWriteMetricsReporter(metricsReporter), + execComponents.get, + rapidsConf.shuffleMultiThreadedWriterThreads) case _ => wrapped.getWriter(handle, mapId, context, metricsReporter) } @@ -738,18 +1210,18 @@ abstract class RapidsShuffleInternalManagerBase(conf: SparkConf, val isDriver: B context: TaskContext, metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { handle match { - case gpu: GpuShuffleHandle[_, _] => - logInfo(s"Asking map output tracker for dependency ${gpu.dependency}, " + - s"map output sizes for: ${gpu.shuffleId}, parts=$startPartition-$endPartition") - if (gpu.dependency.keyOrdering.isDefined) { + case gpuHandle: GpuShuffleHandle[_, _] => + logInfo(s"Asking map output tracker for dependency ${gpuHandle.dependency}, " + + s"map output sizes for: ${gpuHandle.shuffleId}, parts=$startPartition-$endPartition") + if (gpuHandle.dependency.keyOrdering.isDefined) { // very unlikely, but just in case throw new IllegalStateException("A key ordering was requested for a gpu shuffle " - + s"dependency ${gpu.dependency.keyOrdering.get}, this is not supported.") + + s"dependency ${gpuHandle.dependency.keyOrdering.get}, this is not supported.") } val nvtxRange = new NvtxRange("getMapSizesByExecId", NvtxColor.CYAN) val blocksByAddress = try { - SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(gpu.shuffleId, + SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(gpuHandle.shuffleId, startMapIndex, endMapIndex, startPartition, endPartition) } finally { nvtxRange.close() @@ -761,7 +1233,50 @@ abstract class RapidsShuffleInternalManagerBase(conf: SparkConf, val isDriver: B metrics, transport, getCatalogOrThrow, - gpu.dependency.sparkTypes) + gpuHandle.dependency.sparkTypes) + case other: ShuffleHandle if + rapidsConf.isMultiThreadedShuffleManagerMode + && rapidsConf.shuffleMultiThreadedReaderThreads > 0 => + // we enable a multi-threaded reader in the case where we have 1 or + // more threads and we have enbled the MULTITHREADED shuffle mode. + // We special case the threads=1 case in the reader to behave like regular + // spark, but this allows us to add extra metrics that Spark normally + // doesn't look at while materializing blocks. + val baseHandle = other.asInstanceOf[BaseShuffleHandle[K, C, C]] + + // we check that the dependency is a `GpuShuffleDependency` and if not + // we go back to the regular path (e.g. is a GpuColumnarExchange?) + // TODO: it may make sense to expand this code (and the writer code) to include + // regular Exchange nodes. For now this is being conservative and a few changes + // would need to be made to deal with missing metrics, for example, for a regular + // Exchange node. + baseHandle.dependency match { + case gpuDep: GpuShuffleDependency[K, C, C] => + // We want to use batch fetch in the non-push shuffle case. Spark + // checks for a config to see if batch fetch is enabled (this check), and + // it also checks when getting (potentially merged) map status from + // the MapOutputTracker. + val canUseBatchFetch = + SortShuffleManager.canUseBatchFetch(startPartition, endPartition, context) + + val shuffleHandleWithMetrics = new ShuffleHandleWithMetrics( + baseHandle.shuffleId, gpuDep.metrics, gpuDep) + new RapidsShuffleThreadedReader( + startMapIndex, + endMapIndex, + startPartition, + endPartition, + shuffleHandleWithMetrics, + context, + metrics, + rapidsConf.shuffleMultiThreadedMaxBytesInFlight, + canUseBatchFetch = canUseBatchFetch, + numReaderThreads = rapidsConf.shuffleMultiThreadedReaderThreads) + case _ => + val shuffleHandle = RapidsShuffleInternalManagerBase.unwrapHandle(other) + wrapped.getReader(shuffleHandle, startMapIndex, endMapIndex, startPartition, + endPartition, context, metrics) + } case other => val shuffleHandle = RapidsShuffleInternalManagerBase.unwrapHandle(other) wrapped.getReader(shuffleHandle, startMapIndex, endMapIndex, startPartition, diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExecBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExecBase.scala index bccbc871326..6fb840359aa 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExecBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExecBase.scala @@ -174,7 +174,22 @@ abstract class GpuShuffleExchangeExecBase( lazy val readMetrics = SQLShuffleReadMetricsReporter.createShuffleReadMetrics(sparkContext) override lazy val additionalMetrics : Map[String, GpuMetric] = Map( - "dataSize" -> createSizeMetric(ESSENTIAL_LEVEL,"data size") + // dataSize and dataReadSize are uncompressed, one is on write and the + // other on read + "dataSize" -> createSizeMetric(ESSENTIAL_LEVEL,"data size"), + "dataReadSize" -> createSizeMetric(MODERATE_LEVEL, "data read size"), + "rapidsShuffleSerializationTime" -> + createNanoTimingMetric(DEBUG_LEVEL,"rs. serialization time"), + "rapidsShuffleDeserializationTime" -> + createNanoTimingMetric(DEBUG_LEVEL,"rs. deserialization time"), + "rapidsShuffleWriteTime" -> + createNanoTimingMetric(ESSENTIAL_LEVEL,"rs. shuffle write time"), + "rapidsShuffleCombineTime" -> + createNanoTimingMetric(DEBUG_LEVEL,"rs. shuffle combine time"), + "rapidsShuffleWriteIoTime" -> + createNanoTimingMetric(DEBUG_LEVEL,"rs. shuffle write io time"), + "rapidsShuffleReadTime" -> + createNanoTimingMetric(ESSENTIAL_LEVEL,"rs. shuffle read time") ) ++ GpuMetric.wrap(readMetrics) ++ GpuMetric.wrap(writeMetrics) // Spark doesn't report totalTime for this operator so we override metrics diff --git a/tests/src/test/320+-noncdh-nondb/scala/org/apache/spark/sql/rapids/RapidsShuffleThreadedReaderSuite.scala b/tests/src/test/320+-noncdh-nondb/scala/org/apache/spark/sql/rapids/RapidsShuffleThreadedReaderSuite.scala new file mode 100644 index 00000000000..2d017928ed9 --- /dev/null +++ b/tests/src/test/320+-noncdh-nondb/scala/org/apache/spark/sql/rapids/RapidsShuffleThreadedReaderSuite.scala @@ -0,0 +1,175 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.rapids + +import java.io.{ByteArrayOutputStream, InputStream} +import java.nio.ByteBuffer + +import com.nvidia.spark.rapids.{Arm, GpuColumnarBatchSerializer, GpuColumnVector, NoopMetric, SparkSessionHolder} +import org.mockito.ArgumentMatchers.{eq => meq} +import org.mockito.Mockito.{mock, when} +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark._ +import org.apache.spark.internal.config +import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.serializer.SerializerManager +import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedReader +import org.apache.spark.storage.{BlockManager, BlockManagerId, ShuffleBlockId} + +/** + * + * Code ported over from `BlockStoreShuffleReaderSuite` in Apache Spark. + * + * Wrapper for a managed buffer that keeps track of how many times retain and release are called. + * + * We need to define this class ourselves instead of using a spy because the NioManagedBuffer class + * is final (final classes cannot be spied on). + */ +class RecordingManagedBuffer(underlyingBuffer: NioManagedBuffer) extends ManagedBuffer { + var callsToRetain = 0 + var callsToRelease = 0 + + override def size(): Long = underlyingBuffer.size() + override def nioByteBuffer(): ByteBuffer = underlyingBuffer.nioByteBuffer() + override def createInputStream(): InputStream = underlyingBuffer.createInputStream() + override def convertToNetty(): AnyRef = underlyingBuffer.convertToNetty() + + override def retain(): ManagedBuffer = { + callsToRetain += 1 + underlyingBuffer.retain() + } + override def release(): ManagedBuffer = { + callsToRelease += 1 + underlyingBuffer.release() + } +} + +class RapidsShuffleThreadedReaderSuite + extends FunSuite with BeforeAndAfterAll with Arm { + + override def afterAll(): Unit = { + RapidsShuffleInternalManagerBase.stopThreadPool() + } + + /** + * This test makes sure that, when data is read from a HashShuffleReader, the underlying + * ManagedBuffers that contain the data are eventually released. + */ + Seq(1, 2).foreach { numReaderThreads => + test(s"read() releases resources on completion - numThreads=$numReaderThreads") { + val testConf = new SparkConf(false) + // this sets the session and the SparkEnv + SparkSessionHolder.withSparkSession(testConf, _ => { + if (numReaderThreads > 1) { + RapidsShuffleInternalManagerBase.startThreadPoolIfNeeded(0, numReaderThreads) + } + + val reduceId = 15 + val shuffleId = 22 + val numMaps = 6 + val keyValuePairsPerMap = 10 + val serializer = new GpuColumnarBatchSerializer(NoopMetric) + + // Make a mock BlockManager that will return RecordingManagedByteBuffers of data, so that we + // can ensure retain() and release() are properly called. + val blockManager = mock(classOf[BlockManager]) + + // Create a buffer with some randomly generated key-value pairs to use as the shuffle data + // from each mappers (all mappers return the same shuffle data). + val byteOutputStream = new ByteArrayOutputStream() + val serializationStream = serializer.newInstance().serializeStream(byteOutputStream) + withResource(GpuColumnVector.emptyBatchFromTypes(Array.empty)) { emptyBatch => + (0 until keyValuePairsPerMap).foreach { i => + serializationStream.writeKey(i) + serializationStream.writeValue(GpuColumnVector.incRefCounts(emptyBatch)) + } + } + + // Setup the mocked BlockManager to return RecordingManagedBuffers. + val localBlockManagerId = BlockManagerId("test-client", "test-client", 1) + when(blockManager.blockManagerId).thenReturn(localBlockManagerId) + val buffers = (0 until numMaps).map { mapId => + // Create a ManagedBuffer with the shuffle data. + val nioBuffer = new NioManagedBuffer(ByteBuffer.wrap(byteOutputStream.toByteArray)) + val managedBuffer = new RecordingManagedBuffer(nioBuffer) + + // Setup the blockManager mock so the buffer gets returned when the shuffle code tries to + // fetch shuffle data. + val shuffleBlockId = ShuffleBlockId(shuffleId, mapId, reduceId) + when(blockManager.getLocalBlockData(meq(shuffleBlockId))).thenReturn(managedBuffer) + managedBuffer + } + + // Make a mocked MapOutputTracker for the shuffle reader to use to determine what + // shuffle data to read. + val mapOutputTracker = mock(classOf[MapOutputTracker]) + when(mapOutputTracker.getMapSizesByExecutorId( + shuffleId, 0, numMaps, reduceId, reduceId + 1)).thenReturn { + // Test a scenario where all data is local, to avoid creating a bunch of additional mocks + // for the code to read data over the network. + val shuffleBlockIdsAndSizes = (0 until numMaps).map { mapId => + val shuffleBlockId = ShuffleBlockId(shuffleId, mapId, reduceId) + (shuffleBlockId, byteOutputStream.size().toLong, mapId) + } + Seq((localBlockManagerId, shuffleBlockIdsAndSizes)).iterator + } + + // Create a mocked shuffle handle to pass into HashShuffleReader. + val shuffleHandle = { + val dependency = mock(classOf[GpuShuffleDependency[Int, Int, Int]]) + when(dependency.serializer).thenReturn(serializer) + when(dependency.aggregator).thenReturn(None) + when(dependency.keyOrdering).thenReturn(None) + new ShuffleHandleWithMetrics[Int, Int, Int]( + shuffleId, Map.empty, dependency) + } + + val serializerManager = new SerializerManager( + serializer, + new SparkConf() + .set(config.SHUFFLE_COMPRESS, false) + .set(config.SHUFFLE_SPILL_COMPRESS, false)) + + val taskContext = TaskContext.empty() + val metrics = taskContext.taskMetrics.createTempShuffleReadMetrics() + val shuffleReader = new RapidsShuffleThreadedReader[Int, Int]( + 0, + numMaps, + reduceId, + reduceId + 1, + shuffleHandle, + taskContext, + metrics, + 1024 * 1024, + serializerManager, + blockManager, + mapOutputTracker = mapOutputTracker, + numReaderThreads = numReaderThreads) + + assert(shuffleReader.read().length === keyValuePairsPerMap * numMaps) + + // Calling .length above will have exhausted the iterator; make sure that exhausting the + // iterator caused retain and release to be called on each buffer. + buffers.foreach { buffer => + assert(buffer.callsToRetain === 1) + assert(buffer.callsToRelease === 1) + } + }) + } + } +} diff --git a/tests/src/test/320+-noncdh-nondb/scala/org/apache/spark/sql/rapids/RapidsShuffleThreadedWriterSuite.scala b/tests/src/test/320+-noncdh-nondb/scala/org/apache/spark/sql/rapids/RapidsShuffleThreadedWriterSuite.scala index 9ef94df3d16..be32c4ffb7a 100644 --- a/tests/src/test/320+-noncdh-nondb/scala/org/apache/spark/sql/rapids/RapidsShuffleThreadedWriterSuite.scala +++ b/tests/src/test/320+-noncdh-nondb/scala/org/apache/spark/sql/rapids/RapidsShuffleThreadedWriterSuite.scala @@ -27,10 +27,10 @@ import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.ArgumentMatchers.{any, anyInt, anyLong} import org.mockito.Mockito._ -import org.scalatest.{BeforeAndAfterEach, FunSuite} +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.scalatest.mockito.MockitoSugar -import org.apache.spark.{HashPartitioner, ShuffleDependency, SparkConf, SparkException, TaskContext} +import org.apache.spark.{HashPartitioner, SparkConf, SparkException, TaskContext} import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.internal.config import org.apache.spark.network.shuffle.checksum.ShuffleChecksumHelper @@ -38,7 +38,6 @@ import org.apache.spark.network.util.LimitedInputStream import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.IndexShuffleBlockResolver import org.apache.spark.shuffle.api.ShuffleExecutorComponents -import org.apache.spark.shuffle.sort.BypassMergeSortShuffleHandle import org.apache.spark.shuffle.sort.io.LocalDiskShuffleExecutorComponents import org.apache.spark.sql.rapids.shims.RapidsShuffleThreadedWriter import org.apache.spark.storage.{BlockId, BlockManager, DiskBlockManager, DiskBlockObjectWriter, ShuffleChecksumBlockId, ShuffleDataBlockId, ShuffleIndexBlockId, TempShuffleBlockId} @@ -134,15 +133,17 @@ class TestIndexShuffleBlockResolver( class RapidsShuffleThreadedWriterSuite extends FunSuite with BeforeAndAfterEach + with BeforeAndAfterAll with MockitoSugar with ShuffleChecksumTestHelper { @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ @Mock(answer = RETURNS_SMART_NULLS) private var diskBlockManager: DiskBlockManager = _ @Mock(answer = RETURNS_SMART_NULLS) private var taskContext: TaskContext = _ @Mock(answer = RETURNS_SMART_NULLS) private var blockResolver: TestIndexShuffleBlockResolver = _ - @Mock(answer = RETURNS_SMART_NULLS) private var dependency: ShuffleDependency[Int, Int, Int] = _ @Mock(answer = RETURNS_SMART_NULLS) - private var dependencyBad: ShuffleDependency[Int, BadSerializable, BadSerializable] = _ + private var dependency: GpuShuffleDependency[Int, Int, Int] = _ + @Mock(answer = RETURNS_SMART_NULLS) + private var dependencyBad: GpuShuffleDependency[Int, BadSerializable, BadSerializable] = _ private var taskMetrics: TaskMetrics = _ private var tempDir: File = _ @@ -152,24 +153,33 @@ class RapidsShuffleThreadedWriterSuite extends FunSuite .set("spark.app.id", "sampleApp") private val temporaryFilesCreated: mutable.Buffer[File] = new ArrayBuffer[File]() private val blockIdToFileMap: mutable.Map[BlockId, File] = new mutable.HashMap[BlockId, File] - private var shuffleHandle: BypassMergeSortShuffleHandle[Int, Int] = _ + private var shuffleHandle: ShuffleHandleWithMetrics[Int, Int, Int] = _ + + private val numWriterThreads = 2 + + override def beforeAll(): Unit = { + RapidsShuffleInternalManagerBase.startThreadPoolIfNeeded(numWriterThreads, 0) + } - RapidsShuffleInternalManagerBase.startThreadPoolIfNeeded(2) + override def afterAll(): Unit = { + RapidsShuffleInternalManagerBase.stopThreadPool() + } override def beforeEach(): Unit = { super.beforeEach() + TaskContext.setTaskContext(taskContext) MockitoAnnotations.openMocks(this).close() tempDir = Utils.createTempDir() outputFile = File.createTempFile("shuffle", null, tempDir) - taskMetrics = new TaskMetrics - shuffleHandle = new BypassMergeSortShuffleHandle[Int, Int]( - shuffleId = 0, - dependency = dependency - ) + taskMetrics = spy(new TaskMetrics) + val shuffleWriteMetrics = new ShuffleWriteMetrics + shuffleHandle = new ShuffleHandleWithMetrics[Int, Int, Int]( + 0, Map.empty, dependency) when(dependency.partitioner).thenReturn(new HashPartitioner(7)) when(dependency.serializer).thenReturn(new JavaSerializer(conf)) when(dependencyBad.partitioner).thenReturn(new HashPartitioner(7)) when(dependencyBad.serializer).thenReturn(new JavaSerializer(conf)) + when(taskMetrics.shuffleWriteMetrics).thenReturn(shuffleWriteMetrics) when(taskContext.taskMetrics()).thenReturn(taskMetrics) when(blockResolver.getDataFile(0, 0)).thenReturn(outputFile) when(blockManager.diskBlockManager).thenReturn(diskBlockManager) @@ -245,7 +255,8 @@ class RapidsShuffleThreadedWriterSuite extends FunSuite 0L, // MapId conf, taskContext.taskMetrics().shuffleWriteMetrics, - shuffleExecutorComponents) + shuffleExecutorComponents, + numWriterThreads) writer.write(Iterator.empty) writer.stop( /* success = */ true) assert(writer.getPartitionLengths.sum === 0) @@ -270,7 +281,8 @@ class RapidsShuffleThreadedWriterSuite extends FunSuite 0L, // MapId transferConf, new ThreadSafeShuffleWriteMetricsReporter(taskContext.taskMetrics().shuffleWriteMetrics), - shuffleExecutorComponents) + shuffleExecutorComponents, + numWriterThreads) writer.write(records) writer.stop( /* success = */ true) assert(temporaryFilesCreated.nonEmpty) @@ -305,7 +317,8 @@ class RapidsShuffleThreadedWriterSuite extends FunSuite 0L, // MapId conf, taskContext.taskMetrics().shuffleWriteMetrics, - shuffleExecutorComponents) + shuffleExecutorComponents, + numWriterThreads) intercept[SparkException] { writer.write(records) @@ -326,7 +339,8 @@ class RapidsShuffleThreadedWriterSuite extends FunSuite 0L, // MapId conf, taskContext.taskMetrics().shuffleWriteMetrics, - shuffleExecutorComponents) + shuffleExecutorComponents, + numWriterThreads) intercept[SparkException] { writer.write((0 until 100000).iterator.map(i => { if (i == 99990) { @@ -379,8 +393,9 @@ class RapidsShuffleThreadedWriterSuite extends FunSuite shuffleHandle, mapId, conf, - taskContext.taskMetrics().shuffleWriteMetrics, - new LocalDiskShuffleExecutorComponents(conf, blockManager, blockResolver)) + new ThreadSafeShuffleWriteMetricsReporter(taskContext.taskMetrics().shuffleWriteMetrics), + new LocalDiskShuffleExecutorComponents(conf, blockManager, blockResolver), + numWriterThreads) writer.write(Iterator((0, 0), (1, 1), (2, 2), (3, 3), (4, 4), (5, 5), (6, 6))) writer.stop( /* success = */ true) @@ -397,9 +412,10 @@ class RapidsShuffleThreadedWriterSuite extends FunSuite (5, new BadSerializable(5))) ++ (10 until 100000).iterator.map(x => (2, new BadSerializable(x))) - val shuffleHandle = new BypassMergeSortShuffleHandle[Int, BadSerializable]( - shuffleId = 0, - dependency = dependencyBad + val shuffleHandle = new ShuffleHandleWithMetrics[Int, BadSerializable, BadSerializable]( + 0, + Map.empty, + dependencyBad ) val writer = new RapidsShuffleThreadedWriter[Int, BadSerializable]( blockManager, @@ -407,7 +423,8 @@ class RapidsShuffleThreadedWriterSuite extends FunSuite 0L, // MapId conf, new ThreadSafeShuffleWriteMetricsReporter(taskContext.taskMetrics().shuffleWriteMetrics), - shuffleExecutorComponents) + shuffleExecutorComponents, + numWriterThreads) assertThrows[IOException] { writer.write(records) } From 7937fb8aa45f33d288827058e7eb61c3c44cc2bc Mon Sep 17 00:00:00 2001 From: Nghia Truong Date: Tue, 20 Sep 2022 08:35:45 -0700 Subject: [PATCH 126/190] Allow ORC tests to run with wider range of timestamp input (#6545) * Reverse test Signed-off-by: Nghia Truong * Change `orc_timestamp_gen` Signed-off-by: Nghia Truong * Add comment Signed-off-by: Nghia Truong * Rewrite comments Signed-off-by: Nghia Truong * Fix comment Signed-off-by: Nghia Truong Signed-off-by: Nghia Truong --- .../src/main/python/hive_write_test.py | 6 ++-- integration_tests/src/main/python/orc_test.py | 29 ++++--------------- 2 files changed, 8 insertions(+), 27 deletions(-) diff --git a/integration_tests/src/main/python/hive_write_test.py b/integration_tests/src/main/python/hive_write_test.py index 4df2e4d2852..60e999bb16d 100644 --- a/integration_tests/src/main/python/hive_write_test.py +++ b/integration_tests/src/main/python/hive_write_test.py @@ -20,9 +20,9 @@ from marks import * from spark_session import is_hive_available, is_spark_330_or_later, with_cpu_session -# Using positive timestamps to work around a cudf ORC bug -# https://github.com/rapidsai/cudf/issues/11525 -# Using a limited upper end for timestamps to avoid INT96 overflow on Parquet +# Using timestamps from 1970 to work around a cudf ORC bug +# https://github.com/NVIDIA/spark-rapids/issues/140. +# Using a limited upper end for timestamps to avoid INT96 overflow on Parquet. def _restricted_timestamp(nullable=True): return TimestampGen(start=datetime(1970, 1, 1, tzinfo=timezone.utc), end=datetime(2262, 4, 11, tzinfo=timezone.utc), diff --git a/integration_tests/src/main/python/orc_test.py b/integration_tests/src/main/python/orc_test.py index 10e2f6ee691..1c7d63fba1b 100644 --- a/integration_tests/src/main/python/orc_test.py +++ b/integration_tests/src/main/python/orc_test.py @@ -31,11 +31,11 @@ def read_orc_df(data_path): def read_orc_sql(data_path): return lambda spark : spark.sql('select * from orc.`{}`'.format(data_path)) -# ORC has issues reading timestamps where it is off by 1 second if the timestamp is before -# epoch in 1970 and the microsecond value is between 0 and 1000. -# See https://github.com/rapidsai/cudf/issues/11525. +# Using timestamps from 1590 to work around a cudf ORC bug +# https://github.com/NVIDIA/spark-rapids/issues/131. +# Once the bug is fixed we should remove this and use timestamp_gen. def get_orc_timestamp_gen(nullable=True): - return TimestampGen(start=datetime(1970, 1, 1, tzinfo=timezone.utc), nullable=nullable) + return TimestampGen(start=datetime(1590, 1, 1, tzinfo=timezone.utc), nullable=nullable) orc_timestamp_gen = get_orc_timestamp_gen() @@ -484,26 +484,7 @@ def test_read_struct_without_stream(spark_tmp_path): lambda spark : spark.read.orc(data_path)) -# There is an issue when writing timestamp: https://github.com/NVIDIA/spark-rapids/issues/6312 -# Thus, we exclude timestamp types from the tests here. -# When the issue is resolved, remove all these `*_no_timestamp` generators and use just `flattened_orc_gens` -# for `orc_gen` parameter. -orc_basic_gens_no_timestamp = [gen for gen in orc_basic_gens if not isinstance(gen, TimestampGen)] - -orc_array_gens_sample_no_timestamp = [ArrayGen(sub_gen) for sub_gen in orc_basic_gens_no_timestamp] + [ - ArrayGen(ArrayGen(short_gen, max_length=10), max_length=10), - ArrayGen(ArrayGen(string_gen, max_length=10), max_length=10), - ArrayGen(ArrayGen(decimal_gen_64bit, max_length=10), max_length=10), - ArrayGen(StructGen([['child0', byte_gen], ['child1', string_gen], ['child2', float_gen]]))] - -orc_basic_struct_gen_no_timestamp = StructGen([['child'+str(ind), sub_gen] for ind, sub_gen in enumerate(orc_basic_gens_no_timestamp)]) -orc_struct_gens_sample_no_timestamp = [orc_basic_struct_gen_no_timestamp, - StructGen([['child0', byte_gen], ['child1', orc_basic_struct_gen_no_timestamp]]), - StructGen([['child0', ArrayGen(short_gen)], ['child1', double_gen]])] - -flattened_orc_gens_no_timestamp = orc_basic_gens + orc_array_gens_sample_no_timestamp + orc_struct_gens_sample_no_timestamp - -@pytest.mark.parametrize('orc_gen', flattened_orc_gens_no_timestamp, ids=idfn) +@pytest.mark.parametrize('orc_gen', flattened_orc_gens, ids=idfn) @pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) @pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) @pytest.mark.parametrize('case_sensitive', ["false", "true"]) From e1cd1255f3fe1334173dfceb4e1ea9855e13ad90 Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Tue, 20 Sep 2022 14:07:03 -0500 Subject: [PATCH 127/190] Adds link to spark supporting shuffle classes and fix copyright (#6586) * Adds references to spark for supporting shuffle classes that were ported Signed-off-by: Alessandro Bellina * Fixes license and copyright Signed-off-by: Alessandro Bellina --- .../RapidsShuffleBlockFetcherIterator.scala | 3 +++ .../storage/RapidsPushBasedFetchHelper.scala | 22 +++++++++++++------ .../RapidsShuffleBlockFetcherIterator.scala | 4 ++++ 3 files changed, 22 insertions(+), 7 deletions(-) diff --git a/sql-plugin/src/main/311until320-all/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala b/sql-plugin/src/main/311until320-all/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala index 8133125c32d..a8851a37a86 100644 --- a/sql-plugin/src/main/311until320-all/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala +++ b/sql-plugin/src/main/311until320-all/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala @@ -42,6 +42,9 @@ import org.apache.spark.util.{CompletionIterator, TaskCompletionListener, Utils} * Instead of this iterator owning the result and clearing it on `next`, the * `BufferReleasingInputStream` is in charge of that. This allows for multiple threads * to consume different `BufferReleasingInputStream`s produced from this single iterator. + * + * Compare to https://github.com/apache/spark/blob/branch-3.1: + * ./core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala */ /** diff --git a/sql-plugin/src/main/320+/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala b/sql-plugin/src/main/320+/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala index e1b1c3b90ea..4225cae1840 100644 --- a/sql-plugin/src/main/320+/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala +++ b/sql-plugin/src/main/320+/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala @@ -1,12 +1,11 @@ /* - * 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 + * Copyright (c) 2022, NVIDIA CORPORATION. * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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, @@ -32,6 +31,15 @@ import org.apache.spark.network.shuffle.{BlockStoreClient, MergedBlockMeta, Merg import org.apache.spark.storage.BlockManagerId.SHUFFLE_MERGER_IDENTIFIER import org.apache.spark.storage.RapidsShuffleBlockFetcherIterator._ +/** + * Taken mostly verbatim from `PushBasedFetchHelper` because of the type + * hierarchy (specific type for `iterator`). + * + * Compare to https://github.com/apache/spark/blob/branch-3.2, and + * https://github.com/apache/spark/blob/branch-3.3: + * ./core/src/main/scala/org/apache/spark/storage/PushBasedFetchHelper.scala + */ + /** * Helper class for [[ShuffleBlockFetcherIterator]] that encapsulates all the push-based * functionality to fetch push-merged block meta and shuffle chunks. diff --git a/sql-plugin/src/main/320+/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala b/sql-plugin/src/main/320+/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala index 45272d0f9d9..ee2f903e8cc 100644 --- a/sql-plugin/src/main/320+/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala +++ b/sql-plugin/src/main/320+/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala @@ -50,6 +50,10 @@ import org.apache.spark.util.{CompletionIterator, TaskCompletionListener, Utils} * * Reverts usage of `SparkCoreErrors` to just use the original exception here, * as this was done in Spark 3.3.0 and is rather minor. + * + * Compare to https://github.com/apache/spark/blob/branch-3.2, and + * https://github.com/apache/spark/blob/branch-3.3: + * ./core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala */ /** From 998c76e1703ec3c46929ee156e3875def064375d Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Tue, 20 Sep 2022 14:08:19 -0500 Subject: [PATCH 128/190] Updates UCX to 1.13.1 in Dockerfile-blossom.ubuntu and sets UCX_TLS=^posix (#6573) Signed-off-by: Alessandro Bellina Signed-off-by: Alessandro Bellina --- jenkins/Dockerfile-blossom.ubuntu | 4 ++-- jenkins/spark-premerge-build.sh | 9 ++++++--- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/jenkins/Dockerfile-blossom.ubuntu b/jenkins/Dockerfile-blossom.ubuntu index e30551c4a6d..1011eaf20d3 100644 --- a/jenkins/Dockerfile-blossom.ubuntu +++ b/jenkins/Dockerfile-blossom.ubuntu @@ -22,12 +22,12 @@ # CUDA_VER=11.0+ # UBUNTU_VER=18.04 or 20.04 # UCX_CUDA_VER=11 (major CUDA version) -# UCX_VER=1.12.1 +# UCX_VER=1.13.1 ### ARG CUDA_VER=11.0 ARG UBUNTU_VER=18.04 -ARG UCX_VER=1.12.1 +ARG UCX_VER=1.13.1 ARG UCX_CUDA_VER=11 FROM nvidia/cuda:${CUDA_VER}-runtime-ubuntu${UBUNTU_VER} ARG CUDA_VER diff --git a/jenkins/spark-premerge-build.sh b/jenkins/spark-premerge-build.sh index 87b54c30145..d874cfe5d10 100755 --- a/jenkins/spark-premerge-build.sh +++ b/jenkins/spark-premerge-build.sh @@ -109,9 +109,12 @@ rapids_shuffle_smoke_test() { } # using UCX shuffle - # Disabled temporarily due to: https://github.com/NVIDIA/spark-rapids/issues/6572 - # PYSP_TEST_spark_executorEnv_UCX_ERROR_SIGNALS="" \ - # invoke_shuffle_integration_test + # The UCX_TLS=^posix config is removing posix from the list of memory transports + # so that IPC regions are obtained using SysV API instead. This was done because of + # itermittent test failures. See: https://github.com/NVIDIA/spark-rapids/issues/6572 + PYSP_TEST_spark_executorEnv_UCX_ERROR_SIGNALS="" \ + PYSP_TEST_spark_executorEnv_UCX_TLS="^posix" \ + invoke_shuffle_integration_test # using MULTITHREADED shuffle PYSP_TEST_spark_rapids_shuffle_mode=MULTITHREADED \ From bc0aa045549810473c7074b1213e311b117f3715 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 20 Sep 2022 14:51:20 -0500 Subject: [PATCH 129/190] Delta Lake and AQE on Databricks 10.4 workaround [databricks] (#6587) * Handle Delta and AQE workaround around on Databricks 10.4 due to bug Signed-off-by: Thomas Graves * Update to use SQLConf Signed-off-by: Thomas Graves * cleanup switch to SQLConf * fix line length Signed-off-by: Thomas Graves --- .../scala/com/nvidia/spark/rapids/shims/AQEUtils.scala | 3 ++- .../scala/com/nvidia/spark/rapids/shims/AQEUtils.scala | 3 ++- .../scala/com/nvidia/spark/rapids/shims/AQEUtils.scala | 8 +++++++- .../main/scala/com/nvidia/spark/rapids/GpuOverrides.scala | 6 ++++-- 4 files changed, 15 insertions(+), 5 deletions(-) diff --git a/sql-plugin/src/main/311+-nondb/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala b/sql-plugin/src/main/311+-nondb/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala index c53b6a3b3e6..6a5af442efc 100644 --- a/sql-plugin/src/main/311+-nondb/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala +++ b/sql-plugin/src/main/311+-nondb/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala @@ -18,6 +18,7 @@ package com.nvidia.spark.rapids.shims import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.adaptive.{QueryStageExec, ShuffleQueryStageExec} +import org.apache.spark.sql.internal.SQLConf /** Utility methods for manipulating Catalyst classes involved in Adaptive Query Execution */ object AQEUtils { @@ -26,5 +27,5 @@ object AQEUtils { sqse.newReuseInstance(sqse.id, newOutput) } - def isAdaptiveExecutionSupportedInSparkVersion: Boolean = true + def isAdaptiveExecutionSupportedInSparkVersion(conf: SQLConf): Boolean = true } diff --git a/sql-plugin/src/main/312db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala b/sql-plugin/src/main/312db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala index 33302c1d8d0..f0addcac237 100644 --- a/sql-plugin/src/main/312db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala +++ b/sql-plugin/src/main/312db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala @@ -19,6 +19,7 @@ package com.nvidia.spark.rapids.shims import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.adaptive.{QueryStageExec, ShuffleQueryStageExec} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec +import org.apache.spark.sql.internal.SQLConf /** Utility methods for manipulating Catalyst classes involved in Adaptive Query Execution */ object AQEUtils { @@ -28,5 +29,5 @@ object AQEUtils { ShuffleQueryStageExec(sqse.id, reusedExchange, sqse.originalPlan) } - def isAdaptiveExecutionSupportedInSparkVersion: Boolean = true + def isAdaptiveExecutionSupportedInSparkVersion(conf: SQLConf): Boolean = true } diff --git a/sql-plugin/src/main/321db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala b/sql-plugin/src/main/321db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala index 1830a01e996..20c5714bf54 100644 --- a/sql-plugin/src/main/321db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala +++ b/sql-plugin/src/main/321db/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala @@ -19,6 +19,7 @@ package com.nvidia.spark.rapids.shims import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.adaptive.{QueryStageExec, ShuffleQueryStageExec} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec +import org.apache.spark.sql.internal.SQLConf /** Utility methods for manipulating Catalyst classes involved in Adaptive Query Execution */ object AQEUtils { @@ -28,5 +29,10 @@ object AQEUtils { ShuffleQueryStageExec(sqse.id, reusedExchange, sqse.originalPlan, sqse.isSparkExchange) } - def isAdaptiveExecutionSupportedInSparkVersion: Boolean = true + // Databricks 10.4 has an issue where if you turn off AQE it can still use it for + // certain operations. This causes issues with the plugin so this is to work around + // that. + def isAdaptiveExecutionSupportedInSparkVersion(conf: SQLConf): Boolean = { + conf.getConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED) + } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index cbdc7fd8ca9..c7620eeddf0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -4421,10 +4421,12 @@ case class GpuOverrides() extends Rule[SparkPlan] with Logging { logDebug(s"Fallback for RDDScanExec delta log: $rdd") } found - case aqe: AdaptiveSparkPlanExec if !AQEUtils.isAdaptiveExecutionSupportedInSparkVersion => + case aqe: AdaptiveSparkPlanExec if + !AQEUtils.isAdaptiveExecutionSupportedInSparkVersion(plan.conf) => logDebug(s"AdaptiveSparkPlanExec found on unsupported Spark Version: $aqe") true - case project: ProjectExec if !AQEUtils.isAdaptiveExecutionSupportedInSparkVersion => + case project: ProjectExec if + !AQEUtils.isAdaptiveExecutionSupportedInSparkVersion(plan.conf) => val foundExprs = project.expressions.flatMap { e => PlanUtils.findExpressions(e, { case udf: ScalaUDF => From 6d3f66c00baad8a01f18cc06def1708b2a258d5e Mon Sep 17 00:00:00 2001 From: YanxuanLiu <104543031+YanxuanLiu@users.noreply.github.com> Date: Wed, 21 Sep 2022 13:58:26 +0800 Subject: [PATCH 130/190] Use dist/pom file as source of truth for spark versions (#6437) * get spark shim version from pom Signed-off-by: YanxuanLiu * correct comments Signed-off-by: YanxuanLiu * added switch control for version-def Signed-off-by: YanxuanLiu * replace mvn command with profile var Signed-off-by: YanxuanLiu * fix some its Signed-off-by: YanxuanLiu * add func to get versions from pom Signed-off-by: YanxuanLiu * get version in version-def Signed-off-by: YanxuanLiu * get versioon from version-def in yml Signed-off-by: YanxuanLiu * Update .github/workflows/mvn-verify-check.yml fix the path Co-authored-by: Gera Shegalov * add premergeUT to pom Signed-off-by: YanxuanLiu * fix nits Signed-off-by: YanxuanLiu * fix bug: call version-def in yml Signed-off-by: YanxuanLiu * fix bug Signed-off-by: YanxuanLiu * fix nits and optimize env var names Signed-off-by: YanxuanLiu * set headVersion with env Signed-off-by: YanxuanLiu * fix yml bug Signed-off-by: YanxuanLiu * add echo for debug Signed-off-by: YanxuanLiu * add echo in yml for testing Signed-off-by: YanxuanLiu * fix bug Signed-off-by: YanxuanLiu * add SCRIPT_PATH for wrapper call Signed-off-by: YanxuanLiu * add comment Signed-off-by: YanxuanLiu * add comment Signed-off-by: YanxuanLiu * remove common and seperate ut versions to two parts Signed-off-by: YanxuanLiu * skip base version Signed-off-by: YanxuanLiu * add comment Signed-off-by: YanxuanLiu * added property for utf-8 cases Signed-off-by: YanxuanLiu Signed-off-by: YanxuanLiu Co-authored-by: Gera Shegalov --- .github/workflows/mvn-verify-check.yml | 9 ++--- dist/pom.xml | 39 ++++++++++++++++++ jenkins/spark-premerge-build.sh | 51 ++++++++++++++---------- jenkins/version-def.sh | 55 ++++++++++++++++++++++++-- 4 files changed, 124 insertions(+), 30 deletions(-) diff --git a/.github/workflows/mvn-verify-check.yml b/.github/workflows/mvn-verify-check.yml index ee981311582..99c48e8e785 100644 --- a/.github/workflows/mvn-verify-check.yml +++ b/.github/workflows/mvn-verify-check.yml @@ -42,14 +42,11 @@ jobs: id: noSnapshotVersionsStep run: | set -x - noSnapshotVersionsStr=$(mvn -B help:evaluate -q -pl dist -PnoSnapshots -Dexpression=included_buildvers -DforceStdout) - noSnapshotVersionsStr=$(echo $noSnapshotVersionsStr) - noSnapshotVersionsArr=($(IFS=", "; echo $noSnapshotVersionsStr)) - tailNoSnapshotVersionsArr=(${noSnapshotVersionsArr[@]:1}) - svArrBody=$(printf ",{\"spark-version\":\"%s\"}" "${tailNoSnapshotVersionsArr[@]}") + . jenkins/version-def.sh + svArrBody=$(printf ",{\"spark-version\":\"%s\"}" "${SPARK_SHIM_VERSIONS_NOSNAPSHOTS_TAIL[@]}") svArrBody=${svArrBody:1} svJsonStr=$(printf {\"include\":[%s]} $svArrBody) - echo ::set-output name=headVersion::${noSnapshotVersionsArr[0]} + echo ::set-output name=headVersion::$SPARK_BASE_SHIM_VERSION echo ::set-output name=tailVersions::$svJsonStr package-aggregator: diff --git a/dist/pom.xml b/dist/pom.xml index facf52d388a..feeb3ebda7d 100644 --- a/dist/pom.xml +++ b/dist/pom.xml @@ -58,6 +58,21 @@ 312db, 321db + + + 320 + + + 330 + + + 320 + ${project.build.directory}/${project.build.finalName}-${cuda.version}.jar @@ -78,6 +93,30 @@
+ + premergeUT1 + + + ${premergeUT1.buildvers} + + + + + premergeUT2 + + + ${premergeUT2.buildvers} + + + + + premergeUTF8 + + + ${premergeUTF8.buildvers} + + + + + org.apache.spark + spark-core_${scala.binary.version} + + + org.apache.arrow + arrow-vector + + + provided + + + org.apache.arrow + arrow-vector + ${arrow.cdh.version} + provided + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-annotations + + + io.netty + netty-common + + + + + + + cloudera-repo + https://repository.cloudera.com/artifactory/cloudera-repos/ + + + individual diff --git a/integration_tests/pom.xml b/integration_tests/pom.xml index 27c8b35bc2d..7aaa63c15e2 100644 --- a/integration_tests/pom.xml +++ b/integration_tests/pom.xml @@ -128,6 +128,66 @@ + + release330cdh + + + buildver + 330cdh + + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark330cdh.version} + + + org.apache.arrow + arrow-vector + + + provided + + + org.apache.spark + spark-hive_${scala.binary.version} + ${spark330cdh.version} + + + + org.apache.spark + spark-core_${scala.binary.version} + + + org.apache.arrow + arrow-vector + + + provided + + + org.apache.arrow + arrow-vector + ${arrow.cdh.version} + provided + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-annotations + + + io.netty + netty-common + + + + + + + + ${project.basedir}/src/main/330cdh/scala + ${project.basedir}/src/main/311+-nondb/scala + ${project.basedir}/src/main/320+/scala + ${project.basedir}/src/main/311until340-all/scala + ${project.basedir}/src/main/320+-nondb/scala + ${project.basedir}/src/main/320until340-all/scala + ${project.basedir}/src/main/321+/scala + ${project.basedir}/src/main/330until340/scala + ${project.basedir}/src/main/330+/scala + ${project.basedir}/src/main/post320-treenode/scala + + + + + add-test-profile-src-330cdh + add-test-source + generate-test-sources + + + ${project.basedir}/src/test/330cdh/scala + + + + + + + + + + cloudera-repo + https://repository.cloudera.com/artifactory/cloudera-repos/ + + true + + + true + + + + + common + dist + integration_tests + shuffle-plugin + sql-plugin + tests + udf-compiler + tools + aggregator + + udf-compiler @@ -1027,6 +1105,7 @@ 3.3.0 3.3.1-SNAPSHOT 3.4.0-SNAPSHOT + 3.3.0.3.3.7180.0-274 3.6.0 4.3.0 3.2.0 diff --git a/sql-plugin/pom.xml b/sql-plugin/pom.xml index 92f9d233a72..edbfeeb1989 100644 --- a/sql-plugin/pom.xml +++ b/sql-plugin/pom.xml @@ -135,6 +135,66 @@ + + release330cdh + + + buildver + 330cdh + + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark330cdh.version} + + + org.apache.arrow + arrow-vector + + + provided + + + org.apache.spark + spark-hive_${scala.binary.version} + ${spark330cdh.version} + + + + org.apache.spark + spark-core_${scala.binary.version} + + + org.apache.arrow + arrow-vector + + + provided + + + org.apache.arrow + arrow-vector + ${arrow.cdh.version} + provided + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-annotations + + + io.netty + netty-common + + + + + + + org.apache.spark + spark-core_${scala.binary.version} + + + org.apache.arrow + arrow-vector + + + provided + + + org.apache.arrow + arrow-vector + ${arrow.cdh.version} + provided + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-annotations + + + io.netty + netty-common + + + + + + + cloudera-repo + https://repository.cloudera.com/artifactory/cloudera-repos/ + + + + + org.apache.spark + spark-core_${scala.binary.version} + + + org.apache.arrow + arrow-vector + + + provided + + + org.apache.arrow + arrow-vector + ${arrow.cdh.version} + provided + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-annotations + + + io.netty + netty-common + + + + + + + cloudera-repo + https://repository.cloudera.com/artifactory/cloudera-repos/ + + +
From 86fc02b1546fd718667d586bf8aad531c721f5a6 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Wed, 21 Sep 2022 20:14:36 -0500 Subject: [PATCH 135/190] Add Apache snapshot repository when running Avro tests (#6594) * Add Apache snapshot repository when running Avro tests Signed-off-by: Jason Lowe * Unset PYSP_TEST_spark_jars_repositories after use Signed-off-by: Jason Lowe --- integration_tests/run_pyspark_from_build.sh | 5 +++++ jenkins/spark-tests.sh | 4 ++++ 2 files changed, 9 insertions(+) diff --git a/integration_tests/run_pyspark_from_build.sh b/integration_tests/run_pyspark_from_build.sh index a02c6add20e..1caa847ae3e 100755 --- a/integration_tests/run_pyspark_from_build.sh +++ b/integration_tests/run_pyspark_from_build.sh @@ -289,6 +289,10 @@ else jarOpts+=(--packages "${PYSP_TEST_spark_jars_packages}") fi + if [[ -n "$PYSP_TEST_spark_jars_repositories" ]]; then + jarOpts+=(--repositories "${PYSP_TEST_spark_jars_repositories}") + fi + if [[ -n "$PYSP_TEST_spark_driver_extraClassPath" ]]; then jarOpts+=(--driver-class-path "${PYSP_TEST_spark_driver_extraClassPath}") fi @@ -302,6 +306,7 @@ else unset PYSP_TEST_spark_driver_extraJavaOptions unset PYSP_TEST_spark_jars unset PYSP_TEST_spark_jars_packages + unset PYSP_TEST_spark_jars_repositories unset PYSP_TEST_spark_rapids_memory_gpu_allocSize exec "$SPARK_HOME"/bin/spark-submit "${jarOpts[@]}" \ diff --git a/jenkins/spark-tests.sh b/jenkins/spark-tests.sh index 2fdb81aa49f..ef9ff25dace 100755 --- a/jenkins/spark-tests.sh +++ b/jenkins/spark-tests.sh @@ -216,8 +216,12 @@ run_iceberg_tests() { run_avro_tests() { # Workaround to avoid appending avro jar file by '--jars', # which would be addressed by https://github.com/NVIDIA/spark-rapids/issues/6532 + # Adding the Apache snapshots repository since we may be running with a snapshot + # version of Apache Spark which requires accessing the snapshot repository to + # fetch the spark-avro jar. rm -vf $LOCAL_JAR_PATH/spark-avro*.jar PYSP_TEST_spark_jars_packages="org.apache.spark:spark-avro_2.12:${SPARK_VER}" \ + PYSP_TEST_spark_jars_repositories="https://repository.apache.org/snapshots" \ ./run_pyspark_from_build.sh -k avro } From 5efe6fe1fb0fd3ce544e1d3ecffc3765b98f8ff9 Mon Sep 17 00:00:00 2001 From: Hao Zhu <9665750+viadea@users.noreply.github.com> Date: Thu, 22 Sep 2022 09:48:03 -0700 Subject: [PATCH 136/190] [Doc]Add archived release page[skip ci] (#6432) * Add archived release page Signed-off-by: Hao Zhu <9665750+viadea@users.noreply.github.com> * Update docs/archive.md Co-authored-by: Sameer Raheja * Update docs/download.md Co-authored-by: Sameer Raheja Signed-off-by: Hao Zhu <9665750+viadea@users.noreply.github.com> Co-authored-by: Sameer Raheja --- docs/archive.md | 792 +++++++++++++++++++++++++++++++++++++++++++++++ docs/download.md | 783 +--------------------------------------------- 2 files changed, 794 insertions(+), 781 deletions(-) create mode 100644 docs/archive.md diff --git a/docs/archive.md b/docs/archive.md new file mode 100644 index 00000000000..f0daacd41c3 --- /dev/null +++ b/docs/archive.md @@ -0,0 +1,792 @@ +--- +layout: page +title: Archive +nav_order: 15 +--- +Below are archived releases for RAPIDS Accelerator for Apache Spark. + +## Release v22.06.0 +Hardware Requirements: + +The plugin is tested on the following architectures: + + GPU Models: NVIDIA V100, T4 and A2/A10/A30/A100 GPUs + +Software Requirements: + + OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 + + CUDA & NVIDIA Drivers*: 11.x & v450.80.02+ + + Apache Spark 3.1.1, 3.1.2, 3.1.3, 3.2.0, 3.2.1, 3.3.0, Databricks 9.1 ML LTS or 10.4 ML LTS Runtime and GCP Dataproc 2.0 + + Python 3.6+, Scala 2.12, Java 8 + +*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet +for your hardware's minimum driver version. + +*For Cloudera and EMR support, please refer to the +[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. + +### Download v22.06.0 +* Download the [RAPIDS + Accelerator for Apache Spark 22.06.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.06.0/rapids-4-spark_2.12-22.06.0.jar) + +This package is built against CUDA 11.5 and all CUDA 11.x versions are supported through [CUDA forward +compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested +on V100, T4, A2, A10, A30 and A100 GPUs with CUDA 11.0-11.5. For those using other types of GPUs which +do not have CUDA forward compatibility (for example, GeForce), CUDA 11.5 or later is required. Users will +need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. + +### Verify signature +* Download the [RAPIDS Accelerator for Apache Spark 22.06.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.06.0/rapids-4-spark_2.12-22.06.0.jar) + and [RAPIDS Accelerator for Apache Spark 22.06.0 jars.asc](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.06.0/rapids-4-spark_2.12-22.06.0.jar.asc) +* Download the [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com). +* Import the public key: `gpg --import PUB_KEY` +* Verify the signature: `gpg --verify rapids-4-spark_2.12-22.06.0.jar.asc rapids-4-spark_2.12-22.06.0.jar` + +The output if signature verify: + + gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) " + +### Release Notes +New functionality and performance improvements for this release include: +* Combined cuDF jar and rapids-4-spark jar to a single rapids-4-spark jar. + The RAPIDS Accelerator jar (rapids-4-spark jar) is the only jar that needs to be passed to Spark. + The cuDF jar is now bundled with the rapids-4-spark jar and should not be specified. +* Enable CSV read by default +* Enable regular expression by default +* Enable some float related configurations by default +* Improved ANSI support +* Add a UI for the Qualification tool +* Support function map_filter +* Enable MIG with YARN on Dataproc 2.0 +* Changed to ASYNC allocator from ARENA by default + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + +## Release v22.04.0 +Hardware Requirements: + +The plugin is tested on the following architectures: + + GPU Models: NVIDIA V100, T4 and A2/A10/A30/A100 GPUs + +Software Requirements: + + OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 + + CUDA & NVIDIA Drivers*: 11.x & v450.80.02+ + + Apache Spark 3.1.1, 3.1.2, 3.1.3, 3.2.0, 3.2.1, Databricks 9.1 ML LTS or 10.4 ML LTS Runtime and GCP Dataproc 2.0 + + Python 3.6+, Scala 2.12, Java 8 + +*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet +for your hardware's minimum driver version. + +*For Cloudera and EMR support, please refer to the +[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. + +### Download v22.04.0 +* Download the [RAPIDS + Accelerator for Apache Spark 22.04.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.04.0/rapids-4-spark_2.12-22.04.0.jar) +* Download the [RAPIDS cuDF 22.04.0 jar](https://repo1.maven.org/maven2/ai/rapids/cudf/22.04.0/cudf-22.04.0-cuda11.jar) + +This package is built against CUDA 11.5 and all CUDA 11.x versions are supported through [CUDA forward +compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested +on V100, T4, A2, A10, A30 and A100 GPUs with CUDA 11.0-11.5. For those using other types of GPUs which +do not have CUDA forward compatibility (for example, GeForce), CUDA 11.5 or later is required. Users will +need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. + +### Verify signature +* Download the [RAPIDS Accelerator for Apache Spark 22.04.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.04.0/rapids-4-spark_2.12-22.04.0.jar) + and [RAPIDS Accelerator for Apache Spark 22.04.0 jars.asc](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.04.0/rapids-4-spark_2.12-22.04.0.jar.asc) +* Download the [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com). +* Import the public key: `gpg --import PUB_KEY` +* Verify the signature: `gpg --verify rapids-4-spark_2.12-22.04.0.jar.asc rapids-4-spark_2.12-22.04.0.jar` + +The output if signature verify: + + gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) " + +### Release Notes +New functionality and performance improvements for this release include: +* Avro reader for primitive types +* ExistenceJoin support +* ArrayExists support +* GetArrayStructFields support +* Function str_to_map support +* Function percent_rank support +* Regular expression support for function split on string +* Support function approx_percentile in reduction context +* Support function element_at with non-literal index +* Spark cuSpatial UDF + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + +## Release v22.02.0 +Hardware Requirements: + +The plugin is tested on the following architectures: + + GPU Models: NVIDIA V100, T4 and A2/A10/A30/A100 GPUs + +Software Requirements: + + OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 + + CUDA & NVIDIA Drivers*: 11.x & v450.80.02+ + + Apache Spark 3.0.1, 3.0.2, 3.0.3, 3.1.1, 3.1.2, 3.2.0, 3.2.1, Databricks 7.3 ML LTS or 9.1 ML LTS Runtime and GCP Dataproc 2.0 + + Python 3.6+, Scala 2.12, Java 8 + +*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet +for your hardware's minimum driver version. + +*For Cloudera and EMR support, please refer to the +[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. + +### Download v22.02.0 +* Download the [RAPIDS + Accelerator for Apache Spark 22.02.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.02.0/rapids-4-spark_2.12-22.02.0.jar) +* Download the [RAPIDS cuDF 22.02.0 jar](https://repo1.maven.org/maven2/ai/rapids/cudf/22.02.0/cudf-22.02.0-cuda11.jar) + +This package is built against CUDA 11.5 and all CUDA 11.x versions are supported through [CUDA forward +compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested +on V100, T4, A2, A10, A30 and A100 GPUs with CUDA 11.0-11.5. For those using other types of GPUs which +do not have CUDA forward compatibility (for example, GeForce), CUDA 11.5 or later is required. Users will +need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. + +### Verify signature +* Download the [RAPIDS Accelerator for Apache Spark 22.02.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.02.0/rapids-4-spark_2.12-22.02.0.jar) + and [RAPIDS Accelerator for Apache Spark 22.02.0 jars.asc](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.02.0/rapids-4-spark_2.12-22.02.0.jar.asc) +* Download the [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com). +* Import the public key: `gpg --import PUB_KEY` +* Verify the signature: `gpg --verify rapids-4-spark_2.12-22.02.0.jar.asc rapids-4-spark_2.12-22.02.0.jar` + +The output if signature verify: + + gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) " + +### Release Notes +New functionality and performance improvements for this release include: +* Parquet reader and writer support for decimal precision up to 38 digits (128-bits) +* Decimal 128-bits casting + * Casting of decimal 128-bits values in nested types + * Casting to String from decimal 128-bits + * Casting from String to decimal 128-bits +* MIG on YARN support +* GPU explain only mode for Spark 3.x and 2.x +* JSON reader support +* Sequence function support +* regexp_extract function support +* Min and max on single-level struct +* CreateMap updates and enable CreateMap by default +* Cast from array to string +* Add regular expression support to regexp_replace function +* Support for conditional joins using libcudf's mixed join feature + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + +## Release v21.12.0 +Hardware Requirements: + +The plugin is tested on the following architectures: + + GPU Models: NVIDIA V100, T4 and A2/A10/A30/A100 GPUs + +Software Requirements: + + OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 + + CUDA & NVIDIA Drivers*: 11.x & v450.80.02+ + + Apache Spark 3.0.1, 3.0.2, 3.0.3, 3.1.1, 3.1.2, 3.2.0, Databricks 7.3 ML LTS or 9.1 ML LTS Runtime and GCP Dataproc 2.0 + + Python 3.6+, Scala 2.12, Java 8 + +*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet +for your hardware's minimum driver version. + +*For Cloudera and EMR support, please refer to the +[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. + +### Download v21.12.0 +* Download the [RAPIDS + Accelerator for Apache Spark 21.12.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.12.0/rapids-4-spark_2.12-21.12.0.jar) +* Download the [RAPIDS cuDF 21.12.2 jar](https://repo1.maven.org/maven2/ai/rapids/cudf/21.12.2/cudf-21.12.2-cuda11.jar) + +This package is built against CUDA 11.5 and all CUDA 11.x versions are supported through [CUDA forward +compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested +on V100, T4, A2, A10, A30 and A100 GPUs with CUDA 11.0-11.5. For those using other types of GPUs which +do not have CUDA forward compatibility (for example, GeForce), CUDA 11.5 or later is required. Users will +need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. + +### Verify signature +* Download the [RAPIDS Accelerator for Apache Spark 21.12.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.12.0/rapids-4-spark_2.12-21.12.0.jar) + and [RAPIDS Accelerator for Apache Spark 21.12.0 jars.asc](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.12.0/rapids-4-spark_2.12-21.12.0.jar.asc) +* Download the [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com). +* Import the public key: `gpg --import PUB_KEY` +* Verify the signature: `gpg --verify rapids-4-spark_2.12-21.12.0.jar.asc rapids-4-spark_2.12-21.12.0.jar` + +The output if signature verify: + + gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) " + +### Release Notes +New functionality and performance improvements for this release include: +* Support decimal precision up to 38 digits (128-bits) +* Support stddev on double in window context +* Support CPU row-based UDF +* CreateArray outputs array of struct +* collect_set outputs array of struct +* ORC reader and writer support for decimal precision up to 38 digits (128-bits) +* ORC writer supports array, map, and struct +* Support SampleExec, rlike +* regexp_replace supports more patterns such as replacing null +* ParquetCachedBatchSerializer supports map +* Add function explainPotentialGpuPlan to print GPU query plan in a CPU Spark cluster +* Qualification Tool + * Detect RDD APIs and JDBC Scan +* Profiling Tool + * Catch OOM errors and log a hint to increase java heap size + * Print potential problems + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + +## Release v21.10.0 +Hardware Requirements: + +The plugin is tested on the following architectures: + + GPU Models: NVIDIA V100, T4 and A2/A10/A30/A100 GPUs + +Software Requirements: + + OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 + + CUDA & NVIDIA Drivers*: 11.0-11.4 & v450.80.02+ + + Apache Spark 3.0.1, 3.0.2, 3.0.3, 3.1.1, 3.1.2, 3.2.0, Databricks 7.3 ML LTS or 8.2 ML Runtime, GCP Dataproc 2.0, and Azure Synapse + + Python 3.6+, Scala 2.12, Java 8 + +*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet +for your hardware's minimum driver version. + +*For Cloudera and EMR support, please refer to the +[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. + +### Download v21.10.0 +* Download the [RAPIDS + Accelerator for Apache Spark 21.10.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.10.0/rapids-4-spark_2.12-21.10.0.jar) +* Download the [RAPIDS cuDF 21.10.0 jar](https://repo1.maven.org/maven2/ai/rapids/cudf/21.10.0/cudf-21.10.0-cuda11.jar) + +This package is built against CUDA 11.2 and all CUDA 11.x versions are supported through [CUDA forward +compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested +on V100, T4, A2, A10, A30 and A100 GPUs with CUDA 11.0-11.4. For those using other types of GPUs which +do not have CUDA forward compatibility (for example, GeForce), CUDA 11.2 is required. Users will +need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. + +### Release Notes +New functionality and performance improvements for this release include: +* Support collect_list and collect_set in group-by aggregation +* Support stddev, percentile_approx in group-by aggregation +* RunningWindow operations on map +* HashAggregate on struct and nested struct +* Sorting on nested structs +* Explode on map, array, struct +* Union-all on map, array and struct of maps +* Parquet writing of map +* ORC reader supports reading map/struct columns +* ORC reader support decimal64 +* Qualification Tool + * Add conjunction and disjunction filters + * Filtering specific configuration values + * Filtering user name + * Reporting nested data types + * Reporting write data formats +* Profiling Tool + * Generating structured output format + * Improved profiling tool performance + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + +## Release v21.08.0 +Hardware Requirements: + +The plugin is tested on the following architectures: + + GPU Models: NVIDIA V100, T4 and A10/A30/A100 GPUs + +Software Requirements: + + OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 + + CUDA & NVIDIA Drivers*: 11.0-11.4 & v450.80.02+ + + Apache Spark 3.0.1, 3.0.2, 3.0.3, 3.1.1, 3.1.2, Databricks 7.3 ML LTS or 8.2 ML Runtime, and GCP Dataproc 2.0 + + Python 3.6+, Scala 2.12, Java 8 + +*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet +for your hardware's minimum driver version. + +*For Cloudera and EMR support, please refer to the +[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. + +### Download v21.08.0 +* Download the [RAPIDS + Accelerator for Apache Spark 21.08.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.08.0/rapids-4-spark_2.12-21.08.0.jar) +* Download the [RAPIDS cuDF 21.08.2 jar](https://repo1.maven.org/maven2/ai/rapids/cudf/21.08.2/cudf-21.08.2-cuda11.jar) + +This package is built against CUDA 11.2 and all CUDA 11.x versions are supported through [CUDA forward +compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested +on V100, T4, A30 and A100 GPUs with CUDA 11.0-11.4. For those using other types of GPUs which +do not have CUDA forward compatibility (for example, GeForce), CUDA 11.2 is required. Users will +need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. + +### Release Notes +New functionality and performance improvements for this release include: +* Handling data sets that spill out of GPU memory for group by and windowing operations +* Running window rank and dense rank operations on the GPU +* Support for the `LEGACY` timestamp +* Unioning of nested structs +* Adoption of UCX 1.11 for improved error handling for RAPIDS Spark Accelerated Shuffle +* Ability to read cached data from the GPU on the supported Databricks runtimes +* Enabling Parquet writing of array data types from the GPU +* Optimized reads for small files for ORC +* Qualification and Profiling Tools + * Additional filtering capabilities + * Reporting on data types + * Reporting on read data formats + * Ability to run the Qualification tool on Spark 2.x logs + * Ability to run the tool on Apache Spark 3.x, AWS EMR 6.3.0, Dataproc 2.0, Microsoft Azure, and + Databricks 7.3 and 8.2 logs + * Improved Qualification tool performance + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + +## Release v21.06.2 +This is a patch release to address an issue with the plugin in the Databricks 8.2 ML runtime. + +Hardware Requirements: + + GPU Models: NVIDIA V100, T4 or A10/A30/A100 GPUs + +Software Requirements: + + OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 + + CUDA & NVIDIA Drivers*: 11.0 or 11.2 & v450.80.02+ + + Apache Spark 3.0.1, 3.0.2, 3.1.1, 3.1.2, Databricks 7.3 ML LTS or 8.2 ML Runtime, and GCP Dataproc 2.0 + + Python 3.6+, Scala 2.12, Java 8 + +*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet +for your hardware's minimum driver version. + +*For Cloudera and EMR support, please refer to the +[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. + +### Download v21.06.2 +* Download the [RAPIDS + Accelerator for Apache Spark 21.06.2 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.06.2/rapids-4-spark_2.12-21.06.2.jar) +* Download the [RAPIDS cuDF 21.06.1 jar](https://repo1.maven.org/maven2/ai/rapids/cudf/21.06.1/cudf-21.06.1-cuda11.jar) + +This package is built against CUDA 11.2 and all CUDA 11.x versions are supported through [CUDA forward +compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested +on Tesla datacenter GPUs with CUDA 11.0 and 11.2. For those using other types of GPUs which +do not have CUDA forward compatibility (for example, GeForce), CUDA 11.2 is required. Users will +need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. + +### Release Notes +This release patches the plugin to address a backwards incompatible change to Parquet filters made +by Databricks in the Databricks 8.2 ML runtime. More information is in [issue +3191](https://github.com/NVIDIA/spark-rapids/issues/3191) in the RAPIDS Spark repository. See the +[Release v21.06.0](#release-v21060) release notes for more detail about new features in 21.06. + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + +## Release v21.06.1 +This is a patch release to address an issue with the plugin in the Databricks 7.3 ML LTS runtime. + +Hardware Requirements: + + GPU Models: NVIDIA V100, T4 or A10/A30/A100 GPUs + +Software Requirements: + + OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 + + CUDA & NVIDIA Drivers*: 11.0 or 11.2 & v450.80.02+ + + Apache Spark 3.0.1, 3.0.2, 3.1.1, 3.1.2, and GCP Dataproc 2.0 + + Python 3.6+, Scala 2.12, Java 8 + +*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet +for your hardware's minimum driver version. + +*For Cloudera and EMR support, please refer to the +[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. + +### Download v21.06.1 +* Download the [RAPIDS + Accelerator for Apache Spark 21.06.1 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.06.1/rapids-4-spark_2.12-21.06.1.jar) +* Download the [RAPIDS cuDF 21.06.1 jar](https://repo1.maven.org/maven2/ai/rapids/cudf/21.06.1/cudf-21.06.1-cuda11.jar) + +This package is built against CUDA 11.2 and all CUDA 11.x versions are supported through [CUDA forward +compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested +on Tesla datacenter GPUs with CUDA 11.0 and 11.2. For those using other types of GPUs which +do not have CUDA forward compatibility (for example, GeForce), CUDA 11.2 is required. Users will +need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. + +### Release Notes +This release patches the plugin to address a backwards incompatible change to Parquet filters made +by Databricks in the Databricks 7.3 ML LTS runtime. More information is in [issue +3098](https://github.com/NVIDIA/spark-rapids/issues/3098) in the RAPIDS Spark repository. See the +[Release v21.06.0](#release-v21060) release notes for more detail about new features in 21.06. + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + +## Release v21.06.0 +Starting with release 21.06.0, the project is moving to calendar versioning, with the first two +digits representing the year, the second two digits representing the month, and the last digit +representing the patch version of the release. + +Hardware Requirements: + +The plugin is tested on the following architectures: + + GPU Models: NVIDIA V100, T4 and A10/A30/A100 GPUs + +Software Requirements: + + OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 + + CUDA & NVIDIA Drivers*: 11.0 or 11.2 & v450.80.02+ + + Apache Spark 3.0.1, 3.0.2, 3.1.1, 3.1.2, Databricks 8.2 ML Runtime, and GCP Dataproc 2.0 + + Python 3.6+, Scala 2.12, Java 8 + +*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet +for your hardware's minimum driver version. + +*For Cloudera and EMR support, please refer to the +[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. + +### Download v21.06.0 +* Download the [RAPIDS + Accelerator for Apache Spark 21.06.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.06.0/rapids-4-spark_2.12-21.06.0.jar) +* Download the [RAPIDS cuDF 21.06.1 jar](https://repo1.maven.org/maven2/ai/rapids/cudf/21.06.1/cudf-21.06.1-cuda11.jar) + +This package is built against CUDA 11.2 and all CUDA 11.x versions are supported through [CUDA forward +compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested +on V100, T4, A30 and A100 GPUs with CUDA 11.0 and 11.2. For those using other types of GPUs which +do not have CUDA forward compatibility (for example, GeForce), CUDA 11.2 is required. Users will +need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. + +### Release Notes +New functionality for this release includes: +* Support for running on Cloudera CDP 7.1.6, CDP 7.1.7 and Databricks 8.2 ML +* New functionality related to arrays: + * Concatenation of array columns + * Casting arrays of floats to arrays of doubles + * Creation of 2D array types + * Hash partitioning with arrays + * Explode takes expressions that generate arrays +* New functionality for struct types: + * Sorting on struct keys + * Structs with map values + * Caching of structs +* New windowing functionality: + * Window lead / lag for arrays + * Range windows supporting non-timestamp order by expressions +* Enabling large joins that can spill out of memory +* Support for the `concat_ws` operator +* Qualification and Profiling Tools + * Qualification tool looks at a set of applications to determine if the RAPIDS Accelerator for + Apache Spark is a good fit + * Profiling tool to generate information used for debugging and profiling applications + +Performance improvements for this release include: +* Moving RAPIDS Shuffle out of beta +* Updates to UCX error handling +* GPUDirect Storage for spilling + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + +## Release v0.5.0 + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + +Hardware Requirements: + + GPU Architecture: NVIDIA Pascal™ or better (Tested on V100, T4 and A100 GPU) + +Software Requirements: + + OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS8 + + CUDA & NVIDIA Drivers: 10.1.2 & v418.87+, 10.2 & v440.33+ or 11.0 & v450.36+ + + Apache Spark 3.0.0, 3.0.1, 3.0.2, 3.1.1, Databricks 7.3 ML LTS Runtime, or GCP Dataproc 2.0 + + Python 3.6+, Scala 2.12, Java 8 + +### Download v0.5.0 +* Download the [RAPIDS Accelerator for Apache Spark v0.5.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/0.5.0/rapids-4-spark_2.12-0.5.0.jar) +* Download the RAPIDS cuDF 0.19.2 jar for your system: + * [For CUDA 11.0 & NVIDIA driver 450.36+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.19.2/cudf-0.19.2-cuda11.jar) + * [For CUDA 10.2 & NVIDIA driver 440.33+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.19.2/cudf-0.19.2-cuda10-2.jar) + * [For CUDA 10.1 & NVIDIA driver 418.87+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.19.2/cudf-0.19.2-cuda10-1.jar) + +### Release Notes +New functionality for this release includes: +* Additional support for structs, including casting structs to string, hashing structs, unioning + structs, and allowing array types and structs to pass through when doing joins +* Support for `get_json_object`, `pivot`, `explode` operators +* Casting string to decimal and decimal to string + +Performance improvements for this release include: +* Optimizing unnecessary columnar->row->columnar transitions with AQE +* Supporting out of core sorts +* Initial support for cost based optimization +* Decimal32 support +* Accelerate data transfer for map Pandas UDF +* Allow spilled buffers to be unspilled + + +## Release v0.4.1 +### Download v0.4.1 +* Download the [RAPIDS Accelerator For Apache Spark v0.4.1 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/0.4.1/rapids-4-spark_2.12-0.4.1.jar) +* Download the RAPIDS cuDF 0.18.1 jar for your system: + * [For CUDA 11.0 & NVIDIA driver 450.36+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.18.1/cudf-0.18.1-cuda11.jar) + * [For CUDA 10.2 & NVIDIA driver 440.33+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.18.1/cudf-0.18.1-cuda10-2.jar) + * [For CUDA 10.1 & NVIDIA driver 418.87+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.18.1/cudf-0.18.1-cuda10-1.jar) + +### Requirements +Hardware Requirements: + + GPU Architecture: NVIDIA Pascal™ or better (Tested on V100, T4 and A100 GPU) + +Software Requirements: + + OS: Ubuntu 16.04, Ubuntu 18.04 or CentOS 7 + + CUDA & NVIDIA Drivers: 10.1.2 & v418.87+, 10.2 & v440.33+ or 11.0 & v450.36+ + + Apache Spark 3.0, 3.0.1, 3.0.2, 3.1.1, Databricks 7.3 ML LTS Runtime, or GCP Dataproc 2.0 + + Python 3.6+, Scala 2.12, Java 8 + +### Release Notes +This is a patch release based on version 0.4.0 with the following additional fixes: +* Broadcast exchange can fail when job group is set + +The release is supported on Apache Spark 3.0.0, 3.0.1, 3.0.2, 3.1.1, Databricks 7.3 ML LTS and +Google Cloud Platform Dataproc 2.0. + +The list of all supported operations is provided [here](supported_ops.md). + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + +**_Note:_** Using NVIDIA driver release 450.80.02, 450.102.04 or 460.32.03 in combination with the +CUDA 10.1 or 10.2 toolkit may result in long read times when reading a file that is snappy +compressed. In those cases we recommend either running with the CUDA 11.0 toolkit or using a newer +driver. This issue is resolved in the 0.5.0 and higher releases. + +## Release v0.4.0 +### Download v0.4.0 +* Download [RAPIDS Accelerator For Apache Spark v0.4.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/0.4.0/rapids-4-spark_2.12-0.4.0.jar) +* Download RAPIDS cuDF 0.18.1 for your system: + * [For CUDA 11.0 & NVIDIA driver 450.36+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.18.1/cudf-0.18.1-cuda11.jar) + * [For CUDA 10.2 & NVIDIA driver 440.33+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.18.1/cudf-0.18.1-cuda10-2.jar) + * [For CUDA 10.1 & NVIDIA driver 418.87+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.18.1/cudf-0.18.1-cuda10-1.jar) + +### Requirements +Hardware Requirements: + + GPU Architecture: NVIDIA Pascal™ or better (Tested on V100, T4 and A100 GPU) + +Software Requirements: + + OS: Ubuntu 16.04, Ubuntu 18.04 or CentOS 7 + + CUDA & NVIDIA Drivers: 10.1.2 & v418.87+, 10.2 & v440.33+ or 11.0 & v450.36+ + + Apache Spark 3.0, 3.0.1, 3.0.2, 3.1.1, Databricks 7.3 ML LTS Runtime, or GCP Dataproc 2.0 + + Python 3.6+, Scala 2.12, Java 8 + +### Release Notes +New functionality for the release includes +* Decimal support up to 64 bit, including reading and writing decimal from Parquet (can be enabled + by setting `spark.rapids.sql.decimalType.enabled` to True) +* Ability for users to provide GPU versions of Scala, Java or Hive UDFs +* Shuffle and sort support for `struct` data types +* `array_contains` for list operations +* `collect_list` and `average` for windowing operations +* Murmur3 `hash` operation +* Improved performance when reading from DataSource v2 when the source produces data in the Arrow format + +This release includes additional performance improvements, including +* RAPIDS Shuffle with UCX performance improvements +* Instructions on how to use [Alluxio caching](get-started/getting-started-alluxio.md) with Spark to + leverage caching. + +The release is supported on Apache Spark 3.0.0, 3.0.1, 3.0.2, 3.1.1, Databricks 7.3 ML LTS and +Google Cloud Platform Dataproc 2.0. + +The list of all supported operations is provided [here](supported_ops.md). + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + +**_Note:_** Using NVIDIA driver release 450.80.02, 450.102.04 or 460.32.03 in combination with the +CUDA 10.1 or 10.2 toolkit may result in long read times when reading a file that is snappy +compressed. In those cases we recommend either running with the CUDA 11.0 toolkit or using a newer +driver. This issue is resolved in the 0.5.0 and higher releases. + +## Release v0.3.0 +### Download v0.3.0 +* Download [RAPIDS Accelerator For Apache Spark v0.3.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/0.3.0/rapids-4-spark_2.12-0.3.0.jar) +* Download RAPIDS cuDF 0.17 for your system: + * [For CUDA 11.0 & NVIDIA driver 450.36+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.17/cudf-0.17-cuda11.jar) + * [For CUDA 10.2 & NVIDIA driver 440.33+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.17/cudf-0.17-cuda10-2.jar) + * [For CUDA 10.1 & NVIDIA driver 418.87+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.17/cudf-0.17-cuda10-1.jar) + +### Requirements +Hardware Requirements: + + GPU Architecture: NVIDIA Pascal™ or better (Tested on V100, T4 and A100 GPU) + +Software Requirements: + + OS: Ubuntu 16.04, Ubuntu 18.04 or CentOS 7 + + CUDA & NVIDIA Drivers: 10.1.2 & v418.87+, 10.2 & v440.33+ or 11.0 & v450.36+ + + Apache Spark 3.0, 3.0.1, Databricks 7.3 ML LTS Runtime, or GCP Dataproc 2.0 + + Python 3.6+, Scala 2.12, Java 8 + +### Release Notes +This release includes additional performance improvements, including +* Use of per thread default stream to make more efficient use of the GPU +* Further supporting Spark's adaptive query execution, with more rewritten query plans now able to + run on the GPU +* Performance improvements for reading small Parquet files +* RAPIDS Shuffle with UCX updated to UCX 1.9.0 + +New functionality for the release includes +* Parquet reading for lists and structs, +* Lead/lag for windows, and +* Greatest/least operators + +The release is supported on Apache Spark 3.0.0, 3.0.1, Databricks 7.3 ML LTS and Google Cloud +Platform Dataproc 2.0. + +The list of all supported operations is provided [here](supported_ops.md). + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + +**_Note:_** Using NVIDIA driver release 450.80.02, 450.102.04 or 460.32.03 in combination with the +CUDA 10.1 or 10.2 toolkit may result in long read times when reading a file that is snappy +compressed. In those cases we recommend either running with the CUDA 11.0 toolkit or using a newer +driver. This issue is resolved in the 0.5.0 and higher releases. + +## Release v0.2.0 +### Download v0.2.0 +* Download [RAPIDS Accelerator For Apache Spark v0.2.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/0.2.0/rapids-4-spark_2.12-0.2.0.jar) +* Download RAPIDS cuDF 0.15 for your system: + * [For CUDA 11.0 & NVIDIA driver 450.36+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.15/cudf-0.15-cuda11.jar) + * [For CUDA 10.2 & NVIDIA driver 440.33+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.15/cudf-0.15-cuda10-2.jar) + * [For CUDA 10.1 & NVIDIA driver 418.87+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.15/cudf-0.15-cuda10-1.jar) + +### Requirements +Hardware Requirements: + + GPU Architecture: NVIDIA Pascal™ or better (Tested on V100, T4 and A100 GPU) + +Software Requirements: + + OS: Ubuntu 16.04, Ubuntu 18.04 or CentOS 7 + + CUDA & NVIDIA Drivers: 10.1.2 & v418.87+, 10.2 & v440.33+ or 11.0 & v450.36+ + + Apache Spark 3.0, 3.0.1 + + Python 3.x, Scala 2.12, Java 8 + +### Release Notes +This is the second release of the RAPIDS Accelerator for Apache Spark. Adaptive Query Execution +[SPARK-31412](https://issues.apache.org/jira/browse/SPARK-31412) is a new enhancement that was +included in Spark 3.0 that alters the physical execution plan dynamically to improve the performance +of the query. The RAPIDS Accelerator v0.2 introduces Adaptive Query Execution (AQE) for GPUs and +leverages columnar processing [SPARK-32332](https://issues.apache.org/jira/browse/SPARK-32332) +starting from Spark 3.0.1. + +Another enhancement in v0.2 is improvement in reading small Parquet files. This feature takes into +account the scenario where input data can be stored across many small files. By leveraging multiple +CPU threads v0.2 delivers up to 6x performance improvement over the previous release for small +Parquet file reads. + +The RAPIDS Accelerator introduces a beta feature that accelerates +[Spark shuffle for GPUs](get-started/getting-started-on-prem.md#enabling-rapids-shuffle-manager). Accelerated +shuffle makes use of high bandwidth transfers between GPUs (NVLink or p2p over PCIe) and leverages +RDMA (RoCE or Infiniband) for remote transfers. + +The list of all supported operations is provided +[here](configs.md#supported-gpu-operators-and-fine-tuning). + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + +**_Note:_** Using NVIDIA driver release 450.80.02, 450.102.04 or 460.32.03 in combination with the +CUDA 10.1 or 10.2 toolkit may result in long read times when reading a file that is snappy +compressed. In those cases we recommend either running with the CUDA 11.0 toolkit or using a newer +driver. This issue is resolved in the 0.5.0 and higher releases. + +## Release v0.1.0 +### Download v0.1.0 +* Download [RAPIDS Accelerator For Apache Spark v0.1.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/0.1.0/rapids-4-spark_2.12-0.1.0.jar) +* Download RAPIDS cuDF 0.14 for your system: + * [For CUDA 10.2 & NVIDIA driver 440.33+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.14/cudf-0.14-cuda10-2.jar) + * [For CUDA 10.1 & NVIDIA driver 418.87+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.14/cudf-0.14-cuda10-1.jar) + +### Requirements +Hardware Requirements: + + GPU Architecture: NVIDIA Pascal™ or better (Tested on V100 and T4 GPU) + +Software Requirements: + + OS: Ubuntu 16.04, Ubuntu 18.04 or CentOS 7 + (RHEL 7 support is provided through CentOS 7 builds/installs) + + CUDA & NVIDIA Drivers: 10.1.2 & v418.87+ or 10.2 & v440.33+ + + Apache Spark 3.0 + + Python 3.x, Scala 2.12, Java 8 + + + diff --git a/docs/download.md b/docs/download.md index f43970f29df..a429086d012 100644 --- a/docs/download.md +++ b/docs/download.md @@ -79,785 +79,6 @@ New functionality and performance improvements for this release include: For a detailed list of changes, please refer to the [CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). -## Release v22.06.0 -Hardware Requirements: - -The plugin is tested on the following architectures: - - GPU Models: NVIDIA V100, T4 and A2/A10/A30/A100 GPUs - -Software Requirements: - - OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 - - CUDA & NVIDIA Drivers*: 11.x & v450.80.02+ - - Apache Spark 3.1.1, 3.1.2, 3.1.3, 3.2.0, 3.2.1, 3.3.0, Databricks 9.1 ML LTS or 10.4 ML LTS Runtime and GCP Dataproc 2.0 - - Python 3.6+, Scala 2.12, Java 8 - -*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet -for your hardware's minimum driver version. - -*For Cloudera and EMR support, please refer to the -[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. - -### Download v22.06.0 -* Download the [RAPIDS - Accelerator for Apache Spark 22.06.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.06.0/rapids-4-spark_2.12-22.06.0.jar) - -This package is built against CUDA 11.5 and all CUDA 11.x versions are supported through [CUDA forward -compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested -on V100, T4, A2, A10, A30 and A100 GPUs with CUDA 11.0-11.5. For those using other types of GPUs which -do not have CUDA forward compatibility (for example, GeForce), CUDA 11.5 or later is required. Users will -need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. - -### Verify signature -* Download the [RAPIDS Accelerator for Apache Spark 22.06.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.06.0/rapids-4-spark_2.12-22.06.0.jar) - and [RAPIDS Accelerator for Apache Spark 22.06.0 jars.asc](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.06.0/rapids-4-spark_2.12-22.06.0.jar.asc) -* Download the [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com). -* Import the public key: `gpg --import PUB_KEY` -* Verify the signature: `gpg --verify rapids-4-spark_2.12-22.06.0.jar.asc rapids-4-spark_2.12-22.06.0.jar` - -The output if signature verify: - - gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) " - -### Release Notes -New functionality and performance improvements for this release include: -* Combined cuDF jar and rapids-4-spark jar to a single rapids-4-spark jar. - The RAPIDS Accelerator jar (rapids-4-spark jar) is the only jar that needs to be passed to Spark. - The cuDF jar is now bundled with the rapids-4-spark jar and should not be specified. -* Enable CSV read by default -* Enable regular expression by default -* Enable some float related configurations by default -* Improved ANSI support -* Add a UI for the Qualification tool -* Support function map_filter -* Enable MIG with YARN on Dataproc 2.0 -* Changed to ASYNC allocator from ARENA by default - -For a detailed list of changes, please refer to the -[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). - -## Release v22.04.0 -Hardware Requirements: - -The plugin is tested on the following architectures: - - GPU Models: NVIDIA V100, T4 and A2/A10/A30/A100 GPUs - -Software Requirements: - - OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 - - CUDA & NVIDIA Drivers*: 11.x & v450.80.02+ - - Apache Spark 3.1.1, 3.1.2, 3.1.3, 3.2.0, 3.2.1, Databricks 9.1 ML LTS or 10.4 ML LTS Runtime and GCP Dataproc 2.0 - - Python 3.6+, Scala 2.12, Java 8 - -*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet -for your hardware's minimum driver version. - -*For Cloudera and EMR support, please refer to the -[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. - -### Download v22.04.0 -* Download the [RAPIDS - Accelerator for Apache Spark 22.04.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.04.0/rapids-4-spark_2.12-22.04.0.jar) -* Download the [RAPIDS cuDF 22.04.0 jar](https://repo1.maven.org/maven2/ai/rapids/cudf/22.04.0/cudf-22.04.0-cuda11.jar) - -This package is built against CUDA 11.5 and all CUDA 11.x versions are supported through [CUDA forward -compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested -on V100, T4, A2, A10, A30 and A100 GPUs with CUDA 11.0-11.5. For those using other types of GPUs which -do not have CUDA forward compatibility (for example, GeForce), CUDA 11.5 or later is required. Users will -need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. - -### Verify signature -* Download the [RAPIDS Accelerator for Apache Spark 22.04.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.04.0/rapids-4-spark_2.12-22.04.0.jar) - and [RAPIDS Accelerator for Apache Spark 22.04.0 jars.asc](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.04.0/rapids-4-spark_2.12-22.04.0.jar.asc) -* Download the [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com). -* Import the public key: `gpg --import PUB_KEY` -* Verify the signature: `gpg --verify rapids-4-spark_2.12-22.04.0.jar.asc rapids-4-spark_2.12-22.04.0.jar` - -The output if signature verify: - - gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) " - -### Release Notes -New functionality and performance improvements for this release include: -* Avro reader for primitive types -* ExistenceJoin support -* ArrayExists support -* GetArrayStructFields support -* Function str_to_map support -* Function percent_rank support -* Regular expression support for function split on string -* Support function approx_percentile in reduction context -* Support function element_at with non-literal index -* Spark cuSpatial UDF - -For a detailed list of changes, please refer to the -[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). - -## Release v22.02.0 -Hardware Requirements: - -The plugin is tested on the following architectures: - - GPU Models: NVIDIA V100, T4 and A2/A10/A30/A100 GPUs - -Software Requirements: - - OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 - - CUDA & NVIDIA Drivers*: 11.x & v450.80.02+ - - Apache Spark 3.0.1, 3.0.2, 3.0.3, 3.1.1, 3.1.2, 3.2.0, 3.2.1, Databricks 7.3 ML LTS or 9.1 ML LTS Runtime and GCP Dataproc 2.0 - - Python 3.6+, Scala 2.12, Java 8 - -*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet -for your hardware's minimum driver version. - -*For Cloudera and EMR support, please refer to the -[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. - -### Download v22.02.0 -* Download the [RAPIDS - Accelerator for Apache Spark 22.02.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.02.0/rapids-4-spark_2.12-22.02.0.jar) -* Download the [RAPIDS cuDF 22.02.0 jar](https://repo1.maven.org/maven2/ai/rapids/cudf/22.02.0/cudf-22.02.0-cuda11.jar) - -This package is built against CUDA 11.5 and all CUDA 11.x versions are supported through [CUDA forward -compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested -on V100, T4, A2, A10, A30 and A100 GPUs with CUDA 11.0-11.5. For those using other types of GPUs which -do not have CUDA forward compatibility (for example, GeForce), CUDA 11.5 or later is required. Users will -need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. - -### Verify signature -* Download the [RAPIDS Accelerator for Apache Spark 22.02.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.02.0/rapids-4-spark_2.12-22.02.0.jar) - and [RAPIDS Accelerator for Apache Spark 22.02.0 jars.asc](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.02.0/rapids-4-spark_2.12-22.02.0.jar.asc) -* Download the [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com). -* Import the public key: `gpg --import PUB_KEY` -* Verify the signature: `gpg --verify rapids-4-spark_2.12-22.02.0.jar.asc rapids-4-spark_2.12-22.02.0.jar` - -The output if signature verify: - - gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) " - -### Release Notes -New functionality and performance improvements for this release include: -* Parquet reader and writer support for decimal precision up to 38 digits (128-bits) -* Decimal 128-bits casting - * Casting of decimal 128-bits values in nested types - * Casting to String from decimal 128-bits - * Casting from String to decimal 128-bits -* MIG on YARN support -* GPU explain only mode for Spark 3.x and 2.x -* JSON reader support -* Sequence function support -* regexp_extract function support -* Min and max on single-level struct -* CreateMap updates and enable CreateMap by default -* Cast from array to string -* Add regular expression support to regexp_replace function -* Support for conditional joins using libcudf's mixed join feature - -For a detailed list of changes, please refer to the -[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). - -## Release v21.12.0 -Hardware Requirements: - -The plugin is tested on the following architectures: - - GPU Models: NVIDIA V100, T4 and A2/A10/A30/A100 GPUs - -Software Requirements: - - OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 - - CUDA & NVIDIA Drivers*: 11.x & v450.80.02+ - - Apache Spark 3.0.1, 3.0.2, 3.0.3, 3.1.1, 3.1.2, 3.2.0, Databricks 7.3 ML LTS or 9.1 ML LTS Runtime and GCP Dataproc 2.0 - - Python 3.6+, Scala 2.12, Java 8 - -*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet -for your hardware's minimum driver version. - -*For Cloudera and EMR support, please refer to the -[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. - -### Download v21.12.0 -* Download the [RAPIDS - Accelerator for Apache Spark 21.12.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.12.0/rapids-4-spark_2.12-21.12.0.jar) -* Download the [RAPIDS cuDF 21.12.2 jar](https://repo1.maven.org/maven2/ai/rapids/cudf/21.12.2/cudf-21.12.2-cuda11.jar) - -This package is built against CUDA 11.5 and all CUDA 11.x versions are supported through [CUDA forward -compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested -on V100, T4, A2, A10, A30 and A100 GPUs with CUDA 11.0-11.5. For those using other types of GPUs which -do not have CUDA forward compatibility (for example, GeForce), CUDA 11.5 or later is required. Users will -need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. - -### Verify signature -* Download the [RAPIDS Accelerator for Apache Spark 21.12.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.12.0/rapids-4-spark_2.12-21.12.0.jar) - and [RAPIDS Accelerator for Apache Spark 21.12.0 jars.asc](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.12.0/rapids-4-spark_2.12-21.12.0.jar.asc) -* Download the [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com). -* Import the public key: `gpg --import PUB_KEY` -* Verify the signature: `gpg --verify rapids-4-spark_2.12-21.12.0.jar.asc rapids-4-spark_2.12-21.12.0.jar` - -The output if signature verify: - - gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) " - -### Release Notes -New functionality and performance improvements for this release include: -* Support decimal precision up to 38 digits (128-bits) -* Support stddev on double in window context -* Support CPU row-based UDF -* CreateArray outputs array of struct -* collect_set outputs array of struct -* ORC reader and writer support for decimal precision up to 38 digits (128-bits) -* ORC writer supports array, map, and struct -* Support SampleExec, rlike -* regexp_replace supports more patterns such as replacing null -* ParquetCachedBatchSerializer supports map -* Add function explainPotentialGpuPlan to print GPU query plan in a CPU Spark cluster -* Qualification Tool - * Detect RDD APIs and JDBC Scan -* Profiling Tool - * Catch OOM errors and log a hint to increase java heap size - * Print potential problems - -For a detailed list of changes, please refer to the -[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). - -## Release v21.10.0 -Hardware Requirements: - -The plugin is tested on the following architectures: - - GPU Models: NVIDIA V100, T4 and A2/A10/A30/A100 GPUs - -Software Requirements: - - OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 - - CUDA & NVIDIA Drivers*: 11.0-11.4 & v450.80.02+ - - Apache Spark 3.0.1, 3.0.2, 3.0.3, 3.1.1, 3.1.2, 3.2.0, Databricks 7.3 ML LTS or 8.2 ML Runtime, GCP Dataproc 2.0, and Azure Synapse - - Python 3.6+, Scala 2.12, Java 8 - -*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet -for your hardware's minimum driver version. - -*For Cloudera and EMR support, please refer to the -[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. - -### Download v21.10.0 -* Download the [RAPIDS - Accelerator for Apache Spark 21.10.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.10.0/rapids-4-spark_2.12-21.10.0.jar) -* Download the [RAPIDS cuDF 21.10.0 jar](https://repo1.maven.org/maven2/ai/rapids/cudf/21.10.0/cudf-21.10.0-cuda11.jar) - -This package is built against CUDA 11.2 and all CUDA 11.x versions are supported through [CUDA forward -compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested -on V100, T4, A2, A10, A30 and A100 GPUs with CUDA 11.0-11.4. For those using other types of GPUs which -do not have CUDA forward compatibility (for example, GeForce), CUDA 11.2 is required. Users will -need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. - -### Release Notes -New functionality and performance improvements for this release include: -* Support collect_list and collect_set in group-by aggregation -* Support stddev, percentile_approx in group-by aggregation -* RunningWindow operations on map -* HashAggregate on struct and nested struct -* Sorting on nested structs -* Explode on map, array, struct -* Union-all on map, array and struct of maps -* Parquet writing of map -* ORC reader supports reading map/struct columns -* ORC reader support decimal64 -* Qualification Tool - * Add conjunction and disjunction filters - * Filtering specific configuration values - * Filtering user name - * Reporting nested data types - * Reporting write data formats -* Profiling Tool - * Generating structured output format - * Improved profiling tool performance - -For a detailed list of changes, please refer to the -[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). - -## Release v21.08.0 -Hardware Requirements: - -The plugin is tested on the following architectures: - - GPU Models: NVIDIA V100, T4 and A10/A30/A100 GPUs - -Software Requirements: - - OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 - - CUDA & NVIDIA Drivers*: 11.0-11.4 & v450.80.02+ - - Apache Spark 3.0.1, 3.0.2, 3.0.3, 3.1.1, 3.1.2, Databricks 7.3 ML LTS or 8.2 ML Runtime, and GCP Dataproc 2.0 - - Python 3.6+, Scala 2.12, Java 8 - -*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet -for your hardware's minimum driver version. - -*For Cloudera and EMR support, please refer to the -[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. - -### Download v21.08.0 -* Download the [RAPIDS - Accelerator for Apache Spark 21.08.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.08.0/rapids-4-spark_2.12-21.08.0.jar) -* Download the [RAPIDS cuDF 21.08.2 jar](https://repo1.maven.org/maven2/ai/rapids/cudf/21.08.2/cudf-21.08.2-cuda11.jar) - -This package is built against CUDA 11.2 and all CUDA 11.x versions are supported through [CUDA forward -compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested -on V100, T4, A30 and A100 GPUs with CUDA 11.0-11.4. For those using other types of GPUs which -do not have CUDA forward compatibility (for example, GeForce), CUDA 11.2 is required. Users will -need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. - -### Release Notes -New functionality and performance improvements for this release include: -* Handling data sets that spill out of GPU memory for group by and windowing operations -* Running window rank and dense rank operations on the GPU -* Support for the `LEGACY` timestamp -* Unioning of nested structs -* Adoption of UCX 1.11 for improved error handling for RAPIDS Spark Accelerated Shuffle -* Ability to read cached data from the GPU on the supported Databricks runtimes -* Enabling Parquet writing of array data types from the GPU -* Optimized reads for small files for ORC -* Qualification and Profiling Tools - * Additional filtering capabilities - * Reporting on data types - * Reporting on read data formats - * Ability to run the Qualification tool on Spark 2.x logs - * Ability to run the tool on Apache Spark 3.x, AWS EMR 6.3.0, Dataproc 2.0, Microsoft Azure, and - Databricks 7.3 and 8.2 logs - * Improved Qualification tool performance - -For a detailed list of changes, please refer to the -[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). - -## Release v21.06.2 -This is a patch release to address an issue with the plugin in the Databricks 8.2 ML runtime. - -Hardware Requirements: - - GPU Models: NVIDIA V100, T4 or A10/A30/A100 GPUs - -Software Requirements: - - OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 - - CUDA & NVIDIA Drivers*: 11.0 or 11.2 & v450.80.02+ - - Apache Spark 3.0.1, 3.0.2, 3.1.1, 3.1.2, Databricks 7.3 ML LTS or 8.2 ML Runtime, and GCP Dataproc 2.0 - - Python 3.6+, Scala 2.12, Java 8 - -*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet -for your hardware's minimum driver version. - -*For Cloudera and EMR support, please refer to the -[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. - -### Download v21.06.2 -* Download the [RAPIDS - Accelerator for Apache Spark 21.06.2 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.06.2/rapids-4-spark_2.12-21.06.2.jar) -* Download the [RAPIDS cuDF 21.06.1 jar](https://repo1.maven.org/maven2/ai/rapids/cudf/21.06.1/cudf-21.06.1-cuda11.jar) - -This package is built against CUDA 11.2 and all CUDA 11.x versions are supported through [CUDA forward -compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested -on Tesla datacenter GPUs with CUDA 11.0 and 11.2. For those using other types of GPUs which -do not have CUDA forward compatibility (for example, GeForce), CUDA 11.2 is required. Users will -need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. - -### Release Notes -This release patches the plugin to address a backwards incompatible change to Parquet filters made -by Databricks in the Databricks 8.2 ML runtime. More information is in [issue -3191](https://github.com/NVIDIA/spark-rapids/issues/3191) in the RAPIDS Spark repository. See the -[Release v21.06.0](#release-v21060) release notes for more detail about new features in 21.06. - -For a detailed list of changes, please refer to the -[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). - -## Release v21.06.1 -This is a patch release to address an issue with the plugin in the Databricks 7.3 ML LTS runtime. - -Hardware Requirements: - - GPU Models: NVIDIA V100, T4 or A10/A30/A100 GPUs - -Software Requirements: - - OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 - - CUDA & NVIDIA Drivers*: 11.0 or 11.2 & v450.80.02+ - - Apache Spark 3.0.1, 3.0.2, 3.1.1, 3.1.2, and GCP Dataproc 2.0 - - Python 3.6+, Scala 2.12, Java 8 - -*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet -for your hardware's minimum driver version. - -*For Cloudera and EMR support, please refer to the -[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. - -### Download v21.06.1 -* Download the [RAPIDS - Accelerator for Apache Spark 21.06.1 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.06.1/rapids-4-spark_2.12-21.06.1.jar) -* Download the [RAPIDS cuDF 21.06.1 jar](https://repo1.maven.org/maven2/ai/rapids/cudf/21.06.1/cudf-21.06.1-cuda11.jar) - -This package is built against CUDA 11.2 and all CUDA 11.x versions are supported through [CUDA forward -compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested -on Tesla datacenter GPUs with CUDA 11.0 and 11.2. For those using other types of GPUs which -do not have CUDA forward compatibility (for example, GeForce), CUDA 11.2 is required. Users will -need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. - -### Release Notes -This release patches the plugin to address a backwards incompatible change to Parquet filters made -by Databricks in the Databricks 7.3 ML LTS runtime. More information is in [issue -3098](https://github.com/NVIDIA/spark-rapids/issues/3098) in the RAPIDS Spark repository. See the -[Release v21.06.0](#release-v21060) release notes for more detail about new features in 21.06. - -For a detailed list of changes, please refer to the -[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). - -## Release v21.06.0 -Starting with release 21.06.0, the project is moving to calendar versioning, with the first two -digits representing the year, the second two digits representing the month, and the last digit -representing the patch version of the release. - -Hardware Requirements: - -The plugin is tested on the following architectures: - - GPU Models: NVIDIA V100, T4 and A10/A30/A100 GPUs - -Software Requirements: - - OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS 8 - - CUDA & NVIDIA Drivers*: 11.0 or 11.2 & v450.80.02+ - - Apache Spark 3.0.1, 3.0.2, 3.1.1, 3.1.2, Databricks 8.2 ML Runtime, and GCP Dataproc 2.0 - - Python 3.6+, Scala 2.12, Java 8 - -*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet -for your hardware's minimum driver version. - -*For Cloudera and EMR support, please refer to the -[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. - -### Download v21.06.0 -* Download the [RAPIDS - Accelerator for Apache Spark 21.06.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/21.06.0/rapids-4-spark_2.12-21.06.0.jar) -* Download the [RAPIDS cuDF 21.06.1 jar](https://repo1.maven.org/maven2/ai/rapids/cudf/21.06.1/cudf-21.06.1-cuda11.jar) - -This package is built against CUDA 11.2 and all CUDA 11.x versions are supported through [CUDA forward -compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested -on V100, T4, A30 and A100 GPUs with CUDA 11.0 and 11.2. For those using other types of GPUs which -do not have CUDA forward compatibility (for example, GeForce), CUDA 11.2 is required. Users will -need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. - -### Release Notes -New functionality for this release includes: -* Support for running on Cloudera CDP 7.1.6, CDP 7.1.7 and Databricks 8.2 ML -* New functionality related to arrays: - * Concatenation of array columns - * Casting arrays of floats to arrays of doubles - * Creation of 2D array types - * Hash partitioning with arrays - * Explode takes expressions that generate arrays -* New functionality for struct types: - * Sorting on struct keys - * Structs with map values - * Caching of structs -* New windowing functionality: - * Window lead / lag for arrays - * Range windows supporting non-timestamp order by expressions -* Enabling large joins that can spill out of memory -* Support for the `concat_ws` operator -* Qualification and Profiling Tools - * Qualification tool looks at a set of applications to determine if the RAPIDS Accelerator for - Apache Spark is a good fit - * Profiling tool to generate information used for debugging and profiling applications - -Performance improvements for this release include: -* Moving RAPIDS Shuffle out of beta -* Updates to UCX error handling -* GPUDirect Storage for spilling - -For a detailed list of changes, please refer to the -[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). - -## Release v0.5.0 - -For a detailed list of changes, please refer to the -[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). - -Hardware Requirements: - - GPU Architecture: NVIDIA Pascal™ or better (Tested on V100, T4 and A100 GPU) - -Software Requirements: - - OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, CentOS8 - - CUDA & NVIDIA Drivers: 10.1.2 & v418.87+, 10.2 & v440.33+ or 11.0 & v450.36+ - - Apache Spark 3.0.0, 3.0.1, 3.0.2, 3.1.1, Databricks 7.3 ML LTS Runtime, or GCP Dataproc 2.0 - - Python 3.6+, Scala 2.12, Java 8 - -### Download v0.5.0 -* Download the [RAPIDS Accelerator for Apache Spark v0.5.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/0.5.0/rapids-4-spark_2.12-0.5.0.jar) -* Download the RAPIDS cuDF 0.19.2 jar for your system: - * [For CUDA 11.0 & NVIDIA driver 450.36+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.19.2/cudf-0.19.2-cuda11.jar) - * [For CUDA 10.2 & NVIDIA driver 440.33+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.19.2/cudf-0.19.2-cuda10-2.jar) - * [For CUDA 10.1 & NVIDIA driver 418.87+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.19.2/cudf-0.19.2-cuda10-1.jar) - -### Release Notes -New functionality for this release includes: -* Additional support for structs, including casting structs to string, hashing structs, unioning - structs, and allowing array types and structs to pass through when doing joins -* Support for `get_json_object`, `pivot`, `explode` operators -* Casting string to decimal and decimal to string - -Performance improvements for this release include: -* Optimizing unnecessary columnar->row->columnar transitions with AQE -* Supporting out of core sorts -* Initial support for cost based optimization -* Decimal32 support -* Accelerate data transfer for map Pandas UDF -* Allow spilled buffers to be unspilled - - -## Release v0.4.1 -### Download v0.4.1 -* Download the [RAPIDS Accelerator For Apache Spark v0.4.1 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/0.4.1/rapids-4-spark_2.12-0.4.1.jar) -* Download the RAPIDS cuDF 0.18.1 jar for your system: - * [For CUDA 11.0 & NVIDIA driver 450.36+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.18.1/cudf-0.18.1-cuda11.jar) - * [For CUDA 10.2 & NVIDIA driver 440.33+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.18.1/cudf-0.18.1-cuda10-2.jar) - * [For CUDA 10.1 & NVIDIA driver 418.87+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.18.1/cudf-0.18.1-cuda10-1.jar) - -### Requirements -Hardware Requirements: - - GPU Architecture: NVIDIA Pascal™ or better (Tested on V100, T4 and A100 GPU) - -Software Requirements: - - OS: Ubuntu 16.04, Ubuntu 18.04 or CentOS 7 - - CUDA & NVIDIA Drivers: 10.1.2 & v418.87+, 10.2 & v440.33+ or 11.0 & v450.36+ - - Apache Spark 3.0, 3.0.1, 3.0.2, 3.1.1, Databricks 7.3 ML LTS Runtime, or GCP Dataproc 2.0 - - Python 3.6+, Scala 2.12, Java 8 - -### Release Notes -This is a patch release based on version 0.4.0 with the following additional fixes: -* Broadcast exchange can fail when job group is set - -The release is supported on Apache Spark 3.0.0, 3.0.1, 3.0.2, 3.1.1, Databricks 7.3 ML LTS and -Google Cloud Platform Dataproc 2.0. - -The list of all supported operations is provided [here](supported_ops.md). - -For a detailed list of changes, please refer to the -[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). - -**_Note:_** Using NVIDIA driver release 450.80.02, 450.102.04 or 460.32.03 in combination with the -CUDA 10.1 or 10.2 toolkit may result in long read times when reading a file that is snappy -compressed. In those cases we recommend either running with the CUDA 11.0 toolkit or using a newer -driver. This issue is resolved in the 0.5.0 and higher releases. - -## Release v0.4.0 -### Download v0.4.0 -* Download [RAPIDS Accelerator For Apache Spark v0.4.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/0.4.0/rapids-4-spark_2.12-0.4.0.jar) -* Download RAPIDS cuDF 0.18.1 for your system: - * [For CUDA 11.0 & NVIDIA driver 450.36+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.18.1/cudf-0.18.1-cuda11.jar) - * [For CUDA 10.2 & NVIDIA driver 440.33+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.18.1/cudf-0.18.1-cuda10-2.jar) - * [For CUDA 10.1 & NVIDIA driver 418.87+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.18.1/cudf-0.18.1-cuda10-1.jar) - -### Requirements -Hardware Requirements: - - GPU Architecture: NVIDIA Pascal™ or better (Tested on V100, T4 and A100 GPU) - -Software Requirements: - - OS: Ubuntu 16.04, Ubuntu 18.04 or CentOS 7 - - CUDA & NVIDIA Drivers: 10.1.2 & v418.87+, 10.2 & v440.33+ or 11.0 & v450.36+ - - Apache Spark 3.0, 3.0.1, 3.0.2, 3.1.1, Databricks 7.3 ML LTS Runtime, or GCP Dataproc 2.0 - - Python 3.6+, Scala 2.12, Java 8 - -### Release Notes -New functionality for the release includes -* Decimal support up to 64 bit, including reading and writing decimal from Parquet (can be enabled - by setting `spark.rapids.sql.decimalType.enabled` to True) -* Ability for users to provide GPU versions of Scala, Java or Hive UDFs -* Shuffle and sort support for `struct` data types -* `array_contains` for list operations -* `collect_list` and `average` for windowing operations -* Murmur3 `hash` operation -* Improved performance when reading from DataSource v2 when the source produces data in the Arrow format - -This release includes additional performance improvements, including -* RAPIDS Shuffle with UCX performance improvements -* Instructions on how to use [Alluxio caching](get-started/getting-started-alluxio.md) with Spark to - leverage caching. - -The release is supported on Apache Spark 3.0.0, 3.0.1, 3.0.2, 3.1.1, Databricks 7.3 ML LTS and -Google Cloud Platform Dataproc 2.0. - -The list of all supported operations is provided [here](supported_ops.md). - -For a detailed list of changes, please refer to the -[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). - -**_Note:_** Using NVIDIA driver release 450.80.02, 450.102.04 or 460.32.03 in combination with the -CUDA 10.1 or 10.2 toolkit may result in long read times when reading a file that is snappy -compressed. In those cases we recommend either running with the CUDA 11.0 toolkit or using a newer -driver. This issue is resolved in the 0.5.0 and higher releases. - -## Release v0.3.0 -### Download v0.3.0 -* Download [RAPIDS Accelerator For Apache Spark v0.3.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/0.3.0/rapids-4-spark_2.12-0.3.0.jar) -* Download RAPIDS cuDF 0.17 for your system: - * [For CUDA 11.0 & NVIDIA driver 450.36+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.17/cudf-0.17-cuda11.jar) - * [For CUDA 10.2 & NVIDIA driver 440.33+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.17/cudf-0.17-cuda10-2.jar) - * [For CUDA 10.1 & NVIDIA driver 418.87+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.17/cudf-0.17-cuda10-1.jar) - -### Requirements -Hardware Requirements: - - GPU Architecture: NVIDIA Pascal™ or better (Tested on V100, T4 and A100 GPU) - -Software Requirements: - - OS: Ubuntu 16.04, Ubuntu 18.04 or CentOS 7 - - CUDA & NVIDIA Drivers: 10.1.2 & v418.87+, 10.2 & v440.33+ or 11.0 & v450.36+ - - Apache Spark 3.0, 3.0.1, Databricks 7.3 ML LTS Runtime, or GCP Dataproc 2.0 - - Python 3.6+, Scala 2.12, Java 8 - -### Release Notes -This release includes additional performance improvements, including -* Use of per thread default stream to make more efficient use of the GPU -* Further supporting Spark's adaptive query execution, with more rewritten query plans now able to -run on the GPU -* Performance improvements for reading small Parquet files -* RAPIDS Shuffle with UCX updated to UCX 1.9.0 - -New functionality for the release includes -* Parquet reading for lists and structs, -* Lead/lag for windows, and -* Greatest/least operators - -The release is supported on Apache Spark 3.0.0, 3.0.1, Databricks 7.3 ML LTS and Google Cloud -Platform Dataproc 2.0. - -The list of all supported operations is provided [here](supported_ops.md). - -For a detailed list of changes, please refer to the -[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). - -**_Note:_** Using NVIDIA driver release 450.80.02, 450.102.04 or 460.32.03 in combination with the -CUDA 10.1 or 10.2 toolkit may result in long read times when reading a file that is snappy -compressed. In those cases we recommend either running with the CUDA 11.0 toolkit or using a newer -driver. This issue is resolved in the 0.5.0 and higher releases. - -## Release v0.2.0 -### Download v0.2.0 -* Download [RAPIDS Accelerator For Apache Spark v0.2.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/0.2.0/rapids-4-spark_2.12-0.2.0.jar) -* Download RAPIDS cuDF 0.15 for your system: - * [For CUDA 11.0 & NVIDIA driver 450.36+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.15/cudf-0.15-cuda11.jar) - * [For CUDA 10.2 & NVIDIA driver 440.33+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.15/cudf-0.15-cuda10-2.jar) - * [For CUDA 10.1 & NVIDIA driver 418.87+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.15/cudf-0.15-cuda10-1.jar) - -### Requirements -Hardware Requirements: - - GPU Architecture: NVIDIA Pascal™ or better (Tested on V100, T4 and A100 GPU) - -Software Requirements: - - OS: Ubuntu 16.04, Ubuntu 18.04 or CentOS 7 - - CUDA & NVIDIA Drivers: 10.1.2 & v418.87+, 10.2 & v440.33+ or 11.0 & v450.36+ - - Apache Spark 3.0, 3.0.1 - - Python 3.x, Scala 2.12, Java 8 - -### Release Notes -This is the second release of the RAPIDS Accelerator for Apache Spark. Adaptive Query Execution -[SPARK-31412](https://issues.apache.org/jira/browse/SPARK-31412) is a new enhancement that was -included in Spark 3.0 that alters the physical execution plan dynamically to improve the performance -of the query. The RAPIDS Accelerator v0.2 introduces Adaptive Query Execution (AQE) for GPUs and -leverages columnar processing [SPARK-32332](https://issues.apache.org/jira/browse/SPARK-32332) -starting from Spark 3.0.1. - -Another enhancement in v0.2 is improvement in reading small Parquet files. This feature takes into -account the scenario where input data can be stored across many small files. By leveraging multiple -CPU threads v0.2 delivers up to 6x performance improvement over the previous release for small -Parquet file reads. - -The RAPIDS Accelerator introduces a beta feature that accelerates -[Spark shuffle for GPUs](get-started/getting-started-on-prem.md#enabling-rapids-shuffle-manager). Accelerated -shuffle makes use of high bandwidth transfers between GPUs (NVLink or p2p over PCIe) and leverages -RDMA (RoCE or Infiniband) for remote transfers. - -The list of all supported operations is provided -[here](configs.md#supported-gpu-operators-and-fine-tuning). - -For a detailed list of changes, please refer to the -[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). - -**_Note:_** Using NVIDIA driver release 450.80.02, 450.102.04 or 460.32.03 in combination with the -CUDA 10.1 or 10.2 toolkit may result in long read times when reading a file that is snappy -compressed. In those cases we recommend either running with the CUDA 11.0 toolkit or using a newer -driver. This issue is resolved in the 0.5.0 and higher releases. - -## Release v0.1.0 -### Download v0.1.0 -* Download [RAPIDS Accelerator For Apache Spark v0.1.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/0.1.0/rapids-4-spark_2.12-0.1.0.jar) -* Download RAPIDS cuDF 0.14 for your system: - * [For CUDA 10.2 & NVIDIA driver 440.33+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.14/cudf-0.14-cuda10-2.jar) - * [For CUDA 10.1 & NVIDIA driver 418.87+](https://repo1.maven.org/maven2/ai/rapids/cudf/0.14/cudf-0.14-cuda10-1.jar) - -### Requirements -Hardware Requirements: - - GPU Architecture: NVIDIA Pascal™ or better (Tested on V100 and T4 GPU) - -Software Requirements: - - OS: Ubuntu 16.04, Ubuntu 18.04 or CentOS 7 - (RHEL 7 support is provided through CentOS 7 builds/installs) - - CUDA & NVIDIA Drivers: 10.1.2 & v418.87+ or 10.2 & v440.33+ - - Apache Spark 3.0 +## Archived releases - Python 3.x, Scala 2.12, Java 8 +As new releases come out, previous ones will still be available in [archived releases](./archive.md). \ No newline at end of file From 7c37e56bda64f73fc113e774a166e4cefea2495d Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 22 Sep 2022 15:03:09 -0500 Subject: [PATCH 137/190] Add in support for casting binary to string (#6599) Signed-off-by: Robert (Bobby) Evans --- docs/supported_ops.md | 4 ++-- .../src/main/python/cast_test.py | 3 +++ .../com/nvidia/spark/rapids/GpuCast.scala | 21 ++++++++++++++++++- .../com/nvidia/spark/rapids/TypeChecks.scala | 2 +- 4 files changed, 26 insertions(+), 4 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index d2b8603d8ad..bcc2036cdc8 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -17973,7 +17973,7 @@ and the accelerator produces the same result. -NS +S S @@ -18377,7 +18377,7 @@ and the accelerator produces the same result. -NS +S S diff --git a/integration_tests/src/main/python/cast_test.py b/integration_tests/src/main/python/cast_test.py index e5de7acac9f..674f8f6adad 100644 --- a/integration_tests/src/main/python/cast_test.py +++ b/integration_tests/src/main/python/cast_test.py @@ -632,3 +632,6 @@ def getDf(spark): # 106751991 > Byte.MaxValue return spark.createDataFrame([(True, MAX_DAY_TIME_INTERVAL), (False, (timedelta(microseconds=0)))], "c_b boolean, c_dt interval day to second").repartition(1) assert_gpu_and_cpu_are_equal_collect(lambda spark: getDf(spark).selectExpr("if(c_b, 0, cast(c_dt as byte))", "if(c_b, 0, cast(c_dt as short))", "if(c_b, 0, cast(c_dt as int))")) + +def test_cast_binary_to_string(): + assert_gpu_and_cpu_are_equal_collect(lambda spark: unary_op_df(spark, binary_gen).selectExpr("a", "CAST(a AS STRING) as str")) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala index f7b16c8ac0f..8e35b157eab 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala @@ -18,6 +18,7 @@ package com.nvidia.spark.rapids import java.text.SimpleDateFormat import java.time.DateTimeException +import java.util.Optional import scala.collection.mutable.ArrayBuffer @@ -481,6 +482,9 @@ object GpuCast extends Arm { case (ShortType | IntegerType | LongType | ByteType | StringType, BinaryType) => input.asByteList(true) + case (BinaryType, StringType) => + castBinToString(input) + case (_: DecimalType, StringType) => input.castTo(DType.STRING) @@ -1370,6 +1374,21 @@ object GpuCast extends Arm { } } + private def castBinToString(input: ColumnView): ColumnVector = { + // Spark interprets the binary as UTF-8 bytes. So the layout of the + // binary and the layout of the string are the same. We just need to play some games with + // the CPU side metadata to make CUDF think it is a String. + // Sadly there is no simple CUDF API to do this, so for now we pull it apart and put + // it back together again + withResource(input.getChildColumnView(0)) { dataCol => + withResource(new ColumnView(DType.STRING, input.getRowCount, + Optional.of[java.lang.Long](input.getNullCount), + dataCol.getData, input.getValid, input.getOffsets)) { cv => + cv.copyToColumnVector() + } + } + } + private def castIntegralsToDecimal( input: ColumnView, dt: DecimalType, @@ -1647,4 +1666,4 @@ case class GpuCast( doCast(input.getBase, input.dataType(), dataType, ansiMode, legacyCastToString, stringToDateAnsiModeEnabled) -} +} \ No newline at end of file diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index 512cba00d11..7911eef9cee 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -1367,7 +1367,7 @@ class CastChecks extends ExprChecks { val sparkStringSig: TypeSig = cpuNumeric + BOOLEAN + TIMESTAMP + DATE + CALENDAR + STRING + BINARY + GpuTypeShims.additionalTypesStringCanCastTo - val binaryChecks: TypeSig = BINARY + val binaryChecks: TypeSig = STRING + BINARY val sparkBinarySig: TypeSig = STRING + BINARY val decimalChecks: TypeSig = gpuNumeric + STRING From 3f377ef6105be74f5ee578bc06bf38289e0af44d Mon Sep 17 00:00:00 2001 From: Peixin Date: Fri, 23 Sep 2022 11:22:20 +0800 Subject: [PATCH 138/190] Fix version-def script to correctly set list of shims (#6602) Signed-off-by: Peixin Li Signed-off-by: Peixin Li --- jenkins/version-def.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/jenkins/version-def.sh b/jenkins/version-def.sh index 031c0ebd446..7f0c1767ca7 100755 --- a/jenkins/version-def.sh +++ b/jenkins/version-def.sh @@ -77,11 +77,11 @@ SPARK_SHIM_VERSIONS_NOSNAPSHOTS=("${SPARK_SHIM_VERSIONS_ARR[@]}") # Spark shim versions list based on given profile option (snapshots or noSnapshots) case $PHASE_TYPE in pre-release) - SPARK_SHIM_VERSIONS=("${SPARK_SHIM_VERSIONS_SNAPSHOTS[@]}") + SPARK_SHIM_VERSIONS=("${SPARK_SHIM_VERSIONS_NOSNAPSHOTS[@]}") ;; *) - SPARK_SHIM_VERSIONS=("${SPARK_SHIM_VERSIONS_NOSNAPSHOTS[@]}") + SPARK_SHIM_VERSIONS=("${SPARK_SHIM_VERSIONS_SNAPSHOTS[@]}") ;; esac # base version @@ -100,4 +100,4 @@ SPARK_SHIM_VERSIONS_PREMERGE_UT_2=("${SPARK_SHIM_VERSIONS_ARR[@]}") set_env_var_SPARK_SHIM_VERSIONS_ARR -PpremergeUTF8 SPARK_SHIM_VERSIONS_PREMERGE_UTF8=("${SPARK_SHIM_VERSIONS_ARR[@]}") -echo "SPARK_BASE_SHIM_VERSION: $SPARK_BASE_SHIM_VERSION" \ No newline at end of file +echo "SPARK_BASE_SHIM_VERSION: $SPARK_BASE_SHIM_VERSION" From 26a6a96e225ba112a6431ac4f46afc386290781d Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Sat, 24 Sep 2022 11:24:04 +0800 Subject: [PATCH 139/190] Add dynamic partition concurrent writer to avoid the full sort [databricks] (#6569) * Add dynamic partition concurrent writer to avoid the full sort Signed-off-by: Chong Gao --- docs/configs.md | 1 + integration_tests/src/main/python/data_gen.py | 11 + integration_tests/src/main/python/orc_test.py | 2 +- .../src/main/python/orc_write_test.py | 35 +- .../src/main/python/parquet_write_test.py | 58 +- .../nvidia/spark/rapids/GpuOverrides.scala | 6 +- .../com/nvidia/spark/rapids/GpuSortExec.scala | 16 +- .../com/nvidia/spark/rapids/RapidsConf.scala | 19 +- .../spark/rapids/basicPhysicalOperators.scala | 8 +- ...CreateDataSourceTableAsSelectCommand.scala | 6 +- .../spark/sql/rapids/GpuDataSource.scala | 13 +- .../sql/rapids/GpuFileFormatDataWriter.scala | 645 ++++++++++++++++-- .../sql/rapids/GpuFileFormatWriter.scala | 73 +- ...GpuInsertIntoHadoopFsRelationCommand.scala | 6 +- 14 files changed, 816 insertions(+), 83 deletions(-) diff --git a/docs/configs.md b/docs/configs.md index 9108f125919..fb38715ecdc 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -76,6 +76,7 @@ Name | Description | Default Value spark.rapids.sql.castStringToTimestamp.enabled|When set to true, casting from string to timestamp is supported on the GPU. The GPU only supports a subset of formats when casting strings to timestamps. Refer to the CAST documentation for more details.|false spark.rapids.sql.coalescing.reader.numFilterParallel|This controls the number of files the coalescing reader will run in each thread when it filters blocks for reading. If this value is greater than zero the files will be filtered in a multithreaded manner where each thread filters the number of files set by this config. If this is set to zero the files are filtered serially. This uses the same thread pool as the multithreaded reader, see spark.rapids.sql.multiThreadedRead.numThreads. Note that filtering multithreaded is useful with Alluxio.|0 spark.rapids.sql.concurrentGpuTasks|Set the number of tasks that can execute concurrently per GPU. Tasks may temporarily block when the number of concurrent tasks in the executor exceeds this amount. Allowing too many concurrent tasks on the same GPU may lead to GPU out of memory errors.|1 +spark.rapids.sql.concurrentWriterPartitionFlushSize|The flush size of the concurrent writer cache in bytes for each partition. If specified spark.sql.maxConcurrentOutputFileWriters, use concurrent writer to write data. Concurrent writer first caches data for each partition and begins to flush the data if it finds one partition with a size that is greater than or equal to this config. The default value is 0, which will try to select a size based off of file type specific configs. E.g.: It uses `write.parquet.row-group-size-bytes` config for Parquet type and `orc.stripe.size` config for Orc type. If the value is greater than 0, will use this positive value.Max value may get better performance but not always, because concurrent writer uses spillable cache and big value may cause more IO swaps.|0 spark.rapids.sql.csv.read.decimal.enabled|CSV reading is not 100% compatible when reading decimals.|false spark.rapids.sql.csv.read.double.enabled|CSV reading is not 100% compatible when reading doubles.|true spark.rapids.sql.csv.read.float.enabled|CSV reading is not 100% compatible when reading floats.|true diff --git a/integration_tests/src/main/python/data_gen.py b/integration_tests/src/main/python/data_gen.py index 8e423532d7a..72391e6c1f4 100644 --- a/integration_tests/src/main/python/data_gen.py +++ b/integration_tests/src/main/python/data_gen.py @@ -1078,3 +1078,14 @@ def append_unique_to_rows(rows): disable_parquet_field_id_write = {"spark.sql.parquet.fieldId.write.enabled": "false"} # default is true enable_parquet_field_id_write = {"spark.sql.parquet.fieldId.write.enabled": "true"} enable_parquet_field_id_read = {"spark.sql.parquet.fieldId.read.enabled": "true"} # default is false + + +# generate a df with c1 and c2 column have 25 combinations +def get_25_partitions_df(spark): + schema = StructType([ + StructField("c1", IntegerType()), + StructField("c2", IntegerType()), + StructField("c3", IntegerType())]) + data = [[i, j, k] for i in range(0, 5) for j in range(0, 5) for k in range(0, 100)] + return spark.createDataFrame(data, schema) + diff --git a/integration_tests/src/main/python/orc_test.py b/integration_tests/src/main/python/orc_test.py index 1c7d63fba1b..528a76fa7ec 100644 --- a/integration_tests/src/main/python/orc_test.py +++ b/integration_tests/src/main/python/orc_test.py @@ -15,7 +15,7 @@ import pytest from asserts import assert_cpu_and_gpu_are_equal_sql_with_capture, assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_row_counts_equal, assert_gpu_fallback_collect, \ - assert_cpu_and_gpu_are_equal_collect_with_capture + assert_cpu_and_gpu_are_equal_collect_with_capture, assert_gpu_and_cpu_writes_are_equal_collect from data_gen import * from marks import * from pyspark.sql.types import * diff --git a/integration_tests/src/main/python/orc_write_test.py b/integration_tests/src/main/python/orc_write_test.py index 7861f5c58ee..875279ffd38 100644 --- a/integration_tests/src/main/python/orc_write_test.py +++ b/integration_tests/src/main/python/orc_write_test.py @@ -15,7 +15,7 @@ import pytest from asserts import assert_gpu_and_cpu_writes_are_equal_collect, assert_gpu_fallback_write -from spark_session import is_databricks104_or_later +from spark_session import is_before_spark_320 from datetime import date, datetime, timezone from data_gen import * from marks import * @@ -169,3 +169,36 @@ def create_empty_df(spark, path): lambda spark, path: spark.read.orc(path), data_path, conf={'spark.rapids.sql.format.orc.write.enabled': True}) + + +@ignore_order +@pytest.mark.skipif(is_before_spark_320(), reason="is only supported in Spark 320+") +def test_concurrent_writer(spark_tmp_path): + data_path = spark_tmp_path + '/PARQUET_DATA' + assert_gpu_and_cpu_writes_are_equal_collect( + lambda spark, path: get_25_partitions_df(spark) # df has 25 partitions for (c1, c2) + .repartition(2) + .write.mode("overwrite").partitionBy('c1', 'c2').orc(path), + lambda spark, path: spark.read.orc(path), + data_path, + copy_and_update( + # 26 > 25, will not fall back to single writer + {"spark.sql.maxConcurrentOutputFileWriters": 26} + )) + + +@ignore_order +@pytest.mark.skipif(is_before_spark_320(), reason="is only supported in Spark 320+") +def test_fallback_to_single_writer_from_concurrent_writer(spark_tmp_path): + data_path = spark_tmp_path + '/PARQUET_DATA' + assert_gpu_and_cpu_writes_are_equal_collect( + lambda spark, path: get_25_partitions_df(spark) # df has 25 partitions for (c1, c2) + .repartition(2) + .write.mode("overwrite").partitionBy('c1', 'c2').orc(path), + lambda spark, path: spark.read.orc(path), + data_path, + copy_and_update( + # 10 < 25, will fall back to single writer + {"spark.sql.maxConcurrentOutputFileWriters": 10}, + {"spark.rapids.sql.concurrentWriterPartitionFlushSize": 64 * 1024 * 1024} + )) diff --git a/integration_tests/src/main/python/parquet_write_test.py b/integration_tests/src/main/python/parquet_write_test.py index 84326957a99..706143b220b 100644 --- a/integration_tests/src/main/python/parquet_write_test.py +++ b/integration_tests/src/main/python/parquet_write_test.py @@ -19,7 +19,7 @@ from data_gen import * from marks import * from pyspark.sql.types import * -from spark_session import with_cpu_session, with_gpu_session, is_before_spark_330 +from spark_session import with_cpu_session, with_gpu_session, is_before_spark_330, is_before_spark_320 import pyspark.sql.functions as f import pyspark.sql.utils import random @@ -504,3 +504,59 @@ def test_write_daytime_interval(spark_tmp_path): lambda spark, path: spark.read.parquet(path), data_path, conf=writer_confs) + +@ignore_order +@pytest.mark.skipif(is_before_spark_320(), reason="is only supported in Spark 320+") +def test_concurrent_writer(spark_tmp_path): + data_path = spark_tmp_path + '/PARQUET_DATA' + assert_gpu_and_cpu_writes_are_equal_collect( + lambda spark, path: get_25_partitions_df(spark) # df has 25 partitions for (c1, c2) + .repartition(2) + .write.mode("overwrite").partitionBy('c1', 'c2').parquet(path), + lambda spark, path: spark.read.parquet(path), + data_path, + copy_and_update( + # 26 > 25, will not fall back to single writer + {"spark.sql.maxConcurrentOutputFileWriters": 26} + )) + + +@ignore_order +@pytest.mark.skipif(is_before_spark_320(), reason="is only supported in Spark 320+") +def test_fallback_to_single_writer_from_concurrent_writer(spark_tmp_path): + data_path = spark_tmp_path + '/PARQUET_DATA' + assert_gpu_and_cpu_writes_are_equal_collect( + lambda spark, path: get_25_partitions_df(spark) # df has 25 partitions for (c1, c2) + .repartition(2) + .write.mode("overwrite").partitionBy('c1', 'c2').parquet(path), + lambda spark, path: spark.read.parquet(path), + data_path, + copy_and_update( + # 10 < 25, will fall back to single writer + {"spark.sql.maxConcurrentOutputFileWriters": 10}, + {"spark.rapids.sql.concurrentWriterPartitionFlushSize": 64 * 1024 * 1024} + )) + + +@pytest.mark.skipif(True, reason="currently not support write emtpy data: https://github.com/NVIDIA/spark-rapids/issues/6453") +def test_write_empty_data_concurrent_writer(spark_tmp_path): + schema = StructType( + [StructField("c1", StringType()), StructField("c2", IntegerType()), StructField("c3", IntegerType())]) + data = [] # empty data + data_path = spark_tmp_path + '/PARQUET_DATA' + with_gpu_session(lambda spark: spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + .write.mode("overwrite").partitionBy('c1', 'c2').parquet(data_path), + # concurrent writer + {"spark.sql.maxConcurrentOutputFileWriters": 10}) + with_cpu_session(lambda spark: spark.read.parquet(data_path).collect()) + + +@pytest.mark.skipif(True, reason="currently not support write emtpy data: https://github.com/NVIDIA/spark-rapids/issues/6453") +def test_write_empty_data_single_writer(spark_tmp_path): + schema = StructType( + [StructField("c1", StringType()), StructField("c2", IntegerType()), StructField("c3", IntegerType())]) + data = [] # empty data + data_path = spark_tmp_path + '/PARQUET_DATA' + with_gpu_session(lambda spark: spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + .write.mode("overwrite").partitionBy('c1', 'c2').parquet(data_path)) + with_cpu_session(lambda spark: spark.read.parquet(data_path).collect()) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index c7620eeddf0..2c7d48ad685 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -360,7 +360,8 @@ final class InsertIntoHadoopFsRelationCommandMeta( cmd.catalogTable, cmd.fileIndex, cmd.outputColumnNames, - conf.stableSort) + conf.stableSort, + conf.concurrentWriterPartitionFlushSize) } } @@ -410,7 +411,8 @@ final class CreateDataSourceTableAsSelectCommandMeta( cmd.outputColumnNames, origProvider, newProvider, - conf.stableSort) + conf.stableSort, + conf.concurrentWriterPartitionFlushSize) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortExec.scala index f8352debad6..6f7b5a9c5b7 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortExec.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder, UnsafePr import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering import org.apache.spark.sql.catalyst.plans.physical.{Distribution, OrderedDistribution, Partitioning, UnspecifiedDistribution} import org.apache.spark.sql.execution.{SortExec, SparkPlan} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.vectorized.ColumnarBatch @@ -66,6 +67,15 @@ class GpuSortMeta( } } +object GpuSortExec { + def targetSize(sqlConf: SQLConf): Long = { + // To avoid divide by zero errors, underflow and overflow issues in tests + // that want the targetSize to be 0, we set it to something more reasonable + val targetSize = RapidsConf.GPU_BATCH_SIZE_BYTES.get(sqlConf) + math.max(16 * 1024, targetSize) + } +} + case class GpuSortExec( gpuSortOrder: Seq[SortOrder], global: Boolean, @@ -113,7 +123,7 @@ case class GpuSortExec( } } - private [this] lazy val targetSize = RapidsConf.GPU_BATCH_SIZE_BYTES.get(conf) + private lazy val targetSize = GpuSortExec.targetSize(conf) override def doExecuteColumnar(): RDD[ColumnarBatch] = { val sorter = new GpuSorter(gpuSortOrder, output) @@ -130,9 +140,7 @@ case class GpuSortExec( val cpuOrd = new LazilyGeneratedOrdering(sorter.cpuOrdering) val spillCallback = GpuMetric.makeSpillCallback(allMetrics) val iter = GpuOutOfCoreSortIterator(cbIter, sorter, cpuOrd, - // To avoid divide by zero errors, underflow and overflow issues in tests - // that want the targetSize to be 0, we set it to something more reasonable - math.max(16 * 1024, targetSize), opTime, sortTime, outputBatch, outputRows, + targetSize, opTime, sortTime, outputBatch, outputRows, peakDevMemory, spillCallback) TaskContext.get().addTaskCompletionListener(_ -> iter.close()) iter diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 4c732f51be2..811aa8dd2c6 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -1587,6 +1587,21 @@ object RapidsConf { .integerConf .createWithDefault(value = 0) + val CONCURRENT_WRITER_PARTITION_FLUSH_SIZE = + conf("spark.rapids.sql.concurrentWriterPartitionFlushSize") + .doc("The flush size of the concurrent writer cache in bytes for each partition. " + + "If specified spark.sql.maxConcurrentOutputFileWriters, use concurrent writer to " + + "write data. Concurrent writer first caches data for each partition and begins to " + + "flush the data if it finds one partition with a size that is greater than or equal " + + "to this config. The default value is 0, which will try to select a size based off " + + "of file type specific configs. E.g.: It uses `write.parquet.row-group-size-bytes` " + + "config for Parquet type and `orc.stripe.size` config for Orc type. " + + "If the value is greater than 0, will use this positive value." + + "Max value may get better performance but not always, because concurrent writer uses " + + "spillable cache and big value may cause more IO swaps.") + .bytesConf(ByteUnit.BYTE) + .createWithDefault(0L) + private def printSectionHeader(category: String): Unit = println(s"\n### $category") @@ -2117,7 +2132,7 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isRegExpEnabled: Boolean = get(ENABLE_REGEXP) lazy val maxRegExpStateMemory: Long = { - val size = get(REGEXP_MAX_STATE_MEMORY_BYTES) + val size = get(REGEXP_MAX_STATE_MEMORY_BYTES) if (size > 3 * gpuTargetBatchSizeBytes) { logWarning(s"${REGEXP_MAX_STATE_MEMORY_BYTES.key} is more than 3 times " + s"${GPU_BATCH_SIZE_BYTES.key}. This may cause regular expression operations to " + @@ -2134,6 +2149,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isDetectDeltaLogQueries: Boolean = get(DETECT_DELTA_LOG_QUERIES) + lazy val concurrentWriterPartitionFlushSize:Long = get(CONCURRENT_WRITER_PARTITION_FLUSH_SIZE) + private val optimizerDefaults = Map( // this is not accurate because CPU projections do have a cost due to appending values // to each row that is produced, but this needs to be a really small number because diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala index b3cbdcea024..07cbecc5761 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala @@ -84,7 +84,13 @@ object GpuProjectExec extends Arm { case _ => None } - private def extractSingleBoundIndex(boundExprs: Seq[Expression]): Seq[Option[Int]] = + private def isAllSingleBoundIndex(boundExprs: Seq[Expression]): Boolean = + extractSingleBoundIndex(boundExprs).forall { + case Some(index) => true + case _ => false + } + + def extractSingleBoundIndex(boundExprs: Seq[Expression]): Seq[Option[Int]] = boundExprs.map(extractSingleBoundIndex) def isNoopProject(cb: ColumnarBatch, boundExprs: Seq[Expression]): Boolean = { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuCreateDataSourceTableAsSelectCommand.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuCreateDataSourceTableAsSelectCommand.scala index 2c80e3f1e7c..ba19df07ede 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuCreateDataSourceTableAsSelectCommand.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuCreateDataSourceTableAsSelectCommand.scala @@ -37,7 +37,8 @@ case class GpuCreateDataSourceTableAsSelectCommand( outputColumnNames: Seq[String], origProvider: Class[_], gpuFileFormat: ColumnarFileFormat, - useStableSort: Boolean) + useStableSort: Boolean, + concurrentWriterPartitionFlushSize: Long) extends GpuDataWritingCommand { override def runColumnar(sparkSession: SparkSession, child: SparkPlan): Seq[ColumnarBatch] = { @@ -119,7 +120,8 @@ case class GpuCreateDataSourceTableAsSelectCommand( origProvider = origProvider, gpuFileFormat = gpuFileFormat) try { - dataSource.writeAndRead(mode, query, outputColumnNames, physicalPlan, useStableSort) + dataSource.writeAndRead(mode, query, outputColumnNames, physicalPlan, useStableSort, + concurrentWriterPartitionFlushSize) } catch { case ex: AnalysisException => logError(s"Failed to write to table ${table.identifier.unquotedString}", ex) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSource.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSource.scala index 66b18bd84db..fe24c083d5b 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSource.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSource.scala @@ -312,7 +312,8 @@ case class GpuDataSource( private def planForWritingFileFormat( format: ColumnarFileFormat, mode: SaveMode, - data: LogicalPlan, useStableSort: Boolean): GpuInsertIntoHadoopFsRelationCommand = { + data: LogicalPlan, useStableSort: Boolean, + concurrentWriterPartitionFlushSize: Long): GpuInsertIntoHadoopFsRelationCommand = { // Don't glob path for the write path. The contracts here are: // 1. Only one output path can be specified on the write path; // 2. Output path must be a legal HDFS style file system path; @@ -351,7 +352,8 @@ case class GpuDataSource( catalogTable = catalogTable, fileIndex = fileIndex, outputColumnNames = data.output.map(_.name), - useStableSort) + useStableSort = useStableSort, + concurrentWriterPartitionFlushSize = concurrentWriterPartitionFlushSize) } /** @@ -373,14 +375,17 @@ case class GpuDataSource( mode: SaveMode, data: LogicalPlan, outputColumnNames: Seq[String], - physicalPlan: SparkPlan, useStableSort: Boolean): BaseRelation = { + physicalPlan: SparkPlan, + useStableSort: Boolean, + concurrentWriterPartitionFlushSize: Long): BaseRelation = { val outputColumns = DataWritingCommand.logicalPlanOutputWithNames(data, outputColumnNames) if (outputColumns.map(_.dataType).exists(_.isInstanceOf[CalendarIntervalType])) { throw new AnalysisException("Cannot save interval data type into external storage.") } // Only currently support ColumnarFileFormat - val cmd = planForWritingFileFormat(gpuFileFormat, mode, data, useStableSort) + val cmd = planForWritingFileFormat(gpuFileFormat, mode, data, useStableSort, + concurrentWriterPartitionFlushSize) val resolvedPartCols = cmd.partitionColumns.map { col => // The partition columns created in `planForWritingFileFormat` should always be // `UnresolvedAttribute` with a single name part. diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala index 919a8454aec..c08a81311f2 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala @@ -17,21 +17,27 @@ package org.apache.spark.sql.rapids import scala.collection.mutable +import scala.collection.mutable.{ArrayBuffer, ListBuffer} -import ai.rapids.cudf.{ContiguousTable, OrderByArg, Table} +import ai.rapids.cudf.{ColumnVector, ContiguousTable, OrderByArg, Table} import com.nvidia.spark.TimingUtils import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ +import com.nvidia.spark.rapids.StorageTier.StorageTier import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils -import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeSet, Cast, Concat, Expression, Literal, NullsFirst, ScalaUDF, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeSet, Cast, Concat, Expression, Literal, NullsFirst, ScalaUDF, SortOrder, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering import org.apache.spark.sql.connector.write.DataWriter -import org.apache.spark.sql.execution.datasources.{ExecutedWriteSummary, PartitioningUtils, WriteTaskResult} +import org.apache.spark.sql.execution.datasources.{BucketingUtils, ExecutedWriteSummary, PartitioningUtils, WriteTaskResult} +import org.apache.spark.sql.rapids.GpuFileFormatWriter.GpuConcurrentOutputWriterSpec import org.apache.spark.sql.types.{DataType, StringType} import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration @@ -75,6 +81,13 @@ abstract class GpuFileFormatDataWriter( releaseCurrentWriter() } + /** Write an iterator of column batch. */ + def writeWithIterator(iterator: Iterator[ColumnarBatch]): Unit = { + while (iterator.hasNext) { + write(iterator.next()) + } + } + /** Writes a columnar batch of records */ def write(batch: ColumnarBatch): Unit @@ -176,20 +189,58 @@ class GpuSingleDirectoryDataWriter( } /** - * Writes data to using dynamic partition writes, meaning this single function can write to - * multiple directories (partitions) or files (bucketing). + * Dynamic partition writer with single writer, meaning only one writer is opened at any time + * for writing, meaning this single function can write to multiple directories (partitions) + * or files (bucketing). The data to be written are required to be sorted on partition and/or + * bucket column(s) before writing. */ -class GpuDynamicPartitionDataWriter( +class GpuDynamicPartitionDataSingleWriter( description: GpuWriteJobDescription, taskAttemptContext: TaskAttemptContext, committer: FileCommitProtocol) - extends GpuFileFormatDataWriter(description, taskAttemptContext, committer) { + extends GpuFileFormatDataWriter(description, taskAttemptContext, committer) with Arm { + + /** Wrapper class for status of a unique single output writer. */ + protected class WriterStatus( + // output writer + var outputWriter: ColumnarOutputWriter, + + /** Number of records in current file. */ + var recordsInFile: Long = 0, + + /** + * File counter for writing current partition or bucket. For same partition or bucket, + * we may have more than one file, due to number of records limit per file. + */ + var fileCounter: Int = 0 + ) + + /** Wrapper class for status and caches of a unique concurrent output writer. + * Used by `GpuDynamicPartitionDataConcurrentWriter` + */ + class WriterStatusWithCaches( + // writer status + var writerStatus: WriterStatus, + + // caches for this partition or writer + val tableCaches: ListBuffer[SpillableColumnarBatch] = ListBuffer(), + + // current device bytes for the above caches + var deviceBytes: Long = 0 + ) /** Flag saying whether or not the data to be written out is partitioned. */ - private val isPartitioned = description.partitionColumns.nonEmpty + protected val isPartitioned: Boolean = description.partitionColumns.nonEmpty /** Flag saying whether or not the data to be written out is bucketed. */ - private val isBucketed = description.bucketSpec.isDefined + protected val isBucketed: Boolean = description.bucketSpec.isDefined + + protected var currentPartPath: String = "" + + protected var currentWriterStatus: WriterStatus = _ + + // All data is sorted ascending with default null ordering + private val nullsSmallest = Ascending.defaultNullOrdering == NullsFirst if (isBucketed) { throw new UnsupportedOperationException("Bucketing is not supported on the GPU yet.") @@ -201,12 +252,8 @@ class GpuDynamicPartitionDataWriter( |GpuWriteJobDescription: $description """.stripMargin) - private var fileCounter: Int = _ - private var recordsInFile: Long = _ - private var currentPartPath: String = "" - /** Extracts the partition values out of an input batch. */ - private lazy val getPartitionColumns: ColumnarBatch => Table = { + protected lazy val getPartitionColumns: ColumnarBatch => Table = { val expressions = GpuBindReferences.bindGpuReferences( description.partitionColumns, description.allColumns) @@ -221,7 +268,7 @@ class GpuDynamicPartitionDataWriter( } /** Extracts the output values of an input batch. */ - private lazy val getOutputColumns: ColumnarBatch => Table = { + protected lazy val getOutputColumns: ColumnarBatch => Table = { val expressions = GpuBindReferences.bindGpuReferences( description.dataColumns, description.allColumns) @@ -235,6 +282,16 @@ class GpuDynamicPartitionDataWriter( } } + /** Extracts the output values of an input batch. */ + protected lazy val getOutputCb: ColumnarBatch => ColumnarBatch = { + val expressions = GpuBindReferences.bindGpuReferences( + description.dataColumns, + description.allColumns) + cb => { + GpuProjectExec.project(cb, expressions) + } + } + /** * Expression that given partition columns builds a path string like: col1=val/col2=val/... * This is used after we pull the unique partition values back to the host. @@ -251,44 +308,63 @@ class GpuDynamicPartitionDataWriter( /** Evaluates the `partitionPathExpression` above on a row of `partitionValues` and returns * the partition string. */ - private lazy val getPartitionPath: InternalRow => String = { + protected lazy val getPartitionPath: InternalRow => String = { val proj = UnsafeProjection.create(Seq(partitionPathExpression), description.partitionColumns) row => proj(row).getString(0) } + /** Release resources of writer. */ + private [rapids] def releaseWriter(writer: ColumnarOutputWriter): Unit = { + if (writer != null) { + val path = writer.path() + writer.close() + statsTrackers.foreach(_.closeFile(path)) + } + } + + /** + * Opens a new OutputWriter given a partition key and/or a bucket id. + * If bucket id is specified, we will append it to the end of the file name, but before the + * file extension, e.g. part-r-00009-ea518ad4-455a-4431-b471-d24e03814677-00002.gz.parquet + * + * @param partDir the partition directory + * @param bucketId the bucket which all tuples being written by this OutputWriter belong to, + * currently does not support `bucketId`, it's always None + */ @scala.annotation.nowarn( "msg=method newTaskTempFile.* in class FileCommitProtocol is deprecated" ) - private def newOutputWriter(partDir: String): Unit = { - recordsInFile = 0 - releaseResources() - + protected def newWriter( + partDir: String, + bucketId: Option[Int], // Currently it's always None + fileCounter: Int + ): ColumnarOutputWriter = { updatedPartitions.add(partDir) + // Currently will be empty + val bucketIdStr = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") // This must be in a form that matches our bucketing format. See BucketingUtils. - val ext = f".c$fileCounter%03d" + - description.outputWriterFactory.getFileExtension(taskAttemptContext) + val ext = f"$bucketIdStr.c$fileCounter%03d" + + description.outputWriterFactory.getFileExtension(taskAttemptContext) - val customPath = - description.customPartitionLocations.get(PartitioningUtils.parsePathFragment(partDir)) + val customPath = description.customPartitionLocations + .get(PartitioningUtils.parsePathFragment(partDir)) val currentPath = if (customPath.isDefined) { committer.newTaskTempFileAbsPath(taskAttemptContext, customPath.get, ext) } else { - committer.newTaskTempFile(taskAttemptContext, Some(partDir), ext) + committer.newTaskTempFile(taskAttemptContext, Option(partDir), ext) } - currentWriter = description.outputWriterFactory.newInstance( + val newWriter = description.outputWriterFactory.newInstance( path = currentPath, dataSchema = description.dataColumns.toStructType, context = taskAttemptContext) statsTrackers.foreach(_.newFile(currentPath)) + newWriter } - // All data is sorted ascending with default null ordering - private val nullsSmallest = Ascending.defaultNullOrdering == NullsFirst - // distinct value sorted the same way the input data is sorted. private def distinctAndSort(t: Table): Table = { val columnIds = 0 until t.getNumberOfColumns @@ -313,7 +389,7 @@ class GpuDynamicPartitionDataWriter( } // Convert a table to a ColumnarBatch on the host, so we can iterate through it. - private def copyToHostAsBatch(input: Table, colTypes: Array[DataType]): ColumnarBatch = { + protected def copyToHostAsBatch(input: Table, colTypes: Array[DataType]): ColumnarBatch = { val tmp = GpuColumnVector.from(input, colTypes) try { new ColumnarBatch(GpuColumnVector.extractColumns(tmp).map(_.copyToHost()), tmp.numRows()) @@ -323,6 +399,22 @@ class GpuDynamicPartitionDataWriter( } override def write(cb: ColumnarBatch): Unit = { + // this single writer always passes `cachesMap` as None + write(cb, cachesMap = None) + } + + /** + * Write columnar batch. + * If the `cachesMap` is not empty, this single writer should restore the writers and caches in + * the `cachesMap`, this single writer should first combine the caches and current split data + * for a specific partition before write. + * + * @param cb the column batch + * @param cachesMap used by `GpuDynamicPartitionDataConcurrentWriter` when fall back to single + * writer, single writer should handle the stored writers and the pending caches + */ + def write(cb: ColumnarBatch, + cachesMap: Option[mutable.HashMap[String, WriterStatusWithCaches]]): Unit = { // We have an entire batch that is sorted, so we need to split it up by key var needToCloseBatch = true var partitionColumns: Table = null @@ -364,24 +456,85 @@ class GpuDynamicPartitionDataWriter( val table = combined._2.getTable val batch = GpuColumnVector.from(table, outDataTypes) val partPath = combined._1 + + // If fall back from for `GpuDynamicPartitionDataConcurrentWriter`, we should get the + // saved status + var savedStatus: Option[WriterStatusWithCaches] = None + if (currentPartPath != partPath) { + val previousPartPath = currentPartPath currentPartPath = partPath - statsTrackers.foreach(_.newPartition()) - fileCounter = 0 - newOutputWriter(currentPartPath) + + // see a new partition, close the old writer + val previousWriterStatus = currentWriterStatus + if (previousWriterStatus != null) { + releaseWriter(previousWriterStatus.outputWriter) + } + + if (cachesMap.isDefined) { + savedStatus = cachesMap.get.get(currentPartPath) + if (savedStatus.isDefined) { + // first try to restore the saved writer status, + // `GpuDynamicPartitionDataConcurrentWriter` may already opened the writer, and may + // have pending caches + currentWriterStatus = savedStatus.get.writerStatus + // entire batch that is sorted, see a new partition, the old write status is useless + cachesMap.get.remove(previousPartPath) + } else { + // create a new one + val writer = newWriter(partPath, None, 0) + currentWriterStatus = new WriterStatus(writer) + statsTrackers.foreach(_.newPartition()) + } + } else { + // create a new one + val writer = newWriter(partPath, None, 0) + currentWriterStatus = new WriterStatus(writer) + statsTrackers.foreach(_.newPartition()) + } } else if (description.maxRecordsPerFile > 0 && - recordsInFile >= description.maxRecordsPerFile) { + currentWriterStatus.recordsInFile >= description.maxRecordsPerFile) { // Exceeded the threshold in terms of the number of records per file. // Create a new file by increasing the file counter. - fileCounter += 1 - assert(fileCounter < MAX_FILE_COUNTER, - s"File counter $fileCounter is beyond max value $MAX_FILE_COUNTER") - - newOutputWriter(currentPartPath) + currentWriterStatus.fileCounter += 1 + assert(currentWriterStatus.fileCounter < MAX_FILE_COUNTER, + s"File counter ${currentWriterStatus.fileCounter} " + + s"is beyond max value $MAX_FILE_COUNTER") + + // will create a new file, close the old writer + val previousWriterStatus = currentWriterStatus + if (previousWriterStatus != null) { + releaseWriter(previousWriterStatus.outputWriter) + } + + // create a new writer and update the writer in the status + currentWriterStatus.outputWriter = + newWriter(partPath, None, currentWriterStatus.fileCounter) } statsTrackers.foreach(_.newBatch(batch)) - recordsInFile += batch.numRows - currentWriter.write(batch, statsTrackers) + + if (savedStatus.isDefined && savedStatus.get.tableCaches.nonEmpty) { + // convert caches seq to tables and close caches seq + val subTables = convertSpillBatchesToTables(savedStatus.get.tableCaches) + // clear the caches + savedStatus.get.tableCaches.clear() + // concat the caches and this `batch` + val concat = withResource(subTables) { _ => + withResource(batch) { _ => + // append `batch` to sub tables + subTables += GpuColumnVector.from(batch) + } + withResource(Table.concatenate(subTables: _*)) { concat => + GpuColumnVector.from(concat, outDataTypes) + } + } + // write concat table + currentWriterStatus.outputWriter.write(concat, statsTrackers) + currentWriterStatus.recordsInFile += concat.numRows + } else { + currentWriterStatus.outputWriter.write(batch, statsTrackers) + currentWriterStatus.recordsInFile += batch.numRows + } }) } finally { if (needToCloseBatch) { @@ -407,6 +560,413 @@ class GpuDynamicPartitionDataWriter( } } } + + /** Release all resources. */ + override def releaseResources(): Unit = { + // does not use `currentWriter`, single writer use `currentWriterStatus` + assert(currentWriter == null) + + if (currentWriterStatus != null) { + try { + currentWriterStatus.outputWriter.close() + statsTrackers.foreach(_.closeFile(currentWriterStatus.outputWriter.path())) + } finally { + currentWriterStatus = null + } + } + } + + /** + * convert spillable columnar batch seq to tables and close the input `spills` + * + * @param spills spillable columnar batch seq + * @return table array + */ + def convertSpillBatchesToTables(spills: Seq[SpillableColumnarBatch]): ArrayBuffer[Table] = { + withResource(spills) { _ => + val subTablesBuffer = new ArrayBuffer[Table] + spills.foreach { spillableCb => + withResource(spillableCb.getColumnarBatch()) { cb => + val currTable = GpuColumnVector.from(cb) + subTablesBuffer += currTable + } + } + subTablesBuffer + } + } +} + +/** + * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened + * for writing. + * + * The process has the following steps: + * - Step 1: Maintain a map of output writers per each partition columns. Keep all + * writers opened; Cache the inputted batches by splitting them into sub-groups and + * each partition holds a list of spillable sub-groups; Find and write the max pending + * partition data if the total caches exceed the limitation. + * - Step 2: If number of concurrent writers exceeds limit, fall back to sort-based write + * (`GpuDynamicPartitionDataSingleWriter`), sort rest of batches on partition. + * Write batch by batch, and eagerly close the writer when finishing + * Caller is expected to call `writeWithIterator()` instead of `write()` to write records. + * Note: when fall back to `GpuDynamicPartitionDataSingleWriter`, the single writer should + * restore un-closed writers and should handle un-flushed spillable caches. + */ +class GpuDynamicPartitionDataConcurrentWriter( + description: GpuWriteJobDescription, + taskAttemptContext: TaskAttemptContext, + committer: FileCommitProtocol, + spec: GpuConcurrentOutputWriterSpec) + extends GpuDynamicPartitionDataSingleWriter(description, taskAttemptContext, committer) + with Arm with Logging { + + // Keep all the unclosed writers, key is partition directory string. + // Note: if fall back to sort-based mode, also use the opened writers in the map. + private val concurrentWriters = mutable.HashMap[String, WriterStatusWithCaches]() + + // guarantee to close the caches and writers when task is finished + TaskContext.get().addTaskCompletionListener[Unit](_ => closeCachesAndWriters()) + + private val outDataTypes = description.dataColumns.map(_.dataType).toArray + + val spillCallback = RapidsBuffer.defaultSpillCallback + + val partitionFlushSize = if (description.concurrentWriterPartitionFlushSize <= 0) { + // if the property is equal or less than 0, use default value of parquet or orc + val extension = description.outputWriterFactory + .getFileExtension(taskAttemptContext).toLowerCase() + if (extension.endsWith("parquet")) { + taskAttemptContext.getConfiguration.getLong("write.parquet.row-group-size-bytes", + 128L * 1024L * 1024L) // 128M + } else if (extension.endsWith("orc")) { + taskAttemptContext.getConfiguration.getLong("orc.stripe.size", + 64L * 1024L * 1024L) // 64M + } else { + 128L * 1024L * 1024L // 128M + } + } else { + // if the property is greater than 0, use the property value + description.concurrentWriterPartitionFlushSize + } + + // refer to current batch if should fall back to `single writer` + var currentFallbackColumnarBatch: ColumnarBatch = _ + + override def abort(): Unit = { + try { + closeCachesAndWriters() + } finally { + committer.abortTask(taskAttemptContext) + } + } + + /** + * State to indicate if we are falling back to sort-based writer. + * Because we first try to use concurrent writers, its initial value is false. + */ + private var fallBackToSortBased: Boolean = false + + def writeWithSingleWriter(cb: ColumnarBatch): Unit = { + // invoke `GpuDynamicPartitionDataSingleWriter`.write, + // single writer will take care of the unclosed writers and the pending caches + // in `concurrentWriters` + super.write(cb, Some(concurrentWriters)) + } + + def writeWithConcurrentWriter(cb: ColumnarBatch): Unit = { + this.write(cb) + } + + /** + * Write an iterator of column batch. + * + * @param iterator the iterator of column batch + */ + override def writeWithIterator(iterator: Iterator[ColumnarBatch]): Unit = { + // 1: try concurrent writer + while (iterator.hasNext && !fallBackToSortBased) { + // concurrently write and update the `concurrentWriters` map + // the `` will be updated + writeWithConcurrentWriter(iterator.next()) + } + + // 2: fall back to single writer + // Note single writer should restore writer status and handle the pending caches + if (fallBackToSortBased) { + // concat the put back batch and un-coming batches + val newIterator = Iterator.single(currentFallbackColumnarBatch) ++ iterator + // sort the all the batches in `iterator` + val sortIterator: GpuOutOfCoreSortIterator = getSorted(newIterator) + while (sortIterator.hasNext) { + // write with sort-based single writer + writeWithSingleWriter(sortIterator.next()) + } + } + } + + /** + * Sort the input iterator by out of core sort + * + * @param iterator the input iterator + * @return sorted iterator + */ + private def getSorted(iterator: Iterator[ColumnarBatch]): GpuOutOfCoreSortIterator = { + val gpuSortOrder: Seq[SortOrder] = spec.sortOrder + val output: Seq[Attribute] = spec.exec.output + val sorter = new GpuSorter(gpuSortOrder, output) + val cpuOrd = new LazilyGeneratedOrdering(sorter.cpuOrdering) + + // use noop metrics below + val sortTime = NoopMetric + val peakDevMemory = NoopMetric + val opTime = NoopMetric + val outputBatch = NoopMetric + val outputRows = NoopMetric + val spillCallback = new SpillCallback { + override def apply(from: StorageTier, to: StorageTier, amount: Long): Unit = { + } + + override def semaphoreWaitTime: GpuMetric = NoopMetric + } + val targetSize = GpuSortExec.targetSize(spec.exec.conf) + // out of core sort the entire iterator + GpuOutOfCoreSortIterator(iterator, sorter, cpuOrd, targetSize, + opTime, sortTime, outputBatch, outputRows, + peakDevMemory, spillCallback) + } + + /** + * concurrent write the columnar batch + * Note: if new partitions number in `cb` plus existing partitions number is greater than + * `maxWriters` limit, will put back the whole `cb` to 'single writer` + * + * @param cb the columnar batch + */ + override def write(cb: ColumnarBatch): Unit = { + assert(isPartitioned) + assert(!isBucketed) + + if (cb.numRows() == 0) { + // TODO https://github.com/NVIDIA/spark-rapids/issues/6453 + // To solve above issue, I assume that an empty batch will be wrote for saving metadata. + // If the assumption it's true, this concurrent writer should write the metadata here, + // and should not run into below splitting and caching logic + return + } + + // 1. combine partition columns and `cb` columns into a column array + val columnsWithPartition = ArrayBuffer[ColumnVector]() + withResource(getPartitionColumns(cb)) { partitionColumnsTable => + for (i <- 0 until partitionColumnsTable.getNumberOfColumns) { + // append partition column + columnsWithPartition += partitionColumnsTable.getColumn(i) + } + } + val cols = GpuColumnVector.extractBases(cb) + columnsWithPartition ++= cols + + // 2. group by the partition columns + // get sub-groups for each partition and get unique keys for each partition + val groupsAndKeys = withResource(new Table(columnsWithPartition: _*)) { colsWithPartitionTbl => + // [0, partition columns number - 1] + val partitionIndices = description.partitionColumns.indices + + // group by partition columns + val op = colsWithPartitionTbl.groupBy(partitionIndices: _*) + // return groups and uniq keys table + // Each row in uniq keys table is corresponding to a group + op.contiguousSplitGroupsAndGenUniqKeys() + } + + withResource(groupsAndKeys) { _ => + // groups number should equal to uniq keys number + assert(groupsAndKeys.getGroups.length == groupsAndKeys.getUniqKeyTable.getRowCount) + + val (groups, keys) = (groupsAndKeys.getGroups, groupsAndKeys.getUniqKeyTable) + + // 3. generate partition strings for all sub-groups in advance + val partDataTypes = description.partitionColumns.map(_.dataType).toArray + val dataTypes = GpuColumnVector.extractTypes(cb) + // generate partition string list for all groups + val partitionStrList = getPartitionStrList(keys, partDataTypes) + // key table is useless now + groupsAndKeys.closeUniqKeyTable() + + // 4. cache each group according to each partitionStr + withResource(groups) { _ => + + // first update fallBackToSortBased + withResource(cb) { _ => + var newPartitionNum = 0 + var groupIndex = 0 + while (!fallBackToSortBased && groupIndex < groups.length) { + // get the partition string + val partitionStr = partitionStrList(groupIndex) + groupIndex += 1 + if (!concurrentWriters.contains(partitionStr)) { + newPartitionNum += 1 + if (newPartitionNum + concurrentWriters.size >= spec.maxWriters) { + fallBackToSortBased = true + currentFallbackColumnarBatch = cb + // `cb` should be put back to single writer + GpuColumnVector.incRefCounts(cb) + } + } + } + } + + if (!fallBackToSortBased) { + // not fall, collect all caches + var groupIndex = 0 + while (groupIndex < groups.length) { + // get the partition string and group pair + val (partitionStr, group) = (partitionStrList(groupIndex), groups(groupIndex)) + val groupTable = group.getTable + groupIndex += 1 + + // create writer if encounter a new partition and put into `concurrentWriters` map + if (!concurrentWriters.contains(partitionStr)) { + val w = newWriter(partitionStr, None, 0) + val ws = new WriterStatus(w) + concurrentWriters.put(partitionStr, new WriterStatusWithCaches(ws)) + statsTrackers.foreach(_.newPartition()) + } + + // get data columns, tail part is data columns + val dataColumns = ArrayBuffer[ColumnVector]() + for (i <- description.partitionColumns.length until groupTable.getNumberOfColumns) { + dataColumns += groupTable.getColumn(i) + } + withResource(new Table(dataColumns: _*)) { dataTable => + withResource(GpuColumnVector.from(dataTable, dataTypes)) { cb => + val outputCb = getOutputCb(cb) + // convert to spillable cache and add to the pending cache + val currWriterStatus = concurrentWriters(partitionStr) + // create SpillableColumnarBatch to take the owner of `outputCb` + currWriterStatus.tableCaches += SpillableColumnarBatch( + outputCb, SpillPriorities.ACTIVE_ON_DECK_PRIORITY, spillCallback) + currWriterStatus.deviceBytes += GpuColumnVector.getTotalDeviceMemoryUsed(outputCb) + } + } + } + } + } + } + + // 5. find all big enough partitions and write + if(!fallBackToSortBased) { + for ((partitionDir, ws) <- findBigPartitions(partitionFlushSize)) { + writeAndCloseCache(partitionDir, ws) + } + } + } + + private def getPartitionStrList( + uniqKeysTable: Table, partDataTypes: Array[DataType]): Array[String] = { + withResource(copyToHostAsBatch(uniqKeysTable, partDataTypes)) { oneRowCb => + import scala.collection.JavaConverters._ + oneRowCb.rowIterator().asScala.map(getPartitionPath).toArray + } + } + + private def writeAndCloseCache(partitionDir: String, status: WriterStatusWithCaches): Unit = { + assert(status.tableCaches.nonEmpty) + + // convert spillable caches to tables, and close `status.tableCaches` + val subTables = convertSpillBatchesToTables(status.tableCaches) + + // get concat table or the single table + val t: Table = if (status.tableCaches.length >= 2) { + // concat the sub batches to write in once. + val concat = Table.concatenate(subTables: _*) + // close sub tables after concat + subTables.safeClose() + concat + } else { + // only one single table + subTables.head + } + + withResource(t) { _ => + if (description.maxRecordsPerFile > 0 && + status.writerStatus.recordsInFile + t.getRowCount >= description.maxRecordsPerFile) { + // TODO: This does not handle batch splitting to make sure maxRecordsPerFile + // is not exceeded. See: https://github.com/NVIDIA/spark-rapids/issues/6452 + // write data + val batch = GpuColumnVector.from(t, outDataTypes) + // the batch is closed in `write` + status.writerStatus.outputWriter.write(batch, statsTrackers) + statsTrackers.foreach(_.newBatch(batch)) + status.writerStatus.recordsInFile += t.getRowCount + + // check max file number + assert(status.writerStatus.fileCounter < MAX_FILE_COUNTER, + s"File counter ${status.writerStatus.fileCounter} is beyond max value $MAX_FILE_COUNTER") + + // will create a new file, close the former writer + status.writerStatus.outputWriter.close() + + // start a new writer + val w = newWriter(partitionDir, None, status.writerStatus.fileCounter) + status.writerStatus.outputWriter = w + status.writerStatus.recordsInFile = 0L + status.writerStatus.fileCounter += 1 + } else { + // write data + val batch = GpuColumnVector.from(t, outDataTypes) + // the batch is closed in `write` + status.writerStatus.outputWriter.write(batch, statsTrackers) + statsTrackers.foreach(_.newBatch(batch)) + status.writerStatus.recordsInFile += t.getRowCount + } + status.tableCaches.clear() + status.deviceBytes = 0 + } + } + + def closeCachesAndWriters(): Unit = { + // collect all caches and writers + val allResources = ArrayBuffer[AutoCloseable]() + allResources ++= concurrentWriters.values.flatMap(ws => ws.tableCaches) + allResources ++= concurrentWriters.values.map { ws => + new AutoCloseable() { + override def close(): Unit = { + ws.writerStatus.outputWriter.close() + statsTrackers.foreach(_.closeFile(ws.writerStatus.outputWriter.path())) + } + } + } + + // safe close all the caches and writers + allResources.safeClose() + + // clear `concurrentWriters` map + concurrentWriters.values.foreach(ws => ws.tableCaches.clear()) + concurrentWriters.clear() + } + + /** Release all resources. */ + override def releaseResources(): Unit = { + // does not use `currentWriter`, only use the writers in the concurrent writer map + assert(currentWriter == null) + + if (fallBackToSortBased) { + // Note: we should close the last partition writer in the single writer. + super.releaseResources() + } + + // write all caches + concurrentWriters.filter(pair => pair._2.tableCaches.nonEmpty) + .foreach(pair => writeAndCloseCache(pair._1, pair._2)) + + // close all resources + closeCachesAndWriters() + } + + private def findBigPartitions( + sizeThreshold: Long): mutable.Map[String, WriterStatusWithCaches] = { + concurrentWriters.filter(pair => pair._2.deviceBytes >= sizeThreshold) + } } /** @@ -435,7 +995,8 @@ class GpuWriteJobDescription( val customPartitionLocations: Map[TablePartitionSpec, String], val maxRecordsPerFile: Long, val timeZoneId: String, - val statsTrackers: Seq[ColumnarWriteJobStatsTracker]) + val statsTrackers: Seq[ColumnarWriteJobStatsTracker], + val concurrentWriterPartitionFlushSize: Long) extends Serializable { assert(AttributeSet(allColumns) == AttributeSet(partitionColumns ++ dataColumns), diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala index c2055ed7922..8d25bf78ffe 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala @@ -80,6 +80,10 @@ object GpuFileFormatWriter extends Logging { } } + /** Describes how concurrent output writers should be executed. */ + case class GpuConcurrentOutputWriterSpec(maxWriters: Int, exec: GpuExec, + sortOrder: Seq[SortOrder]) + /** * Basic work flow of this command is: * 1. Driver side setup, including output committer initialization and data source specific @@ -105,7 +109,8 @@ object GpuFileFormatWriter extends Logging { bucketSpec: Option[BucketSpec], statsTrackers: Seq[ColumnarWriteJobStatsTracker], options: Map[String, String], - useStableSort: Boolean): Set[String] = { + useStableSort: Boolean, + concurrentWriterPartitionFlushSize: Long): Set[String] = { val job = Job.getInstance(hadoopConf) job.setOutputKeyClass(classOf[Void]) @@ -168,7 +173,8 @@ object GpuFileFormatWriter extends Logging { .getOrElse(sparkSession.sessionState.conf.maxRecordsPerFile), timeZoneId = caseInsensitiveOptions.get(DateTimeUtils.TIMEZONE_OPTION) .getOrElse(sparkSession.sessionState.conf.sessionLocalTimeZone), - statsTrackers = statsTrackers + statsTrackers = statsTrackers, + concurrentWriterPartitionFlushSize = concurrentWriterPartitionFlushSize ) // We should first sort by partition columns, then bucket id, and finally sorting columns. @@ -195,29 +201,46 @@ object GpuFileFormatWriter extends Logging { // prepares the job, any exception thrown from here shouldn't cause abortJob() to be called. committer.setupJob(job) + // check if we can use concurrent writer + val maxWriterConfKey = "spark.sql.maxConcurrentOutputFileWriters" + val maxWriters: Int = if (sparkSession.sessionState.conf.contains(maxWriterConfKey)) { + sparkSession.sessionState.conf.getConfString(maxWriterConfKey).toInt + } else { + // Spark 31x does not support concurrent writer + 0 + } + val concurrentWritersEnabled = maxWriters > 0 && sortColumns.isEmpty + try { - val rdd = if (orderingMatched) { - empty2NullPlan.executeColumnar() + val (rdd, concurrentOutputWriterSpec) = if (orderingMatched) { + (empty2NullPlan.executeColumnar(), None) } else { // SPARK-21165: the `requiredOrdering` is based on the attributes from analyzed plan, and // the physical plan may have different attribute ids due to optimizer removing some // aliases. Here we bind the expression ahead to avoid potential attribute ids mismatch. val orderingExpr = GpuBindReferences.bindReferences( requiredOrdering - .map(attr => SortOrder(attr, Ascending)), finalOutputSpec.outputColumns) - val sortType = if (useStableSort) { - FullSortSingleBatch + .map(attr => SortOrder(attr, Ascending)), finalOutputSpec.outputColumns) + if (concurrentWritersEnabled) { + (empty2NullPlan.executeColumnar(), + Some(GpuConcurrentOutputWriterSpec(maxWriters, empty2NullPlan.asInstanceOf[GpuExec], + orderingExpr))) } else { - OutOfCoreSort + val sortType = if (useStableSort) { + FullSortSingleBatch + } else { + OutOfCoreSort + } + // TODO: Using a GPU ordering as a CPU ordering here. Should be OK for now since we do not + // support bucket expressions yet and the rest should be simple attributes. + val sort = GpuSortExec( + orderingExpr, + global = false, + child = empty2NullPlan, + sortType = sortType + )(orderingExpr).executeColumnar() + (sort, None) } - // TODO: Using a GPU ordering as a CPU ordering here. Should be OK for now since we do not - // support bucket expressions yet and the rest should be simple attributes. - GpuSortExec( - orderingExpr, - global = false, - child = empty2NullPlan, - sortType = sortType - )(orderingExpr).executeColumnar() } // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single @@ -240,7 +263,8 @@ object GpuFileFormatWriter extends Logging { sparkPartitionId = taskContext.partitionId(), sparkAttemptNumber = taskContext.taskAttemptId().toInt & Integer.MAX_VALUE, committer, - iterator = iter) + iterator = iter, + concurrentOutputWriterSpec = concurrentOutputWriterSpec) }, rddWithNonEmptyPartitions.partitions.indices, (index, res: WriteTaskResult) => { @@ -273,7 +297,8 @@ object GpuFileFormatWriter extends Logging { sparkPartitionId: Int, sparkAttemptNumber: Int, committer: FileCommitProtocol, - iterator: Iterator[ColumnarBatch]): WriteTaskResult = { + iterator: Iterator[ColumnarBatch], + concurrentOutputWriterSpec: Option[GpuConcurrentOutputWriterSpec]): WriteTaskResult = { val jobId = SparkHadoopWriterUtils.createJobID(new Date(jobIdInstant), sparkStageId) val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId) @@ -301,15 +326,19 @@ object GpuFileFormatWriter extends Logging { } else if (description.partitionColumns.isEmpty && description.bucketSpec.isEmpty) { new GpuSingleDirectoryDataWriter(description, taskAttemptContext, committer) } else { - new GpuDynamicPartitionDataWriter(description, taskAttemptContext, committer) + concurrentOutputWriterSpec match { + case Some(spec) => + new GpuDynamicPartitionDataConcurrentWriter( + description, taskAttemptContext, committer, spec) + case _ => + new GpuDynamicPartitionDataSingleWriter(description, taskAttemptContext, committer) + } } try { Utils.tryWithSafeFinallyAndFailureCallbacks(block = { // Execute the task to write rows out and commit the task. - while (iterator.hasNext) { - dataWriter.write(iterator.next()) - } + dataWriter.writeWithIterator(iterator) dataWriter.commit() })(catchBlock = { // If there is an error, abort the task diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala index fbd2f1da0d8..bd628b49d75 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala @@ -49,7 +49,8 @@ case class GpuInsertIntoHadoopFsRelationCommand( catalogTable: Option[CatalogTable], fileIndex: Option[FileIndex], outputColumnNames: Seq[String], - useStableSort: Boolean) + useStableSort: Boolean, + concurrentWriterPartitionFlushSize: Long) extends GpuDataWritingCommand { override def runColumnar(sparkSession: SparkSession, child: SparkPlan): Seq[ColumnarBatch] = { @@ -165,7 +166,8 @@ case class GpuInsertIntoHadoopFsRelationCommand( bucketSpec = bucketSpec, statsTrackers = Seq(gpuWriteJobStatsTracker(hadoopConf)), options = options, - useStableSort = useStableSort) + useStableSort = useStableSort, + concurrentWriterPartitionFlushSize = concurrentWriterPartitionFlushSize) // update metastore partition metadata From f5fbfbdcfc94a19ab3ec9fba8a8ae915b40dafef Mon Sep 17 00:00:00 2001 From: Peixin Date: Mon, 26 Sep 2022 10:32:09 +0800 Subject: [PATCH 140/190] Enable automerge from 22.10 to 22.12 (#6609) Signed-off-by: Peixin Li Signed-off-by: Peixin Li --- .github/workflows/auto-merge.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/auto-merge.yml b/.github/workflows/auto-merge.yml index 566dd32ece4..0c927869d2e 100644 --- a/.github/workflows/auto-merge.yml +++ b/.github/workflows/auto-merge.yml @@ -18,7 +18,7 @@ name: auto-merge HEAD to BASE on: pull_request_target: branches: - - branch-22.08 + - branch-22.10 types: [closed] jobs: @@ -29,13 +29,13 @@ jobs: steps: - uses: actions/checkout@v2 with: - ref: branch-22.08 # force to fetch from latest upstream instead of PR ref + ref: branch-22.10 # force to fetch from latest upstream instead of PR ref - name: auto-merge job uses: ./.github/workflows/auto-merge env: OWNER: NVIDIA REPO_NAME: spark-rapids - HEAD: branch-22.08 - BASE: branch-22.10 + HEAD: branch-22.10 + BASE: branch-22.12 AUTOMERGE_TOKEN: ${{ secrets.AUTOMERGE_TOKEN }} # use to merge PR From 9b23fee9bfc683b4adcab78b07c3e3d55fc0587b Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Mon, 26 Sep 2022 21:08:49 +0800 Subject: [PATCH 141/190] Add alluxio reliability doc (#6506) * Add alluxio reliability doc Signed-off-by: Chong Gao * Systemd service for alluxio * Update * Address comments * Update * Address comments * Change block read max duration to 1 minute Signed-off-by: Chong Gao Co-authored-by: Chong Gao --- docs/get-started/getting-started-alluxio.md | 55 +++++++++++++++++++++ 1 file changed, 55 insertions(+) diff --git a/docs/get-started/getting-started-alluxio.md b/docs/get-started/getting-started-alluxio.md index 47af0f0ef65..56905b1b22c 100644 --- a/docs/get-started/getting-started-alluxio.md +++ b/docs/get-started/getting-started-alluxio.md @@ -365,3 +365,58 @@ This section will give some links about how to configure, tune Alluxio and some - [Running Spark on Alluxio](https://docs.alluxio.io/os/user/stable/en/compute/Spark.html) - [Performance Tuning](https://docs.alluxio.io/os/user/stable/en/operation/Performance-Tuning.html ) - [Alluxio troubleshooting](https://docs.alluxio.io/os/user/stable/en/operation/Troubleshooting.html) + +## Alluxio reliability +The properties mentioned in this section can be found in [Alluxio configuration](https://docs.alluxio.io/os/user/stable/en/reference/Properties-List.html) + +### Dealing with Client side delays in response from master or workers +If the master is not responding, possibly due to it crashing or GC pause, +`alluxio.user.rpc.retry.max.duration` will cause the client to retry for 2 minutes. +This is a very long time and can cause delays in the running job, so we suggest lowering this value to 10 seconds. + +If the worker is not responding, possibly due to it crashing or GC pause, +`alluxio.user.block.read.retry.max.duration` will cause the client to retry for 5 minutes. +This is a very long time and can cause delays in the running job, so we suggest lowering this value to 1 minute. + +See relative configs also: +``` +alluxio.user.rpc.retry.max.duration +alluxio.user.rpc.retry.max.sleep +alluxio.user.rpc.retry.base.sleep + +alluxio.user.block.read.retry.max.duration +alluxio.user.block.read.retry.sleep.max +alluxio.user.block.read.retry.sleep.base +``` +Above configurations define the `ExponentialTimeBoundedRetry` retry policies and `max durations`, we can adjust them to appropriate values. + +Set these properties on Spark because Spark invokes Alluxio client. +``` +$SPARK_HOME/bin/spark-shell \ +...... +--conf spark.driver.extraJavaOptions='-Dalluxio.user.rpc.retry.max.duration=10sec -Dalluxio.user.block.read.retry.max.duration=1min' \ +--conf spark.executor.extraJavaOptions='-Dalluxio.user.rpc.retry.max.duration=10sec -Dalluxio.user.block.read.retry.max.duration=1min' \ +...... +``` + +### Worker server tunings to fail fast +By default, `alluxio.master.worker.timeout` is 5min, this is the timeout between master and worker indicating a lost worker. +If the worker holding cache is killed but the elapsed time does not exceed the timeout, +the master still marks the worker as alive. The client will connect this dead worker to pull data, and will fail. +If the worker holding cache is killed and the elapsed time exceeds the timeout, the master marks the worker as lost. +In this case, if cluster has one alive worker, the client will query an alive worker +and the alive worker will pull data from external file system if it is not holding the requested cache. + +To avoid failures when master marking an actual dead worker as alive, set the timeout to a reasonable value, like 1 minute. +vi $ALLUXIO_HOME/conf/alluxio-site.properties +``` +alluxio.master.worker.timeout=60sec +``` + +### The logs +By default, the log path is /logs. +See the master.log and worker.log in this path. + +### Auto start Alluxio the master and workers +After installing Alluxio master and workers, it's better to add a systemd service for each process of master and workers. +Systemd service can automatically start process if process is terminated. \ No newline at end of file From 213b34180a068329618f9ba85acf20584b650b57 Mon Sep 17 00:00:00 2001 From: Gary Shen Date: Mon, 26 Sep 2022 21:13:35 +0800 Subject: [PATCH 142/190] Fix spark2-sql-plugin (#6584) * Add FromUTCTimestamp Signed-off-by: Gary Shen * Fix compiling Signed-off-by: gashen * Fix part of spark2-sql-plugin Signed-off-by: Gary Shen * Update spark2-sql-plugin code base on a82a3a040 Signed-off-by: Gary Shen * Add diff for GpuOptimizedCreateHiveTableAsSelectCommand Signed-off-by: Gary Shen * Update GpuHiveOverrides Signed-off-by: Gary Shen * Update spark2diff with diff v3.6 Signed-off-by: gashen * Address the comments Signed-off-by: Gary Shen * Sync up to commit 3f377ef61 Signed-off-by: gashen * Sync up to commit 26a6a96e2 Signed-off-by: gashen * Fix scalastyle failure Signed-off-by: gashen Signed-off-by: Gary Shen Signed-off-by: gashen --- scripts/rundiffspark2.sh | 13 + .../spark2diffs/FromUTCTimestampExprMeta.diff | 8 + scripts/spark2diffs/GpuBatchUtils.diff | 4 + scripts/spark2diffs/GpuDecimalDivide.diff | 4 +- scripts/spark2diffs/GpuHashJoin.diff | 6 +- scripts/spark2diffs/GpuHiveOverrides.diff | 25 +- ...timizedCreateHiveTableAsSelectCommand.diff | 143 ++++ scripts/spark2diffs/GpuOrcScanBase.diff | 10 +- scripts/spark2diffs/GpuOverrides.diff | 669 ++++++++++-------- scripts/spark2diffs/GpuParquetFileFormat.diff | 8 +- scripts/spark2diffs/GpuRLikeMeta.diff | 2 +- scripts/spark2diffs/GpuReadOrcFileFormat.diff | 2 +- scripts/spark2diffs/GpuRegExpExtractMeta.diff | 2 +- scripts/spark2diffs/GpuRegExpReplaceMeta.diff | 6 +- scripts/spark2diffs/GpuRegExpUtils.diff | 8 +- .../GpuSpecifiedWindowFrameMetaBase.diff | 6 +- .../GpuWindowExpressionMetaBase.diff | 2 +- scripts/spark2diffs/HiveProviderImpl.diff | 17 +- scripts/spark2diffs/RapidsConf.diff | 14 +- scripts/spark2diffs/RapidsMeta.diff | 114 +-- scripts/spark2diffs/RegexParser.diff | 2 +- scripts/spark2diffs/TypeChecks.diff | 64 +- scripts/spark2diffs/UnixTimeExprMeta.diff | 2 - scripts/spark2diffs/aggregate.diff | 113 ++- scripts/spark2diffs/average.diff | 2 +- .../com/nvidia/spark/rapids/DateUtils.scala | 4 +- .../nvidia/spark/rapids/GpuBatchUtils.scala | 152 ++++ ...imizedCreateHiveTableAsSelectCommand.scala | 78 ++ .../com/nvidia/spark/rapids/GpuOrcScan.scala | 5 - .../nvidia/spark/rapids/GpuOverrides.scala | 307 ++++---- .../spark/rapids/GpuParquetFileFormat.scala | 18 +- .../spark/rapids/GpuReadOrcFileFormat.scala | 3 - .../spark/rapids/GpuRegExpReplaceMeta.scala | 14 +- .../nvidia/spark/rapids/GpuWindowMeta.scala | 2 +- .../nvidia/spark/rapids/HiveProvider.scala | 4 + .../com/nvidia/spark/rapids/RapidsConf.scala | 138 +++- .../com/nvidia/spark/rapids/RapidsMeta.scala | 10 +- .../rapids/RegexComplexityEstimator.scala | 71 ++ .../com/nvidia/spark/rapids/RegexParser.scala | 82 ++- .../spark/rapids/ShimGpuOverrides.scala | 29 +- .../com/nvidia/spark/rapids/TypeChecks.scala | 49 +- .../spark/rapids/shims/gpuWindows.scala | 11 +- .../sql/hive/rapids/GpuHiveOverrides.scala | 37 +- .../sql/hive/rapids/HiveProviderImpl.scala | 14 +- .../apache/spark/sql/rapids/arithmetic.scala | 30 + .../sql/rapids/datetimeExpressionsMeta.scala | 31 +- .../sql/rapids/execution/GpuHashJoin.scala | 3 +- .../apache/spark/sql/rapids/stringMeta.scala | 15 +- 48 files changed, 1623 insertions(+), 730 deletions(-) create mode 100644 scripts/spark2diffs/FromUTCTimestampExprMeta.diff create mode 100644 scripts/spark2diffs/GpuBatchUtils.diff create mode 100644 scripts/spark2diffs/GpuOptimizedCreateHiveTableAsSelectCommand.diff create mode 100644 spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchUtils.scala create mode 100644 spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOptimizedCreateHiveTableAsSelectCommand.scala create mode 100644 spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexComplexityEstimator.scala diff --git a/scripts/rundiffspark2.sh b/scripts/rundiffspark2.sh index 5fd471d6538..e10f27937a6 100755 --- a/scripts/rundiffspark2.sh +++ b/scripts/rundiffspark2.sh @@ -80,6 +80,11 @@ sed -n '/abstract class UnixTimeExprMeta/,/sealed trait TimeParserPolicy/{/seal diff $tmp_dir/UnixTimeExprMeta_new.out $tmp_dir/UnixTimeExprMeta_old.out > $tmp_dir/UnixTimeExprMeta.newdiff || true diff -c spark2diffs/UnixTimeExprMeta.diff $tmp_dir/UnixTimeExprMeta.newdiff +sed -n '/class FromUTCTimestampExprMeta/,/override def convertToGpu/{/override def convertToGpu/!p}' ../spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressionsMeta.scala > $tmp_dir/FromUTCTimestampExprMeta_new.out +sed -n '/class FromUTCTimestampExprMeta/,/override def convertToGpu/{/override def convertToGpu/!p}' ../sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala > $tmp_dir/FromUTCTimestampExprMeta_old.out +diff $tmp_dir/FromUTCTimestampExprMeta_new.out $tmp_dir/FromUTCTimestampExprMeta_old.out > $tmp_dir/FromUTCTimestampExprMeta.newdiff || true +diff -c spark2diffs/FromUTCTimestampExprMeta.diff $tmp_dir/FromUTCTimestampExprMeta.newdiff + sed -n '/object GpuToTimestamp/,/abstract class UnixTimeExprMeta/{/abstract class UnixTimeExprMeta/!p}' ../spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressionsMeta.scala > $tmp_dir/GpuToTimestamp_new.out sed -n '/object GpuToTimestamp/,/val REMOVE_WHITESPACE_FROM_MONTH_DAY/{/val REMOVE_WHITESPACE_FROM_MONTH_DAY/!p}' ../sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala > $tmp_dir/GpuToTimestamp_old.out diff $tmp_dir/GpuToTimestamp_new.out $tmp_dir/GpuToTimestamp_old.out > $tmp_dir/GpuToTimestamp.newdiff || true @@ -99,6 +104,14 @@ sed -n '/class GpuRegExpExtractMeta/,/override def convertToGpu/{/override def diff $tmp_dir/GpuRegExpExtractMeta_new.out $tmp_dir/GpuRegExpExtractMeta_old.out > $tmp_dir/GpuRegExpExtractMeta.newdiff || true diff -c spark2diffs/GpuRegExpExtractMeta.diff $tmp_dir/GpuRegExpExtractMeta.newdiff +diff ../sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexComplexityEstimator.scala ../spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexComplexityEstimator.scala + +diff ../sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchUtils.scala ../spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchUtils.scala > $tmp_dir/GpuBatchUtils.newdiff || true +diff -c spark2diffs/GpuBatchUtils.diff $tmp_dir/GpuBatchUtils.newdiff + +diff ../spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOptimizedCreateHiveTableAsSelectCommand.scala ../sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOptimizedCreateHiveTableAsSelectCommand.scala > $tmp_dir/GpuOptimizedCreateHiveTableAsSelectCommand.newdiff || true +diff -c spark2diffs/GpuOptimizedCreateHiveTableAsSelectCommand.diff $tmp_dir/GpuOptimizedCreateHiveTableAsSelectCommand.newdiff + sed -n '/class SubstringIndexMeta/,/^}/{/^}/!p}' ../spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringMeta.scala > $tmp_dir/SubstringIndexMeta_new.out sed -n '/class SubstringIndexMeta/,/override def convertToGpu/{/override def convertToGpu/!p}' ../sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala > $tmp_dir/SubstringIndexMeta_old.out diff $tmp_dir/SubstringIndexMeta_new.out $tmp_dir/SubstringIndexMeta_old.out > $tmp_dir/SubstringIndexMeta.newdiff || true diff --git a/scripts/spark2diffs/FromUTCTimestampExprMeta.diff b/scripts/spark2diffs/FromUTCTimestampExprMeta.diff new file mode 100644 index 00000000000..60ec4d05acc --- /dev/null +++ b/scripts/spark2diffs/FromUTCTimestampExprMeta.diff @@ -0,0 +1,8 @@ +4c4 +< override val parent: Option[RapidsMeta[_, _]], +--- +> override val parent: Option[RapidsMeta[_, _, _]], +25c25 +< } +--- +> diff --git a/scripts/spark2diffs/GpuBatchUtils.diff b/scripts/spark2diffs/GpuBatchUtils.diff new file mode 100644 index 00000000000..2f1c597400e --- /dev/null +++ b/scripts/spark2diffs/GpuBatchUtils.diff @@ -0,0 +1,4 @@ +2c2 +< * Copyright (c) 2020-2021, NVIDIA CORPORATION. +--- +> * Copyright (c) 2022, NVIDIA CORPORATION. diff --git a/scripts/spark2diffs/GpuDecimalDivide.diff b/scripts/spark2diffs/GpuDecimalDivide.diff index 0a29774f57d..31eb1823c46 100644 --- a/scripts/spark2diffs/GpuDecimalDivide.diff +++ b/scripts/spark2diffs/GpuDecimalDivide.diff @@ -1,8 +1,8 @@ -27c27 +57c57 < GpuOverrides.DECIMAL128_MAX_PRECISION) --- > DType.DECIMAL128_MAX_PRECISION) -51,52c51,52 +81,82c81,82 < math.min(outputType.precision + 1, GpuOverrides.DECIMAL128_MAX_PRECISION), < math.min(outputType.scale + 1, GpuOverrides.DECIMAL128_MAX_PRECISION)) --- diff --git a/scripts/spark2diffs/GpuHashJoin.diff b/scripts/spark2diffs/GpuHashJoin.diff index 6cc06c26fd3..d21081a49b7 100644 --- a/scripts/spark2diffs/GpuHashJoin.diff +++ b/scripts/spark2diffs/GpuHashJoin.diff @@ -2,13 +2,13 @@ < def tagForGpu(joinType: JoinType, meta: RapidsMeta[_, _]): Unit = { --- > def tagForGpu(joinType: JoinType, meta: RapidsMeta[_, _, _]): Unit = { -72c72 +73c73 < object GpuHashJoin { --- > object GpuHashJoin extends Arm { -98a99 +99a100 > -119c120 +120c121 < } --- > diff --git a/scripts/spark2diffs/GpuHiveOverrides.diff b/scripts/spark2diffs/GpuHiveOverrides.diff index 764da9001b7..438a088bd3b 100644 --- a/scripts/spark2diffs/GpuHiveOverrides.diff +++ b/scripts/spark2diffs/GpuHiveOverrides.diff @@ -2,22 +2,23 @@ < * Copyright (c) 2020-2022, NVIDIA CORPORATION. --- > * Copyright (c) 2022, NVIDIA CORPORATION. -19c19,21 -< import com.nvidia.spark.rapids.{ExprRule, ShimLoader} +19c19 +< import com.nvidia.spark.rapids.{DataWritingCommandRule, ExprRule, HiveProvider, ShimLoader} --- -> import com.nvidia.spark.RapidsUDF -> import com.nvidia.spark.rapids.{ExprChecks, ExprMeta, ExprRule, GpuOverrides, RapidsConf, RepeatingParamCheck, TypeSig} -> import com.nvidia.spark.rapids.GpuUserDefinedFunction.udfTypeSig -21a24 -> import org.apache.spark.sql.hive.{HiveGenericUDF, HiveSimpleUDF} -26,27c29,30 +> import com.nvidia.spark.rapids.{DataWritingCommandRule, ExprRule, HiveProvider} +27,28c27,28 < ShimLoader.loadClass("org.apache.spark.sql.hive.HiveSessionStateBuilder") < ShimLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf") --- > getClass().getClassLoader.loadClass("org.apache.spark.sql.hive.HiveSessionStateBuilder") > getClass().getClassLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf") -40c43,44 -< ShimLoader.newHiveProvider().getExprs +37c37 +< ShimLoader.newHiveProvider() --- -> // don't use the ShimLoader for Spark 2.x -> new HiveProviderImpl().getExprs +> new HiveProviderImpl() +56,57c56,57 +< * Builds the expression rules that are specific to spark-hive Catalyst nodes +< * This will return an empty mapping if spark-hive is unavailable. +--- +> * Builds the rules that are specific to spark-hive Catalyst nodes. This will return an empty +> * mapping if spark-hive is unavailable. diff --git a/scripts/spark2diffs/GpuOptimizedCreateHiveTableAsSelectCommand.diff b/scripts/spark2diffs/GpuOptimizedCreateHiveTableAsSelectCommand.diff new file mode 100644 index 00000000000..4a45f8db1d8 --- /dev/null +++ b/scripts/spark2diffs/GpuOptimizedCreateHiveTableAsSelectCommand.diff @@ -0,0 +1,143 @@ +21c21,30 +< import org.apache.spark.sql.{SparkSession} +--- +> import scala.util.control.NonFatal +> +> import com.nvidia.spark.rapids.shims.CharVarcharUtilsShims +> +> import org.apache.spark.sql.{SaveMode, SparkSession} +> import org.apache.spark.sql.catalyst.TableIdentifier +> import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SessionCatalog} +> import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +> import org.apache.spark.sql.execution.SparkPlan +> import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand +24c33,145 +< import org.apache.spark.sql.rapids.{GpuOrcFileFormat} +--- +> import org.apache.spark.sql.rapids.{GpuInsertIntoHadoopFsRelationCommand, GpuOrcFileFormat} +> import org.apache.spark.sql.rapids.execution.TrampolineUtil +> import org.apache.spark.sql.rapids.shims.RapidsErrorUtils +> import org.apache.spark.sql.vectorized.ColumnarBatch +> +> /** GPU version of Spark's CreateHiveTableAsSelectBase */ +> trait GpuCreateHiveTableAsSelectBase extends GpuDataWritingCommand { +> val tableDesc: CatalogTable +> val query: LogicalPlan +> val outputColumnNames: Seq[String] +> val mode: SaveMode +> +> protected val tableIdentifier: TableIdentifier = tableDesc.identifier +> +> override def runColumnar(sparkSession: SparkSession, child: SparkPlan): Seq[ColumnarBatch] = { +> val catalog = sparkSession.sessionState.catalog +> val tableExists = catalog.tableExists(tableIdentifier) +> +> if (tableExists) { +> assert(mode != SaveMode.Overwrite, +> s"Expect the table $tableIdentifier has been dropped when the save mode is Overwrite") +> +> if (mode == SaveMode.ErrorIfExists) { +> throw RapidsErrorUtils.tableIdentifierExistsError(tableIdentifier) +> } +> if (mode == SaveMode.Ignore) { +> // Since the table already exists and the save mode is Ignore, we will just return. +> return Seq.empty +> } +> +> val command = getWritingCommand(catalog, tableDesc, tableExists = true) +> command.runColumnar(sparkSession, child) +> GpuDataWritingCommand.propogateMetrics(sparkSession.sparkContext, command, metrics) +> } else { +> tableDesc.storage.locationUri.foreach { p => +> GpuDataWritingCommand.assertEmptyRootPath(p, mode, sparkSession.sessionState.newHadoopConf) +> } +> // TODO ideally, we should get the output data ready first and then +> // add the relation into catalog, just in case of failure occurs while data +> // processing. +> val tableSchema = CharVarcharUtilsShims.getRawSchema( +> outputColumns.toStructType, sparkSession.sessionState.conf) +> assert(tableDesc.schema.isEmpty) +> catalog.createTable( +> tableDesc.copy(schema = tableSchema), ignoreIfExists = false) +> +> try { +> // Read back the metadata of the table which was created just now. +> val createdTableMeta = catalog.getTableMetadata(tableDesc.identifier) +> val command = getWritingCommand(catalog, createdTableMeta, tableExists = false) +> command.runColumnar(sparkSession, child) +> GpuDataWritingCommand.propogateMetrics(sparkSession.sparkContext, command, metrics) +> } catch { +> case NonFatal(e) => +> // drop the created table. +> catalog.dropTable(tableIdentifier, ignoreIfNotExists = true, purge = false) +> throw e +> } +> } +> +> Seq.empty[ColumnarBatch] +> } +> +> // Returns `GpuDataWritingCommand` which actually writes data into the table. +> def getWritingCommand( +> catalog: SessionCatalog, +> tableDesc: CatalogTable, +> tableExists: Boolean): GpuDataWritingCommand +> +> // A subclass should override this with the Class name of the concrete type expected to be +> // returned from `getWritingCommand`. +> def writingCommandClassName: String +> +> override def argString(maxFields: Int): String = { +> s"[Database: ${tableDesc.database}, " + +> s"TableName: ${tableDesc.identifier.table}, " + +> s"$writingCommandClassName]" +> } +> } +> +> case class GpuOptimizedCreateHiveTableAsSelectCommand( +> tableDesc: CatalogTable, +> query: LogicalPlan, +> outputColumnNames: Seq[String], +> mode: SaveMode, +> cpuCmd: OptimizedCreateHiveTableAsSelectCommand) extends GpuCreateHiveTableAsSelectBase { +> override def getWritingCommand( +> catalog: SessionCatalog, +> tableDesc: CatalogTable, +> tableExists: Boolean): GpuDataWritingCommand = { +> // Leverage the existing support for InsertIntoHadoopFsRelationCommand to do the write +> cpuCmd.getWritingCommand(catalog, tableDesc, tableExists) match { +> case c: InsertIntoHadoopFsRelationCommand => +> val rapidsConf = new RapidsConf(conf) +> val rule = GpuOverrides.dataWriteCmds(c.getClass) +> val meta = new InsertIntoHadoopFsRelationCommandMeta(c, rapidsConf, None, rule) +> meta.tagForGpu() +> if (!meta.canThisBeReplaced) { +> throw new IllegalStateException("Unable to convert writing command: " + +> meta.explain(all = false)) +> } +> meta.convertToGpu() +> case c => +> throw new UnsupportedOperationException(s"Unsupported write command: $c") +> } +> } +> +> override def writingCommandClassName: String = +> TrampolineUtil.getSimpleName(classOf[GpuInsertIntoHadoopFsRelationCommand]) +> +> // Do not support partitioned or bucketed writes +> override def requireSingleBatch: Boolean = false +> } +29c150 +< parent: Option[RapidsMeta[_, _]], +--- +> parent: Option[RapidsMeta[_, _, _]], +68a190,198 +> } +> +> override def convertToGpu(): GpuDataWritingCommand = { +> GpuOptimizedCreateHiveTableAsSelectCommand( +> wrapped.tableDesc, +> wrapped.query, +> wrapped.outputColumnNames, +> wrapped.mode, +> cmd) diff --git a/scripts/spark2diffs/GpuOrcScanBase.diff b/scripts/spark2diffs/GpuOrcScanBase.diff index 91c2b24fc46..493a0f6cb08 100644 --- a/scripts/spark2diffs/GpuOrcScanBase.diff +++ b/scripts/spark2diffs/GpuOrcScanBase.diff @@ -1,19 +1,17 @@ -1,2c1,9 +1,2c1,7 < object GpuOrcScan { < // spark 2.x doesn't have data source v2 so not ScanMeta --- > object GpuOrcScan extends Arm { +> > def tagSupport(scanMeta: ScanMeta[OrcScan]): Unit = { > val scan = scanMeta.wrapped > val schema = StructType(scan.readDataSchema ++ scan.readPartitionSchema) -> if (scan.options.getBoolean("mergeSchema", false)) { -> scanMeta.willNotWorkOnGpu("mergeSchema and schema evolution is not supported yet") -> } > tagSupport(scan.sparkSession, schema, scanMeta) > } -7c14 +7c12 < meta: RapidsMeta[_, _]): Unit = { --- > meta: RapidsMeta[_, _, _]): Unit = { -24a32 +19a25 > diff --git a/scripts/spark2diffs/GpuOverrides.diff b/scripts/spark2diffs/GpuOverrides.diff index b0801536e53..af05cdeeed4 100644 --- a/scripts/spark2diffs/GpuOverrides.diff +++ b/scripts/spark2diffs/GpuOverrides.diff @@ -106,7 +106,7 @@ > None 334d344 < } -335a346,365 +335a346,366 > override def convertToGpu(): GpuDataWritingCommand = { > val format = fileFormat.getOrElse( > throw new IllegalStateException("fileFormat missing, tagSelfForGpu not called?")) @@ -124,14 +124,15 @@ > cmd.catalogTable, > cmd.fileIndex, > cmd.outputColumnNames, -> conf.stableSort) +> conf.stableSort, +> conf.concurrentWriterPartitionFlushSize) > } > } -340c370 +340c371 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -345,350c375 +345,350c376 < < // spark 2.3 doesn't have this so just code it here < def sparkSessionActive: SparkSession = { @@ -140,23 +141,23 @@ < } --- > private var gpuProvider: Option[ColumnarFileFormat] = None -360c385 +360c386 < val spark = sparkSessionActive --- > val spark = SparkSession.active -362c387 +362c388 < GpuDataSource.lookupDataSource(cmd.table.provider.get, spark.sessionState.conf) --- > GpuDataSource.lookupDataSourceWithFallback(cmd.table.provider.get, spark.sessionState.conf) -365c390 +365c391 < origProvider.getConstructor().newInstance() match { --- > gpuProvider = origProvider.getConstructor().newInstance() match { -368d392 +368d393 < None -372d395 +372d396 < None -378a402,414 +378a403,416 > override def convertToGpu(): GpuDataWritingCommand = { > val newProvider = gpuProvider.getOrElse( > throw new IllegalStateException("fileFormat unexpected, tagSelfForGpu not called?")) @@ -168,29 +169,30 @@ > cmd.outputColumnNames, > origProvider, > newProvider, -> conf.stableSort) +> conf.stableSort, +> conf.concurrentWriterPartitionFlushSize) > } -380a417,419 +380a419,421 > /** > * Listener trait so that tests can confirm that the expected optimizations are being applied > */ -398d436 +398d438 < -402d439 +402d441 < -406d442 +406d444 < -419,421d454 +419,421d456 < // copy here for 2.x < sealed abstract class Optimization < -423,427d455 +423,427d457 < // Spark 2.x - don't pull in cudf so hardcode here < val DECIMAL32_MAX_PRECISION = 9 < val DECIMAL64_MAX_PRECISION = 18 < val DECIMAL128_MAX_PRECISION = 38 < -480a509,571 +480a511,573 > private def convertPartToGpuIfPossible(part: Partitioning, conf: RapidsConf): Partitioning = { > part match { > case _: GpuPartitioning => part @@ -254,19 +256,15 @@ > } > } > -573c664 +573c666 < case dt: DecimalType if allowDecimal => dt.precision <= GpuOverrides.DECIMAL64_MAX_PRECISION --- > case dt: DecimalType if allowDecimal => dt.precision <= DType.DECIMAL64_MAX_PRECISION -596c687 +596c689 < def checkAndTagFloatAgg(dataType: DataType, conf: RapidsConf, meta: RapidsMeta[_,_]): Unit = { --- > def checkAndTagFloatAgg(dataType: DataType, conf: RapidsConf, meta: RapidsMeta[_,_,_]): Unit = { -610c701 -< meta: RapidsMeta[_,_]): Unit = { ---- -> meta: RapidsMeta[_,_,_]): Unit = { -620a712,743 +605a699,730 > /** > * Helper function specific to ANSI mode for the aggregate functions that should > * fallback, since we don't have the same overflow checks that Spark provides in @@ -299,11 +297,11 @@ > } > } > -624c747 +609c734 < doWrap: (INPUT, RapidsConf, Option[RapidsMeta[_, _]], DataFromReplacementRule) --- > doWrap: (INPUT, RapidsConf, Option[RapidsMeta[_, _, _]], DataFromReplacementRule) -631a755,764 +616a742,751 > def scan[INPUT <: Scan]( > desc: String, > doWrap: (INPUT, RapidsConf, Option[RapidsMeta[_, _, _]], DataFromReplacementRule) @@ -314,60 +312,62 @@ > new ScanRule[INPUT](doWrap, desc, tag) > } > -635c768 +620c755 < doWrap: (INPUT, RapidsConf, Option[RapidsMeta[_, _]], DataFromReplacementRule) --- > doWrap: (INPUT, RapidsConf, Option[RapidsMeta[_, _, _]], DataFromReplacementRule) -653c786 +638c773 < p: Option[RapidsMeta[_, _]], --- > p: Option[RapidsMeta[_, _, _]], -662c795 +647c782 < doWrap: (INPUT, RapidsConf, Option[RapidsMeta[_, _]], DataFromReplacementRule) --- > doWrap: (INPUT, RapidsConf, Option[RapidsMeta[_, _, _]], DataFromReplacementRule) -672c805 +657c792 < doWrap: (INPUT, RapidsConf, Option[RapidsMeta[_, _]], DataFromReplacementRule) --- > doWrap: (INPUT, RapidsConf, Option[RapidsMeta[_, _, _]], DataFromReplacementRule) -683c816 +668c803 < parent: Option[RapidsMeta[_, _]]): BaseExprMeta[INPUT] = --- > parent: Option[RapidsMeta[_, _, _]]): BaseExprMeta[INPUT] = -741a875 +728a864 > override def convertToGpu(child: Expression): GpuExpression = GpuSignum(child) -750a885,886 +738a875,876 > override def convertToGpu(child: Expression): GpuExpression = > GpuAlias(child, a.name)(a.exprId, a.qualifier, a.explicitMetadata) -761a898 +749a888 > override def convertToGpu(): Expression = att -775a913 +763a903 > override def convertToGpu(child: Expression): GpuExpression = GpuPromotePrecision(child) -793,794c931 +781,782c921 < // allowNegativeScaleOfDecimalEnabled is not in 2.x assume its default false < val t = if (s < 0 && !false) { --- > val t = if (s < 0 && !SQLConf.get.allowNegativeScaleOfDecimalEnabled) { -806,807c943 +794,795c933 < // Spark 2.X only has Cast, no AnsiCast so no CastBase, hardcode here to Cast < case p: PromotePrecision if p.child.isInstanceOf[Cast] && --- > case p: PromotePrecision if p.child.isInstanceOf[CastBase] && -809c945 +797c935 < val c = p.child.asInstanceOf[Cast] --- > val c = p.child.asInstanceOf[CastBase] -865c1001 -< if (intermediatePrecision > GpuOverrides.DECIMAL128_MAX_PRECISION) { ---- -> if (intermediatePrecision > DType.DECIMAL128_MAX_PRECISION) { -880c1016 -< if (intermediatePrecision > GpuOverrides.DECIMAL128_MAX_PRECISION) { ---- -> if (intermediatePrecision > DType.DECIMAL128_MAX_PRECISION) { -893a1030,1047 -> +836a975,1002 > override def convertToGpu(): GpuExpression = { +> // Prior to Spark 3.4.0 +> // Division and Multiplication of Decimal types is a little odd. Spark will cast the +> // inputs to a common wider value where the scale is the max of the two input scales, +> // and the precision is max of the two input non-scale portions + the new scale. Then it +> // will do the divide or multiply as a BigDecimal value but lie about the return type. +> // Finally here in CheckOverflow it will reset the scale and check the precision so that +> // Spark knows it fits in the final desired result. +> // Here we try to strip out the extra casts, etc to get to as close to the original +> // query as possible. This lets us then calculate what CUDF needs to get the correct +> // answer, which in some cases is a lot smaller. +> > a.child match { > case _: Divide => > // GpuDecimalDivide includes the overflow check in it. @@ -378,68 +378,68 @@ > GpuDecimalMultiply.nonRoundedIntermediatePrecision(lhsDecimalType, > rhsDecimalType, a.dataType) > GpuDecimalMultiply(lhs.convertToGpu(), rhs.convertToGpu(), wrapped.dataType, -> needsExtraOverflowChecks = intermediatePrecision > DType.DECIMAL128_MAX_PRECISION) +> useLongMultiply = intermediatePrecision > DType.DECIMAL128_MAX_PRECISION) > case _ => > GpuCheckOverflow(childExprs.head.convertToGpu(), > wrapped.dataType, wrapped.nullOnOverflow) > } > } -898a1053 +841a1008 > override def convertToGpu(child: Expression): GpuToDegrees = GpuToDegrees(child) -903a1059 +846a1014 > override def convertToGpu(child: Expression): GpuToRadians = GpuToRadians(child) -942a1099 +885a1054 > override def convertToGpu(): GpuExpression = GpuSpecialFrameBoundary(currentRow) -948a1106 +891a1061 > override def convertToGpu(): GpuExpression = GpuSpecialFrameBoundary(unboundedPreceding) -954a1113 +897a1068 > override def convertToGpu(): GpuExpression = GpuSpecialFrameBoundary(unboundedFollowing) -959a1119 +902a1074 > override def convertToGpu(): GpuExpression = GpuRowNumber -968a1129 +911a1084 > override def convertToGpu(): GpuExpression = GpuRank(childExprs.map(_.convertToGpu())) -977a1139 +920a1094 > override def convertToGpu(): GpuExpression = GpuDenseRank(childExprs.map(_.convertToGpu())) -986a1149,1150 +929a1104,1105 > override def convertToGpu(): GpuExpression = > GpuPercentRank(childExprs.map(_.convertToGpu())) -1006a1171,1172 +949a1126,1127 > override def convertToGpu(): GpuExpression = > GpuLead(input.convertToGpu(), offset.convertToGpu(), default.convertToGpu()) -1026a1193,1194 +969a1148,1149 > override def convertToGpu(): GpuExpression = > GpuLag(input.convertToGpu(), offset.convertToGpu(), default.convertToGpu()) -1036a1205,1206 +979a1160,1161 > override def convertToGpu(child: Expression): GpuExpression = > GpuPreciseTimestampConversion(child, a.fromType, a.toType) -1045,1046c1215 +988,989c1170 < // val ansiEnabled = SQLConf.get.ansiEnabled < val ansiEnabled = false --- > val ansiEnabled = SQLConf.get.ansiEnabled -1049,1050d1217 +992,993d1172 < // Spark 2.x - ansi in not in 2.x < /* -1054,1055d1220 +997,998d1175 < < */ -1056a1222,1224 +999a1177,1179 > > override def convertToGpu(child: Expression): GpuExpression = > GpuUnaryMinus(child, ansiEnabled) -1064a1233 +1007a1188 > override def convertToGpu(child: Expression): GpuExpression = GpuUnaryPositive(child) -1069a1239 +1012a1194 > override def convertToGpu(child: Expression): GpuExpression = GpuYear(child) -1074a1245 +1017a1200 > override def convertToGpu(child: Expression): GpuExpression = GpuMonth(child) -1079a1251 +1022a1206 > override def convertToGpu(child: Expression): GpuExpression = GpuQuarter(child) -1084a1257 +1027a1212 > override def convertToGpu(child: Expression): GpuExpression = GpuDayOfMonth(child) -1089a1263 +1032a1218 > override def convertToGpu(child: Expression): GpuExpression = GpuDayOfYear(child) -1094a1269,1280 +1037a1224,1235 > override def convertToGpu(child: Expression): GpuExpression = GpuAcos(child) > }), > expr[Acosh]( @@ -452,7 +452,9 @@ > } else { > GpuAcoshCompat(child) > } -1099a1286,1306 +1039d1236 +< // Acosh is not supported in spark 2.x +1043a1241,1261 > override def convertToGpu(child: Expression): GpuExpression = GpuAsin(child) > }), > expr[Asinh]( @@ -474,80 +476,82 @@ > s"${RapidsConf.IMPROVED_FLOAT_OPS.key} is enabled") > } > } -1104a1312 +1045d1262 +< // Asinh is not supported in spark 2.x +1049a1267 > override def convertToGpu(child: Expression): GpuExpression = GpuSqrt(child) -1109a1318 +1054a1273 > override def convertToGpu(child: Expression): GpuExpression = GpuCbrt(child) -1118a1328,1329 +1063a1283,1284 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuHypot(lhs, rhs) -1130c1341 +1075c1296 < if (precision > GpuOverrides.DECIMAL128_MAX_PRECISION) { --- > if (precision > DType.DECIMAL128_MAX_PRECISION) { -1137a1349,1352 +1082a1304,1307 > override def convertToGpu(child: Expression): GpuExpression = { > // use Spark `Floor.dataType` to keep consistent between Spark versions. > GpuFloor(child, a.dataType) > } -1149c1364 +1094c1319 < if (precision > GpuOverrides.DECIMAL128_MAX_PRECISION) { --- > if (precision > DType.DECIMAL128_MAX_PRECISION) { -1156a1372,1375 +1101a1327,1330 > override def convertToGpu(child: Expression): GpuExpression = { > // use Spark `Ceil.dataType` to keep consistent between Spark versions. > GpuCeil(child, a.dataType) > } -1162a1382 +1107a1337 > override def convertToGpu(child: Expression): GpuExpression = GpuNot(child) -1171a1392 +1116a1347 > override def convertToGpu(child: Expression): GpuExpression = GpuIsNull(child) -1180a1402 +1125a1357 > override def convertToGpu(child: Expression): GpuExpression = GpuIsNotNull(child) -1186a1409 +1131a1364 > override def convertToGpu(child: Expression): GpuExpression = GpuIsNan(child) -1191a1415 +1136a1370 > override def convertToGpu(child: Expression): GpuExpression = GpuRint(child) -1197a1422 +1142a1377 > override def convertToGpu(child: Expression): GpuExpression = GpuBitwiseNot(child) -1206a1432,1433 +1151a1387,1388 > def convertToGpu(): GpuExpression = > GpuAtLeastNNonNulls(a.n, childExprs.map(_.convertToGpu())) -1215a1443,1444 +1160a1398,1399 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuDateAdd(lhs, rhs) -1224a1454,1455 +1169a1409,1410 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuDateSub(lhs, rhs) -1231a1463,1464 +1176a1418,1419 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuNaNvl(lhs, rhs) -1238a1472,1473 +1183a1427,1428 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuShiftLeft(lhs, rhs) -1245a1481,1482 +1190a1436,1437 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuShiftRight(lhs, rhs) -1252a1490,1491 +1197a1445,1446 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuShiftRightUnsigned(lhs, rhs) -1260a1500,1501 +1205a1455,1456 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuBitwiseAnd(lhs, rhs) -1268a1510,1511 +1213a1465,1466 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuBitwiseOr(lhs, rhs) -1276a1520,1521 +1221a1475,1476 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuBitwiseXor(lhs, rhs) -1288a1534 +1233a1489 > override def convertToGpu(): GpuExpression = GpuCoalesce(childExprs.map(_.convertToGpu())) -1297a1544 +1242a1499 > override def convertToGpu(): GpuExpression = GpuLeast(childExprs.map(_.convertToGpu())) -1306a1554 +1251a1509 > override def convertToGpu(): GpuExpression = GpuGreatest(childExprs.map(_.convertToGpu())) -1311a1560,1566 +1256a1515,1521 > override def convertToGpu(child: Expression): GpuExpression = GpuAtan(child) > }), > expr[Atanh]( @@ -555,43 +559,45 @@ > ExprChecks.mathUnaryWithAst, > (a, conf, p, r) => new UnaryAstExprMeta[Atanh](a, conf, p, r) { > override def convertToGpu(child: Expression): GpuExpression = GpuAtanh(child) -1316a1572 +1258d1522 +< // Atanh is not supported in spark 2.x +1262a1527 > override def convertToGpu(child: Expression): GpuExpression = GpuCos(child) -1321a1578 +1267a1533 > override def convertToGpu(child: Expression): GpuExpression = GpuExp(child) -1326a1584 +1272a1539 > override def convertToGpu(child: Expression): GpuExpression = GpuExpm1(child) -1332a1591 +1278a1546 > override def convertToGpu(child: Expression): GpuExpression = GpuInitCap(child) -1337a1597 +1283a1552 > override def convertToGpu(child: Expression): GpuExpression = GpuLog(child) -1342a1603,1607 +1288a1558,1562 > override def convertToGpu(child: Expression): GpuExpression = { > // No need for overflow checking on the GpuAdd in Double as Double handles overflow > // the same in all modes. > GpuLog(GpuAdd(child, GpuLiteral(1d, DataTypes.DoubleType), false)) > } -1347a1613,1614 +1293a1568,1569 > override def convertToGpu(child: Expression): GpuExpression = > GpuLogarithm(child, GpuLiteral(2d, DataTypes.DoubleType)) -1352a1620,1621 +1298a1575,1576 > override def convertToGpu(child: Expression): GpuExpression = > GpuLogarithm(child, GpuLiteral(10d, DataTypes.DoubleType)) -1359a1629,1631 +1305a1584,1586 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > // the order of the parameters is transposed intentionally > GpuLogarithm(rhs, lhs) -1364a1637 +1310a1592 > override def convertToGpu(child: Expression): GpuExpression = GpuSin(child) -1369a1643 +1315a1598 > override def convertToGpu(child: Expression): GpuExpression = GpuSinh(child) -1374a1649 +1320a1604 > override def convertToGpu(child: Expression): GpuExpression = GpuCosh(child) -1379a1655 +1325a1610 > override def convertToGpu(child: Expression): GpuExpression = GpuCot(child) -1384a1661 +1330a1616 > override def convertToGpu(child: Expression): GpuExpression = GpuTanh(child) -1389a1667,1685 +1335a1622,1638 > override def convertToGpu(child: Expression): GpuExpression = GpuTan(child) > }), > expr[NormalizeNaNAndZero]( @@ -605,23 +611,26 @@ > }), > expr[KnownFloatingPointNormalized]( > "Tag to prevent redundant normalization", -> ExprChecks.unaryProjectInputMatchesOutput( -> TypeSig.DOUBLE + TypeSig.FLOAT, -> TypeSig.DOUBLE + TypeSig.FLOAT), +> ExprChecks.unaryProjectInputMatchesOutput(TypeSig.all, TypeSig.all), > (a, conf, p, r) => new UnaryExprMeta[KnownFloatingPointNormalized](a, conf, p, r) { > override def convertToGpu(child: Expression): GpuExpression = > GpuKnownFloatingPointNormalized(child) -1396a1693,1694 +1337,1338d1639 +< // NormalizeNaNAndZero is not supported in spark 2.x +< // KnownFloatingPointNormalized is not supported in spark 2.x +1344a1646,1647 > override def convertToGpu(child: Expression): GpuExpression = > GpuKnownNotNull(child) -1403a1702,1704 +1351a1655,1657 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > GpuDateDiff(lhs, rhs) > } -1421a1723,1724 +1369a1676,1677 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuTimeAdd(lhs, rhs) -1422a1726,1746 +1371c1679,1699 +< // DateAddInterval is not supported in spark 2.x +--- > expr[DateAddInterval]( > "Adds interval to date", > ExprChecks.binaryProject(TypeSig.DATE, TypeSig.DATE, @@ -643,10 +652,10 @@ > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuDateAddInterval(lhs, rhs) > }), -1430a1755,1756 +1379a1708,1709 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuDateFormatClass(lhs, rhs, strfFormat) -1442a1769,1776 +1391a1722,1729 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > if (conf.isImprovedTimestampOpsEnabled) { > // passing the already converted strf string for a little optimization @@ -655,7 +664,7 @@ > GpuToUnixTimestamp(lhs, rhs, sparkFormat, strfFormat) > } > } -1453a1788,1795 +1402a1741,1748 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > if (conf.isImprovedTimestampOpsEnabled) { > // passing the already converted strf string for a little optimization @@ -664,59 +673,59 @@ > GpuUnixTimestamp(lhs, rhs, sparkFormat, strfFormat) > } > } -1460a1803 +1409a1756 > override def convertToGpu(expr: Expression): GpuExpression = GpuHour(expr) -1466a1810,1812 +1415a1763,1765 > > override def convertToGpu(expr: Expression): GpuExpression = > GpuMinute(expr) -1472a1819,1821 +1421a1772,1774 > > override def convertToGpu(expr: Expression): GpuExpression = > GpuSecond(expr) -1478a1828,1829 +1427a1781,1782 > override def convertToGpu(expr: Expression): GpuExpression = > GpuWeekDay(expr) -1484a1836,1837 +1433a1789,1790 > override def convertToGpu(expr: Expression): GpuExpression = > GpuDayOfWeek(expr) -1489a1843,1844 +1438a1796,1797 > override def convertToGpu(expr: Expression): GpuExpression = > GpuLastDay(expr) -1498a1854,1856 +1447a1807,1809 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > // passing the already converted strf string for a little optimization > GpuFromUnixTime(lhs, rhs, strfFormat) -1505a1864,1865 +1472a1835,1836 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuPmod(lhs, rhs) -1518c1878 +1485c1849 < private val ansiEnabled = false --- > private val ansiEnabled = SQLConf.get.ansiEnabled -1520a1881,1883 +1487a1852,1854 > if (ansiEnabled && GpuAnsi.needBasicOpOverflowCheck(a.dataType)) { > willNotWorkInAst("AST Addition does not support ANSI mode.") > } -1522a1886,1887 +1489a1857,1858 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuAdd(lhs, rhs, failOnError = ansiEnabled) -1535c1900 +1502c1871 < private val ansiEnabled = false --- > private val ansiEnabled = SQLConf.get.ansiEnabled -1537a1903,1905 +1504a1874,1876 > if (ansiEnabled && GpuAnsi.needBasicOpOverflowCheck(a.dataType)) { > willNotWorkInAst("AST Subtraction does not support ANSI mode.") > } -1539a1908,1909 +1506a1879,1880 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuSubtract(lhs, rhs, ansiEnabled) -1552a1923,1925 +1519a1894,1896 > if (SQLConf.get.ansiEnabled && GpuAnsi.needBasicOpOverflowCheck(a.dataType)) { > willNotWorkOnGpu("GPU Multiplication does not support ANSI mode") > } -1554a1928,1935 +1521a1899,1906 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > a.dataType match { > case _: DecimalType => throw new IllegalStateException( @@ -725,37 +734,37 @@ > GpuMultiply(lhs, rhs) > } > } -1561a1943,1944 +1528a1914,1915 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuAnd(lhs, rhs) -1568a1952,1953 +1535a1923,1924 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuOr(lhs, rhs) -1580a1966,1967 +1547a1937,1938 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuEqualNullSafe(lhs, rhs) -1593a1981,1982 +1560a1952,1953 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuEqualTo(lhs, rhs) -1606a1996,1997 +1573a1967,1968 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuGreaterThan(lhs, rhs) -1619a2011,2012 +1586a1982,1983 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuGreaterThanOrEqual(lhs, rhs) -1639a2033,2034 +1606a2004,2005 > override def convertToGpu(): GpuExpression = > GpuInSet(childExprs.head.convertToGpu(), in.list.asInstanceOf[Seq[Literal]].map(_.value)) -1650a2046,2047 +1617a2017,2018 > override def convertToGpu(): GpuExpression = > GpuInSet(childExprs.head.convertToGpu(), in.hset.toSeq) -1663a2061,2062 +1630a2032,2033 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuLessThan(lhs, rhs) -1676a2076,2077 +1643a2047,2048 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuLessThanOrEqual(lhs, rhs) -1681a2083,2094 +1648a2054,2065 > override def convertToGpu(): GpuExpression = { > val branches = childExprs.grouped(2).flatMap { > case Seq(cond, value) => Some((cond.convertToGpu(), value.convertToGpu())) @@ -768,15 +777,15 @@ > } > GpuCaseWhen(branches, elseValue) > } -1698a2112,2115 +1665a2083,2086 > override def convertToGpu(): GpuExpression = { > val Seq(boolExpr, trueExpr, falseExpr) = childExprs.map(_.convertToGpu()) > GpuIf(boolExpr, trueExpr, falseExpr) > } -1706a2124,2125 +1673a2095,2096 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuPow(lhs, rhs) -1720a2140,2159 +1684a2108,2127 > // Division of Decimal types is a little odd. To work around some issues with > // what Spark does the tagging/checks are in CheckOverflow instead of here. > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = @@ -797,15 +806,17 @@ > (a, conf, p, r) => new BinaryExprMeta[IntegralDivide](a, conf, p, r) { > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuIntegralDivide(lhs, rhs) -1728a2168,2169 +1686d2128 +< // IntegralDivide is not supported in spark 2.x +1693a2136,2137 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuRemainder(lhs, rhs) -1743c2184,2185 +1708c2152,2153 < // No filter parameter in 2.x --- > private val filter: Option[BaseExprMeta[_]] = > a.filter.map(GpuOverrides.wrapExpr(_, conf, Some(this))) -1747c2189,2204 +1712c2157,2172 < childrenExprMeta --- > childrenExprMeta ++ filter.toSeq @@ -824,11 +835,11 @@ > GpuAggregateExpression(childExprs.head.convertToGpu().asInstanceOf[GpuAggregateFunction], > a.mode, a.isDistinct, filter.map(_.convertToGpu()), resultId) > } -1770a2228,2230 +1735a2196,2198 > // One of the few expressions that are not replaced with a GPU version > override def convertToGpu(): Expression = > sortOrder.withNewChildren(childExprs.map(_.convertToGpu())) -1793a2254,2260 +1757a2221,2227 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = { > val Seq(pivotColumn, valueColumn) = childExprs > GpuPivotFirst(pivotColumn, valueColumn, pivot.pivotColumnValues) @@ -836,24 +847,22 @@ > > // Pivot does not overflow, so it doesn't need the ANSI check > override val needsAnsiCheck: Boolean = false -1806a2274,2275 +1770a2241,2242 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = > GpuCount(childExprs) -1833a2303,2308 -> +1791a2264,2268 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = > GpuMax(childExprs.head) > > // Max does not overflow, so it doesn't need the ANSI check > override val needsAnsiCheck: Boolean = false -1860a2336,2341 -> +1812a2290,2294 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = > GpuMin(childExprs.head) > > // Min does not overflow, so it doesn't need the ANSI check > override val needsAnsiCheck: Boolean = false -1873a2355,2375 +1825a2308,2328 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = > GpuSum(childExprs.head, a.dataType) > }), @@ -875,87 +884,87 @@ > > // nth does not overflow, so it doesn't need the ANSI check > override val needsAnsiCheck: Boolean = false -1875d2376 +1827d2329 < // Spark 2.x doesn't have NthValue -1887a2389,2393 +1839a2342,2346 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = > GpuFirst(childExprs.head, a.ignoreNulls) > > // First does not overflow, so it doesn't need the ANSI check > override val needsAnsiCheck: Boolean = false -1890c2396 +1842c2349 < "last aggregate operator", --- > "last aggregate operator", -1900a2407,2411 +1852a2360,2364 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = > GpuLast(childExprs.head, a.ignoreNulls) > > // Last does not overflow, so it doesn't need the ANSI check > override val needsAnsiCheck: Boolean = false -1919a2431,2432 +1871a2384,2385 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuBRound(lhs, rhs, a.dataType) -1938a2452,2453 +1890a2405,2406 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuRound(lhs, rhs, a.dataType) -1960a2476,2480 +1912a2429,2433 > > override def convertToGpu(): GpuExpression = > GpuPythonUDF(a.name, a.func, a.dataType, > childExprs.map(_.convertToGpu()), > a.evalType, a.udfDeterministic, a.resultId) -1969a2490 +1921a2443 > override def convertToGpu(child: Expression): GpuExpression = GpuRand(child) -1974a2496 +1926a2449 > override def convertToGpu(): GpuExpression = GpuSparkPartitionID() -1979a2502 +1931a2455 > override def convertToGpu(): GpuExpression = GpuMonotonicallyIncreasingID() -1984a2508 +1936a2461 > override def convertToGpu(): GpuExpression = GpuInputFileName() -1989a2514 +1941a2467 > override def convertToGpu(): GpuExpression = GpuInputFileBlockStart() -1994a2520 +1946a2473 > override def convertToGpu(): GpuExpression = GpuInputFileBlockLength() -2000a2527 +1952a2480 > override def convertToGpu(child: Expression): GpuExpression = GpuMd5(child) -2005a2533 +1957a2486 > override def convertToGpu(child: Expression): GpuExpression = GpuUpper(child) -2011a2540 +1963a2493 > override def convertToGpu(child: Expression): GpuExpression = GpuLower(child) -2028a2558,2562 +1980a2511,2515 > override def convertToGpu( > str: Expression, > width: Expression, > pad: Expression): GpuExpression = > GpuStringLPad(str, width, pad) -2044a2579,2583 +1996a2532,2536 > override def convertToGpu( > str: Expression, > width: Expression, > pad: Expression): GpuExpression = > GpuStringRPad(str, width, pad) -2065a2605,2606 +2017a2558,2559 > override def convertToGpu(arr: Expression): GpuExpression = > GpuGetStructField(arr, expr.ordinal, expr.name) -2077a2619,2620 +2029a2572,2573 > override def convertToGpu(arr: Expression, ordinal: Expression): GpuExpression = > GpuGetArrayItem(arr, ordinal, in.failOnError) -2088a2632,2633 +2047a2592,2593 > override def convertToGpu(map: Expression, key: Expression): GpuExpression = > GpuGetMapValue(map, key, in.failOnError) -2092c2637 +2051c2597 < "Returns value for the given key in value if column is map", --- > "Returns value for the given key in value if column is map.", -2136a2682,2684 +2096a2643,2645 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > GpuElementAt(lhs, rhs, failOnError = in.failOnError) > } -2147a2696,2697 +2107a2657,2658 > override def convertToGpu(child: Expression): GpuExpression = > GpuMapKeys(child) -2158a2709,2724 +2118a2670,2685 > override def convertToGpu(child: Expression): GpuExpression = > GpuMapValues(child) > }), @@ -964,31 +973,32 @@ > ExprChecks.unaryProject( > // Technically the return type is an array of struct, but we cannot really express that > TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + -> TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), +> TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY), > TypeSig.ARRAY.nested(TypeSig.all), > TypeSig.MAP.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + -> TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), +> TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY), > TypeSig.MAP.nested(TypeSig.all)), > (in, conf, p, r) => new UnaryExprMeta[MapEntries](in, conf, p, r) { > override def convertToGpu(child: Expression): GpuExpression = > GpuMapEntries(child) -2178a2745,2747 -> +2120d2686 +< // MapEntries is not supported in spark 2.x +2135a2702,2703 > override def convertToGpu(child: Expression): GpuExpression = > GpuArrayMin(child) -2192a2762,2763 +2144a2713,2714 > override def convertToGpu(child: Expression): GpuExpression = > GpuArrayMax(child) -2203a2775,2776 +2155a2726,2727 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuArrayRepeat(lhs, rhs) -2209a2783,2784 +2161a2734,2735 > override def convertToGpu(): GpuExpression = > GpuCreateNamedStruct(childExprs.map(_.convertToGpu())) -2225a2801,2802 +2177a2752,2753 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuArrayContains(lhs, rhs) -2238c2815,2819 +2190c2766,2770 < }), --- > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { @@ -996,10 +1006,10 @@ > } > } > ), -2263a2845,2846 +2215a2796,2797 > override def convertToGpu(): GpuExpression = > GpuCreateArray(childExprs.map(_.convertToGpu()), wrapped.useStringTypeWhenEmpty) -2279a2863,2869 +2231a2814,2820 > override def convertToGpu(): GpuExpression = { > val func = childExprs.head > val args = childExprs.tail @@ -1007,15 +1017,15 @@ > args.map(_.convertToGpu().asInstanceOf[NamedExpression]), > in.hidden) > } -2287a2878,2880 +2239a2829,2831 > override def convertToGpu(): GpuExpression = { > GpuNamedLambdaVariable(in.name, in.dataType, in.nullable, in.exprId) > } -2304a2898,2900 +2256a2849,2851 > override def convertToGpu(): GpuExpression = { > GpuArrayTransform(childExprs.head.convertToGpu(), childExprs(1).convertToGpu()) > } -2315a2912,2918 +2267a2863,2869 > override def convertToGpu(): GpuExpression = { > GpuArrayExists( > childExprs.head.convertToGpu(), @@ -1023,15 +1033,15 @@ > SQLConf.get.getConf(SQLConf.LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC) > ) > } -2329a2933,2935 +2281a2884,2886 > override def convertToGpu(): GpuExpression = { > GpuArraysZip(childExprs.map(_.convertToGpu())) > } -2343a2950,2952 +2295a2901,2903 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > GpuArrayExcept(lhs, rhs) > } -2362a2972,2995 +2314a2923,2946 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > GpuArrayIntersect(lhs, rhs) > } @@ -1056,17 +1066,15 @@ > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > GpuArrayUnion(lhs, rhs) > } -2369d3001 +2321d2952 < // ArrayUnion is not supported in Spark 2.x -2372c3004 -< "have no common element and they are both non-empty and either of them contains a null " + ---- -> "have no common element and they are both non-empty and either of them contains a null " + -2381a3014,3016 +2333a2965,2967 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > GpuArraysOverlap(lhs, rhs) > } -2388c3023,3084 +2340,2342c2974,3035 +< // TransformKeys is not supported in Spark 2.x +< // TransformValues is not supported in Spark 2.x < // spark 2.x doesn't have MapFilter --- > expr[TransformKeys]( @@ -1131,66 +1139,66 @@ > GpuMapFilter(childExprs.head.convertToGpu(), childExprs(1).convertToGpu()) > } > }), -2395a3092,3096 +2349a3043,3047 > override def convertToGpu( > val0: Expression, > val1: Expression, > val2: Expression): GpuExpression = > GpuStringLocate(val0, val1, val2) -2403a3105,3109 +2357a3056,3060 > override def convertToGpu( > column: Expression, > position: Expression, > length: Expression): GpuExpression = > GpuSubstring(column, position, length) -2419a3126,3128 +2373a3077,3079 > override def convertToGpu( > input: Expression, > repeatTimes: Expression): GpuExpression = GpuStringRepeat(input, repeatTimes) -2427a3137,3141 +2381a3088,3092 > override def convertToGpu( > column: Expression, > target: Expression, > replace: Expression): GpuExpression = > GpuStringReplace(column, target, replace) -2435a3150,3153 +2389a3101,3104 > override def convertToGpu( > column: Expression, > target: Option[Expression] = None): GpuExpression = > GpuStringTrim(column, target) -2444a3163,3166 +2398a3114,3117 > override def convertToGpu( > column: Expression, > target: Option[Expression] = None): GpuExpression = > GpuStringTrimLeft(column, target) -2453a3176,3179 +2407a3127,3130 > override def convertToGpu( > column: Expression, > target: Option[Expression] = None): GpuExpression = > GpuStringTrimRight(column, target) -2460a3187,3188 +2414a3138,3139 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuStartsWith(lhs, rhs) -2467a3196,3197 +2421a3147,3148 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuEndsWith(lhs, rhs) -2480a3211 +2434a3162 > override def convertToGpu(child: Seq[Expression]): GpuExpression = GpuConcat(child) -2502a3234 +2445a3174 > override def convertToGpu(child: Seq[Expression]): GpuExpression = GpuMapConcat(child) -2519a3252,3253 +2462a3192,3193 > override final def convertToGpu(): GpuExpression = > GpuConcatWs(childExprs.map(_.convertToGpu())) -2529a3264,3265 +2472a3204,3205 > def convertToGpu(): GpuExpression = > GpuMurmur3Hash(childExprs.map(_.convertToGpu()), a.seed) -2536a3273,3274 +2479a3213,3214 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuContains(lhs, rhs) -2543a3282,3283 +2486a3222,3223 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuLike(lhs, rhs, a.escapeChar) -2569c3309,3316 +2512c3249,3256 < // RegExpExtractAll is not supported in Spark 2.x --- > expr[RegExpExtractAll]( @@ -1201,21 +1209,21 @@ > ParamCheck("regexp", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING), > ParamCheck("idx", TypeSig.lit(TypeEnum.INT), TypeSig.INT))), > (a, conf, p, r) => new GpuRegExpExtractAllMeta(a, conf, p, r)), -2574a3322 +2517a3262 > override def convertToGpu(child: Expression): GpuExpression = GpuLength(child) -2582a3331,3332 +2525a3271,3272 > override def convertToGpu(child: Expression): GpuExpression = > GpuSize(child, a.legacySizeOfNull) -2589a3340 +2532a3280 > override def convertToGpu(child: Expression): GpuExpression = GpuUnscaledValue(child) -2595a3347,3348 +2538a3287,3288 > override def convertToGpu(child: Expression): GpuExpression = > GpuMakeDecimal(child, a.precision, a.scale, a.nullOnOverflow) -2609a3363 +2552a3303 > override def convertToGpu(): GpuExpression = GpuExplode(childExprs.head.convertToGpu()) -2623a3378 +2566a3318 > override def convertToGpu(): GpuExpression = GpuPosExplode(childExprs.head.convertToGpu()) -2636,2637c3391,3453 +2579,2580c3331,3411 < }), < // spark 2.x CollectList and CollectSet use TypedImperative which isn't in 2.x --- @@ -1256,13 +1264,31 @@ > "Collect a set of unique elements, not supported in reduction", > ExprChecks.fullAgg( > TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + -> TypeSig.NULL + TypeSig.STRUCT), +> TypeSig.NULL + TypeSig.STRUCT + TypeSig.ARRAY), > TypeSig.ARRAY.nested(TypeSig.all), > Seq(ParamCheck("input", > (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + -> TypeSig.NULL + TypeSig.STRUCT).nested(), +> TypeSig.NULL + +> TypeSig.STRUCT + +> TypeSig.ARRAY).nested(), > TypeSig.all))), > (c, conf, p, r) => new TypedImperativeAggExprMeta[CollectSet](c, conf, p, r) { +> +> private def isNestedArrayType(dt: DataType): Boolean = { +> dt match { +> case StructType(fields) => +> fields.exists { field => +> field.dataType match { +> case sdt: StructType => isNestedArrayType(sdt) +> case _: ArrayType => true +> case _ => false +> } +> } +> case ArrayType(et, _) => et.isInstanceOf[ArrayType] || et.isInstanceOf[StructType] +> case _ => false +> } +> } +> > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = > GpuCollectSet(childExprs.head, c.mutableAggBufferOffset, c.inputAggBufferOffset) > @@ -1282,53 +1308,53 @@ > // Last does not overflow, so it doesn't need the ANSI check > override val needsAnsiCheck: Boolean = false > }), -2643a3460,3463 +2586a3418,3421 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = { > val legacyStatisticalAggregate = SQLConf.get.legacyStatisticalAggregate > GpuStddevPop(childExprs.head, !legacyStatisticalAggregate) > } -2651a3472,3475 +2594a3430,3433 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = { > val legacyStatisticalAggregate = SQLConf.get.legacyStatisticalAggregate > GpuStddevSamp(childExprs.head, !legacyStatisticalAggregate) > } -2658a3483,3486 +2601a3441,3444 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = { > val legacyStatisticalAggregate = SQLConf.get.legacyStatisticalAggregate > GpuVariancePop(childExprs.head, !legacyStatisticalAggregate) > } -2665a3494,3497 +2608a3452,3455 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = { > val legacyStatisticalAggregate = SQLConf.get.legacyStatisticalAggregate > GpuVarianceSamp(childExprs.head, !legacyStatisticalAggregate) > } -2705a3538,3543 +2648a3496,3501 > > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = > GpuApproximatePercentile(childExprs.head, > childExprs(1).asInstanceOf[GpuLiteral], > childExprs(2).asInstanceOf[GpuLiteral]) > -2720a3559,3560 +2663a3517,3518 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuGetJsonObject(lhs, rhs) -2732c3572,3574 +2675c3530,3532 < }), --- > override def convertToGpu(): GpuExpression = GpuScalarSubquery(a.plan, a.exprId) > } > ), -2737c3579,3581 +2680c3537,3539 < }), --- > override def convertToGpu(): GpuExpression = GpuCreateMap(childExprs.map(_.convertToGpu())) > } > ), -2755a3600 +2698a3558 > override def convertToGpu(child: Expression): GpuExpression = GpuBitLength(child) -2762a3608 +2705a3566 > override def convertToGpu(child: Expression): GpuExpression = GpuOctetLength(child) -2776,2778c3622,3632 +2719,2721c3580,3590 < ) < // Spark 2.x doesn't have RaiseError or ansicast < ).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap @@ -1344,7 +1370,7 @@ > }), > SparkShimImpl.ansiCastRule > ).collect { case r if r != null => (r.getClassFor.asSubclass(classOf[Expression]), r)}.toMap -2783c3637,3684 +2726c3595,3642 < ShimGpuOverrides.shimExpressions --- > SparkShimImpl.getExprs @@ -1395,14 +1421,14 @@ > > val scans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = > commonScans ++ SparkShimImpl.getScans ++ ExternalSource.getScans -2788c3689 +2731c3647 < parent: Option[RapidsMeta[_, _]]): PartMeta[INPUT] = --- > parent: Option[RapidsMeta[_, _, _]]): PartMeta[INPUT] = -2803a3705,3706 +2761a3678,3679 > override def convertToGpu(): GpuPartitioning = > GpuHashPartitioning(childExprs.map(_.convertToGpu()), hp.numPartitions) -2813a3717,3724 +2771a3690,3697 > override def convertToGpu(): GpuPartitioning = { > if (rp.numPartitions > 1) { > val gpuOrdering = childExprs.map(_.convertToGpu()).asInstanceOf[Seq[SortOrder]] @@ -1411,21 +1437,21 @@ > GpuSinglePartitioning > } > } -2818a3730,3732 +2776a3703,3705 > override def convertToGpu(): GpuPartitioning = { > GpuRoundRobinPartitioning(rrp.numPartitions) > } -2823a3738 +2781a3711 > override def convertToGpu(): GpuPartitioning = GpuSinglePartitioning -2830c3745 +2788c3718 < parent: Option[RapidsMeta[_, _]]): DataWritingCommandMeta[INPUT] = --- > parent: Option[RapidsMeta[_, _, _]]): DataWritingCommandMeta[INPUT] = -2848c3763 +2810c3740 < parent: Option[RapidsMeta[_, _]]): SparkPlanMeta[INPUT] = --- > parent: Option[RapidsMeta[_, _, _]]): SparkPlanMeta[INPUT] = -2872c3787,3805 +2834c3764,3782 < (range, conf, p, r) => new SparkPlanMeta[RangeExec](range, conf, p, r) { --- > (range, conf, p, r) => { @@ -1439,7 +1465,7 @@ > "The backend for most file input", > ExecChecks( > (TypeSig.commonCudfTypes + TypeSig.STRUCT + TypeSig.MAP + TypeSig.ARRAY + -> TypeSig.DECIMAL_128).nested(), +> TypeSig.DECIMAL_128 + TypeSig.BINARY).nested(), > TypeSig.all), > (p, conf, parent, r) => new SparkPlanMeta[BatchScanExec](p, conf, parent, r) { > override val childScans: scala.Seq[ScanMeta[_]] = @@ -1447,14 +1473,16 @@ > > override def convertToGpu(): GpuExec = > GpuBatchScanExec(p.output, childScans.head.convertToGpu()) -2879a3813,3814 +2841a3790,3791 > override def convertToGpu(): GpuExec = > GpuCoalesceExec(coalesce.numPartitions, childPlans.head.convertIfNeeded()) -2893a3829,3831 +2849a3800 +> TypeSig.BINARY.withPsNote(TypeEnum.BINARY, "Only supported for Parquet") + +2855a3807,3809 > override def convertToGpu(): GpuExec = > GpuDataWritingCommandExec(childDataWriteCmds.head.convertToGpu(), > childPlans.head.convertIfNeeded()) -2908a3847,3871 +2870a3825,3849 > override def convertToGpu(): GpuExec = { > // To avoid metrics confusion we split a single stage up into multiple parts but only > // if there are multiple partitions to make it worth doing. @@ -1480,26 +1508,26 @@ > )(takeExec.sortOrder) > } > } -2916a3880,3881 +2878a3858,3859 > override def convertToGpu(): GpuExec = > GpuLocalLimitExec(localLimitExec.limit, childPlans.head.convertIfNeeded()) -2924a3890,3891 +2886a3868,3869 > override def convertToGpu(): GpuExec = > GpuGlobalLimitExec(globalLimitExec.limit, childPlans.head.convertIfNeeded(), 0) -2931,2934c3898 +2893,2896c3876 < (collectLimitExec, conf, p, r) => < new SparkPlanMeta[CollectLimitExec](collectLimitExec, conf, p, r) { < override val childParts: scala.Seq[PartMeta[_]] = < Seq(GpuOverrides.wrapPart(collectLimitExec.outputPartitioning, conf, Some(this)))}) --- > (collectLimitExec, conf, p, r) => new GpuCollectLimitMeta(collectLimitExec, conf, p, r)) -2943a3908,3909 +2905a3886,3887 > override def convertToGpu(): GpuExec = > GpuFilterExec(childExprs.head.convertToGpu(), childPlans.head.convertIfNeeded()) -2965a3932,3933 +2927a3910,3911 > override def convertToGpu(): GpuExec = > GpuUnionExec(childPlans.map(_.convertIfNeeded())) -2996a3965,3975 +2958a3943,3953 > override def convertToGpu(): GpuExec = { > val Seq(left, right) = childPlans.map(_.convertIfNeeded()) > val joinExec = GpuCartesianProductExec( @@ -1511,7 +1539,7 @@ > // as a filter after the join when possible. > condition.map(c => GpuFilterExec(c.convertToGpu(), joinExec)).getOrElse(joinExec) > } -3009a3989,4004 +2973a3969,3984 > exec[ObjectHashAggregateExec]( > "The backend for hash based aggregations supporting TypedImperativeAggregate functions", > ExecChecks( @@ -1522,30 +1550,33 @@ > .nested() > .withPsNote(TypeEnum.BINARY, "only allowed when aggregate buffers can be " + > "converted between CPU and GPU") -> .withPsNote(Seq(TypeEnum.ARRAY, TypeEnum.MAP), +> .withPsNote(Seq(TypeEnum.ARRAY, TypeEnum.MAP), > "not allowed for grouping expressions") > .withPsNote(TypeEnum.STRUCT, > "not allowed for grouping expressions if containing Array or Map as child"), > TypeSig.all), > (agg, conf, p, r) => new GpuObjectHashAggregateExecMeta(agg, conf, p, r)), -3016,3017d4010 +2980,2981d3990 < // SPARK 2.x we can't check for the TypedImperativeAggregate properly so -< // map/arrya/struct left off -3020c4013 +< // map/array/struct left off +2984c3993 < TypeSig.MAP + TypeSig.BINARY) --- > TypeSig.MAP + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.BINARY) -3024c4017,4020 -< .withPsNote(TypeEnum.MAP, "not allowed for grouping expressions"), +2986d3994 +< .withPsNote(TypeEnum.MAP, "not allowed for grouping expressions") +2988c3996,4000 +< "converted between CPU and GPU"), --- -> .withPsNote(Seq(TypeEnum.ARRAY, TypeEnum.MAP), +> "converted between CPU and GPU") +> .withPsNote(Seq(TypeEnum.ARRAY, TypeEnum.MAP), > "not allowed for grouping expressions") > .withPsNote(TypeEnum.STRUCT, > "not allowed for grouping expressions if containing Array or Map as child"), -3027,3028d4022 +2991,2992d4002 < // SPARK 2.x we can't check for the TypedImperativeAggregate properly so don't say we do the < // ObjectHashAggregate -3065c4059,4064 +3029c4039,4044 < (sample, conf, p, r) => new GpuSampleExecMeta(sample, conf, p, r) {} --- > (sample, conf, p, r) => new GpuSampleExecMeta(sample, conf, p, r) @@ -1554,27 +1585,27 @@ > "Plan to collect and transform the broadcast key values", > ExecChecks(TypeSig.all, TypeSig.all), > (s, conf, p, r) => new GpuSubqueryBroadcastMeta(s, conf, p, r) -3066a4066 +3030a4046 > SparkShimImpl.aqeShuffleReaderExec, -3085c4085 +3049c4065 < e.output.map(GpuOverrides.wrapExpr(_, conf, Some(this))) --- > e.resultAttrs.map(GpuOverrides.wrapExpr(_, conf, Some(this))) -3090a4091,4096 +3054a4071,4076 > > override def convertToGpu(): GpuExec = > GpuArrowEvalPythonExec(udfs.map(_.convertToGpu()).asInstanceOf[Seq[GpuPythonUDF]], > resultAttrs.map(_.convertToGpu()).asInstanceOf[Seq[Attribute]], > childPlans.head.convertIfNeeded(), > e.evalType) -3091a4098,4103 +3055a4078,4083 > exec[FlatMapCoGroupsInPandasExec]( > "The backend for CoGrouped Aggregation Pandas UDF, it runs on CPU itself now but supports" + > " scheduling GPU resources for the Python process when enabled", > ExecChecks.hiddenHack(), > (flatCoPy, conf, p, r) => new GpuFlatMapCoGroupsInPandasExecMeta(flatCoPy, conf, p, r)) > .disabledByDefault("Performance is not ideal now"), -3097a4110,4116 +3061a4090,4096 > exec[MapInPandasExec]( > "The backend for Map Pandas Iterator UDF. Accelerates the data transfer between the" + > " Java process and the Python process. It also supports scheduling GPU resources" + @@ -1582,7 +1613,7 @@ > ExecChecks((TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT).nested(), > TypeSig.all), > (mapPy, conf, p, r) => new GpuMapInPandasExecMeta(mapPy, conf, p, r)), -3103,3105c4122,4143 +3067,3069c4102,4123 < // SparkShimImpl.aqeShuffleReaderExec, < // SparkShimImpl.neverReplaceShowCurrentNamespaceCommand, < neverReplaceExec[ExecutedCommandExec]("Table metadata operation") @@ -1609,23 +1640,23 @@ > neverReplaceExec[AdaptiveSparkPlanExec]("Wrapper for adaptive query plan"), > neverReplaceExec[BroadcastQueryStageExec]("Broadcast query stage"), > neverReplaceExec[ShuffleQueryStageExec]("Shuffle query stage") -3109c4147 +3073c4127 < commonExecs ++ ShimGpuOverrides.shimExecs --- > commonExecs ++ SparkShimImpl.getExecs -3112,3114c4150 +3076,3078c4130 < // val key = SQLConf.LEGACY_TIME_PARSER_POLICY.key < val key = "2xgone" < val policy = SQLConf.get.getConfString(key, "EXCEPTION") --- > val policy = SQLConf.get.getConfString(SQLConf.LEGACY_TIME_PARSER_POLICY.key, "EXCEPTION") -3121a4158,4162 +3085a4138,4142 > val preRowToColProjection = TreeNodeTag[Seq[NamedExpression]]("rapids.gpu.preRowToColProcessing") > > val postColToRowProjection = TreeNodeTag[Seq[NamedExpression]]( > "rapids.gpu.postColToRowProcessing") > -3127a4169,4176 +3091a4149,4156 > private def doConvertPlan(wrap: SparkPlanMeta[SparkPlan], conf: RapidsConf, > optimizations: Seq[Optimization]): SparkPlan = { > val convertedPlan = wrap.convertIfNeeded() @@ -1634,7 +1665,7 @@ > sparkPlan > } > -3130c4179,4227 +3094c4159,4207 < Seq.empty --- > if (conf.optimizerEnabled) { @@ -1686,7 +1717,7 @@ > } > } > operator.withNewChildren(children) -3140,3141c4237,4243 +3104,3105c4217,4223 < // Only run the explain and don't actually convert or run on GPU. < def explainPotentialGpuPlan(df: DataFrame, explain: String = "ALL"): String = { --- @@ -1697,7 +1728,7 @@ > * to make it close to when the columnar rules would normally run on the plan. > */ > def explainPotentialGpuPlan(df: DataFrame, explain: String): String = { -3167a4270,4290 +3131a4250,4270 > /** > * Use explain mode on an active SQL plan as its processed through catalyst. > * This path is the same as being run through the plugin running on hosts with @@ -1719,17 +1750,17 @@ > } > } > -3190c4313 +3154c4293 < // case c2r: ColumnarToRowExec => prepareExplainOnly(c2r.child) --- > case c2r: ColumnarToRowExec => prepareExplainOnly(c2r.child) -3192,3193c4315,4316 +3156,3157c4295,4296 < // case aqe: AdaptiveSparkPlanExec => < // prepareExplainOnly(SparkShimImpl.getAdaptiveInputPlan(aqe)) --- > case aqe: AdaptiveSparkPlanExec => > prepareExplainOnly(SparkShimImpl.getAdaptiveInputPlan(aqe)) -3200,3204c4323,4450 +3164,3168c4303,4385 < // Spark 2.x < object GpuUserDefinedFunction { < // UDFs can support all types except UDT which does not have a clear columnar representation. @@ -1769,7 +1800,7 @@ > override def apply(sparkPlan: SparkPlan): SparkPlan = GpuOverrideUtil.tryOverride { plan => > // Note that we disregard the GPU plan returned here and instead rely on side effects of > // tagging the underlying SparkPlan. -> GpuOverrides().apply(plan) +> GpuOverrides().applyWithContext(plan, Some("AQE Query Stage Prep")) > // return the original plan which is now modified as a side-effect of invoking GpuOverrides > plan > }(sparkPlan) @@ -1779,13 +1810,22 @@ > > // Spark calls this method once for the whole plan when AQE is off. When AQE is on, it > // gets called once for each query stage (where a query stage is an `Exchange`). -> override def apply(sparkPlan: SparkPlan): SparkPlan = GpuOverrideUtil.tryOverride { plan => +> override def apply(sparkPlan: SparkPlan): SparkPlan = applyWithContext(sparkPlan, None) +> +> def applyWithContext(sparkPlan: SparkPlan, context: Option[String]): SparkPlan = +> GpuOverrideUtil.tryOverride { plan => > val conf = new RapidsConf(plan.conf) > if (conf.isSqlEnabled && conf.isSqlExecuteOnGPU) { > GpuOverrides.logDuration(conf.shouldExplain, > t => f"Plan conversion to the GPU took $t%.2f ms") { > val updatedPlan = updateForAdaptivePlan(plan, conf) -> applyOverrides(updatedPlan, conf) +> val newPlan = applyOverrides(updatedPlan, conf) +> if (conf.logQueryTransformations) { +> val logPrefix = context.map(str => s"[$str]").getOrElse("") +> logWarning(s"${logPrefix}Transformed query:" + +> s"\nOriginal Plan:\n$plan\nTransformed Plan:\n$newPlan") +> } +> newPlan > } > } else if (conf.isSqlEnabled && conf.isSqlExplainOnlyEnabled) { > // this mode logs the explain output and returns the original CPU plan @@ -1810,7 +1850,7 @@ > plan > } > } -> +3169a4387,4481 > /** > * Determine whether query is running against Delta Lake _delta_log JSON files or > * if Delta is doing stats collection that ends up hardcoding the use of AQE, @@ -1818,31 +1858,44 @@ > * check for a ScalaUDF using a tahoe.Snapshot function and if we ever see > * an AdaptiveSparkPlan on a Spark version we don't expect, fallback to the > * CPU for those plans. +> * Note that the Delta Lake delta log checkpoint parquet files are just inefficient +> * to have to copy the data to GPU and then back off after it does the scan on +> * Delta Table Checkpoint, so have the entire plan fallback to CPU at that point. > */ > def isDeltaLakeMetadataQuery(plan: SparkPlan): Boolean = { > val deltaLogScans = PlanUtils.findOperators(plan, { +> case f: FileSourceScanExec if f.requiredSchema.fields +> .exists(_.name.startsWith("_databricks_internal")) => +> logDebug(s"Fallback for FileSourceScanExec with _databricks_internal: $f") +> true > case f: FileSourceScanExec => > // example filename: "file:/tmp/delta-table/_delta_log/00000000000000000000.json" -> val found = f.relation.inputFiles.exists(name => -> name.contains("/_delta_log/") && name.endsWith(".json")) +> val found = f.relation.inputFiles.exists { name => +> name.contains("/_delta_log/") && name.endsWith(".json") +> } > if (found) { > logDebug(s"Fallback for FileSourceScanExec delta log: $f") > } > found > case rdd: RDDScanExec => -> // example rdd name: "Delta Table State #1 - file:///tmp/delta-table/_delta_log" +> // example rdd name: "Delta Table State #1 - file:///tmp/delta-table/_delta_log" or +> // "Scan ExistingRDD Delta Table Checkpoint with Stats #1 - +> // file:///tmp/delta-table/_delta_log" > val found = rdd.inputRDD != null && > rdd.inputRDD.name != null && -> rdd.inputRDD.name.startsWith("Delta Table State") && +> (rdd.inputRDD.name.startsWith("Delta Table State") +> || rdd.inputRDD.name.startsWith("Delta Table Checkpoint")) && > rdd.inputRDD.name.endsWith("/_delta_log") > if (found) { > logDebug(s"Fallback for RDDScanExec delta log: $rdd") > } > found -> case aqe: AdaptiveSparkPlanExec if !AQEUtils.isAdaptiveExecutionSupportedInSparkVersion => +> case aqe: AdaptiveSparkPlanExec if +> !AQEUtils.isAdaptiveExecutionSupportedInSparkVersion(plan.conf) => > logDebug(s"AdaptiveSparkPlanExec found on unsupported Spark Version: $aqe") > true -> case project: ProjectExec if !AQEUtils.isAdaptiveExecutionSupportedInSparkVersion => +> case project: ProjectExec if +> !AQEUtils.isAdaptiveExecutionSupportedInSparkVersion(plan.conf) => > val foundExprs = project.expressions.flatMap { e => > PlanUtils.findExpressions(e, { > case udf: ScalaUDF => @@ -1864,7 +1917,7 @@ > }) > deltaLogScans.nonEmpty > } -3205a4452,4479 +> > private def applyOverrides(plan: SparkPlan, conf: RapidsConf): SparkPlan = { > val wrap = GpuOverrides.wrapAndTagPlan(plan, conf) > if (conf.isDetectDeltaLogQueries && isDeltaLakeMetadataQuery(plan)) { diff --git a/scripts/spark2diffs/GpuParquetFileFormat.diff b/scripts/spark2diffs/GpuParquetFileFormat.diff index b0d381eecc1..d72b1a53977 100644 --- a/scripts/spark2diffs/GpuParquetFileFormat.diff +++ b/scripts/spark2diffs/GpuParquetFileFormat.diff @@ -8,7 +8,7 @@ > schema: StructType): Option[GpuParquetFileFormat] = { 8a9 > -61,65c62 +73,77c74 < // Spark 2.x doesn't have the rebase mode because the changes of calendar type weren't made < // so just skip the checks, since this is just explain only it would depend on how < // they set when they get to 3.x. The default in 3.x is EXCEPTION which would be good @@ -16,7 +16,7 @@ < /* --- > -87c84,89 +99c96,101 < */ --- > @@ -25,12 +25,12 @@ > } else { > None > } -90,91c92 +102,103c104 < // SPARK 2.X - just return String rather then CompressionType < def parseCompressionType(compressionType: String): Option[String] = { --- > def parseCompressionType(compressionType: String): Option[CompressionType] = { -93,94c94,95 +105,106c106,107 < case "NONE" | "UNCOMPRESSED" => Some("NONE") < case "SNAPPY" => Some("SNAPPY") --- diff --git a/scripts/spark2diffs/GpuRLikeMeta.diff b/scripts/spark2diffs/GpuRLikeMeta.diff index 523abb697b6..0c3f2e026a3 100644 --- a/scripts/spark2diffs/GpuRLikeMeta.diff +++ b/scripts/spark2diffs/GpuRLikeMeta.diff @@ -2,5 +2,5 @@ < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -23a24 +26a27 > diff --git a/scripts/spark2diffs/GpuReadOrcFileFormat.diff b/scripts/spark2diffs/GpuReadOrcFileFormat.diff index a2a89b0932f..ff503c14a73 100644 --- a/scripts/spark2diffs/GpuReadOrcFileFormat.diff +++ b/scripts/spark2diffs/GpuReadOrcFileFormat.diff @@ -1,4 +1,4 @@ -8c8 +5c5 < fsse.sqlContext.sparkSession, --- > SparkShimImpl.sessionFromPlan(fsse), diff --git a/scripts/spark2diffs/GpuRegExpExtractMeta.diff b/scripts/spark2diffs/GpuRegExpExtractMeta.diff index 6bc051f0afa..69f6428fe89 100644 --- a/scripts/spark2diffs/GpuRegExpExtractMeta.diff +++ b/scripts/spark2diffs/GpuRegExpExtractMeta.diff @@ -7,5 +7,5 @@ < /* 21d18 < */ -52a50 +54a52 > diff --git a/scripts/spark2diffs/GpuRegExpReplaceMeta.diff b/scripts/spark2diffs/GpuRegExpReplaceMeta.diff index 5e29325c8a4..5000ed35ce6 100644 --- a/scripts/spark2diffs/GpuRegExpReplaceMeta.diff +++ b/scripts/spark2diffs/GpuRegExpReplaceMeta.diff @@ -10,12 +10,12 @@ < val (pat, repl) = --- > val (pat, repl) = -45,46c45 +46,47c46 < // Spark 2.x is ternary and doesn't have pos parameter < /* --- > -52d50 +53d51 < */ -53a52 +54a53 > diff --git a/scripts/spark2diffs/GpuRegExpUtils.diff b/scripts/spark2diffs/GpuRegExpUtils.diff index e4a46eea871..99442cee84f 100644 --- a/scripts/spark2diffs/GpuRegExpUtils.diff +++ b/scripts/spark2diffs/GpuRegExpUtils.diff @@ -8,19 +8,19 @@ < case _ => --- > case _ => -80,81c80,81 +87,88c87,88 < * Recursively check if pattern contains only zero-match repetitions < * ?, *, {0,}, or {0,n} or any combination of them. --- > * Recursively check if pattern contains only zero-match repetitions > * ?, *, {0,}, or {0,n} or any combination of them. -93c93 +100c100 < isASTEmptyRepetition(term) --- > isASTEmptyRepetition(term) -105c105 +112c112 < * Returns the number of groups in regexp --- > * Returns the number of groups in regexp -116a117 +123a124 > diff --git a/scripts/spark2diffs/GpuSpecifiedWindowFrameMetaBase.diff b/scripts/spark2diffs/GpuSpecifiedWindowFrameMetaBase.diff index c1f6b464417..12b0444ab52 100644 --- a/scripts/spark2diffs/GpuSpecifiedWindowFrameMetaBase.diff +++ b/scripts/spark2diffs/GpuSpecifiedWindowFrameMetaBase.diff @@ -2,12 +2,12 @@ < parent: Option[RapidsMeta[_,_]], --- > parent: Option[RapidsMeta[_,_,_]], -47,49d46 +48,50d47 < // Spark 2.x different - no days, just months and microseconds < // could remove this catch but leaving for now < /* -53,54d49 +54,55d50 < */ < ci.microseconds -119a115 +120a116 > diff --git a/scripts/spark2diffs/GpuWindowExpressionMetaBase.diff b/scripts/spark2diffs/GpuWindowExpressionMetaBase.diff index 5fb307e1a07..711991cb9d9 100644 --- a/scripts/spark2diffs/GpuWindowExpressionMetaBase.diff +++ b/scripts/spark2diffs/GpuWindowExpressionMetaBase.diff @@ -2,7 +2,7 @@ < parent: Option[RapidsMeta[_,_]], --- > parent: Option[RapidsMeta[_,_,_]], -124a125,128 +128a129,132 > > /** > * Convert what this wraps to a GPU enabled version. diff --git a/scripts/spark2diffs/HiveProviderImpl.diff b/scripts/spark2diffs/HiveProviderImpl.diff index 2f942a8780b..f03a7615d7a 100644 --- a/scripts/spark2diffs/HiveProviderImpl.diff +++ b/scripts/spark2diffs/HiveProviderImpl.diff @@ -1,8 +1,17 @@ 20c20 -< import com.nvidia.spark.rapids.{ExprChecks, ExprMeta, ExprRule, GpuExpression, GpuOverrides, HiveProvider, RapidsConf, RepeatingParamCheck, TypeSig} +< import com.nvidia.spark.rapids.{DataWritingCommandRule, ExprChecks, ExprMeta, ExprRule, GpuExpression, GpuOverrides, HiveProvider, OptimizedCreateHiveTableAsSelectCommandMeta, RapidsConf, RepeatingParamCheck, TypeSig} --- -> import com.nvidia.spark.rapids.{ExprChecks, ExprMeta, ExprRule, GpuOverrides, HiveProvider, RapidsConf, RepeatingParamCheck, TypeSig} -54,73d53 +> import com.nvidia.spark.rapids.{DataWritingCommandRule, ExprChecks, ExprMeta, ExprRule, GpuOverrides, HiveProvider, OptimizedCreateHiveTableAsSelectCommandMeta, RapidsConf, RepeatingParamCheck, TypeSig} +34c34 +< DataWritingCommandRule[_ <: DataWritingCommand]] = Seq ( +--- +> DataWritingCommandRule[_ <: DataWritingCommand]] = Seq ( +41c41,42 +< * Builds the expression rules that are specific to spark-hive Catalyst nodes. +--- +> * Builds the rules that are specific to spark-hive Catalyst nodes. This will return an empty +> * mapping if spark-hive is unavailable. +65,84d65 < < override def convertToGpu(): GpuExpression = { < opRapidsFunc.map { _ => @@ -23,7 +32,7 @@ < childExprs.map(_.convertToGpu())) < } < } -93,113d72 +104,124d84 < } < } < diff --git a/scripts/spark2diffs/RapidsConf.diff b/scripts/spark2diffs/RapidsConf.diff index fb38f511bc5..1ad0996bdc6 100644 --- a/scripts/spark2diffs/RapidsConf.diff +++ b/scripts/spark2diffs/RapidsConf.diff @@ -17,11 +17,11 @@ < .createWithDefault(ByteUnit.MiB.toBytes(8).toLong) --- > .createWithDefault(ByteUnit.MiB.toBytes(8)) -1547c1549 -< |$SPARK_HOME/bin/spark-shell --jars rapids-4-spark_2.12-22.08.0-SNAPSHOT-cuda11.jar \ +1633c1635 +< |$SPARK_HOME/bin/spark-shell --jars rapids-4-spark_2.12-22.10.0-SNAPSHOT-cuda11.jar \ --- -> |${SPARK_HOME}/bin/spark-shell --jars rapids-4-spark_2.12-22.08.0-SNAPSHOT-cuda11.jar \ -1602,1606c1604,1607 +> |${SPARK_HOME}/bin/spark-shell --jars rapids-4-spark_2.12-22.10.0-SNAPSHOT-cuda11.jar \ +1688,1692c1690,1693 < // scans not supported in Spark 2.x < // if (asTable) { < // printToggleHeader("Scans\n") @@ -32,12 +32,12 @@ > printToggleHeader("Scans\n") > } > GpuOverrides.scans.values.toSeq.sortBy(_.tag.toString).foreach(_.confHelp(asTable)) -1614c1615 +1700c1701 < // com.nvidia.spark.rapids.python.PythonConfEntries.init() --- > com.nvidia.spark.rapids.python.PythonConfEntries.init() -1688,1689d1688 +1776,1777d1776 < // Spark 2.x doesn't have access to Cuda in CUDF so just allow < /* -1705d1703 +1793d1791 < */ diff --git a/scripts/spark2diffs/RapidsMeta.diff b/scripts/spark2diffs/RapidsMeta.diff index ecf26535cbc..982d9c99fc0 100644 --- a/scripts/spark2diffs/RapidsMeta.diff +++ b/scripts/spark2diffs/RapidsMeta.diff @@ -59,33 +59,33 @@ > p.getTagValue(gpuSupportedTag).getOrElse(Set.empty) + because) > case _ => > } -214a242,246 +218a246,250 > * Returns true iff all of the scans can be replaced. > */ > def canScansBeReplaced: Boolean = childScans.forall(_.canThisBeReplaced) > > /** -244a277 +248a281 > childScans.foreach(_.tagForGpu()) -385a419 +389a423 > childScans.foreach(_.print(append, depth + 1, all)) -408c442 +412c446 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -410,412c444 +414,416c448 < extends RapidsMeta[INPUT, Partitioning](part, conf, parent, rule) { < // 2.x - replaced GpuPartitioning with Partitioning, should be fine < // since BASE only used for convert --- > extends RapidsMeta[INPUT, Partitioning, GpuPartitioning](part, conf, parent, rule) { -415a448 +419a452 > override val childScans: Seq[ScanMeta[_]] = Seq.empty -436c469 +440c473 < parent: Option[RapidsMeta[_, _]]) --- > parent: Option[RapidsMeta[_, _, _]]) -442a476,514 +446a480,518 > override def convertToGpu(): GpuPartitioning = > throw new IllegalStateException("Cannot be converted to GPU") > } @@ -125,57 +125,57 @@ > > override def convertToGpu(): Scan = > throw new IllegalStateException("Cannot be converted to GPU") -451c523 +455c527 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -453c525 +457c529 < extends RapidsMeta[INPUT, DataWritingCommand](cmd, conf, parent, rule) { --- > extends RapidsMeta[INPUT, DataWritingCommand, GpuDataWritingCommand](cmd, conf, parent, rule) { -456a529 +460a533 > override val childScans: Seq[ScanMeta[_]] = Seq.empty -469c542 +473c546 < parent: Option[RapidsMeta[_, _]]) --- > parent: Option[RapidsMeta[_, _, _]]) -474a548,550 +478a552,554 > > override def convertToGpu(): GpuDataWritingCommand = > throw new IllegalStateException("Cannot be converted to GPU") -482c558 +486c562 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -484c560 +488c564 < extends RapidsMeta[INPUT, SparkPlan](plan, conf, parent, rule) { --- > extends RapidsMeta[INPUT, SparkPlan, GpuExec](plan, conf, parent, rule) { -489a566 +493a570 > childScans.foreach(_.recursiveSparkPlanPreventsRunningOnGpu()) -494a572 +498a576 > childScans.foreach(_.recursiveSparkPlanRemoved()) -519a598 +523a602 > override val childScans: Seq[ScanMeta[_]] = Seq.empty -555a635,639 +559a639,643 > > childPlans.head.wrapped > .getTagValue(GpuOverrides.preRowToColProjection).foreach { r2c => > wrapped.setTagValue(GpuOverrides.preRowToColProjection, r2c) > } -597c681 +601c685 < /*if (!canScansBeReplaced) { --- > if (!canScansBeReplaced) { -599c683 +603c687 < } */ --- > } -618a703,705 +622a707,709 > wrapped.getTagValue(RapidsMeta.gpuSupportedTag) > .foreach(_.diff(cannotBeReplacedReasons.get) > .foreach(willNotWorkOnGpu)) -642c729,733 +646c733,737 < convertToCpu --- > if (canThisBeReplaced) { @@ -183,27 +183,27 @@ > } else { > convertToCpu() > } -712c803 +716c807 < parent: Option[RapidsMeta[_, _]]) --- > parent: Option[RapidsMeta[_, _, _]]) -716a808,810 +720a812,814 > > override def convertToGpu(): GpuExec = > throw new IllegalStateException("Cannot be converted to GPU") -725c819 +729c823 < parent: Option[RapidsMeta[_, _]]) --- > parent: Option[RapidsMeta[_, _, _]]) -732a827,829 +736a831,833 > > override def convertToGpu(): GpuExec = > throw new IllegalStateException("Cannot be converted to GPU") -773c870 +777c874 < case agg: SparkPlan if agg.isInstanceOf[WindowExec] => --- > case agg: SparkPlan if SparkShimImpl.isWindowFunctionExec(agg) => -775,782c872 +779,786c876 < case agg: HashAggregateExec => < // Spark 2.x doesn't have the BaseAggregateExec class < if (agg.groupingExpressions.isEmpty) { @@ -214,54 +214,54 @@ < case agg: SortAggregateExec => --- > case agg: BaseAggregateExec => -793c883 +797c887 < def getRegularOperatorContext(meta: RapidsMeta[_, _]): ExpressionContext = meta.wrapped match { --- > def getRegularOperatorContext(meta: RapidsMeta[_, _, _]): ExpressionContext = meta.wrapped match { -849c939 +853c943 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -851c941 +855c945 < extends RapidsMeta[INPUT, Expression](expr, conf, parent, rule) { --- > extends RapidsMeta[INPUT, Expression, Expression](expr, conf, parent, rule) { -857a948 +861a952 > override val childScans: Seq[ScanMeta[_]] = Seq.empty -907c998 +911c1002 < case tzAware: TimeZoneAwareExpression => checkTimeZoneId(tzAware.timeZone.toZoneId) --- > case tzAware: TimeZoneAwareExpression => checkTimeZoneId(tzAware.zoneId) -1000c1091 +1004c1095 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -1002a1094,1095 +1006a1098,1099 > > override def convertToGpu(): GpuExpression -1011c1104 +1015c1108 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -1014a1108,1112 +1018a1112,1116 > override final def convertToGpu(): GpuExpression = > convertToGpu(childExprs.head.convertToGpu()) > > def convertToGpu(child: Expression): GpuExpression > -1030c1128 +1034c1132 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -1041c1139 +1045c1143 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -1046a1145,1147 +1050a1149,1151 > if (needsAnsiCheck) { > GpuOverrides.checkAndTagAnsiAgg(ansiTypeToCheck, this) > } -1050a1152,1164 +1054a1156,1168 > > override final def convertToGpu(): GpuExpression = > convertToGpu(childExprs.map(_.convertToGpu())) @@ -275,18 +275,18 @@ > // The type to use to determine whether the aggregate could overflow. > // Set to None, if we should fallback for all types > val ansiTypeToCheck: Option[DataType] = Some(expr.dataType) -1059c1173 +1063c1177 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -1061a1176,1177 +1065a1180,1181 > > def convertToGpu(childExprs: Seq[Expression]): GpuExpression -1070c1186 +1074c1190 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -1081a1198,1213 +1085a1202,1217 > * Returns a buffer converter who can generate a Expression to transform the aggregation buffer > * of wrapped function from CPU format to GPU format. The conversion occurs on the CPU, so the > * generated expression should be a CPU Expression executed by row. @@ -303,11 +303,11 @@ > throw new NotImplementedError("The method should be implemented by specific functions") > > /** -1095c1227 +1099c1231 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -1097a1230,1236 +1101a1234,1240 > > override final def convertToGpu(): GpuExpression = { > val Seq(lhs, rhs) = childExprs.map(_.convertToGpu()) @@ -315,15 +315,15 @@ > } > > def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression -1104c1243 +1108c1247 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -1122c1261 +1126c1265 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -1124a1264,1290 +1128a1268,1294 > > override final def convertToGpu(): GpuExpression = { > val Seq(child0, child1, child2) = childExprs.map(_.convertToGpu()) @@ -351,11 +351,11 @@ > > def convertToGpu(val0: Expression, val1: Expression, > val2: Expression, val3: Expression): GpuExpression -1130c1296 +1134c1300 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -1132a1299,1305 +1136a1303,1309 > > override final def convertToGpu(): GpuExpression = { > val gpuCol :: gpuTrimParam = childExprs.map(_.convertToGpu()) @@ -363,20 +363,20 @@ > } > > def convertToGpu(column: Expression, target: Option[Expression] = None): GpuExpression -1141c1314 +1145c1318 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -1143a1317,1320 +1147a1321,1324 > override final def convertToGpu(): GpuExpression = > convertToGpu(childExprs.map(_.convertToGpu())) > > def convertToGpu(childExprs: Seq[Expression]): GpuExpression -1152c1329 +1156c1333 < parent: Option[RapidsMeta[_, _]]) --- > parent: Option[RapidsMeta[_, _, _]]) -1156a1334,1336 +1160a1338,1340 > > override def convertToGpu(): GpuExpression = > throw new IllegalStateException("Cannot be converted to GPU") diff --git a/scripts/spark2diffs/RegexParser.diff b/scripts/spark2diffs/RegexParser.diff index 56d589a683e..5ff88a19475 100644 --- a/scripts/spark2diffs/RegexParser.diff +++ b/scripts/spark2diffs/RegexParser.diff @@ -2,7 +2,7 @@ < * Copyright (c) 2022, NVIDIA CORPORATION. --- > * Copyright (c) 2021-2022, NVIDIA CORPORATION. -1798c1798 +1834c1834 < } --- > } diff --git a/scripts/spark2diffs/TypeChecks.diff b/scripts/spark2diffs/TypeChecks.diff index 3fe8064f9c9..6fa895241e4 100644 --- a/scripts/spark2diffs/TypeChecks.diff +++ b/scripts/spark2diffs/TypeChecks.diff @@ -14,23 +14,31 @@ < meta: RapidsMeta[_, _], --- > meta: RapidsMeta[_, _, _], -615c615 +613c613 < val DECIMAL_64: TypeSig = decimal(GpuOverrides.DECIMAL64_MAX_PRECISION) --- > val DECIMAL_64: TypeSig = decimal(DType.DECIMAL64_MAX_PRECISION) -622c622 +620c620 < val DECIMAL_128: TypeSig = decimal(GpuOverrides.DECIMAL128_MAX_PRECISION) --- > val DECIMAL_128: TypeSig = decimal(DType.DECIMAL128_MAX_PRECISION) -767c767 +765c765 < def tag(meta: RapidsMeta[_, _]): Unit --- > def tag(meta: RapidsMeta[_, _, _]): Unit -780c780 +777c777 +< /** +--- +> /** +783c783 +< meta: RapidsMeta[_, _] +--- +> meta: RapidsMeta[_, _, _] +808c808 < meta: RapidsMeta[_, _], --- > meta: RapidsMeta[_, _, _], -804,814d803 +834,844d833 < // Spark 2.X doesn't have getZoneId - copy it here < private def getZoneId(timeZoneId: String): ZoneId = { < val formattedZoneId = timeZoneId @@ -42,68 +50,68 @@ < ZoneId.of(formattedZoneId, ZoneId.SHORT_IDS) < } < -816,817c805 +846,847c835 < // Spark 2.X doesn't have getZoneId < val zoneId = getZoneId(zoneIdString) --- > val zoneId = DateTimeUtils.getZoneId(zoneIdString) -854c842 +889c877 < override def tag(rapidsMeta: RapidsMeta[_, _]): Unit = { --- > override def tag(rapidsMeta: RapidsMeta[_, _, _]): Unit = { -858c846 +893c881 < private[this] def tagBase(rapidsMeta: RapidsMeta[_, _], willNotWork: String => Unit): Unit = { --- > private[this] def tagBase(rapidsMeta: RapidsMeta[_, _, _], willNotWork: String => Unit): Unit = { -916c904 +951c939 < def tag(meta: RapidsMeta[_, _], --- > def tag(meta: RapidsMeta[_, _, _], -927c915 +962c950 < override def tag(meta: RapidsMeta[_, _]): Unit = --- > override def tag(meta: RapidsMeta[_, _, _]): Unit = -953c941 +988c976 < def tag(meta: RapidsMeta[_, _], --- > def tag(meta: RapidsMeta[_, _, _], -974c962 +1009c997 < override def tag(rapidsMeta: RapidsMeta[_, _]): Unit = { --- > override def tag(rapidsMeta: RapidsMeta[_, _, _]): Unit = { -1049c1037 +1084c1072 < override def tag(meta: RapidsMeta[_, _]): Unit = { --- > override def tag(meta: RapidsMeta[_, _, _]): Unit = { -1101c1089 +1136c1124 < override def tag(meta: RapidsMeta[_, _]): Unit = { --- > override def tag(meta: RapidsMeta[_, _, _]): Unit = { -1144c1132 +1179c1167 < override def tag(meta: RapidsMeta[_, _]): Unit = { --- > override def tag(meta: RapidsMeta[_, _, _]): Unit = { -1194c1182 +1229c1217 < override def tag(meta: RapidsMeta[_, _]): Unit = { --- > override def tag(meta: RapidsMeta[_, _, _]): Unit = { -1240c1228 +1275c1263 < override def tag(meta: RapidsMeta[_, _]): Unit = { --- > override def tag(meta: RapidsMeta[_, _, _]): Unit = { -1276c1264 +1311c1299 < override def tag(meta: RapidsMeta[_, _]): Unit = { --- > override def tag(meta: RapidsMeta[_, _, _]): Unit = { -1430c1418 +1465c1453 < override def tag(meta: RapidsMeta[_, _]): Unit = { --- > override def tag(meta: RapidsMeta[_, _, _]): Unit = { -1440c1428 +1475c1463 < private[this] def tagBase(meta: RapidsMeta[_, _], willNotWork: String => Unit): Unit = { --- > private[this] def tagBase(meta: RapidsMeta[_, _, _], willNotWork: String => Unit): Unit = { -1775,1783d1762 +1810,1818d1797 < def getSparkVersion: String = { < // hack for databricks, try to find something more reliable? < if (SPARK_BUILD_USER.equals("Databricks")) { @@ -113,27 +121,27 @@ < } < } < -1798c1777 +1833c1812 < println(s"against version ${getSparkVersion} of Spark. Most of this should still") --- > println(s"against version ${ShimLoader.getSparkVersion} of Spark. Most of this should still") -1906c1885 +1941c1920 < val allData = allSupportedTypes.toList.map { t => --- > val allData = allSupportedTypes.map { t => -1989c1968 +2024c2003 < val allData = allSupportedTypes.toList.map { t => --- > val allData = allSupportedTypes.map { t => -2093c2072 +2128c2107 < val allData = allSupportedTypes.toList.map { t => --- > val allData = allSupportedTypes.map { t => -2294d2272 +2329d2307 < /* -2370d2347 +2405d2382 < */ -2374,2375c2351,2352 +2409,2410c2386,2387 < // case a if a.equals("execs") => outputSupportedExecs() < // case expr if expr.equals("exprs") => outputSupportedExpressions() --- diff --git a/scripts/spark2diffs/UnixTimeExprMeta.diff b/scripts/spark2diffs/UnixTimeExprMeta.diff index 998415abdca..06b6551d986 100644 --- a/scripts/spark2diffs/UnixTimeExprMeta.diff +++ b/scripts/spark2diffs/UnixTimeExprMeta.diff @@ -2,5 +2,3 @@ < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -22a23 -> diff --git a/scripts/spark2diffs/aggregate.diff b/scripts/spark2diffs/aggregate.diff index 78de885d866..11e291130af 100644 --- a/scripts/spark2diffs/aggregate.diff +++ b/scripts/spark2diffs/aggregate.diff @@ -19,9 +19,10 @@ > import org.apache.spark.rdd.RDD > import org.apache.spark.sql.catalyst.InternalRow > import org.apache.spark.sql.catalyst.expressions.{Alias, Ascending, Attribute, AttributeReference, AttributeSeq, AttributeSet, Expression, ExprId, If, NamedExpression, NullsFirst, SortOrder} -24,25c36,44 +24,26c36,46 < import org.apache.spark.sql.execution.{SortExec, SparkPlan, TrampolineUtil} < import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} +< import org.apache.spark.sql.types.{ArrayType, DataType, MapType} --- > import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering > import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -32,14 +33,14 @@ > import org.apache.spark.sql.execution.aggregate.{BaseAggregateExec, HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} > import org.apache.spark.sql.rapids.{CpuToGpuAggregateBufferConverter, CudfAggregate, GpuAggregateExpression, GpuToCpuAggregateBufferConverter} > import org.apache.spark.sql.rapids.execution.{GpuShuffleMeta, TrampolineUtil} -26a46 +> import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType} > import org.apache.spark.sql.vectorized.ColumnarBatch 28,29c48 < // Spark 2.x - had to copy the GpuBaseAggregateMeta into each Hash and Sort Meta because no < // BaseAggregateExec class in Spark 2.x --- > object AggregateUtils { -31,32c50,835 +31,32c50,851 < class GpuHashAggregateMeta( < val agg: HashAggregateExec, --- @@ -173,6 +174,7 @@ > * @param modeInfo identifies which aggregation modes are being used > * @param metrics metrics that will be updated during aggregation > * @param configuredTargetBatchSize user-specified value for the targeted input batch size +> * @param useTieredProject user-specified option to enable tiered projections > */ > class GpuHashAggregateIterator( > cbIter: Iterator[ColumnarBatch], @@ -183,7 +185,8 @@ > resultExpressions: Seq[NamedExpression], > modeInfo: AggregateModeInfo, > metrics: GpuHashAggregateMetrics, -> configuredTargetBatchSize: Long) +> configuredTargetBatchSize: Long, +> useTieredProject: Boolean) > extends Iterator[ColumnarBatch] with Arm with AutoCloseable with Logging { > > // Partial mode: @@ -274,7 +277,7 @@ > > /** Aggregate all input batches and place the results in the aggregatedBatches queue. */ > private def aggregateInputBatches(): Unit = { -> val aggHelper = new AggHelper(forceMerge = false) +> val aggHelper = new AggHelper(forceMerge = false, useTieredProject = useTieredProject) > while (cbIter.hasNext) { > withResource(cbIter.next()) { childBatch => > val isLastInputBatch = GpuColumnVector.isTaggedAsFinalBatch(childBatch) @@ -379,7 +382,8 @@ > wasBatchMerged > } > -> private lazy val concatAndMergeHelper = new AggHelper(forceMerge = true) +> private lazy val concatAndMergeHelper = +> new AggHelper(forceMerge = true, useTieredProject = useTieredProject) > > /** > * Concatenate batches together and perform a merge aggregation on the result. The input batches @@ -461,7 +465,8 @@ > new Iterator[ColumnarBatch] { > override def hasNext: Boolean = keyBatchingIter.hasNext > -> private val mergeSortedHelper = new AggHelper(true, isSorted = true) +> private val mergeSortedHelper = +> new AggHelper(true, isSorted = true, useTieredProject = useTieredProject) > > override def next(): ColumnarBatch = { > // batches coming out of the sort need to be merged @@ -623,8 +628,10 @@ > * the merge steps for each aggregate function > * @param isSorted - if the batch is sorted this is set to true and is passed to cuDF > * as an optimization hint +> * @param useTieredProject - if true, used tiered project for input projections > */ -> class AggHelper(forceMerge: Boolean, isSorted: Boolean = false) { +> class AggHelper(forceMerge: Boolean, isSorted: Boolean = false, +> useTieredProject : Boolean = true) { > // `CudfAggregate` instances to apply, either update or merge aggregates > private val cudfAggregates = new mutable.ArrayBuffer[CudfAggregate]() > @@ -686,10 +693,16 @@ > } > > // a bound expression that is applied before the cuDF aggregate -> private val preStepBound = if (forceMerge) { -> GpuBindReferences.bindGpuReferences(preStep.toList, aggBufferAttributes.toList) +> private val preStepAttributes = if (forceMerge) { +> aggBufferAttributes > } else { -> GpuBindReferences.bindGpuReferences(preStep, inputAttributes) +> inputAttributes +> } +> private val (preStepBound, preStepBoundTiered) = if (useTieredProject) { +> (None, Some(GpuBindReferences.bindGpuReferencesTiered(preStep.toList, +> preStepAttributes.toList))) +> } else { +> (Some(GpuBindReferences.bindGpuReferences(preStep, preStepAttributes.toList)), None) > } > > // a bound expression that is applied after the cuDF aggregate @@ -704,7 +717,11 @@ > */ > def preProcess(toAggregateBatch: ColumnarBatch): ColumnarBatch = { > withResource(new NvtxRange("pre-process", NvtxColor.DARK_GREEN)) { _ => -> GpuProjectExec.project(toAggregateBatch, preStepBound) +> if (useTieredProject) { +> preStepBoundTiered.get.tieredProject(toAggregateBatch) +> } else { +> GpuProjectExec.project(toAggregateBatch, preStepBound.get) +> } > } > } > @@ -829,7 +846,7 @@ > abstract class GpuBaseAggregateMeta[INPUT <: SparkPlan]( > plan: INPUT, > aggRequiredChildDistributionExpressions: Option[Seq[Expression]], -34,36c837,840 +34,36c853,856 < parent: Option[RapidsMeta[_, _]], < rule: DataFromReplacementRule) < extends SparkPlanMeta[HashAggregateExec](agg, conf, parent, rule) { @@ -838,7 +855,37 @@ > rule: DataFromReplacementRule) extends SparkPlanMeta[INPUT](plan, conf, parent, rule) { > > val agg: BaseAggregateExec -60a865,877 +51c871 +< // We don't support Arrays and Maps as GroupBy keys yet, even they are nested in Structs. So, +--- +> // We don't support Maps as GroupBy keys yet, even if they are nested in Structs. So, +53c873,882 +< val arrayOrMapGroupings = agg.groupingExpressions.exists(e => +--- +> val mapGroupings = agg.groupingExpressions.exists(e => +> TrampolineUtil.dataTypeExistsRecursively(e.dataType, +> dt => dt.isInstanceOf[MapType])) +> if (mapGroupings) { +> willNotWorkOnGpu("MapTypes in grouping expressions are not supported") +> } +> +> // We support Arrays as grouping expression but not if the child is a struct. So we need to +> // run recursive type check on the lists of structs +> val arrayWithStructsGroupings = agg.groupingExpressions.exists(e => +55,57c884,891 +< dt => dt.isInstanceOf[ArrayType] || dt.isInstanceOf[MapType])) +< if (arrayOrMapGroupings) { +< willNotWorkOnGpu("ArrayTypes or MapTypes in grouping expressions are not supported") +--- +> dt => dt match { +> case ArrayType(_: StructType, _) => true +> case _ => false +> }) +> ) +> if (arrayWithStructsGroupings) { +> willNotWorkOnGpu("ArrayTypes with Struct children in grouping expressions are not " + +> "supported") +60a895,907 > > if (agg.aggregateExpressions.exists(expr => expr.isDistinct) > && agg.aggregateExpressions.exists(expr => expr.filter.isDefined)) { @@ -852,7 +899,7 @@ > if (AggregationTagging.mustReplaceBoth) { > tagForMixedReplacement() > } -124a942,980 +124a972,1011 > > /** Prevent mixing of CPU and GPU aggregations */ > private def tagForMixedReplacement(): Unit = { @@ -890,9 +937,10 @@ > aggregateAttributes.map(_.convertToGpu().asInstanceOf[Attribute]), > resultExpressions.map(_.convertToGpu().asInstanceOf[NamedExpression]), > childPlans.head.convertIfNeeded(), -> conf.gpuTargetBatchSizeBytes) +> conf.gpuTargetBatchSizeBytes, +> conf.isTieredProjectEnabled) > } -127,128c983,989 +127,128c1014,1020 < class GpuSortAggregateExecMeta( < val agg: SortAggregateExec, --- @@ -903,7 +951,7 @@ > abstract class GpuTypedImperativeSupportedAggregateExecMeta[INPUT <: BaseAggregateExec]( > plan: INPUT, > aggRequiredChildDistributionExpressions: Option[Seq[Expression]], -130,132c991,999 +130,132c1022,1030 < parent: Option[RapidsMeta[_, _]], < rule: DataFromReplacementRule) < extends SparkPlanMeta[SortAggregateExec](agg, conf, parent, rule) { @@ -917,7 +965,7 @@ > expr.aggregateFunction.isInstanceOf[TypedImperativeAggregate[_]] && > (expr.mode == Partial || expr.mode == PartialMerge) > } -134,141c1001,1002 +134,141c1032,1033 < val groupingExpressions: Seq[BaseExprMeta[_]] = < agg.groupingExpressions.map(GpuOverrides.wrapExpr(_, conf, Some(this))) < val aggregateExpressions: Seq[BaseExprMeta[_]] = @@ -929,7 +977,7 @@ --- > // overriding data types of Aggregation Buffers if necessary > if (mayNeedAggBufferConversion) overrideAggBufTypes() -143,144c1004,1015 +143,144c1035,1046 < override val childExprs: Seq[BaseExprMeta[_]] = < groupingExpressions ++ aggregateExpressions ++ aggregateAttributes ++ resultExpressions --- @@ -945,7 +993,7 @@ > case aggMeta: TypedImperativeAggExprMeta[_] => aggMeta.supportBufferConversion > case _ => true > } -147,153c1018,1062 +147,153c1049,1094 < // We don't support Arrays and Maps as GroupBy keys yet, even they are nested in Structs. So, < // we need to run recursive type check on the structs. < val arrayOrMapGroupings = agg.groupingExpressions.exists(e => @@ -996,12 +1044,13 @@ > aggAttributes.map(_.convertToGpu().asInstanceOf[Attribute]), > retExpressions.map(_.convertToGpu().asInstanceOf[NamedExpression]), > childPlans.head.convertIfNeeded(), -> conf.gpuTargetBatchSizeBytes) +> conf.gpuTargetBatchSizeBytes, +> conf.isTieredProjectEnabled) > } else { > super.convertToGpu() -154a1064 +154a1096 > } -156c1066,1312 +156c1098,1344 < tagForReplaceMode() --- > /** @@ -1251,7 +1300,7 @@ > agg.requiredChildDistributionExpressions, conf, parent, rule) { > override def tagPlanForGpu(): Unit = { > super.tagPlanForGpu() -194,256d1349 +194,256d1381 < < /** < * Tagging checks tied to configs that control the aggregation modes that are replaced. @@ -1315,15 +1364,15 @@ < s"Set ${conf.partialMergeDistinctEnabled} to true if desired") < } < } -259,261d1351 +259,261d1383 < // SPARK 2.x we can't check for the TypedImperativeAggregate properly so don't say we do the < // ObjectHashAggregate < /* -265c1355 +265c1387 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -270c1360,1563 +270c1392,1599 < */ --- > /** @@ -1339,6 +1388,7 @@ > * node should project) > * @param child incoming plan (where we get input columns from) > * @param configuredTargetBatchSize user-configured maximum device memory size of a batch +> * @param configuredTieredProjectEnabled configurable optimization to use tiered projections > */ > case class GpuHashAggregateExec( > requiredChildDistributionExpressions: Option[Seq[Expression]], @@ -1347,7 +1397,8 @@ > aggregateAttributes: Seq[Attribute], > resultExpressions: Seq[NamedExpression], > child: SparkPlan, -> configuredTargetBatchSize: Long) extends ShimUnaryExecNode with GpuExec with Arm { +> configuredTargetBatchSize: Long, +> configuredTieredProjectEnabled: Boolean) extends ShimUnaryExecNode with GpuExec with Arm { > > // lifted directly from `BaseAggregateExec.inputAttributes`, edited comment. > def inputAttributes: Seq[Attribute] = { @@ -1425,6 +1476,7 @@ > val resultExprs = resultExpressions > val modeInfo = AggregateModeInfo(uniqueModes) > val targetBatchSize = configuredTargetBatchSize +> val useTieredProject = configuredTieredProjectEnabled > > val rdd = child.executeColumnar() > @@ -1438,7 +1490,8 @@ > resultExprs, > modeInfo, > aggMetrics, -> targetBatchSize) +> targetBatchSize, +> useTieredProject) > } > } > diff --git a/scripts/spark2diffs/average.diff b/scripts/spark2diffs/average.diff index 2ac0dc5485d..541777f520d 100644 --- a/scripts/spark2diffs/average.diff +++ b/scripts/spark2diffs/average.diff @@ -1,4 +1,4 @@ -32a33,37 +13a14,18 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = > GpuAverage(childExprs.head) > diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/DateUtils.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/DateUtils.scala index d6d205f68a8..9fe9c4edc3c 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/DateUtils.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/DateUtils.scala @@ -36,7 +36,7 @@ object DateUtils { "u", "uu", "uuu", "uuuu", "uuuuu", "uuuuuu", "uuuuuuu", "uuuuuuuu", "uuuuuuuuu", "uuuuuuuuuu", "y", "yyy", "yyyyy", "yyyyyy", "yyyyyyy", "yyyyyyyy", "yyyyyyyyy", "yyyyyyyyyy", "D", "DD", "DDD", "s", "m", "H", "h", "M", "MMM", "MMMM", "MMMMM", "L", "LLL", "LLLL", "LLLLL", - "d", "S", "SS", "SSS", "SSSS", "SSSSS", "SSSSSSSSS", "SSSSSSS", "SSSSSSSS") + "d", "S", "SS", "SSSS", "SSSSS", "SSSSSSS", "SSSSSSSS", "SSSSSSSSS") // we support "yy" in some cases, but not when parsing strings // https://github.com/NVIDIA/spark-rapids/issues/2118 @@ -44,7 +44,7 @@ object DateUtils { val conversionMap = Map( "MM" -> "%m", "LL" -> "%m", "dd" -> "%d", "mm" -> "%M", "ss" -> "%S", "HH" -> "%H", - "yy" -> "%y", "yyyy" -> "%Y", "SSSSSS" -> "%f") + "yy" -> "%y", "yyyy" -> "%Y", "SSS" -> "%3f", "SSSSSS" -> "%6f") val ONE_SECOND_MICROSECONDS = 1000000 diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchUtils.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchUtils.scala new file mode 100644 index 00000000000..9e8623c1d82 --- /dev/null +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchUtils.scala @@ -0,0 +1,152 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.types.{ArrayType, DataType, DataTypes, MapType, StructType} + +/** + * Utility class with methods for calculating various metrics about GPU memory usage + * prior to allocation. + */ +object GpuBatchUtils { + + /** Validity buffers are 64 byte aligned */ + val VALIDITY_BUFFER_BOUNDARY_BYTES = 64 + + /** Validity buffers are 64 byte aligned and each byte represents 8 rows */ + val VALIDITY_BUFFER_BOUNDARY_ROWS = VALIDITY_BUFFER_BOUNDARY_BYTES * 8 + + /** Number of bytes per offset (32 bit) */ + val OFFSET_BYTES = 4 + + /** Estimate the number of rows required to meet a batch size limit */ + def estimateRowCount( + desiredBatchSizeBytes: Long, + currentBatchSize: Long, + currentBatchRowCount: Long): Int = { + assert(currentBatchRowCount > 0, "batch must contain at least one row") + val targetRowCount: Long = if (currentBatchSize > desiredBatchSizeBytes) { + currentBatchRowCount + } else if (currentBatchSize == 0) { + // batch size can be 0 when doing a count() operation and the actual data isn't needed + currentBatchRowCount + } else { + ((desiredBatchSizeBytes / currentBatchSize.floatValue()) * currentBatchRowCount).toLong + } + targetRowCount.min(Integer.MAX_VALUE).toInt + } + + /** Estimate the amount of GPU memory a batch of rows will occupy once converted */ + def estimateGpuMemory(schema: StructType, rowCount: Long): Long = { + schema.fields.indices.map(estimateGpuMemory(schema, _, rowCount)).sum + } + + /** Estimate the amount of GPU memory a batch of rows will occupy once converted */ + def estimateGpuMemory(schema: StructType, columnIndex: Int, rowCount: Long): Long = { + val field = schema.fields(columnIndex) + estimateGpuMemory(field.dataType, field.nullable, rowCount) + } + + def estimateGpuMemory(dataType: DataType, nullable: Boolean, rowCount: Long): Long = { + val validityBufferSize = if (nullable) { + calculateValidityBufferSize(rowCount) + } else { + 0 + } + val dataSize = dataType match { + case dt@DataTypes.BinaryType => + val offsetBufferSize = calculateOffsetBufferSize(rowCount) + val dataSize = dt.defaultSize * rowCount + dataSize + offsetBufferSize + case dt@DataTypes.StringType => + val offsetBufferSize = calculateOffsetBufferSize(rowCount) + val dataSize = dt.defaultSize * rowCount + dataSize + offsetBufferSize + case dt: MapType => + // The Spark default map size assumes one entry for good or bad + calculateOffsetBufferSize(rowCount) + + estimateGpuMemory(dt.keyType, false, rowCount) + + estimateGpuMemory(dt.valueType, dt.valueContainsNull, rowCount) + case dt: ArrayType => + // The Spark default array size assumes one entry for good or bad + calculateOffsetBufferSize(rowCount) + + estimateGpuMemory(dt.elementType, dt.containsNull, rowCount) + case dt: StructType => + dt.fields.map { f => + estimateGpuMemory(f.dataType, f.nullable, rowCount) + }.sum + case dt => + dt.defaultSize * rowCount + } + dataSize + validityBufferSize + } + + def calculateValidityBufferSize(rows: Long): Long = { + roundToBoundary((rows + 7)/8, 64) + } + + def calculateOffsetBufferSize(rows: Long): Long = { + (rows+1) * 4 // 32 bit offsets + } + + /** + * Generate indices which evenly splitting input batch + * + * @param rows number of rows of input batch + * @param numSplits desired number of splits + * @return splitting indices + */ + def generateSplitIndices(rows: Long, numSplits: Int): Array[Int] = { + require(rows > 0, s"invalid input rows $rows") + require(numSplits > 0, s"invalid numSplits $numSplits") + val baseIncrement = (rows / numSplits).toInt + var extraIncrements = (rows % numSplits).toInt + val indicesBuf = ArrayBuffer[Int]() + (1 until numSplits).foldLeft(0) { case (last, _) => + val current = if (extraIncrements > 0) { + extraIncrements -= 1 + last + baseIncrement + 1 + } else { + last + baseIncrement + } + indicesBuf += current + current + } + indicesBuf.toArray + } + + def isVariableWidth(dt: DataType): Boolean = !isFixedWidth(dt) + + def isFixedWidth(dt: DataType): Boolean = dt match { + case DataTypes.StringType | DataTypes.BinaryType => false + case _: ArrayType => false + case _: StructType => false + case _: MapType => false + case _ => true + } + + private def roundToBoundary(bytes: Long, boundary: Int): Long = { + val remainder = bytes % boundary + if (remainder > 0) { + bytes + boundary - remainder + } else { + bytes + } + } +} diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOptimizedCreateHiveTableAsSelectCommand.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOptimizedCreateHiveTableAsSelectCommand.scala new file mode 100644 index 00000000000..cef934c1e8f --- /dev/null +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOptimizedCreateHiveTableAsSelectCommand.scala @@ -0,0 +1,78 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids + +import java.util.Locale + +import org.apache.spark.sql.{SparkSession} +import org.apache.spark.sql.hive.execution.OptimizedCreateHiveTableAsSelectCommand +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.rapids.{GpuOrcFileFormat} + +final class OptimizedCreateHiveTableAsSelectCommandMeta( + cmd: OptimizedCreateHiveTableAsSelectCommand, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _]], + rule: DataFromReplacementRule) + extends DataWritingCommandMeta[OptimizedCreateHiveTableAsSelectCommand]( + cmd, conf, parent, rule) { + + override def tagSelfForGpu(): Unit = { + // It would be cleaner if we could simply call `cmd.getWritingCommand` and let + // InsertIntoHadoopFsRelationCommandMeta tag the result, but calling getWritingCommand + // before the table exists will crash. So this ends up replicating a portion of the logic + // from OptimizedCreateHiveTableAsSelectCommand.getWritingCommand and underlying + // utility methods to be able to tag whether we can support the optimized Hive write. + val spark = SparkSession.active + val tableDesc = cmd.tableDesc + + if (tableDesc.partitionColumnNames.nonEmpty) { + willNotWorkOnGpu("partitioned writes are not supported") + } + + if (tableDesc.bucketSpec.isDefined) { + willNotWorkOnGpu("bucketing is not supported") + } + + val serde = tableDesc.storage.serde.getOrElse("").toLowerCase(Locale.ROOT) + if (serde.contains("parquet")) { + val mergeSchemaConfKey = "spark.sql.hive.convertMetastoreParquet.mergeSchema" + val shouldMergeSchema = SQLConf.get.getConfString(mergeSchemaConfKey, "false").toBoolean + if (shouldMergeSchema) { + willNotWorkOnGpu("Merging Parquet schemas across part files is not supported, " + + s"see $mergeSchemaConfKey") + } + val options = tableDesc.properties.filterKeys(isParquetProperty) ++ + tableDesc.storage.properties + GpuParquetFileFormat.tagGpuSupport(this, spark, options, cmd.query.schema) + } else if (serde.contains("orc")) { + val options = tableDesc.properties.filterKeys(isOrcProperty) ++ + tableDesc.storage.properties + GpuOrcFileFormat.tagGpuSupport(this, spark, options, cmd.query.schema) + } else { + willNotWorkOnGpu(s"unsupported serde detected: $serde") + } + } + + // Return true for Apache ORC and Hive ORC-related configuration names. + // Note that Spark doesn't support configurations like `hive.merge.orcfile.stripe.level`. + private def isOrcProperty(key: String) = + key.startsWith("orc.") || key.contains(".orc.") + + private def isParquetProperty(key: String) = + key.startsWith("parquet.") || key.contains(".parquet.") +} diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index 8d12fe0dcd7..fe2674578db 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -37,10 +37,5 @@ object GpuOrcScan { } FileFormatChecks.tag(meta, schema, OrcFormatType, ReadFileOp) - - if (sparkSession.conf - .getOption("spark.sql.orc.mergeSchema").exists(_.toBoolean)) { - meta.willNotWorkOnGpu("mergeSchema and schema evolution is not supported yet") - } } } diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index b474bcd816b..434abdc5899 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -678,9 +678,11 @@ object GpuOverrides extends Logging { sparkSig = TypeSig.cpuAtomics)), (ParquetFormatType, FileFormatChecks( cudfRead = (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.STRUCT + - TypeSig.ARRAY + TypeSig.MAP + GpuTypeShims.additionalParquetSupportedTypes).nested(), + TypeSig.ARRAY + TypeSig.MAP + TypeSig.BINARY + + GpuTypeShims.additionalParquetSupportedTypes).nested(), cudfWrite = (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.STRUCT + - TypeSig.ARRAY + TypeSig.MAP + GpuTypeShims.additionalParquetSupportedTypes).nested(), + TypeSig.ARRAY + TypeSig.MAP + TypeSig.BINARY + + GpuTypeShims.additionalParquetSupportedTypes).nested(), sparkSig = (TypeSig.cpuAtomics + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP + TypeSig.UDT + GpuTypeShims.additionalParquetSupportedTypes).nested())), (OrcFormatType, FileFormatChecks( @@ -703,11 +705,11 @@ object GpuOverrides extends Logging { sparkSig = (TypeSig.cpuAtomics + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP + TypeSig.UDT).nested())), (IcebergFormatType, FileFormatChecks( - cudfRead = (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.STRUCT + + cudfRead = (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.STRUCT + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.MAP + GpuTypeShims.additionalParquetSupportedTypes).nested(), cudfWrite = TypeSig.none, sparkSig = (TypeSig.cpuAtomics + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP + - TypeSig.UDT + GpuTypeShims.additionalParquetSupportedTypes).nested()))) + TypeSig.BINARY + TypeSig.UDT + GpuTypeShims.additionalParquetSupportedTypes).nested()))) val commonExpressions: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Seq( expr[Literal]( @@ -715,9 +717,9 @@ object GpuOverrides extends Logging { ExprChecks.projectAndAst( TypeSig.astTypes, (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.CALENDAR - + TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT) + + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT) .nested(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT), + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT), TypeSig.all), (lit, conf, p, r) => new LiteralExprMeta(lit, conf, p, r)), expr[Signum]( @@ -730,7 +732,8 @@ object GpuOverrides extends Logging { ExprChecks.unaryProjectAndAstInputMatchesOutput( TypeSig.astTypes + GpuTypeShims.additionalCommonOperatorSupportedTypes, (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.MAP + TypeSig.ARRAY + TypeSig.STRUCT - + TypeSig.DECIMAL_128 + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), + + TypeSig.DECIMAL_128 + TypeSig.BINARY + + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), TypeSig.all), (a, conf, p, r) => new UnaryAstExprMeta[Alias](a, conf, p, r) { }), @@ -739,7 +742,7 @@ object GpuOverrides extends Logging { ExprChecks.projectAndAst( TypeSig.astTypes + GpuTypeShims.additionalArithmeticSupportedTypes, (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.MAP + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.DECIMAL_128 + + TypeSig.STRUCT + TypeSig.DECIMAL_128 + TypeSig.BINARY + GpuTypeShims.additionalArithmeticSupportedTypes).nested(), TypeSig.all), (att, conf, p, r) => new BaseExprMeta[AttributeReference](att, conf, p, r) { @@ -831,51 +834,6 @@ object GpuOverrides extends Logging { private[this] lazy val rhsDecimalType = DecimalUtil.asDecimalType(rhs.wrapped.asInstanceOf[Expression].dataType) - override def tagExprForGpu(): Unit = { - a.child match { - // Division and Multiplication of Decimal types is a little odd. Spark will cast the - // inputs to a common wider value where the scale is the max of the two input scales, - // and the precision is max of the two input non-scale portions + the new scale. Then it - // will do the divide or multiply as a BigDecimal value but lie about the return type. - // Finally here in CheckOverflow it will reset the scale and check the precision so that - // Spark knows it fits in the final desired result. - // Here we try to strip out the extra casts, etc to get to as close to the original - // query as possible. This lets us then calculate what CUDF needs to get the correct - // answer, which in some cases is a lot smaller. - case _: Divide => - val intermediatePrecision = - GpuDecimalDivide.nonRoundedIntermediateArgPrecision(lhsDecimalType, - rhsDecimalType, a.dataType) - - if (intermediatePrecision > GpuOverrides.DECIMAL128_MAX_PRECISION) { - if (conf.needDecimalGuarantees) { - binExpr.willNotWorkOnGpu(s"the intermediate precision of " + - s"$intermediatePrecision that is required to guarantee no overflow issues " + - s"for this divide is too large to be supported on the GPU") - } else { - logWarning("Decimal overflow guarantees disabled for " + - s"${lhs.dataType} / ${rhs.dataType} produces ${a.dataType} with an " + - s"intermediate precision of $intermediatePrecision") - } - } - case _: Multiply => - val intermediatePrecision = - GpuDecimalMultiply.nonRoundedIntermediatePrecision(lhsDecimalType, - rhsDecimalType, a.dataType) - if (intermediatePrecision > GpuOverrides.DECIMAL128_MAX_PRECISION) { - if (conf.needDecimalGuarantees) { - binExpr.willNotWorkOnGpu(s"the intermediate precision of " + - s"$intermediatePrecision that is required to guarantee no overflow issues " + - s"for this multiply is too large to be supported on the GPU") - } else { - logWarning("Decimal overflow guarantees disabled for " + - s"${lhs.dataType} * ${rhs.dataType} produces ${a.dataType} with an " + - s"intermediate precision of $intermediatePrecision") - } - } - case _ => // NOOP - } - } }), expr[ToDegrees]( "Converts radians to degrees", @@ -910,10 +868,10 @@ object GpuOverrides extends Logging { TypeSig.numericAndInterval, Seq( ParamCheck("lower", - TypeSig.CALENDAR + TypeSig.NULL + TypeSig.integral, + TypeSig.CALENDAR + TypeSig.NULL + TypeSig.integral + TypeSig.DECIMAL_128, TypeSig.numericAndInterval), ParamCheck("upper", - TypeSig.CALENDAR + TypeSig.NULL + TypeSig.integral, + TypeSig.CALENDAR + TypeSig.NULL + TypeSig.integral + TypeSig.DECIMAL_128, TypeSig.numericAndInterval))), (windowFrame, conf, p, r) => new GpuSpecifiedWindowFrameMeta(windowFrame, conf, p, r) ), expr[WindowSpecDefinition]( @@ -1078,11 +1036,13 @@ object GpuOverrides extends Logging { ExprChecks.mathUnaryWithAst, (a, conf, p, r) => new UnaryAstExprMeta[Acos](a, conf, p, r) { }), + // Acosh is not supported in spark 2.x expr[Asin]( "Inverse sine", ExprChecks.mathUnaryWithAst, (a, conf, p, r) => new UnaryAstExprMeta[Asin](a, conf, p, r) { }), + // Asinh is not supported in spark 2.x expr[Sqrt]( "Square root", ExprChecks.mathUnaryWithAst, @@ -1150,7 +1110,7 @@ object GpuOverrides extends Logging { "Checks if a value is null", ExprChecks.unaryProject(TypeSig.BOOLEAN, TypeSig.BOOLEAN, (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.MAP + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.DECIMAL_128 + + TypeSig.STRUCT + TypeSig.DECIMAL_128 + TypeSig.BINARY + GpuTypeShims.additionalPredicateSupportedTypes).nested(), TypeSig.all), (a, conf, p, r) => new UnaryExprMeta[IsNull](a, conf, p, r) { @@ -1159,7 +1119,7 @@ object GpuOverrides extends Logging { "Checks if a value is not null", ExprChecks.unaryProject(TypeSig.BOOLEAN, TypeSig.BOOLEAN, (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.MAP + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.DECIMAL_128+ + TypeSig.STRUCT + TypeSig.DECIMAL_128 + TypeSig.BINARY + GpuTypeShims.additionalPredicateSupportedTypes).nested(), TypeSig.all), (a, conf, p, r) => new UnaryExprMeta[IsNotNull](a, conf, p, r) { @@ -1185,8 +1145,8 @@ object GpuOverrides extends Logging { "Checks if number of non null/Nan values is greater than a given value", ExprChecks.projectOnly(TypeSig.BOOLEAN, TypeSig.BOOLEAN, repeatingParamCheck = Some(RepeatingParamCheck("input", - (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP + - TypeSig.ARRAY + TypeSig.STRUCT).nested(), + (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.BINARY + + TypeSig.MAP + TypeSig.ARRAY + TypeSig.STRUCT).nested(), TypeSig.all))), (a, conf, p, r) => new ExprMeta[AtLeastNNonNulls](a, conf, p, r) { }), @@ -1263,11 +1223,11 @@ object GpuOverrides extends Logging { expr[Coalesce] ( "Returns the first non-null argument if exists. Otherwise, null", ExprChecks.projectOnly( - (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + + (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.BINARY + GpuTypeShims.additionalArithmeticSupportedTypes).nested(), TypeSig.all, repeatingParamCheck = Some(RepeatingParamCheck("param", - (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT+ + (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.BINARY + GpuTypeShims.additionalArithmeticSupportedTypes).nested(), TypeSig.all))), (a, conf, p, r) => new ExprMeta[Coalesce](a, conf, p, r) { @@ -1295,6 +1255,7 @@ object GpuOverrides extends Logging { ExprChecks.mathUnaryWithAst, (a, conf, p, r) => new UnaryAstExprMeta[Atan](a, conf, p, r) { }), + // Atanh is not supported in spark 2.x expr[Cos]( "Cosine", ExprChecks.mathUnaryWithAst, @@ -1373,6 +1334,8 @@ object GpuOverrides extends Logging { ExprChecks.mathUnaryWithAst, (a, conf, p, r) => new UnaryAstExprMeta[Tan](a, conf, p, r) { }), + // NormalizeNaNAndZero is not supported in spark 2.x + // KnownFloatingPointNormalized is not supported in spark 2.x expr[KnownNotNull]( "Tag an expression as known to not be null", ExprChecks.unaryProjectInputMatchesOutput( @@ -1405,6 +1368,7 @@ object GpuOverrides extends Logging { } }), + // DateAddInterval is not supported in spark 2.x expr[DateFormatClass]( "Converts timestamp to a value of string in the format specified by the date format", ExprChecks.binaryProject(TypeSig.STRING, TypeSig.STRING, @@ -1482,12 +1446,30 @@ object GpuOverrides extends Logging { TypeSig.STRING)), (a, conf, p, r) => new UnixTimeExprMeta[FromUnixTime](a, conf, p, r) { }), + expr[FromUTCTimestamp]( + "Render the input UTC timestamp in the input timezone", + ExprChecks.binaryProject(TypeSig.TIMESTAMP, TypeSig.TIMESTAMP, + ("timestamp", TypeSig.TIMESTAMP, TypeSig.TIMESTAMP), + ("timezone", TypeSig.lit(TypeEnum.STRING) + .withPsNote(TypeEnum.STRING, "Only timezones equivalent to UTC are supported"), + TypeSig.lit(TypeEnum.STRING))), + (a, conf, p, r) => new FromUTCTimestampExprMeta(a, conf, p, r) + ), expr[Pmod]( "Pmod", ExprChecks.binaryProject(TypeSig.gpuNumeric, TypeSig.cpuNumeric, - ("lhs", TypeSig.gpuNumeric, TypeSig.cpuNumeric), + ("lhs", TypeSig.gpuNumeric.withPsNote(TypeEnum.DECIMAL, + s"decimals with precision ${DecimalType.MAX_PRECISION} are not supported"), + TypeSig.cpuNumeric), ("rhs", TypeSig.gpuNumeric, TypeSig.cpuNumeric)), (a, conf, p, r) => new BinaryExprMeta[Pmod](a, conf, p, r) { + override def tagExprForGpu(): Unit = { + a.dataType match { + case dt: DecimalType if dt.precision == DecimalType.MAX_PRECISION => + willNotWorkOnGpu("pmod at maximum decimal precision is not supported") + case _ => + } + } }), expr[Add]( "Addition", @@ -1556,11 +1538,11 @@ object GpuOverrides extends Logging { "Check if the values are equal including nulls <=>", ExprChecks.binaryProject( TypeSig.BOOLEAN, TypeSig.BOOLEAN, - ("lhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("lhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.comparable), - ("rhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("rhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.comparable)), (a, conf, p, r) => new BinaryExprMeta[EqualNullSafe](a, conf, p, r) { }), @@ -1569,11 +1551,11 @@ object GpuOverrides extends Logging { ExprChecks.binaryProjectAndAst( TypeSig.comparisonAstTypes, TypeSig.BOOLEAN, TypeSig.BOOLEAN, - ("lhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("lhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.comparable), - ("rhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("rhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.comparable)), (a, conf, p, r) => new BinaryAstExprMeta[EqualTo](a, conf, p, r) { }), @@ -1582,11 +1564,11 @@ object GpuOverrides extends Logging { ExprChecks.binaryProjectAndAst( TypeSig.comparisonAstTypes, TypeSig.BOOLEAN, TypeSig.BOOLEAN, - ("lhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("lhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.orderable), - ("rhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("rhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.orderable)), (a, conf, p, r) => new BinaryAstExprMeta[GreaterThan](a, conf, p, r) { }), @@ -1595,11 +1577,11 @@ object GpuOverrides extends Logging { ExprChecks.binaryProjectAndAst( TypeSig.comparisonAstTypes, TypeSig.BOOLEAN, TypeSig.BOOLEAN, - ("lhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("lhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.orderable), - ("rhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("rhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.orderable)), (a, conf, p, r) => new BinaryAstExprMeta[GreaterThanOrEqual](a, conf, p, r) { }), @@ -1639,11 +1621,11 @@ object GpuOverrides extends Logging { ExprChecks.binaryProjectAndAst( TypeSig.comparisonAstTypes, TypeSig.BOOLEAN, TypeSig.BOOLEAN, - ("lhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("lhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.orderable), - ("rhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("rhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.orderable)), (a, conf, p, r) => new BinaryAstExprMeta[LessThan](a, conf, p, r) { }), @@ -1652,11 +1634,11 @@ object GpuOverrides extends Logging { ExprChecks.binaryProjectAndAst( TypeSig.comparisonAstTypes, TypeSig.BOOLEAN, TypeSig.BOOLEAN, - ("lhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("lhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.orderable), - ("rhs", TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - GpuTypeShims.additionalPredicateSupportedTypes, + ("rhs", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + GpuTypeShims.additionalPredicateSupportedTypes + TypeSig.STRUCT).nested(), TypeSig.orderable)), (a, conf, p, r) => new BinaryAstExprMeta[LessThanOrEqual](a, conf, p, r) { }), @@ -1668,17 +1650,17 @@ object GpuOverrides extends Logging { expr[If]( "IF expression", ExprChecks.projectOnly( - (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + - TypeSig.MAP + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), + (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + + TypeSig.BINARY + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), TypeSig.all, Seq(ParamCheck("predicate", TypeSig.BOOLEAN, TypeSig.BOOLEAN), ParamCheck("trueValue", - (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + - TypeSig.MAP + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), + (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + + TypeSig.BINARY + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), TypeSig.all), ParamCheck("falseValue", - (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + - TypeSig.MAP + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), + (_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + + TypeSig.BINARY + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), TypeSig.all))), (a, conf, p, r) => new ExprMeta[If](a, conf, p, r) { }), @@ -1693,10 +1675,7 @@ object GpuOverrides extends Logging { expr[Divide]( "Division", ExprChecks.binaryProject( - TypeSig.DOUBLE + TypeSig.DECIMAL_128 + - TypeSig.psNote(TypeEnum.DECIMAL, - "Because of Spark's inner workings the full range of decimal precision " + - "(even for 128-bit values) is not supported."), + TypeSig.DOUBLE + TypeSig.DECIMAL_128, TypeSig.DOUBLE + TypeSig.DECIMAL_128, ("lhs", TypeSig.DOUBLE + TypeSig.DECIMAL_128, TypeSig.DOUBLE + TypeSig.DECIMAL_128), @@ -1704,6 +1683,7 @@ object GpuOverrides extends Logging { TypeSig.DOUBLE + TypeSig.DECIMAL_128)), (a, conf, p, r) => new BinaryExprMeta[Divide](a, conf, p, r) { }), + // IntegralDivide is not supported in spark 2.x expr[Remainder]( "Remainder or modulo", ExprChecks.binaryProject( @@ -2029,10 +2009,10 @@ object GpuOverrides extends Logging { "Gets the named field of the struct", ExprChecks.unaryProject( (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.NULL + - TypeSig.DECIMAL_128).nested(), + TypeSig.DECIMAL_128 + TypeSig.BINARY).nested(), TypeSig.all, TypeSig.STRUCT.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.MAP + TypeSig.NULL + TypeSig.DECIMAL_128), + TypeSig.STRUCT + TypeSig.MAP + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.BINARY), TypeSig.STRUCT.nested(TypeSig.all)), (expr, conf, p, r) => new UnaryExprMeta[GetStructField](expr, conf, p, r) { }), @@ -2040,10 +2020,10 @@ object GpuOverrides extends Logging { "Gets the field at `ordinal` in the Array", ExprChecks.binaryProject( (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL + - TypeSig.DECIMAL_128 + TypeSig.MAP).nested(), + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY).nested(), TypeSig.all, ("array", TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP), + TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY), TypeSig.ARRAY.nested(TypeSig.all)), ("ordinal", TypeSig.INT, TypeSig.INT)), (in, conf, p, r) => new BinaryExprMeta[GetArrayItem](in, conf, p, r) { @@ -2052,23 +2032,30 @@ object GpuOverrides extends Logging { "Gets Value from a Map based on a key", ExprChecks.binaryProject( (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL + - TypeSig.DECIMAL_128 + TypeSig.MAP).nested(), + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY).nested(), TypeSig.all, ("map", TypeSig.MAP.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + - TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP), TypeSig.MAP.nested(TypeSig.all)), + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY), + TypeSig.MAP.nested(TypeSig.all)), ("key", TypeSig.commonCudfTypes + TypeSig.DECIMAL_128, TypeSig.all)), (in, conf, p, r) => new BinaryExprMeta[GetMapValue](in, conf, p, r) { + override def tagExprForGpu(): Unit = { + if (isLit(in.left) && (!isLit(in.right))) { + willNotWorkOnGpu("Looking up Map Scalars with Key Vectors " + + "is not currently unsupported.") + } + } }), expr[ElementAt]( "Returns element of array at given(1-based) index in value if column is array. " + "Returns value for the given key in value if column is map", ExprChecks.binaryProject( (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL + - TypeSig.DECIMAL_128 + TypeSig.MAP).nested(), TypeSig.all, + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY).nested(), TypeSig.all, ("array/map", TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP) + + TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY) + TypeSig.MAP.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + - TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP) + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY) .withPsNote(TypeEnum.MAP ,"If it's map, only primitive key types are supported."), TypeSig.ARRAY.nested(TypeSig.all) + TypeSig.MAP.nested(TypeSig.all)), ("index/key", (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128) @@ -2085,21 +2072,22 @@ object GpuOverrides extends Logging { // Match exactly with the checks for GetMapValue ExprChecks.binaryProject( (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL + - TypeSig.DECIMAL_128 + TypeSig.MAP).nested(), + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY).nested(), TypeSig.all, ("map", TypeSig.MAP.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + - TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP), + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY), TypeSig.MAP.nested(TypeSig.all)), ("key", TypeSig.commonCudfTypes + TypeSig.DECIMAL_128, TypeSig.all)) case _: ArrayType => // Match exactly with the checks for GetArrayItem ExprChecks.binaryProject( (TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.NULL + - TypeSig.DECIMAL_128 + TypeSig.MAP).nested(), + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY).nested(), TypeSig.all, ("array", TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP), + TypeSig.STRUCT + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP + + TypeSig.BINARY), TypeSig.ARRAY.nested(TypeSig.all)), ("ordinal", TypeSig.INT, TypeSig.INT)) case _ => throw new IllegalStateException("Only Array or Map is supported as input.") @@ -2111,10 +2099,10 @@ object GpuOverrides extends Logging { "Returns an unordered array containing the keys of the map", ExprChecks.unaryProject( TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + - TypeSig.ARRAY + TypeSig.STRUCT).nested(), + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.BINARY).nested(), TypeSig.ARRAY.nested(TypeSig.all - TypeSig.MAP), // Maps cannot have other maps as keys TypeSig.MAP.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY), TypeSig.MAP.nested(TypeSig.all)), (in, conf, p, r) => new UnaryExprMeta[MapKeys](in, conf, p, r) { }), @@ -2122,13 +2110,14 @@ object GpuOverrides extends Logging { "Returns an unordered array containing the values of the map", ExprChecks.unaryProject( TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY), TypeSig.ARRAY.nested(TypeSig.all), TypeSig.MAP.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY), TypeSig.MAP.nested(TypeSig.all)), (in, conf, p, r) => new UnaryExprMeta[MapValues](in, conf, p, r) { }), + // MapEntries is not supported in spark 2.x expr[StringToMap]( "Creates a map after splitting the input string into pairs of key-value strings", ExprChecks.projectOnly(TypeSig.MAP.nested(TypeSig.STRING), TypeSig.MAP.nested(TypeSig.STRING), @@ -2204,14 +2193,14 @@ object GpuOverrides extends Logging { ExprChecks.projectOnly( TypeSig.ARRAY.nested(TypeSig.gpuNumeric + TypeSig.NULL + TypeSig.STRING + TypeSig.BOOLEAN + TypeSig.DATE + TypeSig.TIMESTAMP + - TypeSig.ARRAY + TypeSig.STRUCT), + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.BINARY), TypeSig.ARRAY.nested(TypeSig.all), repeatingParamCheck = Some(RepeatingParamCheck("arg", TypeSig.gpuNumeric + TypeSig.NULL + TypeSig.STRING + - TypeSig.BOOLEAN + TypeSig.DATE + TypeSig.TIMESTAMP + TypeSig.STRUCT + + TypeSig.BOOLEAN + TypeSig.DATE + TypeSig.TIMESTAMP + TypeSig.STRUCT + TypeSig.BINARY + TypeSig.ARRAY.nested(TypeSig.gpuNumeric + TypeSig.NULL + TypeSig.STRING + TypeSig.BOOLEAN + TypeSig.DATE + TypeSig.TIMESTAMP + TypeSig.STRUCT + - TypeSig.ARRAY), + TypeSig.ARRAY + TypeSig.BINARY), TypeSig.all))), (in, conf, p, r) => new ExprMeta[CreateArray](in, conf, p, r) { @@ -2282,12 +2271,12 @@ object GpuOverrides extends Logging { "Returns a merged array of structs in which the N-th struct contains" + " all N-th values of input arrays.", ExprChecks.projectOnly(TypeSig.ARRAY.nested( - TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + + TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), TypeSig.ARRAY.nested(TypeSig.all), repeatingParamCheck = Some(RepeatingParamCheck("children", TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), TypeSig.ARRAY.nested(TypeSig.all)))), (in, conf, p, r) => new ExprMeta[ArraysZip](in, conf, p, r) { } @@ -2348,6 +2337,8 @@ object GpuOverrides extends Logging { "3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were " + "not treated as being equal. We have chosen to break with compatibility for " + "the older versions of Spark in this instance and handle NaNs the same as 3.1.3+"), + // TransformKeys is not supported in Spark 2.x + // TransformValues is not supported in Spark 2.x // spark 2.x doesn't have MapFilter expr[StringLocate]( "Substring search operator", @@ -2433,21 +2424,17 @@ object GpuOverrides extends Logging { "List/String concatenate", ExprChecks.projectOnly((TypeSig.STRING + TypeSig.ARRAY).nested( TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY), (TypeSig.STRING + TypeSig.BINARY + TypeSig.ARRAY).nested(TypeSig.all), repeatingParamCheck = Some(RepeatingParamCheck("input", (TypeSig.STRING + TypeSig.ARRAY).nested( TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY), (TypeSig.STRING + TypeSig.BINARY + TypeSig.ARRAY).nested(TypeSig.all)))), (a, conf, p, r) => new ComplexTypeMergingExprMeta[Concat](a, conf, p, r) { }), expr[MapConcat]( "Returns the union of all the given maps", - // Currently, GpuMapConcat supports nested values but not nested keys. - // We will add the nested key support after - // cuDF can fully support nested types in lists::drop_list_duplicates. - // Issue link: https://github.com/rapidsai/cudf/issues/11093 ExprChecks.projectOnly(TypeSig.MAP.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.NULL + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), TypeSig.MAP.nested(TypeSig.all), @@ -2456,13 +2443,6 @@ object GpuOverrides extends Logging { TypeSig.NULL + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), TypeSig.MAP.nested(TypeSig.all)))), (a, conf, p, r) => new ComplexTypeMergingExprMeta[MapConcat](a, conf, p, r) { - override def tagExprForGpu(): Unit = { - a.dataType.keyType match { - case MapType(_,_,_) | ArrayType(_,_) | StructType(_) => willNotWorkOnGpu( - s"GpuMapConcat does not currently support the key type ${a.dataType.keyType}.") - case _ => - } - } }), expr[ConcatWs]( "Concatenates multiple input strings or array of strings into a single " + @@ -2540,7 +2520,7 @@ object GpuOverrides extends Logging { "The size of an array or a map", ExprChecks.unaryProject(TypeSig.INT, TypeSig.INT, (TypeSig.ARRAY + TypeSig.MAP).nested(TypeSig.commonCudfTypes + TypeSig.NULL - + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + + TypeSig.DECIMAL_128 + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), (TypeSig.ARRAY + TypeSig.MAP).nested(TypeSig.all)), (a, conf, p, r) => new UnaryExprMeta[Size](a, conf, p, r) { }), @@ -2563,10 +2543,10 @@ object GpuOverrides extends Logging { // Here is a walk-around representation, since multi-level nested type is not supported yet. // related issue: https://github.com/NVIDIA/spark-rapids/issues/1901 TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), TypeSig.ARRAY.nested(TypeSig.all), (TypeSig.ARRAY + TypeSig.MAP).nested(TypeSig.commonCudfTypes + TypeSig.NULL + - TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.DECIMAL_128 + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), (TypeSig.ARRAY + TypeSig.MAP).nested(TypeSig.all)), (a, conf, p, r) => new GeneratorExprMeta[Explode](a, conf, p, r) { override val supportOuter: Boolean = true @@ -2577,10 +2557,10 @@ object GpuOverrides extends Logging { // Here is a walk-around representation, since multi-level nested type is not supported yet. // related issue: https://github.com/NVIDIA/spark-rapids/issues/1901 TypeSig.ARRAY.nested(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), TypeSig.ARRAY.nested(TypeSig.all), (TypeSig.ARRAY + TypeSig.MAP).nested(TypeSig.commonCudfTypes + TypeSig.NULL + - TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), + TypeSig.DECIMAL_128 + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP), (TypeSig.ARRAY + TypeSig.MAP).nested(TypeSig.all)), (a, conf, p, r) => new GeneratorExprMeta[PosExplode](a, conf, p, r) { override val supportOuter: Boolean = true @@ -2759,11 +2739,26 @@ object GpuOverrides extends Logging { // This needs to match what murmur3 supports. PartChecks(RepeatingParamCheck("hash_key", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.STRUCT).nested(), TypeSig.all)), + TypeSig.STRUCT + TypeSig.ARRAY).nested(), + TypeSig.all) + ), (hp, conf, p, r) => new PartMeta[HashPartitioning](hp, conf, p, r) { override val childExprs: Seq[BaseExprMeta[_]] = hp.expressions.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + override def tagPartForGpu(): Unit = { + val arrayWithStructsHashing = hp.expressions.exists(e => + TrampolineUtil.dataTypeExistsRecursively(e.dataType, + dt => dt match { + case ArrayType(_: StructType, _) => true + case _ => false + }) + ) + if (arrayWithStructsHashing) { + willNotWorkOnGpu("hashing arrays with structs is not supported") + } + } + }), part[RangePartitioning]( "Range partitioning", @@ -2795,7 +2790,7 @@ object GpuOverrides extends Logging { .map(r => r.wrap(writeCmd, conf, parent, r).asInstanceOf[DataWritingCommandMeta[INPUT]]) .getOrElse(new RuleNotFoundDataWritingCommandMeta(writeCmd, conf, parent)) - val dataWriteCmds: Map[Class[_ <: DataWritingCommand], + val commonDataWriteCmds: Map[Class[_ <: DataWritingCommand], DataWritingCommandRule[_ <: DataWritingCommand]] = Seq( dataWriteCmd[InsertIntoHadoopFsRelationCommand]( "Write to Hadoop filesystem", @@ -2805,6 +2800,10 @@ object GpuOverrides extends Logging { (a, conf, p, r) => new CreateDataSourceTableAsSelectCommandMeta(a, conf, p, r)) ).map(r => (r.getClassFor.asSubclass(classOf[DataWritingCommand]), r)).toMap + val dataWriteCmds: Map[Class[_ <: DataWritingCommand], + DataWritingCommandRule[_ <: DataWritingCommand]] = + commonDataWriteCmds ++ GpuHiveOverrides.dataWriteCmds + def wrapPlan[INPUT <: SparkPlan]( plan: INPUT, conf: RapidsConf, @@ -2817,15 +2816,15 @@ object GpuOverrides extends Logging { exec[GenerateExec] ( "The backend for operations that generate more output rows than input rows like explode", ExecChecks( - (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.ARRAY + - TypeSig.STRUCT + TypeSig.MAP).nested(), + (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.BINARY + + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP).nested(), TypeSig.all), (gen, conf, p, r) => new GpuGenerateExecSparkPlanMeta(gen, conf, p, r)), exec[ProjectExec]( "The backend for most select, withColumn and dropColumn statements", ExecChecks( (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.STRUCT + TypeSig.MAP + - TypeSig.ARRAY + TypeSig.DECIMAL_128 + + TypeSig.ARRAY + TypeSig.DECIMAL_128 + TypeSig.BINARY + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), TypeSig.all), (proj, conf, p, r) => new GpuProjectExecMeta(proj, conf, p, r)), @@ -2837,7 +2836,7 @@ object GpuOverrides extends Logging { exec[CoalesceExec]( "The backend for the dataframe coalesce method", ExecChecks((_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.STRUCT + TypeSig.ARRAY + - TypeSig.MAP + GpuTypeShims.additionalArithmeticSupportedTypes).nested(), + TypeSig.MAP + TypeSig.BINARY + GpuTypeShims.additionalArithmeticSupportedTypes).nested(), TypeSig.all), (coalesce, conf, parent, r) => new SparkPlanMeta[CoalesceExec](coalesce, conf, parent, r) { }), @@ -2901,19 +2900,19 @@ object GpuOverrides extends Logging { exec[FilterExec]( "The backend for most filter statements", ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.STRUCT + TypeSig.MAP + - TypeSig.ARRAY + TypeSig.DECIMAL_128 + + TypeSig.ARRAY + TypeSig.DECIMAL_128 + TypeSig.BINARY + GpuTypeShims.additionalCommonOperatorSupportedTypes).nested(), TypeSig.all), (filter, conf, p, r) => new SparkPlanMeta[FilterExec](filter, conf, p, r) { }), exec[ShuffleExchangeExec]( "The backend for most data being exchanged between processes", - ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + GpuTypeShims.additionalArithmeticSupportedTypes).nested() .withPsNote(TypeEnum.STRUCT, "Round-robin partitioning is not supported for nested " + s"structs if ${SQLConf.SORT_BEFORE_REPARTITION.key} is true") .withPsNote( - Seq(TypeEnum.ARRAY, TypeEnum.MAP), + Seq(TypeEnum.MAP), "Round-robin partitioning is not supported if " + s"${SQLConf.SORT_BEFORE_REPARTITION.key} is true"), TypeSig.all), @@ -2929,7 +2928,7 @@ object GpuOverrides extends Logging { }), exec[BroadcastExchangeExec]( "The backend for broadcast exchange of data", - ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP).nested(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.STRUCT), TypeSig.all), @@ -2946,9 +2945,9 @@ object GpuOverrides extends Logging { (join, conf, p, r) => new GpuBroadcastNestedLoopJoinMeta(join, conf, p, r)), exec[CartesianProductExec]( "Implementation of join using brute force", - ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT) - .nested(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + .nested(TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.BINARY + TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT), TypeSig.all), (join, conf, p, r) => new SparkPlanMeta[CartesianProductExec](join, conf, p, r) { @@ -2962,10 +2961,12 @@ object GpuOverrides extends Logging { "The backend for hash based aggregations", ExecChecks( (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.MAP + TypeSig.ARRAY + TypeSig.STRUCT) + TypeSig.MAP + TypeSig.STRUCT + TypeSig.ARRAY) .nested() - .withPsNote(Seq(TypeEnum.ARRAY, TypeEnum.MAP), + .withPsNote(TypeEnum.MAP, "not allowed for grouping expressions") + .withPsNote(TypeEnum.ARRAY, + "not allowed for grouping expressions if containing Struct as child") .withPsNote(TypeEnum.STRUCT, "not allowed for grouping expressions if containing Array or Map as child"), TypeSig.all), @@ -2977,14 +2978,14 @@ object GpuOverrides extends Logging { exec[SortAggregateExec]( "The backend for sort based aggregations", // SPARK 2.x we can't check for the TypedImperativeAggregate properly so - // map/arrya/struct left off + // map/array/struct left off ExecChecks( (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + TypeSig.MAP + TypeSig.BINARY) .nested() + .withPsNote(TypeEnum.MAP, "not allowed for grouping expressions") .withPsNote(TypeEnum.BINARY, "only allowed when aggregate buffers can be " + - "converted between CPU and GPU") - .withPsNote(TypeEnum.MAP, "not allowed for grouping expressions"), + "converted between CPU and GPU"), TypeSig.all), (agg, conf, p, r) => new GpuSortAggregateExecMeta(agg, conf, p, r)), // SPARK 2.x we can't check for the TypedImperativeAggregate properly so don't say we do the diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala index 97471b03e05..6b520d4ea71 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala @@ -33,10 +33,22 @@ object GpuParquetFileFormat { val sqlConf = spark.sessionState.conf val parquetOptions = new ParquetOptions(options, sqlConf) - val columnEncryption = options.getOrElse("parquet.encryption.column.keys", "") - val footerEncryption = options.getOrElse("parquet.encryption.footer.key", "") + // lookup encryption keys in the options, then Hadoop conf, then Spark runtime conf + def lookupEncryptionConfig(key: String): String = { + options.getOrElse(key, { + val hadoopConf = spark.sparkContext.hadoopConfiguration.get(key, "") + if (hadoopConf.nonEmpty) { + hadoopConf + } else { + spark.conf.get(key, "") + } + }) + } + + val columnEncryption = lookupEncryptionConfig("parquet.encryption.column.keys") + val footerEncryption = lookupEncryptionConfig("parquet.encryption.footer.key") - if (!columnEncryption.isEmpty || !footerEncryption.isEmpty) { + if (columnEncryption.nonEmpty || footerEncryption.nonEmpty) { meta.willNotWorkOnGpu("Encryption is not yet supported on GPU. If encrypted Parquet " + "writes are not required unset the \"parquet.encryption.column.keys\" and " + "\"parquet.encryption.footer.key\" in Parquet options") diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala index c21d08f61b0..d9cca11a44f 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala @@ -21,9 +21,6 @@ import org.apache.spark.sql.execution.FileSourceScanExec object GpuReadOrcFileFormat { def tagSupport(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { val fsse = meta.wrapped - if (fsse.relation.options.getOrElse("mergeSchema", "false").toBoolean) { - meta.willNotWorkOnGpu("mergeSchema and schema evolution is not supported yet") - } GpuOrcScan.tagSupport( fsse.sqlContext.sparkSession, fsse.requiredSchema, diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRegExpReplaceMeta.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRegExpReplaceMeta.scala index 93b102f9dce..21ee30bd7f5 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRegExpReplaceMeta.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRegExpReplaceMeta.scala @@ -49,9 +49,10 @@ class GpuRegExpReplaceMeta( try { javaPattern = Some(s.toString()) val (pat, repl) = - new CudfRegexTranspiler(RegexReplaceMode).transpile(s.toString, replacement) - cudfPattern = Some(pat) - repl.map(GpuRegExpUtils.backrefConversion).foreach { + new CudfRegexTranspiler(RegexReplaceMode).getTranspiledAST(s.toString, replacement) + GpuRegExpUtils.validateRegExpComplexity(this, pat) + cudfPattern = Some(pat.toRegexString) + repl.map { r => GpuRegExpUtils.backrefConversion(r.toRegexString) }.foreach { case (hasBackref, convertedRep) => containsBackref = hasBackref replacement = Some(GpuRegExpUtils.unescapeReplaceString(convertedRep)) @@ -154,6 +155,13 @@ object GpuRegExpUtils { } } + def validateRegExpComplexity(meta: ExprMeta[_], regex: RegexAST): Unit = { + if(!RegexComplexityEstimator.isValid(meta.conf, regex)) { + meta.willNotWorkOnGpu(s"estimated memory needed for regular expression exceeds the maximum." + + s" Set ${RapidsConf.REGEXP_MAX_STATE_MEMORY_BYTES} to change it.") + } + } + /** * Recursively check if pattern contains only zero-match repetitions * ?, *, {0,}, or {0,n} or any combination of them. diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowMeta.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowMeta.scala index be11ec177c8..081e045d966 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowMeta.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowMeta.scala @@ -34,7 +34,7 @@ abstract class GpuBaseWindowExecMeta[WindowExecType <: SparkPlan] (windowExec: W conf: RapidsConf, parent: Option[RapidsMeta[_, _]], rule: DataFromReplacementRule) - extends SparkPlanMeta[WindowExecType](windowExec, conf, parent, rule) with Logging { + extends SparkPlanMeta[WindowExecType](windowExec, conf, parent, rule) { /** * Extracts window-expression from WindowExecType. diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/HiveProvider.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/HiveProvider.scala index 9f2f67b3673..58027aa4210 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/HiveProvider.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/HiveProvider.scala @@ -17,6 +17,7 @@ package com.nvidia.spark.rapids import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.execution.command.DataWritingCommand /** * The subclass of HiveProvider imports spark-hive classes. This file should not imports @@ -24,5 +25,8 @@ import org.apache.spark.sql.catalyst.expressions.Expression * runtime. Details see: https://github.com/NVIDIA/spark-rapids/issues/5648 */ trait HiveProvider { + def getDataWriteCmds: Map[Class[_ <: DataWritingCommand], + DataWritingCommandRule[_ <: DataWritingCommand]] + def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] } diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index dd7fbaf128d..818337c7c6a 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -715,6 +715,12 @@ object RapidsConf { .booleanConf .createWithDefault(false) + val ENABLE_TIERED_PROJECT = conf("spark.rapids.sql.tiered.project.enabled") + .doc("Enable tiered project for aggregations.") + .internal() + .booleanConf + .createWithDefault(true) + // FILE FORMATS val MULTITHREAD_READ_NUM_THREADS = conf("spark.rapids.sql.multiThreadedRead.numThreads") .doc("The maximum number of threads on each executor to use for reading small " + @@ -855,6 +861,19 @@ object RapidsConf { .booleanConf .createWithDefault(true) + val ENABLE_ORC_FLOAT_TYPES_TO_STRING = + conf("spark.rapids.sql.format.orc.floatTypesToString.enable") + .doc("When reading an ORC file, the source data schemas(schemas of ORC file) may differ " + + "from the target schemas (schemas of the reader), we need to handle the castings from " + + "source type to target type. Since float/double numbers in GPU have different precision " + + "with CPU, when casting float/double to string, the result of GPU is different from " + + "result of CPU spark. Its default value is `true` (this means the strings result will " + + "differ from result of CPU). If it's set `false` explicitly and there exists casting " + + "from float/double to string in the job, then such behavior will cause an exception, " + + "and the job will fail.") + .booleanConf + .createWithDefault(true) + val ORC_READER_TYPE = conf("spark.rapids.sql.format.orc.reader.type") .doc("Sets the ORC reader type. We support different types that are optimized for " + "different environments. The original Spark style reader can be selected by setting this " + @@ -1050,6 +1069,13 @@ object RapidsConf { .booleanConf .createWithDefault(true) + val ENABLE_RANGE_WINDOW_DECIMAL: ConfEntryWithDefault[Boolean] = + conf("spark.rapids.sql.window.range.decimal.enabled") + .doc("When set to false, this disables the range window acceleration for the " + + "DECIMAL type order-by column") + .booleanConf + .createWithDefault(true) + val ENABLE_REGEXP = conf("spark.rapids.sql.regexp.enabled") .doc("Specifies whether supported regular expressions will be evaluated on the GPU. " + "Unsupported expressions will fall back to CPU. However, there are some known edge cases " + @@ -1059,6 +1085,16 @@ object RapidsConf { .booleanConf .createWithDefault(true) + val REGEXP_MAX_STATE_MEMORY_BYTES = conf("spark.rapids.sql.regexp.maxStateMemoryBytes") + .doc("Specifies the maximum memory on GPU to be used for regular expressions." + + "The memory usage is an estimate based on an upper-bound approximation on the " + + "complexity of the regular expression. Note that the actual memory usage may " + + "still be higher than this estimate depending on the number of rows in the data" + + "column and the input strings themselves. It is recommended to not set this to " + + s"more than 3 times ${GPU_BATCH_SIZE_BYTES.key}") + .bytesConf(ByteUnit.BYTE) + .createWithDefault(Integer.MAX_VALUE) + // INTERNAL TEST AND DEBUG CONFIGS val TEST_CONF = conf("spark.rapids.sql.test.enabled") @@ -1084,6 +1120,12 @@ object RapidsConf { .toSequence .createWithDefault(Nil) + val LOG_TRANSFORMATIONS = conf("spark.rapids.sql.debug.logTransformations") + .doc("When enabled, all query transformations will be logged.") + .internal() + .booleanConf + .createWithDefault(false) + val PARQUET_DEBUG_DUMP_PREFIX = conf("spark.rapids.sql.parquet.debug.dumpPrefix") .doc("A path prefix where Parquet split file data is dumped for debugging.") .internal() @@ -1274,12 +1316,33 @@ object RapidsConf { .bytesConf(ByteUnit.BYTE) .createWithDefault(64 * 1024) + val SHUFFLE_MULTITHREADED_MAX_BYTES_IN_FLIGHT = + conf("spark.rapids.shuffle.multiThreaded.maxBytesInFlight") + .doc("The size limit, in bytes, that the RAPIDS shuffle manager configured in " + + "\"MULTITHREADED\" mode will allow to be deserialized concurrently.") + .bytesConf(ByteUnit.BYTE) + .createWithDefault(Integer.MAX_VALUE) + val SHUFFLE_MULTITHREADED_WRITER_THREADS = conf("spark.rapids.shuffle.multiThreaded.writer.threads") - .doc("The number of threads to use for writing shuffle blocks per executor.") + .doc("The number of threads to use for writing shuffle blocks per executor in the " + + "RAPIDS shuffle manager configured in \"MULTITHREADED\" mode. " + + "There are two special values: " + + "0 = feature is disabled, falls back to Spark built-in shuffle writer; " + + "1 = our implementation of Spark's built-in shuffle writer with extra metrics.") .integerConf .createWithDefault(20) + val SHUFFLE_MULTITHREADED_READER_THREADS = + conf("spark.rapids.shuffle.multiThreaded.reader.threads") + .doc("The number of threads to use for reading shuffle blocks per executor in the " + + "RAPIDS shuffle manager configured in \"MULTITHREADED\" mode. " + + "There are two special values: " + + "0 = feature is disabled, falls back to Spark built-in shuffle reader; " + + "1 = our implementation of Spark's built-in shuffle reader with extra metrics.") + .integerConf + .createWithDefault(20) + // ALLUXIO CONFIGS val ALLUXIO_PATHS_REPLACE = conf("spark.rapids.alluxio.pathsToReplace") @@ -1333,6 +1396,18 @@ object RapidsConf { .toSequence .createWithDefault(Seq("su", "ubuntu", "-c", "/opt/alluxio-2.8.0/bin/alluxio")) + val ALLUXIO_REPLACEMENT_ALGO = conf("spark.rapids.alluxio.replacement.algo") + .doc("The algorithm used when replacing the UFS path with the Alluxio path. CONVERT_TIME " + + "and SELECTION_TIME are the valid options. CONVERT_TIME indicates that we do it when " + + "we convert it to a GPU file read, this has extra overhead of creating an entirely new " + + "file index, which requires listing the files and getting all new file info from Alluxio. " + + "SELECTION_TIME indicates we do it when the file reader is selecting the partitions " + + "to process and just replaces the path without fetching the file information again, this " + + "is faster but doesn't update locality information if that were to work with Alluxio.") + .stringConf + .checkValues(Set("CONVERT_TIME", "SELECTION_TIME")) + .createWithDefault("SELECTION_TIME") + // USER FACING DEBUG CONFIGS val SHUFFLE_COMPRESSION_MAX_BATCH_MEMORY = @@ -1499,6 +1574,32 @@ object RapidsConf { .booleanConf .createWithDefault(value = true) + val NUM_FILES_FILTER_PARALLEL = conf("spark.rapids.sql.coalescing.reader.numFilterParallel") + .doc("This controls the number of files the coalescing reader will run " + + "in each thread when it filters blocks for reading. If this value is greater than zero " + + "the files will be filtered in a multithreaded manner where each thread filters " + + "the number of files set by this config. If this is set to zero the files are " + + "filtered serially. This uses the same thread pool as the multithreaded reader, " + + s"see $MULTITHREAD_READ_NUM_THREADS. Note that filtering multithreaded " + + "is useful with Alluxio.") + .integerConf + .createWithDefault(value = 0) + + val CONCURRENT_WRITER_PARTITION_FLUSH_SIZE = + conf("spark.rapids.sql.concurrentWriterPartitionFlushSize") + .doc("The flush size of the concurrent writer cache in bytes for each partition. " + + "If specified spark.sql.maxConcurrentOutputFileWriters, use concurrent writer to " + + "write data. Concurrent writer first caches data for each partition and begins to " + + "flush the data if it finds one partition with a size that is greater than or equal " + + "to this config. The default value is 0, which will try to select a size based off " + + "of file type specific configs. E.g.: It uses `write.parquet.row-group-size-bytes` " + + "config for Parquet type and `orc.stripe.size` config for Orc type. " + + "If the value is greater than 0, will use this positive value." + + "Max value may get better performance but not always, because concurrent writer uses " + + "spillable cache and big value may cause more IO swaps.") + .bytesConf(ByteUnit.BYTE) + .createWithDefault(0L) + private def printSectionHeader(category: String): Unit = println(s"\n### $category") @@ -1660,6 +1761,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val validateExecsInGpuPlan: Seq[String] = get(TEST_VALIDATE_EXECS_ONGPU) + lazy val logQueryTransformations: Boolean = get(LOG_TRANSFORMATIONS) + lazy val rmmDebugLocation: String = get(RMM_DEBUG) lazy val gpuOomDumpDir: Option[String] = get(GPU_OOM_DUMP_DIR) @@ -1775,6 +1878,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isProjectAstEnabled: Boolean = get(ENABLE_PROJECT_AST) + lazy val isTieredProjectEnabled: Boolean = get(ENABLE_TIERED_PROJECT) + lazy val multiThreadReadNumThreads: Int = { // Use the largest value set among all the options. val deprecatedConfs = Seq( @@ -1792,6 +1897,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { values.max } + lazy val numFilesFilterParallel: Int = get(NUM_FILES_FILTER_PARALLEL) + lazy val isParquetEnabled: Boolean = get(ENABLE_PARQUET) lazy val isParquetInt96WriteEnabled: Boolean = get(ENABLE_PARQUET_INT96_WRITE) @@ -1831,6 +1938,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isOrcWriteEnabled: Boolean = get(ENABLE_ORC_WRITE) + lazy val isOrcFloatTypesToStringEnable: Boolean = get(ENABLE_ORC_FLOAT_TYPES_TO_STRING) + lazy val isOrcPerFileReadEnabled: Boolean = RapidsReaderType.withName(get(ORC_READER_TYPE)) == RapidsReaderType.PERFILE @@ -1936,8 +2045,13 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val shuffleCompressionMaxBatchMemory: Long = get(SHUFFLE_COMPRESSION_MAX_BATCH_MEMORY) + lazy val shuffleMultiThreadedMaxBytesInFlight: Long = + get(SHUFFLE_MULTITHREADED_MAX_BYTES_IN_FLIGHT) + lazy val shuffleMultiThreadedWriterThreads: Int = get(SHUFFLE_MULTITHREADED_WRITER_THREADS) + lazy val shuffleMultiThreadedReaderThreads: Int = get(SHUFFLE_MULTITHREADED_READER_THREADS) + def isUCXShuffleManagerMode: Boolean = RapidsShuffleManagerMode .withName(get(SHUFFLE_MANAGER_MODE)) == RapidsShuffleManagerMode.UCX @@ -1997,6 +2111,14 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val getAlluxioCmd: Seq[String] = get(ALLUXIO_CMD) + lazy val getAlluxioReplacementAlgo: String = get(ALLUXIO_REPLACEMENT_ALGO) + + lazy val isAlluxioReplacementAlgoSelectTime: Boolean = + get(ALLUXIO_REPLACEMENT_ALGO) == "SELECTION_TIME" + + lazy val isAlluxioReplacementAlgoConvertTime: Boolean = + get(ALLUXIO_REPLACEMENT_ALGO) == "CONVERT_TIME" + lazy val driverTimeZone: Option[String] = get(DRIVER_TIMEZONE) lazy val isRangeWindowByteEnabled: Boolean = get(ENABLE_RANGE_WINDOW_BYTES) @@ -2007,8 +2129,20 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isRangeWindowLongEnabled: Boolean = get(ENABLE_RANGE_WINDOW_LONG) + lazy val isRangeWindowDecimalEnabled: Boolean = get(ENABLE_RANGE_WINDOW_DECIMAL) + lazy val isRegExpEnabled: Boolean = get(ENABLE_REGEXP) + lazy val maxRegExpStateMemory: Long = { + val size = get(REGEXP_MAX_STATE_MEMORY_BYTES) + if (size > 3 * gpuTargetBatchSizeBytes) { + logWarning(s"${REGEXP_MAX_STATE_MEMORY_BYTES.key} is more than 3 times " + + s"${GPU_BATCH_SIZE_BYTES.key}. This may cause regular expression operations to " + + s"encounter GPU out of memory errors.") + } + size + } + lazy val getSparkGpuResourceName: String = get(SPARK_GPU_RESOURCE_NAME) lazy val isCpuBasedUDFEnabled: Boolean = get(ENABLE_CPU_BASED_UDF) @@ -2017,6 +2151,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isDetectDeltaLogQueries: Boolean = get(DETECT_DELTA_LOG_QUERIES) + lazy val concurrentWriterPartitionFlushSize:Long = get(CONCURRENT_WRITER_PARTITION_FLUSH_SIZE) + private val optimizerDefaults = Map( // this is not accurate because CPU projections do have a cost due to appending values // to each row that is produced, but this needs to be a really small number because diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index 8f4c8296fd8..7957f742575 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -168,6 +168,10 @@ abstract class RapidsMeta[INPUT <: BASE, BASE]( */ final def entirePlanWillNotWork(because: String): Unit = { cannotReplaceAnyOfPlanReasons.get.add(because) + // recursively tag the plan so that AQE does not attempt + // to run any of the child query stages on the GPU + willNotWorkOnGpu(because) + childPlans.foreach(_.entirePlanWillNotWork(because)) } final def shouldBeRemoved(because: String): Unit = @@ -202,8 +206,8 @@ abstract class RapidsMeta[INPUT <: BASE, BASE]( * set means the entire plan is ok to be replaced, do the normal checking * per exec and children. */ - final def entirePlanExcludedReasons: Seq[String] = { - cannotReplaceAnyOfPlanReasons.getOrElse(mutable.Set.empty).toSeq + final def entirePlanExcludedReasons: Set[String] = { + cannotReplaceAnyOfPlanReasons.getOrElse(mutable.Set.empty).toSet } /** @@ -530,7 +534,7 @@ abstract class SparkPlanMeta[INPUT <: SparkPlan](plan: INPUT, wrapped.withNewChildren(childPlans.map(_.convertIfNeeded())) } - def getReasonsNotToReplaceEntirePlan: Seq[String] = { + def getReasonsNotToReplaceEntirePlan: Set[String] = { val childReasons = childPlans.flatMap(_.getReasonsNotToReplaceEntirePlan) entirePlanExcludedReasons ++ childReasons } diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexComplexityEstimator.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexComplexityEstimator.scala new file mode 100644 index 00000000000..9046ab00197 --- /dev/null +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexComplexityEstimator.scala @@ -0,0 +1,71 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.nvidia.spark.rapids + +import org.apache.spark.sql.types.DataTypes + +object RegexComplexityEstimator { + private def countStates(regex: RegexAST): Int = { + regex match { + case RegexSequence(parts) => + parts.map(countStates).sum + case RegexGroup(true, term) => + 1 + countStates(term) + case RegexGroup(false, term) => + countStates(term) + case RegexCharacterClass(_, _) => + 1 + case RegexChoice(left, right) => + countStates(left) + countStates(right) + case RegexRepetition(term, QuantifierFixedLength(length)) => + length * countStates(term) + case RegexRepetition(term, SimpleQuantifier(ch)) => + ch match { + case '*' => + countStates(term) + case '+' => + 1 + countStates(term) + case '?' => + 1 + countStates(term) + } + case RegexRepetition(term, QuantifierVariableLength(minLength, maxLengthOption)) => + maxLengthOption match { + case Some(maxLength) => + maxLength * countStates(term) + case None => + minLength.max(1) * countStates(term) + } + case RegexChar(_) | RegexEscaped(_) | RegexHexDigit(_) | RegexOctalChar(_) => + 1 + case _ => + 0 + } + } + + private def estimateGpuMemory(numStates: Int, desiredBatchSizeBytes: Long): Long = { + val numRows = GpuBatchUtils.estimateRowCount( + desiredBatchSizeBytes, DataTypes.StringType.defaultSize, 1) + + // cuDF requests num_instructions * num_threads * 2 when allocating the memory on the device + // (ignoring memory alignment). We are trying to reproduce that calculation here: + numStates * numRows * 2 + } + + def isValid(conf: RapidsConf, regex: RegexAST): Boolean = { + val numStates = countStates(regex) + estimateGpuMemory(numStates, conf.gpuTargetBatchSizeBytes) <= conf.maxRegExpStateMemory + } +} diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala index 99d7cfa04e7..96884c7ae83 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala @@ -173,7 +173,7 @@ class RegexParser(pattern: String) { } private def parseCharacterClass(): RegexCharacterClass = { - val supportedMetaCharacters = "\\^-]+" + val supportedMetaCharacters = "\\^-[]+" def getEscapedComponent(): RegexCharacterClassComponent = { peek() match { @@ -207,7 +207,7 @@ class RegexParser(pattern: String) { RegexEscaped(ch) } else { throw new RegexUnsupportedException( - s"Unsupported escaped character in character class", Some(pos-1)) + s"Unsupported escaped character '$ch' in character class", Some(pos-1)) } } case None => @@ -682,6 +682,22 @@ class CudfRegexTranspiler(mode: RegexMode) { * @return Regular expression and optional replacement in cuDF format */ def transpile(pattern: String, repl: Option[String]): (String, Option[String]) = { + val (cudfRegex, replacement) = getTranspiledAST(pattern, repl) + + // write out to regex string, performing minor transformations + // such as adding additional escaping + (cudfRegex.toRegexString, replacement.map(_.toRegexString)) + } + + /** + * Parse Java regular expression and translate into cuDF regular expression in AST form. + * + * @param pattern Regular expression that is valid in Java's engine + * @param repl Optional replacement pattern + * @return Regular expression AST and optional replacement in cuDF format + */ + def getTranspiledAST( + pattern: String, repl: Option[String]): (RegexAST, Option[RegexReplacement]) = { // parse the source regular expression val regex = new RegexParser(pattern).parse() // if we have a replacement, parse the replacement string using the regex parser to account @@ -690,11 +706,10 @@ class CudfRegexTranspiler(mode: RegexMode) { // validate that the regex is supported by cuDF val cudfRegex = transpile(regex, replacement, None) - // write out to regex string, performing minor transformations - // such as adding additional escaping - (cudfRegex.toRegexString, replacement.map(_.toRegexString)) - } + (cudfRegex, replacement) + } + def transpileToSplittableString(e: RegexAST): Option[String] = { e match { case RegexEscaped(ch) if regexMetaChars.contains(ch) => Some(ch.toString) @@ -808,22 +823,20 @@ class CudfRegexTranspiler(mode: RegexMode) { private def negateCharacterClass(components: Seq[RegexCharacterClassComponent]): RegexAST = { // There are differences between cuDF and Java handling of newlines // for negative character matches. The expression `[^a]` will match - // `\r` and `\n` in Java but not in cuDF, so we replace `[^a]` with - // `(?:[\r\n]|[^a])`. We also have to take into account whether any + // `\r` in Java but not in cuDF, so we replace `[^a]` with + // `(?:[\r]|[^a])`. We also have to take into account whether any // newline characters are included in the character range. // // Examples: // - // `[^a]` => `(?:[\r\n]|[^a])` + // `[^a]` => `(?:[\r]|[^a])` // `[^a\r]` => `(?:[\n]|[^a])` // `[^a\n]` => `(?:[\r]|[^a])` // `[^a\r\n]` => `[^a]` - // `[^\r\n]` => `[^\r\n]` val distinctComponents = components.distinct val linefeedCharsInPattern = distinctComponents.flatMap { - case RegexChar(ch) if ch == '\n' || ch == '\r' => Seq(ch) - case RegexEscaped(ch) if ch == 'n' => Seq('\n') + case RegexChar(ch) if ch == '\r' => Seq(ch) case RegexEscaped(ch) if ch == 'r' => Seq('\r') case RegexCharacterRange(startRegex, RegexChar(end)) => val start = startRegex match { @@ -834,11 +847,7 @@ class CudfRegexTranspiler(mode: RegexMode) { s"Unexpected expression at start of character range: ${other.toRegexString}", other.position) } - if (start <= '\n' && end >= '\r') { - Seq('\n', '\r') - } else if (start <= '\n' && end >= '\n') { - Seq('\n') - } else if (start <= '\r' && end >= '\r') { + if (start <= '\r' && end >= '\r') { Seq('\r') } else { Seq.empty @@ -847,12 +856,9 @@ class CudfRegexTranspiler(mode: RegexMode) { } val onlyLinefeedChars = distinctComponents.length == linefeedCharsInPattern.length - val negatedNewlines = Seq('\r', '\n').diff(linefeedCharsInPattern.distinct) + val negatedNewlines = Seq('\r').diff(linefeedCharsInPattern.distinct) - if (onlyLinefeedChars && linefeedCharsInPattern.length == 2) { - // special case for `[^\r\n]` and `[^\\r\\n]` - RegexCharacterClass(negated = true, ListBuffer(distinctComponents: _*)) - } else if (negatedNewlines.isEmpty) { + if (negatedNewlines.isEmpty) { RegexCharacterClass(negated = true, ListBuffer(distinctComponents: _*)) } else { RegexGroup(capture = false, @@ -1377,7 +1383,11 @@ class CudfRegexTranspiler(mode: RegexMode) { s"cuDF does not support repetition of group containing: " + s"${unsupportedTerm.toRegexString}", term.position) } - case (RegexGroup(_, _), SimpleQuantifier(ch)) if ch == '?' => + case (RegexGroup(_, term), SimpleQuantifier(ch)) if ch == '?' => + if (isEntirelyWordBoundary(term) || isEntirelyLineAnchor(term)) { + throw new RegexUnsupportedException( + s"cuDF does not support repetition of: ${term.toRegexString}", term.position) + } RegexRepetition(rewrite(base, replacement, None), quantifier) case (RegexEscaped(ch), SimpleQuantifier('+')) if "AZ".contains(ch) => // \A+ can be transpiled to \A (dropping the repetition) @@ -1498,6 +1508,30 @@ class CudfRegexTranspiler(mode: RegexMode) { } } + private def isEntirely(regex: RegexAST, f: RegexAST => Boolean): Boolean = { + regex match { + case RegexSequence(parts) if parts.nonEmpty => + parts.forall(f) + case RegexGroup(_, term) => + isEntirely(term, f) + case _ => f(regex) + } + } + + private def isEntirelyWordBoundary(regex: RegexAST): Boolean = { + isEntirely(regex, { + case RegexEscaped(ch) if "bB".contains(ch) => true + case _ => false + }) + } + + private def isEntirelyLineAnchor(regex: RegexAST): Boolean = { + isEntirely(regex, { + case RegexEscaped('A') => true + case other => isBeginOrEndLineAnchor(other) + }) + } + private def endsWith(regex: RegexAST, f: RegexAST => Boolean): Boolean = { regex match { case RegexSequence(parts) if parts.nonEmpty => @@ -1506,6 +1540,8 @@ class CudfRegexTranspiler(mode: RegexMode) { case _ => true } endsWith(parts(j), f) + case RegexGroup(_, term) => + endsWith(term, f) case _ => f(regex) } } diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimGpuOverrides.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimGpuOverrides.scala index dcaee65cca7..853fadef553 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimGpuOverrides.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimGpuOverrides.scala @@ -54,26 +54,7 @@ object ShimGpuOverrides extends Logging { TypeSig.cpuNumeric))), (a, conf, p, r) => new AggExprMeta[Average](a, conf, p, r) { override def tagAggForGpu(): Unit = { - // For Decimal Average the SUM adds a precision of 10 to avoid overflowing - // then it divides by the count with an output scale that is 4 more than the input - // scale. With how our divide works to match Spark, this means that we will need a - // precision of 5 more. So 38 - 10 - 5 = 23 - val dataType = a.child.dataType - dataType match { - case dt: DecimalType => - if (dt.precision > 23) { - if (conf.needDecimalGuarantees) { - willNotWorkOnGpu("GpuAverage cannot guarantee proper overflow checks for " + - s"a precision large than 23. The current precision is ${dt.precision}") - } else { - logWarning("Decimal overflow guarantees disabled for " + - s"Average(${a.child.dataType}) produces ${dt} with an " + - s"intermediate precision of ${dt.precision + 15}") - } - } - case _ => // NOOP - } - GpuOverrides.checkAndTagFloatAgg(dataType, conf, this) + GpuOverrides.checkAndTagFloatAgg(a.child.dataType, conf, this) } }), @@ -87,10 +68,10 @@ object ShimGpuOverrides extends Logging { ).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap val shimExecs: Map[Class[_ <: SparkPlan], ExecRule[_ <: SparkPlan]] = Seq( - GpuOverrides.exec[FileSourceScanExec]( - "Reading data from files, often from Hive tables", - ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.STRUCT + TypeSig.MAP + - TypeSig.ARRAY + TypeSig.DECIMAL_128).nested(), TypeSig.all), + GpuOverrides.exec[FileSourceScanExec]( + "Reading data from files, often from Hive tables", + ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.STRUCT + TypeSig.MAP + + TypeSig.ARRAY + TypeSig.BINARY + TypeSig.DECIMAL_128).nested(), TypeSig.all), (fsse, conf, p, r) => new SparkPlanMeta[FileSourceScanExec](fsse, conf, p, r) { // partition filters and data filters are not run on the GPU diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index f2943ecfc39..1f63a3e4a44 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -360,8 +360,7 @@ final class TypeSig private( case DoubleType => check.contains(TypeEnum.DOUBLE) case DateType => check.contains(TypeEnum.DATE) case TimestampType if check.contains(TypeEnum.TIMESTAMP) => - TypeChecks.areTimestampsSupported(ZoneId.systemDefault()) && - TypeChecks.areTimestampsSupported(SQLConf.get.sessionLocalTimeZone) + TypeChecks.areTimestampsSupported() case StringType => check.contains(TypeEnum.STRING) case dt: DecimalType => check.contains(TypeEnum.DECIMAL) && @@ -422,8 +421,7 @@ final class TypeSig private( basicNotSupportedMessage(dataType, TypeEnum.DATE, check, isChild) case TimestampType => if (check.contains(TypeEnum.TIMESTAMP) && - (!TypeChecks.areTimestampsSupported(ZoneId.systemDefault()) || - !TypeChecks.areTimestampsSupported(SQLConf.get.sessionLocalTimeZone))) { + !TypeChecks.areTimestampsSupported()) { Seq(withChild(isChild, s"$dataType is not supported with timezone settings: (JVM:" + s" ${ZoneId.systemDefault()}, session: ${SQLConf.get.sessionLocalTimeZone})." + s" Set both of the timezones to UTC to enable $dataType support")) @@ -776,6 +774,36 @@ abstract class TypeChecks[RET] { }.mkString(", ") } + /** + * Original log does not print enough info when timezone is not UTC, + * here check again to add UTC info. + */ + private def tagTimezoneInfoIfHasTimestampType( + unsupportedTypes: Map[DataType, Set[String]], + meta: RapidsMeta[_, _] + ): Unit = { + def checkTimestampType(dataType: DataType): Unit = dataType match { + case TimestampType if !TypeChecks.areTimestampsSupported() => { + meta.willNotWorkOnGpu(s"your timezone isn't in UTC (JVM:" + + s" ${ZoneId.systemDefault()}, session: ${SQLConf.get.sessionLocalTimeZone})." + + s" Set both of the timezones to UTC to enable TimestampType support") + return + } + case ArrayType(elementType, _) => + checkTimestampType(elementType) + case MapType(keyType, valueType, _) => + checkTimestampType(keyType) + checkTimestampType(valueType) + case StructType(fields) => + fields.foreach(field => checkTimestampType(field.dataType)) + case _ => + // do nothing + } + unsupportedTypes.foreach { case (dataType, nameSet) => + checkTimestampType(dataType) + } + } + protected def tagUnsupportedTypes( meta: RapidsMeta[_, _], sig: TypeSig, @@ -787,6 +815,8 @@ abstract class TypeChecks[RET] { .groupBy(_.dataType) .mapValues(_.map(_.name).toSet) + tagTimezoneInfoIfHasTimestampType(unsupportedTypes, meta) + if (unsupportedTypes.nonEmpty) { meta.willNotWorkOnGpu(msgFormat.format(stringifyTypeAttributeMap(unsupportedTypes))) } @@ -817,6 +847,11 @@ object TypeChecks { val zoneId = getZoneId(zoneIdString) areTimestampsSupported(zoneId) } + + def areTimestampsSupported(): Boolean = { + areTimestampsSupported(ZoneId.systemDefault()) && + areTimestampsSupported(SQLConf.get.sessionLocalTimeZone) + } } /** @@ -1132,7 +1167,7 @@ case class ExprChecksImpl(contexts: Map[ExpressionContext, ContextChecks]) */ object CaseWhenCheck extends ExprChecks { val check: TypeSig = (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP).nested() + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.BINARY).nested() val sparkSig: TypeSig = TypeSig.all @@ -1263,7 +1298,7 @@ object CreateNamedStructCheck extends ExprChecks { val nameSig: TypeSig = TypeSig.lit(TypeEnum.STRING) val sparkNameSig: TypeSig = TypeSig.lit(TypeEnum.STRING) val valueSig: TypeSig = (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT).nested() + TypeSig.ARRAY + TypeSig.MAP + TypeSig.STRUCT + TypeSig.BINARY).nested() val sparkValueSig: TypeSig = TypeSig.all val resultSig: TypeSig = TypeSig.STRUCT.nested(valueSig) val sparkResultSig: TypeSig = TypeSig.STRUCT.nested(sparkValueSig) @@ -1344,7 +1379,7 @@ class CastChecks extends ExprChecks { val sparkStringSig: TypeSig = cpuNumeric + BOOLEAN + TIMESTAMP + DATE + CALENDAR + STRING + BINARY + GpuTypeShims.additionalTypesStringCanCastTo - val binaryChecks: TypeSig = none + val binaryChecks: TypeSig = STRING + BINARY val sparkBinarySig: TypeSig = STRING + BINARY val decimalChecks: TypeSig = gpuNumeric + STRING diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala index 1ca155bf0f2..c0bd09ea736 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala @@ -98,7 +98,7 @@ abstract class GpuWindowExpressionMetaBase( val orderByTypeSupported = orderSpec.forall { so => so.dataType match { case ByteType | ShortType | IntegerType | LongType | - DateType | TimestampType => true + DateType | TimestampType | DecimalType() => true case _ => false } } @@ -125,13 +125,17 @@ abstract class GpuWindowExpressionMetaBase( s"Range window frame is not 100% compatible when the order by type is " + s"long and the range value calculated has overflow. " + s"To enable it please set ${RapidsConf.ENABLE_RANGE_WINDOW_LONG} to true.") + case DecimalType() => if (!conf.isRangeWindowDecimalEnabled) willNotWorkOnGpu( + s"To enable DECIMAL order by columns with Range window frames, " + + s"please set ${RapidsConf.ENABLE_RANGE_WINDOW_DECIMAL} to true.") case _ => // never reach here } } // check whether the boundaries are supported or not. Seq(spec.lower, spec.upper).foreach { - case l @ Literal(_, ByteType | ShortType | IntegerType | LongType) => + case l @ Literal(_, ByteType | ShortType | IntegerType | + LongType | DecimalType()) => checkRangeBoundaryConfig(l.dataType) case Literal(ci: CalendarInterval, CalendarIntervalType) => // interval is only working for TimeStampType @@ -190,6 +194,7 @@ abstract class GpuSpecifiedWindowFrameMetaBase( case Literal(value, ShortType) => value.asInstanceOf[Short].toLong case Literal(value, IntegerType) => value.asInstanceOf[Int].toLong case Literal(value, LongType) => value.asInstanceOf[Long] + case Literal(value: Decimal, DecimalType()) => value.toLong case Literal(ci: CalendarInterval, CalendarIntervalType) => if (ci.months != 0) { willNotWorkOnGpu("interval months isn't supported") @@ -307,7 +312,7 @@ object GpuWindowUtil { } } -case class ParsedBoundary(isUnbounded: Boolean, valueAsLong: Long) +case class ParsedBoundary(isUnbounded: Boolean, value: Either[BigInt, Long]) class GpuWindowSpecDefinitionMeta( windowSpec: WindowSpecDefinition, diff --git a/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveOverrides.scala b/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveOverrides.scala index 416d10ab73e..cec276ca975 100644 --- a/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveOverrides.scala +++ b/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveOverrides.scala @@ -16,15 +16,13 @@ package org.apache.spark.sql.hive.rapids -import com.nvidia.spark.RapidsUDF -import com.nvidia.spark.rapids.{ExprChecks, ExprMeta, ExprRule, GpuOverrides, RapidsConf, RepeatingParamCheck, TypeSig} -import com.nvidia.spark.rapids.GpuUserDefinedFunction.udfTypeSig +import com.nvidia.spark.rapids.{DataWritingCommandRule, ExprRule, HiveProvider} import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.hive.{HiveGenericUDF, HiveSimpleUDF} +import org.apache.spark.sql.execution.command.DataWritingCommand object GpuHiveOverrides { - def isSparkHiveAvailable: Boolean = { + val isSparkHiveAvailable: Boolean = { try { getClass().getClassLoader.loadClass("org.apache.spark.sql.hive.HiveSessionStateBuilder") getClass().getClassLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf") @@ -34,16 +32,29 @@ object GpuHiveOverrides { } } - /** - * Builds the rules that are specific to spark-hive Catalyst nodes. This will return an empty - * mapping if spark-hive is unavailable. - */ - def exprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = { + private lazy val hiveProvider: HiveProvider = { if (isSparkHiveAvailable) { - // don't use the ShimLoader for Spark 2.x - new HiveProviderImpl().getExprs + new HiveProviderImpl() } else { - Map.empty + new HiveProvider() { + override def getDataWriteCmds: Map[Class[_ <: DataWritingCommand], + DataWritingCommandRule[_ <: DataWritingCommand]] = Map.empty + override def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Map.empty + } } } + + + /** + * Builds the data writing command rules that are specific to spark-hive Catalyst nodes. + * This will return an empty mapping if spark-hive is unavailable + */ + def dataWriteCmds: Map[Class[_ <: DataWritingCommand], + DataWritingCommandRule[_ <: DataWritingCommand]] = hiveProvider.getDataWriteCmds + + /** + * Builds the rules that are specific to spark-hive Catalyst nodes. This will return an empty + * mapping if spark-hive is unavailable. + */ + def exprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = hiveProvider.getExprs } diff --git a/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/HiveProviderImpl.scala b/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/HiveProviderImpl.scala index c2dcb5895c1..eb1f26e396b 100644 --- a/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/HiveProviderImpl.scala +++ b/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/HiveProviderImpl.scala @@ -17,14 +17,26 @@ package org.apache.spark.sql.hive.rapids import com.nvidia.spark.RapidsUDF -import com.nvidia.spark.rapids.{ExprChecks, ExprMeta, ExprRule, GpuOverrides, HiveProvider, RapidsConf, RepeatingParamCheck, TypeSig} +import com.nvidia.spark.rapids.{DataWritingCommandRule, ExprChecks, ExprMeta, ExprRule, GpuOverrides, HiveProvider, OptimizedCreateHiveTableAsSelectCommandMeta, RapidsConf, RepeatingParamCheck, TypeSig} import com.nvidia.spark.rapids.GpuUserDefinedFunction.udfTypeSig import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.execution.command.DataWritingCommand import org.apache.spark.sql.hive.{HiveGenericUDF, HiveSimpleUDF} +import org.apache.spark.sql.hive.execution.OptimizedCreateHiveTableAsSelectCommand class HiveProviderImpl extends HiveProvider { + /** + * Builds the data writing command rules that are specific to spark-hive Catalyst nodes. + */ + override def getDataWriteCmds: Map[Class[_ <: DataWritingCommand], + DataWritingCommandRule[_ <: DataWritingCommand]] = Seq ( + GpuOverrides.dataWriteCmd[OptimizedCreateHiveTableAsSelectCommand]( + "Create a Hive table from a query result using Spark data source APIs", + (a, conf, p, r) => new OptimizedCreateHiveTableAsSelectCommandMeta(a, conf, p, r)) + ).map(r => (r.getClassFor.asSubclass(classOf[DataWritingCommand]), r)).toMap + /** * Builds the rules that are specific to spark-hive Catalyst nodes. This will return an empty * mapping if spark-hive is unavailable. diff --git a/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala b/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala index 023386ba5b4..79c37edffe3 100644 --- a/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala +++ b/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala @@ -122,6 +122,36 @@ object GpuDecimalMultiply { } object GpuDecimalDivide { + // This comes from DecimalType.MINIMUM_ADJUSTED_SCALE, but for some reason it is gone + // in databricks so we have it here. + private val MINIMUM_ADJUSTED_SCALE = 6 + + def calcOrigSparkOutputType(lhs: DecimalType, rhs: DecimalType): DecimalType = { + // This comes almost directly from Spark unchanged + val allowPrecisionLoss = SQLConf.get.decimalOperationsAllowPrecisionLoss + val p1 = lhs.precision + val s1 = lhs.scale + val p2 = rhs.precision + val s2 = rhs.scale + if (allowPrecisionLoss) { + // Precision: p1 - s1 + s2 + max(6, s1 + p2 + 1) + // Scale: max(6, s1 + p2 + 1) + val intDig = p1 - s1 + s2 + val scale = math.max(MINIMUM_ADJUSTED_SCALE, s1 + p2 + 1) + val prec = intDig + scale + DecimalType.adjustPrecisionScale(prec, scale) + } else { + var intDig = math.min(DecimalType.MAX_SCALE, p1 - s1 + s2) + var decDig = math.min(DecimalType.MAX_SCALE, math.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) + } + } + // For Spark the final desired output is // new_scale = max(6, lhs.scale + rhs.precision + 1) // new_precision = lhs.precision - lhs.scale + rhs.scale + new_scale diff --git a/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressionsMeta.scala b/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressionsMeta.scala index 52c0d849d50..94f483a20fa 100644 --- a/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressionsMeta.scala +++ b/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressionsMeta.scala @@ -16,11 +16,13 @@ package org.apache.spark.sql.rapids +import java.time.ZoneId + import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.DateUtils.TimestampFormatConversionException import com.nvidia.spark.rapids.GpuOverrides.extractStringLit -import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, TimeZoneAwareExpression} +import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, FromUTCTimestamp, TimeZoneAwareExpression} import org.apache.spark.sql.types._ case class ParseFormatMeta(separator: Option[Char], isTimestamp: Boolean, validRegex: String) @@ -105,3 +107,30 @@ abstract class UnixTimeExprMeta[A <: BinaryExpression with TimeZoneAwareExpressi } } } + +// sealed trait TimeParserPolicy for rundiffspark2.sh +class FromUTCTimestampExprMeta( + expr: FromUTCTimestamp, + override val conf: RapidsConf, + override val parent: Option[RapidsMeta[_, _]], + rule: DataFromReplacementRule) + extends BinaryExprMeta[FromUTCTimestamp](expr, conf, parent, rule) { + + override def tagExprForGpu(): Unit = { + extractStringLit(expr.right) match { + case None => + willNotWorkOnGpu("timezone input must be a literal string") + case Some(timezoneShortID) => + if (timezoneShortID != null) { + val utc = ZoneId.of("UTC").normalized + // This is copied from Spark, to convert `(+|-)h:mm` into `(+|-)0h:mm`. + val timezone = ZoneId.of(timezoneShortID.replaceFirst("(\\+|\\-)(\\d):", "$10$2:"), + ZoneId.SHORT_IDS).normalized + + if (timezone != utc) { + willNotWorkOnGpu("only timezones equivalent to UTC are supported") + } + } + } + } +} diff --git a/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala b/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala index 283e1e7b108..c0c8fe4238e 100644 --- a/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala +++ b/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala @@ -64,7 +64,8 @@ object JoinTypeChecks { private[this] val sparkSupportedJoinKeyTypes = TypeSig.all - TypeSig.MAP.nested() private[this] val joinRideAlongTypes = - (cudfSupportedKeyTypes + TypeSig.DECIMAL_128 + TypeSig.ARRAY + TypeSig.MAP).nested() + (cudfSupportedKeyTypes + TypeSig.DECIMAL_128 + TypeSig.BINARY + + TypeSig.ARRAY + TypeSig.MAP).nested() val equiJoinExecChecks: ExecChecks = ExecChecks( joinRideAlongTypes, diff --git a/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringMeta.scala b/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringMeta.scala index f5ee029e57a..2a85ab5ec95 100644 --- a/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringMeta.scala +++ b/spark2-sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringMeta.scala @@ -38,7 +38,10 @@ class GpuRLikeMeta( case Literal(str: UTF8String, DataTypes.StringType) if str != null => try { // verify that we support this regex and can transpile it to cuDF format - pattern = Some(new CudfRegexTranspiler(RegexFindMode).transpile(str.toString, None)._1) + val (transpiledAST, _) = + new CudfRegexTranspiler(RegexFindMode).getTranspiledAST(str.toString, None) + GpuRegExpUtils.validateRegExpComplexity(this, transpiledAST) + pattern = Some(transpiledAST.toRegexString) } catch { case e: RegexUnsupportedException => willNotWorkOnGpu(e.getMessage) @@ -76,8 +79,10 @@ class GpuRegExpExtractMeta( try { val javaRegexpPattern = str.toString // verify that we support this regex and can transpile it to cuDF format - pattern = Some(new CudfRegexTranspiler(RegexFindMode) - .transpile(javaRegexpPattern, None)._1) + val (transpiledAST, _) = + new CudfRegexTranspiler(RegexFindMode).getTranspiledAST(javaRegexpPattern, None) + GpuRegExpUtils.validateRegExpComplexity(this, transpiledAST) + pattern = Some(transpiledAST.toRegexString) numGroups = GpuRegExpUtils.countGroups(javaRegexpPattern) } catch { case e: RegexUnsupportedException => @@ -259,7 +264,9 @@ abstract class StringSplitRegExpMeta[INPUT <: TernaryExpression](expr: INPUT, pattern = simplified case None => try { - pattern = transpiler.transpile(utf8Str.toString, None)._1 + val (transpiledAST, _) = transpiler.getTranspiledAST(utf8Str.toString, None) + GpuRegExpUtils.validateRegExpComplexity(this, transpiledAST) + pattern = transpiledAST.toRegexString isRegExp = true } catch { case e: RegexUnsupportedException => From 36f9387b064d2e74b41f22dc8d13256a96838385 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Mon, 26 Sep 2022 15:57:25 -0500 Subject: [PATCH 143/190] Enable tests that were missed when binary support was extended (#6607) Signed-off-by: Robert (Bobby) Evans Signed-off-by: Robert (Bobby) Evans --- .../scala/com/nvidia/spark/rapids/CastOpSuite.scala | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala index bb612aed41f..8ec81a4152b 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala @@ -481,8 +481,7 @@ class CastOpSuite extends GpuExpressionTestSuite { col("longs").cast(StringType), col("more_longs").cast(BooleanType), col("more_longs").cast(ByteType), - // Test requires ProjectExec support BinaryType, tested within md5 hash functionality instead - // col("longs").cast(BinaryType), + col("longs").cast(BinaryType), col("longs").cast(ShortType), col("longs").cast(FloatType), col("longs").cast(DoubleType), @@ -643,11 +642,10 @@ class CastOpSuite extends GpuExpressionTestSuite { col("c0").cast(FloatType)) } - // Test requires ProjectExec support BinaryType, tested within md5 hash functionality instead - // testSparkResultsAreEqual("Test cast from strings to binary", floatsAsStrings) { - // frame => frame.select( - // col("c0").cast(BinaryType)) - // } + testSparkResultsAreEqual("Test cast from strings to binary", floatsAsStrings) { + frame => frame.select( + col("c0").cast(BinaryType)) + } test("cast short to decimal") { List(-4, -2, 0, 1, 5, 15).foreach { scale => From 0ef9202422d3e179971e533adad9deefef2d9e65 Mon Sep 17 00:00:00 2001 From: Peixin Date: Tue, 27 Sep 2022 08:45:55 +0800 Subject: [PATCH 144/190] Disable 22.10 snapshot builds (#6610) * Disable 22.10 snapshot builds Signed-off-by: Peixin Li * use ENV to avoid transitive params passing Signed-off-by: Peixin Li --- dist/pom.xml | 11 ++++++++--- jenkins/spark-premerge-build.sh | 4 ++-- jenkins/version-def.sh | 19 ++++++------------- 3 files changed, 16 insertions(+), 18 deletions(-) diff --git a/dist/pom.xml b/dist/pom.xml index a0f81c40d79..055dd4235e4 100644 --- a/dist/pom.xml +++ b/dist/pom.xml @@ -49,11 +49,16 @@ 321cdh, 322, 330, - 330cdh + 330cdh, + 331 + - 314, - 331 + 314 312db, diff --git a/jenkins/spark-premerge-build.sh b/jenkins/spark-premerge-build.sh index 8d34db75e42..ef80f15a219 100755 --- a/jenkins/spark-premerge-build.sh +++ b/jenkins/spark-premerge-build.sh @@ -165,11 +165,11 @@ BUILD_MAINTENANCE_VERSION_SNAPSHOTS="false" # controls whether we build snapshots for the next Spark major or feature version like 3.4.0 or 4.0.0 BUILD_FEATURE_VERSION_SNAPSHOTS="false" PREMERGE_PROFILES="-PnoSnapshots,pre-merge" -if [[ ${PROJECT_VER} =~ ^22\.10\. ]]; then # enable snapshot builds for active development branch only +if [[ ${PROJECT_VER} =~ ^22\.12\. ]]; then # enable snapshot builds for active development branch only BUILD_MAINTENANCE_VERSION_SNAPSHOTS="true" BUILD_FEATURE_VERSION_SNAPSHOTS="false" PREMERGE_PROFILES="-Psnapshots,pre-merge" -elif [[ ${PROJECT_VER} =~ ^22\.12\. ]]; then +elif [[ ${PROJECT_VER} =~ ^23\.02\. ]]; then BUILD_MAINTENANCE_VERSION_SNAPSHOTS="true" BUILD_FEATURE_VERSION_SNAPSHOTS="true" PREMERGE_PROFILES="-Psnapshots,pre-merge" diff --git a/jenkins/version-def.sh b/jenkins/version-def.sh index 7f0c1767ca7..fb6f8fc162c 100755 --- a/jenkins/version-def.sh +++ b/jenkins/version-def.sh @@ -17,18 +17,6 @@ set -e -# PHASE_TYPE: CICD phase at which the script is called, to specify Spark shim versions. -# regular: noSnapshots + snapshots -# pre-release: noSnapshots only -PHASE_TYPE=regular - -if [[ $# -eq 1 ]]; then - PHASE_TYPE=$1 -elif [[ $# -gt 1 ]]; then - >&2 echo "ERROR: too many parameters are provided" - exit 1 -fi - # Split abc=123 from $OVERWRITE_PARAMS # $OVERWRITE_PARAMS patten 'abc=123;def=456;' PRE_IFS=$IFS @@ -74,8 +62,13 @@ SPARK_SHIM_VERSIONS_SNAPSHOTS=("${SPARK_SHIM_VERSIONS_ARR[@]}") # PnoSnapshots: noSnapshots only set_env_var_SPARK_SHIM_VERSIONS_ARR -PnoSnapshots SPARK_SHIM_VERSIONS_NOSNAPSHOTS=("${SPARK_SHIM_VERSIONS_ARR[@]}") -# Spark shim versions list based on given profile option (snapshots or noSnapshots) + +# PHASE_TYPE: CICD phase at which the script is called, to specify Spark shim versions. +# regular: noSnapshots + snapshots +# pre-release: noSnapshots only +PHASE_TYPE=${PHASE_TYPE:-"pre-release"} # TODO: update it to regular in branch-22.12 when CI is available case $PHASE_TYPE in + # SPARK_SHIM_VERSIONS will be used for nightly artifact build pre-release) SPARK_SHIM_VERSIONS=("${SPARK_SHIM_VERSIONS_NOSNAPSHOTS[@]}") ;; From c6ad4e93ecbfdb22d683c1a6db9433350c7d20ec Mon Sep 17 00:00:00 2001 From: zhanga5 Date: Tue, 27 Sep 2022 09:12:27 +0800 Subject: [PATCH 145/190] Tolerate event log folder existence when to create it to avoid raising 'FileExistsError' (#6612) It's happened when tests are executed in parallel, e.g. on CDP Signed-off-by: Alex Zhang Signed-off-by: Alex Zhang --- integration_tests/src/main/python/spark_init_internal.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/integration_tests/src/main/python/spark_init_internal.py b/integration_tests/src/main/python/spark_init_internal.py index a97d31db1a4..f5916fdccd3 100644 --- a/integration_tests/src/main/python/spark_init_internal.py +++ b/integration_tests/src/main/python/spark_init_internal.py @@ -156,7 +156,9 @@ def _handle_event_log_dir(sb, wid): return d = "./eventlog_{}".format(wid) if not os.path.exists(d): - os.makedirs(d) + # Set 'exist_ok' as True to avoid raising 'FileExistsError' as the folder might be created + # by other tests when they are executed in parallel + os.makedirs(d, exist_ok=True) logging.info('Spark event logs will appear under {}. Set the environmnet variable ' 'SPARK_EVENTLOG_ENABLED=false if you want to disable it'.format(d)) From c4292a405fe683d9d4f9841a33b57587b0d67cd9 Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Tue, 27 Sep 2022 09:39:44 +0800 Subject: [PATCH 146/190] Fix a delta deleting issue (#6617) Signed-off-by: Firestarman --- .../org/apache/spark/sql/rapids/shims/GpuFileScanRDD.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/sql-plugin/src/main/311+-db/scala/org/apache/spark/sql/rapids/shims/GpuFileScanRDD.scala b/sql-plugin/src/main/311+-db/scala/org/apache/spark/sql/rapids/shims/GpuFileScanRDD.scala index bcaf8efee25..26250760003 100644 --- a/sql-plugin/src/main/311+-db/scala/org/apache/spark/sql/rapids/shims/GpuFileScanRDD.scala +++ b/sql-plugin/src/main/311+-db/scala/org/apache/spark/sql/rapids/shims/GpuFileScanRDD.scala @@ -72,6 +72,7 @@ class GpuFileScanRDD( context.killTaskIfInterrupted() (currentIterator != null && currentIterator.hasNext) || nextIterator() } + def next(): Object = { val nextElement = currentIterator.next() // TODO: we should have a better separation of row based and batch based scan, so that we @@ -169,7 +170,11 @@ class GpuFileScanRDD( } } else { currentFile = null - InputFileBlockHolder.unset() + // Removed "InputFileBlockHolder.unset()", because for some cases on DB, + // cleaning the input file holder here is too early. Some operators are still + // leveraging this holder to get the current file path even after hasNext + // returns false. + // See https://github.com/NVIDIA/spark-rapids/issues/6592 false } } From db4e67411422cd13b023c5a9eee65bcb3b840944 Mon Sep 17 00:00:00 2001 From: Peixin Date: Tue, 27 Sep 2022 13:12:08 +0800 Subject: [PATCH 147/190] Move spark331 back to list of snapshot shims (#6627) Signed-off-by: Peixin Li Signed-off-by: Peixin Li --- dist/pom.xml | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/dist/pom.xml b/dist/pom.xml index 055dd4235e4..a0f81c40d79 100644 --- a/dist/pom.xml +++ b/dist/pom.xml @@ -49,16 +49,11 @@ 321cdh, 322, 330, - 330cdh, - 331 + 330cdh - - 314 + 314, + 331 312db, From fac9db0160ab3627607467b4f9acf5b437dcb251 Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Tue, 27 Sep 2022 12:18:20 -0500 Subject: [PATCH 148/190] Allow AutoTuner to accept remore path for WorkerInfo (#6630) * Allow AutoTuner to accept remote path for WorkerInfo Signed-off-by: Ahmed Hussein (amahussein) --- .../rapids/tool/profiling/AutoTuner.scala | 109 ++++++++++-------- 1 file changed, 64 insertions(+), 45 deletions(-) diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala index 46641ac0804..dabd10019ca 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala @@ -18,9 +18,12 @@ package com.nvidia.spark.rapids.tool.profiling import java.io.FileNotFoundException -import org.yaml.snakeyaml.Yaml import scala.collection.JavaConverters._ +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, FSDataInputStream, Path} +import org.yaml.snakeyaml.Yaml + import org.apache.spark.internal.Logging /** @@ -479,55 +482,71 @@ object AutoTuner extends Logging { val DEFAULT_WORKER_INFO: String = "." val SUPPORTED_SIZE_UNITS: Seq[String] = Seq("b", "k", "m", "g", "t", "p") + def loadSystemProperties(fileInput: String): Option[java.util.Map[String, Any]] = { + val filePath = new Path(fileInput) + val fs = FileSystem.get(filePath.toUri, new Configuration()) + val yaml = new Yaml() + var fsIs: FSDataInputStream = null + try { + fsIs = fs.open(filePath) + Some(yaml.load(fsIs).asInstanceOf[java.util.Map[String, Any]]) + } finally { + if (fsIs != null) { + fsIs.close() + } + } + } + /** * Parses the yaml file and returns system and gpu properties. * See [[SystemProps]] and [[GpuProps]]. */ def parseSystemInfo(yamlFile: String): (SystemProps, Option[String]) = { - try { - val yaml = new Yaml() - val file = scala.io.Source.fromFile(yamlFile) - val text = file.mkString - val rawProps = yaml.load(text).asInstanceOf[java.util.Map[String, Any]] - .asScala.toMap.filter { case (_, v) => v != null } - val rawSystemProps = rawProps("system").asInstanceOf[java.util.Map[String, Any]] - .asScala.toMap.filter { case (_, v) => v != null } - - if (rawSystemProps.nonEmpty) { - val rawGpuProps = rawProps("gpu").asInstanceOf[java.util.Map[String, Any]] - .asScala.toMap.filter { case (_, v) => v != null } - - val gpuProps = if (rawGpuProps.nonEmpty) { - GpuProps( - rawGpuProps("count").toString.toInt, - rawGpuProps("memory").toString, - rawGpuProps("name").toString) - } else { - null - } - - (SystemProps( - rawSystemProps.getOrElse("num_cores", 1).toString.toInt, - rawSystemProps.getOrElse("cpu_arch", "").toString, - rawSystemProps.getOrElse("memory", "0b").toString, - rawSystemProps.getOrElse("free_disk_space", "0b").toString, - rawSystemProps.getOrElse("time_zone", "").toString, - rawSystemProps.get("num_workers").map(_.toString.toInt), - gpuProps), None) - } else { - (null, Some("System properties was empty")) - } - } catch { - case e: FileNotFoundException => - logError("Exception: " + e.getStackTrace.mkString("Array(", ", ", ")")) - (null, Some("System properties file was not found")) - case e: NullPointerException => - logError("Exception: " + e.getStackTrace.mkString("Array(", ", ", ")")) - (null, Some("System properties file was not formatted correctly.")) - case e: Exception => - logError("Exception: " + e.getStackTrace.mkString("Array(", ", ", ")")) - (null, Some(e.toString)) - } + try { + val loadedProps = loadSystemProperties(yamlFile) + if (loadedProps.isDefined) { + val rawProps = loadedProps.get.asScala.toMap.filter { case (_, v) => v != null } + val rawSystemProps = rawProps("system").asInstanceOf[java.util.Map[String, Any]] + .asScala.toMap.filter { case (_, v) => v != null } + if (rawSystemProps.nonEmpty) { + val rawGpuProps = rawProps("gpu").asInstanceOf[java.util.Map[String, Any]] + .asScala.toMap.filter { case (_, v) => v != null } + + val gpuProps = if (rawGpuProps.nonEmpty) { + GpuProps( + rawGpuProps("count").toString.toInt, + rawGpuProps("memory").toString, + rawGpuProps("name").toString) + } else { + null + } + + (SystemProps( + rawSystemProps.getOrElse("num_cores", 1).toString.toInt, + rawSystemProps.getOrElse("cpu_arch", "").toString, + rawSystemProps.getOrElse("memory", "0b").toString, + rawSystemProps.getOrElse("free_disk_space", "0b").toString, + rawSystemProps.getOrElse("time_zone", "").toString, + rawSystemProps.get("num_workers").map(_.toString.toInt), + gpuProps), None) + } else { + (null, Some("System properties was empty")) + } + } + else { + (null, Some("System properties was empty")) + } + } catch { + case e: FileNotFoundException => + logError("Exception: " + e.getStackTrace.mkString("Array(", ", ", ")")) + (null, Some("System properties file was not found")) + case e: NullPointerException => + logError("Exception: " + e.getStackTrace.mkString("Array(", ", ", ")")) + (null, Some("System properties file was not formatted correctly.")) + case e: Exception => + logError("Exception: " + e.getStackTrace.mkString("Array(", ", ", ")")) + (null, Some(e.toString)) + } } /** From 4a7a7e1fe46920e34b0c1be91df6ffdf95aeb297 Mon Sep 17 00:00:00 2001 From: Jim Brennan Date: Tue, 27 Sep 2022 13:05:51 -0500 Subject: [PATCH 149/190] Print nvidia-smi output when an executor fails due to a cuda fatal exception. (#6606) Signed-off-by: Jim Brennan --- .../com/nvidia/spark/rapids/Plugin.scala | 49 +++++++++++++++++-- 1 file changed, 45 insertions(+), 4 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala index e9d85408745..1a8c9c87c24 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} import scala.collection.JavaConverters._ import scala.collection.mutable.{Map => MutableMap} +import scala.sys.process._ import scala.util.Try import scala.util.matching.Regex @@ -269,10 +270,13 @@ class RapidsExecutorPlugin extends ExecutorPlugin with Logging { logInfo(s"The number of concurrent GPU tasks allowed is $concurrentGpuTasks") GpuSemaphore.initialize(concurrentGpuTasks) } catch { + // Exceptions in executor plugin can cause a single thread to die but the executor process + // sticks around without any useful info until it hearbeat times out. Print what happened + // and exit immediately. + case e: CudaException => + logError("Exception in the executor plugin, shutting down!", e) + logGpuDebugInfoAndExit(systemExitCode = 1) case e: Throwable => - // Exceptions in executor plugin can cause a single thread to die but the executor process - // sticks around without any useful info until it hearbeat times out. Print what happened - // and exit immediately. logError("Exception in the executor plugin, shutting down!", e) System.exit(1) } @@ -324,6 +328,43 @@ class RapidsExecutorPlugin extends ExecutorPlugin with Logging { } } + // Wait for command spawned via Process + private def waitForProcess(cmd: Process, durationMs: Long): Option[Int] = { + val endTime = System.currentTimeMillis() + durationMs + do { + Thread.sleep(10) + if (!cmd.isAlive()) { + return Some(cmd.exitValue()) + } + } while (System.currentTimeMillis() < endTime) + // Timed out + cmd.destroy() + None + } + + // Try to run nvidia-smi when task fails due to a cuda exception. + private def logGpuDebugInfoAndExit(systemExitCode: Int) = synchronized { + try { + val nvidiaSmiStdout = new StringBuilder + val nvidiaSmiStderr = new StringBuilder + val cmd = "nvidia-smi".run( + ProcessLogger(s => nvidiaSmiStdout.append(s + "\n"), s => nvidiaSmiStderr.append(s + "\n"))) + waitForProcess(cmd, 10000) match { + case Some(exitStatus) => + if (exitStatus == 0) { + logWarning("nvidia-smi:\n" + nvidiaSmiStdout) + } else { + logWarning("nvidia-smi failed with: " + nvidiaSmiStdout + nvidiaSmiStderr) + } + case None => logWarning("nvidia-smi command timed out") + } + } catch { + case e: Throwable => + logWarning("nvidia-smi process failed", e) + } + System.exit(systemExitCode) + } + override def shutdown(): Unit = { GpuSemaphore.shutdown() PythonWorkerSemaphore.shutdown() @@ -338,7 +379,7 @@ class RapidsExecutorPlugin extends ExecutorPlugin with Logging { case Some(_: CudaFatalException) => logError("Stopping the Executor based on exception being a fatal CUDA error: " + s"${ef.toErrorString}") - System.exit(20) + logGpuDebugInfoAndExit(systemExitCode = 20) case Some(_: CudaException) => logDebug(s"Executor onTaskFailed because of a non-fatal CUDA error: " + s"${ef.toErrorString}") From 78a5c76b90ea0e97b500c7d162c99174073fae96 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 28 Sep 2022 14:20:38 -0500 Subject: [PATCH 150/190] Qualification tool application time calculation can count stages twice if in separate sql queries (#6642) * Handle deduping stages properly in qualification tool application time calculation Signed-off-by: Thomas Graves * Add test * update test to take less time * update comment Signed-off-by: Thomas Graves --- .../qualification/QualificationAppInfo.scala | 6 ++-- .../qualification/QualificationSuite.scala | 28 +++++++++++++++++++ 2 files changed, 32 insertions(+), 2 deletions(-) diff --git a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala index 05a7f1419c5..f8978b294b1 100644 --- a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala +++ b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala @@ -179,7 +179,7 @@ class QualificationAppInfo( if (execInfo.stages.size > 1) { execInfo.stages.map((_, execInfo)) } else if (execInfo.stages.size < 1) { - // we don't know what stage its in our its duration + // we don't know what stage its in or its duration logDebug(s"No stage associated with ${execInfo.exec} " + s"so speedup factor isn't applied anywhere.") execsWithoutStages += execInfo @@ -238,7 +238,6 @@ class QualificationAppInfo( def summarizeStageLevel(execInfos: Seq[ExecInfo], sqlID: Long): Set[StageQualSummaryInfo] = { val (allStagesToExecs, execsNoStage) = getStageToExec(execInfos) - if (allStagesToExecs.isEmpty) { // use job level // also get the job ids associated with the SQLId @@ -387,7 +386,10 @@ class QualificationAppInfo( 0L } + // the same stage might be referenced from multiple sql queries, we have to dedup them + // with the assumption the stage was reused so time only counts once val allStagesSummary = perSqlStageSummary.flatMap(_.stageSum) + .map(sum => sum.stageId -> sum).toMap.values.toSeq val sqlDataframeTaskDuration = allStagesSummary.map(s => s.stageTaskTime).sum val unsupportedSQLTaskDuration = calculateSQLUnsupportedTaskDuration(allStagesSummary) val endDurationEstimated = this.appEndTime.isEmpty && appDuration > 0 diff --git a/tools/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala b/tools/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala index 154db3fc4b9..5136fd4ef05 100644 --- a/tools/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala +++ b/tools/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala @@ -592,6 +592,34 @@ class QualificationSuite extends FunSuite with BeforeAndAfterEach with Logging { } } + test("test with stage reuse") { + TrampolineUtil.withTempDir { outpath => + TrampolineUtil.withTempDir { eventLogDir => + val (eventLog, _) = ToolTestUtils.generateEventLog(eventLogDir, "dot") { spark => + import spark.implicits._ + val df = spark.sparkContext.makeRDD(1 to 1000, 6).toDF + val df2 = spark.sparkContext.makeRDD(1 to 1000, 6).toDF + val j1 = df.select( $"value" as "a") + .join(df2.select($"value" as "b"), $"a" === $"b").cache() + j1.count() + j1.union(j1).count() + // count above is important thing, here we just make up small df to return + spark.sparkContext.makeRDD(1 to 2).toDF + } + + val allArgs = Array( + "--output-directory", + outpath.getAbsolutePath()) + val appArgs = new QualificationArgs(allArgs ++ Array(eventLog)) + val (exit, appSum) = QualificationMain.mainInternal(appArgs) + assert(exit == 0) + assert(appSum.size == 1) + // note this would have failed an assert with total task time to small if we + // didn't dedup stages + } + } + } + test("test generate udf different sql ops") { TrampolineUtil.withTempDir { outpath => From 498038b4cae42a97d21e9271e8406a5f6adb7583 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 29 Sep 2022 12:25:19 -0500 Subject: [PATCH 151/190] delta lake deletes/updates on Databricks fail when using alluxio [databricks] (#6601) * Fallback to CPU for delta log checkpoint file reads Signed-off-by: Thomas Graves * fix stylie * Handle xxx.checkpoint.yyy.parquet file names * Fix to use or instead of and * cleanup Signed-off-by: Thomas Graves * wrapping * Change alluxio logs to be debug Signed-off-by: Thomas Graves * Do the parquet filter with the coalescing reader in paralell Signed-off-by: Thomas Graves * fix filter param * Add filter time to datasource v2 * cleanup code and commonize * cleanup * change metric level * removing log warning messages * Add tests and logging * fix syntax * STart to change the way alluxio replaces urls Signed-off-by: Thomas Graves * continue adding support * Add in file source and change methods hardcoded * compilation fixes * use runtimeconfig * revert * Add conf for algo * put code back in * add debug logs * Rework to check for the algorithms and add option to get new filestatus * remove reuse option * fix merge conflict * code cleanup * remove debug log * Shim FileStatus * fix style * fix build for databricks * fix relation replacement * add config doc * update comment * Change to handle keeping the input file the original when replacing with alluxio Signed-off-by: Thomas Graves * fix build issues * Update how we get master * remove import * fix port * fixes merge * replace coalescing when task time * add task time for per file * update debug * Update documentation * commonize some code * fix replacing path only if alluxio is on * pass along if any Alluxio mounted * fix compileation and comment * get rid of need for rapidsconf and also deal with delta lake workaround when aqe off * Checkpoint in changing dealing with input_file_name with Alluxio * checkpoint * finish changing to original file for RDD * line length * cleanup imports * compilation fixes * fix iceberg * Handle Delta and AQE workaround around on Databricks 10.4 due to bug Signed-off-by: Thomas Graves * fixing compilation in java * Update to use SQLConf Signed-off-by: Thomas Graves * cleanup switch to SQLConf * fix line length * change to allow calling java * update to call map empty from java * comments * import order * fix import db * change where relation is created * fix extra scheme * Update to have last / * add debug * Add in unit test suite * fix * first real test * remove Suite * Add logging for testing * cleanup * document alluxio support * add some more logging * add more logging * fixes * don't use perfile reader with alluxio * updates and support specific path replacement with coalescing * update message to debug * fixes and debug * fix missing ) * fix 100 * line lengths * Update text Signed-off-by: Thomas Graves * Update configs.md Signed-off-by: Thomas Graves * remove extra import Signed-off-by: Thomas Graves * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala Co-authored-by: Alessandro Bellina * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala Co-authored-by: Alessandro Bellina * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala Co-authored-by: Alessandro Bellina * rework from review * fix missed _1 * update config text * rename alluxionPathReplacementMap and fix case class names Signed-off-by: Thomas Graves Signed-off-by: Thomas Graves Co-authored-by: Alessandro Bellina --- docs/configs.md | 4 +- .../spark/rapids/shims/Spark31XShims.scala | 42 +- .../spark/rapids/shims/Spark31XdbShims.scala | 43 +- .../rapids/shims/Spark320PlusNonDBShims.scala | 21 +- .../rapids/shims/Spark320PlusShims.scala | 9 +- .../spark/rapids/shims/SparkShims.scala | 43 +- .../spark/source/GpuMultiFileBatchReader.java | 4 +- .../iceberg/spark/source/GpuSparkScan.java | 2 +- .../nvidia/spark/rapids/AlluxioUtils.scala | 466 +++++++++++++----- .../spark/rapids/GpuMultiFileReader.scala | 72 ++- .../com/nvidia/spark/rapids/GpuOrcScan.scala | 1 + .../nvidia/spark/rapids/GpuParquetScan.scala | 89 ++-- .../spark/rapids/GpuReadCSVFileFormat.scala | 4 +- .../rapids/GpuReadFileFormatWithMetrics.scala | 4 +- .../spark/rapids/GpuReadOrcFileFormat.scala | 4 +- .../rapids/GpuReadParquetFileFormat.scala | 7 +- .../rapids/PartitionReaderIterator.scala | 3 +- .../com/nvidia/spark/rapids/RapidsConf.scala | 27 +- .../com/nvidia/spark/rapids/SparkShims.scala | 7 +- .../json/rapids/GpuReadJsonFileFormat.scala | 4 +- .../rapids/GpuPartitioningUtils.scala | 18 +- .../apache/spark/sql/rapids/GpuAvroScan.scala | 1 + .../sql/rapids/GpuFileSourceScanExec.scala | 63 +-- .../sql/rapids/GpuReadAvroFileFormat.scala | 4 +- .../rapids/GpuMultiFileReaderSuite.scala | 3 +- 25 files changed, 609 insertions(+), 336 deletions(-) diff --git a/docs/configs.md b/docs/configs.md index fb38715ecdc..17b5bb93057 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -29,11 +29,11 @@ scala> spark.conf.set("spark.rapids.sql.concurrentGpuTasks", 2) Name | Description | Default Value -----|-------------|-------------- -spark.rapids.alluxio.automount.enabled|Enable the feature of auto mounting the cloud storage to Alluxio. It requires the Alluxio master is the same node of Spark driver node. When it's true, it requires an environment variable ALLUXIO_HOME be set properly. The default value of ALLUXIO_HOME is "/opt/alluxio-2.8.0". You can set it as an environment variable when running a spark-submit or you can use spark.yarn.appMasterEnv.ALLUXIO_HOME to set it on Yarn. The Alluxio master's host and port will be read from alluxio.master.hostname and alluxio.master.rpc.port(default: 19998) from ALLUXIO_HOME/conf/alluxio-site.properties, then replace a cloud path which matches spark.rapids.alluxio.bucket.regex like "s3://bar/b.csv" to "alluxio://0.1.2.3:19998/bar/b.csv", and the bucket "s3://bar" will be mounted to "/bar" in Alluxio automatically.|false +spark.rapids.alluxio.automount.enabled|Enable the feature of auto mounting the cloud storage to Alluxio. It requires the Alluxio master is the same node of Spark driver node. When it's true, it requires an environment variable ALLUXIO_HOME be set properly. The default value of ALLUXIO_HOME is "/opt/alluxio-2.8.0". You can set it as an environment variable when running a spark-submit or you can use spark.yarn.appMasterEnv.ALLUXIO_HOME to set it on Yarn. The Alluxio master's host and port will be read from alluxio.master.hostname and alluxio.master.rpc.port(default: 19998) from ALLUXIO_HOME/conf/alluxio-site.properties, then replace a cloud path which matches spark.rapids.alluxio.bucket.regex like "s3://bar/b.csv" to "alluxio://0.1.2.3:19998/bar/b.csv", and the bucket "s3://bar" will be mounted to "/bar" in Alluxio automatically.This config should be enabled when initially starting the application but it can be turned off and one programmatically after that.|false spark.rapids.alluxio.bucket.regex|A regex to decide which bucket should be auto-mounted to Alluxio. E.g. when setting as "^s3://bucket.*", the bucket which starts with "s3://bucket" will be mounted to Alluxio and the path "s3://bucket-foo/a.csv" will be replaced to "alluxio://0.1.2.3:19998/bucket-foo/a.csv". It's only valid when setting spark.rapids.alluxio.automount.enabled=true. The default value matches all the buckets in "s3://" or "s3a://" scheme.|^s3a{0,1}://.* spark.rapids.alluxio.cmd|Provide the Alluxio command, which is used to mount or get information. The default value is "su,ubuntu,-c,/opt/alluxio-2.8.0/bin/alluxio", it means: run Process(Seq("su", "ubuntu", "-c", "/opt/alluxio-2.8.0/bin/alluxio fs mount --readonly /bucket-foo s3://bucket-foo")), to mount s3://bucket-foo to /bucket-foo. the delimiter "," is used to convert to Seq[String] when you need to use a special user to run the mount command.|List(su, ubuntu, -c, /opt/alluxio-2.8.0/bin/alluxio) spark.rapids.alluxio.pathsToReplace|List of paths to be replaced with corresponding Alluxio scheme. E.g. when configure is set to "s3://foo->alluxio://0.1.2.3:19998/foo,gs://bar->alluxio://0.1.2.3:19998/bar", it means: "s3://foo/a.csv" will be replaced to "alluxio://0.1.2.3:19998/foo/a.csv" and "gs://bar/b.csv" will be replaced to "alluxio://0.1.2.3:19998/bar/b.csv". To use this config, you have to mount the buckets to Alluxio by yourself. If you set this config, spark.rapids.alluxio.automount.enabled won't be valid.|None -spark.rapids.alluxio.replacement.algo|The algorithm used when replacing the UFS path with the Alluxio path. CONVERT_TIME and SELECTION_TIME are the valid options. CONVERT_TIME indicates that we do it when we convert it to a GPU file read, this has extra overhead of creating an entirely new file index, which requires listing the files and getting all new file info from Alluxio. SELECTION_TIME indicates we do it when the file reader is selecting the partitions to process and just replaces the path without fetching the file information again, this is faster but doesn't update locality information if that were to work with Alluxio.|SELECTION_TIME +spark.rapids.alluxio.replacement.algo|The algorithm used when replacing the UFS path with the Alluxio path. CONVERT_TIME and TASK_TIME are the valid options. CONVERT_TIME indicates that we do it when we convert it to a GPU file read, this has extra overhead of creating an entirely new file index, which requires listing the files and getting all new file info from Alluxio. TASK_TIME replaces the path as late as possible inside of the task. By waiting and replacing it at task time, it just replaces the path without fetching the file information again, this is faster but doesn't update locality information if that has a bit impact on performance.|TASK_TIME spark.rapids.cloudSchemes|Comma separated list of additional URI schemes that are to be considered cloud based filesystems. Schemes already included: abfs, abfss, dbfs, gs, s3, s3a, s3n, wasbs. Cloud based stores generally would be total separate from the executors and likely have a higher I/O read cost. Many times the cloud filesystems also get better throughput when you have multiple readers in parallel. This is used with spark.rapids.sql.format.parquet.reader.type|None spark.rapids.gpu.resourceName|The name of the Spark resource that represents a GPU that you want the plugin to use if using custom resources with Spark.|gpu spark.rapids.memory.gpu.allocFraction|The fraction of available (free) GPU memory that should be allocated for pooled memory. This must be less than or equal to the maximum limit configured via spark.rapids.memory.gpu.maxAllocFraction, and greater than or equal to the minimum limit configured via spark.rapids.memory.gpu.minAllocFraction.|1.0 diff --git a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/Spark31XShims.scala b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/Spark31XShims.scala index a65074bfb6e..14975d62d09 100644 --- a/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/Spark31XShims.scala +++ b/sql-plugin/src/main/311until320-nondb/scala/com/nvidia/spark/rapids/shims/Spark31XShims.scala @@ -19,7 +19,7 @@ package com.nvidia.spark.rapids.shims import scala.collection.mutable.ListBuffer import com.nvidia.spark.rapids._ -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.FileStatus import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -71,23 +71,6 @@ abstract class Spark31XShims extends SparkShims with Spark31Xuntil33XShims with fileIndex.allFiles() } - override def alluxioReplacePathsPartitionDirectory( - pd: PartitionDirectory, - replaceFunc: Option[Path => Path]): (Seq[FileStatus], PartitionDirectory) = { - val updatedFileStatus = pd.files.map { f => - val replaced = replaceFunc.get(f.getPath) - // Alluxio caches the entire file, so the size should be the same. - // Just hardcode block replication to 1 since we don't know what it really - // is in Alluxio and its not used by splits. The modification time shouldn't be - // affected by Alluxio. Blocksize is also not used. Note that we will not - // get new block locations with this so if Alluxio would return new ones - // this isn't going to get them. From my current experiments, Alluxio is not - // returning the block locations of the cached blocks anyway. - new FileStatus(f.getLen, f.isDirectory, 1, f.getBlockSize, f.getModificationTime, replaced) - } - (updatedFileStatus, PartitionDirectory(pd.values, updatedFileStatus)) - } - def broadcastModeTransform(mode: BroadcastMode, rows: Array[InternalRow]): Any = mode.transform(rows) @@ -368,15 +351,16 @@ abstract class Spark31XShims extends SparkShims with Spark31Xuntil33XShims with val sparkSession = wrapped.relation.sparkSession val options = wrapped.relation.options - val location = if (conf.isAlluxioReplacementAlgoConvertTime) { - AlluxioUtils.replacePathIfNeeded( - conf, - wrapped.relation, - partitionFilters, - wrapped.dataFilters) - } else { - wrapped.relation.location - } + val (location, alluxioPathsToReplaceMap) = + if (conf.isAlluxioReplacementAlgoConvertTime) { + AlluxioUtils.replacePathIfNeeded( + conf, + wrapped.relation, + partitionFilters, + wrapped.dataFilters) + } else { + (wrapped.relation.location, None) + } val newRelation = HadoopFsRelation( location, @@ -395,7 +379,9 @@ abstract class Spark31XShims extends SparkShims with Spark31Xuntil33XShims with wrapped.optionalNumCoalescedBuckets, wrapped.dataFilters, wrapped.tableIdentifier, - wrapped.disableBucketedScan)(conf) + wrapped.disableBucketedScan, + queryUsesInputFile = false, + alluxioPathsToReplaceMap)(conf) } }) ).map(r => (r.getClassFor.asSubclass(classOf[SparkPlan]), r)).toMap diff --git a/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/Spark31XdbShims.scala b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/Spark31XdbShims.scala index ec66bb6f74d..b0cd1bdda2f 100644 --- a/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/Spark31XdbShims.scala +++ b/sql-plugin/src/main/31xdb/scala/com/nvidia/spark/rapids/shims/Spark31XdbShims.scala @@ -18,7 +18,7 @@ package com.nvidia.spark.rapids.shims import com.databricks.sql.execution.window.RunningWindowFunctionExec import com.nvidia.spark.rapids._ -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.FileStatus import org.apache.spark.TaskContext import org.apache.spark.internal.Logging @@ -246,15 +246,17 @@ abstract class Spark31XdbShims extends Spark31XdbShimsBase with Logging { val sparkSession = wrapped.relation.sparkSession val options = wrapped.relation.options - val location = if (conf.isAlluxioReplacementAlgoConvertTime) { - AlluxioUtils.replacePathIfNeeded( - conf, - wrapped.relation, - partitionFilters, - wrapped.dataFilters) - } else { - wrapped.relation.location - } + val (location, alluxioPathsToReplaceMap) = + if (conf.isAlluxioReplacementAlgoConvertTime) { + AlluxioUtils.replacePathIfNeeded( + conf, + wrapped.relation, + partitionFilters, + wrapped.dataFilters) + } else { + (wrapped.relation.location, None) + } + val newRelation = HadoopFsRelation( location, wrapped.relation.partitionSchema, @@ -273,7 +275,9 @@ abstract class Spark31XdbShims extends Spark31XdbShimsBase with Logging { None, wrapped.dataFilters, wrapped.tableIdentifier, - wrapped.disableBucketedScan)(conf) + wrapped.disableBucketedScan, + queryUsesInputFile = false, + alluxioPathsToReplaceMap)(conf) } }) ).map(r => (r.getClassFor.asSubclass(classOf[SparkPlan]), r)).toMap @@ -319,23 +323,6 @@ abstract class Spark31XdbShims extends Spark31XdbShimsBase with Logging { fileCatalog.allFiles().map(_.toFileStatus) } - override def alluxioReplacePathsPartitionDirectory( - pd: PartitionDirectory, - replaceFunc: Option[Path => Path]): (Seq[FileStatus], PartitionDirectory) = { - val updatedFileStatus = pd.files.map { f => - val replaced = replaceFunc.get(f.getPath) - // Alluxio caches the entire file, so the size should be the same. - // Just hardcode block replication to 1 since we don't know what it really - // is in Alluxio and its not used by splits. The modification time shouldn't be - // affected by Alluxio. Blocksize is also not used. Note that we will not - // get new block locations with this so if Alluxio would return new ones - // this isn't going to get them. From my current experiments, Alluxio is not - // returning the block locations of the cached blocks anyway. - new FileStatus(f.length, f.isDir, 1, f.blockSize, f.modificationTime, replaced) - } - (updatedFileStatus, PartitionDirectory(pd.values, updatedFileStatus.toArray)) - } - override def isEmptyRelation(relation: Any): Boolean = false override def tryTransformIfEmptyRelation(mode: BroadcastMode): Option[Any] = None diff --git a/sql-plugin/src/main/320+-nondb/scala/com/nvidia/spark/rapids/shims/Spark320PlusNonDBShims.scala b/sql-plugin/src/main/320+-nondb/scala/com/nvidia/spark/rapids/shims/Spark320PlusNonDBShims.scala index 5691dab603d..5ce350fa0e8 100644 --- a/sql-plugin/src/main/320+-nondb/scala/com/nvidia/spark/rapids/shims/Spark320PlusNonDBShims.scala +++ b/sql-plugin/src/main/320+-nondb/scala/com/nvidia/spark/rapids/shims/Spark320PlusNonDBShims.scala @@ -16,14 +16,14 @@ package com.nvidia.spark.rapids.shims import com.nvidia.spark.rapids.SparkShims -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.NamedExpression import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.adaptive.{BroadcastQueryStageExec, ShuffleQueryStageExec} -import org.apache.spark.sql.execution.datasources.{PartitionDirectory, PartitioningAwareFileIndex} +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.python.WindowInPandasExec @@ -44,23 +44,6 @@ trait Spark320PlusNonDBShims extends SparkShims { fileIndex.allFiles() } - override def alluxioReplacePathsPartitionDirectory( - pd: PartitionDirectory, - replaceFunc: Option[Path => Path]): (Seq[FileStatus], PartitionDirectory) = { - val updatedFileStatus = pd.files.map { f => - val replaced = replaceFunc.get(f.getPath) - // Alluxio caches the entire file, so the size should be the same. - // Just hardcode block replication to 1 since we don't know what it really - // is in Alluxio and its not used by splits. The modification time shouldn't be - // affected by Alluxio. Blocksize is also not used. Note that we will not - // get new block locations with this so if Alluxio would return new ones - // this isn't going to get them. From my current experiments, Alluxio is not - // returning the block locations of the cached blocks anyway. - new FileStatus(f.getLen, f.isDirectory, 1, f.getBlockSize, f.getModificationTime, replaced) - } - (updatedFileStatus, PartitionDirectory(pd.values, updatedFileStatus)) - } - def getWindowExpressions(winPy: WindowInPandasExec): Seq[NamedExpression] = winPy.windowExpression /** diff --git a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala index 31ceba8da26..3d6238bb59c 100644 --- a/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala +++ b/sql-plugin/src/main/320+/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala @@ -384,15 +384,16 @@ trait Spark320PlusShims extends SparkShims with RebaseShims with Logging { val sparkSession = wrapped.relation.sparkSession val options = wrapped.relation.options - val location = if (conf.isAlluxioReplacementAlgoConvertTime) { + val (location, alluxioPathsToReplaceMap) = if (conf.isAlluxioReplacementAlgoConvertTime) { AlluxioUtils.replacePathIfNeeded( conf, wrapped.relation, partitionFilters, wrapped.dataFilters) } else { - wrapped.relation.location + (wrapped.relation.location, None) } + val newRelation = HadoopFsRelation( location, wrapped.relation.partitionSchema, @@ -410,7 +411,9 @@ trait Spark320PlusShims extends SparkShims with RebaseShims with Logging { wrapped.optionalNumCoalescedBuckets, wrapped.dataFilters, wrapped.tableIdentifier, - wrapped.disableBucketedScan)(conf) + wrapped.disableBucketedScan, + queryUsesInputFile = false, + alluxioPathsToReplaceMap)(conf) } } diff --git a/sql-plugin/src/main/321db/scala/com/nvidia/spark/rapids/shims/SparkShims.scala b/sql-plugin/src/main/321db/scala/com/nvidia/spark/rapids/shims/SparkShims.scala index 763a6f8ad58..b93bc2eb519 100644 --- a/sql-plugin/src/main/321db/scala/com/nvidia/spark/rapids/shims/SparkShims.scala +++ b/sql-plugin/src/main/321db/scala/com/nvidia/spark/rapids/shims/SparkShims.scala @@ -18,7 +18,7 @@ package com.nvidia.spark.rapids.shims import com.databricks.sql.execution.window.RunningWindowFunctionExec import com.nvidia.spark.rapids._ -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.FileStatus import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.memory.TaskMemoryManager @@ -72,23 +72,6 @@ object SparkShimImpl extends Spark321PlusShims with Spark320until340Shims { fileCatalog.allFiles().map(_.toFileStatus) } - override def alluxioReplacePathsPartitionDirectory( - pd: PartitionDirectory, - replaceFunc: Option[Path => Path]): (Seq[FileStatus], PartitionDirectory) = { - val updatedFileStatus = pd.files.map { f => - val replaced = replaceFunc.get(f.getPath) - // Alluxio caches the entire file, so the size should be the same. - // Just hardcode block replication to 1 since we don't know what it really - // is in Alluxio and its not used by splits. The modification time shouldn't be - // affected by Alluxio. Blocksize is also not used. Note that we will not - // get new block locations with this so if Alluxio would return new ones - // this isn't going to get them. From my current experiments, Alluxio is not - // returning the block locations of the cached blocks anyway. - new FileStatus(f.length, f.isDir, 1, f.blockSize, f.modificationTime, replaced) - } - (updatedFileStatus, PartitionDirectory(pd.values, updatedFileStatus.toArray)) - } - override def neverReplaceShowCurrentNamespaceCommand: ExecRule[_ <: SparkPlan] = null override def getWindowExpressions(winPy: WindowInPandasExec): Seq[NamedExpression] = @@ -156,15 +139,17 @@ object SparkShimImpl extends Spark321PlusShims with Spark320until340Shims { val sparkSession = wrapped.relation.sparkSession val options = wrapped.relation.options - val location = if (conf.isAlluxioReplacementAlgoConvertTime) { - AlluxioUtils.replacePathIfNeeded( - conf, - wrapped.relation, - partitionFilters, - wrapped.dataFilters) - } else { - wrapped.relation.location - } + val (location, alluxioPathsToReplaceMap) = + if (conf.isAlluxioReplacementAlgoConvertTime) { + AlluxioUtils.replacePathIfNeeded( + conf, + wrapped.relation, + partitionFilters, + wrapped.dataFilters) + } else { + (wrapped.relation.location, None) + } + val newRelation = HadoopFsRelation( location, wrapped.relation.partitionSchema, @@ -183,7 +168,9 @@ object SparkShimImpl extends Spark321PlusShims with Spark320until340Shims { None, wrapped.dataFilters, wrapped.tableIdentifier, - wrapped.disableBucketedScan)(conf) + wrapped.disableBucketedScan, + queryUsesInputFile = false, + alluxioPathsToReplaceMap)(conf) } }), GpuOverrides.exec[RunningWindowFunctionExec]( diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java index 8755fa27289..5e09feeef54 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java @@ -323,7 +323,9 @@ protected FilePartitionReaderBase createRapidsReader(PartitionedFile[] pFiles, numThreads, maxNumFileProcessed, false, // ignoreMissingFiles false, // ignoreCorruptFiles - false // useFieldId + false, // useFieldId + scala.collection.immutable.Map$.MODULE$.empty(), // alluxioPathReplacementMap + false // alluxioReplacementTaskTime ); } diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java index 2c5b670fc3a..8226b13d017 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java @@ -268,7 +268,7 @@ private scala.Tuple3 multiFileReadCheck(ReadTask r .toArray(String[]::new); // Get the final decision for the subtype of the Rapids reader. boolean useMultiThread = MultiFileReaderUtils.useMultiThreadReader( - canUseCoalescing, canUseMultiThread, files, allCloudSchemes); + canUseCoalescing, canUseMultiThread, files, allCloudSchemes, false); return scala.Tuple3.apply(canAccelerateRead, useMultiThread, ff); } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala index fd268eacd39..5c8d785891e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala @@ -22,18 +22,87 @@ import java.util.Properties import scala.io.{BufferedSource, Source} import scala.sys.process.{Process, ProcessLogger} -import com.nvidia.spark.rapids.shims.SparkShimImpl import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.sql.RuntimeConfig import org.apache.spark.sql.catalyst.expressions.{Expression, PlanExpression} -import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, FileIndex, HadoopFsRelation, InMemoryFileIndex, PartitionDirectory, PartitioningAwareFileIndex, PartitionSpec} +import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, FileFormat, FileIndex, HadoopFsRelation, InMemoryFileIndex, PartitionDirectory, PartitionedFile, PartitioningAwareFileIndex, PartitionSpec} +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.datasources.rapids.GpuPartitioningUtils +/* + * Utilities for using Alluxio with the plugin for reading. + * Currently we only support Alluxio with the Datasource v1 Parquet reader. + * We currently support 2 different replacement algorithms: + * CONVERT_TIME: this replaces the file path when we convert a FileSourceScanExec to + * a GpuFileSourceScanExec. This will create an entirely new FileIndex and potentially + * has to re-infer the partitioning if its not a FileIndex type we know. So this can + * cause an extra list leaf files which for many files will run another job and thus + * has additional overhead. This will update the file locations to be the + * Alluxio specific ones if the data is already cached. In order to support the + * input_file_name functionality we have to convert the alluxio:// path back to its + * original url when we go to actually read the file. + * TASK_TIME: this replaces the file path as late as possible on the task side when + * we actually go to read the file. This makes is so that the original non-Alluxio + * path gets reported for the input_file_name properly without having to convert + * paths back to the original. This also has the benefit that it can be more performant + * if it doesn't have to do the extra list leaf files, but you don't get the + * locality information updated. So for small Alluxio clusters or with Spark + * clusters short on task slots this may be a better fit. + * + * The way we do the actual replacement algorithm differs depending on the file reader + * type we use: PERFILE, COALESCING or MULTITHREADED. + * PERFILE is not supported with Alluxio due to not easily being able to fix up + * input_file_name. We could but would require copying the FileScanRDD so skip for now. + * The COALESCING reader is not support when input_file_name is requested so it falls + * back to the MULTITHREADED reader if that is used, when input_file_name is not requested, + * we replace the paths properly based on the replacement algorithm and don't have to worry + * about calculating the original path. The MULTITHREADED reader supports input_file_name + * so it handles calculating the original file path in the case of the convert time algorithm. + * In order to do the replacement at task time and to output the original path for convert + * time, we need to have a mapping of the original scheme to the alluxio scheme. This has been + * made a parameter to many of the readers. With auto mount and task time replacement, + * we make a pass through the files on the driver side in GpuFileSourceScanExec in order to + * do the mounting before the tasks try to access alluxio. + * Note that Delta Lake uses the input_file_name functionality to do things like + * Updates and Deletes and will fail if the path has the alluxio:// in it. + * + * Below we support 2 configs to turn on Alluxio, we have the automount which uses a regex + * to replace paths and then we have the config that specifies direct paths to replace and + * user has to manually mount those. + */ object AlluxioUtils extends Logging { private val checkedAlluxioPath = scala.collection.mutable.HashSet[String]() + private val ALLUXIO_SCHEME = "alluxio://" + private val mountedBuckets: scala.collection.mutable.Map[String, String] = + scala.collection.mutable.Map() + private var alluxioCmd: Seq[String] = null + private var alluxioMasterHost: Option[String] = None + private var alluxioPathsToReplaceMap: Option[Map[String, String]] = None + private var alluxioHome: String = "/opt/alluxio-2.8.0" + private var isInit: Boolean = false + + def checkAlluxioNotSupported(rapidsConf: RapidsConf): Unit = { + if (rapidsConf.isParquetPerFileReadEnabled && + (rapidsConf.getAlluxioAutoMountEnabled || rapidsConf.getAlluxioPathsToReplace.isDefined)) { + throw new IllegalArgumentException("Alluxio is currently not supported with the PERFILE " + + "reader, please use one of the other reader types.") + } + } + + def isAlluxioAutoMountTaskTime(rapidsConf: RapidsConf, + fileFormat: FileFormat): Boolean = { + rapidsConf.getAlluxioAutoMountEnabled && rapidsConf.isAlluxioReplacementAlgoTaskTime && + fileFormat.isInstanceOf[ParquetFileFormat] + } + + def isAlluxioPathsToReplaceTaskTime(rapidsConf: RapidsConf, + fileFormat: FileFormat): Boolean = { + rapidsConf.getAlluxioPathsToReplace.isDefined && rapidsConf.isAlluxioReplacementAlgoTaskTime && + fileFormat.isInstanceOf[ParquetFileFormat] + } private def checkAlluxioMounted( hadoopConfiguration: Configuration, @@ -54,12 +123,25 @@ object AlluxioUtils extends Logging { } } - private val mountedBuckets: scala.collection.mutable.Map[String, String] = - scala.collection.mutable.Map() - private var alluxioCmd: Seq[String] = null - private var alluxioMasterHost: Option[String] = None - private var alluxioHome: String = "/opt/alluxio-2.8.0" - private var isInit: Boolean = false + // Default to read from /opt/alluxio-2.8.0 if not setting ALLUXIO_HOME + private def readAlluxioMasterAndPort: (String, String) = { + var buffered_source: BufferedSource = null + try { + buffered_source = Source.fromFile(alluxioHome + "/conf/alluxio-site.properties") + val prop : Properties = new Properties() + prop.load(buffered_source.bufferedReader()) + val alluxio_master = prop.getProperty("alluxio.master.hostname") + val alluxio_port = prop.getProperty("alluxio.master.rpc.port", "19998") + (alluxio_master, alluxio_port) + } catch { + case e: FileNotFoundException => + throw new RuntimeException(s"Not found Alluxio config in " + + s"$alluxioHome/conf/alluxio-site.properties, " + + "please check if ALLUXIO_HOME is set correctly") + } finally { + if (buffered_source != null) buffered_source.close + } + } // Read out alluxio.master.hostname, alluxio.master.rpc.port // from Alluxio's conf alluxio-site.properties @@ -67,55 +149,44 @@ object AlluxioUtils extends Logging { // This function will only read once from ALLUXIO/conf. private def initAlluxioInfo(conf: RapidsConf): Unit = { this.synchronized { + // left outside isInit to allow changing at runtime alluxioHome = scala.util.Properties.envOrElse("ALLUXIO_HOME", "/opt/alluxio-2.8.0") alluxioCmd = conf.getAlluxioCmd + checkAlluxioNotSupported(conf) if (!isInit) { - // Default to read from /opt/alluxio-2.8.0 if not setting ALLUXIO_HOME - var alluxio_port: String = null - var alluxio_master: String = null - var buffered_source: BufferedSource = null - try { - buffered_source = Source.fromFile(alluxioHome + "/conf/alluxio-site.properties") - val prop : Properties = new Properties() - prop.load(buffered_source.bufferedReader()) - alluxio_master = prop.getProperty("alluxio.master.hostname") - alluxio_port = prop.getProperty("alluxio.master.rpc.port", "19998") - } catch { - case e: FileNotFoundException => - throw new RuntimeException(s"Not found Alluxio config in " + - s"$alluxioHome/conf/alluxio-site.properties, " + - "please check if ALLUXIO_HOME is set correctly") - } finally { - if (buffered_source != null) buffered_source.close - } - - if (alluxio_master == null) { - throw new RuntimeException( - s"Can't find alluxio.master.hostname from $alluxioHome/conf/alluxio-site.properties.") - } - alluxioMasterHost = Some(alluxio_master + ":" + alluxio_port) - // load mounted point by call Alluxio mount command. - val (ret, output) = runAlluxioCmd("fs mount") - if (ret == 0) { - // parse the output, E.g. - // s3a://bucket-foo/ on /bucket-foo - // s3a://bucket-another/ on /bucket-another - // /local_path on / - for (line <- output) { - val items = line.trim.split(" +") - logDebug(line) - if (items.length >= 3) { - // if the first item contains the "://", it means it's a remote path. - // record it as a mounted point - if (items(0).contains("://")) { - mountedBuckets(items(2)) = items(0) - logDebug(s"Found mounted bucket ${items(0)} to ${items(2)}") + if (conf.getAlluxioAutoMountEnabled) { + val (alluxio_master, alluxio_port) = readAlluxioMasterAndPort + if (alluxio_master == null) { + throw new RuntimeException( + s"Can't find alluxio.master.hostname from $alluxioHome/conf/alluxio-site.properties.") + } + alluxioMasterHost = Some(alluxio_master + ":" + alluxio_port) + val alluxioBucketRegex: String = conf.getAlluxioBucketRegex + // load mounted point by call Alluxio mount command. + val (ret, output) = runAlluxioCmd("fs mount") + if (ret == 0) { + // parse the output, E.g. + // s3a://bucket-foo/ on /bucket-foo + // s3a://bucket-another/ on /bucket-another + // /local_path on / + for (line <- output) { + val items = line.trim.split(" +") + logDebug(line) + if (items.length >= 3) { + // if the first item contains the "://", it means it's a remote path. + // record it as a mounted point + if (items(0).contains("://")) { + mountedBuckets(items(2)) = items(0) + logDebug(s"Found mounted bucket ${items(0)} to ${items(2)}") + } } } + } else { + logWarning(s"Failed to run alluxio fs mount $ret") } } else { - logWarning(s"Failed to run alluxio fs mount $ret") + alluxioPathsToReplaceMap = getReplacementMapOption(conf) } isInit = true } @@ -130,7 +201,7 @@ object AlluxioUtils extends Logging { throw new RuntimeException(s"path $path is not expected for Alluxio auto mount") } val bucket = i(1).split("/")(0) - (scheme, bucket) + (scheme + "://", bucket) } private def runAlluxioCmd(param : String) : (Int, @@ -162,7 +233,7 @@ object AlluxioUtils extends Logging { secret_key: Option[String]): Unit = { // to match the output of alluxio fs mount, append / to remote_path // and add / before bucket name for absolute path in Alluxio - val remote_path = scheme + "://" + bucket + "/" + val remote_path = scheme + bucket + "/" val local_bucket = "/" + bucket this.synchronized { if (!mountedBuckets.contains(local_bucket)) { @@ -214,8 +285,16 @@ object AlluxioUtils extends Logging { } } + private def replaceSchemeWithAlluxio(file: String, scheme: String, masterPort: String): String = { + // replace s3://foo/.. to alluxio://alluxioMasterHost/foo/... + val newFile = file.replaceFirst(scheme, ALLUXIO_SCHEME + masterPort + "/") + logDebug(s"Replace $file to ${newFile}") + newFile + } + private def genFuncForPathReplacement( - replaceMapOption: Option[Map[String, String]]): Option[Path => Path] = { + replaceMapOption: Option[Map[String, String]]) + : Option[Path => AlluxioPathReplaceConvertTime] = { if (replaceMapOption.isDefined) { Some((f: Path) => { val pathStr = f.toString @@ -226,9 +305,13 @@ object AlluxioUtils extends Logging { s"from ${RapidsConf.ALLUXIO_PATHS_REPLACE.key} which requires only 1 rule " + s"for each file path") } else if (matchedSet.size == 1) { - new Path(pathStr.replaceFirst(matchedSet.head._1, matchedSet.head._2)) + val res = AlluxioPathReplaceConvertTime( + new Path(pathStr.replaceFirst(matchedSet.head._1, matchedSet.head._2)), + Some(matchedSet.head._1)) + logDebug(s"Specific path replacement, replacing paths with: $res") + res } else { - f + AlluxioPathReplaceConvertTime(f, None) } }) } else { @@ -237,43 +320,65 @@ object AlluxioUtils extends Logging { } private def genFuncForAutoMountReplacement( - conf: RapidsConf, runtimeConf: RuntimeConfig, hadoopConf: Configuration, - alluxioBucketRegex: String) : Option[Path => Path] = { + alluxioBucketRegex: String): Option[Path => AlluxioPathReplaceConvertTime] = { Some((f: Path) => { val pathStr = f.toString - if (pathStr.matches(alluxioBucketRegex)) { - initAlluxioInfo(conf) + val res = if (pathStr.matches(alluxioBucketRegex)) { val (access_key, secret_key) = getKeyAndSecret(hadoopConf, runtimeConf) - val (scheme, bucket) = getSchemeAndBucketFromPath(pathStr) autoMountBucket(scheme, bucket, access_key, secret_key) + assert(alluxioMasterHost.isDefined) + AlluxioPathReplaceConvertTime( + new Path(replaceSchemeWithAlluxio(pathStr, scheme, alluxioMasterHost.get)), + Some(scheme)) + } else { + AlluxioPathReplaceConvertTime(f, None) + } + logDebug(s"Automount replacing paths: $res") + res + }) + } - // replace s3://foo/.. to alluxio://alluxioMasterHost/foo/... - val newPath = new Path(pathStr.replaceFirst( - scheme + ":/", "alluxio://" + alluxioMasterHost.get)) - logDebug(s"Replace $pathStr to ${newPath.toString}") - newPath + // Contains the file string to read and contains a boolean indicating if the + // path was updated to an alluxio:// path. + case class AlluxioPathReplaceTaskTime(fileStr: String, wasReplaced: Boolean) + + // Contains the file Path to read and optionally contains the prefix of the original path. + // The original path is needed when using the input_file_name option with the reader so + // it reports the original path and not the alluxio version + case class AlluxioPathReplaceConvertTime(filePath: Path, origPrefix: Option[String]) + + // Replaces the file name with Alluxio one if it matches. + // Returns a tuple with the file path and whether or not it replaced the + // scheme with the Alluxio one. + private def genFuncForTaskTimeReplacement(pathsToReplace: Map[String, String]) + : Option[String => AlluxioPathReplaceTaskTime] = { + Some((pathStr: String) => { + // pathsToReplace contain strings of exact paths to replace + val matchedSet = pathsToReplace.filter { case (pattern, _) => pathStr.startsWith(pattern) } + if (matchedSet.size > 1) { + // never reach here since replaceMap is a Map + throw new IllegalArgumentException(s"Found ${matchedSet.size} same replacing rules " + + s"from ${RapidsConf.ALLUXIO_PATHS_REPLACE.key} which requires only 1 rule " + + s"for each file path") + } else if (matchedSet.size == 1) { + AlluxioPathReplaceTaskTime( + pathStr.replaceFirst(matchedSet.head._1, matchedSet.head._2), true) } else { - f + AlluxioPathReplaceTaskTime(pathStr, false) } }) } - private def getReplacementOptions( - conf: RapidsConf, - runtimeConf: RuntimeConfig, - hadoopConf: Configuration): (Option[Path => Path], Option[Map[String, String]]) = { + private def getReplacementMapOption(conf: RapidsConf): Option[Map[String, String]] = { val alluxioPathsReplace: Option[Seq[String]] = conf.getAlluxioPathsToReplace - val alluxioAutoMountEnabled = conf.getAlluxioAutoMountEnabled - val alluxioBucketRegex: String = conf.getAlluxioBucketRegex - // alluxioPathsReplace: Seq("key->value", "key1->value1") // turn the rules to the Map with eg // { s3://foo -> alluxio://0.1.2.3:19998/foo, // gs://bar -> alluxio://0.1.2.3:19998/bar } - val replaceMapOption = if (alluxioPathsReplace.isDefined) { + if (alluxioPathsReplace.isDefined) { alluxioPathsReplace.map(rules => { rules.map(rule => { val split = rule.split("->") @@ -288,73 +393,181 @@ object AlluxioUtils extends Logging { } else { None } + } - val replaceFunc = if (replaceMapOption.isDefined) { - genFuncForPathReplacement(replaceMapOption) - } else if (alluxioAutoMountEnabled) { - genFuncForAutoMountReplacement(conf, runtimeConf, hadoopConf, alluxioBucketRegex) + private def getReplacementFunc( + conf: RapidsConf, + runtimeConf: RuntimeConfig, + hadoopConf: Configuration): Option[Path => AlluxioPathReplaceConvertTime] = { + if (conf.getAlluxioPathsToReplace.isDefined) { + genFuncForPathReplacement(alluxioPathsToReplaceMap) + } else if (conf.getAlluxioAutoMountEnabled) { + val alluxioBucketRegex: String = conf.getAlluxioBucketRegex + genFuncForAutoMountReplacement(runtimeConf, hadoopConf, alluxioBucketRegex) } else { None } - (replaceFunc, replaceMapOption) } - // This function just replaces the path in the PartitionDirectory files - // and does not do a new file listing after replacing with alluxio://. - // This saves time but would not update block locations or other metadata. - def replacePathInPDIfNeeded( + // assumes Alluxio directories already mounted at this point + def updateFilesTaskTimeIfAlluxio( + origFiles: Array[PartitionedFile], + alluxioPathReplacementMap: Option[Map[String, String]]) + : Array[PartitionedFileInfoOptAlluxio] = { + val res: Array[PartitionedFileInfoOptAlluxio] = + alluxioPathReplacementMap.map { pathsToReplace => + replacePathInPartitionFileTaskTimeIfNeeded(pathsToReplace, origFiles) + }.getOrElse(origFiles.map(PartitionedFileInfoOptAlluxio(_, None))) + logDebug(s"Updated files at TASK_TIME for Alluxio: ${res.mkString(",")}") + res + } + + // Replaces the path if needed and returns the replaced path and optionally the + // original file if it replaced the scheme with an Alluxio scheme. + def replacePathInPartitionFileTaskTimeIfNeeded( + pathsToReplace: Map[String, String], + files: Array[PartitionedFile]): Array[PartitionedFileInfoOptAlluxio] = { + val replaceFunc = genFuncForTaskTimeReplacement(pathsToReplace) + if (replaceFunc.isDefined) { + files.map { file => + val replacedFileInfo = replaceFunc.get(file.filePath) + if (replacedFileInfo.wasReplaced) { + logDebug(s"TASK_TIME replaced ${file.filePath} with ${replacedFileInfo.fileStr}") + PartitionedFileInfoOptAlluxio(PartitionedFile(file.partitionValues, + replacedFileInfo.fileStr, file.start, file.length), + Some(file)) + } else { + PartitionedFileInfoOptAlluxio(file, None) + } + } + } else { + files.map(PartitionedFileInfoOptAlluxio(_, None)) + } + } + + def autoMountIfNeeded( conf: RapidsConf, - pd: PartitionDirectory, + pds: Seq[PartitionDirectory], hadoopConf: Configuration, - runtimeConf: RuntimeConfig): PartitionDirectory = { - val (replaceFunc, replaceMapOption) = getReplacementOptions(conf, runtimeConf, hadoopConf) - if (replaceFunc.isDefined) { - val (alluxPaths, updatedPD) = - SparkShimImpl.alluxioReplacePathsPartitionDirectory(pd, replaceFunc) - // check the alluxio paths in root paths exist or not - // throw out an exception to stop the job when any of them is not mounted - if (replaceMapOption.isDefined) { - alluxPaths.map(_.getPath).foreach { rootPath => - replaceMapOption.get.values.find(value => rootPath.toString.startsWith(value)). - foreach(matched => - checkAlluxioMounted(hadoopConf, matched)) + runtimeConf: RuntimeConfig): Option[Map[String, String]] = { + val alluxioAutoMountEnabled = conf.getAlluxioAutoMountEnabled + val alluxioBucketRegex: String = conf.getAlluxioBucketRegex + initAlluxioInfo(conf) + if (alluxioAutoMountEnabled) { + val (access_key, secret_key) = getKeyAndSecret(hadoopConf, runtimeConf) + val replacedSchemes = pds.flatMap { pd => + pd.files.map(_.getPath.toString).flatMap { file => + if (file.matches(alluxioBucketRegex)) { + val (scheme, bucket) = getSchemeAndBucketFromPath(file) + autoMountBucket(scheme, bucket, access_key, secret_key) + Some(scheme) + } else { + None + } } } - updatedPD + if (replacedSchemes.nonEmpty) { + Some(replacedSchemes.map(_ -> (ALLUXIO_SCHEME + alluxioMasterHost.get + "/")).toMap) + } else { + None + } } else { - pd + None } } + def checkIfNeedsReplaced( + conf: RapidsConf, + pds: Seq[PartitionDirectory]): Option[Map[String, String]] = { + initAlluxioInfo(conf) + val anyToReplace = pds.map { pd => + pd.files.map(_.getPath.toString).map { file => + val matchedSet = alluxioPathsToReplaceMap.get.filter(a => file.startsWith(a._1)) + if (matchedSet.size > 1) { + // never reach here since replaceMap is a Map + throw new IllegalArgumentException(s"Found ${matchedSet.size} same replacing rules " + + s"from ${RapidsConf.ALLUXIO_PATHS_REPLACE.key} which requires only 1 rule " + + s"for each file path") + } else if (matchedSet.size == 1) { + true + } else { + false + } + }.contains(true) + }.contains(true) + if (anyToReplace) { + alluxioPathsToReplaceMap + } else { + None + } + } + + // reverse the replacePathIfNeeded, returns a tuple of the file passed in and then if it + // was replaced the original file + def getOrigPathFromReplaced(pfs: Array[PartitionedFile], + pathsToReplace: Map[String,String]): Array[PartitionedFileInfoOptAlluxio] = { + pfs.map { pf => + val file = pf.filePath + // pathsToReplace contain strings of exact paths to replace + val matchedSet = pathsToReplace.filter { case (_, alluxPattern) => + file.startsWith(alluxPattern) + } + if (matchedSet.size > 1) { + // never reach here since replaceMap is a Map + throw new IllegalArgumentException(s"Found ${matchedSet.size} same replacing rules " + + s"from ${RapidsConf.ALLUXIO_PATHS_REPLACE.key} which requires only 1 rule " + + s"for each file path") + } else if (matchedSet.size == 1) { + val replacedFile = file.replaceFirst(matchedSet.head._2, matchedSet.head._1) + logDebug(s"getOrigPath replacedFile: $replacedFile") + PartitionedFileInfoOptAlluxio(pf, + Some(PartitionedFile(pf.partitionValues, replacedFile, pf.start, file.length))) + } else { + PartitionedFileInfoOptAlluxio(pf, None) + } + } + } + + // This is used when replacement algorithm is CONVERT_TIME and causes + // a new lookup on the alluxio files. For unknown FileIndex types it can + // also cause us to have to infer the partitioning again. def replacePathIfNeeded( conf: RapidsConf, relation: HadoopFsRelation, partitionFilters: Seq[Expression], - dataFilters: Seq[Expression]): FileIndex = { + dataFilters: Seq[Expression]): (FileIndex, Option[Map[String, String]])= { val hadoopConf = relation.sparkSession.sparkContext.hadoopConfiguration val runtimeConf = relation.sparkSession.conf - val (replaceFunc, replaceMapOption) = getReplacementOptions(conf, runtimeConf, hadoopConf) + initAlluxioInfo(conf) + val replaceFunc = getReplacementFunc(conf, runtimeConf, hadoopConf) - if (replaceFunc.isDefined) { - def replacePathsInPartitionSpec(spec: PartitionSpec): PartitionSpec = { + val (location, allReplacedPrefixes) = if (replaceFunc.isDefined) { + def replacePathsInPartitionSpec(spec: PartitionSpec): (PartitionSpec, Seq[String]) = { val partitionsWithPathsReplaced = spec.partitions.map { p => - val replacedPath = replaceFunc.get(p.path) - org.apache.spark.sql.execution.datasources.PartitionPath(p.values, replacedPath) + val replacedPathAndPrefix = replaceFunc.get(p.path) + (org.apache.spark.sql.execution.datasources.PartitionPath(p.values, + replacedPathAndPrefix.filePath), + replacedPathAndPrefix.origPrefix) } - PartitionSpec(spec.partitionColumns, partitionsWithPathsReplaced) + val paths = partitionsWithPathsReplaced.map(_._1) + val replacedPrefixes = partitionsWithPathsReplaced.flatMap(_._2) + (PartitionSpec(spec.partitionColumns, paths), replacedPrefixes) } def createNewFileIndexWithPathsReplaced( spec: PartitionSpec, - rootPaths: Seq[Path]): InMemoryFileIndex = { - val specAdjusted = replacePathsInPartitionSpec(spec) - val replacedPaths = rootPaths.map(replaceFunc.get) - new InMemoryFileIndex( + rootPaths: Seq[Path]): (InMemoryFileIndex, Seq[String]) = { + val (specAdjusted, replacedPrefixes) = replacePathsInPartitionSpec(spec) + val replacedPathsAndIndicator = rootPaths.map(replaceFunc.get) + val replacedPaths = replacedPathsAndIndicator.map(_.filePath) + val didReplaceAnyRoots = replacedPathsAndIndicator.flatMap(_.origPrefix) + val fi = new InMemoryFileIndex( relation.sparkSession, replacedPaths, relation.options, Option(relation.dataSchema), userSpecifiedPartitionSpec = Some(specAdjusted)) + (fi, didReplaceAnyRoots ++ replacedPrefixes) } // If we know the type of file index, try to reuse as much of the existing @@ -372,7 +585,7 @@ object AlluxioUtils extends Logging { logDebug("Handling CatalogFileIndex") val memFI = cfi.filterPartitions(Nil) createNewFileIndexWithPathsReplaced(memFI.partitionSpec(), memFI.rootPaths) - case _ => { + case _ => logDebug(s"Handling file index type: ${relation.location.getClass}") // With the base Spark FileIndex type we don't know how to modify it to @@ -384,18 +597,23 @@ object AlluxioUtils extends Logging { partitionFilters.filterNot(isDynamicPruningFilter), dataFilters) // replace all of input files - val inputFiles: Seq[Path] = partitionDirs.flatMap(partitionDir => { + val inputFilesAndDidReplace = partitionDirs.flatMap(partitionDir => { partitionDir.files.map(f => replaceFunc.get(f.getPath)) }) + val inputFiles = inputFilesAndDidReplace.map(_.filePath) + val didReplaceAny = inputFilesAndDidReplace.flatMap(_.origPrefix) // replace all of rootPaths which are already unique - val rootPaths = relation.location.rootPaths.map(replaceFunc.get) + val rootPathsAndDidReplace = relation.location.rootPaths.map(replaceFunc.get) + val rootPaths = rootPathsAndDidReplace.map(_.filePath) + val rootPathsDidReplace = rootPathsAndDidReplace.flatMap(_.origPrefix) // check the alluxio paths in root paths exist or not // throw out an exception to stop the job when any of them is not mounted - if (replaceMapOption.isDefined) { + if (alluxioPathsToReplaceMap.isDefined) { rootPaths.foreach { rootPath => - replaceMapOption.get.values.find(value => rootPath.toString.startsWith(value)). + alluxioPathsToReplaceMap.get.values. + find(value => rootPath.toString.startsWith(value)). foreach(matched => checkAlluxioMounted(hadoopConf, matched)) } } @@ -403,7 +621,7 @@ object AlluxioUtils extends Logging { val parameters: Map[String, String] = relation.options // infer PartitionSpec - val partitionSpec = GpuPartitioningUtils.inferPartitioning( + val (partitionSpec, replacedBasePath) = GpuPartitioningUtils.inferPartitioning( relation.sparkSession, rootPaths, inputFiles, @@ -411,17 +629,31 @@ object AlluxioUtils extends Logging { Option(relation.dataSchema), replaceFunc.get) + val allReplacedPrefixes = didReplaceAny ++ rootPathsDidReplace ++ replacedBasePath // generate a new InMemoryFileIndex holding paths with alluxio schema - new InMemoryFileIndex( + val fi = new InMemoryFileIndex( relation.sparkSession, inputFiles, parameters, Option(relation.dataSchema), userSpecifiedPartitionSpec = Some(partitionSpec)) - } + (fi, allReplacedPrefixes) } } else { - relation.location + (relation.location, Seq.empty) + } + val mapIfReplacedPaths = if (allReplacedPrefixes.nonEmpty) { + // with alluxio.automount.enabled we only have a regex so we need to track + // the exact schemes we replaced in order to set the input_file_name properly, + // with the alluxio.pathsToReplace it already contains the exact paths + if (conf.getAlluxioAutoMountEnabled) { + Some(allReplacedPrefixes.map(_ -> (ALLUXIO_SCHEME + alluxioMasterHost.get + "/")).toMap) + } else { + alluxioPathsToReplaceMap + } + } else { + None } + (location, mapIfReplacedPaths) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala index 57c21a0fb9a..a28bb64bfa2 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala @@ -54,6 +54,8 @@ import org.apache.spark.util.SerializableConfiguration trait HostMemoryBuffersWithMetaDataBase { // PartitionedFile to be read def partitionedFile: PartitionedFile + // Original PartitionedFile if path was replaced with Alluxio + def origPartitionedFile: Option[PartitionedFile] = None // An array of BlockChunk(HostMemoryBuffer and its data size) read from PartitionedFile def memBuffersAndSizes: Array[(HostMemoryBuffer, Long)] // Total bytes read @@ -164,13 +166,16 @@ object MultiFileReaderUtils { filePaths.exists(fp => cloudSchemes.contains(fp.getScheme)) } + // If Alluxio is enabled and we do task time replacement we have to take that + // into account here so we use the Coalescing reader instead of the MultiThreaded reader. def useMultiThreadReader( coalescingEnabled: Boolean, multiThreadEnabled: Boolean, files: Array[String], - cloudSchemes: Set[String]): Boolean = - !coalescingEnabled || (multiThreadEnabled && hasPathInCloud(files, cloudSchemes)) - + cloudSchemes: Set[String], + anyAlluxioPathsReplaced: Boolean = false): Boolean = + !coalescingEnabled || (multiThreadEnabled && + (!anyAlluxioPathsReplaced && hasPathInCloud(files, cloudSchemes))) } /** @@ -180,11 +185,15 @@ object MultiFileReaderUtils { * @param sqlConf the SQLConf * @param broadcastedConf the Hadoop configuration * @param rapidsConf the Rapids configuration + * @param alluxioPathReplacementMap Optional map containing mapping of DFS + * scheme to Alluxio scheme */ abstract class MultiFilePartitionReaderFactoryBase( @transient sqlConf: SQLConf, broadcastedConf: Broadcast[SerializableConfiguration], - @transient rapidsConf: RapidsConf) extends PartitionReaderFactory with Arm with Logging { + @transient rapidsConf: RapidsConf, + alluxioPathReplacementMap: Option[Map[String, String]] = None) + extends PartitionReaderFactory with Arm with Logging { protected val maxReadBatchSizeRows = rapidsConf.maxReadBatchSizeRows protected val maxReadBatchSizeBytes = rapidsConf.maxReadBatchSizeBytes @@ -256,8 +265,8 @@ abstract class MultiFilePartitionReaderFactoryBase( /** for testing */ private[rapids] def useMultiThread(filePaths: Array[String]): Boolean = - MultiFileReaderUtils.useMultiThreadReader( - canUseCoalesceFilesReader, canUseMultiThreadReader, filePaths, allCloudSchemes) + MultiFileReaderUtils.useMultiThreadReader(canUseCoalesceFilesReader, + canUseMultiThreadReader, filePaths, allCloudSchemes, alluxioPathReplacementMap.isDefined) } /** @@ -315,6 +324,11 @@ abstract class FilePartitionReaderBase(conf: Configuration, execMetrics: Map[Str } } +// Contains the actual file path to read from and then an optional original path if its read from +// Alluxio. To make it transparent to the user, we return the original non-Alluxio path +// for input_file_name. +case class PartitionedFileInfoOptAlluxio(toRead: PartitionedFile, original: Option[PartitionedFile]) + /** * The Abstract multi-file cloud reading framework * @@ -330,15 +344,20 @@ abstract class FilePartitionReaderBase(conf: Configuration, execMetrics: Map[Str * @param filters push down filters * @param execMetrics the metrics * @param ignoreCorruptFiles Whether to ignore corrupt files when GPU failed to decode the files + * @param alluxioPathReplacementMap Map containing mapping of DFS scheme to Alluxio scheme + * @param alluxioReplacementTaskTime Whether the Alluxio replacement algorithm is set to task time */ abstract class MultiFileCloudPartitionReaderBase( conf: Configuration, - files: Array[PartitionedFile], + inputFiles: Array[PartitionedFile], numThreads: Int, maxNumFileProcessed: Int, filters: Array[Filter], execMetrics: Map[String, GpuMetric], - ignoreCorruptFiles: Boolean = false) extends FilePartitionReaderBase(conf, execMetrics) { + ignoreCorruptFiles: Boolean = false, + alluxioPathReplacementMap: Map[String, String] = Map.empty, + alluxioReplacementTaskTime: Boolean = false) + extends FilePartitionReaderBase(conf, execMetrics) { private var filesToRead = 0 protected var currentFileHostBuffers: Option[HostMemoryBuffersWithMetaDataBase] = None @@ -348,21 +367,36 @@ abstract class MultiFileCloudPartitionReaderBase( private[this] val inputMetrics = Option(TaskContext.get).map(_.taskMetrics().inputMetrics) .getOrElse(TrampolineUtil.newInputMetrics()) + private val files: Array[PartitionedFileInfoOptAlluxio] = { + if (alluxioPathReplacementMap.nonEmpty) { + if (alluxioReplacementTaskTime) { + AlluxioUtils.updateFilesTaskTimeIfAlluxio(inputFiles, Some(alluxioPathReplacementMap)) + } else { + // was done at CONVERT_TIME, need to recalculate the original path to set for + // input_file_name + AlluxioUtils.getOrigPathFromReplaced(inputFiles, alluxioPathReplacementMap) + } + } else { + inputFiles.map(PartitionedFileInfoOptAlluxio(_, None)) + } + } + private def initAndStartReaders(): Unit = { // limit the number we submit at once according to the config if set val limit = math.min(maxNumFileProcessed, files.length) val tc = TaskContext.get for (i <- 0 until limit) { val file = files(i) + logDebug(s"MultiFile reader using file ${file.toRead}, orig file is ${file.original}") // Add these in the order as we got them so that we can make sure // we process them in the same order as CPU would. val threadPool = MultiFileReaderThreadPool.getOrCreateThreadPool(numThreads) - tasks.add(threadPool.submit(getBatchRunner(tc, file, conf, filters))) + tasks.add(threadPool.submit(getBatchRunner(tc, file.toRead, file.original, conf, filters))) } // queue up any left to add once others finish for (i <- limit until files.length) { val file = files(i) - tasksToRun.enqueue(getBatchRunner(tc, file, conf, filters)) + tasksToRun.enqueue(getBatchRunner(tc, file.toRead, file.original, conf, filters)) } isInitted = true filesToRead = files.length @@ -374,6 +408,7 @@ abstract class MultiFileCloudPartitionReaderBase( * * @param tc task context to use * @param file file to be read + * @param origFile optional original unmodified file if replaced with Alluxio * @param conf the Configuration parameters * @param filters push down filters * @return Callable[HostMemoryBuffersWithMetaDataBase] @@ -381,6 +416,7 @@ abstract class MultiFileCloudPartitionReaderBase( def getBatchRunner( tc: TaskContext, file: PartitionedFile, + origFile: Option[PartitionedFile], conf: Configuration, filters: Array[Filter]): Callable[HostMemoryBuffersWithMetaDataBase] @@ -418,8 +454,7 @@ abstract class MultiFileCloudPartitionReaderBase( readBatch(currentFileHostBuffers.get) } catch { case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles => - logWarning( - s"Skipped the corrupted file: ${file}", e) + logWarning(s"Skipped the corrupted file: ${file}", e) None } } @@ -440,10 +475,14 @@ abstract class MultiFileCloudPartitionReaderBase( filesToRead -= 1 TrampolineUtil.incBytesRead(inputMetrics, fileBufsAndMeta.bytesRead) + // if we replaced the path with Alluxio, set it to the original filesystem file + // since Alluxio replacement is supposed to be transparent to the user + val inputFileToSet = + fileBufsAndMeta.origPartitionedFile.getOrElse(fileBufsAndMeta.partitionedFile) InputFileUtils.setInputFileBlock( - fileBufsAndMeta.partitionedFile.filePath, - fileBufsAndMeta.partitionedFile.start, - fileBufsAndMeta.partitionedFile.length) + inputFileToSet.filePath, + inputFileToSet.start, + inputFileToSet.length) if (getSizeOfHostBuffers(fileBufsAndMeta) == 0) { // if sizes are 0 means no rows and no data so skip to next file @@ -457,8 +496,7 @@ abstract class MultiFileCloudPartitionReaderBase( readBatch(fileBufsAndMeta) } catch { case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles => - logWarning( - s"Skipped the corrupted file: ${file}", e) + logWarning(s"Skipped the corrupted file: ${file}", e) None } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index 03222aecde5..da79d1e70be 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -1708,6 +1708,7 @@ class MultiFileCloudOrcPartitionReader( override def getBatchRunner( tc: TaskContext, file: PartitionedFile, + origFile: Option[PartitionedFile], conf: Configuration, filters: Array[Filter]): Callable[HostMemoryBuffersWithMetaDataBase] = { new ReadBatchRunner(tc, file, conf, filters) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index 31f1c300391..ee45223fe9f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -117,11 +117,11 @@ case class GpuParquetScan( logInfo("Using the original per file parquet reader") GpuParquetPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, dataSchema, readDataSchema, readPartitionSchema, pushedFilters, rapidsConf, metrics, - options.asScala.toMap) + options.asScala.toMap, None) } else { GpuParquetMultiFilePartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, dataSchema, readDataSchema, readPartitionSchema, pushedFilters, rapidsConf, metrics, - queryUsesInputFile) + queryUsesInputFile, None) } } @@ -934,8 +934,10 @@ case class GpuParquetMultiFilePartitionReaderFactory( filters: Array[Filter], @transient rapidsConf: RapidsConf, metrics: Map[String, GpuMetric], - queryUsesInputFile: Boolean) - extends MultiFilePartitionReaderFactoryBase(sqlConf, broadcastedConf, rapidsConf) { + queryUsesInputFile: Boolean, + alluxioPathReplacementMap: Option[Map[String, String]]) + extends MultiFilePartitionReaderFactoryBase(sqlConf, broadcastedConf, + rapidsConf, alluxioPathReplacementMap) { private val isCaseSensitive = sqlConf.caseSensitiveAnalysis private val debugDumpPrefix = rapidsConf.parquetDebugDumpPrefix @@ -948,10 +950,13 @@ case class GpuParquetMultiFilePartitionReaderFactory( private val filterHandler = GpuParquetFileFilterHandler(sqlConf) private val readUseFieldId = ParquetSchemaClipShims.useFieldId(sqlConf) private val numFilesFilterParallel = rapidsConf.numFilesFilterParallel + private val alluxioReplacementTaskTime = rapidsConf.isAlluxioReplacementAlgoTaskTime - // we can't use the coalescing files reader when InputFileName, InputFileBlockStart, + // We can't use the coalescing files reader when InputFileName, InputFileBlockStart, // or InputFileBlockLength because we are combining all the files into a single buffer - // and we don't know which file is associated with each row. + // and we don't know which file is associated with each row. If this changes we need to + // make sure the Alluxio path replacement also handles setting the input file name to + // the non-Alluxio path like the multi-threaded reader does. override val canUseCoalesceFilesReader: Boolean = rapidsConf.isParquetCoalesceFileReadEnabled && !(queryUsesInputFile || ignoreCorruptFiles) @@ -973,7 +978,8 @@ case class GpuParquetMultiFilePartitionReaderFactory( new MultiFileCloudParquetPartitionReader(conf, files, filterFunc, isCaseSensitive, debugDumpPrefix, maxReadBatchSizeRows, maxReadBatchSizeBytes, metrics, partitionSchema, numThreads, maxNumFileProcessed, - ignoreMissingFiles, ignoreCorruptFiles, readUseFieldId) + ignoreMissingFiles, ignoreCorruptFiles, readUseFieldId, + alluxioPathReplacementMap.getOrElse(Map.empty), alluxioReplacementTaskTime) } private def filterBlocksForCoalescingReader( @@ -983,6 +989,7 @@ case class GpuParquetMultiFilePartitionReaderFactory( filters: Array[Filter], readDataSchema: StructType): BlockMetaWithPartFile = { try { + logDebug(s"Filtering blocks for coalescing reader, file: ${file.filePath}") val meta = filterHandler.filterBlocks(footerReadType, file, conf, filters, readDataSchema) BlockMetaWithPartFile(meta, file) @@ -1033,8 +1040,18 @@ case class GpuParquetMultiFilePartitionReaderFactory( * @return coalescing reading PartitionReader */ override def buildBaseColumnarReaderForCoalescing( - files: Array[PartitionedFile], + origFiles: Array[PartitionedFile], conf: Configuration): PartitionReader[ColumnarBatch] = { + // update the file paths for Alluxio if needed, the coalescing reader doesn't support + // input_file_name so no need to track what the non Alluxio file name is + val files = if (alluxioReplacementTaskTime) { + AlluxioUtils.updateFilesTaskTimeIfAlluxio(origFiles, alluxioPathReplacementMap).map(_.toRead) + } else { + // Since coalescing reader isn't supported if input_file_name is used, so won't + // ever get here with that. So with convert time or no Alluxio just use the files as + // passed in. + origFiles + } val clippedBlocks = ArrayBuffer[ParquetSingleDataBlockMeta]() val startTime = System.nanoTime() val metaAndFilesArr = if (numFilesFilterParallel > 0) { @@ -1093,7 +1110,8 @@ case class GpuParquetPartitionReaderFactory( filters: Array[Filter], @transient rapidsConf: RapidsConf, metrics: Map[String, GpuMetric], - @transient params: Map[String, String]) + @transient params: Map[String, String], + alluxioPathReplacementMap: Option[Map[String, String]]) extends ShimFilePartitionReaderFactory(params) with Arm with Logging { private val isCaseSensitive = sqlConf.caseSensitiveAnalysis @@ -1105,6 +1123,7 @@ case class GpuParquetPartitionReaderFactory( private val filterHandler = GpuParquetFileFilterHandler(sqlConf) private val readUseFieldId = ParquetSchemaClipShims.useFieldId(sqlConf) + private val alluxioReplacementTaskTime = rapidsConf.isAlluxioReplacementAlgoTaskTime override def supportColumnarReads(partition: InputPartition): Boolean = true @@ -1723,6 +1742,8 @@ class MultiFileParquetPartitionReader( * @param ignoreMissingFiles Whether to ignore missing files * @param ignoreCorruptFiles Whether to ignore corrupt files * @param useFieldId Whether to use field id for column matching + * @param alluxioPathReplacementMap Map containing mapping of DFS scheme to Alluxio scheme + * @param alluxioReplacementTaskTime Whether the Alluxio replacement algorithm is set to task time */ class MultiFileCloudParquetPartitionReader( override val conf: Configuration, @@ -1738,26 +1759,30 @@ class MultiFileCloudParquetPartitionReader( maxNumFileProcessed: Int, ignoreMissingFiles: Boolean, ignoreCorruptFiles: Boolean, - useFieldId: Boolean) + useFieldId: Boolean, + alluxioPathReplacementMap: Map[String, String], + alluxioReplacementTaskTime: Boolean) extends MultiFileCloudPartitionReaderBase(conf, files, numThreads, maxNumFileProcessed, null, - execMetrics, ignoreCorruptFiles) with ParquetPartitionReaderBase { + execMetrics, ignoreCorruptFiles, alluxioPathReplacementMap, alluxioReplacementTaskTime) + with ParquetPartitionReaderBase { private case class HostMemoryEmptyMetaData( - override val partitionedFile: PartitionedFile, - bufferSize: Long, - override val bytesRead: Long, - isCorrectRebaseMode: Boolean, - isCorrectInt96RebaseMode: Boolean, - hasInt96Timestamps: Boolean, - clippedSchema: MessageType, - readSchema: StructType) extends HostMemoryBuffersWithMetaDataBase { - + override val partitionedFile: PartitionedFile, + override val origPartitionedFile: Option[PartitionedFile], + bufferSize: Long, + override val bytesRead: Long, + isCorrectRebaseMode: Boolean, + isCorrectInt96RebaseMode: Boolean, + hasInt96Timestamps: Boolean, + clippedSchema: MessageType, + readSchema: StructType) extends HostMemoryBuffersWithMetaDataBase { override def memBuffersAndSizes: Array[(HostMemoryBuffer, Long)] = Array(null.asInstanceOf[HostMemoryBuffer] -> bufferSize) } case class HostMemoryBuffersWithMetaData( override val partitionedFile: PartitionedFile, + override val origPartitionedFile: Option[PartitionedFile], override val memBuffersAndSizes: Array[(HostMemoryBuffer, Long)], override val bytesRead: Long, isCorrectRebaseMode: Boolean, @@ -1768,6 +1793,7 @@ class MultiFileCloudParquetPartitionReader( private class ReadBatchRunner( file: PartitionedFile, + origPartitionedFile: Option[PartitionedFile], filterFunc: PartitionedFile => ParquetFileInfoWithBlockMeta, taskContext: TaskContext) extends Callable[HostMemoryBuffersWithMetaDataBase] with Logging { @@ -1788,13 +1814,13 @@ class MultiFileCloudParquetPartitionReader( } catch { case e: FileNotFoundException if ignoreMissingFiles => logWarning(s"Skipped missing file: ${file.filePath}", e) - HostMemoryEmptyMetaData(file, 0, 0, false, false, false, null, null) + HostMemoryEmptyMetaData(file, origPartitionedFile, 0, 0, false, false, false, null, null) // Throw FileNotFoundException even if `ignoreCorruptFiles` is true case e: FileNotFoundException if !ignoreMissingFiles => throw e case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles => logWarning( s"Skipped the rest of the content in the corrupted file: ${file.filePath}", e) - HostMemoryEmptyMetaData(file, 0, 0, false, false, false, null, null) + HostMemoryEmptyMetaData(file, origPartitionedFile, 0, 0, false, false, false, null, null) } finally { TrampolineUtil.unsetTaskContext() } @@ -1813,7 +1839,7 @@ class MultiFileCloudParquetPartitionReader( if (fileBlockMeta.blocks.isEmpty) { val bytesRead = fileSystemBytesRead() - startingBytesRead // no blocks so return null buffer and size 0 - HostMemoryEmptyMetaData(file, 0, bytesRead, + HostMemoryEmptyMetaData(file, origPartitionedFile, 0, bytesRead, fileBlockMeta.isCorrectedRebaseMode, fileBlockMeta.isCorrectedInt96RebaseMode, fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema) } else { @@ -1821,7 +1847,7 @@ class MultiFileCloudParquetPartitionReader( if (isDone) { val bytesRead = fileSystemBytesRead() - startingBytesRead // got close before finishing - HostMemoryEmptyMetaData(file, 0, bytesRead, + HostMemoryEmptyMetaData(file, origPartitionedFile, 0, bytesRead, fileBlockMeta.isCorrectedRebaseMode, fileBlockMeta.isCorrectedInt96RebaseMode, fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema) } else { @@ -1829,7 +1855,7 @@ class MultiFileCloudParquetPartitionReader( val bytesRead = fileSystemBytesRead() - startingBytesRead val numRows = fileBlockMeta.blocks.map(_.getRowCount).sum.toInt // overload size to be number of rows with null buffer - HostMemoryEmptyMetaData(file, numRows, bytesRead, + HostMemoryEmptyMetaData(file, origPartitionedFile, numRows, bytesRead, fileBlockMeta.isCorrectedRebaseMode, fileBlockMeta.isCorrectedInt96RebaseMode, fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema) } else { @@ -1843,13 +1869,15 @@ class MultiFileCloudParquetPartitionReader( if (isDone) { // got close before finishing hostBuffers.foreach(_._1.safeClose()) - HostMemoryEmptyMetaData(file, 0, bytesRead, + HostMemoryEmptyMetaData(file, origPartitionedFile, 0, bytesRead, fileBlockMeta.isCorrectedRebaseMode, fileBlockMeta.isCorrectedInt96RebaseMode, fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema) } else { - HostMemoryBuffersWithMetaData(file, hostBuffers.toArray, bytesRead, - fileBlockMeta.isCorrectedRebaseMode, fileBlockMeta.isCorrectedInt96RebaseMode, - fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema) + + HostMemoryBuffersWithMetaData(file, origPartitionedFile, hostBuffers.toArray, + bytesRead, fileBlockMeta.isCorrectedRebaseMode, + fileBlockMeta.isCorrectedInt96RebaseMode, fileBlockMeta.hasInt96Timestamps, + fileBlockMeta.schema, fileBlockMeta.readSchema) } } } @@ -1877,9 +1905,10 @@ class MultiFileCloudParquetPartitionReader( override def getBatchRunner( tc: TaskContext, file: PartitionedFile, + origFile: Option[PartitionedFile], conf: Configuration, filters: Array[Filter]): Callable[HostMemoryBuffersWithMetaDataBase] = { - new ReadBatchRunner(file, filterFunc, tc) + new ReadBatchRunner(file, origFile, filterFunc, tc) } /** diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala index 22e3e1cdbd1..7cf2a9fbd6e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala @@ -41,7 +41,9 @@ class GpuReadCSVFileFormat extends CSVFileFormat with GpuReadFileFormatWithMetri filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration, - metrics: Map[String, GpuMetric]): PartitionedFile => Iterator[InternalRow] = { + metrics: Map[String, GpuMetric], + alluxioPathReplacementMap: Option[Map[String, String]] = None) + : PartitionedFile => Iterator[InternalRow] = { val sqlConf = sparkSession.sessionState.conf val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala index 7c7d9556c68..488aea02039 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala @@ -59,5 +59,7 @@ trait GpuReadFileFormatWithMetrics extends FileFormat { filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration, - metrics: Map[String, GpuMetric]): PartitionedFile => Iterator[InternalRow] + metrics: Map[String, GpuMetric], + alluxioPathReplacementMap: Option[Map[String, String]]) + : PartitionedFile => Iterator[InternalRow] } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala index 82b4c41a364..b6dd08856a0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala @@ -40,7 +40,9 @@ class GpuReadOrcFileFormat extends OrcFileFormat with GpuReadFileFormatWithMetri filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration, - metrics: Map[String, GpuMetric]): PartitionedFile => Iterator[InternalRow] = { + metrics: Map[String, GpuMetric], + alluxioPathReplacementMap: Option[Map[String, String]] = None) + : PartitionedFile => Iterator[InternalRow] = { val sqlConf = sparkSession.sessionState.conf val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala index 504ed0f4ae1..8718219222c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala @@ -40,7 +40,9 @@ class GpuReadParquetFileFormat extends ParquetFileFormat with GpuReadFileFormatW filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration, - metrics: Map[String, GpuMetric]): PartitionedFile => Iterator[InternalRow] = { + metrics: Map[String, GpuMetric], + alluxioPathReplacementMap: Option[Map[String, String]]) + : PartitionedFile => Iterator[InternalRow] = { val sqlConf = sparkSession.sessionState.conf val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) @@ -53,7 +55,8 @@ class GpuReadParquetFileFormat extends ParquetFileFormat with GpuReadFileFormatW filters.toArray, new RapidsConf(sqlConf), metrics, - options) + options, + alluxioPathReplacementMap) PartitionReaderIterator.buildReader(factory) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/PartitionReaderIterator.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/PartitionReaderIterator.scala index 3f9a67b408e..27760396c75 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/PartitionReaderIterator.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/PartitionReaderIterator.scala @@ -53,7 +53,8 @@ class PartitionReaderIterator(reader: PartitionReader[ColumnarBatch]) } object PartitionReaderIterator { - def buildReader(factory: FilePartitionReaderFactory): PartitionedFile => Iterator[InternalRow] = { + def buildReader(factory: FilePartitionReaderFactory) + : PartitionedFile => Iterator[InternalRow] = { file: PartitionedFile => { val reader = factory.buildColumnarReader(file) new PartitionReaderIterator(reader).asInstanceOf[Iterator[InternalRow]] diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 811aa8dd2c6..b0b8ec05cb4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -1371,7 +1371,9 @@ object RapidsConf { "alluxio.master.rpc.port(default: 19998) from ALLUXIO_HOME/conf/alluxio-site.properties, " + "then replace a cloud path which matches spark.rapids.alluxio.bucket.regex like " + "\"s3://bar/b.csv\" to \"alluxio://0.1.2.3:19998/bar/b.csv\", " + - "and the bucket \"s3://bar\" will be mounted to \"/bar\" in Alluxio automatically.") + "and the bucket \"s3://bar\" will be mounted to \"/bar\" in Alluxio automatically." + + "This config should be enabled when initially starting the application but it " + + "can be turned off and one programmatically after that.") .booleanConf .createWithDefault(false) @@ -1400,15 +1402,16 @@ object RapidsConf { val ALLUXIO_REPLACEMENT_ALGO = conf("spark.rapids.alluxio.replacement.algo") .doc("The algorithm used when replacing the UFS path with the Alluxio path. CONVERT_TIME " + - "and SELECTION_TIME are the valid options. CONVERT_TIME indicates that we do it when " + - "we convert it to a GPU file read, this has extra overhead of creating an entirely new " + - "file index, which requires listing the files and getting all new file info from Alluxio. " + - "SELECTION_TIME indicates we do it when the file reader is selecting the partitions " + - "to process and just replaces the path without fetching the file information again, this " + - "is faster but doesn't update locality information if that were to work with Alluxio.") + "and TASK_TIME are the valid options. CONVERT_TIME indicates that we do it " + + "when we convert it to a GPU file read, this has extra overhead of creating an entirely " + + "new file index, which requires listing the files and getting all new file info from " + + "Alluxio. TASK_TIME replaces the path as late as possible inside of the task. " + + "By waiting and replacing it at task time, it just replaces " + + "the path without fetching the file information again, this is faster " + + "but doesn't update locality information if that has a bit impact on performance.") .stringConf - .checkValues(Set("CONVERT_TIME", "SELECTION_TIME")) - .createWithDefault("SELECTION_TIME") + .checkValues(Set("CONVERT_TIME", "TASK_TIME")) + .createWithDefault("TASK_TIME") // USER FACING DEBUG CONFIGS @@ -2111,12 +2114,12 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val getAlluxioReplacementAlgo: String = get(ALLUXIO_REPLACEMENT_ALGO) - lazy val isAlluxioReplacementAlgoSelectTime: Boolean = - get(ALLUXIO_REPLACEMENT_ALGO) == "SELECTION_TIME" - lazy val isAlluxioReplacementAlgoConvertTime: Boolean = get(ALLUXIO_REPLACEMENT_ALGO) == "CONVERT_TIME" + lazy val isAlluxioReplacementAlgoTaskTime: Boolean = + get(ALLUXIO_REPLACEMENT_ALGO) == "TASK_TIME" + lazy val driverTimeZone: Option[String] = get(DRIVER_TIMEZONE) lazy val isRangeWindowByteEnabled: Boolean = get(ENABLE_RANGE_WINDOW_BYTES) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala index 4007d3815ef..589840d078b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala @@ -16,7 +16,7 @@ package com.nvidia.spark.rapids -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.FileStatus import org.apache.parquet.schema.MessageType import org.apache.spark.rdd.RDD @@ -30,7 +30,7 @@ import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, BroadcastQueryStageExec} import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionDirectory, PartitionedFile, PartitioningAwareFileIndex} +import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile, PartitioningAwareFileIndex} import org.apache.spark.sql.execution.datasources.parquet.ParquetFilters import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.internal.SQLConf @@ -114,9 +114,6 @@ trait SparkShims { def filesFromFileIndex(fileCatalog: PartitioningAwareFileIndex): Seq[FileStatus] - def alluxioReplacePathsPartitionDirectory( pd: PartitionDirectory, - replaceFunc: Option[Path => Path]): (Seq[FileStatus], PartitionDirectory) - def isEmptyRelation(relation: Any): Boolean def broadcastModeTransform(mode: BroadcastMode, toArray: Array[InternalRow]): Any diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala index 7830c1af7c1..7fe59b983cc 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala @@ -42,7 +42,9 @@ class GpuReadJsonFileFormat extends JsonFileFormat with GpuReadFileFormatWithMet filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration, - metrics: Map[String, GpuMetric]): PartitionedFile => Iterator[InternalRow] = { + metrics: Map[String, GpuMetric], + alluxioPathReplacementMap: Option[Map[String, String]] = None) + : PartitionedFile => Iterator[InternalRow] = { val sqlConf = sparkSession.sessionState.conf val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/execution/datasources/rapids/GpuPartitioningUtils.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/execution/datasources/rapids/GpuPartitioningUtils.scala index db27e2dd172..d4a88a25c52 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/execution/datasources/rapids/GpuPartitioningUtils.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/execution/datasources/rapids/GpuPartitioningUtils.scala @@ -25,6 +25,7 @@ import scala.collection.mutable.ArrayBuffer import scala.util.Try import scala.util.control.NonFatal +import com.nvidia.spark.rapids.AlluxioUtils.AlluxioPathReplaceConvertTime import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -57,7 +58,8 @@ object GpuPartitioningUtils extends SQLConfHelper { * @param userSpecifiedSchema an optional user specified schema that will be use to provide * types for the discovered partitions * @param replaceFunc the alluxio replace function - * @return the specification of the partitions inferred from the data + * @return the specification of the partitions inferred from the data and if it was replaced the + * original path * * Mainly copied from PartitioningAwareFileIndex.inferPartitioning */ @@ -67,12 +69,13 @@ object GpuPartitioningUtils extends SQLConfHelper { leafFiles: Seq[Path], parameters: Map[String, String], userSpecifiedSchema: Option[StructType], - replaceFunc: Path => Path): PartitionSpec = { + replaceFunc: Path => AlluxioPathReplaceConvertTime) + : (PartitionSpec, Option[String]) = { val recursiveFileLookup = parameters.getOrElse("recursiveFileLookup", "false").toBoolean if (recursiveFileLookup) { - PartitionSpec.emptySpec + (PartitionSpec.emptySpec, None) } else { val caseInsensitiveOptions = CaseInsensitiveMap(parameters) val timeZoneId = caseInsensitiveOptions.get(DateTimeUtils.TIMEZONE_OPTION) @@ -81,15 +84,17 @@ object GpuPartitioningUtils extends SQLConfHelper { // filter out non-data path and get unique leaf dirs of inputFiles val leafDirs: Seq[Path] = leafFiles.filter(isDataPath).map(_.getParent).distinct - val basePathOption = parameters.get(BASE_PATH_PARAM).map(file => { + val basePathAndAnyReplacedOption = parameters.get(BASE_PATH_PARAM).map { file => // need to replace the base path replaceFunc(new Path(file)) - }) + } + val basePathOption = basePathAndAnyReplacedOption.map(_.filePath) + val anyReplacedBase = basePathAndAnyReplacedOption.flatMap(_.origPrefix) val basePaths = getBasePaths(sparkSession.sessionState.newHadoopConfWithOptions(parameters), basePathOption, rootPaths, leafFiles) - parsePartitions( + val parsed = parsePartitions( leafDirs, typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled, basePaths = basePaths, @@ -97,6 +102,7 @@ object GpuPartitioningUtils extends SQLConfHelper { caseSensitive = sparkSession.sqlContext.conf.caseSensitiveAnalysis, validatePartitionColumns = sparkSession.sqlContext.conf.validatePartitionColumns, timeZoneId = timeZoneId) + (parsed, anyReplacedBase) } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala index 309ee6ee15e..b91a0b5aae6 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala @@ -658,6 +658,7 @@ class GpuMultiFileCloudAvroPartitionReader( override def getBatchRunner( tc: TaskContext, file: PartitionedFile, + origFile: Option[PartitionedFile], config: Configuration, filters: Array[Filter]): Callable[HostMemoryBuffersWithMetaDataBase] = new ReadBatchRunner(tc, file, config, filters) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala index 1f5caf1ccfd..eb743d9a9bb 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala @@ -61,6 +61,7 @@ import org.apache.spark.util.collection.BitSet * off in GpuTransitionOverrides if InputFileName, * InputFileBlockStart, or InputFileBlockLength are used * @param disableBucketedScan Disable bucketed scan based on physical query plan. + * @param alluxioPathsMap Map containing mapping of DFS scheme to Alluxio scheme */ case class GpuFileSourceScanExec( @transient relation: HadoopFsRelation, @@ -72,10 +73,17 @@ case class GpuFileSourceScanExec( dataFilters: Seq[Expression], tableIdentifier: Option[TableIdentifier], disableBucketedScan: Boolean = false, - queryUsesInputFile: Boolean = false)(@transient val rapidsConf: RapidsConf) + queryUsesInputFile: Boolean = false, + alluxioPathsMap: Option[Map[String, String]])(@transient val rapidsConf: RapidsConf) extends GpuDataSourceScanExec with GpuExec { import GpuMetric._ + private val isAlluxioReplacementTaskTime = rapidsConf.isAlluxioReplacementAlgoTaskTime + + // this is set only when we either explicitly replaced a path for CONVERT_TIME + // or when TASK_TIME if one of the paths will be replaced + private var alluxioPathReplacementMap: Option[Map[String, String]] = alluxioPathsMap + private val isPerFileReadEnabled = relation.fileFormat match { case _: ParquetFileFormat => rapidsConf.isParquetPerFileReadEnabled case _: OrcFileFormat => rapidsConf.isOrcPerFileReadEnabled @@ -84,15 +92,6 @@ case class GpuFileSourceScanExec( case _ => true // For others, default to PERFILE reader } - private val isAlluxioAlgoSelectionTime = { - val alluxioPathsReplace = rapidsConf.getAlluxioPathsToReplace - val alluxioAutoMountEnabled = rapidsConf.getAlluxioAutoMountEnabled - // currently only support Alluxio replacement with Parquet files - (alluxioPathsReplace.isDefined || alluxioAutoMountEnabled) && - rapidsConf.isAlluxioReplacementAlgoSelectTime && - relation.fileFormat.isInstanceOf[ParquetFileFormat] - } - override def otherCopyArgs: Seq[AnyRef] = Seq(rapidsConf) // All expressions are filter expressions used on the CPU. @@ -121,27 +120,25 @@ case class GpuFileSourceScanExec( @transient lazy val selectedPartitions: Array[PartitionDirectory] = { val optimizerMetadataTimeNs = relation.location.metadataOpsTimeNs.getOrElse(0L) val startTime = System.nanoTime() - val origRet = - relation.location.listFiles( + val pds = relation.location.listFiles( partitionFilters.filterNot(isDynamicPruningFilter), dataFilters) - val ret = if (isAlluxioAlgoSelectionTime) { - val res = origRet.map { pd => - AlluxioUtils.replacePathInPDIfNeeded(rapidsConf, pd, - relation.sparkSession.sparkContext.hadoopConfiguration, - relation.sparkSession.conf) - } - res - } else { - origRet + if (AlluxioUtils.isAlluxioAutoMountTaskTime(rapidsConf, relation.fileFormat)) { + alluxioPathReplacementMap = AlluxioUtils.autoMountIfNeeded(rapidsConf, pds, + relation.sparkSession.sparkContext.hadoopConfiguration, + relation.sparkSession.conf) + } else if (AlluxioUtils.isAlluxioPathsToReplaceTaskTime(rapidsConf, relation.fileFormat)) { + // this is not ideal, here we check to see if we will replace any paths, which is an + // extra iteration through paths + alluxioPathReplacementMap = AlluxioUtils.checkIfNeedsReplaced(rapidsConf, pds) } logDebug(s"File listing and possibly replace with Alluxio path " + s"took: ${System.nanoTime() - startTime}") - setFilesNumAndSizeMetric(ret, true) + setFilesNumAndSizeMetric(pds, true) val timeTakenMs = NANOSECONDS.toMillis( (System.nanoTime() - startTime) + optimizerMetadataTimeNs) driverMetrics("metadataTime") = timeTakenMs - ret + pds }.toArray // We can only determine the actual partitions at runtime when a dynamic partition filter is @@ -183,7 +180,8 @@ case class GpuFileSourceScanExec( // exposed for testing lazy val bucketedScan: Boolean = { - if (relation.sparkSession.sessionState.conf.bucketingEnabled && relation.bucketSpec.isDefined + if (relation.sparkSession.sessionState.conf.bucketingEnabled + && relation.bucketSpec.isDefined && !disableBucketedScan) { val spec = relation.bucketSpec.get val bucketColumns = spec.bucketColumnNames.flatMap(n => toAttribute(n)) @@ -255,7 +253,8 @@ case class GpuFileSourceScanExec( @transient private lazy val pushedDownFilters = { - val supportNestedPredicatePushdown = DataSourceUtils.supportNestedPredicatePushdown(relation) + val supportNestedPredicatePushdown = + DataSourceUtils.supportNestedPredicatePushdown(relation) dataFilters.flatMap(DataSourceStrategy.translateFilter(_, supportNestedPredicatePushdown)) } @@ -345,7 +344,8 @@ case class GpuFileSourceScanExec( options = relation.options, hadoopConf = relation.sparkSession.sessionState.newHadoopConfWithOptions(relation.options), - metrics = allMetrics) + metrics = allMetrics, + alluxioPathReplacementMap) Some(reader) } else { None @@ -355,8 +355,7 @@ case class GpuFileSourceScanExec( createBucketedReadRDD(relation.bucketSpec.get, readFile, dynamicallySelectedPartitions, relation) } else { - createNonBucketedReadRDD(readFile, dynamicallySelectedPartitions, - relation) + createNonBucketedReadRDD(readFile, dynamicallySelectedPartitions, relation) } sendDriverMetrics() readRDD @@ -567,6 +566,8 @@ case class GpuFileSourceScanExec( SparkShimImpl.getFileScanRDD(relation.sparkSession, readFile.get, partitions, requiredSchema) } else { + logDebug(s"Using Datasource RDD, files are: " + + s"${partitions.flatMap(_.files).mkString(",")}") // note we use the v2 DataSourceRDD instead of FileScanRDD so we don't have to copy more code GpuDataSourceRDD(relation.sparkSession.sparkContext, partitions, readerFactory) } @@ -592,7 +593,8 @@ case class GpuFileSourceScanExec( pushedDownFilters.toArray, rapidsConf, allMetrics, - queryUsesInputFile) + queryUsesInputFile, + alluxioPathReplacementMap) case _: OrcFileFormat => GpuOrcMultiFilePartitionReaderFactory( sqlConf, @@ -633,7 +635,8 @@ case class GpuFileSourceScanExec( optionalNumCoalescedBuckets, QueryPlan.normalizePredicates(dataFilters, output), None, - queryUsesInputFile)(rapidsConf) + queryUsesInputFile, + alluxioPathsMap = alluxioPathsMap)(rapidsConf) } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala index e2f29233676..0386602de0a 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala @@ -45,7 +45,9 @@ class GpuReadAvroFileFormat extends AvroFileFormat with GpuReadFileFormatWithMet filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration, - metrics: Map[String, GpuMetric]): PartitionedFile => Iterator[InternalRow] = { + metrics: Map[String, GpuMetric], + alluxioPathReplacementMap: Option[Map[String, String]] = None) + : PartitionedFile => Iterator[InternalRow] = { val sqlConf = sparkSession.sessionState.conf val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuMultiFileReaderSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuMultiFileReaderSuite.scala index 14067d8a355..d40b33e7298 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuMultiFileReaderSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuMultiFileReaderSuite.scala @@ -34,7 +34,7 @@ class GpuMultiFileReaderSuite extends FunSuite with Arm { val membuffers = Array((HostMemoryBuffer.allocate(0), 0L)) val multiFileReader = new MultiFileCloudPartitionReaderBase( conf, - files = Array.empty, + inputFiles = Array.empty, numThreads = 1, maxNumFileProcessed = 1, filters = Array.empty, @@ -52,6 +52,7 @@ class GpuMultiFileReaderSuite extends FunSuite with Arm { override def getBatchRunner( tc: TaskContext, file: PartitionedFile, + origFile: Option[PartitionedFile], conf: Configuration, filters: Array[Filter]): Callable[HostMemoryBuffersWithMetaDataBase] = { () => null From 1893cef2c5c0b165596bf05338d51197c459321a Mon Sep 17 00:00:00 2001 From: Matt Ahrens Date: Thu, 29 Sep 2022 13:26:20 -0500 Subject: [PATCH 152/190] Changing toList to toIterator to improve memory optimization and runtime performance (#6651) Signed-off-by: Matt Ahrens Signed-off-by: Matt Ahrens --- .../main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala index ef60fe22787..6f14135857c 100644 --- a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala +++ b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala @@ -114,7 +114,7 @@ abstract class AppBase( val logFiles = reader.listEventLogFiles logFiles.foreach { file => Utils.tryWithResource(openEventLogInternal(file.getPath, fs)) { in => - val lines = Source.fromInputStream(in)(Codec.UTF8).getLines().toList + val lines = Source.fromInputStream(in)(Codec.UTF8).getLines().toIterator // Using find as foreach with conditional to exit early if we are done. // Do NOT use a while loop as it is much much slower. lines.find { line => From d849f96297b357df00ea7b9904b40f2d8769126e Mon Sep 17 00:00:00 2001 From: Niranjan Artal <50492963+nartal1@users.noreply.github.com> Date: Thu, 29 Sep 2022 11:38:22 -0700 Subject: [PATCH 153/190] Qualification tool: Print cluster usage tags to csv and log file (#6638) * Qualification tool: Print cluster usage tags to csv and log file * addressed review comments Signed-off-by: Niranjan --- .../tool/qualification/QualOutputWriter.scala | 66 ++++++++++++++++--- .../RunningQualificationApp.scala | 22 ++++++- .../spark/sql/rapids/tool/ToolUtils.scala | 26 ++++++++ .../qualification/QualificationAppInfo.scala | 25 +++++-- .../QualificationEventProcessor.scala | 7 ++ .../qualification/QualificationSuite.scala | 36 ++++++++++ 6 files changed, 164 insertions(+), 18 deletions(-) diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala index 65cb1206f0e..4e89610d11e 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.{Buffer, LinkedHashMap, ListBuffer} import com.nvidia.spark.rapids.tool.ToolTextFileWriter import com.nvidia.spark.rapids.tool.planparser.{ExecInfo, PlanInfo} import com.nvidia.spark.rapids.tool.profiling.ProfileUtils.replaceDelimiter -import com.nvidia.spark.rapids.tool.qualification.QualOutputWriter.TEXT_DELIMITER +import com.nvidia.spark.rapids.tool.qualification.QualOutputWriter.{CLUSTER_ID, CLUSTER_ID_STR_SIZE, JOB_ID, JOB_ID_STR_SIZE, RUN_NAME, RUN_NAME_STR_SIZE, TEXT_DELIMITER} import org.apache.spark.sql.rapids.tool.ToolUtils import org.apache.spark.sql.rapids.tool.qualification.{EstimatedPerSQLSummaryInfo, EstimatedSummaryInfo, QualificationAppInfo, QualificationSummaryInfo} @@ -186,8 +186,20 @@ class QualOutputWriter(outputDir: String, reportReadSchema: Boolean, sums.map(_.unSupportedExprs.size), QualOutputWriter.UNSUPPORTED_EXPRS_MAX_SIZE, QualOutputWriter.UNSUPPORTED_EXPRS.size) - val headersAndSizes = QualOutputWriter.getSummaryHeaderStringsAndSizes(sums, appIdMaxSize, - unSupExecMaxSize, unSupExprMaxSize) + val hasClusterTags = sums.exists(_.clusterTags.nonEmpty) + val (clusterIdMaxSize, jobIdMaxSize, runNameMaxSize) = if (hasClusterTags) { + (QualOutputWriter.getMaxSizeForHeader(sums.map(_.allClusterTagsMap.getOrElse( + CLUSTER_ID, "").size), QualOutputWriter.CLUSTER_ID), + QualOutputWriter.getMaxSizeForHeader(sums.map(_.allClusterTagsMap.getOrElse( + JOB_ID, "").size), QualOutputWriter.JOB_ID), + QualOutputWriter.getMaxSizeForHeader(sums.map(_.allClusterTagsMap.getOrElse( + RUN_NAME, "").size), QualOutputWriter.RUN_NAME)) + } else { + (CLUSTER_ID_STR_SIZE, JOB_ID_STR_SIZE, RUN_NAME_STR_SIZE) + } + val headersAndSizes = QualOutputWriter.getSummaryHeaderStringsAndSizes( + sums, appIdMaxSize, unSupExecMaxSize, unSupExprMaxSize, hasClusterTags, + clusterIdMaxSize, jobIdMaxSize, runNameMaxSize) val entireHeader = QualOutputWriter.constructOutputRowFromMap(headersAndSizes, TEXT_DELIMITER, true) val sep = "=" * (entireHeader.size - 1) @@ -204,7 +216,8 @@ class QualOutputWriter(outputDir: String, reportReadSchema: Boolean, val finalSums = estSum.take(numOutputRows) finalSums.foreach { sumInfo => val wStr = QualOutputWriter.constructAppSummaryInfo(sumInfo, headersAndSizes, - appIdMaxSize, unSupExecMaxSize, unSupExprMaxSize, TEXT_DELIMITER, true) + appIdMaxSize, unSupExecMaxSize, unSupExprMaxSize, hasClusterTags, clusterIdMaxSize, + jobIdMaxSize, runNameMaxSize, TEXT_DELIMITER, true) writer.write(wStr) if (printStdout) print(wStr) } @@ -239,7 +252,8 @@ case class FormattedQualificationSummaryInfo( taskSpeedupFactor: Double, endDurationEstimated: Boolean, unSupportedExecs: String, - unSupportedExprs: String) + unSupportedExprs: String, + clusterTags: Map[String, String]) object QualOutputWriter { val NON_SQL_TASK_DURATION_STR = "NonSQL Task Duration" @@ -284,6 +298,10 @@ object QualOutputWriter { val STAGE_ESTIMATED_STR = "Stage Estimated" val UNSUPPORTED_EXECS = "Unsupported Execs" val UNSUPPORTED_EXPRS = "Unsupported Expressions" + val CLUSTER_TAGS = "Cluster Tags" + val CLUSTER_ID = "ClusterId" + val JOB_ID = "JobId" + val RUN_NAME = "RunName" val APP_DUR_STR_SIZE: Int = APP_DUR_STR.size val SQL_DUR_STR_SIZE: Int = SQL_DUR_STR.size @@ -293,6 +311,9 @@ object QualOutputWriter { val GPU_OPPORTUNITY_STR_SIZE: Int = GPU_OPPORTUNITY_STR.size val UNSUPPORTED_EXECS_MAX_SIZE: Int = 25 val UNSUPPORTED_EXPRS_MAX_SIZE: Int = 25 + val CLUSTER_ID_STR_SIZE: Int = CLUSTER_ID.size + val JOB_ID_STR_SIZE: Int = JOB_ID.size + val RUN_NAME_STR_SIZE: Int = RUN_NAME.size val CSV_DELIMITER = "," val TEXT_DELIMITER = "|" @@ -329,7 +350,7 @@ object QualOutputWriter { Math.min(maxSQLDescLength, maxSizeOfDesc) } - private def getMaxSizeForHeader(sizes: Seq[Int], headerTxtStr: String): Int = { + def getMaxSizeForHeader(sizes: Seq[Int], headerTxtStr: String): Int = { if (sizes.size > 0 && sizes.max > headerTxtStr.size) { sizes.max } else { @@ -425,6 +446,10 @@ object QualOutputWriter { UNSUPPORTED_EXECS -> UNSUPPORTED_EXECS.size, UNSUPPORTED_EXPRS -> UNSUPPORTED_EXPRS.size ) + if (appInfos.exists(_.clusterTags.nonEmpty)) { + detailedHeadersAndFields += (CLUSTER_TAGS -> getMaxSizeForHeader( + appInfos.map(_.clusterTags.length), CLUSTER_TAGS)) + } if (reportReadSchema) { detailedHeadersAndFields += (READ_SCHEMA_STR -> @@ -437,8 +462,12 @@ object QualOutputWriter { appInfos: Seq[QualificationSummaryInfo], appIdMaxSize: Int, unSupExecMaxSize: Int = UNSUPPORTED_EXECS_MAX_SIZE, - unSupExprMaxSize: Int = UNSUPPORTED_EXPRS_MAX_SIZE): LinkedHashMap[String, Int] = { - LinkedHashMap[String, Int]( + unSupExprMaxSize: Int = UNSUPPORTED_EXPRS_MAX_SIZE, + hasClusterTags: Boolean = false, + clusterIdMaxSize: Int = CLUSTER_ID_STR_SIZE, + jobIdMaxSize: Int = JOB_ID_STR_SIZE, + runNameMaxSize: Int = RUN_NAME_STR_SIZE): LinkedHashMap[String, Int] = { + val data = LinkedHashMap[String, Int]( APP_NAME_STR -> getMaxSizeForHeader(appInfos.map(_.appName.size), APP_NAME_STR), APP_ID_STR -> appIdMaxSize, APP_DUR_STR -> APP_DUR_STR_SIZE, @@ -451,6 +480,12 @@ object QualOutputWriter { UNSUPPORTED_EXECS -> unSupExecMaxSize, UNSUPPORTED_EXPRS -> unSupExprMaxSize ) + if (hasClusterTags) { + data += (CLUSTER_ID -> clusterIdMaxSize) + data += (JOB_ID -> jobIdMaxSize) + data += (RUN_NAME -> runNameMaxSize) + } + data } def constructAppSummaryInfo( @@ -459,6 +494,10 @@ object QualOutputWriter { appIdMaxSize: Int, unSupExecMaxSize: Int, unSupExprMaxSize: Int, + hasClusterTags: Boolean, + clusterIdMaxSize: Int, + jobIdMaxSize: Int, + runNameMaxSize: Int, delimiter: String, prettyPrint: Boolean): String = { val data = ListBuffer[(String, Int)]( @@ -475,6 +514,11 @@ object QualOutputWriter { sumInfo.unsupportedExecs -> unSupExecMaxSize, sumInfo.unsupportedExprs -> unSupExprMaxSize ) + if (hasClusterTags) { + data += (sumInfo.allTagsMap.getOrElse(CLUSTER_ID, "") -> clusterIdMaxSize) + data += (sumInfo.allTagsMap.getOrElse(JOB_ID, "") -> jobIdMaxSize) + data += (sumInfo.allTagsMap.getOrElse(RUN_NAME, "") -> runNameMaxSize) + } constructOutputRow(data, delimiter, prettyPrint) } @@ -688,7 +732,8 @@ object QualOutputWriter { ToolUtils.truncateDoubleToTwoDecimal(appInfo.taskSpeedupFactor), appInfo.endDurationEstimated, appInfo.unSupportedExecs, - appInfo.unSupportedExprs + appInfo.unSupportedExprs, + appInfo.allClusterTagsMap ) } @@ -724,6 +769,9 @@ object QualOutputWriter { appInfo.unSupportedExecs.toString -> headersAndSizes(UNSUPPORTED_EXECS), appInfo.unSupportedExprs.toString -> headersAndSizes(UNSUPPORTED_EXPRS) ) + if (appInfo.clusterTags.nonEmpty) { + data += appInfo.clusterTags.mkString(";") -> headersAndSizes(CLUSTER_TAGS) + } if (reportReadSchema) { data += (stringIfempty(appInfo.readFileFormats) -> headersAndSizes(READ_SCHEMA_STR)) } diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualificationApp.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualificationApp.scala index ffd49ce65d9..f7f22ce635e 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualificationApp.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualificationApp.scala @@ -94,13 +94,29 @@ class RunningQualificationApp() extends QualificationAppInfo(None, None, Seq(info).map(_.unSupportedExprs.size), QualOutputWriter.UNSUPPORTED_EXPRS_MAX_SIZE, QualOutputWriter.UNSUPPORTED_EXPRS.size) + val hasClusterTags = info.clusterTags.nonEmpty + val (clusterIdMax, jobIdMax, runNameMax) = if (hasClusterTags) { + (QualOutputWriter.getMaxSizeForHeader(Seq(info).map( + _.allClusterTagsMap.getOrElse(QualOutputWriter.CLUSTER_ID, "").size), + QualOutputWriter.CLUSTER_ID), + QualOutputWriter.getMaxSizeForHeader(Seq(info).map( + _.allClusterTagsMap.getOrElse(QualOutputWriter.JOB_ID, "").size), + QualOutputWriter.JOB_ID), + QualOutputWriter.getMaxSizeForHeader(Seq(info).map( + _.allClusterTagsMap.getOrElse(QualOutputWriter.RUN_NAME, "").size), + QualOutputWriter.RUN_NAME)) + } else { + (QualOutputWriter.CLUSTER_ID_STR_SIZE, QualOutputWriter.JOB_ID_STR_SIZE, + QualOutputWriter.RUN_NAME_STR_SIZE) + } val headersAndSizes = QualOutputWriter.getSummaryHeaderStringsAndSizes(Seq(info), - appIdMaxSize, unSupExecMaxSize, unSupExprMaxSize) + appIdMaxSize, unSupExecMaxSize, unSupExprMaxSize, hasClusterTags, + clusterIdMax, jobIdMax, runNameMax) val headerStr = QualOutputWriter.constructOutputRowFromMap(headersAndSizes, delimiter, prettyPrint) val appInfoStr = QualOutputWriter.constructAppSummaryInfo(info.estimatedInfo, - headersAndSizes, appIdMaxSize, unSupExecMaxSize, unSupExprMaxSize, - delimiter, prettyPrint) + headersAndSizes, appIdMaxSize, unSupExecMaxSize, unSupExprMaxSize, hasClusterTags, + clusterIdMax, jobIdMax, runNameMax, delimiter, prettyPrint) headerStr + appInfoStr case None => logWarning(s"Unable to get qualification information for this application") diff --git a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/ToolUtils.scala b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/ToolUtils.scala index 1e7f91b2037..1f1d3e940c3 100644 --- a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/ToolUtils.scala +++ b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/ToolUtils.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.rapids.tool import com.nvidia.spark.rapids.tool.profiling.ProfileUtils.replaceDelimiter import com.nvidia.spark.rapids.tool.qualification.QualOutputWriter +import org.json4s.DefaultFormats +import org.json4s.jackson.JsonMethods.parse import org.apache.spark.internal.{config, Logging} import org.apache.spark.sql.DataFrame @@ -33,6 +35,30 @@ object ToolUtils extends Logging { df.showString(numRows, 0) } + /** + * Parses the string which contains configs in JSON format ( key : value ) pairs and + * returns the Map of [String, String] + * @param clusterTag String which contains property clusterUsageTags.clusterAllTags in + * JSON format + * @return Map of ClusterTags + */ + def parseClusterTags(clusterTag: String): Map[String, String] = { + // clusterTags will be in this format - + // [{"key":"Vendor","value":"Databricks"}, + // {"key":"Creator","value":"abc@company.com"},{"key":"ClusterName", + // "value":"job-215-run-1"},{"key":"ClusterId","value":"0617-131246-dray530"}, + // {"key":"JobId","value":"215"},{"key":"RunName","value":"test73longer"}, + // {"key":"DatabricksEnvironment","value":"workerenv-7026851462233806"}] + + // case class to hold key -> value pairs + case class ClusterTags(key: String, value: String) + implicit val formats = DefaultFormats + val listOfClusterTags = parse(clusterTag) + val clusterTagsMap = listOfClusterTags.extract[List[ClusterTags]].map( + x => x.key -> x.value).toMap + clusterTagsMap + } + // given to duration values, calculate a human readable percent // rounded to 2 decimal places. ie 39.12% def calculateDurationPercent(first: Long, total: Long): Double = { diff --git a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala index f8978b294b1..5d12790e401 100644 --- a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala +++ b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala @@ -56,6 +56,8 @@ class QualificationAppInfo( val notSupportFormatAndTypes: HashMap[String, Set[String]] = HashMap[String, Set[String]]() var sqlPlans: HashMap[Long, SparkPlanInfo] = HashMap.empty[Long, SparkPlanInfo] + var clusterTags: String = "" + private lazy val eventProcessor = new QualificationEventProcessor(this) /** @@ -364,6 +366,12 @@ class QualificationAppInfo( } val appName = appInfo.map(_.appName).getOrElse("") + + val allClusterTagsMap = if (clusterTags.nonEmpty) { + ToolUtils.parseClusterTags(clusterTags) + } else { + Map.empty[String, String] + } val perSqlInfos = if (reportSqlLevel) { Some(planInfos.flatMap { pInfo => sqlIdToInfo.get(pInfo.sqlID).map { info => @@ -424,7 +432,7 @@ class QualificationAppInfo( val estimatedInfo = QualificationAppInfo.calculateEstimatedInfoSummary(estimatedGPURatio, sparkSQLDFWallClockDuration, appDuration, taskSpeedupFactor, appName, appId, - sqlIdsWithFailures.nonEmpty, unSupportedExecs, unSupportedExprs) + sqlIdsWithFailures.nonEmpty, unSupportedExecs, unSupportedExprs, allClusterTagsMap) QualificationSummaryInfo(info.appName, appId, problems, executorCpuTimePercent, endDurationEstimated, sqlIdsWithFailures, @@ -433,7 +441,7 @@ class QualificationAppInfo( nonSQLTaskDuration, unsupportedSQLTaskDuration, supportedSQLTaskDuration, taskSpeedupFactor, info.sparkUser, info.startTime, origPlanInfos, perSqlStageSummary.map(_.stageSum).flatten, estimatedInfo, perSqlInfos, - unSupportedExecs, unSupportedExprs) + unSupportedExecs, unSupportedExprs, clusterTags, allClusterTagsMap) } } @@ -495,7 +503,8 @@ case class EstimatedSummaryInfo( estimatedGpuTimeSaved: Double, // app_duration - estimated_gpu_duration recommendation: String, unsupportedExecs: String, - unsupportedExprs: String) + unsupportedExprs: String, + allTagsMap: Map[String, String]) // Estimate based on wall clock times for each SQL query case class EstimatedPerSQLSummaryInfo( @@ -561,7 +570,9 @@ case class QualificationSummaryInfo( estimatedInfo: EstimatedSummaryInfo, perSQLEstimatedInfo: Option[Seq[EstimatedPerSQLSummaryInfo]], unSupportedExecs: String, - unSupportedExprs: String) + unSupportedExprs: String, + clusterTags: String, + allClusterTagsMap: Map[String, String]) case class StageQualSummaryInfo( stageId: Int, @@ -596,7 +607,8 @@ object QualificationAppInfo extends Logging { def calculateEstimatedInfoSummary(estimatedRatio: Double, sqlDataFrameDuration: Long, appDuration: Long, speedupFactor: Double, appName: String, appId: String, hasFailures: Boolean, unsupportedExecs: String = "", - unsupportedExprs: String = ""): EstimatedSummaryInfo = { + unsupportedExprs: String = "", + allClusterTagsMap: Map[String, String] = Map.empty[String, String]): EstimatedSummaryInfo = { val sqlDataFrameDurationToUse = if (sqlDataFrameDuration > appDuration) { // our app duration is shorter then our sql duration, estimate the sql duration down // to app duration @@ -627,7 +639,8 @@ object QualificationAppInfo extends Logging { estimated_gpu_timesaved, recommendation, unsupportedExecs, - unsupportedExprs) + unsupportedExprs, + allClusterTagsMap) } def createApp( diff --git a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationEventProcessor.scala b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationEventProcessor.scala index e245c444060..f5efaedda9f 100644 --- a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationEventProcessor.scala +++ b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationEventProcessor.scala @@ -39,6 +39,8 @@ class QualificationEventProcessor(app: QualificationAppInfo) if (ToolUtils.isPluginEnabled(sparkProperties)) { throw GpuEventLogException(s"Eventlog is from GPU run. Skipping ...") } + app.clusterTags = sparkProperties.getOrElse( + "spark.databricks.clusterUsageTags.clusterAllTags", "") } override def doSparkListenerApplicationStart( @@ -130,6 +132,11 @@ class QualificationEventProcessor(app: QualificationAppInfo) app.stageIdToSqlID.getOrElseUpdate(stageId, sqlID) } } + // If the confs are set after SparkSession initialization, it is captured in this event. + if (app.clusterTags.isEmpty) { + app.clusterTags = event.properties.getProperty( + "spark.databricks.clusterUsageTags.clusterAllTags", "") + } } override def doSparkListenerJobEnd( diff --git a/tools/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala b/tools/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala index 5136fd4ef05..408550f0e50 100644 --- a/tools/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala +++ b/tools/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala @@ -659,6 +659,42 @@ class QualificationSuite extends FunSuite with BeforeAndAfterEach with Logging { } } + test("test clusterTags configs ") { + TrampolineUtil.withTempDir { outpath => + TrampolineUtil.withTempDir { eventLogDir => + + val (eventLog, _) = ToolTestUtils.generateEventLog(eventLogDir, "clustertags") { spark => + import spark.implicits._ + spark.conf.set("spark.databricks.clusterUsageTags.clusterAllTags", + """[{"key":"Vendor", + |"value":"Databricks"},{"key":"Creator","value":"abc@company.com"}, + |{"key":"ClusterName","value":"job-215-run-1"},{"key":"ClusterId", + |"value":"0617-131246-dray530"},{"key":"JobId","value":"215"}, + |{"key":"RunName","value":"test73longer"},{"key":"DatabricksEnvironment", + |"value":"workerenv-7026851462233806"}]""".stripMargin) + + val df1 = spark.sparkContext.makeRDD(1 to 1000, 6).toDF + df1.sample(0.1) + } + val expectedClusterId = "0617-131246-dray530" + val expectedJobId = "215" + val expectedRunName = "test73longer" + + val allArgs = Array( + "--output-directory", + outpath.getAbsolutePath()) + val appArgs = new QualificationArgs(allArgs ++ Array(eventLog)) + val (exit, appSum) = QualificationMain.mainInternal(appArgs) + assert(exit == 0) + assert(appSum.size == 1) + val allTags = appSum.flatMap(_.allClusterTagsMap).toMap + assert(allTags("ClusterId") == expectedClusterId) + assert(allTags("JobId") == expectedJobId) + assert(allTags("RunName") == expectedRunName) + } + } + } + test("test read datasource v1") { val profileLogDir = ToolTestUtils.getTestResourcePath("spark-events-profiling") val logFiles = Array(s"$profileLogDir/eventlog_dsv1.zstd") From 0d4f5f9cf3fee123d4b1213e165314075f3134d1 Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Thu, 29 Sep 2022 14:35:11 -0500 Subject: [PATCH 154/190] Fix case where number of shuffle writer threads is set to 0 (#6641) * Fix case where number of shuffle writer threads is set to 0 Signed-off-by: Alessandro Bellina * Add thread count to shuffle mode log line Signed-off-by: Alessandro Bellina --- .../RapidsShuffleInternalManagerBase.scala | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala index ad8b792c01b..973aa431c28 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala @@ -1053,7 +1053,10 @@ class RapidsShuffleInternalManagerBase(conf: SparkConf, val isDriver: Boolean) if (rapidsConf.isCacheOnlyShuffleManagerMode) { "Transport disabled (local cached blocks only)" } else { - "Experimental threaded shuffle mode" + val numWriteThreads = rapidsConf.shuffleMultiThreadedWriterThreads + val numReadThreads = rapidsConf.shuffleMultiThreadedReaderThreads + s"Experimental threaded shuffle mode " + + s"(write threads=$numWriteThreads, read threads=$numReadThreads)" } } else { s"Transport enabled (remote fetches will use ${rapidsConf.shuffleTransportClassName}" @@ -1181,10 +1184,16 @@ class RapidsShuffleInternalManagerBase(conf: SparkConf, val isDriver: Boolean) gpu.dependency.metrics) case bmssh: BypassMergeSortShuffleHandle[_, _] => bmssh.dependency match { - case gpuDep: GpuShuffleDependency[_, _, _] if gpuDep.useMultiThreadedShuffle => - // cast the handle with specific generic types due to type-erasure + case gpuDep: GpuShuffleDependency[_, _, _] + if gpuDep.useMultiThreadedShuffle && + rapidsConf.shuffleMultiThreadedWriterThreads > 0 => + // use the threaded writer if the number of threads specified is 1 or above, + // with 0 threads we fallback to the Spark-provided writer. val handleWithMetrics = new ShuffleHandleWithMetrics( - bmssh.shuffleId, gpuDep.metrics, gpuDep.asInstanceOf[GpuShuffleDependency[K, V, V]]) + bmssh.shuffleId, + gpuDep.metrics, + // cast the handle with specific generic types due to type-erasure + gpuDep.asInstanceOf[GpuShuffleDependency[K, V, V]]) new RapidsShuffleThreadedWriter[K, V]( blockManager, handleWithMetrics, From 8e065537046e2350b75fb02a4892f028b1c5d165 Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Thu, 29 Sep 2022 17:18:56 -0500 Subject: [PATCH 155/190] Avoid failing test on cleanup when filesystem has issues (#6640) * Avoid failing test on cleanup when filesystem has issues Signed-off-by: Alessandro Bellina * Catch NonFatal exceptions only Signed-off-by: Alessandro Bellina --- .../RapidsShuffleThreadedWriterSuite.scala | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/tests/src/test/320+-noncdh-nondb/scala/org/apache/spark/sql/rapids/RapidsShuffleThreadedWriterSuite.scala b/tests/src/test/320+-noncdh-nondb/scala/org/apache/spark/sql/rapids/RapidsShuffleThreadedWriterSuite.scala index be32c4ffb7a..5217db41ed8 100644 --- a/tests/src/test/320+-noncdh-nondb/scala/org/apache/spark/sql/rapids/RapidsShuffleThreadedWriterSuite.scala +++ b/tests/src/test/320+-noncdh-nondb/scala/org/apache/spark/sql/rapids/RapidsShuffleThreadedWriterSuite.scala @@ -22,6 +22,7 @@ import java.util.zip.CheckedInputStream import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.util.control.NonFatal import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS @@ -32,7 +33,7 @@ import org.scalatest.mockito.MockitoSugar import org.apache.spark.{HashPartitioner, SparkConf, SparkException, TaskContext} import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} -import org.apache.spark.internal.config +import org.apache.spark.internal.{config, Logging} import org.apache.spark.network.shuffle.checksum.ShuffleChecksumHelper import org.apache.spark.network.util.LimitedInputStream import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} @@ -135,7 +136,8 @@ class RapidsShuffleThreadedWriterSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll with MockitoSugar - with ShuffleChecksumTestHelper { + with ShuffleChecksumTestHelper + with Logging { @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ @Mock(answer = RETURNS_SMART_NULLS) private var diskBlockManager: DiskBlockManager = _ @Mock(answer = RETURNS_SMART_NULLS) private var taskContext: TaskContext = _ @@ -239,10 +241,16 @@ class RapidsShuffleThreadedWriterSuite extends FunSuite override def afterEach(): Unit = { TaskContext.unset() + blockIdToFileMap.clear() + temporaryFilesCreated.clear() try { Utils.deleteRecursively(tempDir) - blockIdToFileMap.clear() - temporaryFilesCreated.clear() + } catch { + case NonFatal(e) => + // Catch non-fatal errors here as we are cleaning up directories using a Spark utility + // and we shouldn't fail a test for these exceptions. See: + // https://github.com/NVIDIA/spark-rapids/issues/6515 + logWarning(s"Error while cleaning up $tempDir", e) } finally { super.afterEach() } From ecb2b9b36e9d291a37ea8abec330c4ec1ffe8940 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 30 Sep 2022 08:21:36 -0500 Subject: [PATCH 156/190] Update iceberg doc for split size options (#6648) Signed-off-by: Thomas Graves Signed-off-by: Thomas Graves --- docs/additional-functionality/iceberg-support.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/additional-functionality/iceberg-support.md b/docs/additional-functionality/iceberg-support.md index f32a55be59f..77057d23cf6 100644 --- a/docs/additional-functionality/iceberg-support.md +++ b/docs/additional-functionality/iceberg-support.md @@ -56,6 +56,18 @@ The RAPIDS Accelerator does not support Apache Iceberg tables using the ORC data The RAPIDS Accelerator does not support Apache Iceberg tables using the Avro data format. + +### Reader Split Size + +The maximum number of bytes to pack into a single partition when reading files on Spark is normally +controlled by the config `spark.files.maxPartitionBytes`. But on Iceberg that doesn't apply. +Iceberg has its own configs to control the split size. See the read options in the + [Iceberg Runtime Configuration](https://iceberg.apache.org/docs/latest/spark-configuration/#runtime-configuration) +documentation for details. One example is to use the `split-size` reader option like: +```scala +spark.read.option("split-size", "24217728").table("someTable") +``` + ## Writing Tables The RAPIDS Accelerator for Apache Spark does not accelerate Apache Iceberg writes. Writes From 5664d0395c7b6756bb2522c84a10effc9fb857b6 Mon Sep 17 00:00:00 2001 From: Matt Ahrens Date: Fri, 30 Sep 2022 08:33:35 -0500 Subject: [PATCH 157/190] Workflow to add new issues to Github global project [skip ci] (#6643) * Testing out project workflow to add new issues Signed-off-by: Matt Ahrens * Updating workflow with release version Signed-off-by: Matt Ahrens * Updating project workflow for testing Signed-off-by: Matt Ahrens * Updating project workflow for testing Signed-off-by: Matt Ahrens * Updating project workflow for testing Signed-off-by: Matt Ahrens * Updating project workflow for prod merge Signed-off-by: Matt Ahrens Signed-off-by: Matt Ahrens --- .github/workflows/add-to-project.yml | 31 ++++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) create mode 100644 .github/workflows/add-to-project.yml diff --git a/.github/workflows/add-to-project.yml b/.github/workflows/add-to-project.yml new file mode 100644 index 00000000000..d24d9b212f3 --- /dev/null +++ b/.github/workflows/add-to-project.yml @@ -0,0 +1,31 @@ +# Copyright (c) 2022, NVIDIA CORPORATION. +# +# 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. + +name: Add new issues to project + +on: + issues: + types: + - opened + +jobs: + add-to-project: + if: github.repository == 'NVIDIA/spark-rapids' + name: Add new issues to project + runs-on: ubuntu-latest + steps: + - uses: actions/add-to-project@v0.3.0 + with: + project-url: https://github.com/orgs/NVIDIA/projects/4 + github-token: ${{ secrets.PROJECT_TOKEN }} From fade173ce7a12b75f95d85b4dd19e74ab6151ae9 Mon Sep 17 00:00:00 2001 From: Gary Shen Date: Sat, 1 Oct 2022 05:44:26 +0800 Subject: [PATCH 158/190] Sync up to commit 8e0655370 (#6662) Signed-off-by: Gary Shen Signed-off-by: Gary Shen --- scripts/spark2diffs/RapidsConf.diff | 10 +++---- .../com/nvidia/spark/rapids/RapidsConf.scala | 27 ++++++++++--------- 2 files changed, 20 insertions(+), 17 deletions(-) diff --git a/scripts/spark2diffs/RapidsConf.diff b/scripts/spark2diffs/RapidsConf.diff index 1ad0996bdc6..48708029fb0 100644 --- a/scripts/spark2diffs/RapidsConf.diff +++ b/scripts/spark2diffs/RapidsConf.diff @@ -17,11 +17,11 @@ < .createWithDefault(ByteUnit.MiB.toBytes(8).toLong) --- > .createWithDefault(ByteUnit.MiB.toBytes(8)) -1633c1635 +1636c1638 < |$SPARK_HOME/bin/spark-shell --jars rapids-4-spark_2.12-22.10.0-SNAPSHOT-cuda11.jar \ --- > |${SPARK_HOME}/bin/spark-shell --jars rapids-4-spark_2.12-22.10.0-SNAPSHOT-cuda11.jar \ -1688,1692c1690,1693 +1691,1695c1693,1696 < // scans not supported in Spark 2.x < // if (asTable) { < // printToggleHeader("Scans\n") @@ -32,12 +32,12 @@ > printToggleHeader("Scans\n") > } > GpuOverrides.scans.values.toSeq.sortBy(_.tag.toString).foreach(_.confHelp(asTable)) -1700c1701 +1703c1704 < // com.nvidia.spark.rapids.python.PythonConfEntries.init() --- > com.nvidia.spark.rapids.python.PythonConfEntries.init() -1776,1777d1776 +1779,1780d1779 < // Spark 2.x doesn't have access to Cuda in CUDF so just allow < /* -1793d1791 +1796d1794 < */ diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 818337c7c6a..b6e36ff79f5 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -1369,7 +1369,9 @@ object RapidsConf { "alluxio.master.rpc.port(default: 19998) from ALLUXIO_HOME/conf/alluxio-site.properties, " + "then replace a cloud path which matches spark.rapids.alluxio.bucket.regex like " + "\"s3://bar/b.csv\" to \"alluxio://0.1.2.3:19998/bar/b.csv\", " + - "and the bucket \"s3://bar\" will be mounted to \"/bar\" in Alluxio automatically.") + "and the bucket \"s3://bar\" will be mounted to \"/bar\" in Alluxio automatically." + + "This config should be enabled when initially starting the application but it " + + "can be turned off and one programmatically after that.") .booleanConf .createWithDefault(false) @@ -1398,15 +1400,16 @@ object RapidsConf { val ALLUXIO_REPLACEMENT_ALGO = conf("spark.rapids.alluxio.replacement.algo") .doc("The algorithm used when replacing the UFS path with the Alluxio path. CONVERT_TIME " + - "and SELECTION_TIME are the valid options. CONVERT_TIME indicates that we do it when " + - "we convert it to a GPU file read, this has extra overhead of creating an entirely new " + - "file index, which requires listing the files and getting all new file info from Alluxio. " + - "SELECTION_TIME indicates we do it when the file reader is selecting the partitions " + - "to process and just replaces the path without fetching the file information again, this " + - "is faster but doesn't update locality information if that were to work with Alluxio.") + "and TASK_TIME are the valid options. CONVERT_TIME indicates that we do it " + + "when we convert it to a GPU file read, this has extra overhead of creating an entirely " + + "new file index, which requires listing the files and getting all new file info from " + + "Alluxio. TASK_TIME replaces the path as late as possible inside of the task. " + + "By waiting and replacing it at task time, it just replaces " + + "the path without fetching the file information again, this is faster " + + "but doesn't update locality information if that has a bit impact on performance.") .stringConf - .checkValues(Set("CONVERT_TIME", "SELECTION_TIME")) - .createWithDefault("SELECTION_TIME") + .checkValues(Set("CONVERT_TIME", "TASK_TIME")) + .createWithDefault("TASK_TIME") // USER FACING DEBUG CONFIGS @@ -2113,12 +2116,12 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val getAlluxioReplacementAlgo: String = get(ALLUXIO_REPLACEMENT_ALGO) - lazy val isAlluxioReplacementAlgoSelectTime: Boolean = - get(ALLUXIO_REPLACEMENT_ALGO) == "SELECTION_TIME" - lazy val isAlluxioReplacementAlgoConvertTime: Boolean = get(ALLUXIO_REPLACEMENT_ALGO) == "CONVERT_TIME" + lazy val isAlluxioReplacementAlgoTaskTime: Boolean = + get(ALLUXIO_REPLACEMENT_ALGO) == "TASK_TIME" + lazy val driverTimeZone: Option[String] = get(DRIVER_TIMEZONE) lazy val isRangeWindowByteEnabled: Boolean = get(ENABLE_RANGE_WINDOW_BYTES) From d7966690ef1a1011f140e9423a129ff6deaa8be7 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 30 Sep 2022 16:49:17 -0500 Subject: [PATCH 159/190] Add Qualification tool support for running application - per sql output (#6657) * Adding running qualification event processor * Adding running qualification event processor Signed-off-by: Thomas Graves * Start writing output * in progress not working * more changes for writing output * fixes * update output functions * fixes * handle unknown events * fix processing other event * debugging * flush headers * change for local filesystem to use something where flush works * init writer * fix app writing * fix app reporting log file * debug dbfs * more debug dbfs * more * minor fixes * fix app summary * cleanup per sql reporting * start trying to clean running app * remove unused imports * Start cleaning up Signed-off-by: Thomas Graves * mkdir and permissions * fix up writing headers * put back the TaskStageAccumCase * call into the super for doSparkListenerSQLAdaptiveExecutionUpdate * don't store jobs if not related to sql * cleanup todo's * Add test for running qual app per sql * chagne way check reporting sql * add functions to get per sql headers * test headers * get rid of report per sql * fix compilation * update tests * update tests * text * add api to get sql ids * add test for cleanup * add more tests * move test * Update running qual test * formatting * fix style * beter handle writer failing * fix typo * change to get running spark app hadoop configuration * fix compile * add shutdown hook * start to cleanup based on num files * finish implementing rolling output files * toInt * cleanup files * fixes * fixes * cleanup logs * debug * change to cleanup only after defined * fix cleanup * fix * cleanup * documentation * add some references to output * update test * add test for rolling files * fix test * fix line length * change log level to warning * output header when logging * fixes * update test * fix test to not cleanup before checking * update copyright * Update docs/spark-qualification-tool.md Co-authored-by: Niranjan Artal <50492963+nartal1@users.noreply.github.com> * Update tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/RunningQualificationEventProcessor.scala Co-authored-by: Niranjan Artal <50492963+nartal1@users.noreply.github.com> Signed-off-by: Thomas Graves Signed-off-by: Thomas Graves Co-authored-by: Niranjan Artal <50492963+nartal1@users.noreply.github.com> --- docs/spark-qualification-tool.md | 70 ++++- .../rapids/tool/ToolTextFileWriter.scala | 44 ++- .../tool/planparser/SQLPlanParser.scala | 2 +- .../tool/profiling/CompareApplications.scala | 2 +- .../rapids/tool/profiling/GenerateDot.scala | 2 +- .../tool/qualification/QualOutputWriter.scala | 230 +++++++++------- .../tool/qualification/Qualification.scala | 4 +- .../RunningQualOutputWriter.scala | 90 +++++++ .../RunningQualificationApp.scala | 255 ++++++++++++++---- .../spark/sql/rapids/tool/AppBase.scala | 54 ++++ .../sql/rapids/tool/EventProcessorBase.scala | 21 +- .../tool/profiling/ApplicationInfo.scala | 9 +- .../tool/profiling/EventsProcessor.scala | 27 +- .../qualification/QualificationAppInfo.scala | 39 +-- .../QualificationEventProcessor.scala | 43 +-- .../RunningQualificationEventProcessor.scala | 254 +++++++++++++++++ .../qualification/QualificationSuite.scala | 111 +++++++- 17 files changed, 1021 insertions(+), 236 deletions(-) create mode 100644 tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualOutputWriter.scala create mode 100644 tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/RunningQualificationEventProcessor.scala diff --git a/docs/spark-qualification-tool.md b/docs/spark-qualification-tool.md index c3e0fa27126..2b3c829d993 100644 --- a/docs/spark-qualification-tool.md +++ b/docs/spark-qualification-tool.md @@ -31,9 +31,10 @@ This document covers below topics: ## How to use the Qualification tool -The Qualification tool can be run in two different ways. One is to run it as a standalone tool on the -Spark event logs after the application(s) have run and other is to be integrated into a running Spark -application. +The Qualification tool can be run in three different ways. One is to run it as a standalone tool on the +Spark event logs after the application(s) have run, the second is to be integrated into a running Spark +application using explicit API calls, and the third is to install a Spark listener which can output +results on a per SQL query basis. ## Running the Qualification tool standalone on Spark event logs @@ -301,7 +302,7 @@ For information on the files content and processing the Qualification report and to [Understanding the Qualification tool output](#understanding-the-qualification-tool-output) and [Output Formats](#output-formats) sections below. -## Running the Qualification tool inside a running Spark application +## Running the Qualification tool inside a running Spark application using the API ### Prerequisites - Java 8 or above, Spark 3.0.1+ @@ -384,6 +385,67 @@ For example, if running the spark-shell: $SPARK_HOME/bin/spark-shell --jars rapids-4-spark-tools_2.12-.jar ``` +## Running using a Spark Listener + +We provide a Spark Listener that can be installed at application start that will produce output +for each SQL queries in the running application and indicate if that query is a good fit to try +with the Rapids Accelerator for Spark. + +### Prerequisites +- Java 8 or above, Spark 3.0.1+ + +### Download the tools jar +- Download the jar file from [Maven repository](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark-tools_2.12/22.08.0/) + +### Configuration + +Add the RunningQualificationEventProcess to the spark listeners configuration: +`spark.extraListeners=org.apache.spark.sql.rapids.tool.qualification.RunningQualificationEventProcessor` + +The user should specify the output directory if they want the output to go to separate +files, otherwise it will go to the Spark driver log. If the output directory is specified, it outputs +two different files, one csv and one pretty printed log file. The output directory can be a local directory +or point to a distributed file system or blobstore like S3. + - `spark.rapids.qualification.outputDir` + +By default, this will output results for 10 SQL queries per file and will +keep 100 files. This behavior is because many blob stores don't show files until +they are fully written so you wouldn't be able to see the results for a running +application until it finishes the number of SQL queries per file. This behavior +can be configured with the following configs. + - `spark.rapids.qualification.output.numSQLQueriesPerFile` - default 10 + - `spark.rapids.qualification.output.maxNumFiles` - default 100 + +### Run the Spark application + +Run the application and include the tools jar, `spark.extraListeners` config and optionally the other +configs to control the tools behavior. + +For example: + +```bash +SPARK_HOME/bin/spark-shell \ +--jars rapids-4-spark-tools_2.12-.jar \ +--conf spark.extraListeners=org.apache.spark.sql.rapids.tool.qualification.RunningQualificationEventProcessor \ +--conf spark.rapids.qualification.outputDir=/tmp/qualPerSqlOutput \ +--conf spark.rapids.qualification.output.numSQLQueriesPerFile=5 \ +--conf spark.rapids.qualification.output.maxNumFiles=10 +``` + +After running some SQL queries you can look in the output directory and see files like: + +``` +rapids_4_spark_qualification_output_persql_0.csv +rapids_4_spark_qualification_output_persql_0.log +rapids_4_spark_qualification_output_persql_1.csv +rapids_4_spark_qualification_output_persql_1.log +rapids_4_spark_qualification_output_persql_2.csv +rapids_4_spark_qualification_output_persql_2.log +``` + +See the [Understanding the Qualification tool output](#understanding-the-qualification-tool-output) +section on the file contents details. + ## Understanding the Qualification tool output For each processed Spark application, the Qualification tool generates two main fields to help quantify the expected diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/ToolTextFileWriter.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/ToolTextFileWriter.scala index 83cf533b362..75d39fedcd7 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/ToolTextFileWriter.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/ToolTextFileWriter.scala @@ -15,30 +15,66 @@ */ package com.nvidia.spark.rapids.tool +import java.io.FileOutputStream + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} +import org.apache.hadoop.fs.permission.FsPermission import org.apache.spark.internal.Logging /** * Class for writing local files, allows writing to distributed file systems. */ -class ToolTextFileWriter(finalOutputDir: String, logFileName: String, - finalLocationText: String) extends Logging { +class ToolTextFileWriter( + finalOutputDir: String, + logFileName: String, + finalLocationText: String, + hadoopConf: Option[Configuration] = None) extends Logging { + // use same as Spark event log writer + val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("660", 8).toShort) + val LOG_FOLDER_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) private val textOutputPath = new Path(s"$finalOutputDir/$logFileName") - private val fs = FileSystem.get(textOutputPath.toUri, new Configuration()) + private val hadoopConfToUse = hadoopConf.getOrElse(new Configuration()) + + private val defaultFs = FileSystem.getDefaultUri(hadoopConfToUse).getScheme + private val isDefaultLocal = defaultFs == null || defaultFs == "file" + private val uri = textOutputPath.toUri + + def getFileOutputPath: Path = textOutputPath + + // The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). + // Therefore, for local files, use FileOutputStream instead. // this overwrites existing path - private var outFile: Option[FSDataOutputStream] = Some(fs.create(textOutputPath)) + private var outFile: Option[FSDataOutputStream] = { + val fs = FileSystem.get(uri, hadoopConfToUse) + val outStream = if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") { + FileSystem.mkdirs(fs, new Path(finalOutputDir), LOG_FOLDER_PERMISSIONS) + Some(new FSDataOutputStream(new FileOutputStream(uri.getPath), null)) + } else { + Some(fs.create(textOutputPath)) + } + fs.setPermission(textOutputPath, LOG_FILE_PERMISSIONS) + outStream + } def write(stringToWrite: String): Unit = { outFile.foreach(_.writeBytes(stringToWrite)) } + def flush(): Unit = { + outFile.foreach { file => + file.flush() + file.hflush() + } + } + def close(): Unit = { outFile.foreach { file => logInfo(s"$finalLocationText output location: $textOutputPath") file.flush() + file.hflush() file.close() outFile = None } diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala index 02d5daa6279..97872805ebc 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala @@ -263,7 +263,7 @@ object SQLPlanParser extends Logging { None } case _ => logDebug(s"Incorrect expression - $expr") - None + None } funcName } diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/CompareApplications.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/CompareApplications.scala index 389c8618248..7c8ef4e2570 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/CompareApplications.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/CompareApplications.scala @@ -31,7 +31,7 @@ class CompareApplications(apps: Seq[ApplicationInfo]) extends Logging { def findMatchingStages(): (Seq[CompareProfileResults], Seq[CompareProfileResults]) = { val normalizedByAppId = apps.map { app => - val normalized = app.sqlPlan.mapValues { plan => + val normalized = app.sqlPlans.mapValues { plan => SparkPlanInfoWithStage(plan, app.accumIdToStageId).normalizeForStageComparison } (app.appId, normalized) diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateDot.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateDot.scala index e9cd5216270..e81b97be529 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateDot.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateDot.scala @@ -110,7 +110,7 @@ object GenerateDot { list += row(1) -> row(2) } - val sqlPlansMap = app.sqlPlan.map { case (sqlId, sparkPlanInfo) => + val sqlPlansMap = app.sqlPlans.map { case (sqlId, sparkPlanInfo) => sqlId -> ((sparkPlanInfo, app.physicalPlanDescription(sqlId))) } for ((sqlID, (planInfo, physicalPlan)) <- sqlPlansMap) { diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala index 4e89610d11e..3d62a150782 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala @@ -22,6 +22,7 @@ import com.nvidia.spark.rapids.tool.ToolTextFileWriter import com.nvidia.spark.rapids.tool.planparser.{ExecInfo, PlanInfo} import com.nvidia.spark.rapids.tool.profiling.ProfileUtils.replaceDelimiter import com.nvidia.spark.rapids.tool.qualification.QualOutputWriter.{CLUSTER_ID, CLUSTER_ID_STR_SIZE, JOB_ID, JOB_ID_STR_SIZE, RUN_NAME, RUN_NAME_STR_SIZE, TEXT_DELIMITER} +import org.apache.hadoop.conf.Configuration import org.apache.spark.sql.rapids.tool.ToolUtils import org.apache.spark.sql.rapids.tool.qualification.{EstimatedPerSQLSummaryInfo, EstimatedSummaryInfo, QualificationAppInfo, QualificationSummaryInfo} @@ -33,32 +34,102 @@ import org.apache.spark.sql.rapids.tool.qualification.{EstimatedPerSQLSummaryInf * @param reportReadSchema Whether to include the read data source schema in csv output * @param printStdout Indicates if the summary report should be printed to stdout as well * @param prettyPrintOrder The order in which to print the Text output + * @param hadoopConf Optional Hadoop Configuration to use */ class QualOutputWriter(outputDir: String, reportReadSchema: Boolean, - printStdout: Boolean, prettyPrintOrder: String) { + printStdout: Boolean, prettyPrintOrder: String, + hadoopConf: Option[Configuration] = None) { - // a file extension will be added to this later - private val logFileName = "rapids_4_spark_qualification_output" - - def writeDetailedReport(sums: Seq[QualificationSummaryInfo]): Unit = { - val csvFileWriter = new ToolTextFileWriter(outputDir, s"${logFileName}.csv", "CSV") + def writeDetailedCSVReport(sums: Seq[QualificationSummaryInfo]): Unit = { + val csvFileWriter = new ToolTextFileWriter(outputDir, + s"${QualOutputWriter.LOGFILE_NAME}.csv", "CSV", hadoopConf) try { - val headersAndSizes = QualOutputWriter.getDetailedHeaderStringsAndSizes(sums, - reportReadSchema) - csvFileWriter.write(QualOutputWriter.constructDetailedHeader(headersAndSizes, - QualOutputWriter.CSV_DELIMITER, false)) - sums.foreach { sum => - csvFileWriter.write(QualOutputWriter.constructAppDetailedInfo(sum, headersAndSizes, - QualOutputWriter.CSV_DELIMITER, false, reportReadSchema)) - } + writeDetailedCSVReport(csvFileWriter, sums) } finally { csvFileWriter.close() } } + protected def writeDetailedCSVReport(csvFileWriter: ToolTextFileWriter, + sums: Seq[QualificationSummaryInfo]): Unit = { + val headersAndSizes = QualOutputWriter.getDetailedHeaderStringsAndSizes(sums, + reportReadSchema) + csvFileWriter.write(QualOutputWriter.constructDetailedHeader(headersAndSizes, + QualOutputWriter.CSV_DELIMITER, false)) + sums.foreach { sum => + csvFileWriter.write(QualOutputWriter.constructAppDetailedInfo(sum, headersAndSizes, + QualOutputWriter.CSV_DELIMITER, false, reportReadSchema)) + } + } + + // write the text summary report + def writeTextReport(sums: Seq[QualificationSummaryInfo], estSums: Seq[EstimatedSummaryInfo], + numOutputRows: Int) : Unit = { + val textFileWriter = new ToolTextFileWriter(outputDir, s"${QualOutputWriter.LOGFILE_NAME}.log", + "Summary Report", hadoopConf) + try { + writeTextReport(textFileWriter, sums, estSums, numOutputRows) + } finally { + textFileWriter.close() + } + } + + protected def writeTextReport(writer: ToolTextFileWriter, + sums: Seq[QualificationSummaryInfo], estSum: Seq[EstimatedSummaryInfo], + numOutputRows: Int): Unit = { + val appIdMaxSize = QualOutputWriter.getAppIdSize(sums) + val unSupExecMaxSize = QualOutputWriter.getunSupportedMaxSize( + sums.map(_.unSupportedExecs.size), + QualOutputWriter.UNSUPPORTED_EXECS_MAX_SIZE, + QualOutputWriter.UNSUPPORTED_EXECS.size) + val unSupExprMaxSize = QualOutputWriter.getunSupportedMaxSize( + sums.map(_.unSupportedExprs.size), + QualOutputWriter.UNSUPPORTED_EXPRS_MAX_SIZE, + QualOutputWriter.UNSUPPORTED_EXPRS.size) + val appNameMaxSize = QualOutputWriter.getAppNameSize(sums) + val hasClusterTags = sums.exists(_.clusterTags.nonEmpty) + val (clusterIdMaxSize, jobIdMaxSize, runNameMaxSize) = if (hasClusterTags) { + (QualOutputWriter.getMaxSizeForHeader(sums.map(_.allClusterTagsMap.getOrElse( + CLUSTER_ID, "").size), QualOutputWriter.CLUSTER_ID), + QualOutputWriter.getMaxSizeForHeader(sums.map(_.allClusterTagsMap.getOrElse( + JOB_ID, "").size), QualOutputWriter.JOB_ID), + QualOutputWriter.getMaxSizeForHeader(sums.map(_.allClusterTagsMap.getOrElse( + RUN_NAME, "").size), QualOutputWriter.RUN_NAME)) + } else { + (CLUSTER_ID_STR_SIZE, JOB_ID_STR_SIZE, RUN_NAME_STR_SIZE) + } + val headersAndSizes = QualOutputWriter.getSummaryHeaderStringsAndSizes( + appNameMaxSize, appIdMaxSize, unSupExecMaxSize, unSupExprMaxSize, hasClusterTags, + clusterIdMaxSize, jobIdMaxSize, runNameMaxSize) + val entireHeader = QualOutputWriter.constructOutputRowFromMap(headersAndSizes, + TEXT_DELIMITER, true) + val sep = "=" * (entireHeader.size - 1) + writer.write(s"$sep\n") + writer.write(entireHeader) + writer.write(s"$sep\n") + // write to stdout as well + if (printStdout) { + print("APPLICATION SUMMARY:\n") + print(s"$sep\n") + print(entireHeader) + print(s"$sep\n") + } + val finalSums = estSum.take(numOutputRows) + finalSums.foreach { sumInfo => + val wStr = QualOutputWriter.constructAppSummaryInfo(sumInfo, headersAndSizes, + appIdMaxSize, unSupExecMaxSize, unSupExprMaxSize, hasClusterTags, clusterIdMaxSize, + jobIdMaxSize, runNameMaxSize, TEXT_DELIMITER, true) + writer.write(wStr) + if (printStdout) print(wStr) + } + writer.write(s"$sep\n") + if (printStdout) print(s"$sep\n") + } + def writeStageReport(sums: Seq[QualificationSummaryInfo], order: String) : Unit = { - val csvFileWriter = new ToolTextFileWriter(outputDir, s"${logFileName}_stages.csv", - "Stage Exec Info") + val csvFileWriter = new ToolTextFileWriter(outputDir, + s"${QualOutputWriter.LOGFILE_NAME}_stages.csv", + "Stage Exec Info", hadoopConf) try { val headersAndSizes = QualOutputWriter .getDetailedStagesHeaderStringsAndSizes(sums) @@ -72,14 +143,17 @@ class QualOutputWriter(outputDir: String, reportReadSchema: Boolean, } } - def writePerSqlCSVReport(sums: Seq[QualificationSummaryInfo], maxSQLDescLength: Int) : Unit = { - val csvFileWriter = new ToolTextFileWriter(outputDir, s"${logFileName}_persql.csv", - "Per SQL CSV Report") + def writePerSqlCSVReport(sums: Seq[QualificationSummaryInfo], maxSQLDescLength: Int): Unit = { + val csvFileWriter = new ToolTextFileWriter(outputDir, + s"${QualOutputWriter.LOGFILE_NAME}_persql.csv", + "Per SQL CSV Report", hadoopConf) try { - val plans = sums.flatMap(_.planInfo) - val allExecs = QualOutputWriter.getAllExecsFromPlan(plans) - val headersAndSizes = QualOutputWriter.getDetailedPerSqlHeaderStringsAndSizes(sums, - maxSQLDescLength, QualOutputWriter.CSV_DELIMITER) + val appNameSize = QualOutputWriter.getAppNameSize(sums) + val appIdSize = QualOutputWriter.getAppIdSize(sums) + val sqlDescSize = + QualOutputWriter.getSqlDescSize(sums, maxSQLDescLength, QualOutputWriter.CSV_DELIMITER) + val headersAndSizes = + QualOutputWriter.getDetailedPerSqlHeaderStringsAndSizes(appNameSize, appIdSize, sqlDescSize) csvFileWriter.write(QualOutputWriter.constructDetailedHeader(headersAndSizes, QualOutputWriter.CSV_DELIMITER, false)) val appIdMaxSize = QualOutputWriter.getAppIdSize(sums) @@ -96,9 +170,12 @@ class QualOutputWriter(outputDir: String, reportReadSchema: Boolean, private def writePerSqlTextSummary(writer: ToolTextFileWriter, sums: Seq[QualificationSummaryInfo], numOutputRows: Int, maxSQLDescLength: Int): Unit = { - val appIdMaxSize = QualOutputWriter.getAppIdSize(sums) - val headersAndSizes = QualOutputWriter.getDetailedPerSqlHeaderStringsAndSizes(sums, - maxSQLDescLength, TEXT_DELIMITER) + val appNameSize = QualOutputWriter.getAppNameSize(sums) + val appIdSize = QualOutputWriter.getAppIdSize(sums) + val sqlDescSize = + QualOutputWriter.getSqlDescSize(sums, maxSQLDescLength, QualOutputWriter.TEXT_DELIMITER) + val headersAndSizes = + QualOutputWriter.getDetailedPerSqlHeaderStringsAndSizes(appNameSize, appIdSize, sqlDescSize) val entireHeader = QualOutputWriter.constructOutputRowFromMap(headersAndSizes, TEXT_DELIMITER, true) val sep = "=" * (entireHeader.size - 1) @@ -125,7 +202,7 @@ class QualOutputWriter(outputDir: String, reportReadSchema: Boolean, val finalSums = sorted.take(numOutputRows) finalSums.foreach { estInfo => val wStr = QualOutputWriter.constructPerSqlSummaryInfo(estInfo, headersAndSizes, - appIdMaxSize, TEXT_DELIMITER, true, maxSQLDescLength) + appIdSize, TEXT_DELIMITER, true, maxSQLDescLength) writer.write(wStr) if (printStdout) print(wStr) } @@ -135,8 +212,9 @@ class QualOutputWriter(outputDir: String, reportReadSchema: Boolean, def writePerSqlTextReport(sums: Seq[QualificationSummaryInfo], numOutputRows: Int, maxSQLDescLength: Int) : Unit = { - val textFileWriter = new ToolTextFileWriter(outputDir, s"${logFileName}_persql.log", - "Per SQL Summary Report") + val textFileWriter = new ToolTextFileWriter(outputDir, + s"${QualOutputWriter.LOGFILE_NAME}_persql.log", + "Per SQL Summary Report", hadoopConf) try { writePerSqlTextSummary(textFileWriter, sums, numOutputRows, maxSQLDescLength) } finally { @@ -145,8 +223,9 @@ class QualOutputWriter(outputDir: String, reportReadSchema: Boolean, } def writeExecReport(sums: Seq[QualificationSummaryInfo], order: String) : Unit = { - val csvFileWriter = new ToolTextFileWriter(outputDir, s"${logFileName}_execs.csv", - "Plan Exec Info") + val csvFileWriter = new ToolTextFileWriter(outputDir, + s"${QualOutputWriter.LOGFILE_NAME}_execs.csv", + "Plan Exec Info", hadoopConf) try { val plans = sums.flatMap(_.planInfo) val allExecs = QualOutputWriter.getAllExecsFromPlan(plans) @@ -161,69 +240,6 @@ class QualOutputWriter(outputDir: String, reportReadSchema: Boolean, csvFileWriter.close() } } - - // write the text summary report - def writeReport(sums: Seq[QualificationSummaryInfo], estSums: Seq[EstimatedSummaryInfo], - numOutputRows: Int) : Unit = { - val textFileWriter = new ToolTextFileWriter(outputDir, s"${logFileName}.log", - "Summary Report") - try { - writeTextSummary(textFileWriter, sums, estSums, numOutputRows) - } finally { - textFileWriter.close() - } - } - - private def writeTextSummary(writer: ToolTextFileWriter, - sums: Seq[QualificationSummaryInfo], estSum: Seq[EstimatedSummaryInfo], - numOutputRows: Int): Unit = { - val appIdMaxSize = QualOutputWriter.getAppIdSize(sums) - val unSupExecMaxSize = QualOutputWriter.getunSupportedMaxSize( - sums.map(_.unSupportedExecs.size), - QualOutputWriter.UNSUPPORTED_EXECS_MAX_SIZE, - QualOutputWriter.UNSUPPORTED_EXECS.size) - val unSupExprMaxSize = QualOutputWriter.getunSupportedMaxSize( - sums.map(_.unSupportedExprs.size), - QualOutputWriter.UNSUPPORTED_EXPRS_MAX_SIZE, - QualOutputWriter.UNSUPPORTED_EXPRS.size) - val hasClusterTags = sums.exists(_.clusterTags.nonEmpty) - val (clusterIdMaxSize, jobIdMaxSize, runNameMaxSize) = if (hasClusterTags) { - (QualOutputWriter.getMaxSizeForHeader(sums.map(_.allClusterTagsMap.getOrElse( - CLUSTER_ID, "").size), QualOutputWriter.CLUSTER_ID), - QualOutputWriter.getMaxSizeForHeader(sums.map(_.allClusterTagsMap.getOrElse( - JOB_ID, "").size), QualOutputWriter.JOB_ID), - QualOutputWriter.getMaxSizeForHeader(sums.map(_.allClusterTagsMap.getOrElse( - RUN_NAME, "").size), QualOutputWriter.RUN_NAME)) - } else { - (CLUSTER_ID_STR_SIZE, JOB_ID_STR_SIZE, RUN_NAME_STR_SIZE) - } - val headersAndSizes = QualOutputWriter.getSummaryHeaderStringsAndSizes( - sums, appIdMaxSize, unSupExecMaxSize, unSupExprMaxSize, hasClusterTags, - clusterIdMaxSize, jobIdMaxSize, runNameMaxSize) - val entireHeader = QualOutputWriter.constructOutputRowFromMap(headersAndSizes, - TEXT_DELIMITER, true) - val sep = "=" * (entireHeader.size - 1) - writer.write(s"$sep\n") - writer.write(entireHeader) - writer.write(s"$sep\n") - // write to stdout as well - if (printStdout) { - print("APPLICATION SUMMARY:\n") - print(s"$sep\n") - print(entireHeader) - print(s"$sep\n") - } - val finalSums = estSum.take(numOutputRows) - finalSums.foreach { sumInfo => - val wStr = QualOutputWriter.constructAppSummaryInfo(sumInfo, headersAndSizes, - appIdMaxSize, unSupExecMaxSize, unSupExprMaxSize, hasClusterTags, clusterIdMaxSize, - jobIdMaxSize, runNameMaxSize, TEXT_DELIMITER, true) - writer.write(wStr) - if (printStdout) print(wStr) - } - writer.write(s"$sep\n") - if (printStdout) print(s"$sep\n") - } } case class FormattedQualificationSummaryInfo( @@ -318,11 +334,18 @@ object QualOutputWriter { val CSV_DELIMITER = "," val TEXT_DELIMITER = "|" + // a file extension will be added to this later + val LOGFILE_NAME = "rapids_4_spark_qualification_output" + def getAppIdSize(sums: Seq[QualificationSummaryInfo]): Int = { val sizes = sums.map(_.appId.size) getMaxSizeForHeader(sizes, QualOutputWriter.APP_ID_STR) } + def getAppNameSize(sums: Seq[QualificationSummaryInfo]): Int = { + getMaxSizeForHeader(sums.map(_.appName.size), APP_NAME_STR) + } + def getunSupportedMaxSize(unSupExecs: Seq[Int], maxStringSize: Int, headerSize: Int): Int = { val unSupportedExecsSize = unSupExecs.size val unSupportedExecsMax = if (unSupExecs.nonEmpty) { @@ -459,7 +482,7 @@ object QualOutputWriter { } private[qualification] def getSummaryHeaderStringsAndSizes( - appInfos: Seq[QualificationSummaryInfo], + appNameMaxSize: Int, appIdMaxSize: Int, unSupExecMaxSize: Int = UNSUPPORTED_EXECS_MAX_SIZE, unSupExprMaxSize: Int = UNSUPPORTED_EXPRS_MAX_SIZE, @@ -468,7 +491,7 @@ object QualOutputWriter { jobIdMaxSize: Int = JOB_ID_STR_SIZE, runNameMaxSize: Int = RUN_NAME_STR_SIZE): LinkedHashMap[String, Int] = { val data = LinkedHashMap[String, Int]( - APP_NAME_STR -> getMaxSizeForHeader(appInfos.map(_.appName.size), APP_NAME_STR), + APP_NAME_STR -> appNameMaxSize, APP_ID_STR -> appIdMaxSize, APP_DUR_STR -> APP_DUR_STR_SIZE, SQL_DUR_STR -> SQL_DUR_STR_SIZE, @@ -537,13 +560,14 @@ object QualOutputWriter { execInfos.map(_.children.getOrElse(Seq.empty).map(_.nodeId).mkString(",").size) } def getDetailedPerSqlHeaderStringsAndSizes( - appInfos: Seq[QualificationSummaryInfo], - maxSQLDescLength: Int, delimiter: String): LinkedHashMap[String, Int] = { + appMaxNameSize: Int, + appMaxIdSize: Int, + sqlDescLength: Int): LinkedHashMap[String, Int] = { val detailedHeadersAndFields = LinkedHashMap[String, Int]( - APP_NAME_STR -> getMaxSizeForHeader(appInfos.map(_.appName.size), APP_NAME_STR), - APP_ID_STR -> QualOutputWriter.getAppIdSize(appInfos), + APP_NAME_STR -> appMaxNameSize, + APP_ID_STR -> appMaxIdSize, SQL_ID_STR -> SQL_ID_STR.size, - SQL_DESC_STR -> QualOutputWriter.getSqlDescSize(appInfos, maxSQLDescLength, delimiter), + SQL_DESC_STR -> sqlDescLength, SQL_DUR_STR -> SQL_DUR_STR_SIZE, GPU_OPPORTUNITY_STR -> GPU_OPPORTUNITY_STR_SIZE, ESTIMATED_GPU_DURATION -> ESTIMATED_GPU_DURATION.size, @@ -766,8 +790,8 @@ object QualOutputWriter { appInfo.supportedSQLTaskDuration.toString -> headersAndSizes(SUPPORTED_SQL_TASK_DURATION_STR), appInfo.taskSpeedupFactor.toString -> headersAndSizes(SPEEDUP_FACTOR_STR), appInfo.endDurationEstimated.toString -> headersAndSizes(APP_DUR_ESTIMATED_STR), - appInfo.unSupportedExecs.toString -> headersAndSizes(UNSUPPORTED_EXECS), - appInfo.unSupportedExprs.toString -> headersAndSizes(UNSUPPORTED_EXPRS) + appInfo.unSupportedExecs -> headersAndSizes(UNSUPPORTED_EXECS), + appInfo.unSupportedExprs -> headersAndSizes(UNSUPPORTED_EXPRS) ) if (appInfo.clusterTags.nonEmpty) { data += appInfo.clusterTags.mkString(";") -> headersAndSizes(CLUSTER_TAGS) diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala index 730e7778f73..2d7409e14ba 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala @@ -77,8 +77,8 @@ class Qualification(outputDir: String, numRows: Int, hadoopConf: Configuration, // sort order and limit only applies to the report summary text file, // the csv file we write the entire data in descending order val sortedDescDetailed = sortDescForDetailedReport(allAppsSum) - qWriter.writeReport(allAppsSum, sortForExecutiveSummary(sortedDescDetailed, order), numRows) - qWriter.writeDetailedReport(sortedDescDetailed) + qWriter.writeTextReport(allAppsSum, sortForExecutiveSummary(sortedDescDetailed, order), numRows) + qWriter.writeDetailedCSVReport(sortedDescDetailed) if (reportSqlLevel) { qWriter.writePerSqlTextReport(allAppsSum, numRows, maxSQLDescLength) qWriter.writePerSqlCSVReport(allAppsSum, maxSQLDescLength) diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualOutputWriter.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualOutputWriter.scala new file mode 100644 index 00000000000..8e235033492 --- /dev/null +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualOutputWriter.scala @@ -0,0 +1,90 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.tool.qualification + +import com.nvidia.spark.rapids.tool.ToolTextFileWriter +import com.nvidia.spark.rapids.tool.qualification.QualOutputWriter.TEXT_DELIMITER +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +/** + * This class handles writing output to files for a running qualification app. + * Currently this only supports writing per sql output, not the entire application + * qualification, since doing an entire application may use a lot of memory if its + * long running. + * + * @param appId The id of the application + * @param appName The name of the application + * @param outputDir The directory to output the files to + * @param hadoopConf Optional Hadoop Configuration to use + * @param fileNameSuffix A suffix to add to the output per sql filenames + */ +class RunningQualOutputWriter( + appId: String, + appName: String, + outputDir: String, + hadoopConf: Option[Configuration] = None, + fileNameSuffix: String = "") + extends QualOutputWriter(outputDir, reportReadSchema=false, printStdout=false, + prettyPrintOrder = "desc", hadoopConf) { + + private lazy val csvPerSQLFileWriter = new ToolTextFileWriter(outputDir, + s"${QualOutputWriter.LOGFILE_NAME}_persql_$fileNameSuffix.csv", + "Per SQL CSV Report", hadoopConf) + private lazy val textPerSQLFileWriter = new ToolTextFileWriter(outputDir, + s"${QualOutputWriter.LOGFILE_NAME}_persql_$fileNameSuffix.log", + "Per SQL Summary Report", hadoopConf) + + // we don't know max length since process per query, hardcode for 100 for now + private val SQL_DESC_LENGTH = 100 + private val appNameSize = if (appName.nonEmpty) appName.size else 100 + val headersAndSizes = QualOutputWriter.getDetailedPerSqlHeaderStringsAndSizes(appNameSize, + appId.size, SQL_DESC_LENGTH) + val entireTextHeader = QualOutputWriter.constructOutputRowFromMap(headersAndSizes, + TEXT_DELIMITER, true) + private val sep = "=" * (entireTextHeader.size - 1) + + def getOutputFileNames: Seq[Path] = { + Seq(csvPerSQLFileWriter.getFileOutputPath, textPerSQLFileWriter.getFileOutputPath) + } + + def init(): Unit = { + csvPerSQLFileWriter.write(QualOutputWriter.constructDetailedHeader(headersAndSizes, + QualOutputWriter.CSV_DELIMITER, false)) + textPerSQLFileWriter.write(s"$sep\n") + textPerSQLFileWriter.write(entireTextHeader) + textPerSQLFileWriter.write(s"$sep\n") + csvPerSQLFileWriter.flush() + textPerSQLFileWriter.flush() + } + + def close(): Unit = { + csvPerSQLFileWriter.close() + textPerSQLFileWriter.write(s"$sep\n") + textPerSQLFileWriter.close() + } + + def writePerSqlCSVReport(sqlInfo: String): Unit = { + csvPerSQLFileWriter.write(sqlInfo) + csvPerSQLFileWriter.flush() + } + + def writePerSqlTextReport(sqlInfo: String): Unit = { + textPerSQLFileWriter.write(sqlInfo) + textPerSQLFileWriter.flush() + } +} diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualificationApp.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualificationApp.scala index f7f22ce635e..16322b56a73 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualificationApp.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualificationApp.scala @@ -16,6 +16,9 @@ package com.nvidia.spark.rapids.tool.qualification +import com.nvidia.spark.rapids.tool.planparser.SQLPlanParser +import com.nvidia.spark.rapids.tool.qualification.QualOutputWriter.SQL_DESC_STR + import org.apache.spark.SparkEnv import org.apache.spark.sql.rapids.tool.qualification._ @@ -30,6 +33,11 @@ import org.apache.spark.sql.rapids.tool.qualification._ * like this on a single application, the detailed output may be most useful to look * for potential issues and time spent in Dataframe operations. * + * Please note that this will use additional memory so use with caution if using with a + * long running application. The perSqlOnly option will allow reporting at the per + * SQL query level without tracking all the Application information, but currently does + * not cleanup. There is a cleanupSQL function that the user can force cleanup if required. + * * Create the `RunningQualicationApp`: * {{{ * val qualApp = new com.nvidia.spark.rapids.tool.qualification.RunningQualificationApp() @@ -41,16 +49,44 @@ import org.apache.spark.sql.rapids.tool.qualification._ * spark.sparkContext.addSparkListener(listener) * }}} * - * Run your queries and then get the summary or detailed output to see the results. + * Run your queries and then get the Application summary or detailed output to see the results. * {{{ * // run your sql queries ... * val summaryOutput = qualApp.getSummary() * val detailedOutput = qualApp.getDetailed() * }}} * + * If wanting per sql query output, run your queries and then get the output you are interested in. + * {{{ + * // run your sql queries ... + * val csvHeader = qualApp.getPerSqlCSVHeader + * val txtHeader = qualApp.getPerSqlTextHeader + * val (csvOut, txtOut) = qualApp.getPerSqlTextAndCSVSummary(sqlID) + * // print header and output wherever its useful + * }}} + * + * @param perSqlOnly allows reporting at the SQL query level and doesn't track + * the entire application */ -class RunningQualificationApp() extends QualificationAppInfo(None, None, - new PluginTypeChecker(), reportSqlLevel = false) { +class RunningQualificationApp( + perSqlOnly: Boolean = false, + pluginTypeChecker: PluginTypeChecker = new PluginTypeChecker()) + extends QualificationAppInfo(None, None, pluginTypeChecker, reportSqlLevel=false, perSqlOnly) { + // note we don't use the per sql reporting providing by QualificationAppInfo so we always + // send down false for it + + // we don't know the max sql query name size so lets cap it at 100 + private val SQL_DESC_LENGTH = 100 + private lazy val appName = appInfo.map(_.appName).getOrElse("") + private lazy val appNameSize = if (appName.nonEmpty) appName.size else 100 + private lazy val perSqlHeadersAndSizes = { + QualOutputWriter.getDetailedPerSqlHeaderStringsAndSizes(appNameSize, + appId.size, SQL_DESC_LENGTH) + } + + def this() = { + this(false) + } // since application is running, try to initialize current state private def initApp(): Unit = { @@ -58,7 +94,7 @@ class RunningQualificationApp() extends QualificationAppInfo(None, None, val appIdConf = SparkEnv.get.conf.getOption("spark.app.id") val appStartTime = SparkEnv.get.conf.get("spark.app.startTime", "-1") - // start event doesn't happen so initial it + // start event doesn't happen so initialize it val thisAppInfo = QualApplicationInfo( appName, appIdConf, @@ -75,55 +111,137 @@ class RunningQualificationApp() extends QualificationAppInfo(None, None, initApp() /** - * Get the summary report for qualification. + * Get the IDs of the SQL queries currently being tracked. + * @return a sequence of SQL IDs + */ + def getAvailableSqlIDs: Seq[Long] = { + sqlIdToInfo.keys.toSeq + } + + /** + * Get the per SQL query header in CSV format. + * @return a string with the header + */ + def getPerSqlCSVHeader: String = { + QualOutputWriter.constructDetailedHeader(perSqlHeadersAndSizes, + QualOutputWriter.CSV_DELIMITER, false) + } + + /** + * Get the per SQL query header in TXT format for pretty printing. + * @return a string with the header + */ + def getPerSqlTextHeader: String = { + QualOutputWriter.constructDetailedHeader(perSqlHeadersAndSizes, + QualOutputWriter.TEXT_DELIMITER, true) + } + + /** + * Get the per SQL query header. + * @return a string with the header + */ + def getPerSqlHeader(delimiter: String, + prettyPrint: Boolean, sqlDescLength: Int = SQL_DESC_LENGTH): String = { + perSqlHeadersAndSizes(SQL_DESC_STR) = sqlDescLength + QualOutputWriter.constructDetailedHeader(perSqlHeadersAndSizes, delimiter, prettyPrint) + } + + /** + * Get the per SQL query summary report in both Text and CSV format. + * @param sqlID The sqlID of the query. + * @return a tuple of the CSV summary followed by the Text summary. + */ + def getPerSqlTextAndCSVSummary(sqlID: Long): (String, String) = { + val sqlInfo = aggregatePerSQLStats(sqlID) + val csvResult = constructPerSqlResult(sqlInfo, QualOutputWriter.CSV_DELIMITER, false) + val textResult = constructPerSqlResult(sqlInfo, QualOutputWriter.TEXT_DELIMITER, true) + (csvResult, textResult) + } + + /** + * Get the per SQL query summary report for qualification for the specified sqlID. + * @param sqlID The sqlID of the query. * @param delimiter The delimiter separating fields of the summary report. * @param prettyPrint Whether to include the delimiter at start and end and * add spacing so the data rows align with column headings. - * @return String containing the summary report. + * @param sqlDescLength Maximum length to use for the SQL query description. + * @return String containing the summary report, or empty string if its not available. */ - def getSummary(delimiter: String = "|", prettyPrint: Boolean = true): String = { - val appInfo = super.aggregateStats() - appInfo match { + def getPerSQLSummary(sqlID: Long, delimiter: String = "|", + prettyPrint: Boolean = true, sqlDescLength: Int = SQL_DESC_LENGTH): String = { + val sqlInfo = aggregatePerSQLStats(sqlID) + constructPerSqlResult(sqlInfo, delimiter, prettyPrint, sqlDescLength) + } + + private def constructPerSqlResult( + sqlInfo: Option[EstimatedPerSQLSummaryInfo], + delimiter: String = "|", + prettyPrint: Boolean = true, + sqlDescLength: Int = SQL_DESC_LENGTH): String = { + sqlInfo match { case Some(info) => - val appIdMaxSize = QualOutputWriter.getAppIdSize(Seq(info)) - val unSupExecMaxSize = QualOutputWriter.getunSupportedMaxSize( - Seq(info).map(_.unSupportedExecs.size), - QualOutputWriter.UNSUPPORTED_EXECS_MAX_SIZE, - QualOutputWriter.UNSUPPORTED_EXECS.size) - val unSupExprMaxSize = QualOutputWriter.getunSupportedMaxSize( - Seq(info).map(_.unSupportedExprs.size), - QualOutputWriter.UNSUPPORTED_EXPRS_MAX_SIZE, - QualOutputWriter.UNSUPPORTED_EXPRS.size) - val hasClusterTags = info.clusterTags.nonEmpty - val (clusterIdMax, jobIdMax, runNameMax) = if (hasClusterTags) { - (QualOutputWriter.getMaxSizeForHeader(Seq(info).map( - _.allClusterTagsMap.getOrElse(QualOutputWriter.CLUSTER_ID, "").size), - QualOutputWriter.CLUSTER_ID), - QualOutputWriter.getMaxSizeForHeader(Seq(info).map( - _.allClusterTagsMap.getOrElse(QualOutputWriter.JOB_ID, "").size), - QualOutputWriter.JOB_ID), - QualOutputWriter.getMaxSizeForHeader(Seq(info).map( - _.allClusterTagsMap.getOrElse(QualOutputWriter.RUN_NAME, "").size), - QualOutputWriter.RUN_NAME)) - } else { - (QualOutputWriter.CLUSTER_ID_STR_SIZE, QualOutputWriter.JOB_ID_STR_SIZE, - QualOutputWriter.RUN_NAME_STR_SIZE) - } - val headersAndSizes = QualOutputWriter.getSummaryHeaderStringsAndSizes(Seq(info), - appIdMaxSize, unSupExecMaxSize, unSupExprMaxSize, hasClusterTags, - clusterIdMax, jobIdMax, runNameMax) - val headerStr = QualOutputWriter.constructOutputRowFromMap(headersAndSizes, - delimiter, prettyPrint) - val appInfoStr = QualOutputWriter.constructAppSummaryInfo(info.estimatedInfo, - headersAndSizes, appIdMaxSize, unSupExecMaxSize, unSupExprMaxSize, hasClusterTags, - clusterIdMax, jobIdMax, runNameMax, delimiter, prettyPrint) - headerStr + appInfoStr + perSqlHeadersAndSizes(SQL_DESC_STR) = sqlDescLength + QualOutputWriter.constructPerSqlSummaryInfo(info, perSqlHeadersAndSizes, + appId.size, delimiter, prettyPrint, sqlDescLength) case None => logWarning(s"Unable to get qualification information for this application") "" } } + /** + * Get the summary report for qualification. + * @param delimiter The delimiter separating fields of the summary report. + * @param prettyPrint Whether to include the delimiter at start and end and + * add spacing so the data rows align with column headings. + * @return String containing the summary report. + */ + def getSummary(delimiter: String = "|", prettyPrint: Boolean = true): String = { + if (!perSqlOnly) { + val appInfo = super.aggregateStats() + appInfo match { + case Some(info) => + val unSupExecMaxSize = QualOutputWriter.getunSupportedMaxSize( + Seq(info).map(_.unSupportedExecs.size), + QualOutputWriter.UNSUPPORTED_EXECS_MAX_SIZE, + QualOutputWriter.UNSUPPORTED_EXECS.size) + val unSupExprMaxSize = QualOutputWriter.getunSupportedMaxSize( + Seq(info).map(_.unSupportedExprs.size), + QualOutputWriter.UNSUPPORTED_EXPRS_MAX_SIZE, + QualOutputWriter.UNSUPPORTED_EXPRS.size) + val hasClusterTags = info.clusterTags.nonEmpty + val (clusterIdMax, jobIdMax, runNameMax) = if (hasClusterTags) { + (QualOutputWriter.getMaxSizeForHeader(Seq(info).map( + _.allClusterTagsMap.getOrElse(QualOutputWriter.CLUSTER_ID, "").size), + QualOutputWriter.CLUSTER_ID), + QualOutputWriter.getMaxSizeForHeader(Seq(info).map( + _.allClusterTagsMap.getOrElse(QualOutputWriter.JOB_ID, "").size), + QualOutputWriter.JOB_ID), + QualOutputWriter.getMaxSizeForHeader(Seq(info).map( + _.allClusterTagsMap.getOrElse(QualOutputWriter.RUN_NAME, "").size), + QualOutputWriter.RUN_NAME)) + } else { + (QualOutputWriter.CLUSTER_ID_STR_SIZE, QualOutputWriter.JOB_ID_STR_SIZE, + QualOutputWriter.RUN_NAME_STR_SIZE) + } + val appHeadersAndSizes = QualOutputWriter.getSummaryHeaderStringsAndSizes( + appName.size, info.appId.size, unSupExecMaxSize, unSupExprMaxSize, + hasClusterTags, clusterIdMax, jobIdMax, runNameMax) + val headerStr = QualOutputWriter.constructOutputRowFromMap(appHeadersAndSizes, + delimiter, prettyPrint) + val appInfoStr = QualOutputWriter.constructAppSummaryInfo(info.estimatedInfo, + appHeadersAndSizes, appId.size, unSupExecMaxSize, unSupExprMaxSize, hasClusterTags, + clusterIdMax, jobIdMax, runNameMax, delimiter, prettyPrint) + headerStr + appInfoStr + case None => + logWarning(s"Unable to get qualification information for this application") + "" + } + } else { + "" + } + } + /** * Get the detailed report for qualification. * @param delimiter The delimiter separating fields of the summary report. @@ -133,19 +251,46 @@ class RunningQualificationApp() extends QualificationAppInfo(None, None, */ def getDetailed(delimiter: String = "|", prettyPrint: Boolean = true, reportReadSchema: Boolean = false): String = { - val appInfo = super.aggregateStats() - appInfo match { - case Some(info) => - val headersAndSizes = - QualOutputWriter.getDetailedHeaderStringsAndSizes(Seq(info),reportReadSchema ) - val headerStr = QualOutputWriter.constructDetailedHeader(headersAndSizes, - delimiter, prettyPrint) - val appInfoStr = QualOutputWriter.constructAppDetailedInfo(info, headersAndSizes, delimiter, - prettyPrint, reportReadSchema) - headerStr + appInfoStr - case None => - logWarning(s"Unable to get qualification information for this application") - "" + if (!perSqlOnly) { + val appInfo = super.aggregateStats() + appInfo match { + case Some(info) => + val headersAndSizesToUse = + QualOutputWriter.getDetailedHeaderStringsAndSizes(Seq(info), reportReadSchema) + val headerStr = QualOutputWriter.constructDetailedHeader(headersAndSizesToUse, + delimiter, prettyPrint) + val appInfoStr = QualOutputWriter.constructAppDetailedInfo(info, headersAndSizesToUse, + delimiter, prettyPrint, reportReadSchema) + headerStr + appInfoStr + case None => + logWarning(s"Unable to get qualification information for this application") + "" + } + } else { + "" + } + } + + // don't aggregate at app level, just sql level + private def aggregatePerSQLStats(sqlID: Long): Option[EstimatedPerSQLSummaryInfo] = { + val sqlDesc = sqlIdToInfo.get(sqlID).map(_.description) + val origPlanInfo = sqlPlans.get(sqlID).map { plan => + SQLPlanParser.parseSQLPlan(appId, plan, sqlID, sqlDesc.getOrElse(""), pluginTypeChecker, this) + } + val perSqlInfos = origPlanInfo.flatMap { pInfo => + // filter out any execs that should be removed + val planInfos = removeExecsShouldRemove(Seq(pInfo)) + // get a summary of each SQL Query + val perSqlStageSummary = summarizeSQLStageInfo(planInfos) + sqlIdToInfo.get(pInfo.sqlID).map { sqlInfo => + val wallClockDur = sqlInfo.duration.getOrElse(0L) + // get task duration ratio + val sqlStageSums = perSqlStageSummary.filter(_.sqlID == pInfo.sqlID) + val estimatedInfo = getPerSQLWallClockSummary(sqlStageSums, wallClockDur, + sqlIDtoFailures.get(pInfo.sqlID).nonEmpty, appName) + EstimatedPerSQLSummaryInfo(pInfo.sqlID, pInfo.sqlDesc, estimatedInfo) + } } + perSqlInfos } } diff --git a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala index 6f14135857c..8dcfa5a4b29 100644 --- a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala +++ b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala @@ -54,6 +54,8 @@ abstract class AppBase( val sqlIDtoProblematic: HashMap[Long, Set[String]] = HashMap[Long, Set[String]]() // sqlId to sql info val sqlIdToInfo = new HashMap[Long, SQLExecutionInfoClass]() + // sqlPlans stores HashMap (sqlID <-> SparkPlanInfo) + var sqlPlans: HashMap[Long, SparkPlanInfo] = HashMap.empty[Long, SparkPlanInfo] // accum id to task stage accum info var taskStageAccumMap: HashMap[Long, ArrayBuffer[TaskStageAccumCase]] = @@ -73,6 +75,58 @@ abstract class AppBase( stage } + def getAllStagesForJobsInSqlQuery(sqlID: Long): Seq[Int] = { + val jobsIdsInSQLQuery = jobIdToSqlID.filter { case (_, sqlIdForJob) => + sqlIdForJob == sqlID + }.keys.toSeq + jobsIdsInSQLQuery.flatMap { jId => + jobIdToInfo.get(jId).map(_.stageIds) + }.flatten + } + + def cleanupAccumId(accId: Long): Unit = { + taskStageAccumMap.remove(accId) + driverAccumMap.remove(accId) + accumulatorToStages.remove(accId) + } + + def cleanupStages(stageIds: Set[Int]): Unit = { + // stageIdToInfo can have multiple stage attempts, remove all of them + stageIds.foreach { stageId => + val toRemove = stageIdToInfo.keys.filter(_._1 == stageId) + toRemove.foreach(stageIdToInfo.remove(_)) + } + } + + def cleanupSQL(sqlID: Long): Unit = { + sqlIDToDataSetOrRDDCase.remove(sqlID) + sqlIDtoProblematic.remove(sqlID) + sqlIdToInfo.remove(sqlID) + sqlPlans.remove(sqlID) + val dsToRemove = dataSourceInfo.filter(_.sqlID == sqlID) + dsToRemove.foreach(dataSourceInfo -= _) + + val jobsInSql = jobIdToSqlID.filter { case (_, sqlIdForJob) => + sqlIdForJob == sqlID + }.keys + jobsInSql.foreach { jobId => + // must call cleanupStage first + // clean when no other jobs need that stage + // not sure about races here but lets check the jobs and assume we can clean + // when none of them reference this stage + val stagesNotInOtherJobs = jobIdToInfo.get(jobId).map { jInfo => + val stagesInJobToRemove = jInfo.stageIds.toSet + // check to make sure no other jobs reference the same stage + val allOtherJobs = jobIdToInfo - jobId + val allOtherStageIds = allOtherJobs.values.flatMap(_.stageIds).toSet + stagesInJobToRemove.filter(!allOtherStageIds.contains(_)) + } + stagesNotInOtherJobs.foreach(cleanupStages(_)) + jobIdToSqlID.remove(_) + jobIdToInfo.remove(_) + } + } + def processEvent(event: SparkListenerEvent): Boolean private def openEventLogInternal(log: Path, fs: FileSystem): InputStream = { diff --git a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/EventProcessorBase.scala b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/EventProcessorBase.scala index 66b020e7fbe..ed6146e0016 100644 --- a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/EventProcessorBase.scala +++ b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/EventProcessorBase.scala @@ -16,7 +16,6 @@ package org.apache.spark.sql.rapids.tool -import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal @@ -139,6 +138,7 @@ abstract class EventProcessorBase[T <: AppBase](app: T) extends SparkListener wi "" ) app.sqlIdToInfo.put(event.executionId, sqlExecution) + app.sqlPlans += (event.executionId -> event.sparkPlanInfo) } def doSparkListenerSQLExecutionEnd( @@ -167,7 +167,10 @@ abstract class EventProcessorBase[T <: AppBase](app: T) extends SparkListener wi def doSparkListenerSQLAdaptiveExecutionUpdate( app: T, - event: SparkListenerSQLAdaptiveExecutionUpdate): Unit = {} + event: SparkListenerSQLAdaptiveExecutionUpdate): Unit = { + // AQE plan can override the ones got from SparkListenerSQLExecutionStart + app.sqlPlans += (event.executionId -> event.sparkPlanInfo) + } def doSparkListenerSQLAdaptiveSQLMetricUpdates( app: T, @@ -304,20 +307,6 @@ abstract class EventProcessorBase[T <: AppBase](app: T) extends SparkListener wi logDebug("Processing event: " + event.getClass) val sqlIDString = event.properties.getProperty("spark.sql.execution.id") val sqlID = ProfileUtils.stringToLong(sqlIDString) - // add jobInfoClass - val thisJob = new JobInfoClass( - event.jobId, - event.stageIds, - sqlID, - event.properties.asScala, - event.time, - None, - None, - None, - None, - ProfileUtils.isPluginEnabled(event.properties.asScala) || app.gpuMode - ) - app.jobIdToInfo.put(event.jobId, thisJob) sqlID.foreach(app.jobIdToSqlID(event.jobId) = _) } diff --git a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/ApplicationInfo.scala b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/ApplicationInfo.scala index 61be7da78dc..2faf89d0ea5 100644 --- a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/ApplicationInfo.scala +++ b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/ApplicationInfo.scala @@ -206,9 +206,6 @@ class ApplicationInfo( var appInfo: ApplicationCase = null var appId: String = "" - // sqlPlan stores HashMap (sqlID <-> SparkPlanInfo) - var sqlPlan: mutable.HashMap[Long, SparkPlanInfo] = mutable.HashMap.empty[Long, SparkPlanInfo] - // physicalPlanDescription stores HashMap (sqlID <-> physicalPlanDescription) var physicalPlanDescription: mutable.HashMap[Long, String] = mutable.HashMap.empty[Long, String] @@ -243,7 +240,7 @@ class ApplicationInfo( // Connects Operators to Stages using AccumulatorIDs def connectOperatorToStage(): Unit = { - for ((sqlId, planInfo) <- sqlPlan) { + for ((sqlId, planInfo) <- sqlPlans) { val planGraph = SparkPlanGraph(planInfo) // Maps stages to operators by checking for non-zero intersection // between nodeMetrics and stageAccumulateIDs @@ -260,7 +257,7 @@ class ApplicationInfo( */ def processSQLPlanMetrics(): Unit = { connectOperatorToStage() - for ((sqlID, planInfo) <- sqlPlan) { + for ((sqlID, planInfo) <- sqlPlans) { checkMetadataForReadSchema(sqlID, planInfo) val planGraph = SparkPlanGraph(planInfo) // SQLPlanMetric is a case Class of @@ -344,7 +341,7 @@ class ApplicationInfo( val nodeIds = sqlPlanNodeIdToStageIds.filter { case (_, v) => v.contains(s) }.keys.toSeq - val nodeNames = sqlPlan.get(j.sqlID.get).map { planInfo => + val nodeNames = sqlPlans.get(j.sqlID.get).map { planInfo => val nodes = SparkPlanGraph(planInfo).allNodes val validNodes = nodes.filter { n => nodeIds.contains((j.sqlID.get, n.id)) diff --git a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/EventsProcessor.scala b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/EventsProcessor.scala index 72fcbfdbd7b..49bcdf26454 100644 --- a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/EventsProcessor.scala +++ b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/EventsProcessor.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.rapids.tool.profiling import java.util.concurrent.TimeUnit.NANOSECONDS +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal @@ -35,6 +36,29 @@ import org.apache.spark.sql.rapids.tool.EventProcessorBase class EventsProcessor(app: ApplicationInfo) extends EventProcessorBase[ApplicationInfo](app) with Logging { + override def doSparkListenerJobStart( + app: ApplicationInfo, + event: SparkListenerJobStart): Unit = { + logDebug("Processing event: " + event.getClass) + super.doSparkListenerJobStart(app, event) + val sqlIDString = event.properties.getProperty("spark.sql.execution.id") + val sqlID = ProfileUtils.stringToLong(sqlIDString) + // add jobInfoClass + val thisJob = new JobInfoClass( + event.jobId, + event.stageIds, + sqlID, + event.properties.asScala, + event.time, + None, + None, + None, + None, + ProfileUtils.isPluginEnabled(event.properties.asScala) || app.gpuMode + ) + app.jobIdToInfo.put(event.jobId, thisJob) + } + override def doSparkListenerResourceProfileAddedReflect( app: ApplicationInfo, event: SparkListenerEvent): Boolean = { @@ -243,7 +267,6 @@ class EventsProcessor(app: ApplicationInfo) extends EventProcessorBase[Applicati app: ApplicationInfo, event: SparkListenerSQLExecutionStart): Unit = { super.doSparkListenerSQLExecutionStart(app, event) - app.sqlPlan += (event.executionId -> event.sparkPlanInfo) app.physicalPlanDescription += (event.executionId -> event.physicalPlanDescription) } @@ -287,8 +310,8 @@ class EventsProcessor(app: ApplicationInfo) extends EventProcessorBase[Applicati event: SparkListenerSQLAdaptiveExecutionUpdate): Unit = { logDebug("Processing event: " + event.getClass) // AQE plan can override the ones got from SparkListenerSQLExecutionStart - app.sqlPlan += (event.executionId -> event.sparkPlanInfo) app.physicalPlanDescription += (event.executionId -> event.physicalPlanDescription) + super.doSparkListenerSQLAdaptiveExecutionUpdate(app, event) } override def doSparkListenerSQLAdaptiveSQLMetricUpdates( diff --git a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala index 5d12790e401..b5a8db0d227 100644 --- a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala +++ b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala @@ -34,7 +34,8 @@ class QualificationAppInfo( eventLogInfo: Option[EventLogInfo], hadoopConf: Option[Configuration] = None, pluginTypeChecker: PluginTypeChecker, - reportSqlLevel: Boolean) + reportSqlLevel: Boolean, + perSqlOnly: Boolean = false) extends AppBase(eventLogInfo, hadoopConf) with Logging { var appId: String = "" @@ -43,7 +44,6 @@ class QualificationAppInfo( val writeDataFormat: ArrayBuffer[String] = ArrayBuffer[String]() var appInfo: Option[QualApplicationInfo] = None - val sqlStart: HashMap[Long, QualSQLExecutionInfo] = HashMap[Long, QualSQLExecutionInfo]() val sqlIDToTaskEndSum: HashMap[Long, StageTaskQualificationSummary] = HashMap.empty[Long, StageTaskQualificationSummary] @@ -54,11 +54,9 @@ class QualificationAppInfo( val sqlIDtoFailures: HashMap[Long, ArrayBuffer[String]] = HashMap.empty[Long, ArrayBuffer[String]] val notSupportFormatAndTypes: HashMap[String, Set[String]] = HashMap[String, Set[String]]() - var sqlPlans: HashMap[Long, SparkPlanInfo] = HashMap.empty[Long, SparkPlanInfo] var clusterTags: String = "" - - private lazy val eventProcessor = new QualificationEventProcessor(this) + private lazy val eventProcessor = new QualificationEventProcessor(this, perSqlOnly) /** * Get the event listener the qualification tool uses to process Spark events. @@ -80,6 +78,20 @@ class QualificationAppInfo( false } + override def cleanupStages(stageIds: Set[Int]): Unit = { + stageIds.foreach { stageId => + stageIdToTaskEndSum.remove(stageId) + stageIdToSqlID.remove(stageId) + } + super.cleanupStages(stageIds) + } + + override def cleanupSQL(sqlID: Long): Unit = { + sqlIDToTaskEndSum.remove(sqlID) + sqlIDtoFailures.remove(sqlID) + super.cleanupSQL(sqlID) + } + // time in ms private def calculateAppDuration(startTime: Long): Option[Long] = { if (startTime > 0) { @@ -163,10 +175,10 @@ class QualificationAppInfo( } } - private def checkUnsupportedReadFormats(): Unit = { + protected def checkUnsupportedReadFormats(): Unit = { if (dataSourceInfo.size > 0) { dataSourceInfo.map { ds => - val (readScore, nsTypes) = pluginTypeChecker.scoreReadDataTypes(ds.format, ds.schema) + val (_, nsTypes) = pluginTypeChecker.scoreReadDataTypes(ds.format, ds.schema) if (nsTypes.nonEmpty) { val currentFormat = notSupportFormatAndTypes.get(ds.format).getOrElse(Set.empty[String]) notSupportFormatAndTypes(ds.format) = (currentFormat ++ nsTypes) @@ -207,15 +219,6 @@ class QualificationAppInfo( } } - private def getAllStagesForJobsInSqlQuery(sqlID: Long): Seq[Int] = { - val jobsIdsInSQLQuery = jobIdToSqlID.filter { case (_, sqlIdForJob) => - sqlIdForJob == sqlID - }.keys.toSeq - jobsIdsInSQLQuery.flatMap { jId => - jobIdToInfo(jId).stageIds - } - } - private def stagesSummary(execInfos: Seq[ExecInfo], stages: Seq[Int], estimated: Boolean): Set[StageQualSummaryInfo] = { val allStageTaskTime = stages.map { stageId => @@ -477,7 +480,7 @@ class QualificationAppInfo( sqlIDtoProblematic(sqlID) = existingIssues ++ issues } // Get the write data format - if (node.name.contains("InsertIntoHadoopFsRelationCommand")) { + if (!perSqlOnly && node.name.contains("InsertIntoHadoopFsRelationCommand")) { val writeFormat = node.desc.split(",")(2) writeDataFormat += writeFormat } @@ -650,7 +653,7 @@ object QualificationAppInfo extends Logging { reportSqlLevel: Boolean): Option[QualificationAppInfo] = { val app = try { val app = new QualificationAppInfo(Some(path), Some(hadoopConf), pluginTypeChecker, - reportSqlLevel) + reportSqlLevel, false) logInfo(s"${path.eventLog.toString} has App: ${app.appId}") Some(app) } catch { diff --git a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationEventProcessor.scala b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationEventProcessor.scala index f5efaedda9f..74c0462d7db 100644 --- a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationEventProcessor.scala +++ b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationEventProcessor.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.rapids.tool.qualification import java.util.concurrent.TimeUnit.NANOSECONDS +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import com.nvidia.spark.rapids.tool.profiling._ @@ -26,7 +27,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.sql.execution.ui._ import org.apache.spark.sql.rapids.tool.{EventProcessorBase, GpuEventLogException, ToolUtils} -class QualificationEventProcessor(app: QualificationAppInfo) +class QualificationEventProcessor(app: QualificationAppInfo, perSqlOnly: Boolean) extends EventProcessorBase[QualificationAppInfo](app) { type T = QualificationAppInfo @@ -88,19 +89,7 @@ class QualificationEventProcessor(app: QualificationAppInfo) app: QualificationAppInfo, event: SparkListenerSQLExecutionStart): Unit = { super.doSparkListenerSQLExecutionStart(app, event) - val sqlExecution = QualSQLExecutionInfo( - event.executionId, - event.time, - None, - None, - "", - None, - false, - "" - ) - app.sqlStart += (event.executionId -> sqlExecution) app.processSQLPlan(event.executionId, event.sparkPlanInfo) - app.sqlPlans += (event.executionId -> event.sparkPlanInfo) } override def doSparkListenerSQLExecutionEnd( @@ -108,7 +97,9 @@ class QualificationEventProcessor(app: QualificationAppInfo) event: SparkListenerSQLExecutionEnd): Unit = { super.doSparkListenerSQLExecutionEnd(app, event) logDebug("Processing event: " + event.getClass) - app.lastSQLEndTime = Some(event.time) + if (!perSqlOnly) { + app.lastSQLEndTime = Some(event.time) + } // only include duration if it contains no jobs that failed val failures = app.sqlIDtoFailures.get(event.executionId) if (event.executionFailure.isDefined || failures.isDefined) { @@ -132,6 +123,24 @@ class QualificationEventProcessor(app: QualificationAppInfo) app.stageIdToSqlID.getOrElseUpdate(stageId, sqlID) } } + val sqlID = ProfileUtils.stringToLong(sqlIDString) + // don't store if we are only processing per sql queries and the job isn't + // related to a SQL query + if ((perSqlOnly && sqlID.isDefined) || !perSqlOnly) { + val thisJob = new JobInfoClass( + event.jobId, + event.stageIds, + sqlID, + event.properties.asScala, + event.time, + None, + None, + None, + None, + ProfileUtils.isPluginEnabled(event.properties.asScala) || app.gpuMode + ) + app.jobIdToInfo.put(event.jobId, thisJob) + } // If the confs are set after SparkSession initialization, it is captured in this event. if (app.clusterTags.isEmpty) { app.clusterTags = event.properties.getProperty( @@ -144,7 +153,9 @@ class QualificationEventProcessor(app: QualificationAppInfo) event: SparkListenerJobEnd): Unit = { logDebug("Processing event: " + event.getClass) super.doSparkListenerJobEnd(app, event) - app.lastJobEndTime = Some(event.time) + if (!perSqlOnly) { + app.lastJobEndTime = Some(event.time) + } if (event.jobResult != JobSucceeded) { app.jobIdToSqlID.get(event.jobId) match { case Some(sqlID) => @@ -181,7 +192,7 @@ class QualificationEventProcessor(app: QualificationAppInfo) event: SparkListenerSQLAdaptiveExecutionUpdate): Unit = { logDebug("Processing event: " + event.getClass) // AQE plan can override the ones got from SparkListenerSQLExecutionStart - app.sqlPlans += (event.executionId -> event.sparkPlanInfo) app.processSQLPlan(event.executionId, event.sparkPlanInfo) + super.doSparkListenerSQLAdaptiveExecutionUpdate(app, event) } } diff --git a/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/RunningQualificationEventProcessor.scala b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/RunningQualificationEventProcessor.scala new file mode 100644 index 00000000000..850a918d089 --- /dev/null +++ b/tools/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/RunningQualificationEventProcessor.scala @@ -0,0 +1,254 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.rapids.tool.qualification + +import java.io.IOException +import java.util.concurrent.atomic.AtomicBoolean + +import scala.util.control.NonFatal + +import com.nvidia.spark.rapids.tool.qualification.{RunningQualificationApp, RunningQualOutputWriter} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.security.AccessControlException + +import org.apache.spark.{CleanerListener, SparkConf, SparkContext} +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler._ +import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart} + +/** + * This is a Spark Listener that can be used to determine if the SQL queries in + * the running application are a good fit to try with the Rapids Accelerator for Spark. + * + * This only supports output on a per sql query basis. It supports writing to + * local filesystem and most distributed filesystems and blob stores supported + * by Hadoop. + * + * It can be run in your Spark application by installing it as a listener: + * set spark.extraListeners to + * org.apache.spark.sql.rapids.tool.qualification.RunningQualificationEventProcessor + * and including the tools jar rapids-4-spark-tools_2.12-.jar when you start + * the Spark application. + * + * The user should specify the output directory if they want the output to go to separate + * files, otherwise it will go to the Spark driver log: + * - spark.rapids.qualification.outputDir + * + * By default, this will output results for 10 SQL queries per file and will + * keep 100 files. This behavior is because many blob stores don't show files until + * they are full written so you wouldn't be able to see the results for a running + * application until it finishes the number of SQL queries per file. This behavior + * can be configured with the following configs. + * - spark.rapids.qualification.output.numSQLQueriesPerFile - default 10 + * - spark.rapids.qualification.output.maxNumFiles - default 100 + * + * @param sparkConf Spark Configuration used to get configs used by this listener + */ +class RunningQualificationEventProcessor(sparkConf: SparkConf) extends SparkListener with Logging { + + private val qualApp = new RunningQualificationApp(true) + private val listener = qualApp.getEventListener + private val isInited = new AtomicBoolean(false) + private val maxSQLQueriesPerFile: Long = + sparkConf.get("spark.rapids.qualification.output.numSQLQueriesPerFile", "10").toLong + private val maxNumFiles: Int = + sparkConf.get("spark.rapids.qualification.output.maxNumFiles", "100").toInt + private val outputFileFromConfig = sparkConf.get("spark.rapids.qualification.outputDir", "") + private lazy val appName = qualApp.appInfo.map(_.appName).getOrElse("") + private var fileWriter: Option[RunningQualOutputWriter] = None + private var currentFileNum = 0 + private var currentSQLQueriesWritten = 0 + private val filesWritten = Array.fill[Seq[Path]](maxNumFiles)(Seq.empty) + private lazy val txtHeader = qualApp.getPerSqlTextHeader + + class QualCleanerListener extends SparkListener with CleanerListener with Logging { + def accumCleaned(accId: Long): Unit = { + // remove the accums when no longer referenced + qualApp.cleanupAccumId(accId) + } + def rddCleaned(rddId: Int): Unit = {} + def shuffleCleaned(shuffleId: Int): Unit = {} + def broadcastCleaned(broadcastId: Long): Unit = {} + def checkpointCleaned(rddId: Long): Unit = {} + } + + private def initListener(): Unit = { + // install after startup when SparkContext is available + val sc = SparkContext.getOrCreate(sparkConf) + sc.cleaner.foreach(x => x.attachListener(new QualCleanerListener())) + } + + private def cleanupExistingFiles(id: Int, hadoopConf: Configuration): Unit = { + filesWritten(id).foreach { file => + logWarning(s"Going to remove file: $file") + val fs = FileSystem.get(file.toUri, hadoopConf) + try { + // delete recursive + fs.delete(file, true) + } catch { + case _: AccessControlException => + logInfo(s"No permission to delete $file, ignoring.") + case ioe: IOException => + logError(s"IOException in cleaning $file", ioe) + } + } + } + + private def updateFileWriter(): Unit = { + // get the running Hadoop Configuration so we pick up keys for accessing various + // distributed filesystems + val hadoopConf = SparkContext.getOrCreate(sparkConf).hadoopConfiguration + if (outputFileFromConfig.nonEmpty) { + if (fileWriter.isDefined) { + if (currentFileNum >= maxNumFiles - 1) { + currentFileNum = 0 + } else { + currentFileNum += 1 + } + // if the current slot already had files written, remove those before + // writing a new file + cleanupExistingFiles(currentFileNum, hadoopConf) + } + val writer = try { + logDebug(s"Creating new file output writer for id: $currentFileNum") + val runningWriter = new RunningQualOutputWriter(qualApp.appId, appName, + outputFileFromConfig, Some(hadoopConf), currentFileNum.toString) + filesWritten(currentFileNum) = runningWriter.getOutputFileNames + Some(runningWriter) + } catch { + case NonFatal(e) => + logError("Error creating the RunningQualOutputWriter, output will not be" + + s" saved to a file, error: ${e.getMessage}", e) + None + } + writer.foreach(_.init()) + fileWriter.foreach(_.close()) + fileWriter = writer + } + } + + private def writeSQLDetails(sqlID: Long): Unit = { + val (csvSQLInfo, textSQLInfo) = qualApp.getPerSqlTextAndCSVSummary(sqlID) + if (outputFileFromConfig.nonEmpty) { + // once file has gotten enough SQL queries, switch it to new file + if (currentSQLQueriesWritten >= maxSQLQueriesPerFile || !fileWriter.isDefined) { + updateFileWriter() + currentSQLQueriesWritten = 0 + } + fileWriter.foreach { writer => + logDebug(s"Done with SQL query ${sqlID} summary:: \n $textSQLInfo") + writer.writePerSqlCSVReport(csvSQLInfo) + writer.writePerSqlTextReport(textSQLInfo) + currentSQLQueriesWritten += 1 + } + } else { + // file writer isn't configured so just output to driver logs, us warning + // level so it comes out when using the shell + logWarning("\n" + txtHeader + textSQLInfo) + } + qualApp.cleanupSQL(sqlID) + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { + listener.onStageCompleted(stageCompleted) + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { + listener.onStageSubmitted(stageSubmitted) + } + + override def onTaskStart(onTaskStart: SparkListenerTaskStart): Unit = { + listener.onTaskStart(onTaskStart) + } + + override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult): Unit = { + listener.onTaskGettingResult(taskGettingResult) + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + listener.onTaskEnd(taskEnd) + } + + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + listener.onJobStart(jobStart) + // make sure we have attached the listener on the first job start + if (!isInited.get()) { + initListener() + isInited.set(true) + } + } + + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { + listener.onJobEnd(jobEnd) + } + + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate): Unit = { + listener.onEnvironmentUpdate(environmentUpdate) + } + + override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { + listener.onBlockManagerAdded(blockManagerAdded) + } + + override def onBlockManagerRemoved( + blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit = { + listener.onBlockManagerRemoved(blockManagerRemoved) + } + + override def onApplicationStart(applicationStart: SparkListenerApplicationStart): Unit = { + listener.onApplicationStart(applicationStart) + } + + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { + listener.onApplicationEnd(applicationEnd) + fileWriter.foreach(_.close()) + } + + override def onExecutorMetricsUpdate( + executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit = { + listener.onExecutorMetricsUpdate(executorMetricsUpdate) + } + + override def onStageExecutorMetrics( + executorMetrics: SparkListenerStageExecutorMetrics): Unit = { + listener.onStageExecutorMetrics(executorMetrics) + } + + override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { + listener.onExecutorAdded(executorAdded) + } + + override def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit = { + listener.onExecutorRemoved(executorRemoved) + } + + override def onOtherEvent(event: SparkListenerEvent): Unit = { + listener.onOtherEvent(event) + event match { + case e: SparkListenerSQLExecutionStart => + logDebug("Starting new SQL query") + case e: SparkListenerSQLExecutionEnd => + writeSQLDetails(e.executionId) + case _ => + } + } + + override def onResourceProfileAdded(event: SparkListenerResourceProfileAdded): Unit = { + listener.onResourceProfileAdded(event) + } +} diff --git a/tools/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala b/tools/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala index 408550f0e50..b2ec1b0808b 100644 --- a/tools/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala +++ b/tools/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala @@ -23,14 +23,15 @@ import scala.collection.mutable.{ArrayBuffer, ListBuffer} import scala.io.Source import com.nvidia.spark.rapids.tool.{EventLogPathProcessor, ToolTestUtils} +import org.apache.hadoop.fs.{FileSystem, Path} import org.scalatest.{BeforeAndAfterEach, FunSuite} import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted, SparkListenerTaskEnd} -import org.apache.spark.sql.{DataFrame, SparkSession, TrampolineUtil} +import org.apache.spark.sql.{DataFrame, Dataset, SparkSession, TrampolineUtil} import org.apache.spark.sql.functions.{desc, hex, udf} import org.apache.spark.sql.rapids.tool.{AppBase, AppFilterImpl, ToolUtils} -import org.apache.spark.sql.rapids.tool.qualification.{QualificationAppInfo, QualificationSummaryInfo} +import org.apache.spark.sql.rapids.tool.qualification.{QualificationAppInfo, QualificationSummaryInfo, RunningQualificationEventProcessor} import org.apache.spark.sql.types._ // drop the fields that won't go to DataFrame without encoders @@ -131,6 +132,10 @@ class QualificationSuite extends FunSuite with BeforeAndAfterEach with Logging { Some(perSQLSchema)) } + def readPerSqlTextFile(expected: File): Dataset[String] = { + sparkSession.read.textFile(expected.getPath()) + } + private def createSummaryForDF( appSums: Seq[QualificationSummaryInfo]): Seq[TestQualificationSummary] = { appSums.map { appInfoRec => @@ -187,6 +192,68 @@ class QualificationSuite extends FunSuite with BeforeAndAfterEach with Logging { } } + test("RunningQualificationEventProcessor per sql") { + TrampolineUtil.withTempDir { qualOutDir => + TrampolineUtil.withTempPath { outParquetFile => + TrampolineUtil.withTempPath { outJsonFile => + // note don't close the application here so we test running output + ToolTestUtils.runAndCollect("running per sql") { spark => + val sparkConf = spark.sparkContext.getConf + sparkConf.set("spark.rapids.qualification.output.numSQLQueriesPerFile", "2") + sparkConf.set("spark.rapids.qualification.output.maxNumFiles", "3") + sparkConf.set("spark.rapids.qualification.outputDir", qualOutDir.getPath) + val listener = new RunningQualificationEventProcessor(sparkConf) + spark.sparkContext.addSparkListener(listener) + import spark.implicits._ + val testData = Seq((1, 2), (3, 4)).toDF("a", "b") + testData.write.json(outJsonFile.getCanonicalPath) + testData.write.parquet(outParquetFile.getCanonicalPath) + val df = spark.read.parquet(outParquetFile.getCanonicalPath) + val df2 = spark.read.json(outJsonFile.getCanonicalPath) + // generate a bunch of SQL queries to test the file rolling, should run + // 10 sql queries total with above and below + for (i <- 1 to 7) { + df.join(df2.select($"a" as "a2"), $"a" === $"a2").count() + } + val df3 = df.join(df2.select($"a" as "a2"), $"a" === $"a2") + df3 + } + // the code above that runs the Spark query stops the Sparksession + // so create a new one to read in the csv file + createSparkSession() + val outputDir = qualOutDir.getPath + "/" + val csvOutput0 = outputDir + QualOutputWriter.LOGFILE_NAME + "_persql_0.csv" + val txtOutput0 = outputDir + QualOutputWriter.LOGFILE_NAME + "_persql_0.log" + // check that there are 6 files since configured for 3 and have 1 csv and 1 log + // file each + val outputDirPath = new Path(outputDir) + val fs = FileSystem.get(outputDirPath.toUri, + sparkSession.sparkContext.hadoopConfiguration) + val allFiles = fs.listStatus(outputDirPath) + assert(allFiles.size == 6) + val dfPerSqlActual = readPerSqlFile(new File(csvOutput0)) + assert(dfPerSqlActual.columns.size == 10) + val rows = dfPerSqlActual.collect() + assert(rows.size == 2) + val firstRow = rows(1) + // , should be replaced with ; + assert(firstRow(3).toString.contains("at QualificationSuite.scala")) + + // this reads everything into single column + val dfPerSqlActualTxt = readPerSqlTextFile(new File(txtOutput0)) + assert(dfPerSqlActualTxt.columns.size == 1) + val rowsTxt = dfPerSqlActualTxt.collect() + // have to account for headers + assert(rowsTxt.size == 6) + val headerRowTxt = rowsTxt(1).toString + assert(headerRowTxt.contains("Recommendation")) + val firstValueRow = rowsTxt(3).toString + assert(firstValueRow.contains("QualificationSuite.scala")) + } + } + } + } + test("test order asc") { val logFiles = Array( s"$logDir/dataset_eventlog", @@ -845,8 +912,9 @@ class QualificationSuite extends FunSuite with BeforeAndAfterEach with Logging { val values = rowsSumOut(1).split(",") val appInfo = qualApp.aggregateStats() assert(appInfo.nonEmpty) + val appNameMaxSize = QualOutputWriter.getAppNameSize(Seq(appInfo.get)) assert(headers.size == - QualOutputWriter.getSummaryHeaderStringsAndSizes(Seq(appInfo.get), 0).keys.size) + QualOutputWriter.getSummaryHeaderStringsAndSizes(appNameMaxSize, 0).keys.size) assert(values.size == headers.size - 1) // unSupportedExpr is empty // 3 should be the SQL DF Duration assert(headers(3).contains("SQL DF")) @@ -904,7 +972,7 @@ class QualificationSuite extends FunSuite with BeforeAndAfterEach with Logging { } } - test("running qualification app files") { + test("running qualification app files with per sql") { TrampolineUtil.withTempPath { outParquetFile => TrampolineUtil.withTempPath { outJsonFile => @@ -920,6 +988,34 @@ class QualificationSuite extends FunSuite with BeforeAndAfterEach with Logging { val df2 = spark.read.json(outJsonFile.getCanonicalPath) df.join(df2.select($"a" as "a2"), $"a" === $"a2") } + // just basic testing that line exists and has right separator + val csvHeader = qualApp.getPerSqlCSVHeader + assert(csvHeader.contains("App Name,App ID,SQL ID,SQL Description,SQL DF Duration," + + "GPU Opportunity,Estimated GPU Duration,Estimated GPU Speedup," + + "Estimated GPU Time Saved,Recommendation")) + val txtHeader = qualApp.getPerSqlTextHeader + assert(txtHeader.contains("| App Name| App ID|" + + "SQL ID" + + "| " + + "SQL Description|" + + "SQL DF Duration|GPU Opportunity|Estimated GPU Duration|" + + "Estimated GPU Speedup|Estimated GPU Time Saved| Recommendation|")) + val randHeader = qualApp.getPerSqlHeader(";", true, 20) + assert(randHeader.contains("; App Name; App ID" + + ";SQL ID; SQL Description;SQL DF Duration;GPU Opportunity;Estimated GPU Duration;" + + "Estimated GPU Speedup;Estimated GPU Time Saved; Recommendation;")) + val allSQLIds = qualApp.getAvailableSqlIDs + val numSQLIds = allSQLIds.size + assert(numSQLIds > 0) + val sqlIdToLookup = allSQLIds.head + val (csvOut, txtOut) = qualApp.getPerSqlTextAndCSVSummary(sqlIdToLookup) + assert(csvOut.contains("Profiling Tool Unit Tests") && csvOut.contains(","), + s"CSV output was: $csvOut") + assert(txtOut.contains("Profiling Tool Unit Tests") && txtOut.contains("|"), + s"TXT output was: $txtOut") + val sqlOut = qualApp.getPerSQLSummary(sqlIdToLookup, ":", true, 5) + assert(sqlOut.contains("Tool Unit Tests:"), s"SQL output was: $sqlOut") + // test different delimiter val sumOut = qualApp.getSummary(":", false) val rowsSumOut = sumOut.split("\n") @@ -929,7 +1025,7 @@ class QualificationSuite extends FunSuite with BeforeAndAfterEach with Logging { val appInfo = qualApp.aggregateStats() assert(appInfo.nonEmpty) assert(headers.size == - QualOutputWriter.getSummaryHeaderStringsAndSizes(Seq(appInfo.get), 0).keys.size) + QualOutputWriter.getSummaryHeaderStringsAndSizes(30, 30).keys.size) assert(values.size == headers.size - 1) // UnsupportedExpr is empty // 3 should be the SQL DF Duration assert(headers(3).contains("SQL DF")) @@ -942,9 +1038,10 @@ class QualificationSuite extends FunSuite with BeforeAndAfterEach with Logging { // Check Read Schema contains json and parquet val readSchemaIndex = headersDetailed.length - 1 assert(headersDetailed(readSchemaIndex).contains("Read Schema")) - assert( - valuesDetailed(readSchemaIndex).contains("json") && + assert(valuesDetailed(readSchemaIndex).contains("json") && valuesDetailed(readSchemaIndex).contains("parquet")) + qualApp.cleanupSQL(sqlIdToLookup) + assert(qualApp.getAvailableSqlIDs.size == numSQLIds - 1) } } } From 67fea32eb70bda33ba1953b126cf63b10feac772 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 30 Sep 2022 17:41:34 -0600 Subject: [PATCH 160/190] fix resource leaks in regexp_extract_all (#6666) Signed-off-by: Andy Grove Signed-off-by: Andy Grove --- .../org/apache/spark/sql/rapids/stringFunctions.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala index 1288d69ff03..af20375c7a5 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala @@ -1312,20 +1312,24 @@ case class GpuRegExpExtractAll( } } } - ColumnVector.makeList(stringCols: _*) + withResource(stringCols) { _ => + ColumnVector.makeList(stringCols: _*) + } } } } // Filter out null values in the lists - val extractedStrings = withResource(extractedWithNulls.getListOffsetsView) { offsetsCol => + val extractedStrings = withResource(extractedWithNulls) { _ => + withResource(extractedWithNulls.getListOffsetsView) { offsetsCol => withResource(extractedWithNulls.getChildColumnView(0)) { stringCol => withResource(stringCol.isNotNull) { isNotNull => - withResource(isNotNull.makeListFromOffsets(rowCount, offsetsCol)) { booleanMask => + withResource(isNotNull.makeListFromOffsets(rowCount, offsetsCol)) { booleanMask => extractedWithNulls.applyBooleanMask(booleanMask) } } } } + } // If input is null, output should also be null withResource(extractedStrings) { s => withResource(GpuScalar.from(null, DataTypes.createArrayType(DataTypes.StringType))) { From 99e5ca932586d1bf742b385fe4b9f5f936d6c923 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Mon, 3 Oct 2022 06:58:37 -0700 Subject: [PATCH 161/190] Make broadcast tables spillable (#6604) Fixes #836. This is an MVP utilizing SpillableColumnarBatch wrapper. Signed-off-by: Gera Shegalov --- .../spark/rapids/RapidsBufferCatalog.scala | 5 + .../execution/GpuBroadcastExchangeExec.scala | 129 +++++++++--------- .../rapids/execution/GpuBroadcastHelper.scala | 6 +- .../spark/rapids/SerializationSuite.scala | 22 ++- 4 files changed, 87 insertions(+), 75 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala index d4b0e31a01f..39f98e9685f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala @@ -174,6 +174,11 @@ object RapidsBufferCatalog extends Logging with Arm { } } + // For testing + def setDeviceStorage(rdms: RapidsDeviceMemoryStore): Unit = { + deviceStorage = rdms + } + def init(rapidsConf: RapidsConf): Unit = { // We are going to re-initialize so make sure all of the old things were closed... closeImpl() diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala index 5d3959bd0e6..b9fe46efe79 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala @@ -33,6 +33,7 @@ import com.nvidia.spark.rapids.shims.{ShimBroadcastExchangeLike, ShimUnaryExecNo import org.apache.spark.SparkException import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging import org.apache.spark.launcher.SparkLauncher import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -87,18 +88,21 @@ object SerializedHostTableUtils extends Arm { } } +// scalastyle:off no.finalize @SerialVersionUID(100L) class SerializeConcatHostBuffersDeserializeBatch( data: Array[SerializeBatchDeserializeHostBuffer], output: Seq[Attribute]) - extends Serializable with Arm with AutoCloseable { + extends Serializable with Arm with AutoCloseable with Logging { @transient private var dataTypes = output.map(_.dataType).toArray @transient private var headers = data.map(_.header) @transient private var buffers = data.map(_.buffer) - @transient private var batchInternal: ColumnarBatch = null - def batch: ColumnarBatch = this.synchronized { - if (batchInternal == null) { + // used for memoization of deserialization to GPU on Executor + @transient private var batchInternal: SpillableColumnarBatch = null + + def batch: SpillableColumnarBatch = this.synchronized { + Option(batchInternal).getOrElse { if (headers.length > 1) { // This should only happen if the driver is trying to access the batch. That should not be // a common occurrence, so for simplicity just round-trip this through the serialization. @@ -111,30 +115,34 @@ class SerializeConcatHostBuffersDeserializeBatch( } assert(headers.length <= 1 && buffers.length <= 1) withResource(new NvtxRange("broadcast manifest batch", NvtxColor.PURPLE)) { _ => - if (headers.isEmpty) { - batchInternal = GpuColumnVector.emptyBatchFromTypes(dataTypes) - GpuColumnVector.extractBases(batchInternal).foreach(_.noWarnLeakExpected()) - } else { - withResource(JCudfSerialization.readTableFrom(headers.head, buffers.head)) { tableInfo => - val table = tableInfo.getContiguousTable - if (table == null) { - val numRows = tableInfo.getNumRows - batchInternal = new ColumnarBatch(new Array[ColumnVector](0), numRows) - } else { - batchInternal = GpuColumnVectorFromBuffer.from(table, dataTypes) - GpuColumnVector.extractBases(batchInternal).foreach(_.noWarnLeakExpected()) - table.getBuffer.noWarnLeakExpected() + try { + val res = if (headers.isEmpty) { + SpillableColumnarBatch(GpuColumnVector.emptyBatchFromTypes(dataTypes), + SpillPriorities.ACTIVE_BATCHING_PRIORITY, RapidsBuffer.defaultSpillCallback) + } else { + withResource(JCudfSerialization.readTableFrom(headers.head, buffers.head)) { + tableInfo => + val table = tableInfo.getContiguousTable + if (table == null) { + val numRows = tableInfo.getNumRows + SpillableColumnarBatch(new ColumnarBatch(Array.empty[ColumnVector], numRows), + SpillPriorities.ACTIVE_BATCHING_PRIORITY, RapidsBuffer.defaultSpillCallback) + } else { + SpillableColumnarBatch(table, dataTypes, + SpillPriorities.ACTIVE_BATCHING_PRIORITY, RapidsBuffer.defaultSpillCallback) + } } } + batchInternal = res + res + } finally { + // At this point we no longer need the host data and should not need to touch it again. + buffers.safeClose() + headers = null + buffers = null } - - // At this point we no longer need the host data and should not need to touch it again. - buffers.safeClose() - headers = null - buffers = null } } - batchInternal } /** @@ -145,32 +153,35 @@ class SerializeConcatHostBuffersDeserializeBatch( * NOTE: The caller is responsible to release these host columnar batches. */ def hostBatches: Array[ColumnarBatch] = this.synchronized { - batchInternal match { - case batch if batch == null => - withResource(new NvtxRange("broadcast manifest batch", NvtxColor.PURPLE)) { _ => - val columnBatches = new mutable.ArrayBuffer[ColumnarBatch]() - closeOnExcept(columnBatches) { cBatches => - headers.zip(buffers).foreach { case (header, buffer) => - val hostColumns = SerializedHostTableUtils.buildHostColumns( - header, buffer, dataTypes) - val rowCount = header.getNumRows - cBatches += new ColumnarBatch(hostColumns.toArray, rowCount) - } + Option(batchInternal).map { spillable => + withResource(spillable.getColumnarBatch()) { batch => + val hostColumns: Array[ColumnVector] = GpuColumnVector + .extractColumns(batch) + .safeMap(_.copyToHost()) + Array(new ColumnarBatch(hostColumns, numRows)) + } + }.getOrElse { + withResource(new NvtxRange("broadcast manifest batch", NvtxColor.PURPLE)) { _ => + val columnBatches = new mutable.ArrayBuffer[ColumnarBatch]() + closeOnExcept(columnBatches) { cBatches => + headers.zip(buffers).foreach { case (header, buffer) => + val hostColumns = SerializedHostTableUtils.buildHostColumns( + header, buffer, dataTypes) + val rowCount = header.getNumRows + cBatches += new ColumnarBatch(hostColumns.toArray, rowCount) } - columnBatches.toArray } - case batch => - val hostColumns = GpuColumnVector.extractColumns(batch).map(_.copyToHost()) - Array(new ColumnarBatch(hostColumns.toArray, batch.numRows())) + columnBatches.toArray + } } } private def writeObject(out: ObjectOutputStream): Unit = { - if (batchInternal != null) { - val table = GpuColumnVector.from(batchInternal) + Option(batchInternal).map { spillable => + val table = withResource(spillable.getColumnarBatch())(GpuColumnVector.from) JCudfSerialization.writeToStream(table, out, 0, table.getRowCount) out.writeObject(dataTypes) - } else { + }.getOrElse { if (headers.length == 0) { // We didn't get any data back, but we need to write out an empty table that matches withResource(GpuColumnVector.emptyHostColumns(dataTypes)) { hostVectors => @@ -201,35 +212,25 @@ class SerializeConcatHostBuffersDeserializeBatch( } } - def numRows: Int = { - if (batchInternal != null) { - batchInternal.numRows() - } else { - headers.map(_.getNumRows).sum - } - } + def numRows: Int = Option(batchInternal) + .map(_.numRows()) + .getOrElse(headers.map(_.getNumRows).sum) - def dataSize: Long = { - if (batchInternal != null) { - val bases = GpuColumnVector.extractBases(batchInternal).map(_.copyToHost()) - try { - JCudfSerialization.getSerializedSizeInBytes(bases, 0, batchInternal.numRows()) - } finally { - bases.safeClose() - } - } else { - buffers.map(_.getLength).sum - } - } + def dataSize: Long = Option(batchInternal) + .map(_.sizeInBytes) + .getOrElse(buffers.map(_.getLength).sum) override def close(): Unit = this.synchronized { buffers.safeClose() - if (batchInternal != null) { - batchInternal.close() - batchInternal = null - } + Option(batchInternal).foreach(_.close()) + } + + override def finalize(): Unit = { + super.finalize() + close() } } +// scalastyle:on no.finalize @SerialVersionUID(100L) class SerializeBatchDeserializeHostBuffer(batch: ColumnarBatch) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala index 988d855dbf7..092341fbbbf 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala @@ -41,9 +41,7 @@ object GpuBroadcastHelper { broadcastSchema: StructType): ColumnarBatch = { broadcastRelation.value match { case broadcastBatch: SerializeConcatHostBuffersDeserializeBatch => - val builtBatch = broadcastBatch.batch - GpuColumnVector.incRefCounts(builtBatch) - builtBatch + broadcastBatch.batch.getColumnarBatch() case v if SparkShimImpl.isEmptyRelation(v) => GpuColumnVector.emptyBatch(broadcastSchema) case t => @@ -67,7 +65,7 @@ object GpuBroadcastHelper { def getBroadcastBatchNumRows(broadcastRelation: Broadcast[Any]): Int = { broadcastRelation.value match { case broadcastBatch: SerializeConcatHostBuffersDeserializeBatch => - broadcastBatch.batch.numRows() + broadcastBatch.numRows case v if SparkShimImpl.isEmptyRelation(v) => 0 case t => throw new IllegalStateException(s"Invalid broadcast batch received $t") diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/SerializationSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/SerializationSuite.scala index 55906f78732..817e69e43fb 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/SerializationSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/SerializationSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021, NVIDIA CORPORATION. + * Copyright (c) 2021-2022, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,14 +18,20 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.Table import org.apache.commons.lang3.SerializationUtils -import org.scalatest.FunSuite +import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.rapids.execution.{SerializeBatchDeserializeHostBuffer, SerializeConcatHostBuffersDeserializeBatch} import org.apache.spark.sql.types.{DoubleType, IntegerType, StringType} import org.apache.spark.sql.vectorized.ColumnarBatch -class SerializationSuite extends FunSuite with Arm { +class SerializationSuite extends FunSuite + with BeforeAndAfterAll with Arm { + + override def beforeAll(): Unit = { + RapidsBufferCatalog.setDeviceStorage(new RapidsDeviceMemoryStore()) + } + private def buildBatch(): ColumnarBatch = { withResource(new Table.TestBuilder() .column(5, null.asInstanceOf[java.lang.Integer], 3, 1, 1, 1, 1, 1, 1, 1) @@ -68,12 +74,14 @@ class SerializationSuite extends FunSuite with Arm { val buffer = createDeserializedHostBuffer(gpuExpected) val hostBatch = new SerializeConcatHostBuffersDeserializeBatch(Array(buffer), attrs) withResource(hostBatch) { _ => - val gpuBatch = hostBatch.batch - TestUtils.compareBatches(gpuExpected, gpuBatch) + withResource(hostBatch.batch.getColumnarBatch()) { gpuBatch => + TestUtils.compareBatches(gpuExpected, gpuBatch) + } // clone via serialization after manifesting the GPU batch withResource(SerializationUtils.clone(hostBatch)) { clonedObj => - val gpuClonedBatch = clonedObj.batch - TestUtils.compareBatches(gpuExpected, gpuClonedBatch) + withResource(clonedObj.batch.getColumnarBatch()) { gpuClonedBatch => + TestUtils.compareBatches(gpuExpected, gpuClonedBatch) + } // try to clone it again from the cloned object SerializationUtils.clone(clonedObj).close() } From b9c771f126bde1c43462a87b06cff733c41c9aae Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Tue, 4 Oct 2022 11:18:13 -0500 Subject: [PATCH 162/190] Fixes split estimation in explode/explode_outer [databricks] (#6631) * Fixes split estimation in explode/explode_outer Signed-off-by: Alessandro Bellina * Fix leak when outer=true * arrayElements -> arrayLengths * listValues.getRowCount already includes nulls * Cleanup getRowByteCount * Get splits from the size obtained in prefixSum * Add tests and tweak split generation code * Fix leak in test * Remove unnecessary Logging import, and import cudf Scalar * Add javadoc Signed-off-by: Alessandro Bellina --- .../nvidia/spark/rapids/GpuGenerateExec.scala | 203 +++++++- .../spark/rapids/GpuGenerateSuite.scala | 489 ++++++++++++++++++ 2 files changed, 664 insertions(+), 28 deletions(-) create mode 100644 tests/src/test/scala/com/nvidia/spark/rapids/GpuGenerateSuite.scala diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala index a59ac484bf2..46bc82953a7 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala @@ -18,7 +18,7 @@ package com.nvidia.spark.rapids import scala.collection.mutable.ArrayBuffer -import ai.rapids.cudf.{ColumnVector, ContiguousTable, NvtxColor, Table} +import ai.rapids.cudf.{ColumnVector, ContiguousTable, DType, NvtxColor, NvtxRange, OrderByArg, Scalar, Table} import com.nvidia.spark.rapids.shims.{ShimExpression, ShimUnaryExecNode} import org.apache.spark.TaskContext @@ -133,12 +133,14 @@ trait GpuGenerator extends GpuUnevaluable { * @param outer when true, each input row will be output at least once, even if the * output of the given `generator` is empty. * @param targetSizeBytes the target number of bytes for a GPU batch, one of `RapidsConf` + * @param maxRows the target number of rows for a GPU batch, exposed for testing purposes. * @return split indices of input batch */ def inputSplitIndices(inputBatch: ColumnarBatch, generatorOffset: Int, outer: Boolean, - targetSizeBytes: Long): Array[Int] + targetSizeBytes: Long, + maxRows: Int = Int.MaxValue): Array[Int] /** * Extract lazy expressions from generator if exists. @@ -204,7 +206,8 @@ case class GpuReplicateRows(children: Seq[Expression]) extends GpuGenerator with override def inputSplitIndices(inputBatch: ColumnarBatch, generatorOffset: Int, outer: Boolean, - targetSizeBytes: Long): Array[Int] = { + targetSizeBytes: Long, + maxRows: Int = Int.MaxValue): Array[Int] = { val vectors = GpuColumnVector.extractBases(inputBatch) val inputRows = inputBatch.numRows() if (inputRows == 0) return Array() @@ -223,7 +226,7 @@ case class GpuReplicateRows(children: Seq[Expression]) extends GpuGenerator with // how may splits will we need to keep the output size under the target size val numSplitsForTargetSize = math.ceil(estimatedOutputSizeBytes / targetSizeBytes).toInt // how may splits will we need to keep the output rows under max value - val numSplitsForTargetRow = math.ceil(estimatedOutputRows / Int.MaxValue).toInt + val numSplitsForTargetRow = math.ceil(estimatedOutputRows / maxRows).toInt // how may splits will we need to keep replicateRows working safely val numSplits = numSplitsForTargetSize max numSplitsForTargetRow @@ -264,38 +267,182 @@ abstract class GpuExplodeBase extends GpuUnevaluableUnaryExpression with GpuGene } } + private def getPerRowRepetition(explodingColumn: ColumnVector, outer: Boolean): ColumnVector = { + withResource(explodingColumn.countElements()) { arrayLengths => + if (outer) { + // for outer, empty arrays and null arrays will produce a row + withResource(GpuScalar.from(1, IntegerType)) { one => + val noNulls = withResource(arrayLengths.isNotNull) { isLhsNotNull => + isLhsNotNull.ifElse(arrayLengths, one) + } + withResource(noNulls) { _ => + withResource(noNulls.greaterOrEqualTo(one)) { isGeOne => + isGeOne.ifElse(noNulls, one) + } + } + } + } else { + // ensure no nulls in this output + withResource(GpuScalar.from(0, IntegerType)) { zero => + GpuNvl(arrayLengths, zero) + } + } + } + } + + private def getRowByteCount(column: Seq[ColumnVector]): ColumnVector = { + withResource(new NvtxRange("getRowByteCount", NvtxColor.GREEN)) { _ => + val bits = withResource(new Table(column: _*)) { tbl => + tbl.rowBitCount() + } + withResource(bits) { _ => + withResource(Scalar.fromLong(8)) { toBytes => + bits.trueDiv(toBytes, DType.INT64) + } + } + } + } + override def inputSplitIndices(inputBatch: ColumnarBatch, - generatorOffset: Int, - outer: Boolean, - targetSizeBytes: Long): Array[Int] = { + generatorOffset: Int, outer: Boolean, + targetSizeBytes: Long, + maxRows: Int = Int.MaxValue): Array[Int] = { + val inputRows = inputBatch.numRows() + + // if the number of input rows is 1 or less, cannot split + if (inputRows <= 1) return Array() val vectors = GpuColumnVector.extractBases(inputBatch) - val inputRows = inputBatch.numRows() - if (inputRows == 0) return Array() + + val explodingColumn = vectors(generatorOffset) // Get the output size in bytes of the column that we are going to explode // along with an estimate of how many output rows produced by the explode - val (explodeColOutputSize, estimatedOutputRows) = withResource( - vectors(generatorOffset).getChildColumnView(0)) { listValues => - val totalSize = listValues.getDeviceMemorySize - val totalCount = listValues.getRowCount - (totalSize.toDouble, totalCount.toDouble) + val (explodeColOutputSize, estimatedOutputRows) = + withResource(explodingColumn.getChildColumnView(0)) { listValues => + val totalSize = listValues.getDeviceMemorySize + // get the number of elements in the array child + var totalCount = listValues.getRowCount + // when we are calculating an explode_outer, we need to add to the row count + // the cases where the parent element has a null array, as we are going to produce + // these rows. + if (outer) { + totalCount += explodingColumn.getNullCount + } + (totalSize.toDouble, totalCount.toDouble) + } + + // we know we are going to output at least this much + var estimatedOutputSizeBytes = explodeColOutputSize + + val splitIndices = if (generatorOffset == 0) { + // this explodes the array column, and the table has no other columns to go + // along with it + val numSplitsForTargetSize = + math.min(inputRows, + math.ceil(estimatedOutputSizeBytes / targetSizeBytes).toInt) + GpuBatchUtils.generateSplitIndices(inputRows, numSplitsForTargetSize).distinct + } else { + withResource(new NvtxRange("EstimateRepetition", NvtxColor.BLUE)) { _ => + // get the # of repetitions per row of the input for this explode + val perRowRepetition = getPerRowRepetition(explodingColumn, outer) + val repeatingColumns = vectors.slice(0, generatorOffset) + + // get per row byte count of every column, except the exploding one + // NOTE: in the future, we may want to factor in the exploding column size + // into this math, if there is skew in the column to explode. + val repeatingByteCount = + withResource(getRowByteCount(repeatingColumns)) { byteCountBeforeRepetition => + withResource(perRowRepetition) { _ => + byteCountBeforeRepetition.mul(perRowRepetition) + } + } + + // compute prefix sum of byte sizes, this can be used to find input row + // split points at which point the output batch is estimated to be roughly + // prefixSum(row) bytes ( + exploding column size for `row`) + val prefixSum = withResource(repeatingByteCount) { + _.prefixSum + } + + val splitIndices = withResource(prefixSum) { _ => + // the last element of `repeatedSizeEstimate` is the overall sum + val repeatedSizeEstimate = + withResource(prefixSum.subVector((prefixSum.getRowCount - 1).toInt)) { lastRow => + withResource(lastRow.copyToHost()) { hc => + hc.getLong(0) + } + } + + estimatedOutputSizeBytes += repeatedSizeEstimate + + // how may splits will we need to keep the output size under the target size + val numSplitsForTargetSize = + math.min(inputRows, + math.ceil(estimatedOutputSizeBytes / targetSizeBytes).toInt) + + val idealSplits = if (numSplitsForTargetSize == 0) { + Array.empty[Long] + } else { + // we need to apply the splits onto repeated size, because the prefixSum + // is only done for repeated sizes + val sizePerSplit = + math.ceil(repeatedSizeEstimate.toDouble / numSplitsForTargetSize).toLong + (1 until numSplitsForTargetSize).map { s => + s * sizePerSplit + }.toArray + } + + if (idealSplits.length == 0) { + Array.empty[Int] + } else { + val lowerBound = + withResource(new Table(prefixSum)) { prefixSumTable => + withResource(ColumnVector.fromLongs(idealSplits: _*)) { idealBounds => + withResource(new Table(idealBounds)) { idealBoundsTable => + prefixSumTable.lowerBound(idealBoundsTable, OrderByArg.asc(0)) + } + } + } + + val largestSplitIndex = inputRows - 1 + val splits = withResource(lowerBound) { _ => + withResource(lowerBound.copyToHost) { hostBounds => + (0 until hostBounds.getRowCount.toInt).map { s => + // add 1 to the bound because you get the row index of the last + // row at which was smaller or equal to the bound, for example: + // prefixSum=[8, 16, 24, 32] + // if you are looking for a bound of 16, you get index 1. That said, to + // split this, you want the index to be between 16 and 24, so that's index 2. + // Make sure that the maximum bound is inputRows - 1, otherwise we can trigger + // a cuDF exception. + math.min(hostBounds.getInt(s) + 1, largestSplitIndex) + } + } + } + + // apply distinct in the case of extreme skew, where for example we have all nulls + // except for 1 row that has all the data. + splits.distinct.toArray + } + } + splitIndices + } } - // input size of columns to be repeated during exploding - val repeatColsInputSize = vectors.slice(0, generatorOffset).map(_.getDeviceMemorySize).sum - // estimated output size of repeated columns - val repeatColsOutputSize = repeatColsInputSize * estimatedOutputRows / inputRows - // estimated total output size - val estimatedOutputSizeBytes = explodeColOutputSize + repeatColsOutputSize - // how may splits will we need to keep the output size under the target size - val numSplitsForTargetSize = math.ceil(estimatedOutputSizeBytes / targetSizeBytes).toInt - // how may splits will we need to keep the output rows under max value - val numSplitsForTargetRow = math.ceil(estimatedOutputRows / Int.MaxValue).toInt - // how may splits will we need to keep exploding working safely - val numSplits = numSplitsForTargetSize max numSplitsForTargetRow - if (numSplits == 0) Array() - else GpuBatchUtils.generateSplitIndices(inputRows, numSplits) + // how may splits will we need to keep the output rows under max value + val numSplitsForTargetRow = math.ceil(estimatedOutputRows / maxRows).toInt + + // If the number of splits needed to keep the row limits for cuDF is higher than + // the splits we found by size, we need to use the row-based splits. + // Note, given skewed input, we could be left with batches split at bad places, + // e.g. all of the non nulls are in a single split. So we may need to re-split + // that row-based slice using the size approach. + if (numSplitsForTargetRow > splitIndices.length) { + GpuBatchUtils.generateSplitIndices(inputRows, numSplitsForTargetRow) + } else { + splitIndices + } } // Infer result schema of GenerateExec from input schema diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuGenerateSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuGenerateSuite.scala new file mode 100644 index 00000000000..74604e3f7cf --- /dev/null +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuGenerateSuite.scala @@ -0,0 +1,489 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids + +import ai.rapids.cudf.{ColumnVector, DType, Table} +import ai.rapids.cudf.HostColumnVector.{BasicType, ListType} +import java.util +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.types.{ArrayType, DataType, IntegerType} +import org.apache.spark.sql.vectorized.ColumnarBatch + +class GpuGenerateSuite + extends SparkQueryCompareTestSuite + with Arm { + val rapidsConf = new RapidsConf(Map.empty[String, String]) + + def makeListColumn( + numRows: Int, + listSize: Int, + includeNulls: Boolean, + allNulls: Boolean): ColumnVector = { + val list = util.Arrays.asList((0 until listSize): _*) + val rows = (0 until numRows).map { r => + if (allNulls || includeNulls && r % 2 == 0) { + null + } else { + list + } + } + ColumnVector.fromLists( + new ListType(true, + new BasicType(true, DType.INT32)), + rows: _*) + } + + def makeBatch( + numRows: Int, + includeRepeatColumn: Boolean = true, + includeNulls: Boolean = false, + listSize: Int = 4, + allNulls: Boolean = false): (ColumnarBatch, Long) = { + var inputSize: Long = 0L + withResource(makeListColumn(numRows, listSize, includeNulls, allNulls)) { cvList => + inputSize += + withResource(cvList.getChildColumnView(0)) { + _.getDeviceMemorySize + } + val batch = if (includeRepeatColumn) { + val dt: Array[DataType] = Seq(IntegerType, ArrayType(IntegerType)).toArray + val secondColumn = (0 until numRows).map { x => + val i = Int.box(x) + if (allNulls || includeNulls && i % 2 == 0) { + null + } else { + i + } + } + withResource(ColumnVector.fromBoxedInts(secondColumn: _*)) { repeatCol => + inputSize += listSize * repeatCol.getDeviceMemorySize + withResource(new Table(repeatCol, cvList)) { tbl => + GpuColumnVector.from(tbl, dt) + } + } + } else { + val dt: Array[DataType] = Seq(ArrayType(IntegerType)).toArray + withResource(new Table(cvList)) { tbl => + GpuColumnVector.from(tbl, dt) + } + } + (batch, inputSize) + } + } + + def checkSplits(splits: Array[Int], batch: ColumnarBatch): Unit = { + withResource(GpuColumnVector.from(batch)) { tbl => + var totalRows = 0L + // because concatenate does not work with 1 Table and I can't incRefCount a Table + // this is used to close the concatenated table, which would otherwise be leaked. + withResource(new ArrayBuffer[Table]) { tableToClose => + withResource(tbl.contiguousSplit(splits: _*)) { splitted => + splitted.foreach { ct => + totalRows += ct.getRowCount + } + // `getTable` causes Table to be owned by the `ContiguousTable` class + // so they get closed when the `ContiguousTable`s get closed. + val concatted = if (splitted.length == 1) { + splitted(0).getTable + } else { + val tbl = Table.concatenate(splitted.map(_.getTable): _*) + tableToClose += tbl + tbl + } + // Compare row by row the input vs the concatenated splits + withResource(GpuColumnVector.from(batch)) { inputTbl => + assertResult(concatted.getRowCount)(batch.numRows()) + (0 until batch.numCols()).foreach { c => + withResource(concatted.getColumn(c).copyToHost()) { hostConcatCol => + withResource(inputTbl.getColumn(c).copyToHost()) { hostInputCol => + (0 until batch.numRows()).foreach { r => + if (hostInputCol.isNull(r)) { + assertResult(true)(hostConcatCol.isNull(r)) + } else { + if (hostInputCol.getType == DType.LIST) { + // exploding column + compare(hostInputCol.getList(r), hostConcatCol.getList(r)) + } else { + compare(hostInputCol.getInt(r), hostConcatCol.getInt(r)) + } + } + } + } + } + } + } + } + } + } + } + + test("all null inputs") { + val (batch, _) = makeBatch(numRows = 100, allNulls = true) + withResource(batch) { _ => + val e = GpuExplode(null) + assertResult(0)( + e.inputSplitIndices(batch, 1, false, 4).length) + + // Here we are going to split 99 times, since our targetSize is 1 Byte and we are going to + // produce 100 rows. The row byte count for a null row is going to be sizeof(type), and + // because we use `outer=true` we are expecting 4 bytes x 100 rows. + assertResult(99)( + e.inputSplitIndices(batch, generatorOffset = 1 , true, 4).length) + } + } + + test("0-row batches short-circuits to no splits") { + val (batch, _) = makeBatch(numRows = 0) + withResource(batch) { _ => + val e = GpuExplode(null) + assertResult(0)( + e.inputSplitIndices(batch, 1, false, 1).length) + assertResult(0)( + e.inputSplitIndices(batch, generatorOffset = 1 , true, 1).length) + } + } + + test("1-row batches short-circuits to no splits") { + val (batch, _) = makeBatch(numRows = 1) + withResource(batch) { _ => + val e = GpuExplode(null) + var splits = e.inputSplitIndices(batch, 1, false, 1) + assertResult(0)(splits.length) + checkSplits(splits, batch) + + splits = e.inputSplitIndices(batch, generatorOffset = 1 , true, 1) + assertResult(0)(splits.length) + checkSplits(splits, batch) + } + } + + test("2-row batches split in half") { + val (batch, inputSize) = makeBatch(numRows = 2) + withResource(batch) { _ => + val e = GpuExplode(null) + val target = inputSize/2 + var splits = e.inputSplitIndices(batch, 1, false, target) + assertResult(1)(splits.length) + assertResult(1)(splits(0)) + checkSplits(splits, batch) + + splits = e.inputSplitIndices(batch, generatorOffset = 1 , true, target) + assertResult(1)(splits.length) + assertResult(1)(splits(0)) + checkSplits(splits, batch) + } + } + + test("8-row batch splits in half") { + val (batch, inputSize) = makeBatch(numRows = 8) + withResource(batch) { _ => + val e = GpuExplode(null) + val target = inputSize/2 + // here a split at 4 actually means produce two Tables, each with 4 rows. + var splits = e.inputSplitIndices(batch, 1, false, target) + assertResult(1)(splits.length) + assertResult(4)(splits(0)) + checkSplits(splits, batch) + + splits = e.inputSplitIndices(batch, generatorOffset = 1 , true, target) + assertResult(1)(splits.length) + assertResult(4)(splits(0)) + checkSplits(splits, batch) + } + } + + // these next four tests exercise code that just uses the exploding column's size as the limit + test("test batch with a single exploding column") { + val (batch, _) = makeBatch(numRows = 100, includeRepeatColumn = false) + withResource(batch) { _ => + val e = GpuExplode(null) + // the exploded column should be 4 Bytes * 100 rows * 4 reps per row = 1600 Bytes. + val targetSize = 1600 + // 1600 == a single split + var splits = e.inputSplitIndices(batch, 0, false, targetSize) + assertResult(0)(splits.length) + checkSplits(splits, batch) + + // 800 == 1 splits (2 parts) right down the middle + splits = e.inputSplitIndices(batch, 0, false, targetSize/2) + assertResult(1)(splits.length) + assertResult(50)(splits(0)) + checkSplits(splits, batch) + + // 400 == 3 splits (4 parts) + splits = e.inputSplitIndices(batch, 0, false, targetSize/4) + assertResult(3)(splits.length) + assertResult(25)(splits(0)) + assertResult(50)(splits(1)) + assertResult(75)(splits(2)) + checkSplits(splits, batch) + } + } + + test("test batch with a single exploding column with nulls") { + val (batch, inputSize) = makeBatch(numRows=100, includeRepeatColumn=false, includeNulls=true) + withResource(batch) { _ => + val e = GpuExplode(null) + // the exploded column should be 4 Bytes * 100 rows * 4 reps per row = 1600 Bytes. + // with nulls it should be 1/2 that + val targetSize = inputSize + // 800 = no splits + var splits = e.inputSplitIndices(batch, 0, false, targetSize) + assertResult(0)(splits.length) + checkSplits(splits, batch) + + // 400 == 1 splits (2 parts) right down the middle + splits = e.inputSplitIndices(batch, 0, false, targetSize/2) + assertResult(1)(splits.length) + assertResult(50)(splits(0)) + checkSplits(splits, batch) + + // 200 == 8 parts + splits = e.inputSplitIndices(batch, 0, false, targetSize/4) + assertResult(3)(splits.length) + assertResult(25)(splits(0)) + assertResult(50)(splits(1)) + assertResult(75)(splits(2)) + checkSplits(splits, batch) + } + } + + test("outer: test batch with a single exploding column with nulls") { + // for outer, size is the same (nulls are assumed not to occupy any space, which is not true) + // but number of rows is not the same. + val (batch, inputSize) = makeBatch(numRows=100, includeRepeatColumn=false, includeNulls=true) + withResource(batch) { _ => + val e = GpuExplode(null) + // the exploded column should be 4 Bytes * 100 rows * 4 reps per row = 1600 Bytes. + // with nulls it should be 1/2 that + val targetSize = inputSize + // 800 = no splits + assertResult(0)( + e.inputSplitIndices(batch, 0, true, targetSize).length) + + // 400 == 1 splits (2 parts) right down the middle + var splits = e.inputSplitIndices(batch, 0, true, targetSize/2) + assertResult(1)(splits.length) + assertResult(50)(splits(0)) + + // 200 == 3 splits (4 parts) + splits = e.inputSplitIndices(batch, 0, true, targetSize/4) + assertResult(3)(splits.length) + assertResult(25)(splits(0)) + assertResult(50)(splits(1)) + assertResult(75)(splits(2)) + } + } + + test("outer, limit rows: test batch with a single exploding column with nulls") { + // for outer, size is the same (nulls are assumed not to occupy any space, which is not true) + // but number of rows is not the same. + val (batch, inputSize) = makeBatch(numRows=100, includeRepeatColumn=false, includeNulls=true) + withResource(batch) { _ => + val e = GpuExplode(null) + // the exploded column should be 4 Bytes * 100 rows * 4 reps per row = 1600 Bytes. + // with nulls it should be 1/2 that + // 250 rows is the expected number of rows in this case: + // 200 non-null rows (4 reps * 50 non-null) + + // 50 for rows that had nulls, since those are produced as well. + // no-splits + assertResult(0)( + e.inputSplitIndices(batch, 0, true, inputSize, maxRows = 250).length) + + // 1 split (2 parts) + var splits = e.inputSplitIndices(batch, 0, true, inputSize, maxRows = 125) + assertResult(1)(splits.length) + assertResult(50)(splits(0)) + + // 3 splits (4 parts) + splits = e.inputSplitIndices(batch, 0, true, inputSize, maxRows = 63) + assertResult(3)(splits.length) + assertResult(25)(splits(0)) + assertResult(50)(splits(1)) + assertResult(75)(splits(2)) + } + } + + test("test batch with a repeating column") { + val (batch, inputSize) = makeBatch(numRows=100) + withResource(batch) { _ => + val e = GpuExplode(null) + + // no splits + var targetSize = inputSize //3200 Bytes + var splits = e.inputSplitIndices(batch, 1, false, targetSize) + assertResult(0)(splits.length) + checkSplits(splits, batch) + + // 1600 == 1 splits (2 parts) right down the middle + targetSize = inputSize / 2 + splits = e.inputSplitIndices(batch, 1, false, targetSize) + assertResult(1)(splits.length) + assertResult(50)(splits(0)) + checkSplits(splits, batch) + + targetSize = inputSize / 4 + splits = e.inputSplitIndices(batch, 1, false, targetSize) + assertResult(3)(splits.length) + assertResult(25)(splits(0)) + assertResult(50)(splits(1)) + assertResult(75)(splits(2)) + checkSplits(splits, batch) + + targetSize = inputSize / 8 + splits = e.inputSplitIndices(batch, 1, false, targetSize) + assertResult(7)(splits.length) + assertResult(13)(splits(0)) + assertResult(25)(splits(1)) + assertResult(38)(splits(2)) + assertResult(50)(splits(3)) + assertResult(63)(splits(4)) + assertResult(75)(splits(5)) + assertResult(88)(splits(6)) + checkSplits(splits, batch) + } + } + + test("test batch with a repeating column with nulls") { + val (batch, _) = makeBatch(numRows=100, includeNulls = true) + withResource(batch) { _ => + val e = GpuExplode(null) + val inputSize = 1600 + var targetSize = inputSize + var splits = e.inputSplitIndices(batch, 1, false, targetSize) + assertResult(0)(splits.length) + checkSplits(splits, batch) + + // 800 == 1 splits (2 parts) right down the middle + targetSize = inputSize / 2 + splits = e.inputSplitIndices(batch, 1, false, targetSize) + assertResult(1)(splits.length) + checkSplits(splits, batch) + + // 400 == 3 splits (4 parts) + targetSize = inputSize / 4 + splits = e.inputSplitIndices(batch, 1, false, targetSize) + assertResult(3)(splits.length) + checkSplits(splits, batch) + + // we estimate 1600 bytes in this scenario (1000 for repeating col, and 800 for exploding) + // in this case we use 32 bytes as the target size, so that is ceil(1800/32) => 57 splits. + splits = e.inputSplitIndices(batch, 1, false, 32) + assertResult(49)(splits.length) + checkSplits(splits, batch) + + // in this case we use 16 bytes as the target size, resulting in 1600/8 > 100 splits, + // which is more than the number of input rows, so we fallback to splitting at most to + // input number of rows 100. Since the input is half nulls, then we get 50 splits. + splits = e.inputSplitIndices(batch, 1, false, 8) + assertResult(50)(splits.length) + checkSplits(splits, batch) + } + } + + test("outer: test batch with a repeating column with nulls") { + val (batch, inputSize) = makeBatch(numRows = 100, includeNulls = true) + withResource(batch) { _ => + val e = GpuExplode(null) + // the exploded column should be 4 Bytes * 100 rows * 4 reps per row = 1600 Bytes. + // the repeating column is 4 bytes (or 400 bytes total) repeated 4 times (or 1600) + // 1600 == two splits + var targetSize = inputSize // 2656 Bytes + var splits = e.inputSplitIndices(batch, 1, true, targetSize) + checkSplits(splits, batch) + + // 1600 == 1 splits (2 parts) right down the middle + targetSize = inputSize / 2 + splits = e.inputSplitIndices(batch, 1, true, targetSize) + checkSplits(splits, batch) + + // 800 == 3 splits (4 parts) + targetSize = inputSize / 4 + splits = e.inputSplitIndices(batch, 1, true, targetSize) + checkSplits(splits, batch) + + // we estimate 1800 bytes in this scenario (1000 for repeating col, and 800 for exploding) + // this is slightly more splits from the outer=false case, since we weren't counting nulls + // then. + splits = e.inputSplitIndices(batch, 1, true, 32) + assertResult(55)(splits.length) + checkSplits(splits, batch) + + // in this case we use 16 bytes as the target size, resulting in 1800/16 > 100 splits, + // which is more than the number of input rows, so we fallback to splitting at most to + // input number of rows 100. Since the input is half nulls, then we get 50 splits. + splits = e.inputSplitIndices(batch, 1, true, 16) + assertResult(50)(splits.length) + checkSplits(splits, batch) + } + } + + test("outer: test 1000 row batch with a repeating column with nulls") { + val (batch, _) = makeBatch(numRows = 10000, includeNulls = true) + withResource(batch) { _ => + val e = GpuExplode(null) + // the exploded column should be 4 Bytes * 100 rows * 4 reps per row = 1600 Bytes. + // the repeating column is 4 bytes (or 400 bytes total) repeated 4 times (or 1600) + // 1600 == two splits + var splits = e.inputSplitIndices(batch, 1, true, 1600) + checkSplits(splits, batch) + + // 1600 == 1 splits (2 parts) right down the middle + splits = e.inputSplitIndices(batch, 1, true, 800) + checkSplits(splits, batch) + + // 800 == 3 splits (4 parts) + splits = e.inputSplitIndices(batch, 1, true, 400) + checkSplits(splits, batch) + + splits = e.inputSplitIndices(batch, 1, true, 100) + checkSplits(splits, batch) + } + } + + test("if the row limit produces more splits, prefer splitting using maxRows") { + val (batch, inputSize) = makeBatch(numRows = 10000, includeNulls = true) + withResource(batch) { _ => + val e = GpuExplode(null) + // by size try to no splits, instead we should get 2 (by maxRows) + // we expect 40000 rows (4x1000 given 4 items in the list per row), but we included nulls, + // so this should return 20000 rows (given that this is not outer) + var splits = e.inputSplitIndices(batch, 1, false, inputSize, maxRows = 20000) + assertResult(0)(splits.length) + checkSplits(splits, batch) + + splits = e.inputSplitIndices(batch, 1, false, inputSize, maxRows = 10000) + assertResult(1)(splits.length) + assertResult(5000)(splits(0)) + checkSplits(splits, batch) + } + } + + test("outer: if the row limit produces more splits, prefer splitting using maxRows") { + val (batch, inputSize) = makeBatch(numRows = 10000, includeNulls = true) + withResource(batch) { _ => + val e = GpuExplode(null) + // by size try to no splits, instead we should get 2 (by maxRows) + // we expect 40000 rows (4x1000 given 4 items in the list per row) + val splits = e.inputSplitIndices(batch, 1, true, inputSize, maxRows = 20000) + assertResult(1)(splits.length) + assertResult(5000)(splits(0)) + checkSplits(splits, batch) + } + } +} From c05ac2de84bbd34db644d82eec38f98b2d1a6fcd Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Tue, 4 Oct 2022 10:19:24 -0700 Subject: [PATCH 163/190] Revert "Add support for arrays in hashaggregate [databricks] (#6066)" (#6679) This reverts commit 122e107b977daa012479456794916e292aa1f6e4. Signed-off-by: Raza Jafri Signed-off-by: Raza Jafri Co-authored-by: Raza Jafri --- docs/supported_ops.md | 72 +++++++++---------- .../src/main/python/hash_aggregate_test.py | 15 +--- .../src/main/python/repart_test.py | 15 +--- .../nvidia/spark/rapids/GpuOverrides.scala | 29 ++------ .../com/nvidia/spark/rapids/aggregate.scala | 26 ++----- .../spark/sql/rapids/AggregateFunctions.scala | 12 ++-- tools/src/main/resources/supportedExprs.csv | 4 +- 7 files changed, 59 insertions(+), 114 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index bcc2036cdc8..c890ed89f74 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -556,7 +556,7 @@ Accelerator supports are described below. S NS NS -PS
not allowed for grouping expressions if containing Struct as child;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
+PS
not allowed for grouping expressions;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
PS
not allowed for grouping expressions;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
PS
not allowed for grouping expressions if containing Array or Map as child;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT
NS @@ -724,7 +724,7 @@ Accelerator supports are described below. S S NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
+PS
Round-robin partitioning is not supported if spark.sql.execution.sortBeforeRepartition is true;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
PS
Round-robin partitioning is not supported if spark.sql.execution.sortBeforeRepartition is true;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
PS
Round-robin partitioning is not supported for nested structs if spark.sql.execution.sortBeforeRepartition is true;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT
NS @@ -7690,45 +7690,45 @@ are limited. None project input + + + + + S S -S -S -S -S -S -S -PS
UTC is only supported TZ for TIMESTAMP
-S -S -S -S -S -PS
UTC is only supported TZ for child TIMESTAMP
-PS
UTC is only supported TZ for child TIMESTAMP
-PS
UTC is only supported TZ for child TIMESTAMP
-S + + + + + + + + + + + result + + + + + S S -S -S -S -S -S -S -PS
UTC is only supported TZ for TIMESTAMP
-S -S -S -S -S -PS
UTC is only supported TZ for child TIMESTAMP
-PS
UTC is only supported TZ for child TIMESTAMP
-PS
UTC is only supported TZ for child TIMESTAMP
-S + + + + + + + + + + + KnownNotNull @@ -18547,9 +18547,9 @@ as `a` don't show up in the table. They are controlled by the rules for S NS NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
NS -PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, MAP, UDT
+NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, ARRAY, MAP, UDT
NS diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index 672a0ecc60e..b1f378f9ba6 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -116,19 +116,6 @@ ('b', FloatGen(nullable=(True, 10.0), special_cases=[(float('nan'), 10.0)])), ('c', LongGen())] -# grouping single-level lists -_grpkey_list_with_non_nested_children = [[('a', RepeatSeqGen(ArrayGen(data_gen), length=3)), - ('b', IntegerGen())] for data_gen in all_basic_gens + decimal_gens] - -#grouping mutliple-level structs with arrays -_grpkey_nested_structs_with_array_basic_child = [ - ('a', RepeatSeqGen(StructGen([ - ['aa', IntegerGen()], - ['ab', ArrayGen(IntegerGen())]]), - length=20)), - ('b', IntegerGen()), - ('c', NullGen())] - _nan_zero_float_special_cases = [ (float('nan'), 5.0), (NEG_FLOAT_NAN_MIN_VALUE, 5.0), @@ -331,7 +318,7 @@ def test_hash_reduction_decimal_overflow_sum(precision): # some optimizations are conspiring against us. conf = {'spark.rapids.sql.batchSizeBytes': '128m'}) -@pytest.mark.parametrize('data_gen', [_grpkey_nested_structs_with_array_basic_child, _longs_with_nulls] + _grpkey_list_with_non_nested_children, ids=idfn) +@pytest.mark.parametrize('data_gen', [_longs_with_nulls], ids=idfn) def test_hash_grpby_sum_count_action(data_gen): assert_gpu_and_cpu_row_counts_equal( lambda spark: gen_df(spark, data_gen, length=100).groupby('a').agg(f.sum('b')) diff --git a/integration_tests/src/main/python/repart_test.py b/integration_tests/src/main/python/repart_test.py index b12a680d3eb..7b77b7be426 100644 --- a/integration_tests/src/main/python/repart_test.py +++ b/integration_tests/src/main/python/repart_test.py @@ -214,23 +214,10 @@ def test_round_robin_sort_fallback(data_gen): lambda spark : gen_df(spark, data_gen).withColumn('extra', lit(1)).repartition(13), 'ShuffleExchangeExec') -@allow_non_gpu("ProjectExec", "ShuffleExchangeExec") -@ignore_order(local=True) # To avoid extra data shuffle by 'sort on Spark' for this repartition test. -@pytest.mark.parametrize('num_parts', [2, 10, 17, 19, 32], ids=idfn) -@pytest.mark.parametrize('gen', [([('ag', ArrayGen(StructGen([('b1', long_gen)])))], ['ag'])], ids=idfn) -def test_hash_repartition_exact_fallback(gen, num_parts): - data_gen = gen[0] - part_on = gen[1] - assert_gpu_fallback_collect( - lambda spark : gen_df(spark, data_gen, length=1024) \ - .repartition(num_parts, *part_on) \ - .withColumn('id', f.spark_partition_id()) \ - .selectExpr('*'), "ShuffleExchangeExec") - @ignore_order(local=True) # To avoid extra data shuffle by 'sort on Spark' for this repartition test. @pytest.mark.parametrize('num_parts', [1, 2, 10, 17, 19, 32], ids=idfn) @pytest.mark.parametrize('gen', [ - ([('a', boolean_gen)], ['a']), + ([('a', boolean_gen)], ['a']), ([('a', byte_gen)], ['a']), ([('a', short_gen)], ['a']), ([('a', int_gen)], ['a']), diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 2c7d48ad685..d69f494ad6f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -1632,7 +1632,9 @@ object GpuOverrides extends Logging { }), expr[KnownFloatingPointNormalized]( "Tag to prevent redundant normalization", - ExprChecks.unaryProjectInputMatchesOutput(TypeSig.all, TypeSig.all), + ExprChecks.unaryProjectInputMatchesOutput( + TypeSig.DOUBLE + TypeSig.FLOAT, + TypeSig.DOUBLE + TypeSig.FLOAT), (a, conf, p, r) => new UnaryExprMeta[KnownFloatingPointNormalized](a, conf, p, r) { override def convertToGpu(child: Expression): GpuExpression = GpuKnownFloatingPointNormalized(child) @@ -3655,26 +3657,11 @@ object GpuOverrides extends Logging { // This needs to match what murmur3 supports. PartChecks(RepeatingParamCheck("hash_key", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.STRUCT + TypeSig.ARRAY).nested(), - TypeSig.all) - ), + TypeSig.STRUCT).nested(), TypeSig.all)), (hp, conf, p, r) => new PartMeta[HashPartitioning](hp, conf, p, r) { override val childExprs: Seq[BaseExprMeta[_]] = hp.expressions.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - override def tagPartForGpu(): Unit = { - val arrayWithStructsHashing = hp.expressions.exists(e => - TrampolineUtil.dataTypeExistsRecursively(e.dataType, - dt => dt match { - case ArrayType(_: StructType, _) => true - case _ => false - }) - ) - if (arrayWithStructsHashing) { - willNotWorkOnGpu("hashing arrays with structs is not supported") - } - } - override def convertToGpu(): GpuPartitioning = GpuHashPartitioning(childExprs.map(_.convertToGpu()), hp.numPartitions) }), @@ -3894,7 +3881,7 @@ object GpuOverrides extends Logging { .withPsNote(TypeEnum.STRUCT, "Round-robin partitioning is not supported for nested " + s"structs if ${SQLConf.SORT_BEFORE_REPARTITION.key} is true") .withPsNote( - Seq(TypeEnum.MAP), + Seq(TypeEnum.ARRAY, TypeEnum.MAP), "Round-robin partitioning is not supported if " + s"${SQLConf.SORT_BEFORE_REPARTITION.key} is true"), TypeSig.all), @@ -3956,12 +3943,10 @@ object GpuOverrides extends Logging { "The backend for hash based aggregations", ExecChecks( (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.MAP + TypeSig.STRUCT + TypeSig.ARRAY) + TypeSig.MAP + TypeSig.ARRAY + TypeSig.STRUCT) .nested() - .withPsNote(TypeEnum.MAP, + .withPsNote(Seq(TypeEnum.ARRAY, TypeEnum.MAP), "not allowed for grouping expressions") - .withPsNote(TypeEnum.ARRAY, - "not allowed for grouping expressions if containing Struct as child") .withPsNote(TypeEnum.STRUCT, "not allowed for grouping expressions if containing Array or Map as child"), TypeSig.all), diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala index c7754990d68..46853d542d7 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala @@ -42,7 +42,7 @@ import org.apache.spark.sql.execution.{ExplainUtils, SortExec, SparkPlan} import org.apache.spark.sql.execution.aggregate.{BaseAggregateExec, HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} import org.apache.spark.sql.rapids.{CpuToGpuAggregateBufferConverter, CudfAggregate, GpuAggregateExpression, GpuToCpuAggregateBufferConverter} import org.apache.spark.sql.rapids.execution.{GpuShuffleMeta, TrampolineUtil} -import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType} +import org.apache.spark.sql.types.{ArrayType, DataType, MapType} import org.apache.spark.sql.vectorized.ColumnarBatch object AggregateUtils { @@ -868,27 +868,13 @@ abstract class GpuBaseAggregateMeta[INPUT <: SparkPlan]( groupingExpressions ++ aggregateExpressions ++ aggregateAttributes ++ resultExpressions override def tagPlanForGpu(): Unit = { - // We don't support Maps as GroupBy keys yet, even if they are nested in Structs. So, + // We don't support Arrays and Maps as GroupBy keys yet, even they are nested in Structs. So, // we need to run recursive type check on the structs. - val mapGroupings = agg.groupingExpressions.exists(e => + val arrayOrMapGroupings = agg.groupingExpressions.exists(e => TrampolineUtil.dataTypeExistsRecursively(e.dataType, - dt => dt.isInstanceOf[MapType])) - if (mapGroupings) { - willNotWorkOnGpu("MapTypes in grouping expressions are not supported") - } - - // We support Arrays as grouping expression but not if the child is a struct. So we need to - // run recursive type check on the lists of structs - val arrayWithStructsGroupings = agg.groupingExpressions.exists(e => - TrampolineUtil.dataTypeExistsRecursively(e.dataType, - dt => dt match { - case ArrayType(_: StructType, _) => true - case _ => false - }) - ) - if (arrayWithStructsGroupings) { - willNotWorkOnGpu("ArrayTypes with Struct children in grouping expressions are not " + - "supported") + dt => dt.isInstanceOf[ArrayType] || dt.isInstanceOf[MapType])) + if (arrayOrMapGroupings) { + willNotWorkOnGpu("ArrayTypes or MapTypes in grouping expressions are not supported") } tagForReplaceMode() diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala index a01a649b88f..affe29a776f 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala @@ -559,7 +559,7 @@ case class GpuBasicMin(child: Expression) extends GpuMin(child) */ case class GpuFloatMin(child: Expression) extends GpuMin(child) with GpuReplaceWindowFunction { - + override val dataType: DataType = child.dataType match { case FloatType | DoubleType => child.dataType case t => throw new IllegalStateException(s"child type $t is not FloatType or DoubleType") @@ -606,7 +606,7 @@ case class GpuFloatMin(child: Expression) extends GpuMin(child) // Else return the min value override lazy val postUpdate: Seq[Expression] = Seq( GpuIf( - updateAllNansOrNulls.attr, + updateAllNansOrNulls.attr, GpuIf( updateHasNan.attr, GpuLiteral(nan, dataType), GpuLiteral(null, dataType) ), @@ -668,7 +668,7 @@ object GpuMax { abstract class GpuMax(child: Expression) extends GpuAggregateFunction with GpuBatchedRunningWindowWithFixer with GpuAggregateWindowFunction - with GpuRunningWindowFunction + with GpuRunningWindowFunction with Serializable { override lazy val initialValues: Seq[GpuLiteral] = Seq(GpuLiteral(null, child.dataType)) override lazy val inputProjection: Seq[Expression] = Seq(child) @@ -730,7 +730,7 @@ case class GpuBasicMax(child: Expression) extends GpuMax(child) * column `isNan`. If any value in this column is true, return `Nan`, * Else, return what `GpuBasicMax` returns. */ -case class GpuFloatMax(child: Expression) extends GpuMax(child) +case class GpuFloatMax(child: Expression) extends GpuMax(child) with GpuReplaceWindowFunction{ override val dataType: DataType = child.dataType match { @@ -756,13 +756,13 @@ case class GpuFloatMax(child: Expression) extends GpuMax(child) override lazy val updateAggregates: Seq[CudfAggregate] = Seq(updateMaxVal, updateIsNan) // If there is `Nan` value in the target column, return `Nan` // else return what the `CudfMax` returns - override lazy val postUpdate: Seq[Expression] = + override lazy val postUpdate: Seq[Expression] = Seq( GpuIf(updateIsNan.attr, GpuLiteral(nan, dataType), updateMaxVal.attr) ) // Same logic as the `inputProjection` stage. - override lazy val preMerge: Seq[Expression] = + override lazy val preMerge: Seq[Expression] = Seq(evaluateExpression, GpuIsNan(evaluateExpression)) // Same logic as the `updateAggregates` stage. override lazy val mergeAggregates: Seq[CudfAggregate] = Seq(mergeMaxVal, mergeIsNan) diff --git a/tools/src/main/resources/supportedExprs.csv b/tools/src/main/resources/supportedExprs.csv index 25748ccecf4..def02f728b4 100644 --- a/tools/src/main/resources/supportedExprs.csv +++ b/tools/src/main/resources/supportedExprs.csv @@ -261,8 +261,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -KnownFloatingPointNormalized,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S -KnownFloatingPointNormalized,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S +KnownFloatingPointNormalized,S, ,None,project,input,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +KnownFloatingPointNormalized,S, ,None,project,result,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA KnownNotNull,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,NS,S,S,PS,PS,PS,NS KnownNotNull,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,NS,S,S,PS,PS,PS,NS Lag,S,`lag`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS From ce7c07662682e841f441d4686c6121580849e194 Mon Sep 17 00:00:00 2001 From: Mike Wilson Date: Tue, 4 Oct 2022 15:09:16 -0400 Subject: [PATCH 164/190] String to decimal casting custom kernel (#6616) * Added string to decimal casting in a single kernel call Signed-off-by: Mike Wilson --- .../com/nvidia/spark/rapids/GpuCast.scala | 50 ++----------------- .../rapids/GpuTextBasedPartitionReader.scala | 3 +- 2 files changed, 5 insertions(+), 48 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala index 8e35b157eab..a04c26f794e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala @@ -25,6 +25,7 @@ import scala.collection.mutable.ArrayBuffer import ai.rapids.cudf.{BinaryOp, ColumnVector, ColumnView, DecimalUtils, DType, Scalar} import ai.rapids.cudf import com.nvidia.spark.rapids.RapidsPluginImplicits._ +import com.nvidia.spark.rapids.jni.CastStrings import com.nvidia.spark.rapids.shims.{AnsiUtil, GpuIntervalUtils, GpuTypeShims, SparkShimImpl, YearParseUtil} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult @@ -453,7 +454,7 @@ object GpuCast extends Arm { case (FloatType | DoubleType, StringType) => castFloatingTypeToString(input) case (StringType, ByteType | ShortType | IntegerType | LongType ) => - com.nvidia.spark.rapids.jni.CastStrings.toInteger(input, ansiMode, + CastStrings.toInteger(input, ansiMode, GpuColumnVector.getNonNestedRapidsType(toDataType)) case (StringType, BooleanType | FloatType | DoubleType | DateType | TimestampType) => withResource(input.strip()) { trimmed => @@ -474,7 +475,7 @@ object GpuCast extends Arm { } } case (StringType, dt: DecimalType) => - castStringToDecimal(input, ansiMode, dt) + CastStrings.toDecimal(input, ansiMode, dt.precision, -dt.scale) case (ByteType | ShortType | IntegerType | LongType, dt: DecimalType) => castIntegralsToDecimal(input, dt, ansiMode) @@ -984,51 +985,6 @@ object GpuCast extends Arm { } } - def castStringToDecimal( - input: ColumnView, - ansiEnabled: Boolean, - dt: DecimalType): ColumnVector = { - // 1. Sanitize strings to make sure all are fixed points - // 2. Identify all fixed point values - // 3. Cast String to newDt (newDt = dt. precision + 1, dt.scale + 1). Promote precision if - // needed. This step is required so we can round up if needed in the final step - // 4. Now cast newDt to dt (Decimal to Decimal) - def getInterimDecimalPromoteIfNeeded(dt: DecimalType): DecimalType = { - if (dt.precision + 1 > DecimalType.MAX_PRECISION) { - throw new IllegalArgumentException("One or more values exceed the maximum supported " + - "Decimal precision while conversion") - } - DecimalType(dt.precision + 1, dt.scale + 1) - } - - val interimSparkDt = getInterimDecimalPromoteIfNeeded(dt) - val interimDt = DecimalUtil.createCudfDecimal(interimSparkDt) - val isFixedPoints = withResource(input.strip()) { - // We further filter out invalid values using the cuDF isFixedPoint method. - _.isFixedPoint(interimDt) - } - - withResource(isFixedPoints) { isFixedPoints => - if (ansiEnabled) { - withResource(isFixedPoints.all()) { allFixedPoints => - if (allFixedPoints.isValid && !allFixedPoints.getBoolean) { - throw new ArithmeticException(s"One or more values cannot be " + - s"represented as Decimal(${dt.precision}, ${dt.scale})") - } - } - } - // intermediate step needed so we can make sure we can round up - withResource(input.castTo(interimDt)) { interimDecimals => - withResource(Scalar.fromNull(interimDt)) { nulls => - withResource(isFixedPoints.ifElse(interimDecimals, nulls)) { decimals => - // cast Decimal to the Decimal that's needed - castDecimalToDecimal(decimals, interimSparkDt, dt, ansiEnabled) - } - } - } - } - } - def castStringToFloats( input: ColumnVector, ansiEnabled: Boolean, diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala index 64b45bca054..136fee970a9 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala @@ -23,6 +23,7 @@ import scala.math.max import ai.rapids.cudf.{ColumnVector, DType, HostMemoryBuffer, NvtxColor, NvtxRange, Scalar, Schema, Table} import com.nvidia.spark.rapids.DateUtils.{toStrf, TimestampFormatConversionException} +import com.nvidia.spark.rapids.jni.CastStrings import com.nvidia.spark.rapids.shims.GpuTypeShims import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -393,7 +394,7 @@ abstract class GpuTextBasedPartitionReader( } def castStringToDecimal(input: ColumnVector, dt: DecimalType): ColumnVector = { - GpuCast.castStringToDecimal(input, ansiEnabled = false, dt) + CastStrings.toDecimal(input, false, dt.precision, -dt.scale) } def castStringToInt(input: ColumnVector, intType: DType): ColumnVector = { From 9c7b1c06d16d1190ca4ad4f0fb307dff711b9a5e Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Tue, 4 Oct 2022 16:06:56 -0500 Subject: [PATCH 165/190] Revise AutoTuner to match the BootStrap tool (#6652) * Revise AutoTuner to match the BootStrap Signed-off-by: Ahmed Hussein (amahussein) * handle constructorException Signed-off-by: Ahmed Hussein (amahussein) * fix imports to pass styling checks Signed-off-by: Ahmed Hussein (amahussein) * finish remaining todo Signed-off-by: Ahmed Hussein (amahussein) * add unit tests and code reviews Signed-off-by: Ahmed Hussein (amahussein) * address PR comments Signed-off-by: Ahmed Hussein (amahussein) * fix typos in toString() Signed-off-by: Ahmed Hussein (amahussein) * finish remaining comments Signed-off-by: Ahmed Hussein (amahussein) * address typos in comments messages Signed-off-by: Ahmed Hussein (amahussein) * address typos in comments messages-2 Signed-off-by: Ahmed Hussein (amahussein) * handle missing information in worker info Signed-off-by: Ahmed Hussein (amahussein) * add unit tests for missing workers Signed-off-by: Ahmed Hussein (amahussein) * turn off scala style for static lines exceeding 100 Signed-off-by: Ahmed Hussein (amahussein) * fix typo in the autotuner Signed-off-by: Ahmed Hussein (amahussein) Signed-off-by: Ahmed Hussein (amahussein) --- tools/scripts/discoveryScript.sh | 10 +- .../rapids/tool/profiling/AutoTuner.scala | 1168 +++++++++++------ .../tool/profiling/ClassWarehouse.scala | 4 - .../rapids/tool/profiling/ProfileArgs.scala | 6 +- .../rapids/tool/profiling/Profiler.scala | 40 +- .../tool/profiling/AutoTunerSuite.scala | 351 +++++ 6 files changed, 1158 insertions(+), 421 deletions(-) create mode 100644 tools/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AutoTunerSuite.scala diff --git a/tools/scripts/discoveryScript.sh b/tools/scripts/discoveryScript.sh index cff13f7ffc6..b63aa746785 100644 --- a/tools/scripts/discoveryScript.sh +++ b/tools/scripts/discoveryScript.sh @@ -106,12 +106,12 @@ function read_system_properties() { function write_system_properties() { cat > "$OUTPUT_FILE" << EOF system: - num_cores: $numCores - cpu_arch: $cpuArch + numCores: $numCores + cpuArch: $cpuArch memory: $memInGb - free_disk_space: $freeDiskSpaceInGb - time_zone: $timeZone - num_workers: $NUM_WORKERS + freeDiskSpace: $freeDiskSpaceInGb + timeZone: $timeZone + numWorkers: $NUM_WORKERS gpu: count: $gpuCount memory: $gpuMemoryInMb diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala index dabd10019ca..039e3586097 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala @@ -16,92 +16,242 @@ package com.nvidia.spark.rapids.tool.profiling -import java.io.FileNotFoundException +import java.io.{BufferedReader, InputStreamReader} +import java.util -import scala.collection.JavaConverters._ +import scala.beans.BeanProperty +import scala.collection.JavaConverters.mapAsScalaMapConverter +import scala.collection.mutable +import scala.collection.mutable.ListBuffer import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FSDataInputStream, Path} import org.yaml.snakeyaml.Yaml +import org.yaml.snakeyaml.constructor.{Constructor, ConstructorException} +import org.yaml.snakeyaml.representer.Representer import org.apache.spark.internal.Logging /** - * A wrapper class that stores all the properties that would be recommended by the Auto-tuner. - * Separate getter and setter methods are specified for each property for ease of access. + * A wrapper class that stores all the GPU properties. + * The BeanProperty enables loading and parsing the YAML formatted content using the + * Constructor SnakeYaml approach. */ -class Config { - private val properties: collection.mutable.Map[String, String] = collection.mutable.Map( - "spark.executor.instances" -> null, - "spark.rapids.sql.enabled" -> null, - "spark.executor.cores" -> null, - "spark.executor.memory" -> null, - "spark.rapids.sql.concurrentGpuTasks" -> null, - "spark.task.resource.gpu.amount" -> null, - "spark.sql.shuffle.partitions" -> null, - "spark.sql.files.maxPartitionBytes" -> null, - "spark.rapids.memory.pinnedPool.size" -> null, - "spark.executor.memoryOverhead" -> null - ) - - def getExecutorInstances: Int = properties("spark.executor.instances").toInt - - def getExecutorCores: Int = properties("spark.executor.cores").toInt - - def getExecutorMemory: String = properties("spark.executor.memory") - - def getConcurrentGpuTasks: Int = properties("spark.rapids.sql.concurrentGpuTasks").toInt - - def getTaskResourceGpu: Double = properties("spark.task.resource.gpu.amount").toDouble - - def getShufflePartitions: Int = properties("spark.sql.shuffle.partitions").toInt - - def getMaxPartitionBytes: String = properties("spark.sql.files.maxPartitionBytes") +class GpuWorkerProps( + @BeanProperty var memory: String, + @BeanProperty var count: Int, + @BeanProperty var name: String) { + def this() { + this("0m", 0, "None") + } + def isMissingInfo: Boolean = { + count == 0 || memory.startsWith("0") || name == "None" + } + def isEmpty: Boolean = { + count == 0 && memory.startsWith("0") && name == "None" + } + /** + * If the GPU count is missing, it will set 1 as a default value + * + * @return true if the value has been updated. + */ + def setDefaultGpuCountIfMissing: Boolean = { + if (count == 0) { + count = AutoTuner.DEF_WORKER_GPU_COUNT + true + } else { + false + } + } + def setDefaultGpuNameIfMissing: Boolean = { + if (name == "None") { + name = AutoTuner.DEF_WORKER_GPU_NAME + true + } else { + false + } + } - def getPinnedPoolSize: String = properties("spark.rapids.memory.pinnedPool.size") + /** + * If the GPU memory is missing, it will sets a default valued based on the GPU device and the + * static HashMap [[AutoTuner.DEF_WORKER_GPU_MEMORY_MB]]. + * If it is still missing, it sets a default to 16384m. + * + * @return true if the value has been updated. + */ + def setDefaultGpuMemIfMissing: Boolean = { + if (memory.startsWith("0")) { + memory = AutoTuner.DEF_WORKER_GPU_MEMORY_MB.getOrElse(getName, "16384m") + true + } else { + false + } + } - def getExecutorMemoryOverhead: String = properties("spark.executor.memoryOverhead") + /** + * Sets any missing field and return a list of messages to indicate what has been updated. + * @return a list containing information of what was missing and the default value that has been + * used to initialize the field. + */ + def setMissingFields(): Seq[String] = { + val res = new ListBuffer[String]() + if (setDefaultGpuCountIfMissing) { + res += s"GPU count is missing. Setting default to $getCount." + } + if (setDefaultGpuNameIfMissing) { + res += s"GPU device is missing. Setting default to $getName." + } + if (setDefaultGpuMemIfMissing) { + res += s"GPU memory is missing. Setting default to $getMemory." + } + res + } - def getAllProperties: Map[String, String] = properties.toMap + override def toString: String = + s"{count: $count, memory: $memory, name: $name}" +} - def setExecutorInstances(numInstances: Int): Unit = { - properties("spark.executor.instances") = numInstances.toString +/** + * A wrapper class that stores all the system properties. + * The BeanProperty enables loading and parsing the YAML formatted content using the + * Constructor SnakeYaml approach. + */ +class SystemClusterProps( + @BeanProperty var numCores: Int, + @BeanProperty var memory: String, + @BeanProperty var numWorkers: Int) { + def this() { + this(0, "0m", 0) + } + def isMissingInfo: Boolean = { + // keep for future expansion as we may add more fields later. + numWorkers <= 0 + } + def isEmpty: Boolean = { + // consider the object incorrect if either numCores or memory are not set. + numCores <= 0 || memory.startsWith("0") + } + def setDefaultNumWorkersIfMissing(): Boolean = { + if (numWorkers <= 0) { + numWorkers = AutoTuner.DEF_NUM_WORKERS + true + } else { + false + } } - - def setExecutorCores(executorCores: Int): Unit = { - properties("spark.executor.cores") = executorCores.toString + /** + * Sets any missing field and return a list of messages to indicate what has been updated. + * @return a list containing information of what was missing and the default value that has been + * used to initialize the field. + */ + def setMissingFields(): Seq[String] = { + val res = new ListBuffer[String]() + if (setDefaultNumWorkersIfMissing()) { + res += s"Number of workers is missing. Setting default to $getNumWorkers." + } + res } + override def toString: String = + s"{numCores: $numCores, memory: $memory, numWorkers: $numWorkers}" +} - def setExecutorMemory(executorMemory: String): Unit = { - properties("spark.executor.memory") = executorMemory - } +/** + * A wrapper class that stores all the properties of the cluster. + * The BeanProperty enables loading and parsing the YAML formatted content using the + * Constructor SnakeYaml approach. + * + * @param system wrapper that includes the properties related to system information like cores and + * memory. + * @param gpu wrapper that includes the properties related to GPU. + * @param softwareProperties a set of software properties such as Spark properties. + * The properties are typically loaded from the default cluster + * configurations. + */ +class ClusterProperties( + @BeanProperty var system: SystemClusterProps, + @BeanProperty var gpu: GpuWorkerProps, + @BeanProperty var softwareProperties: util.LinkedHashMap[String, String]) { - def setConcurrentGpuTasks(concurrentGpuTasks: Int): Unit = { - properties("spark.rapids.sql.concurrentGpuTasks") = concurrentGpuTasks.toString - } + import AutoTuner._ - def setTaskResourceGpu(taskResourceGpu: Double): Unit = { - properties("spark.task.resource.gpu.amount") = taskResourceGpu.toString + def this() { + this(new SystemClusterProps(), new GpuWorkerProps(), new util.LinkedHashMap[String, String]()) } - - def setShufflePartitions(shufflePartitions: Int): Unit = { - properties("spark.sql.shuffle.partitions") = shufflePartitions.toString + def isEmpty: Boolean = { + system.isEmpty && gpu.isEmpty + } + def getTargetProperties: mutable.Map[String, String] = { + softwareProperties.asScala.filter(entry => recommendationsTarget.contains(entry._1)) } + override def toString: String = + s"{${system.toString}, ${gpu.toString}, $softwareProperties}" +} - def setMaxPartitionBytes(maxPartitionBytes: String): Unit = { - properties("spark.sql.files.maxPartitionBytes") = maxPartitionBytes +/** + * Wrapper to hold the recommendation of a given criterion. + * + * @param name the property label. + * @param original the value loaded from the spark properties. + * @param recommended the recommended value by the AutoTuner. + */ +class RecommendationEntry(val name: String, + val original: Option[String], + var recommended: Option[String]) { + + def setRecommendedValue(value: String): Unit = { + recommended = Option(value) } - def setPinnedPoolSize(pinnedPoolSize: String): Unit = { - properties("spark.rapids.memory.pinnedPool.size") = pinnedPoolSize + /** + * Used to compare between two properties by converting memory units to + * a equivalent representations. + * @param propValue property to be processed. + * @return the uniform representation of property. + * For Memory, the value is converted to bytes. + */ + private def getRawValue(propValue: Option[String]): Option[String] = { + propValue match { + case None => None + case Some(value) => + if (AutoTuner.containsMemoryUnits(value)) { + // if it is memory return the bytes unit + Some(s"${AutoTuner.convertFromHumanReadableSize(value)}") + } else { + propValue + } + } } - def setExecutorMemoryOverhead(executorMemoryOverhead: String): Unit = { - properties("spark.executor.memoryOverhead") = executorMemoryOverhead + /** + * Returns true when the recommendation is different than the original. + */ + private def recommendsNewValue(): Boolean = { + val originalVal = getRawValue(original) + val recommendedVal = getRawValue(recommended) + (originalVal, recommendedVal) match { + case (None, None) => false + case (Some(orig), Some(rec)) => + orig != rec + case _ => true + } } - def setExecutorMemoryOverheadFactor(executorMemoryOverheadFactor: Double): Unit = { - properties("spark.executor.memoryOverheadFactor") = executorMemoryOverheadFactor.toString + /** + * True or False whether the recommendation is valid. e.g., recommendations that does not change + * the original value returns false if filter is enabled. + * @param filterByUpdated flag to pick only the properties that would be updated by the + * recommendations + */ + def isValid(filterByUpdated: Boolean): Boolean = { + recommended match { + case None => false + case _ => + if (filterByUpdated) { // filter enabled + recommendsNewValue() + } else { + true + } + } } } @@ -109,7 +259,7 @@ class Config { * AutoTuner module that uses event logs and worker's system properties to recommend Spark * RAPIDS configuration based on heuristics. * - * Example (Refer to test suite for more cases): + * Example: * a. Success: * Input: * system: @@ -123,212 +273,384 @@ class Config { * count: 8 * memory: 32gb * name: NVIDIA V100 + * softwareProperties: + * spark.driver.maxResultSize: 7680m + * spark.driver.memory: 15360m + * spark.executor.cores: '8' + * spark.executor.instances: '2' + * spark.executor.memory: 47222m + * spark.executorEnv.OPENBLAS_NUM_THREADS: '1' + * spark.extraListeners: com.google.cloud.spark.performance.DataprocMetricsListener + * spark.scheduler.mode: FAIR + * spark.sql.cbo.enabled: 'true' + * spark.ui.port: '0' + * spark.yarn.am.memory: 640m * * Output: * Spark Properties: * --conf spark.executor.cores=8 - * --conf spark.executor.instances=32 - * --conf spark.executor.memory=63.75g - * --conf spark.executor.memoryOverhead=8.38g - * --conf spark.rapids.memory.pinnedPool.size=2g - * --conf spark.rapids.sql.concurrentGpuTasks=4 - * --conf spark.sql.files.maxPartitionBytes=4g - * --conf spark.sql.shuffle.partitions=200 + * --conf spark.executor.instances=20 + * --conf spark.executor.memory=16384m + * --conf spark.executor.memoryOverhead=5734m + * --conf spark.rapids.memory.pinnedPool.size=4096m + * --conf spark.rapids.sql.concurrentGpuTasks=2 + * --conf spark.sql.files.maxPartitionBytes=4096m * --conf spark.task.resource.gpu.amount=0.125 * + * Comments: + * - 'spark.rapids.sql.concurrentGpuTasks' was not set. + * - 'spark.executor.memoryOverhead' was not set. + * - 'spark.rapids.memory.pinnedPool.size' was not set. + * - 'spark.sql.adaptive.enabled' should be enabled for better performance. + * * b. Failure: - * Input: Empty or Incorrect System Properties + * Input: Incorrect File * Output: + * Cannot recommend properties. See Comments. + * * Comments: + * - java.io.FileNotFoundException: File worker_info.yaml does not exist * - 'spark.executor.memory' should be set to at least 2GB/core. - * - 'spark.executor.instances' should be set to 'num_gpus * num_workers'. - * - 'spark.task.resource.gpu.amount' should be set to 1/#cores. + * - 'spark.executor.instances' should be set to (gpuCount * numWorkers). + * - 'spark.task.resource.gpu.amount' should be set to Max(1, (numCores / gpuCount)). + * - 'spark.rapids.sql.concurrentGpuTasks' should be set to Max(4, (gpuMemory / 8G)). + * - 'spark.rapids.memory.pinnedPool.size' should be set to 2048m. + * - 'spark.sql.adaptive.enabled' should be enabled for better performance. + * + * @param clusterProps The cluster properties including cores, mem, GPU, and software + * (see [[ClusterProperties]]). + * @param appInfo the container holding the profiling result. */ -class AutoTuner(app: ApplicationSummaryInfo, workerInfo: String) extends Logging { - import AutoTuner._ - val DEFAULT_SHUFFLE_PARTITION_MULTIPLIER: Int = 2 - val MAX_JVM_GCTIME_FRACTION: Double = 0.3 +class AutoTuner( + val clusterProps: ClusterProperties, + val appInfo: Option[ApplicationSummaryInfo]) extends Logging { - val DEFAULT_CONCURRENT_GPU_TASKS: Int = 2 - val DEFAULT_CONCURRENT_GPU_TASKS_MULTIPLIER: Double = 0.125 // Currently aggressively set to 1/8 - val MAX_CONCURRENT_GPU_TASKS: Int = 4 + import AutoTuner._ - val DEFAULT_MAX_PARTITION_BYTES: String = "512m" - val MAX_PARTITION_BYTES_BOUND: String = "4g" - val MAX_PARTITION_BYTES_RANGE: String = "256m" - val MIN_PARTITION_BYTES_RANGE: String = "128m" + var comments = new ListBuffer[String]() + var recommendations: mutable.LinkedHashMap[String, RecommendationEntry] = + mutable.LinkedHashMap[String, RecommendationEntry]() + // list of recommendations to be skipped for recommendations + // Note that the recommendations will be computed anyway to avoid breaking dependencies. + private val skippedRecommendations: mutable.HashSet[String] = mutable.HashSet[String]() + // list of recommendations having the calculations disabled, and only depend on default values + private val limitedLogicRecommendations: mutable.HashSet[String] = mutable.HashSet[String]() + // When enabled, the profiler recommendations should only include updated settings. + private var filterByUpdatedPropertiesEnabled: Boolean = true + + private def isCalculationEnabled(prop: String) : Boolean = { + !limitedLogicRecommendations.contains(prop) + } + + private def findPropertyInProfPropertyResults( + key: String, + props: Seq[RapidsPropertyProfileResult]): Option[String] = { + props.collectFirst { + case entry: RapidsPropertyProfileResult + if entry.key == key && entry.rows(1) != "null" => entry.rows(1) + } + } + def getSparkPropertyFromProfile(propKey: String): Option[String] = { + appInfo match { + case None => None + case Some(profInfo) => + val resFromProfSparkProps = findPropertyInProfPropertyResults(propKey, profInfo.sparkProps) + resFromProfSparkProps match { + case None => findPropertyInProfPropertyResults(propKey, profInfo.rapidsProps) + case Some(_) => resFromProfSparkProps + } + } + } - val DEFAULT_PINNED_POOL_SIZE: String = "2g" - val DEFAULT_MEMORY_OVERHEAD_FACTOR: Double = 0.1 - val DEFAULT_SYSTEM_OVERHEAD: String = "2g" // Overhead of other system processes + def getPropertyValue(key: String): Option[String] = { + val fromProfile = getSparkPropertyFromProfile(key) + fromProfile match { + case None => Option(clusterProps.softwareProperties.get(key)) + case Some(_) => fromProfile + } + } - val MAX_PER_EXECUTOR_CORE_COUNT: Int = 16 - val MIN_PER_EXECUTOR_CORE_COUNT: Int = 4 + def initRecommendations(): Unit = { + recommendationsTarget.foreach { key => + // no need to add new records if they are missing from props + getPropertyValue(key).foreach { propVal => + val recommendationVal = new RecommendationEntry(key, Option(propVal), None) + recommendations(key) = recommendationVal + } + } + } - val MAX_EXECUTOR_MEMORY: String = "64g" - val MIN_EXECUTOR_MEMORY: String = "8g" + def appendRecommendation(key: String, value: String): Unit = { + if (!skippedRecommendations.contains(key)) { + val recomRecord = recommendations.getOrElseUpdate(key, + new RecommendationEntry(key, getPropertyValue(key), None)) + if (value != null) { + recomRecord.setRecommendedValue(value) + if (recomRecord.original.isEmpty) { + // add a comment that the value was missing in the cluster properties + appendComment(s"'$key' was not set.") + } + } + } + } - var comments: Seq[String] = Seq() + /** + * Safely appends the recommendation to the given key. + * It skips if the value is 0. + */ + def appendRecommendation(key: String, value: Int): Unit = { + if (value > 0) { + appendRecommendation(key: String, s"$value") + } + } /** - * Recommendation for 'spark.executor.instances' based on number of gpus and workers. - * Assumption - In case GPU properties are not available, it assumes 1 GPU per worker. + * Safely appends the recommendation to the given key. + * It skips if the value is 0.0. */ - private def recommendExecutorInstances(recommendedConfig: Config, - systemProps: SystemProps): Unit = { - systemProps.numWorkers match { - case Some(numWorkers) => - val numInstances = if (systemProps.gpuProps != null) { - numWorkers * systemProps.gpuProps.count - } else { - // Assumption: 1 GPU per worker. - numWorkers - } + def appendRecommendation(key: String, value: Double): Unit = { + if (value > 0.0) { + appendRecommendation(key: String, s"$value") + } + } + /** + * Safely appends the recommendation to the given key. + * It appends "m" to the string value. It skips if the value is 0 or null. + */ + def appendRecommendationForMemoryMB(key: String, value: String): Unit = { + if (value != null && value.toDouble > 0.0) { + appendRecommendation(key, s"${value}m") + } + } - recommendedConfig.setExecutorInstances(numInstances) - case None => - val num_gpus_str = if (systemProps.gpuProps != null) { - systemProps.gpuProps.count.toString - } else { - "num_gpus" - } + /** + * calculated 'spark.executor.instances' based on number of gpus and workers. + * Assumption - cluster properties were updated to have a default values if missing. + */ + def calcExecInstances(): Int = { + clusterProps.gpu.getCount * clusterProps.system.numWorkers + } - comments :+= s"'spark.executor.instances' should be set to $num_gpus_str * num_workers." + /** + * Recommendation for 'spark.executor.instances' based on number of gpus and workers. + * Assumption - If the properties include "spark.dynamicAllocation.enabled=true", then ignore + * spark.executor.instances. + */ + def recommendExecutorInstances(): Unit = { + val execInstancesOpt = getPropertyValue("spark.dynamicAllocation.enabled") match { + case Some(propValue) => + if (propValue.toBoolean) { + None + } else { + Option(calcExecInstances()) + } + case None => Option(calcExecInstances()) + } + if (execInstancesOpt.isDefined) { + appendRecommendation("spark.executor.instances", execInstancesOpt.get) } } /** * Recommendation for 'spark.executor.cores' based on number of cpu cores and gpus. + * Assumption - cluster properties were updated to have a default values if missing. */ - private def recommendExecutorCores(recommendedConfig: Config, - systemProps: SystemProps): Unit = { - val numCores: Int = if (systemProps.gpuProps != null) { - Math.min(systemProps.numCores * 1.0 / systemProps.gpuProps.count, - MAX_PER_EXECUTOR_CORE_COUNT).toInt - } else { - systemProps.numCores - } + def calcNumExecutorCores: Int = { + // clusterProps.gpu.getCount can never be 0. This is verified in processPropsAndCheck() + val executorsPerNode = clusterProps.gpu.getCount + Math.max(1, clusterProps.system.getNumCores / executorsPerNode) + } - if (numCores < MIN_PER_EXECUTOR_CORE_COUNT) { - comments :+= s"Number of cores per executor is very low. " + - s"It is recommended to have at least $MIN_PER_EXECUTOR_CORE_COUNT cores per executor." - } + /** + * Recommendation for 'spark.task.resource.gpu.amount' based on num of cpu cores. + */ + def calcTaskGPUAmount(numExecCoresCalculator: () => Int): Double = { + val numExecutorCores = numExecCoresCalculator.apply() + // can never be 0 since numExecutorCores has to be at least 1 + 1.0 / numExecutorCores + } - if (systemProps.numWorkers.nonEmpty) { - val numInstances = recommendedConfig.getExecutorInstances - if (numCores * numInstances < systemProps.numCores) { - comments :+= "Not all cores in the machine are being used. " + - "It is recommended to use different machine." - } - } + /** + * Recommendation for 'spark.rapids.sql.concurrentGpuTasks' based on gpu memory. + * Assumption - cluster properties were updated to have a default values if missing. + */ + def calcGpuConcTasks(): Long = { + Math.min(MAX_CONC_GPU_TASKS, + convertToMB(clusterProps.gpu.memory) / DEF_GPU_MEM_PER_TASK_MB) + } - recommendedConfig.setExecutorCores(numCores) + /** + * Calculates the available memory for each executor on the worker based on the number of + * executors per node and the memory. + * Assumption - cluster properties were updated to have a default values if missing. + */ + private def calcAvailableMemPerExec(): Double = { + // account for system overhead + val usableWorkerMem = + Math.max(0, convertToMB(clusterProps.system.memory) - DEF_SYSTEM_RESERVE_MB) + // clusterProps.gpu.getCount can never be 0. This is verified in processPropsAndCheck() + (1.0 * usableWorkerMem) / clusterProps.gpu.getCount } /** * Recommendation for 'spark.executor.memory' based on system memory, cluster scheduler * and num of gpus. - * Note - This method requires recommendedConfig to have 'spark.executor.cores' set. */ - private def recommendExecutorMemory(recommendedConfig: Config, - systemProps: SystemProps): Unit = { - val systemMemoryNum: Long = convertFromHumanReadableSize(systemProps.memory) - val systemOverhead: Long = convertFromHumanReadableSize(DEFAULT_SYSTEM_OVERHEAD) - val effectiveSystemMemoryNum: Long = systemMemoryNum - systemOverhead - - convertFromHumanReadableSize(DEFAULT_PINNED_POOL_SIZE) - val maxExecutorMemNum: Long = convertFromHumanReadableSize(MAX_EXECUTOR_MEMORY) - - val executorMemory: Long = if (systemProps.gpuProps != null) { - Math.min(effectiveSystemMemoryNum * 1.0 / systemProps.gpuProps.count, - maxExecutorMemNum).toLong - } else { - Math.min(effectiveSystemMemoryNum * 1.0 / recommendedConfig.getExecutorCores, - maxExecutorMemNum).toLong - } + def calcExecutorHeap(executorContainerMemCalculator: () => Double, + numExecCoresCalculator: () => Int): Long = { + // reserve 10% of heap as memory overhead + val maxExecutorHeap = Math.max(0, + executorContainerMemCalculator() * (1 - DEF_HEAP_OVERHEAD_FRACTION)).toInt + // give up to 2GB of heap to each executor core + Math.min(maxExecutorHeap, DEF_HEAP_PER_CORE_MB * numExecCoresCalculator()) + } - if(executorMemory < convertFromHumanReadableSize(MIN_EXECUTOR_MEMORY)) { - comments :+= s"Executor memory is very low. " + - s"It is recommended to have at least $MIN_EXECUTOR_MEMORY." + /** + * Recommendation for 'spark.rapids.memory.pinnedPool.size' and 'spark.executor.memoryOverhead' + * based on executor memory. + */ + def calcPinnedMemoryWithOverhead( + execHeapCalculator: () => Long, + containerMemCalculator: () => Double): (Long, Long) = { + val executorHeap = execHeapCalculator() + var executorMemOverhead = (executorHeap * DEF_HEAP_OVERHEAD_FRACTION).toLong + // pinned memory uses any unused space up to 4GB + val pinnedMem = Math.min(MAX_PINNED_MEMORY_MB, + containerMemCalculator.apply() - executorHeap - executorMemOverhead).toLong + executorMemOverhead += pinnedMem + (pinnedMem, executorMemOverhead) + } + + private def getSparkVersion: Option[String] = { + appInfo match { + case Some(app) => Option(app.appInfo.head.sparkVersion) + case None => None } - - recommendedConfig.setExecutorMemory(convertToHumanReadableSize(executorMemory)) } /** - * Calculate memory overhead as: pinnedPoolSize + (memoryOverheadFactor * executorMemory) + * Find the label of the memory.overhead based on the spark master configuration and the spark + * version. + * @return "spark.executor.memoryOverhead", "spark.kubernetes.memoryOverheadFactor", + * or "spark.executor.memoryOverheadFactor". */ - private def calculateMemoryOverhead(pinnedPoolSize: String, executorMemory: String): Long = { - val pinnedPoolSizeNum = convertFromHumanReadableSize(pinnedPoolSize) - val executorMemoryNum = convertFromHumanReadableSize(executorMemory) - (pinnedPoolSizeNum + DEFAULT_MEMORY_OVERHEAD_FACTOR * executorMemoryNum).toLong + def memoryOverheadLabel: String = { + val sparkMasterConf = getPropertyValue("spark.master") + val defaultLabel = "spark.executor.memoryOverhead" + sparkMasterConf match { + case None => defaultLabel + case Some(sparkMaster) => + if (sparkMaster.contains("yarn")) { + defaultLabel + } else if (sparkMaster.contains("k8s")) { + getSparkVersion match { + case Some(version) => + if (compareSparkVersion(version, "3.3.0") > 0) { + "spark.executor.memoryOverheadFactor" + } else { + "spark.kubernetes.memoryOverheadFactor" + } + case None => defaultLabel + } + } else { + defaultLabel + } + } } /** - * Recommendation for 'spark.rapids.memory.pinnedPool.size' if it is not set. - * Recommendation for memoryOverhead and memoryOverheadFactor based on cluster scheduler and - * spark version used. - * * Flow: - * if (pinnedPoolSize is not set) -> set pinnedPoolSize, calculate and set memoryOverhead - * else - * if using yarn - * -> if memoryOverhead is not set, calculate and set. - * if using k8s - * -> if memoryOverheadFactor is not set, add comment. + * if "spark.rapids.memory.pinnedPool.size" is set + * if yarn -> recommend "spark.executor.memoryOverhead" + * if using k8s -> + * if version > 3.3.0 recommend "spark.executor.memoryOverheadFactor" and add comment + * else recommend "spark.kubernetes.memoryOverheadFactor" and add comment if missing */ - private def recommendMemoryOverheadProperties(recommendedConfig: Config): Unit = { - val pinnedPoolSize = getSparkProperty(app, "spark.rapids.memory.pinnedPool.size") - if (pinnedPoolSize.isEmpty) { - recommendedConfig.setPinnedPoolSize(DEFAULT_PINNED_POOL_SIZE) - val memoryOverhead = calculateMemoryOverhead(DEFAULT_PINNED_POOL_SIZE, - recommendedConfig.getExecutorMemory) - recommendedConfig.setExecutorMemoryOverhead(convertToHumanReadableSize(memoryOverhead)) - } else { - val sparkMaster = getSparkProperty(app, "spark.master") - if (sparkMaster.contains("yarn")) { - if (getSparkProperty(app, "spark.executor.memoryOverhead").isEmpty) { - val memoryOverhead = calculateMemoryOverhead(pinnedPoolSize.get, - recommendedConfig.getExecutorMemory) - recommendedConfig.setExecutorMemoryOverhead( - convertToHumanReadableSize(memoryOverhead)) - } - } else if (sparkMaster.contains("k8s")) { - if (compareSparkVersion(app.appInfo.head.sparkVersion, "3.3.0") > 0) { - if (getSparkProperty(app, "spark.executor.memoryOverheadFactor").isEmpty) { - comments :+= "'spark.executor.memoryOverheadFactor' must be set " + - "if using 'spark.rapids.memory.pinnedPool.size'" - } - } else { - if (getSparkProperty(app, "spark.kubernetes.memoryOverheadFactor").isEmpty) { - comments :+= "'spark.kubernetes.memoryOverheadFactor' must be set " + - "if using 'spark.rapids.memory.pinnedPool.size'" - } + def addRecommendationForMemoryOverhead(recomValue: String): Unit = { + val memOverheadLookup = memoryOverheadLabel + appendRecommendationForMemoryMB(memOverheadLookup, recomValue) + getPropertyValue("spark.rapids.memory.pinnedPool.size").foreach { lookup => + if (lookup != "spark.executor.memoryOverhead") { + if (getPropertyValue(memOverheadLookup).isEmpty) { + appendComment(s"'$memOverheadLookup' must be set if using " + + s"'spark.rapids.memory.pinnedPool.size") } } } } + def calculateRecommendations(): Unit = { + recommendExecutorInstances() + val numExecutorCores = calcNumExecutorCores + val execCoresExpr = () => numExecutorCores + + appendRecommendation("spark.executor.cores", numExecutorCores) + appendRecommendation("spark.task.resource.gpu.amount", + calcTaskGPUAmount(execCoresExpr)) + appendRecommendation("spark.rapids.sql.concurrentGpuTasks", + calcGpuConcTasks().toInt) + val availableMemPerExec = calcAvailableMemPerExec() + val availableMemPerExecExpr = () => availableMemPerExec + val executorHeap = calcExecutorHeap(availableMemPerExecExpr, execCoresExpr) + val executorHeapExpr = () => executorHeap + appendRecommendationForMemoryMB("spark.executor.memory", s"$executorHeap") + val (pinnedMemory, memoryOverhead) = + calcPinnedMemoryWithOverhead(executorHeapExpr, availableMemPerExecExpr) + appendRecommendationForMemoryMB("spark.rapids.memory.pinnedPool.size", s"$pinnedMemory") + addRecommendationForMemoryOverhead(s"$memoryOverhead") + + recommendMaxPartitionBytes() + recommendShufflePartitions() + recommendGeneralProperties() + } + /** - * Recommendation for 'spark.sql.shuffle.partitions' based on spill size. + * Checks whether the cluster properties are valid. + * If the cluster worker-info is missing entries (i.e., CPU and GPU count), it sets the entries + * to default values. For each default value, a comment is added to the [[comments]]. + * + * @return false if the cluster properties are not loaded. e.g, all entries are set to 0. + * true if the missing information were updated to default initial values. */ - private def recommendShufflePartitions(recommendedConfig: Config): Unit = { - var shufflePartitions: Int = getSparkProperty(app, "spark.sql.shuffle.partitions") - .getOrElse("200").toInt + def processPropsAndCheck: Boolean = { + if (clusterProps.system.isEmpty) { + if (!clusterProps.isEmpty) { + appendComment( + s"Incorrect values in worker system information: ${clusterProps.system}.") + } + false + } else { + if (clusterProps.system.isMissingInfo) { + clusterProps.system.setMissingFields().foreach(m => appendComment(m)) + } + if (clusterProps.gpu.isMissingInfo) { + clusterProps.gpu.setMissingFields().foreach(m => appendComment(m)) + } + true + } + } - // TODO: Need to look at other metrics for GPU spills (DEBUG mode), and batch sizes metric - val totalSpilledMetrics = app.sqlTaskAggMetrics.map { - task => task.diskBytesSpilledSum + task.memoryBytesSpilledSum - }.sum - if (totalSpilledMetrics > 0) { - shufflePartitions *= DEFAULT_SHUFFLE_PARTITION_MULTIPLIER - // Could be memory instead of partitions - comments :+= "\"spark.sql.shuffle.partitions\" should be increased since spilling occurred." + private def recommendGeneralProperties(): Unit = { + val aqeEnabled = getPropertyValue("spark.sql.adaptive.enabled").getOrElse("False") + if (aqeEnabled == "False") { + appendComment(commentsForMissingProps("spark.sql.adaptive.enabled")) + } + if (appInfo.isDefined) { + val jvmGCFraction = appInfo.get.sqlTaskAggMetrics.map { + taskMetrics => taskMetrics.jvmGCTimeSum * 1.0 / taskMetrics.executorCpuTime + } + if (jvmGCFraction.nonEmpty) { // avoid zero division + if ((jvmGCFraction.sum / jvmGCFraction.size) > MAX_JVM_GCTIME_FRACTION) { + appendComment("Average JVM GC time is very high. " + + "Other Garbage Collectors can be used for better performance.") + } + } } - recommendedConfig.setShufflePartitions(shufflePartitions) } /** - * Calculate max partition bytes using task input size. + * Calculate max partition bytes using task input size and existing setting for maxPartitionBytes. + * Note that this won't apply the same on iceberg. * Eg, * MIN_PARTITION_BYTES_RANGE = 128m, MAX_PARTITION_BYTES_RANGE = 256m * (1) Input: maxPartitionBytes = 512m @@ -339,157 +661,250 @@ class AutoTuner(app: ApplicationSummaryInfo, workerInfo: String) extends Logging * Output: newMaxPartitionBytes = 2g / (512m/128m) = 512m */ private def calculateMaxPartitionBytes(maxPartitionBytes: String): String = { - val taskInputSize = - app.sqlTaskAggMetrics.map(_.inputBytesReadAvg).sum / app.sqlTaskAggMetrics.size - val maxPartitionBytesNum = convertFromHumanReadableSize(maxPartitionBytes) - if (taskInputSize > 0 && - taskInputSize < convertFromHumanReadableSize(MIN_PARTITION_BYTES_RANGE)) { - // Increase partition size - val calculatedMaxPartitionBytes = Math.min( - maxPartitionBytesNum * - (convertFromHumanReadableSize(MIN_PARTITION_BYTES_RANGE) / taskInputSize), - convertFromHumanReadableSize(MAX_PARTITION_BYTES_BOUND)) - - convertToHumanReadableSize(calculatedMaxPartitionBytes.toLong) - } else if (taskInputSize > convertFromHumanReadableSize(MAX_PARTITION_BYTES_RANGE)) { - // Decrease partition size - val calculatedMaxPartitionBytes = Math.min( - maxPartitionBytesNum / - (taskInputSize / convertFromHumanReadableSize(MAX_PARTITION_BYTES_RANGE)), - convertFromHumanReadableSize(MAX_PARTITION_BYTES_BOUND)) - - convertToHumanReadableSize(calculatedMaxPartitionBytes.toLong) + val app = appInfo.get + val maxPartitionBytesNum = convertToMB(maxPartitionBytes) + if (app.sqlTaskAggMetrics.isEmpty) { // avoid division by zero + maxPartitionBytesNum.toString } else { - // Do not recommend maxPartitionBytes - null + val taskInputSizeInBytes = + app.sqlTaskAggMetrics.map(_.inputBytesReadAvg).sum / app.sqlTaskAggMetrics.size + val taskInputSizeInMB = taskInputSizeInBytes / (1024 * 1024) + if (taskInputSizeInMB > 0 && + taskInputSizeInMB < MIN_PARTITION_BYTES_RANGE_MB) { + // Increase partition size + val calculatedMaxPartitionBytes = Math.min( + maxPartitionBytesNum * + (MIN_PARTITION_BYTES_RANGE_MB / taskInputSizeInMB), + MAX_PARTITION_BYTES_BOUND_MB) + calculatedMaxPartitionBytes.toLong.toString + } else if (taskInputSizeInMB > MAX_PARTITION_BYTES_RANGE_MB) { + // Decrease partition size + val calculatedMaxPartitionBytes = Math.min( + maxPartitionBytesNum / + (taskInputSizeInMB / MAX_PARTITION_BYTES_RANGE_MB), + MAX_PARTITION_BYTES_BOUND_MB) + calculatedMaxPartitionBytes.toLong.toString + } else { + // Do not recommend maxPartitionBytes + null + } } } /** * Recommendation for 'spark.sql.files.maxPartitionBytes' based on input size for each task. + * Note that the logic can be disabled by adding the property to [[limitedLogicRecommendations]] + * which is one of the arguments of [[getRecommendedProperties()]]. */ - private def recommendMaxPartitionBytes(recommendedConfig: Config): Unit = { - getSparkProperty(app, "spark.sql.files.maxPartitionBytes") match { - case None => - val newMaxPartitionBytes = calculateMaxPartitionBytes(DEFAULT_MAX_PARTITION_BYTES) - recommendedConfig.setMaxPartitionBytes(newMaxPartitionBytes) - - case Some(maxPartitionBytes) => - val newMaxPartitionBytes = calculateMaxPartitionBytes(maxPartitionBytes) - recommendedConfig.setMaxPartitionBytes(newMaxPartitionBytes) - comments :+= s"Although 'spark.sql.files.maxPartitionBytes' was set to $maxPartitionBytes" + - s", recommended value is $newMaxPartitionBytes." - } + private def recommendMaxPartitionBytes(): Unit = { + val maxPartitionProp = + getPropertyValue("spark.sql.files.maxPartitionBytes").getOrElse(MAX_PARTITION_BYTES) + val recommended = + if (isCalculationEnabled("spark.sql.files.maxPartitionBytes")) { + appInfo match { + case None => s"${convertToMB(maxPartitionProp)}" + case Some(_) => + calculateMaxPartitionBytes(maxPartitionProp) + } + } else { + s"${convertToMB(maxPartitionProp)}" + } + appendRecommendationForMemoryMB("spark.sql.files.maxPartitionBytes", recommended) } - private def recommendGeneralProperties() = { - val aqeEnabled = getSparkProperty(app, "spark.sql.adaptive.enabled").getOrElse("False") - if (aqeEnabled == "False") { - comments :+= "'spark.sql.adaptive.enabled' should be enabled for better performance." + /** + * Recommendations for "spark.sql.shuffle.partitions'. + * Note that by default this only recommend the default value for now. + * To enable calculating recommendation based on spills, override the argument + * "limitedLogicList" passed to [[getRecommendedProperties]]. + * + */ + def recommendShufflePartitions(): Unit = { + val lookup = "spark.sql.shuffle.partitions" + var shufflePartitions = + getPropertyValue(lookup).getOrElse(DEF_SHUFFLE_PARTITIONS).toInt + // TODO: Need to look at other metrics for GPU spills (DEBUG mode), and batch sizes metric + if (isCalculationEnabled(lookup)) { + appInfo.foreach { app => + val totalSpilledMetrics = app.sqlTaskAggMetrics.map { task => + task.diskBytesSpilledSum + task.memoryBytesSpilledSum + }.sum + if (totalSpilledMetrics > 0) { + shufflePartitions *= DEF_SHUFFLE_PARTITION_MULTIPLIER + // Could be memory instead of partitions + appendOptionalComment(lookup, + s"'$lookup' should be increased since spilling occurred.") + } + } } + appendRecommendation("spark.sql.shuffle.partitions", s"$shufflePartitions") + } - val jvmGCFraction = app.sqlTaskAggMetrics.map { - taskMetrics => taskMetrics.jvmGCTimeSum * 1.0 / taskMetrics.executorCpuTime - } - if ((jvmGCFraction.sum / jvmGCFraction.size) > MAX_JVM_GCTIME_FRACTION) { - comments :+= "Average JVM GC time is very high. " + - "Other Garbage Collectors can be used for better performance" + def appendOptionalComment(lookup: String, comment: String): Unit = { + if (!skippedRecommendations.contains(lookup)) { + appendComment(comment) } } - /** - * Recommendation for 'spark.task.resource.gpu.amount' based on num of cpu cores. - */ - private def recommendTaskResourceGpu(recommendedConfig: Config): Unit = { - val numCores: Int = recommendedConfig.getExecutorCores - val taskResourceGpu: Double = 1.0 / numCores - recommendedConfig.setTaskResourceGpu(taskResourceGpu) + def appendComment(comment: String): Unit = { + comments += comment } - /** - * Recommendation for 'spark.rapids.sql.concurrentGpuTasks' based on gpu memory. - */ - private def recommendConcurrentGpuTasks(recommendedConfig: Config, - systemProps: SystemProps): Unit = { - val numCores: Int = recommendedConfig.getExecutorCores - val gpuMemoryNum: Long = convertFromHumanReadableSize(systemProps.gpuProps.memory) - val concurrentGpuTasks: Int = Math.min( - gpuMemoryNum * DEFAULT_CONCURRENT_GPU_TASKS_MULTIPLIER, - MAX_CONCURRENT_GPU_TASKS).toInt - recommendedConfig.setConcurrentGpuTasks(concurrentGpuTasks) - - if(numCores < concurrentGpuTasks) { - comments :+= s"For the given GPU, number of CPU cores is very low. It should be" + - s" at least equal to concurrent gpu tasks i.e. $concurrentGpuTasks." - } + def convertClusterPropsToString(): String = { + clusterProps.toString } - /** - * Recommendations for Spark only properties. - */ - private def recommendSparkProperties(recommendedConfig: Config, - systemProps: SystemProps): Unit = { - if (systemProps == null) { - logWarning("System information is not available. Cannot recommend properties.") - comments :+= "'spark.executor.memory' should be set to at least 2GB/core." - comments :+= "'spark.executor.instances' should be set to 'num_gpus * num_workers'." - } else { - recommendExecutorInstances(recommendedConfig, systemProps) - recommendExecutorCores(recommendedConfig, systemProps) - recommendExecutorMemory(recommendedConfig, systemProps) - recommendMemoryOverheadProperties(recommendedConfig) - recommendShufflePartitions(recommendedConfig) - recommendMaxPartitionBytes(recommendedConfig) - recommendGeneralProperties() - } + private def toCommentProfileResult: Seq[RecommendedCommentResult] = { + comments.map(RecommendedCommentResult) } - /** - * Recommendations for GPU (RAPIDS) properties. - */ - private def recommendGpuProperties(recommendedConfig: Config, systemProps: SystemProps): Unit = { - if (systemProps == null || systemProps.gpuProps == null) { - logWarning("GPU information is not available. Cannot recommend properties.") - comments :+= "'spark.task.resource.gpu.amount' should be set to 1/#cores." - comments :+= s"'spark.rapids.sql.concurrentGpuTasks' should be set to" + - s" $DEFAULT_CONCURRENT_GPU_TASKS." - comments :+= s"'spark.rapids.memory.pinnedPool.size' should be set to" + - s" $DEFAULT_PINNED_POOL_SIZE." - } else { - recommendTaskResourceGpu(recommendedConfig) - recommendConcurrentGpuTasks(recommendedConfig, systemProps) - } + private def toRecommendationsProfileResult: Seq[RecommendedPropertyResult] = { + val finalRecommendations = + recommendations.filter(elem => elem._2.isValid(filterByUpdatedPropertiesEnabled)) + finalRecommendations.collect { + case (key, record) => RecommendedPropertyResult(key, record.recommended.get) + }.toSeq.sortBy(_.property) } /** - * Entry point for generating recommendations. + * The Autotuner loads the spark properties from either the ClusterProperties or the eventlog. + * 1- runs the calculation for each criterion and saves it as a [[RecommendationEntry]]. + * 2- The final list of recommendations include any [[RecommendationEntry]] that has a + * recommendation that is different from the original property. + * 3- Null values are excluded. + * 4- A comment is added for each missing property in the spark property. + * + * @param skipList a list of properties to be skipped. If none, all recommendations are + * returned. Note that the recommendations will be computed anyway internally + * in case there are dependencies between the recommendations. + * Default is empty. + * @param limitedLogicList a list of properties that will do simple recommendations based on + * static default values. + * Default is set to "spark.sql.shuffle.partitions". + * @param showOnlyUpdatedProps When enabled, the profiler recommendations should only include + * updated settings. + * @return pair of recommendations and comments. Both sequence can be empty. */ - def getRecommendedProperties: (Seq[RecommendedPropertyResult], - Seq[RecommendedCommentResult]) = { - val (systemProps, message) = parseSystemInfo(workerInfo) - if(message.nonEmpty) { - comments :+= message.get - } - val recommendedConfig = new Config() - recommendSparkProperties(recommendedConfig, systemProps) - recommendGpuProperties(recommendedConfig, systemProps) - (toPropertyProfileResult(recommendedConfig), toCommentProfileResult(comments)) + def getRecommendedProperties( + skipList: Option[Seq[String]] = Some(Seq()), + limitedLogicList: Option[Seq[String]] = Some(Seq("spark.sql.shuffle.partitions")), + showOnlyUpdatedProps: Boolean = true): + (Seq[RecommendedPropertyResult], Seq[RecommendedCommentResult]) = { + filterByUpdatedPropertiesEnabled = showOnlyUpdatedProps + limitedLogicList.foreach { limitedSeq => + limitedSeq.foreach(_ => limitedLogicRecommendations.add(_)) + } + skipList.foreach(skipSeq => skipSeq.foreach(_ => skippedRecommendations.add(_))) + if (processPropsAndCheck) { + initRecommendations() + calculateRecommendations() + } else { + // add all default comments + commentsForMissingProps.foreach(commentEntry => appendComment(commentEntry._2)) + } + (toRecommendationsProfileResult, toCommentProfileResult) } } object AutoTuner extends Logging { - val DEFAULT_WORKER_INFO: String = "." + // Amount of GPU memory to use per concurrent task in megabytes. + // Using a bit less than 8GB here since Dataproc clusters advertise T4s as only having + // around 14.75 GB and we want to run with 2 concurrent by default on T4s. + val DEF_GPU_MEM_PER_TASK_MB = 7500L + // Maximum number of concurrent tasks to run on the GPU + val MAX_CONC_GPU_TASKS = 4L + // Amount of CPU memory to reserve for system overhead (kernel, buffers, etc.) in megabytes + val DEF_SYSTEM_RESERVE_MB: Long = 2 * 1024L + // Fraction of the executor JVM heap size that should be additionally reserved + // for JVM off-heap overhead (thread stacks, native libraries, etc.) + val DEF_HEAP_OVERHEAD_FRACTION = 0.1 + val MAX_JVM_GCTIME_FRACTION = 0.3 + // Ideal amount of JVM heap memory to request per CPU core in megabytes + val DEF_HEAP_PER_CORE_MB: Long = 2 * 1024L + // Maximum amount of pinned memory to use per executor in MB + val MAX_PINNED_MEMORY_MB: Long = 4 * 1024L + // Default pinned memory to use per executor in MB + val DEF_PINNED_MEMORY_MB: Long = 2 * 1024L + // value in MB + val MIN_PARTITION_BYTES_RANGE_MB = 128L + // value in MB + val MAX_PARTITION_BYTES_RANGE_MB = 256L + // value in MB + val MAX_PARTITION_BYTES_BOUND_MB: Int = 4 * 1024 + val MAX_PARTITION_BYTES: String = "512m" + val DEF_SHUFFLE_PARTITIONS = "200" + val DEF_SHUFFLE_PARTITION_MULTIPLIER: Int = 2 + // GPU count defaults to 1 if it is missing. + val DEF_WORKER_GPU_COUNT = 1 + // GPU default device is T4 + val DEF_WORKER_GPU_NAME = "T4" + // T4 default memory is 16G + // A100 set default to 40GB + val DEF_WORKER_GPU_MEMORY_MB: mutable.LinkedHashMap[String, String] = + mutable.LinkedHashMap[String, String]("T4"-> "15109m", "A100" -> "40960m") + // Default Number of Workers 1 + val DEF_NUM_WORKERS = 1 + val DEFAULT_WORKER_INFO_PATH = "./worker_info.yaml" val SUPPORTED_SIZE_UNITS: Seq[String] = Seq("b", "k", "m", "g", "t", "p") - def loadSystemProperties(fileInput: String): Option[java.util.Map[String, Any]] = { - val filePath = new Path(fileInput) - val fs = FileSystem.get(filePath.toUri, new Configuration()) - val yaml = new Yaml() + val commentsForMissingProps: mutable.Map[String, String] = mutable.LinkedHashMap[String, String]( + "spark.executor.memory" -> + "'spark.executor.memory' should be set to at least 2GB/core.", + "spark.executor.instances" -> + "'spark.executor.instances' should be set to (gpuCount * numWorkers).", + "spark.task.resource.gpu.amount" -> + "'spark.task.resource.gpu.amount' should be set to Max(1, (numCores / gpuCount)).", + "spark.rapids.sql.concurrentGpuTasks" -> + s"'spark.rapids.sql.concurrentGpuTasks' should be set to Max(4, (gpuMemory / 8G)).", + "spark.rapids.memory.pinnedPool.size" -> + s"'spark.rapids.memory.pinnedPool.size' should be set to ${DEF_PINNED_MEMORY_MB}m.", + "spark.sql.adaptive.enabled" -> + "'spark.sql.adaptive.enabled' should be enabled for better performance.") + + val recommendationsTarget: Seq[String] = Seq[String]( + "spark.executor.instances", + "spark.rapids.sql.enabled", + "spark.executor.cores", + "spark.executor.memory", + "spark.rapids.sql.concurrentGpuTasks", + "spark.task.resource.gpu.amount", + "spark.sql.shuffle.partitions", + "spark.sql.files.maxPartitionBytes", + "spark.rapids.memory.pinnedPool.size", + "spark.executor.memoryOverhead", + "spark.executor.memoryOverheadFactor", + "spark.kubernetes.memoryOverheadFactor") + + private def handleException( + ex: Exception, + appInfo: Option[ApplicationSummaryInfo]): AutoTuner = { + logError("Exception: " + ex.getStackTrace.mkString("Array(", ", ", ")")) + val tuning = new AutoTuner(new ClusterProperties(), appInfo) + val msg = ex match { + case cEx: ConstructorException => cEx.getContext + case _ => if (ex.getCause != null) ex.getCause.toString else ex.toString + } + tuning.appendComment(msg) + tuning + } + + def loadClusterPropertiesFromContent(clusterProps: String): Option[ClusterProperties] = { + val representer = new Representer + representer.getPropertyUtils.setSkipMissingProperties(true) + val yamlObjNested = new Yaml(new Constructor(classOf[ClusterProperties]), representer) + Option(yamlObjNested.load(clusterProps).asInstanceOf[ClusterProperties]) + } + + def loadClusterProps(filePath: String): Option[ClusterProperties] = { + val path = new Path(filePath) var fsIs: FSDataInputStream = null try { - fsIs = fs.open(filePath) - Some(yaml.load(fsIs).asInstanceOf[java.util.Map[String, Any]]) + val fs = FileSystem.get(path.toUri, new Configuration()) + fsIs = fs.open(path) + val reader = new BufferedReader(new InputStreamReader(fsIs)) + val fileContent = Stream.continually(reader.readLine()).takeWhile(_ != null).mkString("\n") + loadClusterPropertiesFromContent(fileContent) } finally { if (fsIs != null) { fsIs.close() @@ -498,69 +913,34 @@ object AutoTuner extends Logging { } /** - * Parses the yaml file and returns system and gpu properties. - * See [[SystemProps]] and [[GpuProps]]. + * Similar to [[buildAutoTuner]] but it allows constructing the AutoTuner without an + * existing file. This can be used in testing. + * + * @param clusterProps the cluster properties as string. + * @param appInfo Optional of the profiling container. + * @return a new AutoTuner object. */ - def parseSystemInfo(yamlFile: String): (SystemProps, Option[String]) = { + def buildAutoTunerFromProps( + clusterProps: String, + appInfo: Option[ApplicationSummaryInfo]): AutoTuner = { try { - val loadedProps = loadSystemProperties(yamlFile) - if (loadedProps.isDefined) { - val rawProps = loadedProps.get.asScala.toMap.filter { case (_, v) => v != null } - val rawSystemProps = rawProps("system").asInstanceOf[java.util.Map[String, Any]] - .asScala.toMap.filter { case (_, v) => v != null } - if (rawSystemProps.nonEmpty) { - val rawGpuProps = rawProps("gpu").asInstanceOf[java.util.Map[String, Any]] - .asScala.toMap.filter { case (_, v) => v != null } - - val gpuProps = if (rawGpuProps.nonEmpty) { - GpuProps( - rawGpuProps("count").toString.toInt, - rawGpuProps("memory").toString, - rawGpuProps("name").toString) - } else { - null - } - - (SystemProps( - rawSystemProps.getOrElse("num_cores", 1).toString.toInt, - rawSystemProps.getOrElse("cpu_arch", "").toString, - rawSystemProps.getOrElse("memory", "0b").toString, - rawSystemProps.getOrElse("free_disk_space", "0b").toString, - rawSystemProps.getOrElse("time_zone", "").toString, - rawSystemProps.get("num_workers").map(_.toString.toInt), - gpuProps), None) - } else { - (null, Some("System properties was empty")) - } - } - else { - (null, Some("System properties was empty")) - } + val clusterPropsOpt = loadClusterPropertiesFromContent(clusterProps) + new AutoTuner(clusterPropsOpt.getOrElse(new ClusterProperties()), appInfo) } catch { - case e: FileNotFoundException => - logError("Exception: " + e.getStackTrace.mkString("Array(", ", ", ")")) - (null, Some("System properties file was not found")) - case e: NullPointerException => - logError("Exception: " + e.getStackTrace.mkString("Array(", ", ", ")")) - (null, Some("System properties file was not formatted correctly.")) case e: Exception => - logError("Exception: " + e.getStackTrace.mkString("Array(", ", ", ")")) - (null, Some(e.toString)) + handleException(e, appInfo) } } - /** - * Returns the value of Spark property from the application summary info. - * [[RapidsPropertyProfileResult]] is defined as (key:key, rows: [key, value]). - * Returns: - * a. If the value is "null" or key is not found: None - * b. Else: Some(value) - */ - private def getSparkProperty(app: ApplicationSummaryInfo, property: String): Option[String] = { - app.sparkProps.collectFirst { - case propertyProfile: RapidsPropertyProfileResult - if propertyProfile.key == property && propertyProfile.rows(1) != "null" => - propertyProfile.rows(1) + def buildAutoTuner( + filePath: String, + appInfo: Option[ApplicationSummaryInfo]): AutoTuner = { + try { + val clusterPropsOpt = loadClusterProps(filePath) + new AutoTuner(clusterPropsOpt.getOrElse(new ClusterProperties()), appInfo) + } catch { + case e: Exception => + handleException(e, appInfo) } } @@ -576,14 +956,29 @@ object AutoTuner extends Logging { (sizeNum * Math.pow(1024, SUPPORTED_SIZE_UNITS.indexOf(sizeUnit))).toLong } + def containsMemoryUnits(size: String): Boolean = { + val sizesArr = size.toLowerCase.split("(?=[a-z])") + if (sizesArr.length > 1) { + SUPPORTED_SIZE_UNITS.contains(sizesArr(1)) + } else { + false + } + } + + def convertToMB(size: String): Long = { + convertFromHumanReadableSize(size) / (1024 * 1024) + } + /** * Converts size from bytes to human readable. * Eg, 4194304 -> "4m", 633554 -> "618.70k". */ def convertToHumanReadableSize(size: Long): String = { - if(size < 0) return "0b" + if (size < 0L) { + return "0b" + } - val unitIndex = (Math.log10(size)/Math.log10(1024)).toInt + val unitIndex = (Math.log10(size) / Math.log10(1024)).toInt assert(unitIndex < SUPPORTED_SIZE_UNITS.size, s"$size is too large to convert to human readable size") @@ -606,15 +1001,4 @@ object AutoTuner extends Logging { val difference = paddedVersions.find { case (a, b) => a != b } difference.fold(0) { case (a, b) => a.toInt - b.toInt } } - - private def toPropertyProfileResult(recommendedConfig:Config): Seq[RecommendedPropertyResult] = { - val properties = recommendedConfig.getAllProperties - properties.collect { - case (property, value) if value != null => RecommendedPropertyResult(property, value) - }.toSeq.sortBy(_.property) - } - - private def toCommentProfileResult(comments:Seq[String]): Seq[RecommendedCommentResult] = { - comments.map(RecommendedCommentResult) - } } diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ClassWarehouse.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ClassWarehouse.scala index b10786a09e2..ecf8bc172de 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ClassWarehouse.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ClassWarehouse.scala @@ -633,10 +633,6 @@ case class WholeStageCodeGenResults( } } -case class GpuProps(count: Int, memory: String, name: String) -case class SystemProps(numCores: Int, cpuArch: String, memory: String, - diskSpace: String, timeZone: String, numWorkers: Option[Int], gpuProps: GpuProps) - case class RecommendedPropertyResult(property: String, value: String){ override def toString: String = "--conf %s=%s".format(property, value) } diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileArgs.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileArgs.scala index 21d97ee52dd..19772a7ffa5 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileArgs.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileArgs.scala @@ -93,9 +93,9 @@ Usage: java -cp rapids-4-spark-tools_2.12-.jar:$SPARK_HOME/jars/* val workerInfo: ScallopOption[String] = opt[String](required = false, descr = "File path containing the system information of a worker node. It is assumed " + - "that all workers are homogenous. It requires the AutoTuner to be enabled. Default is" + - "current directory.", - default = Some(AutoTuner.DEFAULT_WORKER_INFO)) + "that all workers are homogenous. It requires the AutoTuner to be enabled. Default is " + + "./worker_info.yaml", + default = Some(AutoTuner.DEFAULT_WORKER_INFO_PATH)) validate(filterCriteria) { case crit if (crit.endsWith("-newest-filesystem") || diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala index 39e7cff46bc..3bf578687a3 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala @@ -310,7 +310,7 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs) extends Logging val removedBMs = healthCheck.getRemovedBlockManager val removedExecutors = healthCheck.getRemovedExecutors val unsupportedOps = healthCheck.getPossibleUnsupportedSQLPlan - + if (printPlans) { CollectInformation.printSQLPlans(apps, outputDir) } @@ -450,23 +450,14 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs) extends Logging Some("Unsupported SQL Ops")) if (useAutoTuner) { - val workerInfo: String = appArgs.workerInfo.getOrElse(AutoTuner.DEFAULT_WORKER_INFO) - val autoTuner: AutoTuner = new AutoTuner(app, workerInfo) - val (properties, comments) = autoTuner.getRecommendedProperties + val workerInfoPath = appArgs.workerInfo.getOrElse(AutoTuner.DEFAULT_WORKER_INFO_PATH) + val autoTuner: AutoTuner = AutoTuner.buildAutoTuner(workerInfoPath, Some(app)) + // the autotuner allows skipping some properties + // e.g. getRecommendedProperties(Some(Seq("spark.executor.instances"))) skips the + // recommendation related to executor instances. + val (properties, comments) = autoTuner.getRecommendedProperties() profileOutputWriter.writeText("\n### D. Recommended Configuration ###\n") - - if (properties.nonEmpty) { - val propertiesToStr = properties.map(_.toString).reduce(_ + "\n" + _) - profileOutputWriter.writeText("\nSpark Properties:\n" + propertiesToStr + "\n") - } else { - profileOutputWriter.writeText("Cannot recommend properties. See Comments.\n") - } - - // Comments are optional - if (comments.nonEmpty) { - val commentsToStr = comments.map(_.toString).reduce(_ + "\n" + _) - profileOutputWriter.writeText("\nComments:\n" + commentsToStr + "\n") - } + profileOutputWriter.writeText(Profiler.getAutoTunerResultsAsString(properties, comments)) } } } @@ -478,4 +469,19 @@ object Profiler { val COMPARE_LOG_FILE_NAME_PREFIX = "rapids_4_spark_tools_compare" val COMBINED_LOG_FILE_NAME_PREFIX = "rapids_4_spark_tools_combined" val SUBDIR = "rapids_4_spark_profile" + def getAutoTunerResultsAsString(props: Seq[RecommendedPropertyResult], + comments: Seq[RecommendedCommentResult]): String = { + val propStr = if (props.nonEmpty) { + val propertiesToStr = props.map(_.toString).reduce(_ + "\n" + _) + s"\nSpark Properties:\n$propertiesToStr\n" + } else { + "Cannot recommend properties. See Comments.\n" + } + if (comments.isEmpty) { // Comments are optional + propStr + } else { + val commentsToStr = comments.map(_.toString).reduce(_ + "\n" + _) + propStr + s"\nComments:\n$commentsToStr\n" + } + } } diff --git a/tools/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AutoTunerSuite.scala b/tools/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AutoTunerSuite.scala new file mode 100644 index 00000000000..ebaa1de3788 --- /dev/null +++ b/tools/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AutoTunerSuite.scala @@ -0,0 +1,351 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.tool.profiling + +import java.util + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.scalatest.{BeforeAndAfterEach, FunSuite} +import org.yaml.snakeyaml.{DumperOptions, Yaml} + +import org.apache.spark.internal.Logging + +class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { + + val defaultDataprocProps: mutable.Map[String, String] = { + mutable.LinkedHashMap[String, String]( + "spark.dynamicAllocation.enabled" -> "true", + "spark.driver.maxResultSize" -> "7680m", + "spark.driver.memory" -> "15360m", + "spark.executor.cores" -> "16", + "spark.executor.instances" -> "2", + "spark.executor.resource.gpu.amount" -> "1", + "spark.executor.memory" -> "26742m", + "spark.executor.memoryOverhead" -> "7372m", + "spark.executorEnv.OPENBLAS_NUM_THREADS" -> "1", + "spark.extraListeners" -> "com.google.cloud.spark.performance.DataprocMetricsListener", + "spark.rapids.memory.pinnedPool.size" -> "2048m", + "spark.scheduler.mode" -> "FAIR", + "spark.sql.cbo.enabled" -> "true", + "spark.sql.adaptive.enabled" -> "true", + "spark.ui.port" -> "0", + "spark.yarn.am.memory" -> "640m" + ) + } + + private def buildWorkerInfoAsString( + customProps: Option[mutable.Map[String, String]] = None, + numCores: Option[Int] = Some(32), + systemMemory: Option[String] = Some("122880MiB"), + numWorkers: Option[Int] = Some(4), + gpuCount: Option[Int] = Some(2), + gpuMemory: Option[String] = Some("15109MiB"), + gpuDevice: Option[String] = Some("T4")): String = { + val gpuWorkerProps = new GpuWorkerProps( + gpuMemory.getOrElse("15109MiB"), gpuCount.getOrElse(2), gpuDevice.getOrElse("T4")) + val cpuSystem = new SystemClusterProps( + numCores.getOrElse(32), systemMemory.getOrElse("122880MiB"), numWorkers.getOrElse(4)) + val systemProperties = customProps match { + case None => mutable.Map[String, String]() + case Some(newProps) => newProps + } + val convertedMap = new util.LinkedHashMap[String, String](systemProperties.asJava) + val clusterProps = new ClusterProperties(cpuSystem, gpuWorkerProps, convertedMap) + // set the options to convert the object into formatted yaml content + val options = new DumperOptions() + options.setIndent(2) + options.setPrettyFlow(true) + options.setDefaultFlowStyle(DumperOptions.FlowStyle.BLOCK) + val yaml = new Yaml(options) + val rawString = yaml.dump(clusterProps) + // Skip the first line as it contains "the name of the class" + rawString.split("\n").drop(1).mkString("\n") + } + + test("Load non-existing cluster properties") { + val autoTuner: AutoTuner = AutoTuner.buildAutoTuner("non-existing.yaml", None) + val (properties, comments) = autoTuner.getRecommendedProperties() + val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) + val expectedResults = + s"""|Cannot recommend properties. See Comments. + | + |Comments: + |- java.io.FileNotFoundException: File non-existing.yaml does not exist + |- 'spark.executor.memory' should be set to at least 2GB/core. + |- 'spark.executor.instances' should be set to (gpuCount * numWorkers). + |- 'spark.task.resource.gpu.amount' should be set to Max(1, (numCores / gpuCount)). + |- 'spark.rapids.sql.concurrentGpuTasks' should be set to Max(4, (gpuMemory / 8G)). + |- 'spark.rapids.memory.pinnedPool.size' should be set to 2048m. + |- 'spark.sql.adaptive.enabled' should be enabled for better performance. + |""".stripMargin + assert(expectedResults == autoTunerOutput) + } + + test("Load cluster properties with missing CPU cores") { + val dataprocWorkerInfo = buildWorkerInfoAsString(None, Some(0)) + val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) + val (properties, comments) = autoTuner.getRecommendedProperties() + val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) + // scalastyle:off line.size.limit + val expectedResults = + s"""|Cannot recommend properties. See Comments. + | + |Comments: + |- Incorrect values in worker system information: {numCores: 0, memory: 122880MiB, numWorkers: 4}. + |- 'spark.executor.memory' should be set to at least 2GB/core. + |- 'spark.executor.instances' should be set to (gpuCount * numWorkers). + |- 'spark.task.resource.gpu.amount' should be set to Max(1, (numCores / gpuCount)). + |- 'spark.rapids.sql.concurrentGpuTasks' should be set to Max(4, (gpuMemory / 8G)). + |- 'spark.rapids.memory.pinnedPool.size' should be set to 2048m. + |- 'spark.sql.adaptive.enabled' should be enabled for better performance. + |""".stripMargin + // scalastyle:on line.size.limit + assert(expectedResults == autoTunerOutput) + } + + test("Load cluster properties with missing CPU memory") { + val dataprocWorkerInfo = buildWorkerInfoAsString(None, None, Some("0m")) + val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) + val (properties, comments) = autoTuner.getRecommendedProperties() + val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) + // scalastyle:off line.size.limit + val expectedResults = + s"""|Cannot recommend properties. See Comments. + | + |Comments: + |- Incorrect values in worker system information: {numCores: 32, memory: 0m, numWorkers: 4}. + |- 'spark.executor.memory' should be set to at least 2GB/core. + |- 'spark.executor.instances' should be set to (gpuCount * numWorkers). + |- 'spark.task.resource.gpu.amount' should be set to Max(1, (numCores / gpuCount)). + |- 'spark.rapids.sql.concurrentGpuTasks' should be set to Max(4, (gpuMemory / 8G)). + |- 'spark.rapids.memory.pinnedPool.size' should be set to 2048m. + |- 'spark.sql.adaptive.enabled' should be enabled for better performance. + |""".stripMargin + // scalastyle:on line.size.limit + assert(expectedResults == autoTunerOutput) + } + + test("Load cluster properties with missing number of workers") { + val dataprocWorkerInfo = buildWorkerInfoAsString(None, None, None, Some(0)) + val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) + val (properties, comments) = autoTuner.getRecommendedProperties() + val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) + val expectedResults = + s"""| + |Spark Properties: + |--conf spark.executor.cores=16 + |--conf spark.executor.instances=2 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=7372m + |--conf spark.rapids.memory.pinnedPool.size=4096m + |--conf spark.rapids.sql.concurrentGpuTasks=2 + |--conf spark.sql.files.maxPartitionBytes=512m + |--conf spark.sql.shuffle.partitions=200 + |--conf spark.task.resource.gpu.amount=0.0625 + | + |Comments: + |- Number of workers is missing. Setting default to 1. + |- 'spark.executor.instances' was not set. + |- 'spark.executor.cores' was not set. + |- 'spark.task.resource.gpu.amount' was not set. + |- 'spark.rapids.sql.concurrentGpuTasks' was not set. + |- 'spark.executor.memory' was not set. + |- 'spark.rapids.memory.pinnedPool.size' was not set. + |- 'spark.executor.memoryOverhead' was not set. + |- 'spark.sql.files.maxPartitionBytes' was not set. + |- 'spark.sql.shuffle.partitions' was not set. + |- 'spark.sql.adaptive.enabled' should be enabled for better performance. + |""".stripMargin + assert(expectedResults == autoTunerOutput) + } + + test("Load cluster properties with missing GPU count") { + // the gpuCount should default to 1 + val customProps = mutable.LinkedHashMap( + "spark.executor.cores" -> "16", + "spark.executor.memory" -> "32768m", + "spark.executor.memoryOverhead" -> "7372m", + "spark.rapids.memory.pinnedPool.size" -> "4096m", + "spark.rapids.sql.concurrentGpuTasks" -> "2", + "spark.sql.files.maxPartitionBytes" -> "512m", + "spark.task.resource.gpu.amount" -> "0.0625") + val sparkProps = defaultDataprocProps.++(customProps) + val dataprocWorkerInfo = buildWorkerInfoAsString(Some(sparkProps), None, None, None, Some(0)) + val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) + val (properties, comments) = autoTuner.getRecommendedProperties() + val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) + val expectedResults = + s"""| + |Spark Properties: + |--conf spark.executor.cores=32 + |--conf spark.executor.memory=65536m + |--conf spark.executor.memoryOverhead=10649m + |--conf spark.sql.shuffle.partitions=200 + |--conf spark.task.resource.gpu.amount=0.03125 + | + |Comments: + |- GPU count is missing. Setting default to 1. + |- 'spark.sql.shuffle.partitions' was not set. + |""".stripMargin + assert(expectedResults == autoTunerOutput) + } + + test("Load cluster properties with missing GPU memory") { + // the gpu memory should be set to T4 memory settings + val customProps = mutable.LinkedHashMap( + "spark.executor.cores" -> "16", + "spark.executor.memory" -> "32768m", + "spark.executor.memoryOverhead" -> "7372m", + "spark.rapids.memory.pinnedPool.size" -> "4096m", + "spark.rapids.sql.concurrentGpuTasks" -> "2", + "spark.sql.files.maxPartitionBytes" -> "512m", + "spark.task.resource.gpu.amount" -> "0.0625") + val sparkProps = defaultDataprocProps.++(customProps) + val dataprocWorkerInfo = + buildWorkerInfoAsString(Some(sparkProps), None, None, None, None, Some("0m")) + val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) + val (properties, comments) = autoTuner.getRecommendedProperties() + val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) + val expectedResults = + s"""| + |Spark Properties: + |--conf spark.sql.shuffle.partitions=200 + | + |Comments: + |- GPU memory is missing. Setting default to 15109m. + |- 'spark.sql.shuffle.partitions' was not set. + |""".stripMargin + assert(expectedResults == autoTunerOutput) + } + + test("Load cluster properties with unknown GPU device") { + // with unknown fpu device, the memory won't be set correctly, then it should default to 16G + val customProps = mutable.LinkedHashMap( + "spark.executor.cores" -> "16", + "spark.executor.memory" -> "32768m", + "spark.executor.memoryOverhead" -> "7372m", + "spark.rapids.memory.pinnedPool.size" -> "4096m", + "spark.rapids.sql.concurrentGpuTasks" -> "2", + "spark.sql.files.maxPartitionBytes" -> "512m", + "spark.task.resource.gpu.amount" -> "0.0625") + val sparkProps = defaultDataprocProps.++(customProps) + val dataprocWorkerInfo = + buildWorkerInfoAsString(Some(sparkProps), None, None, None, None, Some("0m"), Some("GPU-X")) + val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) + val (properties, comments) = autoTuner.getRecommendedProperties() + val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) + val expectedResults = + s"""| + |Spark Properties: + |--conf spark.sql.shuffle.partitions=200 + | + |Comments: + |- GPU memory is missing. Setting default to 16384m. + |- 'spark.sql.shuffle.partitions' was not set. + |""".stripMargin + assert(expectedResults == autoTunerOutput) + } + + test("test T4 dataproc cluster with dynamicAllocation enabled") { + val customProps = mutable.LinkedHashMap( + "spark.executor.cores" -> "16", + "spark.executor.memory" -> "32768m", + "spark.executor.memoryOverhead" -> "7372m", + "spark.rapids.memory.pinnedPool.size" -> "4096m", + "spark.rapids.sql.concurrentGpuTasks" -> "2", + "spark.sql.files.maxPartitionBytes" -> "512m", + "spark.task.resource.gpu.amount" -> "0.0625") + val sparkProps = defaultDataprocProps.++(customProps) + val dataprocWorkerInfo = buildWorkerInfoAsString(Some(sparkProps)) + val expectedResults = + s"""| + |Spark Properties: + |--conf spark.sql.shuffle.partitions=200 + | + |Comments: + |- 'spark.sql.shuffle.partitions' was not set. + |""".stripMargin + val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) + val (properties, comments) = autoTuner.getRecommendedProperties() + val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) + assert(expectedResults == autoTunerOutput) + } + + // This mainly to test that the executorInstances will be calculated when the dynamic allocation + // is missing. + test("test T4 dataproc cluster with missing dynamic allocation") { + val customProps = mutable.LinkedHashMap( + "spark.dynamicAllocation.enabled" -> "false", + "spark.executor.cores" -> "16", + "spark.executor.memory" -> "32768m", + "spark.executor.memoryOverhead" -> "7372m", + "spark.rapids.memory.pinnedPool.size" -> "4096m", + "spark.rapids.sql.concurrentGpuTasks" -> "2", + "spark.sql.files.maxPartitionBytes" -> "512m", + "spark.task.resource.gpu.amount" -> "0.0625") + val sparkProps = defaultDataprocProps.++(customProps) + val dataprocWorkerInfo = buildWorkerInfoAsString(Some(sparkProps)) + val expectedResults = + s"""| + |Spark Properties: + |--conf spark.executor.instances=8 + |--conf spark.sql.shuffle.partitions=200 + | + |Comments: + |- 'spark.sql.shuffle.partitions' was not set. + |""".stripMargin + val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) + val (properties, comments) = autoTuner.getRecommendedProperties() + val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) + assert(expectedResults == autoTunerOutput) + } + + test("test AutoTuner with empty sparkProperties" ) { + val dataprocWorkerInfo = buildWorkerInfoAsString(None) + val expectedResults = + s"""| + |Spark Properties: + |--conf spark.executor.cores=16 + |--conf spark.executor.instances=8 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=7372m + |--conf spark.rapids.memory.pinnedPool.size=4096m + |--conf spark.rapids.sql.concurrentGpuTasks=2 + |--conf spark.sql.files.maxPartitionBytes=512m + |--conf spark.sql.shuffle.partitions=200 + |--conf spark.task.resource.gpu.amount=0.0625 + | + |Comments: + |- 'spark.executor.instances' was not set. + |- 'spark.executor.cores' was not set. + |- 'spark.task.resource.gpu.amount' was not set. + |- 'spark.rapids.sql.concurrentGpuTasks' was not set. + |- 'spark.executor.memory' was not set. + |- 'spark.rapids.memory.pinnedPool.size' was not set. + |- 'spark.executor.memoryOverhead' was not set. + |- 'spark.sql.files.maxPartitionBytes' was not set. + |- 'spark.sql.shuffle.partitions' was not set. + |- 'spark.sql.adaptive.enabled' should be enabled for better performance. + |""".stripMargin + val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) + val (properties, comments) = autoTuner.getRecommendedProperties() + val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) + assert(expectedResults == autoTunerOutput) + } +} From 8a12b81efb08f155afd9141a2c1842ac1209d44e Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 5 Oct 2022 08:49:20 -0500 Subject: [PATCH 166/190] Fix the maxPartitionBytes recommendation by AutoTuner to use the max task input bytes (#6689) * Revise AutoTuner to match the BootStrap Signed-off-by: Ahmed Hussein (amahussein) * handle constructorException Signed-off-by: Ahmed Hussein (amahussein) * fix imports to pass styling checks Signed-off-by: Ahmed Hussein (amahussein) * finish remaining todo Signed-off-by: Ahmed Hussein (amahussein) * add unit tests and code reviews Signed-off-by: Ahmed Hussein (amahussein) * Fix autotuner calculation max partition bytes to be based on max Signed-off-by: Thomas Graves * fix inputSize * address PR comments Signed-off-by: Ahmed Hussein (amahussein) * fix typos in toString() Signed-off-by: Ahmed Hussein (amahussein) * finish remaining comments Signed-off-by: Ahmed Hussein (amahussein) * cleanup Signed-off-by: Thomas Graves * rename * address typos in comments messages Signed-off-by: Ahmed Hussein (amahussein) * address typos in comments messages-2 Signed-off-by: Ahmed Hussein (amahussein) * handle missing information in worker info Signed-off-by: Ahmed Hussein (amahussein) * add unit tests for missing workers Signed-off-by: Ahmed Hussein (amahussein) * rename variable * fix merge issue Signed-off-by: Ahmed Hussein (amahussein) Signed-off-by: Thomas Graves Signed-off-by: Thomas Graves Co-authored-by: Ahmed Hussein (amahussein) --- .../rapids/tool/profiling/Analysis.scala | 24 +++++++ .../profiling/ApplicationSummaryInfo.scala | 3 +- .../rapids/tool/profiling/AutoTuner.scala | 27 +++++--- .../tool/profiling/ClassWarehouse.scala | 7 ++ .../rapids/tool/profiling/Profiler.scala | 68 +++++++++++-------- 5 files changed, 89 insertions(+), 40 deletions(-) diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Analysis.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Analysis.scala index a4fd9a0d0fa..1ecd2443735 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Analysis.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Analysis.scala @@ -312,6 +312,30 @@ class Analysis(apps: Seq[ApplicationInfo]) { } } + def getMaxTaskInputSizeBytes(): Seq[SQLMaxTaskInputSizes] = { + apps.map { app => + val maxOfSql = app.sqlIdToInfo.map { case (sqlId, _) => + val jcs = app.jobIdToInfo.filter { case (_, jc) => + jc.sqlID.getOrElse(-1) == sqlId + } + if (jcs.isEmpty) { + 0L + } else { + val stageIdsForSQL = jcs.flatMap(_._2.stageIds).toSeq + val tasksInSQL = app.taskEnd.filter { tc => + stageIdsForSQL.contains(tc.stageId) + } + if (tasksInSQL.isEmpty) { + 0L + } else { + tasksInSQL.map(_.input_bytesRead).max + } + } + }.max + SQLMaxTaskInputSizes(app.index, app.appId, maxOfSql) + } + } + def sqlMetricsAggregationDurationAndCpuTime(): Seq[SQLDurationExecutorTimeProfileResult] = { val allRows = apps.flatMap { app => app.sqlIdToInfo.map { case (sqlId, sqlCase) => diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationSummaryInfo.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationSummaryInfo.scala index 208e4af21e1..5646e68b58b 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationSummaryInfo.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationSummaryInfo.scala @@ -36,4 +36,5 @@ case class ApplicationSummaryInfo( val unsupportedOps: Seq[UnsupportedOpsProfileResult], val sparkProps: Seq[RapidsPropertyProfileResult], val sqlStageInfo: Seq[SQLStageInfoProfileResult], - val wholeStage: Seq[WholeStageCodeGenResults]) + val wholeStage: Seq[WholeStageCodeGenResults], + val maxTaskInputBytesRead: Seq[SQLMaxTaskInputSizes]) diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala index 039e3586097..5605f067455 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala @@ -649,8 +649,10 @@ class AutoTuner( } /** - * Calculate max partition bytes using task input size and existing setting for maxPartitionBytes. - * Note that this won't apply the same on iceberg. + * Calculate max partition bytes using the max task input size and existing setting + * for maxPartitionBytes. Note that this won't apply the same on iceberg. + * The max bytes here does not distinguish between GPU and CPU reads so we could + * improve that in the future. * Eg, * MIN_PARTITION_BYTES_RANGE = 128m, MAX_PARTITION_BYTES_RANGE = 256m * (1) Input: maxPartitionBytes = 512m @@ -662,26 +664,29 @@ class AutoTuner( */ private def calculateMaxPartitionBytes(maxPartitionBytes: String): String = { val app = appInfo.get + // Autotuner only supports a single app right now, so we get whatever value is here + val inputBytesMax = if (app.maxTaskInputBytesRead.nonEmpty) { + app.maxTaskInputBytesRead.head.maxTaskInputBytesRead / 1024 / 1024 + } else { + 0.0 + } val maxPartitionBytesNum = convertToMB(maxPartitionBytes) - if (app.sqlTaskAggMetrics.isEmpty) { // avoid division by zero + if (inputBytesMax == 0.0) { maxPartitionBytesNum.toString } else { - val taskInputSizeInBytes = - app.sqlTaskAggMetrics.map(_.inputBytesReadAvg).sum / app.sqlTaskAggMetrics.size - val taskInputSizeInMB = taskInputSizeInBytes / (1024 * 1024) - if (taskInputSizeInMB > 0 && - taskInputSizeInMB < MIN_PARTITION_BYTES_RANGE_MB) { + if (inputBytesMax > 0 && + inputBytesMax < MIN_PARTITION_BYTES_RANGE_MB) { // Increase partition size val calculatedMaxPartitionBytes = Math.min( maxPartitionBytesNum * - (MIN_PARTITION_BYTES_RANGE_MB / taskInputSizeInMB), + (MIN_PARTITION_BYTES_RANGE_MB / inputBytesMax), MAX_PARTITION_BYTES_BOUND_MB) calculatedMaxPartitionBytes.toLong.toString - } else if (taskInputSizeInMB > MAX_PARTITION_BYTES_RANGE_MB) { + } else if (inputBytesMax > MAX_PARTITION_BYTES_RANGE_MB) { // Decrease partition size val calculatedMaxPartitionBytes = Math.min( maxPartitionBytesNum / - (taskInputSizeInMB / MAX_PARTITION_BYTES_RANGE_MB), + (inputBytesMax / MAX_PARTITION_BYTES_RANGE_MB), MAX_PARTITION_BYTES_BOUND_MB) calculatedMaxPartitionBytes.toLong.toString } else { diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ClassWarehouse.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ClassWarehouse.scala index ecf8bc172de..9f2521de408 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ClassWarehouse.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ClassWarehouse.scala @@ -446,6 +446,13 @@ case class JobStageAggTaskMetricsProfileResult( } } +case class SQLMaxTaskInputSizes( + appIndex: Int, + appId: String, + // Not added to the output since it is used only by the AutoTuner + maxTaskInputBytesRead: Double +) + case class SQLTaskAggMetricsProfileResult( appIndex: Int, appId: String, diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala index 3bf578687a3..e0e57f31c1d 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala @@ -62,35 +62,43 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs) extends Logging if (outputCombined) { logError("Output combined option not valid with compare mode!") } else { - // create all the apps in parallel - val apps = createApps(eventLogInfos) - - if (apps.size < 2) { - logError("At least 2 applications are required for comparison mode. Exiting!") + if (useAutoTuner) { + logError("Autotuner is currently not supported with compare mode!") } else { - val profileOutputWriter = new ProfileOutputWriter(s"$outputDir/compare", - Profiler.COMPARE_LOG_FILE_NAME_PREFIX, numOutputRows, outputCSV = outputCSV) - try { - // we need the info for all of the apps to be able to compare so this happens serially - val (sums, comparedRes) = processApps(apps, printPlans = false, profileOutputWriter) - writeOutput(profileOutputWriter, Seq(sums), false, comparedRes) - } - finally { - profileOutputWriter.close() + // create all the apps in parallel + val apps = createApps(eventLogInfos) + + if (apps.size < 2) { + logError("At least 2 applications are required for comparison mode. Exiting!") + } else { + val profileOutputWriter = new ProfileOutputWriter(s"$outputDir/compare", + Profiler.COMPARE_LOG_FILE_NAME_PREFIX, numOutputRows, outputCSV = outputCSV) + try { + // we need the info for all of the apps to be able to compare so this happens serially + val (sums, comparedRes) = processApps(apps, printPlans = false, profileOutputWriter) + writeOutput(profileOutputWriter, Seq(sums), false, comparedRes) + } + finally { + profileOutputWriter.close() + } } } } } else if (outputCombined) { - // same as collection but combine the output so all apps are in single tables - // We can process all the apps in parallel and get the summary for them and then - // combine them into single tables in the output. - val profileOutputWriter = new ProfileOutputWriter(s"$outputDir/combined", - Profiler.COMBINED_LOG_FILE_NAME_PREFIX, numOutputRows, outputCSV = outputCSV) - val sums = createAppsAndSummarize(eventLogInfos, false, profileOutputWriter) - try { - writeOutput(profileOutputWriter, sums, outputCombined) - } finally { - profileOutputWriter.close() + if (useAutoTuner) { + logError("Autotuner is currently not supported with combined mode!") + } else { + // same as collection but combine the output so all apps are in single tables + // We can process all the apps in parallel and get the summary for them and then + // combine them into single tables in the output. + val profileOutputWriter = new ProfileOutputWriter(s"$outputDir/combined", + Profiler.COMBINED_LOG_FILE_NAME_PREFIX, numOutputRows, outputCSV = outputCSV) + val sums = createAppsAndSummarize(eventLogInfos, false, profileOutputWriter) + try { + writeOutput(profileOutputWriter, sums, outputCombined) + } finally { + profileOutputWriter.close() + } } } else { // Read each application and process it separately to save memory. @@ -302,6 +310,11 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs) extends Logging val sqlTaskAggMetrics = analysis.sqlMetricsAggregation() val durAndCpuMet = analysis.sqlMetricsAggregationDurationAndCpuTime() val skewInfo = analysis.shuffleSkewCheck() + val maxTaskInputInfo = if (useAutoTuner) { + analysis.getMaxTaskInputSizeBytes() + } else { + Seq.empty + } val healthCheck = new HealthCheck(apps) val failedTasks = healthCheck.getFailedTasks @@ -343,7 +356,7 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs) extends Logging (ApplicationSummaryInfo(appInfo, dsInfo, execInfo, jobInfo, rapidsProps, rapidsJar, sqlMetrics, jsMetAgg, sqlTaskAggMetrics, durAndCpuMet, skewInfo, failedTasks, failedStages, failedJobs, removedBMs, removedExecutors, unsupportedOps, sparkProps, sqlStageInfo, - wholeStage), compareRes) + wholeStage, maxTaskInputInfo), compareRes) } def writeOutput(profileOutputWriter: ProfileOutputWriter, @@ -378,8 +391,6 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs) extends Logging val resRows = allRows.map(r => RapidsPropertyProfileResult(r(0), outputHeaders, r)) resRows.sortBy(cols => cols.key) } - - val sorted = appsSum.sortBy( x => x.appInfo.head.appIndex) val reduced = ApplicationSummaryInfo( appsSum.flatMap(_.appInfo).sortBy(_.appIndex), appsSum.flatMap(_.dsInfo).sortBy(_.appIndex), @@ -400,7 +411,8 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs) extends Logging appsSum.flatMap(_.unsupportedOps).sortBy(_.appIndex), combineProps(rapidsOnly=false, appsSum).sortBy(_.key), appsSum.flatMap(_.sqlStageInfo).sortBy(_.duration)(Ordering[Option[Long]].reverse), - appsSum.flatMap(_.wholeStage).sortBy(_.appIndex) + appsSum.flatMap(_.wholeStage).sortBy(_.appIndex), + appsSum.flatMap(_.maxTaskInputBytesRead).sortBy(_.appIndex) ) Seq(reduced) } else { From 47f70ff5e5e000f0157fbe34df7d4c11f33bb7cc Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 5 Oct 2022 13:54:10 -0500 Subject: [PATCH 167/190] Update spark2 code for Revert "Add support for arrays in hashaggregate" (#6690) Signed-off-by: Thomas Graves Signed-off-by: Thomas Graves --- scripts/spark2diffs/GpuOverrides.diff | 344 +++++++++--------- scripts/spark2diffs/aggregate.diff | 61 +--- .../nvidia/spark/rapids/GpuOverrides.scala | 25 +- 3 files changed, 192 insertions(+), 238 deletions(-) diff --git a/scripts/spark2diffs/GpuOverrides.diff b/scripts/spark2diffs/GpuOverrides.diff index af05cdeeed4..7b167bbe26b 100644 --- a/scripts/spark2diffs/GpuOverrides.diff +++ b/scripts/spark2diffs/GpuOverrides.diff @@ -597,7 +597,7 @@ > override def convertToGpu(child: Expression): GpuExpression = GpuCot(child) 1330a1616 > override def convertToGpu(child: Expression): GpuExpression = GpuTanh(child) -1335a1622,1638 +1335a1622,1640 > override def convertToGpu(child: Expression): GpuExpression = GpuTan(child) > }), > expr[NormalizeNaNAndZero]( @@ -611,24 +611,26 @@ > }), > expr[KnownFloatingPointNormalized]( > "Tag to prevent redundant normalization", -> ExprChecks.unaryProjectInputMatchesOutput(TypeSig.all, TypeSig.all), +> ExprChecks.unaryProjectInputMatchesOutput( +> TypeSig.DOUBLE + TypeSig.FLOAT, +> TypeSig.DOUBLE + TypeSig.FLOAT), > (a, conf, p, r) => new UnaryExprMeta[KnownFloatingPointNormalized](a, conf, p, r) { > override def convertToGpu(child: Expression): GpuExpression = > GpuKnownFloatingPointNormalized(child) -1337,1338d1639 +1337,1338d1641 < // NormalizeNaNAndZero is not supported in spark 2.x < // KnownFloatingPointNormalized is not supported in spark 2.x -1344a1646,1647 +1344a1648,1649 > override def convertToGpu(child: Expression): GpuExpression = > GpuKnownNotNull(child) -1351a1655,1657 +1351a1657,1659 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > GpuDateDiff(lhs, rhs) > } -1369a1676,1677 +1369a1678,1679 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuTimeAdd(lhs, rhs) -1371c1679,1699 +1371c1681,1701 < // DateAddInterval is not supported in spark 2.x --- > expr[DateAddInterval]( @@ -652,10 +654,10 @@ > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuDateAddInterval(lhs, rhs) > }), -1379a1708,1709 +1379a1710,1711 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuDateFormatClass(lhs, rhs, strfFormat) -1391a1722,1729 +1391a1724,1731 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > if (conf.isImprovedTimestampOpsEnabled) { > // passing the already converted strf string for a little optimization @@ -664,7 +666,7 @@ > GpuToUnixTimestamp(lhs, rhs, sparkFormat, strfFormat) > } > } -1402a1741,1748 +1402a1743,1750 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > if (conf.isImprovedTimestampOpsEnabled) { > // passing the already converted strf string for a little optimization @@ -673,59 +675,59 @@ > GpuUnixTimestamp(lhs, rhs, sparkFormat, strfFormat) > } > } -1409a1756 +1409a1758 > override def convertToGpu(expr: Expression): GpuExpression = GpuHour(expr) -1415a1763,1765 +1415a1765,1767 > > override def convertToGpu(expr: Expression): GpuExpression = > GpuMinute(expr) -1421a1772,1774 +1421a1774,1776 > > override def convertToGpu(expr: Expression): GpuExpression = > GpuSecond(expr) -1427a1781,1782 +1427a1783,1784 > override def convertToGpu(expr: Expression): GpuExpression = > GpuWeekDay(expr) -1433a1789,1790 +1433a1791,1792 > override def convertToGpu(expr: Expression): GpuExpression = > GpuDayOfWeek(expr) -1438a1796,1797 +1438a1798,1799 > override def convertToGpu(expr: Expression): GpuExpression = > GpuLastDay(expr) -1447a1807,1809 +1447a1809,1811 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > // passing the already converted strf string for a little optimization > GpuFromUnixTime(lhs, rhs, strfFormat) -1472a1835,1836 +1472a1837,1838 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuPmod(lhs, rhs) -1485c1849 +1485c1851 < private val ansiEnabled = false --- > private val ansiEnabled = SQLConf.get.ansiEnabled -1487a1852,1854 +1487a1854,1856 > if (ansiEnabled && GpuAnsi.needBasicOpOverflowCheck(a.dataType)) { > willNotWorkInAst("AST Addition does not support ANSI mode.") > } -1489a1857,1858 +1489a1859,1860 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuAdd(lhs, rhs, failOnError = ansiEnabled) -1502c1871 +1502c1873 < private val ansiEnabled = false --- > private val ansiEnabled = SQLConf.get.ansiEnabled -1504a1874,1876 +1504a1876,1878 > if (ansiEnabled && GpuAnsi.needBasicOpOverflowCheck(a.dataType)) { > willNotWorkInAst("AST Subtraction does not support ANSI mode.") > } -1506a1879,1880 +1506a1881,1882 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuSubtract(lhs, rhs, ansiEnabled) -1519a1894,1896 +1519a1896,1898 > if (SQLConf.get.ansiEnabled && GpuAnsi.needBasicOpOverflowCheck(a.dataType)) { > willNotWorkOnGpu("GPU Multiplication does not support ANSI mode") > } -1521a1899,1906 +1521a1901,1908 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > a.dataType match { > case _: DecimalType => throw new IllegalStateException( @@ -734,37 +736,37 @@ > GpuMultiply(lhs, rhs) > } > } -1528a1914,1915 +1528a1916,1917 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuAnd(lhs, rhs) -1535a1923,1924 +1535a1925,1926 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuOr(lhs, rhs) -1547a1937,1938 +1547a1939,1940 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuEqualNullSafe(lhs, rhs) -1560a1952,1953 +1560a1954,1955 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuEqualTo(lhs, rhs) -1573a1967,1968 +1573a1969,1970 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuGreaterThan(lhs, rhs) -1586a1982,1983 +1586a1984,1985 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuGreaterThanOrEqual(lhs, rhs) -1606a2004,2005 +1606a2006,2007 > override def convertToGpu(): GpuExpression = > GpuInSet(childExprs.head.convertToGpu(), in.list.asInstanceOf[Seq[Literal]].map(_.value)) -1617a2017,2018 +1617a2019,2020 > override def convertToGpu(): GpuExpression = > GpuInSet(childExprs.head.convertToGpu(), in.hset.toSeq) -1630a2032,2033 +1630a2034,2035 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuLessThan(lhs, rhs) -1643a2047,2048 +1643a2049,2050 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuLessThanOrEqual(lhs, rhs) -1648a2054,2065 +1648a2056,2067 > override def convertToGpu(): GpuExpression = { > val branches = childExprs.grouped(2).flatMap { > case Seq(cond, value) => Some((cond.convertToGpu(), value.convertToGpu())) @@ -777,15 +779,15 @@ > } > GpuCaseWhen(branches, elseValue) > } -1665a2083,2086 +1665a2085,2088 > override def convertToGpu(): GpuExpression = { > val Seq(boolExpr, trueExpr, falseExpr) = childExprs.map(_.convertToGpu()) > GpuIf(boolExpr, trueExpr, falseExpr) > } -1673a2095,2096 +1673a2097,2098 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuPow(lhs, rhs) -1684a2108,2127 +1684a2110,2129 > // Division of Decimal types is a little odd. To work around some issues with > // what Spark does the tagging/checks are in CheckOverflow instead of here. > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = @@ -806,17 +808,17 @@ > (a, conf, p, r) => new BinaryExprMeta[IntegralDivide](a, conf, p, r) { > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuIntegralDivide(lhs, rhs) -1686d2128 +1686d2130 < // IntegralDivide is not supported in spark 2.x -1693a2136,2137 +1693a2138,2139 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuRemainder(lhs, rhs) -1708c2152,2153 +1708c2154,2155 < // No filter parameter in 2.x --- > private val filter: Option[BaseExprMeta[_]] = > a.filter.map(GpuOverrides.wrapExpr(_, conf, Some(this))) -1712c2157,2172 +1712c2159,2174 < childrenExprMeta --- > childrenExprMeta ++ filter.toSeq @@ -835,11 +837,11 @@ > GpuAggregateExpression(childExprs.head.convertToGpu().asInstanceOf[GpuAggregateFunction], > a.mode, a.isDistinct, filter.map(_.convertToGpu()), resultId) > } -1735a2196,2198 +1735a2198,2200 > // One of the few expressions that are not replaced with a GPU version > override def convertToGpu(): Expression = > sortOrder.withNewChildren(childExprs.map(_.convertToGpu())) -1757a2221,2227 +1757a2223,2229 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = { > val Seq(pivotColumn, valueColumn) = childExprs > GpuPivotFirst(pivotColumn, valueColumn, pivot.pivotColumnValues) @@ -847,22 +849,22 @@ > > // Pivot does not overflow, so it doesn't need the ANSI check > override val needsAnsiCheck: Boolean = false -1770a2241,2242 +1770a2243,2244 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = > GpuCount(childExprs) -1791a2264,2268 +1791a2266,2270 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = > GpuMax(childExprs.head) > > // Max does not overflow, so it doesn't need the ANSI check > override val needsAnsiCheck: Boolean = false -1812a2290,2294 +1812a2292,2296 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = > GpuMin(childExprs.head) > > // Min does not overflow, so it doesn't need the ANSI check > override val needsAnsiCheck: Boolean = false -1825a2308,2328 +1825a2310,2330 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = > GpuSum(childExprs.head, a.dataType) > }), @@ -884,87 +886,87 @@ > > // nth does not overflow, so it doesn't need the ANSI check > override val needsAnsiCheck: Boolean = false -1827d2329 +1827d2331 < // Spark 2.x doesn't have NthValue -1839a2342,2346 +1839a2344,2348 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = > GpuFirst(childExprs.head, a.ignoreNulls) > > // First does not overflow, so it doesn't need the ANSI check > override val needsAnsiCheck: Boolean = false -1842c2349 +1842c2351 < "last aggregate operator", --- > "last aggregate operator", -1852a2360,2364 +1852a2362,2366 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = > GpuLast(childExprs.head, a.ignoreNulls) > > // Last does not overflow, so it doesn't need the ANSI check > override val needsAnsiCheck: Boolean = false -1871a2384,2385 +1871a2386,2387 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuBRound(lhs, rhs, a.dataType) -1890a2405,2406 +1890a2407,2408 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuRound(lhs, rhs, a.dataType) -1912a2429,2433 +1912a2431,2435 > > override def convertToGpu(): GpuExpression = > GpuPythonUDF(a.name, a.func, a.dataType, > childExprs.map(_.convertToGpu()), > a.evalType, a.udfDeterministic, a.resultId) -1921a2443 +1921a2445 > override def convertToGpu(child: Expression): GpuExpression = GpuRand(child) -1926a2449 +1926a2451 > override def convertToGpu(): GpuExpression = GpuSparkPartitionID() -1931a2455 +1931a2457 > override def convertToGpu(): GpuExpression = GpuMonotonicallyIncreasingID() -1936a2461 +1936a2463 > override def convertToGpu(): GpuExpression = GpuInputFileName() -1941a2467 +1941a2469 > override def convertToGpu(): GpuExpression = GpuInputFileBlockStart() -1946a2473 +1946a2475 > override def convertToGpu(): GpuExpression = GpuInputFileBlockLength() -1952a2480 +1952a2482 > override def convertToGpu(child: Expression): GpuExpression = GpuMd5(child) -1957a2486 +1957a2488 > override def convertToGpu(child: Expression): GpuExpression = GpuUpper(child) -1963a2493 +1963a2495 > override def convertToGpu(child: Expression): GpuExpression = GpuLower(child) -1980a2511,2515 +1980a2513,2517 > override def convertToGpu( > str: Expression, > width: Expression, > pad: Expression): GpuExpression = > GpuStringLPad(str, width, pad) -1996a2532,2536 +1996a2534,2538 > override def convertToGpu( > str: Expression, > width: Expression, > pad: Expression): GpuExpression = > GpuStringRPad(str, width, pad) -2017a2558,2559 +2017a2560,2561 > override def convertToGpu(arr: Expression): GpuExpression = > GpuGetStructField(arr, expr.ordinal, expr.name) -2029a2572,2573 +2029a2574,2575 > override def convertToGpu(arr: Expression, ordinal: Expression): GpuExpression = > GpuGetArrayItem(arr, ordinal, in.failOnError) -2047a2592,2593 +2047a2594,2595 > override def convertToGpu(map: Expression, key: Expression): GpuExpression = > GpuGetMapValue(map, key, in.failOnError) -2051c2597 +2051c2599 < "Returns value for the given key in value if column is map", --- > "Returns value for the given key in value if column is map.", -2096a2643,2645 +2096a2645,2647 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > GpuElementAt(lhs, rhs, failOnError = in.failOnError) > } -2107a2657,2658 +2107a2659,2660 > override def convertToGpu(child: Expression): GpuExpression = > GpuMapKeys(child) -2118a2670,2685 +2118a2672,2687 > override def convertToGpu(child: Expression): GpuExpression = > GpuMapValues(child) > }), @@ -981,24 +983,24 @@ > (in, conf, p, r) => new UnaryExprMeta[MapEntries](in, conf, p, r) { > override def convertToGpu(child: Expression): GpuExpression = > GpuMapEntries(child) -2120d2686 +2120d2688 < // MapEntries is not supported in spark 2.x -2135a2702,2703 +2135a2704,2705 > override def convertToGpu(child: Expression): GpuExpression = > GpuArrayMin(child) -2144a2713,2714 +2144a2715,2716 > override def convertToGpu(child: Expression): GpuExpression = > GpuArrayMax(child) -2155a2726,2727 +2155a2728,2729 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuArrayRepeat(lhs, rhs) -2161a2734,2735 +2161a2736,2737 > override def convertToGpu(): GpuExpression = > GpuCreateNamedStruct(childExprs.map(_.convertToGpu())) -2177a2752,2753 +2177a2754,2755 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuArrayContains(lhs, rhs) -2190c2766,2770 +2190c2768,2772 < }), --- > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { @@ -1006,10 +1008,10 @@ > } > } > ), -2215a2796,2797 +2215a2798,2799 > override def convertToGpu(): GpuExpression = > GpuCreateArray(childExprs.map(_.convertToGpu()), wrapped.useStringTypeWhenEmpty) -2231a2814,2820 +2231a2816,2822 > override def convertToGpu(): GpuExpression = { > val func = childExprs.head > val args = childExprs.tail @@ -1017,15 +1019,15 @@ > args.map(_.convertToGpu().asInstanceOf[NamedExpression]), > in.hidden) > } -2239a2829,2831 +2239a2831,2833 > override def convertToGpu(): GpuExpression = { > GpuNamedLambdaVariable(in.name, in.dataType, in.nullable, in.exprId) > } -2256a2849,2851 +2256a2851,2853 > override def convertToGpu(): GpuExpression = { > GpuArrayTransform(childExprs.head.convertToGpu(), childExprs(1).convertToGpu()) > } -2267a2863,2869 +2267a2865,2871 > override def convertToGpu(): GpuExpression = { > GpuArrayExists( > childExprs.head.convertToGpu(), @@ -1033,15 +1035,15 @@ > SQLConf.get.getConf(SQLConf.LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC) > ) > } -2281a2884,2886 +2281a2886,2888 > override def convertToGpu(): GpuExpression = { > GpuArraysZip(childExprs.map(_.convertToGpu())) > } -2295a2901,2903 +2295a2903,2905 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > GpuArrayExcept(lhs, rhs) > } -2314a2923,2946 +2314a2925,2948 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > GpuArrayIntersect(lhs, rhs) > } @@ -1066,13 +1068,13 @@ > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > GpuArrayUnion(lhs, rhs) > } -2321d2952 +2321d2954 < // ArrayUnion is not supported in Spark 2.x -2333a2965,2967 +2333a2967,2969 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { > GpuArraysOverlap(lhs, rhs) > } -2340,2342c2974,3035 +2340,2342c2976,3037 < // TransformKeys is not supported in Spark 2.x < // TransformValues is not supported in Spark 2.x < // spark 2.x doesn't have MapFilter @@ -1139,66 +1141,66 @@ > GpuMapFilter(childExprs.head.convertToGpu(), childExprs(1).convertToGpu()) > } > }), -2349a3043,3047 +2349a3045,3049 > override def convertToGpu( > val0: Expression, > val1: Expression, > val2: Expression): GpuExpression = > GpuStringLocate(val0, val1, val2) -2357a3056,3060 +2357a3058,3062 > override def convertToGpu( > column: Expression, > position: Expression, > length: Expression): GpuExpression = > GpuSubstring(column, position, length) -2373a3077,3079 +2373a3079,3081 > override def convertToGpu( > input: Expression, > repeatTimes: Expression): GpuExpression = GpuStringRepeat(input, repeatTimes) -2381a3088,3092 +2381a3090,3094 > override def convertToGpu( > column: Expression, > target: Expression, > replace: Expression): GpuExpression = > GpuStringReplace(column, target, replace) -2389a3101,3104 +2389a3103,3106 > override def convertToGpu( > column: Expression, > target: Option[Expression] = None): GpuExpression = > GpuStringTrim(column, target) -2398a3114,3117 +2398a3116,3119 > override def convertToGpu( > column: Expression, > target: Option[Expression] = None): GpuExpression = > GpuStringTrimLeft(column, target) -2407a3127,3130 +2407a3129,3132 > override def convertToGpu( > column: Expression, > target: Option[Expression] = None): GpuExpression = > GpuStringTrimRight(column, target) -2414a3138,3139 +2414a3140,3141 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuStartsWith(lhs, rhs) -2421a3147,3148 +2421a3149,3150 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuEndsWith(lhs, rhs) -2434a3162 +2434a3164 > override def convertToGpu(child: Seq[Expression]): GpuExpression = GpuConcat(child) -2445a3174 +2445a3176 > override def convertToGpu(child: Seq[Expression]): GpuExpression = GpuMapConcat(child) -2462a3192,3193 +2462a3194,3195 > override final def convertToGpu(): GpuExpression = > GpuConcatWs(childExprs.map(_.convertToGpu())) -2472a3204,3205 +2472a3206,3207 > def convertToGpu(): GpuExpression = > GpuMurmur3Hash(childExprs.map(_.convertToGpu()), a.seed) -2479a3213,3214 +2479a3215,3216 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuContains(lhs, rhs) -2486a3222,3223 +2486a3224,3225 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuLike(lhs, rhs, a.escapeChar) -2512c3249,3256 +2512c3251,3258 < // RegExpExtractAll is not supported in Spark 2.x --- > expr[RegExpExtractAll]( @@ -1209,21 +1211,21 @@ > ParamCheck("regexp", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING), > ParamCheck("idx", TypeSig.lit(TypeEnum.INT), TypeSig.INT))), > (a, conf, p, r) => new GpuRegExpExtractAllMeta(a, conf, p, r)), -2517a3262 +2517a3264 > override def convertToGpu(child: Expression): GpuExpression = GpuLength(child) -2525a3271,3272 +2525a3273,3274 > override def convertToGpu(child: Expression): GpuExpression = > GpuSize(child, a.legacySizeOfNull) -2532a3280 +2532a3282 > override def convertToGpu(child: Expression): GpuExpression = GpuUnscaledValue(child) -2538a3287,3288 +2538a3289,3290 > override def convertToGpu(child: Expression): GpuExpression = > GpuMakeDecimal(child, a.precision, a.scale, a.nullOnOverflow) -2552a3303 +2552a3305 > override def convertToGpu(): GpuExpression = GpuExplode(childExprs.head.convertToGpu()) -2566a3318 +2566a3320 > override def convertToGpu(): GpuExpression = GpuPosExplode(childExprs.head.convertToGpu()) -2579,2580c3331,3411 +2579,2580c3333,3413 < }), < // spark 2.x CollectList and CollectSet use TypedImperative which isn't in 2.x --- @@ -1308,53 +1310,53 @@ > // Last does not overflow, so it doesn't need the ANSI check > override val needsAnsiCheck: Boolean = false > }), -2586a3418,3421 +2586a3420,3423 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = { > val legacyStatisticalAggregate = SQLConf.get.legacyStatisticalAggregate > GpuStddevPop(childExprs.head, !legacyStatisticalAggregate) > } -2594a3430,3433 +2594a3432,3435 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = { > val legacyStatisticalAggregate = SQLConf.get.legacyStatisticalAggregate > GpuStddevSamp(childExprs.head, !legacyStatisticalAggregate) > } -2601a3441,3444 +2601a3443,3446 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = { > val legacyStatisticalAggregate = SQLConf.get.legacyStatisticalAggregate > GpuVariancePop(childExprs.head, !legacyStatisticalAggregate) > } -2608a3452,3455 +2608a3454,3457 > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = { > val legacyStatisticalAggregate = SQLConf.get.legacyStatisticalAggregate > GpuVarianceSamp(childExprs.head, !legacyStatisticalAggregate) > } -2648a3496,3501 +2648a3498,3503 > > override def convertToGpu(childExprs: Seq[Expression]): GpuExpression = > GpuApproximatePercentile(childExprs.head, > childExprs(1).asInstanceOf[GpuLiteral], > childExprs(2).asInstanceOf[GpuLiteral]) > -2663a3517,3518 +2663a3519,3520 > override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = > GpuGetJsonObject(lhs, rhs) -2675c3530,3532 +2675c3532,3534 < }), --- > override def convertToGpu(): GpuExpression = GpuScalarSubquery(a.plan, a.exprId) > } > ), -2680c3537,3539 +2680c3539,3541 < }), --- > override def convertToGpu(): GpuExpression = GpuCreateMap(childExprs.map(_.convertToGpu())) > } > ), -2698a3558 +2698a3560 > override def convertToGpu(child: Expression): GpuExpression = GpuBitLength(child) -2705a3566 +2705a3568 > override def convertToGpu(child: Expression): GpuExpression = GpuOctetLength(child) -2719,2721c3580,3590 +2719,2721c3582,3592 < ) < // Spark 2.x doesn't have RaiseError or ansicast < ).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap @@ -1370,7 +1372,7 @@ > }), > SparkShimImpl.ansiCastRule > ).collect { case r if r != null => (r.getClassFor.asSubclass(classOf[Expression]), r)}.toMap -2726c3595,3642 +2726c3597,3644 < ShimGpuOverrides.shimExpressions --- > SparkShimImpl.getExprs @@ -1421,14 +1423,14 @@ > > val scans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = > commonScans ++ SparkShimImpl.getScans ++ ExternalSource.getScans -2731c3647 +2731c3649 < parent: Option[RapidsMeta[_, _]]): PartMeta[INPUT] = --- > parent: Option[RapidsMeta[_, _, _]]): PartMeta[INPUT] = -2761a3678,3679 +2746a3665,3666 > override def convertToGpu(): GpuPartitioning = > GpuHashPartitioning(childExprs.map(_.convertToGpu()), hp.numPartitions) -2771a3690,3697 +2756a3677,3684 > override def convertToGpu(): GpuPartitioning = { > if (rp.numPartitions > 1) { > val gpuOrdering = childExprs.map(_.convertToGpu()).asInstanceOf[Seq[SortOrder]] @@ -1437,21 +1439,21 @@ > GpuSinglePartitioning > } > } -2776a3703,3705 +2761a3690,3692 > override def convertToGpu(): GpuPartitioning = { > GpuRoundRobinPartitioning(rrp.numPartitions) > } -2781a3711 +2766a3698 > override def convertToGpu(): GpuPartitioning = GpuSinglePartitioning -2788c3718 +2773c3705 < parent: Option[RapidsMeta[_, _]]): DataWritingCommandMeta[INPUT] = --- > parent: Option[RapidsMeta[_, _, _]]): DataWritingCommandMeta[INPUT] = -2810c3740 +2795c3727 < parent: Option[RapidsMeta[_, _]]): SparkPlanMeta[INPUT] = --- > parent: Option[RapidsMeta[_, _, _]]): SparkPlanMeta[INPUT] = -2834c3764,3782 +2819c3751,3769 < (range, conf, p, r) => new SparkPlanMeta[RangeExec](range, conf, p, r) { --- > (range, conf, p, r) => { @@ -1473,16 +1475,16 @@ > > override def convertToGpu(): GpuExec = > GpuBatchScanExec(p.output, childScans.head.convertToGpu()) -2841a3790,3791 +2826a3777,3778 > override def convertToGpu(): GpuExec = > GpuCoalesceExec(coalesce.numPartitions, childPlans.head.convertIfNeeded()) -2849a3800 +2834a3787 > TypeSig.BINARY.withPsNote(TypeEnum.BINARY, "Only supported for Parquet") + -2855a3807,3809 +2840a3794,3796 > override def convertToGpu(): GpuExec = > GpuDataWritingCommandExec(childDataWriteCmds.head.convertToGpu(), > childPlans.head.convertIfNeeded()) -2870a3825,3849 +2855a3812,3836 > override def convertToGpu(): GpuExec = { > // To avoid metrics confusion we split a single stage up into multiple parts but only > // if there are multiple partitions to make it worth doing. @@ -1508,26 +1510,26 @@ > )(takeExec.sortOrder) > } > } -2878a3858,3859 +2863a3845,3846 > override def convertToGpu(): GpuExec = > GpuLocalLimitExec(localLimitExec.limit, childPlans.head.convertIfNeeded()) -2886a3868,3869 +2871a3855,3856 > override def convertToGpu(): GpuExec = > GpuGlobalLimitExec(globalLimitExec.limit, childPlans.head.convertIfNeeded(), 0) -2893,2896c3876 +2878,2881c3863 < (collectLimitExec, conf, p, r) => < new SparkPlanMeta[CollectLimitExec](collectLimitExec, conf, p, r) { < override val childParts: scala.Seq[PartMeta[_]] = < Seq(GpuOverrides.wrapPart(collectLimitExec.outputPartitioning, conf, Some(this)))}) --- > (collectLimitExec, conf, p, r) => new GpuCollectLimitMeta(collectLimitExec, conf, p, r)) -2905a3886,3887 +2890a3873,3874 > override def convertToGpu(): GpuExec = > GpuFilterExec(childExprs.head.convertToGpu(), childPlans.head.convertIfNeeded()) -2927a3910,3911 +2912a3897,3898 > override def convertToGpu(): GpuExec = > GpuUnionExec(childPlans.map(_.convertIfNeeded())) -2958a3943,3953 +2943a3930,3940 > override def convertToGpu(): GpuExec = { > val Seq(left, right) = childPlans.map(_.convertIfNeeded()) > val joinExec = GpuCartesianProductExec( @@ -1539,7 +1541,7 @@ > // as a filter after the join when possible. > condition.map(c => GpuFilterExec(c.convertToGpu(), joinExec)).getOrElse(joinExec) > } -2973a3969,3984 +2956a3954,3969 > exec[ObjectHashAggregateExec]( > "The backend for hash based aggregations supporting TypedImperativeAggregate functions", > ExecChecks( @@ -1556,16 +1558,16 @@ > "not allowed for grouping expressions if containing Array or Map as child"), > TypeSig.all), > (agg, conf, p, r) => new GpuObjectHashAggregateExecMeta(agg, conf, p, r)), -2980,2981d3990 +2963,2964d3975 < // SPARK 2.x we can't check for the TypedImperativeAggregate properly so < // map/array/struct left off -2984c3993 +2967c3978 < TypeSig.MAP + TypeSig.BINARY) --- > TypeSig.MAP + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.BINARY) -2986d3994 +2969d3979 < .withPsNote(TypeEnum.MAP, "not allowed for grouping expressions") -2988c3996,4000 +2971c3981,3985 < "converted between CPU and GPU"), --- > "converted between CPU and GPU") @@ -1573,39 +1575,39 @@ > "not allowed for grouping expressions") > .withPsNote(TypeEnum.STRUCT, > "not allowed for grouping expressions if containing Array or Map as child"), -2991,2992d4002 +2974,2975d3987 < // SPARK 2.x we can't check for the TypedImperativeAggregate properly so don't say we do the < // ObjectHashAggregate -3029c4039,4044 +3012c4024 < (sample, conf, p, r) => new GpuSampleExecMeta(sample, conf, p, r) {} --- > (sample, conf, p, r) => new GpuSampleExecMeta(sample, conf, p, r) -> ), +3013a4026,4031 > exec[SubqueryBroadcastExec]( > "Plan to collect and transform the broadcast key values", > ExecChecks(TypeSig.all, TypeSig.all), > (s, conf, p, r) => new GpuSubqueryBroadcastMeta(s, conf, p, r) -3030a4046 +> ), > SparkShimImpl.aqeShuffleReaderExec, -3049c4065 +3032c4050 < e.output.map(GpuOverrides.wrapExpr(_, conf, Some(this))) --- > e.resultAttrs.map(GpuOverrides.wrapExpr(_, conf, Some(this))) -3054a4071,4076 +3037a4056,4061 > > override def convertToGpu(): GpuExec = > GpuArrowEvalPythonExec(udfs.map(_.convertToGpu()).asInstanceOf[Seq[GpuPythonUDF]], > resultAttrs.map(_.convertToGpu()).asInstanceOf[Seq[Attribute]], > childPlans.head.convertIfNeeded(), > e.evalType) -3055a4078,4083 +3038a4063,4068 > exec[FlatMapCoGroupsInPandasExec]( > "The backend for CoGrouped Aggregation Pandas UDF, it runs on CPU itself now but supports" + > " scheduling GPU resources for the Python process when enabled", > ExecChecks.hiddenHack(), > (flatCoPy, conf, p, r) => new GpuFlatMapCoGroupsInPandasExecMeta(flatCoPy, conf, p, r)) > .disabledByDefault("Performance is not ideal now"), -3061a4090,4096 +3044a4075,4081 > exec[MapInPandasExec]( > "The backend for Map Pandas Iterator UDF. Accelerates the data transfer between the" + > " Java process and the Python process. It also supports scheduling GPU resources" + @@ -1613,7 +1615,7 @@ > ExecChecks((TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT).nested(), > TypeSig.all), > (mapPy, conf, p, r) => new GpuMapInPandasExecMeta(mapPy, conf, p, r)), -3067,3069c4102,4123 +3050,3052c4087,4108 < // SparkShimImpl.aqeShuffleReaderExec, < // SparkShimImpl.neverReplaceShowCurrentNamespaceCommand, < neverReplaceExec[ExecutedCommandExec]("Table metadata operation") @@ -1640,23 +1642,23 @@ > neverReplaceExec[AdaptiveSparkPlanExec]("Wrapper for adaptive query plan"), > neverReplaceExec[BroadcastQueryStageExec]("Broadcast query stage"), > neverReplaceExec[ShuffleQueryStageExec]("Shuffle query stage") -3073c4127 +3056c4112 < commonExecs ++ ShimGpuOverrides.shimExecs --- > commonExecs ++ SparkShimImpl.getExecs -3076,3078c4130 +3059,3061c4115 < // val key = SQLConf.LEGACY_TIME_PARSER_POLICY.key < val key = "2xgone" < val policy = SQLConf.get.getConfString(key, "EXCEPTION") --- > val policy = SQLConf.get.getConfString(SQLConf.LEGACY_TIME_PARSER_POLICY.key, "EXCEPTION") -3085a4138,4142 +3068a4123,4127 > val preRowToColProjection = TreeNodeTag[Seq[NamedExpression]]("rapids.gpu.preRowToColProcessing") > > val postColToRowProjection = TreeNodeTag[Seq[NamedExpression]]( > "rapids.gpu.postColToRowProcessing") > -3091a4149,4156 +3074a4134,4141 > private def doConvertPlan(wrap: SparkPlanMeta[SparkPlan], conf: RapidsConf, > optimizations: Seq[Optimization]): SparkPlan = { > val convertedPlan = wrap.convertIfNeeded() @@ -1665,7 +1667,7 @@ > sparkPlan > } > -3094c4159,4207 +3077c4144,4192 < Seq.empty --- > if (conf.optimizerEnabled) { @@ -1717,7 +1719,7 @@ > } > } > operator.withNewChildren(children) -3104,3105c4217,4223 +3087,3088c4202,4208 < // Only run the explain and don't actually convert or run on GPU. < def explainPotentialGpuPlan(df: DataFrame, explain: String = "ALL"): String = { --- @@ -1728,7 +1730,7 @@ > * to make it close to when the columnar rules would normally run on the plan. > */ > def explainPotentialGpuPlan(df: DataFrame, explain: String): String = { -3131a4250,4270 +3114a4235,4255 > /** > * Use explain mode on an active SQL plan as its processed through catalyst. > * This path is the same as being run through the plugin running on hosts with @@ -1750,17 +1752,17 @@ > } > } > -3154c4293 +3137c4278 < // case c2r: ColumnarToRowExec => prepareExplainOnly(c2r.child) --- > case c2r: ColumnarToRowExec => prepareExplainOnly(c2r.child) -3156,3157c4295,4296 +3139,3140c4280,4281 < // case aqe: AdaptiveSparkPlanExec => < // prepareExplainOnly(SparkShimImpl.getAdaptiveInputPlan(aqe)) --- > case aqe: AdaptiveSparkPlanExec => > prepareExplainOnly(SparkShimImpl.getAdaptiveInputPlan(aqe)) -3164,3168c4303,4385 +3147,3151c4288,4370 < // Spark 2.x < object GpuUserDefinedFunction { < // UDFs can support all types except UDT which does not have a clear columnar representation. @@ -1850,7 +1852,7 @@ > plan > } > } -3169a4387,4481 +3152a4372,4466 > /** > * Determine whether query is running against Delta Lake _delta_log JSON files or > * if Delta is doing stats collection that ends up hardcoding the use of AQE, diff --git a/scripts/spark2diffs/aggregate.diff b/scripts/spark2diffs/aggregate.diff index 11e291130af..927ff8d64c0 100644 --- a/scripts/spark2diffs/aggregate.diff +++ b/scripts/spark2diffs/aggregate.diff @@ -19,10 +19,9 @@ > import org.apache.spark.rdd.RDD > import org.apache.spark.sql.catalyst.InternalRow > import org.apache.spark.sql.catalyst.expressions.{Alias, Ascending, Attribute, AttributeReference, AttributeSeq, AttributeSet, Expression, ExprId, If, NamedExpression, NullsFirst, SortOrder} -24,26c36,46 +24,25c36,44 < import org.apache.spark.sql.execution.{SortExec, SparkPlan, TrampolineUtil} < import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} -< import org.apache.spark.sql.types.{ArrayType, DataType, MapType} --- > import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering > import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -33,7 +32,7 @@ > import org.apache.spark.sql.execution.aggregate.{BaseAggregateExec, HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} > import org.apache.spark.sql.rapids.{CpuToGpuAggregateBufferConverter, CudfAggregate, GpuAggregateExpression, GpuToCpuAggregateBufferConverter} > import org.apache.spark.sql.rapids.execution.{GpuShuffleMeta, TrampolineUtil} -> import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType} +26a46 > import org.apache.spark.sql.vectorized.ColumnarBatch 28,29c48 < // Spark 2.x - had to copy the GpuBaseAggregateMeta into each Hash and Sort Meta because no @@ -855,37 +854,7 @@ > rule: DataFromReplacementRule) extends SparkPlanMeta[INPUT](plan, conf, parent, rule) { > > val agg: BaseAggregateExec -51c871 -< // We don't support Arrays and Maps as GroupBy keys yet, even they are nested in Structs. So, ---- -> // We don't support Maps as GroupBy keys yet, even if they are nested in Structs. So, -53c873,882 -< val arrayOrMapGroupings = agg.groupingExpressions.exists(e => ---- -> val mapGroupings = agg.groupingExpressions.exists(e => -> TrampolineUtil.dataTypeExistsRecursively(e.dataType, -> dt => dt.isInstanceOf[MapType])) -> if (mapGroupings) { -> willNotWorkOnGpu("MapTypes in grouping expressions are not supported") -> } -> -> // We support Arrays as grouping expression but not if the child is a struct. So we need to -> // run recursive type check on the lists of structs -> val arrayWithStructsGroupings = agg.groupingExpressions.exists(e => -55,57c884,891 -< dt => dt.isInstanceOf[ArrayType] || dt.isInstanceOf[MapType])) -< if (arrayOrMapGroupings) { -< willNotWorkOnGpu("ArrayTypes or MapTypes in grouping expressions are not supported") ---- -> dt => dt match { -> case ArrayType(_: StructType, _) => true -> case _ => false -> }) -> ) -> if (arrayWithStructsGroupings) { -> willNotWorkOnGpu("ArrayTypes with Struct children in grouping expressions are not " + -> "supported") -60a895,907 +60a881,893 > > if (agg.aggregateExpressions.exists(expr => expr.isDistinct) > && agg.aggregateExpressions.exists(expr => expr.filter.isDefined)) { @@ -899,7 +868,7 @@ > if (AggregationTagging.mustReplaceBoth) { > tagForMixedReplacement() > } -124a972,1011 +124a958,997 > > /** Prevent mixing of CPU and GPU aggregations */ > private def tagForMixedReplacement(): Unit = { @@ -940,7 +909,7 @@ > conf.gpuTargetBatchSizeBytes, > conf.isTieredProjectEnabled) > } -127,128c1014,1020 +127,128c1000,1006 < class GpuSortAggregateExecMeta( < val agg: SortAggregateExec, --- @@ -951,7 +920,7 @@ > abstract class GpuTypedImperativeSupportedAggregateExecMeta[INPUT <: BaseAggregateExec]( > plan: INPUT, > aggRequiredChildDistributionExpressions: Option[Seq[Expression]], -130,132c1022,1030 +130,132c1008,1016 < parent: Option[RapidsMeta[_, _]], < rule: DataFromReplacementRule) < extends SparkPlanMeta[SortAggregateExec](agg, conf, parent, rule) { @@ -965,7 +934,7 @@ > expr.aggregateFunction.isInstanceOf[TypedImperativeAggregate[_]] && > (expr.mode == Partial || expr.mode == PartialMerge) > } -134,141c1032,1033 +134,141c1018,1019 < val groupingExpressions: Seq[BaseExprMeta[_]] = < agg.groupingExpressions.map(GpuOverrides.wrapExpr(_, conf, Some(this))) < val aggregateExpressions: Seq[BaseExprMeta[_]] = @@ -977,7 +946,7 @@ --- > // overriding data types of Aggregation Buffers if necessary > if (mayNeedAggBufferConversion) overrideAggBufTypes() -143,144c1035,1046 +143,144c1021,1032 < override val childExprs: Seq[BaseExprMeta[_]] = < groupingExpressions ++ aggregateExpressions ++ aggregateAttributes ++ resultExpressions --- @@ -993,7 +962,7 @@ > case aggMeta: TypedImperativeAggExprMeta[_] => aggMeta.supportBufferConversion > case _ => true > } -147,153c1049,1094 +147,153c1035,1080 < // We don't support Arrays and Maps as GroupBy keys yet, even they are nested in Structs. So, < // we need to run recursive type check on the structs. < val arrayOrMapGroupings = agg.groupingExpressions.exists(e => @@ -1048,9 +1017,9 @@ > conf.isTieredProjectEnabled) > } else { > super.convertToGpu() -154a1096 +154a1082 > } -156c1098,1344 +156c1084,1330 < tagForReplaceMode() --- > /** @@ -1300,7 +1269,7 @@ > agg.requiredChildDistributionExpressions, conf, parent, rule) { > override def tagPlanForGpu(): Unit = { > super.tagPlanForGpu() -194,256d1381 +194,256d1367 < < /** < * Tagging checks tied to configs that control the aggregation modes that are replaced. @@ -1364,15 +1333,15 @@ < s"Set ${conf.partialMergeDistinctEnabled} to true if desired") < } < } -259,261d1383 +259,261d1369 < // SPARK 2.x we can't check for the TypedImperativeAggregate properly so don't say we do the < // ObjectHashAggregate < /* -265c1387 +265c1373 < parent: Option[RapidsMeta[_, _]], --- > parent: Option[RapidsMeta[_, _, _]], -270c1392,1599 +270c1378,1585 < */ --- > /** diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 434abdc5899..9945b9045ab 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -2739,26 +2739,11 @@ object GpuOverrides extends Logging { // This needs to match what murmur3 supports. PartChecks(RepeatingParamCheck("hash_key", (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.STRUCT + TypeSig.ARRAY).nested(), - TypeSig.all) - ), + TypeSig.STRUCT).nested(), TypeSig.all)), (hp, conf, p, r) => new PartMeta[HashPartitioning](hp, conf, p, r) { override val childExprs: Seq[BaseExprMeta[_]] = hp.expressions.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - override def tagPartForGpu(): Unit = { - val arrayWithStructsHashing = hp.expressions.exists(e => - TrampolineUtil.dataTypeExistsRecursively(e.dataType, - dt => dt match { - case ArrayType(_: StructType, _) => true - case _ => false - }) - ) - if (arrayWithStructsHashing) { - willNotWorkOnGpu("hashing arrays with structs is not supported") - } - } - }), part[RangePartitioning]( "Range partitioning", @@ -2912,7 +2897,7 @@ object GpuOverrides extends Logging { .withPsNote(TypeEnum.STRUCT, "Round-robin partitioning is not supported for nested " + s"structs if ${SQLConf.SORT_BEFORE_REPARTITION.key} is true") .withPsNote( - Seq(TypeEnum.MAP), + Seq(TypeEnum.ARRAY, TypeEnum.MAP), "Round-robin partitioning is not supported if " + s"${SQLConf.SORT_BEFORE_REPARTITION.key} is true"), TypeSig.all), @@ -2961,12 +2946,10 @@ object GpuOverrides extends Logging { "The backend for hash based aggregations", ExecChecks( (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + - TypeSig.MAP + TypeSig.STRUCT + TypeSig.ARRAY) + TypeSig.MAP + TypeSig.ARRAY + TypeSig.STRUCT) .nested() - .withPsNote(TypeEnum.MAP, + .withPsNote(Seq(TypeEnum.ARRAY, TypeEnum.MAP), "not allowed for grouping expressions") - .withPsNote(TypeEnum.ARRAY, - "not allowed for grouping expressions if containing Struct as child") .withPsNote(TypeEnum.STRUCT, "not allowed for grouping expressions if containing Array or Map as child"), TypeSig.all), From 101b70144a0c3565499ab70511975569a4a85f65 Mon Sep 17 00:00:00 2001 From: Hao Zhu <9665750+viadea@users.noreply.github.com> Date: Wed, 5 Oct 2022 16:26:36 -0700 Subject: [PATCH 168/190] [Doc]Add 22.10 download page[skip ci] (#6670) * Add 2210 download page fix GCP custom image script Modify Doc for 2210: such as now we support AQE on Databricks Signed-off-by: Hao Zhu <9665750+viadea@users.noreply.github.com> * Add hive parquet write support Signed-off-by: Hao Zhu <9665750+viadea@users.noreply.github.com> * Add tool features Signed-off-by: Hao Zhu <9665750+viadea@users.noreply.github.com> * Add databricks docker solution Signed-off-by: Hao Zhu <9665750+viadea@users.noreply.github.com> * reword Signed-off-by: Hao Zhu <9665750+viadea@users.noreply.github.com> * add a link Signed-off-by: Hao Zhu <9665750+viadea@users.noreply.github.com> * Update docs/FAQ.md Co-authored-by: Sameer Raheja Signed-off-by: Hao Zhu <9665750+viadea@users.noreply.github.com> Co-authored-by: Sameer Raheja --- docs/FAQ.md | 43 ++++++------- docs/archive.md | 61 +++++++++++++++++++ docs/demo/AWS-EMR/Mortgage-ETL-GPU-EMR.ipynb | 2 +- docs/demo/Databricks/Mortgage-ETL-db.ipynb | 2 +- .../Databricks/generate-init-script.ipynb | 2 +- docs/dev/testing.md | 4 +- docs/download.md | 47 ++++---------- .../get-started/getting-started-databricks.md | 33 +++------- .../gpu_dataproc_packages_ubuntu_sample.sh | 14 +++-- 9 files changed, 117 insertions(+), 91 deletions(-) diff --git a/docs/FAQ.md b/docs/FAQ.md index 5cbef2e443b..b9811883a04 100644 --- a/docs/FAQ.md +++ b/docs/FAQ.md @@ -25,8 +25,7 @@ The RAPIDS Accelerator for Apache Spark officially supports: - [Azure Synapse](get-started/getting-started-azure-synapse-analytics.md) - Cloudera provides the plugin packaged through [CDS 3.2](https://docs.cloudera.com/cdp-private-cloud-base/7.1.7/cds-3/topics/spark-spark-3-overview.html) - which is supported on the following - [CDP Private Cloud Base releases](https://docs.cloudera.com/cdp-private-cloud-base/7.1.7/cds-3/topics/spark-3-requirements.html). + and [CDS 3.3](https://docs.cloudera.com/cdp-private-cloud-base/7.1.8/cds-3/topics/spark-spark-3-overview.html). Most distributions based on a supported Apache Spark version should work, but because the plugin replaces parts of the physical plan that Apache Spark considers to be internal the code for those @@ -40,7 +39,7 @@ release. ### What hardware is supported? -The plugin is tested and supported on V100, T4, A2, A10, A30 and A100 datacenter GPUs. It is possible +The plugin is tested and supported on P100, V100, T4, A2, A10, A30 and A100 datacenter GPUs. It is possible to run the plugin on GeForce desktop hardware with Volta or better architectures. GeForce hardware does not support [CUDA forward compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html#forward-compatibility-title), @@ -60,21 +59,17 @@ More information about cards that support forward compatibility can be found On startup the RAPIDS Accelerator will log a warning message on the Spark driver showing the version with a message that looks something like this: ``` -21/04/14 22:14:55 WARN SQLExecPlugin: RAPIDS Accelerator 21.06.0 using cudf 21.06.0. To disable GPU support set `spark.rapids.sql.enabled` to false +WARN RapidsPluginUtils: RAPIDS Accelerator 22.10.0 using cudf 22.10.0. ``` -The full RAPIDS Accelerator and cudf build properties are logged at `INFO` level in the -Spark driver and executor logs with messages that are similar to the following: +The full RAPIDS Accelerator, RAPIDS Accelerator JNI and cudf build properties are logged at `INFO` +level in the Spark driver and executor logs with messages that are similar to the following: ``` -21/04/14 17:20:20 INFO RapidsExecutorPlugin: RAPIDS Accelerator build: {version=0.5.0-SNAPSHOT, user=jlowe, url=, date=2021-04-14T22:12:14Z, revision=79a5cf8acd615587b2c7835072b0d8b0d4604f8b, cudf_version=0.19-SNAPSHOT, branch=branch-0.5} -21/04/14 17:20:20 INFO RapidsExecutorPlugin: cudf build: {version=0.19-SNAPSHOT, user=, date=2021-04-13T08:42:40Z, revision=a5d2407b93de444a6a7faf9db4b7dbf4ecbfe9ed, branch=HEAD} +INFO RapidsPluginUtils: RAPIDS Accelerator build: {version=22.10.0-SNAPSHOT, user=, url=https://github.com/NVIDIA/spark-rapids.git, date=2022-09-02T12:41:30Z, revision=66450a3549d7cbb23799ec7be2f6f02b253efb85, cudf_version=22.10.0-SNAPSHOT, branch=HEAD} +INFO RapidsPluginUtils: RAPIDS Accelerator JNI build: {version=22.10.0-SNAPSHOT, user=, url=https://github.com/NVIDIA/spark-rapids-jni.git, date=2022-09-02T03:35:21Z, revision=76b71b9ffa1fa4237365b51485d11362cbfb99e5, branch=HEAD} +INFO RapidsPluginUtils: cudf build: {version=22.10.0-SNAPSHOT, user=, url=https://github.com/rapidsai/cudf.git, date=2022-09-02T03:35:21Z, revision=c273da4d6285d6b6f9640585cb3b8cf11310bef6, branch=HEAD} ``` -### What is the right hardware setup to run GPU accelerated Spark? - -GPU accelerated Spark can run on any NVIDIA Pascal or better GPU architecture, including Volta, -Turing or Ampere. - ### What parts of Apache Spark are accelerated? Currently a limited set of SQL and DataFrame operations are supported, please see the @@ -276,9 +271,6 @@ Queries on Databricks will not fail but it can not benefit from DPP. Any operation that is supported on GPU will stay on the GPU when AQE is enabled. -AQE is not supported on Databricks with the plugin. -If AQE is enabled on Databricks, queries may fail with `StackOverflowError` error. - #### Why does my query show as not on the GPU when Adaptive Query Execution is enabled? When running an `explain()` on a query where AQE is on, it is possible that AQE has not finalized @@ -364,11 +356,11 @@ Yes The GPU is not needed on the driver and there is no benefit to having one available on the driver for the RAPIDS plugin. -### How does the performance compare to Databricks' DeltaEngine? +### Are table layout formats supported? -We have not evaluated the performance yet. DeltaEngine is not open source, so any analysis needs to -be done with Databricks in some form. When DeltaEngine is generally available and the terms of -service allow it, we will look into doing a comparison. +Yes, there is GPU support for Delta Lake read +and [Apache Iceberg 0.13 read](./additional-functionality/iceberg-support.md). +Write operations will fall back to the CPU. ### How many tasks can I run per executor? How many should I run per executor? @@ -496,7 +488,7 @@ This is typically caused by the IOMMU being enabled. Please see the for this issue. To fix it you can either disable the IOMMU, or you can disable using pinned memory by setting -[spark.rapids.memory.pinnedPool.size](configs.md#memory.pinnedPool.size) to 0. +[`spark.rapids.memory.pinnedPool.size`](configs.md#memory.pinnedPool.size) to 0. ### Why am I getting a buffer overflow error when using the KryoSerializer? Buffer overflow will happen when trying to serialize an object larger than @@ -558,9 +550,14 @@ If you are getting a warning `Avro library not found by the RAPIDS plugin.` or i Spark job by using the `--jars` or `--packages` option followed by the file path or maven path to RAPIDS jar since that is the preferred way to run RAPIDS accelerator. -Note, you can add locally installed jars for external packages such as Avro Data Sources and the RAPIDS Accelerator jars via `spark.driver.extraClassPath` (--driver-class-path in the client mode) on the driver side, and `spark.executor.extraClassPath` on the executor side. However, you should not mix the deploy methods for either of the external modules. Either deploy both Spark Avro and RAPIDS Accelerator jars as local jars via `extraClassPath` settings or use the `--jars` or `--packages` options. +Note, you can add locally installed jars for external packages such as Avro Data Sources and the +RAPIDS Accelerator jars via `spark.driver.extraClassPath` (--driver-class-path in the client mode) +on the driver side, and `spark.executor.extraClassPath` on the executor side. However, you should not +mix the deploy methods for either of the external modules. Either deploy both Spark Avro and RAPIDS +Accelerator jars as local jars via `extraClassPath` settings or use the `--jars` or `--packages` options. -As a consequence, per Issue #5796, if you also use the RAPIDS Shuffle Manager, your deployment option may be limited to the extraClassPath method. +As a consequence, per [issue-5796](https://github.com/NVIDIA/spark-rapids/issues/5796), if you also +use the RAPIDS Shuffle Manager, your deployment option may be limited to the extraClassPath method. ### What is the default RMM pool allocator? diff --git a/docs/archive.md b/docs/archive.md index f0daacd41c3..8e9fc6233c1 100644 --- a/docs/archive.md +++ b/docs/archive.md @@ -5,6 +5,67 @@ nav_order: 15 --- Below are archived releases for RAPIDS Accelerator for Apache Spark. +## Release v22.08.0 +Hardware Requirements: + +The plugin is tested on the following architectures: + + GPU Models: NVIDIA V100, T4 and A2/A10/A30/A100 GPUs + +Software Requirements: + + OS: Ubuntu 18.04, Ubuntu 20.04 or CentOS 7, Rocky Linux 8 + + CUDA & NVIDIA Drivers*: 11.x & v450.80.02+ + + Apache Spark 3.1.1, 3.1.2, 3.1.3, 3.2.0, 3.2.1, 3.2.2, 3.3.0, Databricks 9.1 ML LTS or 10.4 ML LTS Runtime and GCP Dataproc 2.0 + + Python 3.6+, Scala 2.12, Java 8 + +*Some hardware may have a minimum driver version greater than v450.80.02+. Check the GPU spec sheet +for your hardware's minimum driver version. + +*For Cloudera and EMR support, please refer to the +[Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. + +### Download v22.08.0 +* Download the [RAPIDS + Accelerator for Apache Spark 22.08.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.08.0/rapids-4-spark_2.12-22.08.0.jar) + +This package is built against CUDA 11.5 and all CUDA 11.x versions are supported through [CUDA forward +compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested +on V100, T4, A2, A10, A30 and A100 GPUs with CUDA 11.0-11.5. For those using other types of GPUs which +do not have CUDA forward compatibility (for example, GeForce), CUDA 11.5 or later is required. Users will +need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. + +### Verify signature +* Download the [RAPIDS Accelerator for Apache Spark 22.08.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.08.0/rapids-4-spark_2.12-22.08.0.jar) + and [RAPIDS Accelerator for Apache Spark 22.08.0 jars.asc](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.08.0/rapids-4-spark_2.12-22.08.0.jar.asc) +* Download the [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com). +* Import the public key: `gpg --import PUB_KEY` +* Verify the signature: `gpg --verify rapids-4-spark_2.12-22.08.0.jar.asc rapids-4-spark_2.12-22.08.0.jar` + +The output if signature verify: + + gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) " + +### Release Notes +New functionality and performance improvements for this release include: +* Rocky Linux 8 support +* Ability to build Spark RAPIDS jars for Java versions 9+ +* Zstandard Parquet and ORC read support +* Binary read support from parquet +* Apache Iceberg 0.13 support +* Array function support: array_intersect, array_union, array_except and arrays_overlap +* Support nth_value, first and last in windowing function +* Alluxio auto mount for AWS S3 buckets +* Qualification tool: + * SQL level qualification + * Add application details view + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + ## Release v22.06.0 Hardware Requirements: diff --git a/docs/demo/AWS-EMR/Mortgage-ETL-GPU-EMR.ipynb b/docs/demo/AWS-EMR/Mortgage-ETL-GPU-EMR.ipynb index 3e24c498f23..d136b6471da 100644 --- a/docs/demo/AWS-EMR/Mortgage-ETL-GPU-EMR.ipynb +++ b/docs/demo/AWS-EMR/Mortgage-ETL-GPU-EMR.ipynb @@ -10,7 +10,7 @@ "source": [ "## Data Source\n", "\n", - "Dataset is from Fannie Mae’s [Single-Family Loan Performance Data](http://www.fanniemae.com/portal/funding-the-market/data/loan-performance-data.html). Refer to these [instructions](https://github.com/NVIDIA/spark-rapids-examples/blob/branch-22.08/docs/get-started/xgboost-examples/dataset/mortgage.md) to download the dataset from the Fannie Mae website.\n", + "Dataset is from Fannie Mae’s [Single-Family Loan Performance Data](http://www.fanniemae.com/portal/funding-the-market/data/loan-performance-data.html). Refer to these [instructions](https://github.com/NVIDIA/spark-rapids-examples/blob/branch-22.10/docs/get-started/xgboost-examples/dataset/mortgage.md) to download the dataset from the Fannie Mae website.\n", "\n", "## Prerequisite\n", "\n", diff --git a/docs/demo/Databricks/Mortgage-ETL-db.ipynb b/docs/demo/Databricks/Mortgage-ETL-db.ipynb index 3fb281ce593..0c3e7db8563 100644 --- a/docs/demo/Databricks/Mortgage-ETL-db.ipynb +++ b/docs/demo/Databricks/Mortgage-ETL-db.ipynb @@ -12,7 +12,7 @@ }, "source": [ "## Prerequirement\n", - "Dataset is derived from Fannie Mae’s [Single-Family Loan Performance Data](http://www.fanniemae.com/portal/funding-the-market/data/loan-performance-data.html) with all rights reserved by Fannie Mae. Refer to these [instructions](https://github.com/NVIDIA/spark-rapids-examples/blob/branch-22.08/docs/get-started/xgboost-examples/dataset/mortgage.md) to download the dataset." + "Dataset is derived from Fannie Mae’s [Single-Family Loan Performance Data](http://www.fanniemae.com/portal/funding-the-market/data/loan-performance-data.html) with all rights reserved by Fannie Mae. Refer to these [instructions](https://github.com/NVIDIA/spark-rapids-examples/blob/branch-22.10/docs/get-started/xgboost-examples/dataset/mortgage.md) to download the dataset." ] }, { diff --git a/docs/demo/Databricks/generate-init-script.ipynb b/docs/demo/Databricks/generate-init-script.ipynb index 12b845f3d35..87f8f0ceb90 100644 --- a/docs/demo/Databricks/generate-init-script.ipynb +++ b/docs/demo/Databricks/generate-init-script.ipynb @@ -3,7 +3,7 @@ { "cell_type":"code", "source":[ - "dbutils.fs.mkdirs(\"dbfs:/databricks/init_scripts/\")\n \ndbutils.fs.put(\"/databricks/init_scripts/init.sh\",\"\"\"\n#!/bin/bash\nsudo wget -O /databricks/jars/rapids-4-spark_2.12-22.08.0.jar https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.08.0/rapids-4-spark_2.12-22.08.0.jar\n\"\"\", True)" + "dbutils.fs.mkdirs(\"dbfs:/databricks/init_scripts/\")\n \ndbutils.fs.put(\"/databricks/init_scripts/init.sh\",\"\"\"\n#!/bin/bash\nsudo wget -O /databricks/jars/rapids-4-spark_2.12-22.10.0.jar https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.10.0/rapids-4-spark_2.12-22.10.0.jar\n\"\"\", True)" ], "metadata":{ diff --git a/docs/dev/testing.md b/docs/dev/testing.md index 62a198217ec..b71aa195413 100644 --- a/docs/dev/testing.md +++ b/docs/dev/testing.md @@ -5,5 +5,5 @@ nav_order: 2 parent: Developer Overview --- An overview of testing can be found within the repository at: -* [Unit tests](https://github.com/NVIDIA/spark-rapids/tree/branch-22.08/tests#readme) -* [Integration testing](https://github.com/NVIDIA/spark-rapids/tree/branch-22.08/integration_tests#readme) +* [Unit tests](https://github.com/NVIDIA/spark-rapids/tree/branch-22.10/tests#readme) +* [Integration testing](https://github.com/NVIDIA/spark-rapids/tree/branch-22.10/integration_tests#readme) diff --git a/docs/download.md b/docs/download.md index a429086d012..509bd447f17 100644 --- a/docs/download.md +++ b/docs/download.md @@ -18,12 +18,12 @@ cuDF jar, that is either preinstalled in the Spark classpath on all nodes or sub that uses the RAPIDS Accelerator For Apache Spark. See the [getting-started guide](https://nvidia.github.io/spark-rapids/Getting-Started/) for more details. -## Release v22.08.0 +## Release v22.10.0 Hardware Requirements: The plugin is tested on the following architectures: - GPU Models: NVIDIA V100, T4 and A2/A10/A30/A100 GPUs + GPU Models: NVIDIA P100, V100, T4 and A2/A10/A30/A100 GPUs Software Requirements: @@ -41,40 +41,19 @@ for your hardware's minimum driver version. *For Cloudera and EMR support, please refer to the [Distributions](./FAQ.md#which-distributions-are-supported) section of the FAQ. -### Download v22.08.0 -* Download the [RAPIDS - Accelerator for Apache Spark 22.08.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.08.0/rapids-4-spark_2.12-22.08.0.jar) - -This package is built against CUDA 11.5 and all CUDA 11.x versions are supported through [CUDA forward -compatibility](https://docs.nvidia.com/deploy/cuda-compatibility/index.html). It is tested -on V100, T4, A2, A10, A30 and A100 GPUs with CUDA 11.0-11.5. For those using other types of GPUs which -do not have CUDA forward compatibility (for example, GeForce), CUDA 11.5 or later is required. Users will -need to ensure the minimum driver (450.80.02) and CUDA toolkit are installed on each Spark node. - -### Verify signature -* Download the [RAPIDS Accelerator for Apache Spark 22.08.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.08.0/rapids-4-spark_2.12-22.08.0.jar) - and [RAPIDS Accelerator for Apache Spark 22.08.0 jars.asc](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.08.0/rapids-4-spark_2.12-22.08.0.jar.asc) -* Download the [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com). -* Import the public key: `gpg --import PUB_KEY` -* Verify the signature: `gpg --verify rapids-4-spark_2.12-22.08.0.jar.asc rapids-4-spark_2.12-22.08.0.jar` - -The output if signature verify: - - gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) " - ### Release Notes New functionality and performance improvements for this release include: -* Rocky Linux 8 support -* Ability to build Spark RAPIDS jars for Java versions 9+ -* Zstandard Parquet and ORC read support -* Binary read support from parquet -* Apache Iceberg 0.13 support -* Array function support: array_intersect, array_union, array_except and arrays_overlap -* Support nth_value, first and last in windowing function -* Alluxio auto mount for AWS S3 buckets -* Qualification tool: - * SQL level qualification - * Add application details view +* Dataproc qualification, profiling, bootstrap and diagnostic tool +* Databricks custom docker container +* AQE support on Databricks +* MultiThreaded Shuffle feature +* HashAggregate on Array support +* Binary write support for parquet +* Cast binary to string +* Hive parquet table write support +* Qualification and Profiling tool: + * Print Databricks cluster/job information + * AutoTuner for Profiling tool For a detailed list of changes, please refer to the [CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). diff --git a/docs/get-started/getting-started-databricks.md b/docs/get-started/getting-started-databricks.md index 07586e9644b..82d3d3a4ab2 100644 --- a/docs/get-started/getting-started-databricks.md +++ b/docs/get-started/getting-started-databricks.md @@ -23,17 +23,7 @@ The number of GPUs per node dictates the number of Spark executors that can run ## Limitations -1. Adaptive query execution(AQE) and Delta optimization write do not work. These should be disabled -when using the plugin. Queries may still see significant speedups even with AQE disabled. - - ```bash - spark.databricks.delta.optimizeWrite.enabled false - spark.sql.adaptive.enabled false - ``` - - See [issue-1059](https://github.com/NVIDIA/spark-rapids/issues/1059) for more detail. - -2. Dynamic partition pruning(DPP) does not work. This results in poor performance for queries which +1. Dynamic partition pruning(DPP) does not work. This results in poor performance for queries which would normally benefit from DPP. With DPP on, queries may fail on Databricks when using the plugin. ```bash @@ -42,17 +32,18 @@ when using the plugin. Queries may still see significant speedups even with AQE See [issue-3143](https://github.com/NVIDIA/spark-rapids/issues/3143) for more detail. -3. When selecting GPU nodes, Databricks requires the driver node to be a GPU node. Outside of - Databricks the plugin can operate with the driver as a CPU node and workers as GPU nodes. +2. When selecting GPU nodes, Databricks UI requires the driver node to be a GPU node. However you + can use Databricks API to create a cluster with CPU driver node. + Outside of Databricks the plugin can operate with the driver as a CPU node and workers as GPU nodes. -4. Cannot spin off multiple executors on a multi-GPU node. +3. Cannot spin off multiple executors on a multi-GPU node. Even though it is possible to set `spark.executor.resource.gpu.amount=1` in the in Spark Configuration tab, Databricks overrides this to `spark.executor.resource.gpu.amount=N` (where N is the number of GPUs per node). This will result in failed executors when starting the cluster. -5. Parquet rebase mode is set to "LEGACY" by default. +4. Parquet rebase mode is set to "LEGACY" by default. The following Spark configurations are set to `LEGACY` by default on Databricks: @@ -65,7 +56,7 @@ when using the plugin. Queries may still see significant speedups even with AQE If you do not need `LEGACY` write semantics, set these configs to `EXCEPTION` which is the default value in Apache Spark 3.0 and higher. -6. Databricks makes changes to the runtime without notification. +5. Databricks makes changes to the runtime without notification. Databricks makes changes to existing runtimes, applying patches, without notification. [Issue-3098](https://github.com/NVIDIA/spark-rapids/issues/3098) is one example of this. We run @@ -133,17 +124,13 @@ cluster. Note: Please remove the `spark.task.resource.gpu.amount` config for a single-node Databricks cluster because Spark local mode does not support GPU scheduling. - There is an incompatibility between the Databricks specific implementation of adaptive query - execution (AQE) and the spark-rapids plugin. In order to mitigate this, - `spark.sql.adaptive.enabled` should be set to false. In addition, the plugin does not work with - the Databricks `spark.databricks.delta.optimizeWrite` option. + The plugin does not work with the Databricks `spark.databricks.delta.optimizeWrite` option. ```bash spark.plugins com.nvidia.spark.SQLPlugin spark.task.resource.gpu.amount 0.1 spark.rapids.memory.pinnedPool.size 2G spark.databricks.delta.optimizeWrite.enabled false - spark.sql.adaptive.enabled false spark.sql.optimizer.dynamicPartitionPruning.enabled false spark.rapids.sql.concurrentGpuTasks 2 ``` @@ -163,7 +150,7 @@ cluster. ```bash spark.rapids.sql.python.gpu.enabled true spark.python.daemon.module rapids.daemon_databricks - spark.executorEnv.PYTHONPATH /databricks/jars/rapids-4-spark_2.12-22.08.0.jar:/databricks/spark/python + spark.executorEnv.PYTHONPATH /databricks/jars/rapids-4-spark_2.12-22.10.0.jar:/databricks/spark/python ``` 7. Once you’ve added the Spark config, click “Confirm and Restart”. @@ -171,7 +158,7 @@ cluster. ## Import the GPU Mortgage Example Notebook Import the example [notebook](../demo/Databricks/Mortgage-ETL-db.ipynb) from the repo into your -workspace, then open the notebook. Please find this [instruction](https://github.com/NVIDIA/spark-rapids-examples/blob/branch-22.08/docs/get-started/xgboost-examples/dataset/mortgage.md) +workspace, then open the notebook. Please find this [instruction](https://github.com/NVIDIA/spark-rapids-examples/blob/branch-22.10/docs/get-started/xgboost-examples/dataset/mortgage.md) to download the dataset. ```bash diff --git a/docs/get-started/gpu_dataproc_packages_ubuntu_sample.sh b/docs/get-started/gpu_dataproc_packages_ubuntu_sample.sh index 6029a6978e5..1428eb8516e 100644 --- a/docs/get-started/gpu_dataproc_packages_ubuntu_sample.sh +++ b/docs/get-started/gpu_dataproc_packages_ubuntu_sample.sh @@ -20,7 +20,7 @@ OS_NAME=$(lsb_release -is | tr '[:upper:]' '[:lower:]') readonly OS_NAME OS_DIST=$(lsb_release -cs) readonly OS_DIST -CUDA_VERSION='11.0' +CUDA_VERSION='11.5' readonly CUDA_VERSION readonly NVIDIA_BASE_DL_URL='https://developer.download.nvidia.com/compute' @@ -76,7 +76,7 @@ function install_nvidia_gpu_driver() { local -r cuda_package=cuda-toolkit fi # Without --no-install-recommends this takes a very long time. - execute_with_retries "apt-get install -y -q --no-install-recommends cuda-drivers-460" + execute_with_retries "apt-get install -y -q --no-install-recommends cuda-drivers-495" execute_with_retries "apt-get install -y -q --no-install-recommends ${cuda_package}" ldconfig echo "NVIDIA GPU driver provided by NVIDIA was installed successfully" @@ -139,9 +139,9 @@ EOF systemctl start dataproc-cgroup-device-permissions } -readonly DEFAULT_SPARK_RAPIDS_VERSION="22.08.0" -readonly DEFAULT_CUDA_VERSION="11.0" -readonly DEFAULT_XGBOOST_VERSION="1.6.1" +readonly DEFAULT_SPARK_RAPIDS_VERSION="22.10.0" +readonly DEFAULT_CUDA_VERSION="11.5" +readonly DEFAULT_XGBOOST_VERSION="1.6.2" readonly SPARK_VERSION="3.0" # SPARK config @@ -150,7 +150,7 @@ readonly XGBOOST_VERSION=${DEFAULT_XGBOOST_VERSION} readonly XGBOOST_GPU_SUB_VERSION=${DEFAULT_XGBOOST_GPU_SUB_VERSION} function install_spark_rapids() { - local -r rapids_repo_url='https://repo1.maven.org/maven2/ai/rapids' + local -r nvidia_repo_url='https://repo1.maven.org/maven2/com/nvidia' local -r dmlc_repo_url='https://repo.maven.apache.org/maven2/ml/dmlc' # Convert . to - for URL formatting @@ -188,6 +188,8 @@ spark.executor.resource.gpu.amount=1 spark.plugins=com.nvidia.spark.SQLPlugin spark.executor.resource.gpu.discoveryScript=/usr/lib/spark/scripts/gpu/getGpusResources.sh spark.submit.pyFiles=/usr/lib/spark/jars/xgboost4j-spark_${SPARK_VERSION}-${XGBOOST_VERSION}-${XGBOOST_GPU_SUB_VERSION}.jar +spark.dynamicAllocation.enabled=false +spark.sql.files.maxPartitionBytes=512m ###### END : RAPIDS properties for Spark ${SPARK_VERSION} ###### EOF } From b9f909b0ce9a9d9e821738ca76f60f77a5e1932c Mon Sep 17 00:00:00 2001 From: MithunR Date: Wed, 5 Oct 2022 20:20:56 -0700 Subject: [PATCH 169/190] Fixed validity checks for large decimal window bounds: (#6691) --- .../src/main/python/window_function_test.py | 28 +++++++++++++++++++ .../spark/rapids/GpuWindowExpression.scala | 4 +-- 2 files changed, 30 insertions(+), 2 deletions(-) diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py index 87411cbe93b..a9fcb1a6db2 100644 --- a/integration_tests/src/main/python/window_function_test.py +++ b/integration_tests/src/main/python/window_function_test.py @@ -862,6 +862,34 @@ def test_window_aggregations_for_decimal_ranges(data_gen): conf={}) +# In a distributed setup the order of the partitions returned might be different, so we must ignore the order +# but small batch sizes can make sort very slow, so do the final order by locally +@ignore_order(local=True) +@pytest.mark.parametrize('data_gen', [_grpkey_longs_with_nullable_largest_decimals], + ids=idfn) +def test_window_aggregations_for_big_decimal_ranges(data_gen): + """ + Tests for range window aggregations, with DECIMAL order by columns. + The table schema used: + a: Group By column + b: Order By column (decimal) + c: Aggregation column (incidentally, also decimal) + + Since this test is for the order-by column type, and not for each specific windowing aggregation, + we use COUNT(1) throughout the test, for different window widths and ordering. + Some other aggregation functions are thrown in for variety. + """ + assert_gpu_and_cpu_are_equal_sql( + lambda spark: gen_df(spark, data_gen, length=2048), + "window_agg_table", + 'SELECT ' + ' COUNT(1) OVER (PARTITION BY a ORDER BY b ASC ' + ' RANGE BETWEEN 12345678901234567890123456789012345.12 PRECEDING ' + ' AND 11111111112222222222333333333344444.12 FOLLOWING) ' + 'FROM window_agg_table', + conf={}) + + _gen_data_for_collect_list = [ ('a', RepeatSeqGen(LongGen(), length=20)), ('b', LongRangeGen()), diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala index cd4d5b59090..ac9f5683cea 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala @@ -376,12 +376,12 @@ abstract class GpuSpecifiedWindowFrameMetaBase( return None } - val value: Long = bounds match { + val value: BigInt = bounds match { case Literal(value, ByteType) => value.asInstanceOf[Byte].toLong case Literal(value, ShortType) => value.asInstanceOf[Short].toLong case Literal(value, IntegerType) => value.asInstanceOf[Int].toLong case Literal(value, LongType) => value.asInstanceOf[Long] - case Literal(value: Decimal, DecimalType()) => value.toLong + case Literal(value: Decimal, DecimalType()) => value.toJavaBigDecimal.unscaledValue() case Literal(ci: CalendarInterval, CalendarIntervalType) => if (ci.months != 0) { willNotWorkOnGpu("interval months isn't supported") From 16ef1624c0c887f1b4671e845d74adce6b267e1c Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 6 Oct 2022 07:50:17 -0500 Subject: [PATCH 170/190] Suppress warning for jdk11 Finalize method deprecatio (#6705) Signed-off-by: Thomas Graves Signed-off-by: Thomas Graves --- .../spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala index b9fe46efe79..1daf3b3b074 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala @@ -225,6 +225,7 @@ class SerializeConcatHostBuffersDeserializeBatch( Option(batchInternal).foreach(_.close()) } + @scala.annotation.nowarn("msg=method finalize in class Object is deprecated") override def finalize(): Unit = { super.finalize() close() From 9b6b62aeee7e7119dc40c93504369aa828cddcc3 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 6 Oct 2022 07:51:17 -0500 Subject: [PATCH 171/190] Fix AutoTuner yaml error handling and discovery script rounding (#6697) * Fix misc things with AutoTuner and scripts Signed-off-by: Thomas Graves * kick git test * ifix line length Signed-off-by: Thomas Graves --- tools/scripts/discoveryScript.sh | 5 +- .../rapids/tool/profiling/AutoTuner.scala | 16 +-- .../tool/profiling/AutoTunerSuite.scala | 110 +++++++++++++++--- 3 files changed, 106 insertions(+), 25 deletions(-) mode change 100644 => 100755 tools/scripts/discoveryScript.sh diff --git a/tools/scripts/discoveryScript.sh b/tools/scripts/discoveryScript.sh old mode 100644 new mode 100755 index b63aa746785..1674149e146 --- a/tools/scripts/discoveryScript.sh +++ b/tools/scripts/discoveryScript.sh @@ -80,9 +80,8 @@ function get_gpu_properties() { IFS=',' read -ra gpuInfoArr <<< "$gpuInfo" gpuCount=${gpuInfoArr[0]} gpuName=${gpuInfoArr[1]} - local gpuMemoryInMb="$(echo ${gpuInfoArr[2]} | cut -d' ' -f1)" - gpuMemoryInGb=$((gpuMemoryInMb / 1024)) - gpuMemoryInGb="${gpuMemoryInGb}gb" + # GPU memory should have units but separated by space so remove the space + gpuMemoryInMb="$(echo ${gpuInfoArr[2]} | sed 's/ //g')" } function get_disk_space() { diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala index 5605f067455..c726048b5d9 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala @@ -45,10 +45,12 @@ class GpuWorkerProps( this("0m", 0, "None") } def isMissingInfo: Boolean = { - count == 0 || memory.startsWith("0") || name == "None" + memory == null || memory.isEmpty || name == null || name.isEmpty || + count == 0 || memory.startsWith("0") || name == "None" } def isEmpty: Boolean = { - count == 0 && memory.startsWith("0") && name == "None" + count == 0 && (memory == null || memory.isEmpty || memory.startsWith("0")) && + (name == null || name.isEmpty || name == "None") } /** * If the GPU count is missing, it will set 1 as a default value @@ -64,7 +66,7 @@ class GpuWorkerProps( } } def setDefaultGpuNameIfMissing: Boolean = { - if (name == "None") { + if (name == null || name.isEmpty || name == "None") { name = AutoTuner.DEF_WORKER_GPU_NAME true } else { @@ -75,13 +77,13 @@ class GpuWorkerProps( /** * If the GPU memory is missing, it will sets a default valued based on the GPU device and the * static HashMap [[AutoTuner.DEF_WORKER_GPU_MEMORY_MB]]. - * If it is still missing, it sets a default to 16384m. + * If it is still missing, it sets a default to 15109m. * * @return true if the value has been updated. */ def setDefaultGpuMemIfMissing: Boolean = { - if (memory.startsWith("0")) { - memory = AutoTuner.DEF_WORKER_GPU_MEMORY_MB.getOrElse(getName, "16384m") + if (memory == null || memory.isEmpty || memory.startsWith("0")) { + memory = AutoTuner.DEF_WORKER_GPU_MEMORY_MB.getOrElse(getName, "15109m") true } else { false @@ -129,7 +131,7 @@ class SystemClusterProps( } def isEmpty: Boolean = { // consider the object incorrect if either numCores or memory are not set. - numCores <= 0 || memory.startsWith("0") + memory == null || memory.isEmpty || numCores <= 0 || memory.startsWith("0") } def setDefaultNumWorkersIfMissing(): Boolean = { if (numWorkers <= 0) { diff --git a/tools/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AutoTunerSuite.scala b/tools/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AutoTunerSuite.scala index ebaa1de3788..2e5824f473a 100644 --- a/tools/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AutoTunerSuite.scala +++ b/tools/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AutoTunerSuite.scala @@ -58,9 +58,9 @@ class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { gpuMemory: Option[String] = Some("15109MiB"), gpuDevice: Option[String] = Some("T4")): String = { val gpuWorkerProps = new GpuWorkerProps( - gpuMemory.getOrElse("15109MiB"), gpuCount.getOrElse(2), gpuDevice.getOrElse("T4")) + gpuMemory.getOrElse(""), gpuCount.getOrElse(0), gpuDevice.getOrElse("")) val cpuSystem = new SystemClusterProps( - numCores.getOrElse(32), systemMemory.getOrElse("122880MiB"), numWorkers.getOrElse(4)) + numCores.getOrElse(0), systemMemory.getOrElse(""), numWorkers.getOrElse(0)) val systemProperties = customProps match { case None => mutable.Map[String, String]() case Some(newProps) => newProps @@ -97,7 +97,7 @@ class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { assert(expectedResults == autoTunerOutput) } - test("Load cluster properties with missing CPU cores") { + test("Load cluster properties with CPU cores 0") { val dataprocWorkerInfo = buildWorkerInfoAsString(None, Some(0)) val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) val (properties, comments) = autoTuner.getRecommendedProperties() @@ -119,8 +119,31 @@ class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { assert(expectedResults == autoTunerOutput) } - test("Load cluster properties with missing CPU memory") { - val dataprocWorkerInfo = buildWorkerInfoAsString(None, None, Some("0m")) + test("Load cluster properties with CPU memory missing") { + val dataprocWorkerInfo = buildWorkerInfoAsString(None, Some(32), None) + val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) + val (properties, comments) = autoTuner.getRecommendedProperties() + val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) + // scalastyle:off line.size.limit + val expectedResults = + s"""|Cannot recommend properties. See Comments. + | + |Comments: + |- Incorrect values in worker system information: {numCores: 32, memory: , numWorkers: 4}. + |- 'spark.executor.memory' should be set to at least 2GB/core. + |- 'spark.executor.instances' should be set to (gpuCount * numWorkers). + |- 'spark.task.resource.gpu.amount' should be set to Max(1, (numCores / gpuCount)). + |- 'spark.rapids.sql.concurrentGpuTasks' should be set to Max(4, (gpuMemory / 8G)). + |- 'spark.rapids.memory.pinnedPool.size' should be set to 2048m. + |- 'spark.sql.adaptive.enabled' should be enabled for better performance. + |""".stripMargin + // scalastyle:on line.size.limit + assert(expectedResults == autoTunerOutput) + } + + + test("Load cluster properties with CPU memory 0") { + val dataprocWorkerInfo = buildWorkerInfoAsString(None, Some(32), Some("0m")) val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) @@ -141,8 +164,8 @@ class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { assert(expectedResults == autoTunerOutput) } - test("Load cluster properties with missing number of workers") { - val dataprocWorkerInfo = buildWorkerInfoAsString(None, None, None, Some(0)) + test("Load cluster properties with number of workers 0") { + val dataprocWorkerInfo = buildWorkerInfoAsString(None, Some(32), Some("122880MiB"), Some(0)) val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) @@ -175,7 +198,7 @@ class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { assert(expectedResults == autoTunerOutput) } - test("Load cluster properties with missing GPU count") { + test("Load cluster properties with GPU count of 0") { // the gpuCount should default to 1 val customProps = mutable.LinkedHashMap( "spark.executor.cores" -> "16", @@ -186,7 +209,8 @@ class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { "spark.sql.files.maxPartitionBytes" -> "512m", "spark.task.resource.gpu.amount" -> "0.0625") val sparkProps = defaultDataprocProps.++(customProps) - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(sparkProps), None, None, None, Some(0)) + val dataprocWorkerInfo = buildWorkerInfoAsString(Some(sparkProps), Some(32), + Some("122880MiB"), Some(4), Some(0)) val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) @@ -206,7 +230,7 @@ class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { assert(expectedResults == autoTunerOutput) } - test("Load cluster properties with missing GPU memory") { + test("Load cluster properties with GPU memory is missing") { // the gpu memory should be set to T4 memory settings val customProps = mutable.LinkedHashMap( "spark.executor.cores" -> "16", @@ -218,7 +242,7 @@ class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { "spark.task.resource.gpu.amount" -> "0.0625") val sparkProps = defaultDataprocProps.++(customProps) val dataprocWorkerInfo = - buildWorkerInfoAsString(Some(sparkProps), None, None, None, None, Some("0m")) + buildWorkerInfoAsString(Some(sparkProps), Some(32), Some("122880MiB"), Some(4), Some(2), None) val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) @@ -234,8 +258,64 @@ class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { assert(expectedResults == autoTunerOutput) } + test("Load cluster properties with GPU memory 0") { + // the gpu memory should be set to T4 memory settings + val customProps = mutable.LinkedHashMap( + "spark.executor.cores" -> "16", + "spark.executor.memory" -> "32768m", + "spark.executor.memoryOverhead" -> "7372m", + "spark.rapids.memory.pinnedPool.size" -> "4096m", + "spark.rapids.sql.concurrentGpuTasks" -> "2", + "spark.sql.files.maxPartitionBytes" -> "512m", + "spark.task.resource.gpu.amount" -> "0.0625") + val sparkProps = defaultDataprocProps.++(customProps) + val dataprocWorkerInfo = buildWorkerInfoAsString(Some(sparkProps), Some(32), + Some("122880MiB"), Some(4), Some(2), Some("0M")) + val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) + val (properties, comments) = autoTuner.getRecommendedProperties() + val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) + val expectedResults = + s"""| + |Spark Properties: + |--conf spark.sql.shuffle.partitions=200 + | + |Comments: + |- GPU memory is missing. Setting default to 15109m. + |- 'spark.sql.shuffle.partitions' was not set. + |""".stripMargin + assert(expectedResults == autoTunerOutput) + } + + test("Load cluster properties with GPU name missing") { + val customProps = mutable.LinkedHashMap( + "spark.executor.cores" -> "16", + "spark.executor.memory" -> "32768m", + "spark.executor.memoryOverhead" -> "7372m", + "spark.rapids.memory.pinnedPool.size" -> "4096m", + "spark.rapids.sql.concurrentGpuTasks" -> "2", + "spark.sql.files.maxPartitionBytes" -> "512m", + "spark.task.resource.gpu.amount" -> "0.0625") + val sparkProps = defaultDataprocProps.++(customProps) + val dataprocWorkerInfo = buildWorkerInfoAsString(Some(sparkProps), Some(32), + Some("122880MiB"), Some(4), Some(2), Some("0MiB"), None) + val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) + val (properties, comments) = autoTuner.getRecommendedProperties() + val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) + val expectedResults = + s"""| + |Spark Properties: + |--conf spark.sql.shuffle.partitions=200 + | + |Comments: + |- GPU device is missing. Setting default to T4. + |- GPU memory is missing. Setting default to 15109m. + |- 'spark.sql.shuffle.partitions' was not set. + |""".stripMargin + assert(expectedResults == autoTunerOutput) + } + test("Load cluster properties with unknown GPU device") { - // with unknown fpu device, the memory won't be set correctly, then it should default to 16G + // with unknown gpu device, the memory won't be set correctly, then it should default to 16G val customProps = mutable.LinkedHashMap( "spark.executor.cores" -> "16", "spark.executor.memory" -> "32768m", @@ -245,8 +325,8 @@ class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { "spark.sql.files.maxPartitionBytes" -> "512m", "spark.task.resource.gpu.amount" -> "0.0625") val sparkProps = defaultDataprocProps.++(customProps) - val dataprocWorkerInfo = - buildWorkerInfoAsString(Some(sparkProps), None, None, None, None, Some("0m"), Some("GPU-X")) + val dataprocWorkerInfo = buildWorkerInfoAsString(Some(sparkProps), Some(32), + Some("122880MiB"), Some(4), Some(2), Some("0MiB"), Some("GPU-X")) val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, None) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) @@ -256,7 +336,7 @@ class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { |--conf spark.sql.shuffle.partitions=200 | |Comments: - |- GPU memory is missing. Setting default to 16384m. + |- GPU memory is missing. Setting default to 15109m. |- 'spark.sql.shuffle.partitions' was not set. |""".stripMargin assert(expectedResults == autoTunerOutput) From fb2e5768db10b32efaa86765795e304e0db9ab6e Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Thu, 6 Oct 2022 08:36:44 -0500 Subject: [PATCH 172/190] Take semaphore after first stream batch is materialized (broadcast) (#6709) * Fix semaphore acquisition before stream side IO in broadcast join Signed-off-by: Alessandro Bellina * Break up CloseableBufferedIterator into its own class * Fix typo Signed-off-by: Alessandro Bellina --- .../rapids/CloseableBufferedIterator.scala | 45 ++++++++++++++++ .../rapids/GpuBroadcastHashJoinExec.scala | 53 +++++++++++++++++-- .../rapids/GpuShuffledHashJoinExec.scala | 26 --------- .../rapids/execution/GpuBroadcastHelper.scala | 9 ++-- 4 files changed, 100 insertions(+), 33 deletions(-) create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/CloseableBufferedIterator.scala diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/CloseableBufferedIterator.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/CloseableBufferedIterator.scala new file mode 100644 index 00000000000..84fe51420ea --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/CloseableBufferedIterator.scala @@ -0,0 +1,45 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids + +import org.apache.spark.TaskContext + +/** + * Helper iterator that wraps a BufferedIterator of AutoCloseable subclasses. + * This iterator also implements AutoCloseable, so it can be closed in case + * of exceptions. + * + * @param wrapped the buffered iterator + * @tparam T an AutoCloseable subclass + */ +class CloseableBufferedIterator[T <: AutoCloseable](wrapped: BufferedIterator[T]) + extends BufferedIterator[T] with AutoCloseable { + // register against task completion to close any leaked buffered items + Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit](_ => close())) + + private[this] var isClosed = false + override def head: T = wrapped.head + override def headOption: Option[T] = wrapped.headOption + override def next: T = wrapped.next + override def hasNext: Boolean = wrapped.hasNext + override def close(): Unit = { + if (!isClosed) { + headOption.foreach(_.close()) + isClosed = true + } + } +} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec.scala index 9c217caf1bf..cc3dc10b772 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBroadcastHashJoinExec.scala @@ -16,8 +16,11 @@ package com.nvidia.spark.rapids +import ai.rapids.cudf.{NvtxColor, NvtxRange} import com.nvidia.spark.rapids.shims.ShimBinaryExecNode +import org.apache.spark.TaskContext +import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression @@ -28,6 +31,7 @@ import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, HashedRelationBroadcastMode} import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, GpuBroadcastHelper, GpuHashJoin, JoinTypeChecks} +import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch class GpuBroadcastHashJoinMeta( @@ -149,6 +153,43 @@ case class GpuBroadcastHashJoinExec( throw new IllegalStateException( "GpuBroadcastHashJoin does not support row-based processing") + /** + * Gets the ColumnarBatch for the build side and the stream iterator by + * acquiring the GPU only after first stream batch has been streamed to GPU. + * + * `broadcastRelation` represents the broadcasted build side table on the host. The code + * in this function peaks at the stream side, after having wrapped it in a closeable + * buffered iterator, to cause the stream side to produce the first batch. This delays + * acquiring the semaphore until after the stream side performs all the steps needed + * (including IO) to produce that first batch. Once the first stream batch is produced, + * the build side is materialized to the GPU (while holding the semaphore). + * + * TODO: This could try to trigger the broadcast materialization on the host before + * getting started on the stream side (e.g. call `broadcastRelation.value`). + */ + private def getBroadcastBuiltBatchAndStreamIter( + broadcastRelation: Broadcast[Any], + buildSchema: StructType, + streamIter: Iterator[ColumnarBatch], + coalesceMetricsMap: Map[String, GpuMetric]): (ColumnarBatch, Iterator[ColumnarBatch]) = { + val semWait = coalesceMetricsMap(GpuMetric.SEMAPHORE_WAIT_TIME) + + val bufferedStreamIter = new CloseableBufferedIterator(streamIter.buffered) + closeOnExcept(bufferedStreamIter) { _ => + withResource(new NvtxRange("first stream batch", NvtxColor.RED)) { _ => + if (bufferedStreamIter.hasNext) { + bufferedStreamIter.head + } else { + GpuSemaphore.acquireIfNecessary(TaskContext.get(), semWait) + } + } + + val buildBatch = + GpuBroadcastHelper.getBroadcastBatch(broadcastRelation, buildSchema) + (buildBatch, bufferedStreamIter) + } + } + override def doExecuteColumnar(): RDD[ColumnarBatch] = { val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) @@ -166,10 +207,14 @@ case class GpuBroadcastHashJoinExec( val rdd = streamedPlan.executeColumnar() val buildSchema = buildPlan.schema rdd.mapPartitions { it => - val stIt = new CollectTimeIterator("broadcast join stream", it, streamTime) - withResource( - GpuBroadcastHelper.getBroadcastBatch(broadcastRelation, buildSchema)) { builtBatch => - doJoin(builtBatch, stIt, targetSize, spillCallback, + val (builtBatch, streamIter) = + getBroadcastBuiltBatchAndStreamIter( + broadcastRelation, + buildSchema, + new CollectTimeIterator("broadcast join stream", it, streamTime), + allMetrics) + withResource(builtBatch) { _ => + doJoin(builtBatch, streamIter, targetSize, spillCallback, numOutputRows, joinOutputRows, numOutputBatches, opTime, joinTime) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala index 6cbc6f79dfd..98c4632995a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala @@ -190,32 +190,6 @@ case class GpuShuffledHashJoinExec( } object GpuShuffledHashJoinExec extends Arm { - /** - * Helper iterator that wraps a BufferedIterator of AutoCloseable subclasses. - * This iterator also implements AutoCloseable, so it can be closed in case - * of exceptions. - * - * @param wrapped the buffered iterator - * @tparam T an AutoCloseable subclass - */ - class CloseableBufferedIterator[T <: AutoCloseable](wrapped: BufferedIterator[T]) - extends BufferedIterator[T] with AutoCloseable { - // register against task completion to close any leaked buffered items - Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit](_ => close())) - - private[this] var isClosed = false - override def head: T = wrapped.head - override def headOption: Option[T] = wrapped.headOption - override def next: T = wrapped.next - override def hasNext: Boolean = wrapped.hasNext - override def close(): Unit = { - if (!isClosed) { - headOption.foreach(_.close()) - isClosed = true - } - } - } - /** * Gets a `ColumnarBatch` and stream Iterator[ColumnarBatch] pair by acquiring * the GPU semaphore optimally in the scenario where the build side is relatively diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala index 092341fbbbf..0768b5c916a 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala @@ -16,14 +16,15 @@ package org.apache.spark.sql.rapids.execution -import com.nvidia.spark.rapids.GpuColumnVector +import ai.rapids.cudf.{NvtxColor, NvtxRange} +import com.nvidia.spark.rapids.{Arm, GpuColumnVector} import com.nvidia.spark.rapids.shims.SparkShimImpl import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch -object GpuBroadcastHelper { +object GpuBroadcastHelper extends Arm { /** * Given a broadcast relation get a ColumnarBatch that can be used on the GPU. * @@ -41,7 +42,9 @@ object GpuBroadcastHelper { broadcastSchema: StructType): ColumnarBatch = { broadcastRelation.value match { case broadcastBatch: SerializeConcatHostBuffersDeserializeBatch => - broadcastBatch.batch.getColumnarBatch() + withResource(new NvtxRange("getBroadcastBatch", NvtxColor.YELLOW)) { _ => + broadcastBatch.batch.getColumnarBatch() + } case v if SparkShimImpl.isEmptyRelation(v) => GpuColumnVector.emptyBatch(broadcastSchema) case t => From 01dbcb57cbfd2f533311981e43b03e99d47524d1 Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Thu, 6 Oct 2022 09:12:08 -0500 Subject: [PATCH 173/190] Add AutoTuner documentation [skip ci] (#6701) * Add AutoTuner documentation [skip ci] Signed-off-by: Ahmed Hussein (amahussein) fixes #6320 --- docs/spark-profiling-tool.md | 132 +++++++++++++++++- .../rapids/tool/profiling/ProfileArgs.scala | 2 +- 2 files changed, 130 insertions(+), 4 deletions(-) diff --git a/docs/spark-profiling-tool.md b/docs/spark-profiling-tool.md index 659815e11a2..1238fef8e84 100644 --- a/docs/spark-profiling-tool.md +++ b/docs/spark-profiling-tool.md @@ -7,7 +7,9 @@ nav_order: 9 The Profiling tool analyzes both CPU or GPU generated event logs and generates information which can be used for debugging and profiling Apache Spark applications. -The output information contains the Spark version, executor details, properties, etc. +The output information contains the Spark version, executor details, properties, etc. +Starting with release _22.10_, the Profiling tool optionally provides optimized RAPIDS +configurations based on the worker's information (see [Auto-Tuner support](#auto-tuner-support)). * TOC {:toc} @@ -22,7 +24,7 @@ The output information contains the Spark version, executor details, properties, - The tool does not support nested directories. Event log files or event log directories should be at the top level when specifying a directory. -Note: Spark event logs can be downloaded from Spark UI using a "Download" button on the right side, +Note: Spark event logs can be downloaded from Spark UI using a _Download_ button on the right side, or can be found in the location specified by `spark.eventLog.dir`. See the [Apache Spark Monitoring](http://spark.apache.org/docs/latest/monitoring.html) documentation for more information. @@ -56,6 +58,8 @@ There are 3 modes of operation for the Profiling tool: com.nvidia.spark.rapids.tool.profiling.ProfileMain [options] ``` + Note that this is the only mode that supports the _Auto-Tuner_ option described in more details + in the [Auto-Tuner support](#auto-tuner-support) section. 2. Combined Mode: Combined mode is collection mode but then combines all the applications @@ -511,15 +515,79 @@ Failed jobs: +--------+-----+------+--------+---------------------------------------------------------------------------------------------------+ ``` +#### D. Recommended Configuration + +The _Auto-Tuner_ output has 2 main sections: +1. _Spark Properties_: A list of Apache Spark configurations to tune the performance of the app. + The list is the result of `diff` between the existing app configurations and the recommended + ones. Therefore, a recommendation matches the existing app configuration, it will not show up in + the list. +2. _Comments_: A list of messages to highlight properties that were missing in the app + configurations, or the cause of failure to generate the recommendations. + +**Examples** + +- A succesful run with missing _softwareProperties_: + ``` + Spark Properties: + --conf spark.executor.cores=16 + --conf spark.executor.instances=8 + --conf spark.executor.memory=32768m + --conf spark.executor.memoryOverhead=7372m + --conf spark.rapids.memory.pinnedPool.size=4096m + --conf spark.rapids.sql.concurrentGpuTasks=2 + --conf spark.sql.files.maxPartitionBytes=512m + --conf spark.sql.shuffle.partitions=200 + --conf spark.task.resource.gpu.amount=0.0625 + + Comments: + - 'spark.executor.instances' was not set. + - 'spark.executor.cores' was not set. + - 'spark.task.resource.gpu.amount' was not set. + - 'spark.rapids.sql.concurrentGpuTasks' was not set. + - 'spark.executor.memory' was not set. + - 'spark.rapids.memory.pinnedPool.size' was not set. + - 'spark.executor.memoryOverhead' was not set. + - 'spark.sql.files.maxPartitionBytes' was not set. + - 'spark.sql.shuffle.partitions' was not set. + - 'spark.sql.adaptive.enabled' should be enabled for better performance. + ``` + +- A succesful run with defined _softwareProperties_. In this example, only + two recommendations did not match the existing app app configurations: + ``` + Spark Properties: + --conf spark.executor.instances=8 + --conf spark.sql.shuffle.partitions=200 + + Comments: + - 'spark.sql.shuffle.partitions' was not set. + ``` + +- Failing to load the worker info: + ``` + Cannot recommend properties. See Comments. + + Comments: + - java.io.FileNotFoundException: File worker-info.yaml does not exist + - 'spark.executor.memory' should be set to at least 2GB/core. + - 'spark.executor.instances' should be set to (gpuCount * numWorkers). + - 'spark.task.resource.gpu.amount' should be set to Max(1, (numCores / gpuCount)). + - 'spark.rapids.sql.concurrentGpuTasks' should be set to Max(4, (gpuMemory / 8G)). + - 'spark.rapids.memory.pinnedPool.size' should be set to 2048m. + - 'spark.sql.adaptive.enabled' should be enabled for better performance. + ``` + ## Profiling tool options -```bash +``` Profiling tool for the RAPIDS Accelerator and Apache Spark Usage: java -cp rapids-4-spark-tools_2.12-.jar:$SPARK_HOME/jars/* com.nvidia.spark.rapids.tool.profiling.ProfileMain [options] + -a, --auto-tuner Toggle AutoTuner module. --combined Collect mode but combine all applications into the same tables. -c, --compare Compare Applications (Note this may require @@ -565,6 +633,10 @@ Usage: java -cp rapids-4-spark-tools_2.12-.jar:$SPARK_HOME/jars/* (86400 seconds) and must be greater than 3 seconds. If it times out, it will report what it was able to process up until the timeout. + -w, --worker-info File path containing the system information of + a worker node. It is assumed that all workers + are homogenous. It requires the AutoTuner to + be enabled. Default is ./worker_info.yaml -h, --help Show help message trailing arguments: @@ -573,6 +645,60 @@ Usage: java -cp rapids-4-spark-tools_2.12-.jar:$SPARK_HOME/jars/* /path/to/eventlog2 ``` +### Auto-Tuner support + +Starting with release _22.10_, the Profiling tool a new _Auto-Tuner_ that aims at optimizing +Apache Spark applications by recommending a set of configurations to tune the performance of +Rapids accelerator. +Currently, the _Auto-Tuner_ calculates a set of configurations that impact the performance of Apache +Spark apps executing on GPU. Those calculations can leverage cluster information +(e.g. memory, cores, Spark default configurations) as well as information processed in the +application event logs. +The values loaded from the app logs have higher precedence than the default configs. +Please refer to [Understanding the Profiling tool output](#d-recommended-configuration) for +more details on the output of the _Auto-Tuner_. + +Note the following _Auto-Tuner_ limitations: +- It is currently only supported in the _Collection Mode_ (see [the 3 different modes](#step-2-how-to-run-the-profiling-tool)), and +- It is assumed that all the _worker_ nodes on the cluster are homogenous. + +To run the _Auto-Tuner_, enable the `auto-tuner` flag and pass a valid `--worker-info `. +The _Auto-Tuner_ needs to learn the system properties of the _worker_ nodes that run application +code in the cluster. The argument `FILE_PATH` can either be local or remote file (i.e., HDFS). +A template of the worker information is shown below: + + ``` + system: + numCores: 32 + memory: 212992MiB + numWorkers: 5 + gpu: + memory: 15109MiB + count: 4 + name: T4 + softwareProperties: + spark.driver.maxResultSize: 7680m + spark.driver.memory: 15360m + spark.executor.cores: '8' + spark.executor.instances: '2' + spark.executor.memory: 47222m + spark.executorEnv.OPENBLAS_NUM_THREADS: '1' + spark.scheduler.mode: FAIR + spark.sql.cbo.enabled: 'true' + spark.ui.port: '0' + spark.yarn.am.memory: 640m + ``` + + | Property | Optional | If Missing | + |--------------------|:--------:|------------------------------------------------------------------------------------------------------------------------------| + | system.numCores | No | _Auto-Tuner_ does not calculate recommendations | + | system.memory | No | _Auto-Tuner_ does not calculate any recommendations | + | system.numWorkers | Yes | Default: 1 | + | gpu.name | Yes | Default: T4 (Nvidia Tesla T4) | + | gpu.memory | Yes | Default: 16G | + | softwareProperties | Yes | This section is optional. The _Auto-Tuner_ reads the configs within the logs of the Apache Spark apps with higher precedence | + + ## Profiling tool metrics definitions All the metrics definitions can be found in the diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileArgs.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileArgs.scala index 19772a7ffa5..86fb0c75fd4 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileArgs.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileArgs.scala @@ -88,7 +88,7 @@ Usage: java -cp rapids-4-spark-tools_2.12-.jar:$SPARK_HOME/jars/* "m(months). If a period is not specified it defaults to days.") val autoTuner: ScallopOption[Boolean] = opt[Boolean](required = false, - descr = "Toggle auto-tuner module.", + descr = "Toggle AutoTuner module.", default = Some(false)) val workerInfo: ScallopOption[String] = opt[String](required = false, From c56bdd28bde3e743874598cfc92ef2b0b4620f12 Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Fri, 7 Oct 2022 09:37:38 -0500 Subject: [PATCH 174/190] Fix a couple of markdown links that are now permanently moved [skip ci] (#6721) * Fix a couple of markdown links that are now permanently moved Signed-off-by: Alessandro Bellina * Add a slash at the end to prevent another redirect Signed-off-by: Alessandro Bellina --- docs/additional-functionality/rapids-shuffle.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/additional-functionality/rapids-shuffle.md b/docs/additional-functionality/rapids-shuffle.md index 7690bd08b6f..ade535a0ce0 100644 --- a/docs/additional-functionality/rapids-shuffle.md +++ b/docs/additional-functionality/rapids-shuffle.md @@ -52,8 +52,8 @@ The minimum UCX requirement for the RAPIDS Shuffle Manager is #### Baremetal -1. If you have Mellanox hardware, please ensure you have the [MLNX_OFED driver](https://www.mellanox.com/products/infiniband-drivers/linux/mlnx_ofed), and the -[`nv_peer_mem` kernel module](https://www.mellanox.com/products/GPUDirect-RDMA) installed. UCX packages +1. If you have Mellanox hardware, please ensure you have the [MLNX_OFED driver](https://network.nvidia.com/products/infiniband-drivers/linux/mlnx_ofed/), and the +[`nv_peer_mem` kernel module](https://network.nvidia.com/products/GPUDirect-RDMA/) installed. UCX packages are compatible with MLNX_OFED 5.0+. Please install the latest driver available. With `nv_peer_mem` (GPUDirectRDMA), IB/RoCE-based transfers can perform zero-copy transfers From 702c023cbb2dbe31b53dac590d7d63f09fd1473f Mon Sep 17 00:00:00 2001 From: Sameer Raheja Date: Fri, 7 Oct 2022 11:05:10 -0700 Subject: [PATCH 175/190] Document that we test on JDK8 and JDK11, other versions are untested (#6719) Signed-off-by: Sameer Raheja Signed-off-by: Sameer Raheja --- CONTRIBUTING.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index dd7c23fae09..752b627a369 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -102,8 +102,9 @@ You can build against different versions of the CUDA Toolkit by using qone of th * `-Pcuda11` (CUDA 11.0/11.1/11.2, default) ### Building and Testing with JDK9+ -We support JDK8 as our main JDK version. However, it's possible to build and run with more modern -JDK versions as well. To this end set `JAVA_HOME` in the environment to your JDK root directory. +We support JDK8 as our main JDK version, and test JDK8 and JDK11. It is possible to build and run +with more modern JDK versions, however these are untested. The first step is to set `JAVA_HOME` in +the environment to your JDK root directory. At the time of this writing, the most robust way to run the RAPIDS Accelerator is from a jar dedicated to a single Spark version. To this end please use a single shim and specify `-DallowConventionalDistJar=true` From 2814ffe63113e8c06310fd53158f45cc1ef17cd9 Mon Sep 17 00:00:00 2001 From: Matt Ahrens Date: Fri, 7 Oct 2022 16:44:18 -0500 Subject: [PATCH 176/190] [Doc] Adding Dataproc quick start steps to use new user tools package [skip ci] (#6695) * Adding Dataproc quick start steps to use new user tools package Signed-off-by: Matt Ahrens * Adding Dataproc quick start steps to use new user tools package Signed-off-by: Matt Ahrens * Updating based on review comments Signed-off-by: Matt Ahrens * Switching bash formatting to normal triple ticks Signed-off-by: Matt Ahrens * Adding configs link in bootstrap section Signed-off-by: Matt Ahrens * Adding spark configs link in bootstrap section Signed-off-by: Matt Ahrens Signed-off-by: Matt Ahrens --- docs/get-started/getting-started-gcp.md | 113 +++++++++++++++++++++++- 1 file changed, 112 insertions(+), 1 deletion(-) diff --git a/docs/get-started/getting-started-gcp.md b/docs/get-started/getting-started-gcp.md index eadb8283587..c9024092536 100644 --- a/docs/get-started/getting-started-gcp.md +++ b/docs/get-started/getting-started-gcp.md @@ -7,7 +7,14 @@ parent: Getting-Started # Getting started with RAPIDS Accelerator on GCP Dataproc [Google Cloud Dataproc](https://cloud.google.com/dataproc) is Google Cloud's fully managed Apache - Spark and Hadoop service. This guide will walk through the steps to: + Spark and Hadoop service. The quick start guide will go through: + +* [Quick Start Prerequisites](#quick-start-prerequisites) +* [Qualify CPU workloads for GPU acceleration](#qualify-cpu-workloads-for-gpu-acceleration) +* [Bootstrap GPU cluster with optimized settings](#bootstrap-gpu-cluster-with-optimized-settings) +* [Tune applications on GPU cluster](#tune-applications-on-gpu-cluster) + +The advanced guide will walk through the steps to: * [Create a Dataproc Cluster Accelerated by GPUs](#create-a-dataproc-cluster-accelerated-by-gpus) * [Run Pyspark or Scala ETL and XGBoost training Notebook on a Dataproc Cluster Accelerated by @@ -16,6 +23,110 @@ parent: Getting-Started GPUs](#submit-spark-jobs-to-a-dataproc-cluster-accelerated-by-gpus) * [Build custom Dataproc image to accelerate cluster initialization time](#build-custom-dataproc-image-to-accelerate-cluster-init-time) +## Quick Start Prerequisites + +* gcloud CLI is installed: https://cloud.google.com/sdk/docs/install +* python 3.8+ +* `pip install spark-rapids-user-tools` + +## Qualify CPU Workloads for GPU Acceleration + +The [qualification tool](https://nvidia.github.io/spark-rapids/docs/spark-qualification-tool.html) is launched on a Dataproc cluster that has applications that have already run. +The tool will output the applications recommended for acceleration along with estimated speed-up +and cost saving metrics. Additionally, it will provide information on how to launch a GPU- +accelerated cluster to take advantage of the speed-up and cost savings. + +Usage: `spark_rapids_dataproc qualification --cluster --region ` + +Help (to see all options available): `spark_rapids_dataproc qualification --help` + +Example output: +``` ++----+------------+--------------------------------+----------------------+-----------------+-----------------+---------------+-----------------+ +| | App Name | App ID | Recommendation | Estimated GPU | Estimated GPU | App | Estimated GPU | +| | | | | Speedup | Duration(s) | Duration(s) | Savings(%) | +|----+------------+--------------------------------+----------------------+-----------------+-----------------+---------------+-----------------| +| 0 | query24 | application_1664888311321_0011 | Strongly Recommended | 3.49 | 257.18 | 897.68 | 59.70 | +| 1 | query78 | application_1664888311321_0009 | Strongly Recommended | 3.35 | 113.89 | 382.35 | 58.10 | +| 2 | query23 | application_1664888311321_0010 | Strongly Recommended | 3.08 | 325.77 | 1004.28 | 54.37 | +| 3 | query64 | application_1664888311321_0008 | Strongly Recommended | 2.91 | 150.81 | 440.30 | 51.82 | +| 4 | query50 | application_1664888311321_0003 | Recommended | 2.47 | 101.54 | 250.95 | 43.08 | +| 5 | query16 | application_1664888311321_0005 | Recommended | 2.36 | 106.33 | 251.95 | 40.63 | +| 6 | query38 | application_1664888311321_0004 | Recommended | 2.29 | 67.37 | 154.33 | 38.59 | +| 7 | query87 | application_1664888311321_0006 | Recommended | 2.25 | 75.67 | 170.69 | 37.64 | +| 8 | query51 | application_1664888311321_0002 | Recommended | 1.53 | 53.94 | 82.63 | 8.18 | ++----+------------+--------------------------------+----------------------+-----------------+-----------------+---------------+-----------------+ +To launch a GPU-accelerated cluster with Spark RAPIDS, add the following to your cluster creation script: + --initialization-actions=gs://goog-dataproc-initialization-actions-us-central1/gpu/install_gpu_driver.sh,gs://goog-dataproc-initialization-actions-us-central1/rapids/rapids.sh \ + --worker-accelerator type=nvidia-tesla-t4,count=2 \ + --metadata gpu-driver-provider="NVIDIA" \ + --metadata rapids-runtime=SPARK \ + --cuda-version=11.5 +``` + +## Bootstrap GPU Cluster with Optimized Settings + +The bootstrap tool will apply optimized settings for the RAPIDS Accelerator on Apache Spark on a +GPU cluster for Dataproc. The tool will fetch the characteristics of the cluster -- including +number of workers, worker cores, worker memory, and GPU accelerator type and count. It will use +the cluster properties to then determine the optimal settings for running GPU-accelerated Spark +applications. + +Usage: `spark_rapids_dataproc bootstrap --cluster --region ` + +Help (to see all options available): `spark_rapids_dataproc bootstrap --help` + +Example output: +``` +##### BEGIN : RAPIDS bootstrap settings for gpu-cluster +spark.executor.cores=16 +spark.executor.memory=32768m +spark.executor.memoryOverhead=7372m +spark.rapids.sql.concurrentGpuTasks=2 +spark.rapids.memory.pinnedPool.size=4096m +spark.sql.files.maxPartitionBytes=512m +spark.task.resource.gpu.amount=0.0625 +##### END : RAPIDS bootstrap settings for gpu-cluster +``` + +A detailed description for bootstrap settings with usage information is available in the [RAPIDS Accelerator for Apache Spark Configuration](https://nvidia.github.io/spark-rapids/docs/configs.html) and [Spark Configuration](https://spark.apache.org/docs/latest/configuration.html) page. + +## Tune Applications on GPU Cluster + +Once Spark applications have been run on the GPU cluster, the [profiling tool](https://nvidia.github.io/spark-rapids/docs/spark-profiling-tool.html) can be run to +analyze the event logs of the applications to determine if more optimal settings should be +configured. The tool will output a per-application set of config settings to be adjusted for +enhanced performance. + +Usage: `spark_rapids_dataproc profiling --cluster --region ` + +Help (to see all options available): `spark_rapids_dataproc profiling --help` + +Example output: +``` ++--------------------------------+--------------------------------------------------+--------------------------------------------------------------------------------------------------+ +| App ID | Recommendations | Comments | ++================================+==================================================+==================================================================================================+ +| application_1664894105643_0011 | --conf spark.executor.cores=16 | - 'spark.task.resource.gpu.amount' was not set. | +| | --conf spark.executor.memory=32768m | - 'spark.rapids.sql.concurrentGpuTasks' was not set. | +| | --conf spark.executor.memoryOverhead=7372m | - 'spark.rapids.memory.pinnedPool.size' was not set. | +| | --conf spark.rapids.memory.pinnedPool.size=4096m | - 'spark.executor.memoryOverhead' was not set. | +| | --conf spark.rapids.sql.concurrentGpuTasks=2 | - 'spark.sql.files.maxPartitionBytes' was not set. | +| | --conf spark.sql.files.maxPartitionBytes=1571m | - 'spark.sql.shuffle.partitions' was not set. | +| | --conf spark.sql.shuffle.partitions=200 | | +| | --conf spark.task.resource.gpu.amount=0.0625 | | ++--------------------------------+--------------------------------------------------+--------------------------------------------------------------------------------------------------+ +| application_1664894105643_0002 | --conf spark.executor.cores=16 | - 'spark.task.resource.gpu.amount' was not set. | +| | --conf spark.executor.memory=32768m | - 'spark.rapids.sql.concurrentGpuTasks' was not set. | +| | --conf spark.executor.memoryOverhead=7372m | - 'spark.rapids.memory.pinnedPool.size' was not set. | +| | --conf spark.rapids.memory.pinnedPool.size=4096m | - 'spark.executor.memoryOverhead' was not set. | +| | --conf spark.rapids.sql.concurrentGpuTasks=2 | - 'spark.sql.files.maxPartitionBytes' was not set. | +| | --conf spark.sql.files.maxPartitionBytes=3844m | - 'spark.sql.shuffle.partitions' was not set. | +| | --conf spark.sql.shuffle.partitions=200 | | +| | --conf spark.task.resource.gpu.amount=0.0625 | | ++--------------------------------+--------------------------------------------------+--------------------------------------------------------------------------------------------------+ +``` + ## Create a Dataproc Cluster Accelerated by GPUs You can use [Cloud Shell](https://cloud.google.com/shell) to execute shell commands that will From c75c1f036f7592a53f8057659d485fa67da238cd Mon Sep 17 00:00:00 2001 From: Niranjan Artal <50492963+nartal1@users.noreply.github.com> Date: Fri, 7 Oct 2022 16:11:07 -0700 Subject: [PATCH 177/190] Qualification tool: Read SQL function names for parsing expressions (#6724) * Read SQL function names for parsing expressions Signed-off-by: Niranjan Artal * fix indentation Signed-off-by: Niranjan Artal --- .../qualification/PluginTypeChecker.scala | 34 ++++++++++++++++--- .../tool/planparser/SqlPlanParserSuite.scala | 23 +++++++++++++ 2 files changed, 53 insertions(+), 4 deletions(-) diff --git a/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala b/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala index 4f14af8fca3..35799b25cfd 100644 --- a/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala +++ b/tools/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala @@ -84,7 +84,7 @@ class PluginTypeChecker extends Logging { private def readOperatorsScore: Map[String, Double] = { val source = Source.fromResource(OPERATORS_SCORE_FILE) - readSupportedOperators(source).map(x => (x._1, x._2.toDouble)) + readSupportedOperators(source, "score").map(x => (x._1, x._2.toDouble)) } private def readSupportedExecs: Map[String, String] = { @@ -94,7 +94,10 @@ class PluginTypeChecker extends Logging { private def readSupportedExprs: Map[String, String] = { val source = Source.fromResource(SUPPORTED_EXPRS_FILE) - readSupportedOperators(source).map(x => (x._1.toLowerCase, x._2)) + // Some SQL function names have backquotes(`) around their names, + // so we remove them before saving. + readSupportedOperators(source, "exprs").map( + x => (x._1.toLowerCase.replaceAll("\\`", ""), x._2)) } private def readSupportedTypesForPlugin: ( @@ -103,7 +106,10 @@ class PluginTypeChecker extends Logging { readSupportedTypesForPlugin(source) } - private def readSupportedOperators(source: BufferedSource): Map[String, String] = { + // operatorType can be exprs, score or execs(default). Reads the columns in file depending + // on the operatorType passed to this function. + private def readSupportedOperators(source: BufferedSource, + operatorType: String = "execs"): Map[String, String] = { val supportedOperators = HashMap.empty[String, String] try { val fileContents = source.getLines().toSeq @@ -121,7 +127,27 @@ class PluginTypeChecker extends Logging { s" header length doesn't match rows length. Row that doesn't match is " + s"${cols.mkString(",")}") } - supportedOperators.put(cols(0), cols(1)) + // There are addidtional checks for Expressions. In physical plan, SQL function name is + // printed instead of expression name. We have to save both expression name and + // SQL function name(if there is one) so that we don't miss the expression while + // parsing the execs. + // Ex: Expression name = Substring, SQL function= `substr`; `substring` + // Ex: Expression name = Average, SQL function name = `avg` + if (operatorType.equals("exprs")) { + // save expression name + supportedOperators.put(cols(0), cols(1)) + // Check if there is SQL function name for the above expression + if (cols(2).nonEmpty && cols(2) != None) { + // Split on `;` if there are multiple SQL names as shown in above example and + // save each SQL function name as a separate key. + val sqlFuncNames = cols(2).split(";") + for (i <- sqlFuncNames) { + supportedOperators.put(i, cols(1)) + } + } + } else { + supportedOperators.put(cols(0), cols(1)) + } } } finally { source.close() diff --git a/tools/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala b/tools/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala index 6a79ff10584..031e61e8d27 100644 --- a/tools/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala +++ b/tools/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala @@ -669,4 +669,27 @@ class SQLPlanParserSuite extends FunSuite with BeforeAndAfterEach with Logging { } } } + + test("Parse SQL function Name in HashAggregateExec") { + TrampolineUtil.withTempDir { eventLogDir => + val (eventLog, _) = ToolTestUtils.generateEventLog(eventLogDir, "sqlmetric") { spark => + import spark.implicits._ + val df1 = Seq((1, "a"), (1, "aa"), (1, "a"), (2, "b"), + (2, "b"), (3, "c"), (3, "c")).toDF("num", "letter") + // Average is Expression name and `avg` is SQL function name. + // SQL function name is in the eventlog as shown below. + // HashAggregate(keys=[letter#187], functions=[avg(cast(num#186 as bigint))]) + df1.groupBy("letter").avg("num") + } + val pluginTypeChecker = new PluginTypeChecker() + val app = createAppFromEventlog(eventLog) + assert(app.sqlPlans.size == 1) + val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => + SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) + } + val execInfo = getAllExecsFromPlan(parsedPlans.toSeq) + val hashAggregate = execInfo.filter(_.exec == "HashAggregate") + assertSizeAndSupported(2, hashAggregate, 4.5) + } + } } From 7e4324c6f117e9555a8bb427205562f296591003 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 7 Oct 2022 21:17:53 -0500 Subject: [PATCH 178/190] Fix NullPointerException in iceberg schema parsing code when selecting single column (#6729) * Fix NullPointerException in iceberg scheme code when selecting single column Signed-off-by: Thomas Graves * fix typo * fix spelling * hardcode column to c0 Signed-off-by: Thomas Graves --- .../src/main/python/iceberg_test.py | 18 ++++++++++++++++++ .../iceberg/parquet/GpuParquetReader.java | 16 ++++++++++------ 2 files changed, 28 insertions(+), 6 deletions(-) diff --git a/integration_tests/src/main/python/iceberg_test.py b/integration_tests/src/main/python/iceberg_test.py index 4b4ccfacc10..88aed07bb26 100644 --- a/integration_tests/src/main/python/iceberg_test.py +++ b/integration_tests/src/main/python/iceberg_test.py @@ -72,6 +72,24 @@ def setup_iceberg_table(spark): "spark.rapids.sql.format.parquet.reader.type": reader_type, "spark.sql.optimizer.dynamicPartitionPruning.enabled": "true"}) +@iceberg +@ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering +@pytest.mark.parametrize("data_gens", iceberg_gens_list, ids=idfn) +@pytest.mark.parametrize('reader_type', rapids_reader_types) +def test_iceberg_parquet_read_round_trip_select_one(spark_tmp_table_factory, data_gens, reader_type): + gen_list = [('_c' + str(i), gen) for i, gen in enumerate(data_gens)] + table = spark_tmp_table_factory.get() + tmpview = spark_tmp_table_factory.get() + def setup_iceberg_table(spark): + df = gen_df(spark, gen_list) + df.createOrReplaceTempView(tmpview) + spark.sql("CREATE TABLE {} USING ICEBERG AS SELECT * FROM {}".format(table, tmpview)) + with_cpu_session(setup_iceberg_table) + # explicitly only select 1 column to make sure we test that path in the schema parsing code + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT _c0 FROM {}".format(table)), + conf={'spark.rapids.sql.format.parquet.reader.type': reader_type}) + @iceberg @ignore_order(local=True) # Iceberg plans with a thread pool and is not deterministic in file ordering @pytest.mark.parametrize("data_gens", iceberg_gens_list, ids=idfn) diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/parquet/GpuParquetReader.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/parquet/GpuParquetReader.java index f35a669241c..3398d6b3be1 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/parquet/GpuParquetReader.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/parquet/GpuParquetReader.java @@ -311,10 +311,12 @@ public Type struct(Types.StructType expected, GroupType struct, List field @Override public Type list(Types.ListType expectedList, GroupType list, Type element) { - boolean hasConstant = expectedList.fields().stream() + if (expectedList != null) { + boolean hasConstant = expectedList.fields().stream() .anyMatch(f -> idToConstant.containsKey(f.fieldId())); - if (hasConstant) { - throw new UnsupportedOperationException("constant column in list"); + if (hasConstant) { + throw new UnsupportedOperationException("constant column in list"); + } } GroupType repeated = list.getType(0).asGroupType(); Type originalElement = repeated.getType(0); @@ -326,10 +328,12 @@ public Type list(Types.ListType expectedList, GroupType list, Type element) { @Override public Type map(Types.MapType expectedMap, GroupType map, Type key, Type value) { - boolean hasConstant = expectedMap.fields().stream() + if (expectedMap != null) { + boolean hasConstant = expectedMap.fields().stream() .anyMatch(f -> idToConstant.containsKey(f.fieldId())); - if (hasConstant) { - throw new UnsupportedOperationException("constant column in map"); + if (hasConstant) { + throw new UnsupportedOperationException("constant column in map"); + } } GroupType repeated = map.getFields().get(0).asGroupType(); Type originalKey = repeated.getType(0); From 34d4e792c28540b856efe5c8d7b96ac8f5a9d595 Mon Sep 17 00:00:00 2001 From: Mike Wilson Date: Mon, 10 Oct 2022 22:36:22 -0400 Subject: [PATCH 179/190] Enabling decimal 38,2 casting (#6750) --- .../main/scala/com/nvidia/spark/rapids/GpuCastMeta.scala | 2 -- .../src/main/scala/com/nvidia/spark/rapids/GpuCast.scala | 2 -- .../test/scala/com/nvidia/spark/rapids/CastOpSuite.scala | 9 +++------ 3 files changed, 3 insertions(+), 10 deletions(-) diff --git a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCastMeta.scala b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCastMeta.scala index 7e4f169355c..0728017a177 100644 --- a/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCastMeta.scala +++ b/spark2-sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCastMeta.scala @@ -85,8 +85,6 @@ final class CastExprMeta[INPUT <: Cast]( "converting floating point data types to strings and this can produce results that " + "differ from the default behavior in Spark. To enable this operation on the GPU, set" + s" ${RapidsConf.ENABLE_CAST_FLOAT_TO_STRING} to true.") - case (_: StringType, dt: DecimalType) if dt.precision + 1 > DecimalType.MAX_PRECISION => - willNotWorkOnGpu(s"Because of rounding requirements we cannot support $dt on the GPU") case (_: StringType, _: FloatType | _: DoubleType) if !conf.isCastStringToFloatEnabled => willNotWorkOnGpu("Currently hex values aren't supported on the GPU. Also note " + "that casting from string to float types on the GPU returns incorrect results when " + diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala index a04c26f794e..e877ba269a6 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala @@ -100,8 +100,6 @@ final class CastExprMeta[INPUT <: CastBase]( "converting floating point data types to strings and this can produce results that " + "differ from the default behavior in Spark. To enable this operation on the GPU, set" + s" ${RapidsConf.ENABLE_CAST_FLOAT_TO_STRING} to true.") - case (_: StringType, dt: DecimalType) if dt.precision + 1 > DecimalType.MAX_PRECISION => - willNotWorkOnGpu(s"Because of rounding requirements we cannot support $dt on the GPU") case (_: StringType, _: FloatType | _: DoubleType) if !conf.isCastStringToFloatEnabled => willNotWorkOnGpu("Currently hex values aren't supported on the GPU. Also note " + "that casting from string to float types on the GPU returns incorrect results when " + diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala index 8ec81a4152b..87c44b62e34 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/CastOpSuite.scala @@ -906,12 +906,9 @@ class CastOpSuite extends GpuExpressionTestSuite { } } - test("cast string to decimal (fail)") { - assertThrows[IllegalArgumentException]( - List(-38, 38, 2, 32, 8).foreach { scale => - testCastToDecimal(DataTypes.StringType, scale, - customRandGenerator = Some(new scala.util.Random(1234L))) - }) + test("cast 38,2 string to decimal") { + testCastToDecimal(DataTypes.StringType, scale = 2, precision = 38, + customRandGenerator = Some(new scala.util.Random(1234L))) } test("cast string to decimal (include NaN/INF/-INF)") { From b81ca3e3135191dfb4457a28ed2fa788c64de5ae Mon Sep 17 00:00:00 2001 From: Navin Kumar <97137715+NVnavkumar@users.noreply.github.com> Date: Tue, 11 Oct 2022 12:23:01 -0700 Subject: [PATCH 180/190] Docker container for ease of deployment to Databricks [skip ci] (#6703) * Databricks Docker container source files Signed-off-by: Navin Kumar * Updated to latest version Signed-off-by: Navin Kumar * Add Ganglia to NOTICE Signed-off-by: Navin Kumar * Updated NOTICE-binary with correct copyrights Signed-off-by: Navin Kumar * Update stage names in Dockerfile Signed-off-by: Navin Kumar * remove these copyrights from NOTICE-binary since they are only for the Docker image and not the JAR Signed-off-by: Navin Kumar * Fix README.md missing build instructions Signed-off-by: Navin Kumar * Updated instructions with step re-order and clarification about selection of GPU nodes for workers Signed-off-by: Navin Kumar Signed-off-by: Navin Kumar --- .../00-custom-spark-driver-defaults.conf | 21 ++ docker/databricks/Dockerfile | 227 ++++++++++++ docker/databricks/README.md | 129 +++++++ docker/databricks/build.sh | 53 +++ docker/databricks/ganglia/ganglia.conf | 37 ++ .../ganglia/gconf/conf.d/modpython.conf | 30 ++ .../ganglia/gconf/databricks-gmond.conf | 33 ++ docker/databricks/ganglia/gconf/gmetad.conf | 22 ++ docker/databricks/ganglia/gconf/gmond.conf | 342 ++++++++++++++++++ .../databricks/ganglia/start_spark_slave.sh | 189 ++++++++++ docker/databricks/init.sh | 234 ++++++++++++ docker/databricks/requirements_10.4ML.txt | 241 ++++++++++++ docker/databricks/scripts/monit | 118 ++++++ .../webterminal/setup_ttyd_daemon.sh | 37 ++ .../webterminal/start_ttyd_daemon.sh | 26 ++ .../webterminal/stop_ttyd_daemon.sh | 26 ++ .../databricks/webterminal/webTerminalBashrc | 41 +++ 17 files changed, 1806 insertions(+) create mode 100644 docker/databricks/00-custom-spark-driver-defaults.conf create mode 100644 docker/databricks/Dockerfile create mode 100644 docker/databricks/README.md create mode 100755 docker/databricks/build.sh create mode 100644 docker/databricks/ganglia/ganglia.conf create mode 100644 docker/databricks/ganglia/gconf/conf.d/modpython.conf create mode 100644 docker/databricks/ganglia/gconf/databricks-gmond.conf create mode 100755 docker/databricks/ganglia/gconf/gmetad.conf create mode 100755 docker/databricks/ganglia/gconf/gmond.conf create mode 100644 docker/databricks/ganglia/start_spark_slave.sh create mode 100644 docker/databricks/init.sh create mode 100644 docker/databricks/requirements_10.4ML.txt create mode 100644 docker/databricks/scripts/monit create mode 100644 docker/databricks/webterminal/setup_ttyd_daemon.sh create mode 100644 docker/databricks/webterminal/start_ttyd_daemon.sh create mode 100644 docker/databricks/webterminal/stop_ttyd_daemon.sh create mode 100644 docker/databricks/webterminal/webTerminalBashrc diff --git a/docker/databricks/00-custom-spark-driver-defaults.conf b/docker/databricks/00-custom-spark-driver-defaults.conf new file mode 100644 index 00000000000..a3831565ad1 --- /dev/null +++ b/docker/databricks/00-custom-spark-driver-defaults.conf @@ -0,0 +1,21 @@ +# Copyright (c) 2022, NVIDIA CORPORATION. +# +# 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. +[driver] { + "spark.plugins" = "com.nvidia.spark.SQLPlugin" + "spark.rapids.memory.pinnedPool.size" = "2G" + "spark.databricks.delta.optimizeWrite.enabled" = "false" + "spark.sql.optimizer.dynamicPartitionPruning.enabled" = "false" + "spark.sql.files.maxPartitionBytes" = "512m" + "spark.rapids.sql.concurrentGpuTasks" = "2" +} \ No newline at end of file diff --git a/docker/databricks/Dockerfile b/docker/databricks/Dockerfile new file mode 100644 index 00000000000..b913ebd6eba --- /dev/null +++ b/docker/databricks/Dockerfile @@ -0,0 +1,227 @@ +# Copyright (c) 2022, NVIDIA CORPORATION. +# +# 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. + +############# +# Combine all below Dockerfiles together: +# https://github.com/databricks/containers/blob/master/ubuntu/gpu/cuda-11/base/Dockerfile +# https://github.com/databricks/containers/blob/master/ubuntu/minimal/Dockerfile +# https://github.com/databricks/containers/blob/master/ubuntu/python/Dockerfile +# https://github.com/databricks/containers/blob/master/ubuntu/dbfsfuse/Dockerfile +# https://github.com/databricks/containers/blob/master/ubuntu/standard/Dockerfile +# https://github.com/databricks/containers/blob/master/experimental/ubuntu/ganglia/Dockerfile +# https://github.com/dayananddevarapalli/containers/blob/main/webterminal/Dockerfile +############# +ARG CUDA_VERSION=11.5.2 +FROM nvidia/cuda:${CUDA_VERSION}-cudnn8-runtime-ubuntu20.04 as base + +ARG CUDA_PKG_VERSION=11-5 + +############# +# Install all needed libs +############# +ARG REQUIREMENTS=requirements_10.4ML.txt +COPY ${REQUIREMENTS} /tmp/requirements.txt + +RUN set -ex && \ + cd /etc/apt/sources.list.d && \ + mv cuda.list cuda.list.disabled && \ + apt-get -y update && \ + apt-get -y install wget && \ + wget -qO - https://developer.download.nvidia.com/compute/cuda/repos/ubuntu1604/x86_64/3bf863cc.pub | apt-key add - && \ + cd /etc/apt/sources.list.d && \ + mv cuda.list.disabled cuda.list && \ + apt-get -y update && \ + apt-get -y upgrade && \ + apt-get install -y software-properties-common && \ + add-apt-repository ppa:deadsnakes/ppa -y && \ + apt-get -y install python3.8 virtualenv python3-filelock libcairo2 cuda-cupti-${CUDA_PKG_VERSION} \ + cuda-toolkit-${CUDA_PKG_VERSION}-config-common cuda-toolkit-11-config-common cuda-toolkit-config-common \ + openjdk-8-jdk-headless iproute2 bash sudo coreutils procps gpg fuse openssh-server && \ + apt-get -y install cuda-cudart-dev-${CUDA_PKG_VERSION} cuda-cupti-dev-${CUDA_PKG_VERSION} cuda-driver-dev-${CUDA_PKG_VERSION} \ + cuda-nvcc-${CUDA_PKG_VERSION} cuda-thrust-${CUDA_PKG_VERSION} cuda-toolkit-${CUDA_PKG_VERSION}-config-common cuda-toolkit-11-config-common \ + cuda-toolkit-config-common python3.8-dev libpq-dev libcairo2-dev build-essential unattended-upgrades cmake ccache \ + openmpi-bin linux-headers-5.4.0-117 linux-headers-5.4.0-117-generic linux-headers-generic libopenmpi-dev unixodbc-dev \ + sysstat ssh tmux && \ + apt-get install -y less vim && \ + /var/lib/dpkg/info/ca-certificates-java.postinst configure && \ + # Initialize the default environment that Spark and notebooks will use + virtualenv -p python3.8 --system-site-packages /databricks/python3 \ + && /databricks/python3/bin/pip install --no-cache-dir --upgrade pip \ + && /databricks/python3/bin/pip install --no-cache-dir -r /tmp/requirements.txt \ + # Install Python libraries for Databricks environment + && /databricks/python3/bin/pip cache purge && \ + apt-get -y purge --autoremove software-properties-common cuda-cudart-dev-${CUDA_PKG_VERSION} cuda-cupti-dev-${CUDA_PKG_VERSION} \ + cuda-driver-dev-${CUDA_PKG_VERSION} cuda-nvcc-${CUDA_PKG_VERSION} cuda-thrust-${CUDA_PKG_VERSION} \ + python3.8-dev libpq-dev libcairo2-dev build-essential unattended-upgrades cmake ccache openmpi-bin \ + linux-headers-5.4.0-117 linux-headers-5.4.0-117-generic linux-headers-generic libopenmpi-dev unixodbc-dev \ + virtualenv python3-virtualenv && \ + apt-get clean && \ + rm -rf /var/lib/apt/lists/* /tmp/* /var/tmp/* && \ + mkdir -p /databricks/jars && \ + mkdir -p /mnt/driver-daemon && \ + ############# + # Disable NVIDIA repos to prevent accidental upgrades. + ############# + ln -s /databricks/jars /mnt/driver-daemon/jars && \ + cd /etc/apt/sources.list.d && \ + mv cuda.list cuda.list.disabled && \ + # Create user "ubuntu" + useradd --create-home --shell /bin/bash --groups sudo ubuntu + +############# +# Set all env variables +############# +ARG DATABRICKS_RUNTIME_VERSION=10.4 +ENV PYSPARK_PYTHON=/databricks/python3/bin/python3 +ENV DATABRICKS_RUNTIME_VERSION=${DATABRICKS_RUNTIME_VERSION} +ENV LANG=C.UTF-8 +ENV USER=ubuntu +ENV PATH=/usr/local/nvidia/bin:/databricks/python3/bin:/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin:/snap/bin + +FROM base as with-plugin + +############# +# Spark RAPIDS configuration +############# +ARG DRIVER_CONF_FILE=00-custom-spark-driver-defaults.conf +ARG JAR_FILE=rapids-4-spark_2.12-22.10.0.jar +ARG JAR_URL=https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/22.10.0/${JAR_FILE} +ARG INIT_SCRIPT=init.sh +COPY ${DRIVER_CONF_FILE} /databricks/driver/conf/00-custom-spark-driver-defaults.conf + +WORKDIR /databricks/jars +ADD $JAR_URL /databricks/jars/${JAR_FILE} + +ADD $INIT_SCRIPT /opt/spark-rapids/init.sh +RUN chmod 755 /opt/spark-rapids/init.sh + +WORKDIR /databricks + +############# +# Setup Ganglia +############# +FROM with-plugin as with-ganglia + +WORKDIR /databricks +ENV DEBIAN_FRONTEND=noninteractive +RUN apt-get update && apt-get install -q -y --force-yes --fix-missing --ignore-missing \ + ganglia-monitor \ + ganglia-webfrontend \ + ganglia-monitor-python \ + python3-pip \ + wget \ + rsync \ + cron \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* /tmp/* /var/tmp/* +# Upgrade Ganglia to 3.7.2 to patch XSS bug, see CJ-15250 +# Upgrade Ganglia to 3.7.4 and use private forked repo to patch several security bugs, see CJ-20114 +# SC-17279: We run gmetad as user ganglia, so change the owner from nobody to ganglia for the rrd directory +RUN cd /tmp \ + && export GANGLIA_WEB=ganglia-web-3.7.4-db-4 \ + && wget https://s3-us-west-2.amazonaws.com/databricks-build-files/$GANGLIA_WEB.tar.gz \ + && tar xvzf $GANGLIA_WEB.tar.gz \ + && cd $GANGLIA_WEB \ + && make install \ + && chown ganglia:ganglia /var/lib/ganglia/rrds +# Install Phantom.JS +RUN cd /tmp \ + && export PHANTOM_JS="phantomjs-2.1.1-linux-x86_64" \ + && wget https://s3-us-west-2.amazonaws.com/databricks-build-files/$PHANTOM_JS.tar.bz2 \ + && tar xvjf $PHANTOM_JS.tar.bz2 \ + && mv $PHANTOM_JS /usr/local/share \ + && ln -sf /usr/local/share/$PHANTOM_JS/bin/phantomjs /usr/local/bin +# Apache2 config. The `sites-enabled` config files are loaded into the container +# later. +RUN rm /etc/apache2/sites-enabled/* && a2enmod proxy && a2enmod proxy_http +RUN mkdir -p /etc/monit/conf.d +RUN echo '\ +check process ganglia-monitor with pidfile /var/run/ganglia-monitor.pid\n\ + start program = "/usr/sbin/service ganglia-monitor start"\n\ + stop program = "/usr/sbin/service ganglia-monitor stop"\n\ + if memory usage > 500 MB for 3 cycles then restart\n\ +' > /etc/monit/conf.d/ganglia-monitor-not-active +RUN echo '\ +check process gmetad with pidfile /var/run/gmetad.pid\n\ + start program = "/usr/sbin/service gmetad start"\n\ + stop program = "/usr/sbin/service gmetad stop"\n\ + if memory usage > 500 MB for 3 cycles then restart\n\ +\n\ +check process apache2 with pidfile /var/run/apache2/apache2.pid\n\ + start program = "/usr/sbin/service apache2 start"\n\ + stop program = "/usr/sbin/service apache2 stop"\n\ + if memory usage > 500 MB for 3 cycles then restart\n\ +' > /etc/monit/conf.d/gmetad-not-active +RUN echo '\ +check process spark-slave with pidfile /tmp/spark-root-org.apache.spark.deploy.worker.Worker-1.pid\n\ + start program = "/databricks/spark/scripts/restart-workers"\n\ + stop program = "/databricks/spark/scripts/kill_worker.sh"\n\ +' > /etc/monit/conf.d/spark-slave-not-active +# add Ganglia configuration file indicating the DocumentRoot - Databricks checks this to enable Ganglia upon cluster startup +RUN mkdir -p /etc/apache2/sites-enabled +ADD ganglia/ganglia.conf /etc/apache2/sites-enabled +RUN chmod 775 /etc/apache2/sites-enabled/ganglia.conf +ADD ganglia/gconf/* /etc/ganglia/ +RUN mkdir -p /databricks/spark/scripts/ganglia/ +RUN mkdir -p /databricks/spark/scripts/ +ADD ganglia/start_spark_slave.sh /databricks/spark/scripts/start_spark_slave.sh + +# add local monit shell script in the right location +RUN mkdir -p /etc/init.d +ADD scripts/monit /etc/init.d +RUN chmod 775 /etc/init.d/monit + +############# +# Set up webterminal ssh +############# +FROM with-ganglia as with-webterminal + +RUN wget https://github.com/tsl0922/ttyd/releases/download/1.6.3/ttyd.x86_64 && \ + mkdir -p /databricks/driver/logs && \ + mkdir -p /databricks/spark/scripts/ttyd/ && \ + mkdir -p /etc/monit/conf.d/ && \ + mv ttyd.x86_64 /databricks/spark/scripts/ttyd/ttyd && \ + export TTYD_BIN_FILE=/databricks/spark/scripts/ttyd/ttyd + +ENV TTYD_DIR=/databricks/spark/scripts/ttyd +ENV TTYD_BIN_FILE=$TTYD_DIR/ttyd + +COPY webterminal/setup_ttyd_daemon.sh $TTYD_DIR/setup_ttyd_daemon.sh +COPY webterminal/stop_ttyd_daemon.sh $TTYD_DIR/stop_ttyd_daemon.sh +COPY webterminal/start_ttyd_daemon.sh $TTYD_DIR/start_ttyd_daemon.sh +COPY webterminal/webTerminalBashrc $TTYD_DIR/webTerminalBashrc +RUN echo '\ +check process ttyd with pidfile /var/run/ttyd-daemon.pid\n\ + start program = "/databricks/spark/scripts/ttyd/start_ttyd_daemon.sh"\n\ + stop program = "/databricks/spark/scripts/ttyd/stop_ttyd_daemon.sh"' > /etc/monit/conf.d/ttyd-daemon-not-active + +FROM with-webterminal as with-alluxio +############# +# Setup Alluxio +############# +ARG ALLUXIO_VERSION=2.8.0 +ARG ALLUXIO_HOME="/opt/alluxio-${ALLUXIO_VERSION}" +ARG ALLUXIO_TAR_FILE="alluxio-${ALLUXIO_VERSION}-bin.tar.gz" +ARG ALLUXIO_DOWNLOAD_URL="https://downloads.alluxio.io/downloads/files/${ALLUXIO_VERSION}/${ALLUXIO_TAR_FILE}" + +RUN wget -O /tmp/$ALLUXIO_TAR_FILE ${ALLUXIO_DOWNLOAD_URL} \ + && tar zxf /tmp/${ALLUXIO_TAR_FILE} -C /opt/ \ + && rm -f /tmp/${ALLUXIO_TAR_FILE} \ + && cp ${ALLUXIO_HOME}/client/alluxio-${ALLUXIO_VERSION}-client.jar /databricks/jars/ + +############# +# Allow ubuntu user to sudo without password +############# +RUN echo "ubuntu ALL=(ALL) NOPASSWD:ALL" > /etc/sudoers.d/ubuntu \ + && chmod 555 /etc/sudoers.d/ubuntu diff --git a/docker/databricks/README.md b/docker/databricks/README.md new file mode 100644 index 00000000000..42a82b14508 --- /dev/null +++ b/docker/databricks/README.md @@ -0,0 +1,129 @@ +# RAPIDS Accelerator for Apache Spark Docker container for Databricks + +## Build the container + +You will need to build the Docker container before using it in the Databricks environment. This can be done with the provided build script. You can customize a few options using environment variables, but will at minimum need to set REPO_BASE and TAG_NAME to a repository where you can push the built image. For example to push an image to repository at `i-love-spark/rapids-4-spark-databricks`: + +```bash +$ REPO_BASE=i-love-spark TAG_NAME=rapids-4-spark-databricks ./build.sh +``` + +The script will then build an image with fully qualified tag: `i-love-spark/rapids-4-spark-databricks:22.10.0`. + +If you set `PUSH=true`, if the build completes successfully, the script will push it to specified repository. Only do this if you have authenticated using Docker to the repository and you have the appropriate permissions to push image artifacts. + +```bash +$ REPO_BASE=i-love-spark TAG_NAME=rapids-4-spark-databricks PUSH=true ./build.sh +``` + +There are other customizations possible, see the source in `build.sh` for more information. + +Once this image is pushed to your repository, it is ready to be used on the Databricks environment. + +## Usage + +The easiest way to use the RAPIDS Accelerator for Spark on Databricks is use the pre-built Docker +container and Databricks Container Services. + +Currently the Docker container supports the following Databricks runtime(s) via Databricks Container Services: +- [Databricks 10.4 LTS](https://docs.databricks.com/release-notes/runtime/10.4.html#system-environment) + +See [Customize containers with Databricks Container Services](https://docs.databricks.com/clusters/custom-containers.html) for more information. + +Create a Databricks cluster by going to Clusters, then clicking `+ Create Cluster`. Ensure the +cluster meets the prerequisites above by configuring it as follows: + +1. In the `Databricks runtime version` field, click `Standard` and select `Runtime: 10.4 LTS (Scala 2.12, Spark 3.2.1)` (do NOT use `Runtime: 10.4 LTS ML (GPU, Scala 2.12, Spark 3.2.1)` from the `ML` tab). + +2. Ensure `Use Photon Acceleration` is disabled. + +3. Ensure `Enable autoscaling` is disabled. + +Note that GPU nodes are not available to be selected at this time for the driver or the workers. Therefore, you must first configure the use of the Docker container before configuring the driver and worker nodes. + +4. Under the `Advanced options`, select the `Docker` tab. + +5. Select `Use your own Docker container`. + +6. In the `Docker Image URL` field, enter the image location you pushed to using the build steps. + +7. Set `Authentication` set to `Default` if using a public repository, or configure `Authentication` for the repository you have pushed the image to. + +Now you can configure the driver and worker nodes in the main part of the UI. + +8. Choose the number of workers that matches the number of GPUs you want to use. + +9. Select a worker type. On AWS, use nodes with 1 GPU each such as `p3.2xlarge` or `g4dn.xlarge`. + p2 nodes do not meet the architecture requirements (Pascal or higher) for the Spark worker + (although they can be used for the driver node). For Azure, choose GPU nodes such as + Standard_NC6s_v3. For GCP, choose N1 or A2 instance types with GPUs. + +10. Select a driver type. Generally, this can be set the same as the worker, but you can select a node that + does NOT include a GPU if you don't plan to do any GPU-related operations on the driver. On AWS, this + can be an `i3.xlarge` or larger. + +11. Now select the `Init Scripts` tab. + +12. In the `Destination` field, select `FILE`. + +13. In the `Init script path` field, enter `file:/opt/spark-rapids/init.sh` + +14. Click `Add`. + +15. Add any other configs, such as SSH Key, Logging, or additional Spark configuration. The Docker container uses the configuration in `00-custom-spark-driver-defaults.conf` by default. When adding additional lines to `Spark config` in the UI, the configuration will override those defaults that are configured in the Docker container. + +16. Start the cluster. + +### Enabling Alluxio in the Databricks cluster using the Docker container + +If you would like to enable the Alluxio cluster on your Databricks cluster, you will need to add the following configuration to your cluster. + +1. Edit the desired cluster. + +2. Under the `Advanced options`, select the `Spark` tab. + +3. In the `Spark config` field, add the following lines. The second 2 are good starting points when using Alluxio but could be tuned +if needed. + +``` +spark.databricks.io.cached.enabled false +spark.rapids.alluxio.automount.enabled true +spark.rapids.sql.coalescing.reader.numFilterParallel 2 +spark.rapids.sql.multiThreadedRead.numThreads 40 +``` + +4. In the `Environment variables` field, add the line `ENABLE_ALLUXIO=1`. + +5. Customize Alluxio configuration using the following configs if needed. These should be added in the `Environment variables` field if you wish to change them. + + - The default amount of disk space used for Alluxio on the Workers is 70%. This can be adjusted using the configuration below. `ALLUXIO_STORAGE_PERCENT=70` + + - The default heap size used by the Alluxio Master process is 16GB, this may need to be changed depending on the size of the driver node. Make sure it has enough memory for the Master and the Spark driver processes. `ALLUXIO_MASTER_HEAP=16g` + + - To copy the Alluxio Master and Worker logs off of local disk to be able to look at them after the cluster is shutdown you can configure this to some path accessible via rsync. For instance, on Databricks this might be a path in /dbfs/. `ALLUXIO_COPY_LOG_PATH=/dbfs/somedirectory/` + +5. Click `Confirm` (if the cluster is currently stopped) or `Confirm and Restart` if the cluster is currently running. + +6. Ensure the cluster is started by click `Start` if necessary. + +To verify the alluxio cluster is working, you can use the Web Terminal: + +1. Ensure the cluster is fully up and running. Then in the cluster UI, click the `Apps` tab. + +2. Click `Launch Web Terminal`. + +3. In the new tab that opens, you will get a terminal session. + +4. Run the following command: + +```bash +$ /opt/alluxio-2.8.0/bin/alluxio fsadmin report +``` + +5. You should see a line indicating the number of active workers, ensure this is equal to the configured number of workers you used for the cluster: + +``` +... + Live Workers: 2 +... +``` diff --git a/docker/databricks/build.sh b/docker/databricks/build.sh new file mode 100755 index 00000000000..a43a510648a --- /dev/null +++ b/docker/databricks/build.sh @@ -0,0 +1,53 @@ +#!/bin/bash +# Copyright (c) 2022, NVIDIA CORPORATION. +# +# 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. + +set -ex + +REPO_BASE=${REPO_BASE:-"nvidia"} +TAG_NAME=${TAG_NAME:-"rapids-4-spark-databricks"} +VERSION=${VERSION:-"22.10.0"} +TAG_VERSION=${TAG_VERSION:-$VERSION} +CUDA_VERSION=${CUDA_VERSION:-"11.5.1"} +CUDA_PKG_VERSION=${CUDA_VERSION%.*} +CUDA_PKG_VERSION=${CUDA_PKG_VERSION/./-} + +DOCKERFILE=${DOCKERFILE:-"Dockerfile"} +BASE_JAR_URL=${BASE_JAR_URL:-"https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12"} + +JAR_VERSION=${JAR_VERSION:-$VERSION} +JAR_FILE=${JAR_FILE:-"rapids-4-spark_2.12-${JAR_VERSION}-cuda11.jar"} +JAR_URL=${JAR_URL:-"${BASE_JAR_URL}/${VERSION}/${JAR_FILE}"} + +DRIVER_CONF_FILE=${DRIVER_CONF_FILE:-"00-custom-spark-driver-defaults.conf"} +DATABRICKS_RUNTIME_VERSION=${DATABRICKS_RUNTIME_VERSION:-"10.4"} +REQUIREMENTS="requirements_${DATABRICKS_RUNTIME_VERSION}ML.txt" + +STAGE=${STAGE:-"with-alluxio"} + +docker build \ + --build-arg CUDA_VERSION=${CUDA_VERSION} \ + --build-arg CUDA_PKG_VERSION=${CUDA_PKG_VERSION} \ + --build-arg JAR_URL=${JAR_URL} \ + --build-arg JAR_FILE=${JAR_FILE} \ + --build-arg REQUIREMENTS=${REQUIREMENTS} \ + --build-arg DRIVER_CONF_FILE=${DRIVER_CONF_FILE} \ + --target $STAGE \ + -f ${DOCKERFILE} \ + -t "${REPO_BASE}/${TAG_NAME}:${TAG_VERSION}" \ + . + +if [[ $PUSH == "true" ]]; then + docker push "${REPO_BASE}/${TAG_NAME}:${TAG_VERSION}" +fi diff --git a/docker/databricks/ganglia/ganglia.conf b/docker/databricks/ganglia/ganglia.conf new file mode 100644 index 00000000000..e78a112934c --- /dev/null +++ b/docker/databricks/ganglia/ganglia.conf @@ -0,0 +1,37 @@ +# Copyright (c) 2022, NVIDIA CORPORATION. +# +# 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. + + ServerAdmin webmaster@localhost + + # Show Ganglia UI by default + DocumentRoot /usr/share/ganglia-webfrontend + + # Available loglevels: trace8, ..., trace1, debug, info, notice, warn, + # error, crit, alert, emerg. + # It is also possible to configure the loglevel for particular + # modules, e.g. + #LogLevel info ssl:warn + + ErrorLog ${APACHE_LOG_DIR}/error.log + CustomLog ${APACHE_LOG_DIR}/access.log combined + + # For most configuration files from conf-available/, which are + # enabled or disabled at a global level, it is possible to + # include a line for only one particular virtual host. For example the + # following line enables the CGI configuration for this host only + # after it has been globally disabled with "a2disconf". + #Include conf-available/serve-cgi-bin.conf + + +# vim: syntax=apache ts=4 sw=4 sts=4 sr noet diff --git a/docker/databricks/ganglia/gconf/conf.d/modpython.conf b/docker/databricks/ganglia/gconf/conf.d/modpython.conf new file mode 100644 index 00000000000..e7105969977 --- /dev/null +++ b/docker/databricks/ganglia/gconf/conf.d/modpython.conf @@ -0,0 +1,30 @@ +/* + Copyright (c) 2022, NVIDIA CORPORATION. + + 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. + + params - path to the directory where mod_python + should look for python metric modules + + the "pyconf" files in the include directory below + will be scanned for configurations for those modules +*/ +modules { + module { + name = "python_module" + path = "/usr/lib/ganglia/modpython.so" + params = "/usr/lib/ganglia/python_modules" + } +} + +include ('/etc/ganglia/conf.d/*.pyconf') diff --git a/docker/databricks/ganglia/gconf/databricks-gmond.conf b/docker/databricks/ganglia/gconf/databricks-gmond.conf new file mode 100644 index 00000000000..dea09b08b4f --- /dev/null +++ b/docker/databricks/ganglia/gconf/databricks-gmond.conf @@ -0,0 +1,33 @@ +/* + Copyright (c) 2022, NVIDIA CORPORATION. + + 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. +*/ +cluster { + name = "cluster" + owner = "unspecified" + latlong = "unspecified" + url = "unspecified" +} + +udp_send_channel { + host = 10.126.246.34 + port = 8649 + ttl = 1 +} + +/* This is set for gmond running on the driver to receive metrics from other gmonds, and is not + in use for gmonds running on workers. */ +udp_recv_channel { + port = 8649 +} \ No newline at end of file diff --git a/docker/databricks/ganglia/gconf/gmetad.conf b/docker/databricks/ganglia/gconf/gmetad.conf new file mode 100755 index 00000000000..128f6d8937b --- /dev/null +++ b/docker/databricks/ganglia/gconf/gmetad.conf @@ -0,0 +1,22 @@ +# Copyright (c) 2022, NVIDIA CORPORATION. +# +# 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 configuration file is a set of default ganglia-monitor configuration found after installing +# ganglia 2.5.x at /etc/ganglia/gmond.conf, and is built into the spark container. +# Note that dynamically generated configuration is generated by a script `start_ganglia` +# and is included by the line below. + +data_source "cluster" localhost +setuid_username "ganglia" +xml_port 8651 \ No newline at end of file diff --git a/docker/databricks/ganglia/gconf/gmond.conf b/docker/databricks/ganglia/gconf/gmond.conf new file mode 100755 index 00000000000..b17189c108b --- /dev/null +++ b/docker/databricks/ganglia/gconf/gmond.conf @@ -0,0 +1,342 @@ +# Copyright (c) 2022, NVIDIA CORPORATION. +# +# 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 configuration file is a set of default ganglia-monitor configuration found after installing +# ganglia 2.5.x at /etc/ganglia/gmond.conf, and is built into the spark container. +# Note that dynamically generated configuration is generated by a script `start_ganglia` +# and is included by the line below. + +include ('/etc/ganglia/databricks-gmond.conf') + +/* This configuration is as close to 2.5.x default behavior as possible + The values closely match ./gmond/metric.h definitions in 2.5.x */ +globals { + daemonize = yes + setuid = yes + user = ganglia + debug_level = 0 + max_udp_msg_len = 1472 + mute = no + deaf = no + host_dmax = 0 /*secs */ + cleanup_threshold = 300 /*secs */ + gexec = no + /* See https://github.com/ganglia/monitor-core/wiki/FAQ#when-using-unicast-restarting-the-collector-gmond-causes-metric-graphs-to-disappear */ + send_metadata_interval = 30 /*secs */ +} + +/* The host section describes attributes of the host, like the location */ +host { + location = "unspecified" +} + +/* You can specify as many tcp_accept_channels as you like to share + an xml description of the state of the cluster */ +tcp_accept_channel { + port = 8649 +} + +/* Each metrics module that is referenced by gmond must be specified and + loaded. If the module has been statically linked with gmond, it does not + require a load path. However all dynamically loadable modules must include + a load path. */ +modules { + module { + name = "core_metrics" + } + module { + name = "cpu_module" + path = "/usr/lib/ganglia/modcpu.so" + } + module { + name = "disk_module" + path = "/usr/lib/ganglia/moddisk.so" + } + module { + name = "load_module" + path = "/usr/lib/ganglia/modload.so" + } + module { + name = "mem_module" + path = "/usr/lib/ganglia/modmem.so" + } + module { + name = "net_module" + path = "/usr/lib/ganglia/modnet.so" + } + module { + name = "proc_module" + path = "/usr/lib/ganglia/modproc.so" + } + module { + name = "sys_module" + path = "/usr/lib/ganglia/modsys.so" + } +} + +/* The old internal 2.5.x metric array has been replaced by the following + collection_group directives. What follows is the default behavior for + collecting and sending metrics that is as close to 2.5.x behavior as + possible. */ + +/* This collection group will cause a heartbeat (or beacon) to be sent every + 20 seconds. In the heartbeat is the GMOND_STARTED data which expresses + the age of the running gmond. */ +collection_group { + collect_once = yes + time_threshold = 20 + metric { + name = "heartbeat" + } +} + +/* This collection group will send general info about this host every 300 secs. + This information doesn't change between reboots and is only collected once. */ +collection_group { + collect_once = yes + time_threshold = 300 + metric { + name = "cpu_num" + title = "CPU Count" + } + metric { + name = "cpu_speed" + title = "CPU Speed" + } + metric { + name = "mem_total" + title = "Memory Total" + } + /* Should this be here? Swap can be added/removed between reboots. */ + metric { + name = "swap_total" + title = "Swap Space Total" + } + metric { + name = "boottime" + title = "Last Boot Time" + } + metric { + name = "machine_type" + title = "Machine Type" + } + metric { + name = "os_name" + title = "Operating System" + } + metric { + name = "os_release" + title = "Operating System Release" + } + metric { + name = "location" + title = "Location" + } +} + +/* This collection group will send the status of gexecd for this host every 300 secs */ +/* Unlike 2.5.x the default behavior is to report gexecd OFF. */ +collection_group { + collect_once = yes + time_threshold = 300 + metric { + name = "gexec" + title = "Gexec Status" + } +} + +/* This collection group will collect the CPU status info every 20 secs. + The time threshold is set to 90 seconds. In honesty, this time_threshold could be + set significantly higher to reduce unneccessary network chatter. */ +collection_group { + collect_every = 20 + time_threshold = 90 + /* CPU status */ + metric { + name = "cpu_user" + value_threshold = "1.0" + title = "CPU User" + } + metric { + name = "cpu_system" + value_threshold = "1.0" + title = "CPU System" + } + metric { + name = "cpu_idle" + value_threshold = "5.0" + title = "CPU Idle" + } + metric { + name = "cpu_nice" + value_threshold = "1.0" + title = "CPU Nice" + } + metric { + name = "cpu_aidle" + value_threshold = "5.0" + title = "CPU aidle" + } + metric { + name = "cpu_wio" + value_threshold = "1.0" + title = "CPU wio" + } + /* The next two metrics are optional if you want more detail... + ... since they are accounted for in cpu_system. + metric { + name = "cpu_intr" + value_threshold = "1.0" + title = "CPU intr" + } + metric { + name = "cpu_sintr" + value_threshold = "1.0" + title = "CPU sintr" + } + */ +} + +collection_group { + collect_every = 20 + time_threshold = 90 + /* Load Averages */ + metric { + name = "load_one" + value_threshold = "1.0" + title = "One Minute Load Average" + } + metric { + name = "load_five" + value_threshold = "1.0" + title = "Five Minute Load Average" + } + metric { + name = "load_fifteen" + value_threshold = "1.0" + title = "Fifteen Minute Load Average" + } +} + +/* This group collects the number of running and total processes */ +collection_group { + collect_every = 80 + time_threshold = 950 + metric { + name = "proc_run" + value_threshold = "1.0" + title = "Total Running Processes" + } + metric { + name = "proc_total" + value_threshold = "1.0" + title = "Total Processes" + } +} + +/* This collection group grabs the volatile memory metrics every 40 secs and + sends them at least every 180 secs. This time_threshold can be increased + significantly to reduce unneeded network traffic. */ +collection_group { + collect_every = 40 + time_threshold = 180 + metric { + name = "mem_free" + value_threshold = "1024.0" + title = "Free Memory" + } + metric { + name = "mem_shared" + value_threshold = "1024.0" + title = "Shared Memory" + } + metric { + name = "mem_buffers" + value_threshold = "1024.0" + title = "Memory Buffers" + } + metric { + name = "mem_cached" + value_threshold = "1024.0" + title = "Cached Memory" + } + metric { + name = "swap_free" + value_threshold = "1024.0" + title = "Free Swap Space" + } +} + +collection_group { + collect_every = 40 + time_threshold = 300 + metric { + name = "bytes_out" + value_threshold = 4096 + title = "Bytes Sent" + } + metric { + name = "bytes_in" + value_threshold = 4096 + title = "Bytes Received" + } + metric { + name = "pkts_in" + value_threshold = 256 + title = "Packets Received" + } + metric { + name = "pkts_out" + value_threshold = 256 + title = "Packets Sent" + } +} + +/* Different than 2.5.x default since the old config made no sense */ +collection_group { + collect_every = 1800 + time_threshold = 3600 + metric { + name = "disk_total" + value_threshold = 1.0 + title = "Total Disk Space" + } +} + +collection_group { + collect_every = 40 + time_threshold = 180 + metric { + name = "disk_free" + value_threshold = 1.0 + title = "Disk Space Available" + } + metric { + name = "part_max_used" + value_threshold = 1.0 + title = "Maximum Disk Space Used" + } +} + +/* Ganglia modules are defined in terms of .conf files in /etc/ganglia/conf.d; the directive below +includes all such .conf files. + +Ganglia Python modules (e.g. the NVIDIA python module for monitoring GPUs) are specified +in .pyconf files within /etc/ganglia/conf.d. When we install support for Ganglia python modules, +(via apt-get install ganglia-monitor-python in our GPU base image) the installation process +creates a /etc/ganglia/conf.d/modpython.conf file. The directive below includes the modpython.conf +file, which in turn contains a directive to include all .pyconf files within /etc/ganglia/conf.d. + +Load this after built-in collection groups so gmond sends them first. */ +include ('/etc/ganglia/conf.d/*.conf') diff --git a/docker/databricks/ganglia/start_spark_slave.sh b/docker/databricks/ganglia/start_spark_slave.sh new file mode 100644 index 00000000000..46d09799a6e --- /dev/null +++ b/docker/databricks/ganglia/start_spark_slave.sh @@ -0,0 +1,189 @@ +#!/bin/bash +# Copyright (c) 2022, NVIDIA CORPORATION. +# +# 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. + +# Parameters: +# 1 spark master +# 2 spark master port +# 3 spark master public DNS +# 4 spark master web ui port +# 5 optional: cores to use + +# Write the spark.defaults file for HIPAA compliance if necessary +${DB_HOME}/spark/scripts/hipaa-setup.sh spark-defaults + +export SPARK_HOME=$DB_HOME/spark + +. "${SPARK_HOME}/bin/load-spark-env.sh" + +# Appending ML Runtime Python Path items, if they exist to PYTHONPATH +if [[ -f $MLR_PYTHONPATH ]]; +then + MLR_PYTHONPATH_CONTENT=`cat $MLR_PYTHONPATH` + export PYTHONPATH=$SPARK_HOME/python:$MLR_PYTHONPATH_CONTENT +fi + +# Set R_LIBS to configure library installation locations. +R_LIBS_PRIMARY=$SPARK_HOME/R/lib +# Set environment variables for NFS library directories. These should be kept in sync with +# setup_driver_env.sh. Additionally, DATABRICKS_LIBS_NFS_ROOT_DIR needs to match the value of +# NFS_PATH_SUFFIX set for the sidecar container in Kubernetes-based dataplane, see +# DbrClusterJsonnetInputParams. +export DATABRICKS_LIBS_NFS_ROOT_DIR="${DATABRICKS_LIBS_NFS_ROOT_DIR:=.ephemeral_nfs}" +export DATABRICKS_CLUSTER_LIBS_ROOT_DIR="${DATABRICKS_CLUSTER_LIBS_ROOT_DIR:=cluster_libraries}" +export DATABRICKS_CLUSTER_LIBS_R_ROOT_DIR="${DATABRICKS_CLUSTER_LIBS_R_ROOT_DIR:=r}" +# Get the first element of $SPARK_LOCAL_DIRS. +SPARK_LOCAL_DIR=$(echo $SPARK_LOCAL_DIRS | cut -d , -f 1) +R_CLUSTER_LIBS=$SPARK_LOCAL_DIR/$DATABRICKS_LIBS_NFS_ROOT_DIR/$DATABRICKS_CLUSTER_LIBS_ROOT_DIR/$DATABRICKS_CLUSTER_LIBS_R_ROOT_DIR +export R_LIBS=$R_LIBS_PRIMARY:$R_CLUSTER_LIBS + +SECONDS=0 +mkdir -p $R_LIBS_PRIMARY +R CMD INSTALL --library=$R_LIBS_PRIMARY --no-docs --no-html $SPARK_HOME/R/pkg/ +echo "Took $SECONDS seconds to install Spark R" + +echo "$@" > /tmp/slave-params + +export SPARK_PUBLIC_DNS=$3 +export SPARK_WORKER_WEBUI_PORT=$4 + +if [ "$5" -gt "0" ]; then + export SPARK_WORKER_CORES=$5 +fi + +# Config directory +DBCONF_DIR=$DB_HOME/spark/dbconf +# Log4j config +LOG4J_DIR=$DBCONF_DIR/log4j/master-worker +# The slave must have SPARK_JARS in its classpath in addition +# to $DB_HOME/spark/assembly/target/scala-2.10/spark-assembly-1.0.0-hadoop1.2.1.jar. +# The reason is that this assembly jar is empty, according to ami/charms/worker/runnext. +# +# Include the driver-daemon jars (which includes all relevant spark jars). +SPARK_JARS=`echo /databricks/jars/* | tr " " :` + +SHUFFLE_SERVICE_OPT="" +SPARK_EXECUTOR_LOG_CONF="" +SPARK_VERSION=$(head -n 1 $DB_HOME/spark/VERSION) + +if [[ ${SPARK_VERSION:0:1} -ge 2 ]]; then + # Shuffle service is only enabled if the Spark version is greater than or equal to 2.0.0 + SHUFFLE_SERVICE_OPT="-Dspark.shuffle.service.enabled=true -Dspark.shuffle.service.port=4048" + # Only need to configure executor logging strategy if the Spark version is greater than or equal to 2.0.0 + SPARK_EXECUTOR_LOG_CONF="-Dspark.executor.logs.rolling.strategy=time" + SPARK_EXECUTOR_LOG_CONF="$SPARK_EXECUTOR_LOG_CONF -Dspark.executor.logs.rolling.time.interval=hourly" + SPARK_EXECUTOR_LOG_CONF="$SPARK_EXECUTOR_LOG_CONF -Dspark.executor.logs.rolling.maxRetainedFiles=72" + SPARK_EXECUTOR_LOG_CONF="$SPARK_EXECUTOR_LOG_CONF -Dspark.executor.logs.rolling.enableCompression=true" +fi + +$SPARK_HOME/scripts/install_gpu_driver.sh + +# If it is GPU image (check nvidia-smi) and the spark version is greater or equal to 3.0.0, +# we will set up the configs for GPU scheduling +if [[ ${SPARK_VERSION:0:1} -ge 3 ]] && [[ -x "$(command -v nvidia-smi)" ]]; then + NUM_GPUS=`nvidia-smi --query-gpu=name --format=csv,noheader | wc -l` + SPARK_RESOURCE_CONF="-Dspark.worker.resource.gpu.discoveryScript=/databricks/spark/scripts/gpu/get_gpus_resources.sh -Dspark.worker.resource.gpu.amount=$NUM_GPUS" +fi + +# LOG4J_DIR must be specified before SPARK_JARS +export SPARK_DAEMON_JAVA_OPTS="$SHUFFLE_SERVICE_OPT -XX:-UseContainerSupport -Dspark.worker.cleanup.enabled=false $SPARK_RESOURCE_CONF $SPARK_EXECUTOR_LOG_CONF -cp $LOG4J_DIR:$SPARK_JARS" + +# Inject allowlisted custom spark conf to spark worker +# Same code snippet in start_spark_master.sh +get_custom_spark_conf_location() { + local deploy_conf_dir="$DB_HOME/common/conf" + + local value="$($DB_HOME/spark/scripts/conf_reader \ + --key "databricks.common.driver.sparkConfFilePath" \ + --project-group "untrusted-projects" \ + --conf-dir "$deploy_conf_dir" \ + --quiet 2> /dev/null)" + if [[ -z "$value" ]]; then + echo "/tmp/custom-spark.conf" + else + echo "$value" + fi +} + +custom_spark_conf_file="$(get_custom_spark_conf_location)" + +SPARK_CONF_KEYS_TO_PROPAGATE=( + 'spark.ui.reverseProxy' + 'spark.ui.reverseProxyUrl' + 'spark.worker.aioaLazyConfig.enabled' + 'spark.worker.aioaLazyConfig.iamRoleCheckEnabled' + 'spark.worker.register.initialRegistrationRetries' + 'spark.worker.register.proLongedRegistrationRetries' + 'spark.worker.register.initialRegistrationIntervalSec' + 'spark.worker.register.prolongedRegistrationIntervalSec' +) + +for KEY in "${SPARK_CONF_KEYS_TO_PROPAGATE[@]}"; do + VALUE="$(cat "$custom_spark_conf_file" | grep "^$KEY " | cut -d' ' -f2-)" + if [[ ! -z "$VALUE" ]]; then + SPARK_DAEMON_JAVA_OPTS+=" -D$KEY='$VALUE'" + fi +done + +export SPARK_DIST_CLASSPATH="$LOG4J_DIR:$SPARK_JARS" + +source $SPARK_HOME/scripts/rotate_logs.sh +rotate_active_spark_log $SPARK_HOME/logs + +# On slaves, we use single thread OpenBLAS. +export OPENBLAS_NUM_THREADS=1 + +# The default malloc() implementation is rather aggresive in creating new arenas to handle memory allocations in a multi-threaded workload. +# By default it can create up to 8 x NUM_CORES arenas. It leads to increased memory fragmentation. The Spark executor doesn't typically have +# much space for memory from malloc() as most of it is used by java heap. Therefore, it benefits from a much more conservative malloc() settings. +# See: +# https://github.com/prestodb/presto/issues/8993 +# https://devcenter.heroku.com/articles/tuning-glibc-memory-behavior +export MALLOC_ARENA_MAX=4 + +# Configure metrics.properties to send metrics to gmond on the driver node +echo "*.sink.ganglia.host=$1" >> $SPARK_HOME/conf/metrics.properties + +# START SPARK SLAVE. We will always launch a single worker at here. +# Activate the root python env before start Spark slave. +# - For conda runtime, this is DATABRICKS_ROOT_CONDA_ENV +# - For non-conda runtime, this is DATABRICKS_ROOT_VIRTUALENV_ENV +# The conda script is a no-op in non-conda runtimes. +( + /databricks/spark/scripts/copy_conda_pkgs_dir.sh + . /databricks/spark/scripts/activate_root_python_environment.sh + $SPARK_HOME/sbin/start-slave.sh spark://$1:$2 +) + +# Start gmond and setup monit configurations +$SPARK_HOME/scripts/ganglia/start_ganglia.sh --services ganglia-monitor --driver_ip $1 + +# Enable monit for the slave +if [ ! -e /etc/monit/conf.d/spark-slave.cfg ]; then + cp /etc/monit/conf.d/spark-slave-not-active /etc/monit/conf.d/spark-slave.cfg + /etc/init.d/monit start + /etc/init.d/monit reload +fi + +# Make sure that Python is fully initialized before we start spark worker. +# So, when user ubuntu is used to interact with Python, user ubuntu +# will not see permission related issue. +# We ignore the return code of this call. So, even the python environment +# is in a bad state because of any customization by users, we can still +# start services in the container. +echo "Trigger Python initialization" +/databricks/python/bin/python --version || true + +# Note: This should be the highest priority (most likely to be killed) +nohup $SPARK_HOME/scripts/set_jvms_oom_priority.sh 2000 &> /tmp/set_jvms_oom_priority_output & diff --git a/docker/databricks/init.sh b/docker/databricks/init.sh new file mode 100644 index 00000000000..6867adddf74 --- /dev/null +++ b/docker/databricks/init.sh @@ -0,0 +1,234 @@ +#!/bin/bash +# Copyright (c) 2022, NVIDIA CORPORATION. +# +# 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. + +# Try to run Alluxio directly in init_script +# Driver side: +# 1. Update alluxio-site.properties +# 2. Format master (No need to copyDir conf/) +# 3. Start alluxio master only. +# Worker side: +# 1. Update alluxio-site.properties as same as Driver side +# 2. Start alluxio worker only. Make sure succeed to connect to master. Wait master? + +#################### +# Global constants # +#################### +readonly ALLUXIO_VERSION="2.8.0" +readonly ALLUXIO_HOME="/opt/alluxio-${ALLUXIO_VERSION}" +readonly ALLUXIO_SITE_PROPERTIES="${ALLUXIO_HOME}/conf/alluxio-site.properties" +readonly ALLUXIO_METRICS_PROPERTIES_TEMPLATE="${ALLUXIO_HOME}/conf/metrics.properties.template" +readonly ALLUXIO_METRICS_PROPERTIES="${ALLUXIO_HOME}/conf/metrics.properties" +ALLUXIO_STORAGE_PERCENT=${ALLUXIO_STORAGE_PERCENT:-'70'} +ALLUXIO_MASTER_HEAP=${ALLUXIO_MASTER_HEAP:-'16g'} +# location to copy the Alluxio logs to so that they are kept after cluster is shutdown +# recommended location would be dbfs on Databricks, path has to be accessible via rsync +ALLUXIO_COPY_LOG_PATH=${ALLUXIO_COPY_LOG_PATH:-''} + +# Run a command as a specific user +# Assumes the provided user already exists on the system and user running script has sudo access +# +# Args: +# $1: user +# $2: cmd +doas() { + if [[ "$#" -ne "2" ]]; then + echo "Incorrect number of arguments passed into function doas, expecting 2" + exit 2 + fi + local user="$1" + local cmd="$2" + + runuser -l "${user}" -c "${cmd}" +} + +# Appends or replaces a property KV pair to the alluxio-site.properties file +# +# Args: +# $1: property +# $2: value +set_alluxio_property() { + if [[ "$#" -ne "2" ]]; then + echo "Incorrect number of arguments passed into function set_alluxio_property, expecting 2" + exit 2 + fi + local property="$1" + local value="$2" + + if grep -qe "^\s*${property}=" ${ALLUXIO_SITE_PROPERTIES} 2> /dev/null; then + doas ubuntu "sed -i 's;${property}=.*;${property}=${value};g' ${ALLUXIO_SITE_PROPERTIES}" + echo "Property ${property} already exists in ${ALLUXIO_SITE_PROPERTIES} and is replaced with value ${value}" >&2 + else + doas ubuntu "echo '${property}=${value}' >> ${ALLUXIO_SITE_PROPERTIES}" + fi +} + +# Appends or replaces a property KV pair to the metrics.properties file +set_metrics_property() { + if [[ "$#" -ne "2" ]]; then + echo "Incorrect number of arguments passed into function set_metrics_property, expecting 2" + exit 2 + fi + local property="$1" + local value="$2" + + if [ ! -f "$ALLUXIO_METRICS_PROPERTIES" ]; then + doas ubuntu "cp ${ALLUXIO_METRICS_PROPERTIES_TEMPLATE} ${ALLUXIO_METRICS_PROPERTIES}" + fi + if grep -qe "^\s*${property}=" ${ALLUXIO_METRICS_PROPERTIES} 2> /dev/null; then + doas ubuntu "sed -i 's;${property}=.*;${property}=${value};g' ${ALLUXIO_METRICS_PROPERTIES}" + echo "Property ${property} already exists in ${ALLUXIO_METRICS_PROPERTIES} and is replaced with value ${value}" >&2 + else + doas ubuntu "echo '${property}=${value}' >> ${ALLUXIO_METRICS_PROPERTIES}" + fi +} + + +# Configures Alluxio to use NVMe mounts as storage +# Returns "true" if Alluxio should configure MEM tier when no NVMe mounts are available +# +# Args: +# $1: nvme_capacity_usage - Argument value of [-n ] +configure_nvme() { + if [[ "$#" -ne "1" ]]; then + echo "Incorrect number of arguments passed into function configure_nvme, expecting 1" + exit 2 + fi + nvme_capacity_usage=$1 + + local use_mem="true" + local paths="" + local quotas="" + local medium_type="" + # In databricks instance, /local_disk0 is the local NVME disk. + # in the format of " " + # The block size parameter (-B) is in MB (1024 * 1024) + local -r mount_points="$(df -B 1048576 | grep 'local_disk' | awk '{print $1, $4, $6}')" + set +e + # read returns 1 unless EOF is reached, but we specify -d '' which means always read until EOF + IFS=$'\n' read -d '' -ra mounts <<< "${mount_points}" + set -e + # attempt to configure NVMe, otherwise fallback to MEM + # Should only 1 NVMe disk, I need to + if [[ "${#mounts[@]}" -gt 1 ]]; then + echo "More than 1 NVMe device, need to modify the script to cover it" + exit 2 + fi + if [[ "${#mounts[@]}" == 1 ]]; then + for mount_point in "${mounts[@]}"; do + local path_cap + local mnt_path + local quota_p + path_cap="$(echo "${mount_point}" | awk '{print $2}')" + # mnt_path="$(echo "${mount_point}" | awk '{print $3}')" + quota_p=$((path_cap * nvme_capacity_usage / 100)) + # if alluxio doesn't have permissions to write to this directory it will fail + # fixed to use the /cache folder in alluxio folder + mnt_path="/local_disk0/cache" + paths+="${mnt_path}," + quotas+="${quota_p}MB," + medium_type+="SSD," + done + paths="${paths::-1}" + quotas="${quotas::-1}" + medium_type="${medium_type::-1}" + + use_mem="false" + set_alluxio_property alluxio.worker.tieredstore.level0.alias "SSD" + set_alluxio_property alluxio.worker.tieredstore.level0.dirs.mediumtype "${medium_type}" + set_alluxio_property alluxio.worker.tieredstore.level0.dirs.path "${paths}" + set_alluxio_property alluxio.worker.tieredstore.level0.dirs.quota "${quotas}" + fi + echo "${use_mem}" +} + +# add crontab to rsync alluxio log to /dbfs/cluster-logs/alluxio +set_crontab_alluxio_log() { + if [[ "$#" -ne "1" ]]; then + echo "Incorrect" + exit 2 + fi + local folder=$1 + mkdir -p $ALLUXIO_COPY_LOG_PATH/$folder + # add crond to copy alluxio logs + crontab -l > cron_bkp || true + echo "* * * * * /usr/bin/rsync -a /opt/alluxio-2.8.0/logs $ALLUXIO_COPY_LOG_PATH/$folder >/dev/null 2>&1" >> cron_bkp + crontab cron_bkp + rm cron_bkp +} + +start_ssh() { + # Start SSH + service ssh start +} + +start_alluxio() { + + # create the folder for NVMe caching + mkdir -p /local_disk0/cache + chown ubuntu:ubuntu /local_disk0/cache + + # create the folder for domain socket + mkdir -p /local_disk0/alluxio_domain_socket + chown ubuntu:ubuntu /local_disk0/alluxio_domain_socket + + set_alluxio_property alluxio.master.hostname "${DB_DRIVER_IP}" + set_alluxio_property alluxio.underfs.s3.inherit.acl false + set_alluxio_property alluxio.underfs.s3.default.mode 0755 + set_alluxio_property alluxio.worker.tieredstore.levels "1" + set_alluxio_property alluxio.worker.data.server.domain.socket.address /local_disk0/alluxio_domain_socket + set_alluxio_property alluxio.worker.data.server.domain.socket.as.uuid true + set_alluxio_property alluxio.worker.network.async.cache.manager.queue.max 4000 + set_alluxio_property alluxio.user.short.circuit.preferred true + + set_alluxio_property alluxio.job.worker.threadpool.size 20 + set_alluxio_property alluxio.worker.network.block.writer.threads.max 2048 + set_alluxio_property alluxio.worker.network.block.reader.threads.max 2048 + set_alluxio_property alluxio.master.ufs.block.location.cache.capacity 2000000 + set_alluxio_property alluxio.master.ufs.path.cache.capacity 200000 + + sed -i "s/localhost/${DB_DRIVER_IP}/g" /opt/alluxio-${ALLUXIO_VERSION}/conf/masters + + configure_nvme "${ALLUXIO_STORAGE_PERCENT}" + + if [[ $DB_IS_DRIVER = "TRUE" ]]; then + # On Driver + if [[ -n $ALLUXIO_COPY_LOG_PATH ]]; then + set_crontab_alluxio_log "${DB_DRIVER_IP}-master" + fi + MASTER_HEAP_SETTING="-Xms${ALLUXIO_MASTER_HEAP} -Xmx${ALLUXIO_MASTER_HEAP}" + doas ubuntu "ALLUXIO_MASTER_JAVA_OPTS=\"${MASTER_HEAP_SETTING}\" ${ALLUXIO_HOME}/bin/alluxio-start.sh master" + else + # On Workers + if [[ -n $ALLUXIO_COPY_LOG_PATH ]]; then + set_crontab_alluxio_log "${DB_CONTAINER_IP}-worker" + fi + echo "alluxio.worker.hostname=${DB_CONTAINER_IP}" >> ${ALLUXIO_SITE_PROPERTIES} + echo "alluxio.user.hostname=${DB_CONTAINER_IP}" >> ${ALLUXIO_SITE_PROPERTIES} + + n=0 + until [ "$n" -ge 5 ] + do + doas ubuntu "${ALLUXIO_HOME}/bin/alluxio-start.sh worker" && break + n=$((n+1)) + sleep 10 + done + fi +} + +start_ssh + +if [[ "$ENABLE_ALLUXIO" = "1" ]]; then + start_alluxio +fi diff --git a/docker/databricks/requirements_10.4ML.txt b/docker/databricks/requirements_10.4ML.txt new file mode 100644 index 00000000000..9e67f682b20 --- /dev/null +++ b/docker/databricks/requirements_10.4ML.txt @@ -0,0 +1,241 @@ +# Copyright (c) 2022, NVIDIA CORPORATION. +# +# 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. +absl-py==0.11.0 +appdirs==1.4.4 +argon2-cffi==20.1.0 +astor==0.8.1 +astunparse==1.6.3 +async-generator==1.10 +attrs==20.3.0 +backcall==0.2.0 +bcrypt==3.2.0 +bidict==0.21.4 +bleach==3.3.0 +blis==0.7.4 +boto3==1.16.7 +botocore==1.19.7 +cachetools==4.2.4 +catalogue==2.0.6 +certifi==2020.12.5 +cffi==1.14.5 +chardet==4.0.0 +click==7.1.2 +cloudpickle==1.6.0 +cmdstanpy==0.9.68 +configparser==5.0.1 +convertdate==2.3.2 +cryptography==3.4.7 +cycler==0.10.0 +cymem==2.0.5 +Cython==0.29.23 +databricks-automl-runtime==0.2.6.1 +databricks-cli==0.16.3 +dbl-tempo==0.1.2 +dbus-python==1.2.16 +decorator==5.0.6 +defusedxml==0.7.1 +dill==0.3.2 +diskcache==5.2.1 +distlib==0.3.4 +distro==1.4.0 +distro-info===0.23ubuntu1 +entrypoints==0.3 +ephem==4.1.3 +facets-overview==1.0.0 +fasttext==0.9.2 +filelock==3.0.12 +Flask==1.1.2 +flatbuffers==2.0 +fsspec==0.9.0 +future==0.18.2 +gast==0.4.0 +gitdb==4.0.7 +GitPython==3.1.12 +google-auth==1.22.1 +google-auth-oauthlib==0.4.2 +google-pasta==0.2.0 +grpcio==1.39.0 +gunicorn==20.0.4 +gviz-api==1.10.0 +h5py==3.1.0 +hijri-converter==2.2.3 +holidays==0.12 +horovod==0.23.0 +htmlmin==0.1.12 +huggingface-hub==0.1.2 +idna==2.10 +ImageHash==4.2.1 +imbalanced-learn==0.8.1 +importlib-metadata==3.10.0 +importlib-metadata +ipykernel==5.3.4 +ipython==7.22.0 +ipython-genutils==0.2.0 +ipywidgets==7.6.3 +isodate==0.6.0 +itsdangerous==1.1.0 +jedi==0.17.2 +Jinja2==2.11.3 +jmespath==0.10.0 +joblib==1.0.1 +joblibspark==0.3.0 +jsonschema==3.2.0 +jupyter-client==6.1.12 +jupyter-core==4.7.1 +jupyterlab-pygments==0.1.2 +jupyterlab-widgets==1.0.0 +keras==2.8.0 +Keras-Preprocessing==1.1.2 +kiwisolver==1.3.1 +koalas==1.8.2 +korean-lunar-calendar==0.2.1 +langcodes==3.3.0 +libclang==13.0.0 +lightgbm==3.3.2 +llvmlite==0.38.0 +LunarCalendar==0.0.9 +Mako==1.1.3 +Markdown==3.3.3 +MarkupSafe==2.0.1 +matplotlib==3.4.2 +missingno==0.5.1 +mistune==0.8.4 +#mleap==0.18.1 +mleap +mlflow-skinny==1.24.0 +multimethod==1.7 +murmurhash==1.0.5 +nbclient==0.5.3 +nbconvert==6.0.7 +nbformat==5.1.3 +nest-asyncio==1.5.1 +networkx==2.5 +nltk==3.6.1 +notebook==6.3.0 +numba==0.55.1 +numpy==1.20.1 +oauthlib==3.1.0 +opt-einsum==3.3.0 +packaging==21.3 +pandas==1.2.4 +pandas-profiling==3.1.0 +pandocfilters==1.4.3 +paramiko==2.7.2 +parso==0.7.0 +pathy==0.6.0 +patsy==0.5.1 +petastorm==0.11.4 +pexpect==4.8.0 +phik==0.12.0 +pickleshare==0.7.5 +Pillow==8.2.0 +plotly==5.5.0 +pmdarima==1.8.4 +preshed==3.0.5 +prompt-toolkit==3.0.17 +prophet==1.0.1 +protobuf==3.17.2 +psutil==5.8.0 +psycopg2==2.8.5 +ptyprocess==0.7.0 +pyarrow==4.0.0 +pyasn1==0.4.8 +pyasn1-modules==0.2.8 +pybind11==2.9.1 +pycparser==2.20 +pydantic==1.8.2 +Pygments==2.8.1 +PyGObject==3.36.0 +PyMeeus==0.5.11 +PyNaCl==1.4.0 +pyodbc==4.0.30 +pyparsing==2.4.7 +pyrsistent==0.17.3 +pystan==2.19.1.1 +python-apt +python-dateutil==2.8.1 +python-editor==1.0.4 +python-engineio==4.3.0 +python-socketio==5.4.1 +pytz==2020.5 +PyWavelets==1.1.1 +PyYAML==5.4.1 +pyzmq==20.0.0 +regex==2021.4.4 +requests==2.25.1 +requests-oauthlib==1.3.0 +requests-unixsocket==0.2.0 +rsa==4.7.2 +s3transfer==0.3.7 +sacremoses==0.0.46 +scikit-learn==0.24.1 +scipy==1.6.2 +seaborn==0.11.1 +Send2Trash==1.5.0 +setuptools-git==1.2 +shap==0.40.0 +simplejson==3.17.2 +six==1.15.0 +slicer==0.0.7 +smart-open==5.2.0 +smmap==3.0.5 +spacy==3.2.1 +spacy-legacy==3.0.8 +spacy-loggers==1.0.1 +spark-tensorflow-distributor==1.0.0 +sqlparse==0.4.1 +srsly==2.4.1 +ssh-import-id==5.10 +statsmodels==0.12.2 +tabulate==0.8.7 +tangled-up-in-unicode==0.1.0 +tenacity==6.2.0 +tensorboard==2.8.0 +tensorboard-data-server==0.6.1 +tensorboard-plugin-profile==2.5.0 +tensorboard-plugin-wit==1.8.1 +tensorflow==2.8.0 +tensorflow-estimator==2.8.0 +tensorflow-io-gcs-filesystem==0.24.0 +termcolor==1.1.0 +terminado==0.9.4 +testpath==0.4.4 +tf-estimator-nightly==2.8.0.dev2021122109 +thinc==8.0.12 +threadpoolctl==2.1.0 +tokenizers==0.10.3 +#torch==1.10.2+cu111 +torch==1.10.2 +#torchvision==0.11.3+cu111 +torchvision==0.11.3 +tornado==6.1 +tqdm==4.59.0 +traitlets==5.0.5 +transformers==4.16.2 +typer==0.3.2 +typing-extensions==3.7.4.3 +ujson==4.0.2 +unattended-upgrades==0.1 +urllib3==1.25.11 +virtualenv==20.4.1 +visions==0.7.4 +wasabi==0.8.2 +wcwidth==0.2.5 +webencodings==0.5.1 +websocket-client==0.57.0 +Werkzeug==1.0.1 +widgetsnbextension==3.5.1 +wrapt==1.12.1 +xgboost==1.5.2 +zipp==3.4.1 diff --git a/docker/databricks/scripts/monit b/docker/databricks/scripts/monit new file mode 100644 index 00000000000..1b1ed4d4ee3 --- /dev/null +++ b/docker/databricks/scripts/monit @@ -0,0 +1,118 @@ +#!/bin/sh +# Copyright (c) 2022, NVIDIA CORPORATION. +# +# 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. + +### BEGIN INIT INFO +# Provides: monit +# Required-Start: $remote_fs +# Required-Stop: $remote_fs +# Should-Start: $all +# Should-Stop: $all +# Default-Start: 2 3 4 5 +# Default-Stop: 0 1 6 +# Short-Description: service and resource monitoring daemon +# Description: monit is a utility for managing and monitoring +# processes, programs, files, directories and filesystems +# on a Unix system. Monit conducts automatic maintenance +# and repair and can execute meaningful causal actions +# in error situations. +### END INIT INFO + +set -e + +. /lib/lsb/init-functions + +DAEMON=/usr/bin/monit +CONFIG=/etc/monit/monitrc +NAME=monit +DESC="daemon monitor" +MONIT_OPTS= +PID="/run/$NAME.pid" + +# Check if DAEMON binary exist +[ -f $DAEMON ] || exit 0 + +[ -f "/etc/default/$NAME" ] && . /etc/default/$NAME + +MONIT_OPTS="-c $CONFIG $MONIT_OPTS" + +monit_not_configured () { + if [ "$1" != "stop" ] + then + printf "\tplease configure $NAME and then edit /etc/default/$NAME\n" + printf "\tand set the \"START\" variable to \"yes\" in order to allow\n" + printf "\t$NAME to start\n" + fi + exit 0 +} + +monit_checks () { + # Check if START variable is set to "yes", if not we exit. + if [ "$START" != "yes" ] + then + monit_not_configured $1 + fi +} + +case "$1" in + start) + log_daemon_msg "Starting $DESC" "$NAME" + monit_checks $1 + if start-stop-daemon --start --quiet --oknodo --pidfile $PID --exec $DAEMON -- $MONIT_OPTS 1>/dev/null + then + log_end_msg 0 + else + log_end_msg 1 + fi + ;; + stop) + log_daemon_msg "Stopping $DESC" "$NAME" + if start-stop-daemon --retry TERM/5/KILL/5 --oknodo --stop --quiet --pidfile $PID 1>/dev/null + then + log_end_msg 0 + else + log_end_msg 1 + fi + ;; + reload) + log_daemon_msg "Reloading $DESC configuration" "$NAME" + if start-stop-daemon --stop --signal HUP --quiet --oknodo --pidfile $PID --exec $DAEMON -- $MONIT_OPTS 1>/dev/null + then + log_end_msg 0 + else + log_end_msg 1 + fi + ;; + restart|force-reload) + log_daemon_msg "Restarting $DESC" "$NAME" + start-stop-daemon --retry TERM/5/KILL/5 --oknodo --stop --quiet --pidfile $PID 1>/dev/null + if start-stop-daemon --start --quiet --oknodo --pidfile $PID --exec $DAEMON -- $MONIT_OPTS 1>/dev/null + then + log_end_msg 0 + else + log_end_msg 1 + fi + ;; + syntax) + $DAEMON $MONIT_OPTS -t + ;; + status) + status_of_proc -p $PID $DAEMON $NAME + ;; + *) + log_action_msg "Usage: /etc/init.d/$NAME {start|stop|reload|restart|force-reload|syntax|status}" + ;; +esac + +exit 0 diff --git a/docker/databricks/webterminal/setup_ttyd_daemon.sh b/docker/databricks/webterminal/setup_ttyd_daemon.sh new file mode 100644 index 00000000000..a7e41cb3712 --- /dev/null +++ b/docker/databricks/webterminal/setup_ttyd_daemon.sh @@ -0,0 +1,37 @@ +#!/bin/bash +# Copyright (c) 2022, NVIDIA CORPORATION. +# +# 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. + + +# source spark-env.sh to load SPARK_LOCAL_DIRS environment variables. spark-env.sh may +# not be available under some situation like DriverLocalTest. +SPARK_ENV_PATH="/databricks/spark/conf/spark-env.sh" +if [[ -f SPARK_ENV_PATH ]]; then + source SPARK_ENV_PATH +fi + +TTYD_DAEMON_FILE=/etc/monit/conf.d/ttyd-daemon-not-active + +if [[ -f "$TTYD_DAEMON_FILE" && "$DISABLE_WEB_TERMINAL" != true ]]; then + ln -s $TTYD_DAEMON_FILE /etc/monit/conf.d/ttyd-daemon.cfg + + if [[ -z $(pgrep -f /databricks/spark/scripts/ttyd/ttyd) ]]; then + bash /databricks/spark/scripts/ttyd/start_ttyd_daemon.sh + else + echo "not starting ttyd because ttyd process exists" + fi + /etc/init.d/monit reload +else + echo "ttyd-daemon.cfg does not exist" +fi diff --git a/docker/databricks/webterminal/start_ttyd_daemon.sh b/docker/databricks/webterminal/start_ttyd_daemon.sh new file mode 100644 index 00000000000..5c8dfcaba30 --- /dev/null +++ b/docker/databricks/webterminal/start_ttyd_daemon.sh @@ -0,0 +1,26 @@ +#!/bin/bash +# Copyright (c) 2022, NVIDIA CORPORATION. +# +# 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. + + +mkdir -p /databricks/driver/logs + +TTYD_BIN_FILE=/databricks/spark/scripts/ttyd/ttyd + +if [[ -f "$TTYD_BIN_FILE" ]]; then + $TTYD_BIN_FILE -p 7681 -d 7 -P 30 -t disableReconnect=true -t disableLeaveAlert=true bash --rcfile /databricks/spark/scripts/ttyd/webTerminalBashrc >> /databricks/driver/logs/ttyd_logs 2>&1 & + echo $! > /var/run/ttyd-daemon.pid +else + echo "could not find ttyd binary at /databricks/spark/scripts/ttyd/ttyd" +fi diff --git a/docker/databricks/webterminal/stop_ttyd_daemon.sh b/docker/databricks/webterminal/stop_ttyd_daemon.sh new file mode 100644 index 00000000000..5c8dfcaba30 --- /dev/null +++ b/docker/databricks/webterminal/stop_ttyd_daemon.sh @@ -0,0 +1,26 @@ +#!/bin/bash +# Copyright (c) 2022, NVIDIA CORPORATION. +# +# 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. + + +mkdir -p /databricks/driver/logs + +TTYD_BIN_FILE=/databricks/spark/scripts/ttyd/ttyd + +if [[ -f "$TTYD_BIN_FILE" ]]; then + $TTYD_BIN_FILE -p 7681 -d 7 -P 30 -t disableReconnect=true -t disableLeaveAlert=true bash --rcfile /databricks/spark/scripts/ttyd/webTerminalBashrc >> /databricks/driver/logs/ttyd_logs 2>&1 & + echo $! > /var/run/ttyd-daemon.pid +else + echo "could not find ttyd binary at /databricks/spark/scripts/ttyd/ttyd" +fi diff --git a/docker/databricks/webterminal/webTerminalBashrc b/docker/databricks/webterminal/webTerminalBashrc new file mode 100644 index 00000000000..f8797f0b464 --- /dev/null +++ b/docker/databricks/webterminal/webTerminalBashrc @@ -0,0 +1,41 @@ +# Copyright (c) 2022, NVIDIA CORPORATION. +# +# 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. + +BOLD='\033[1m' +RED='\033[0;31m' +YELLOW='\033[1;33m' +NC='\033[0m' # No Color + +echo -e " +Welcome to Databricks web terminal! Please read the following instructions: + + * This terminal session is ${RED}${BOLD}ephemeral${NC}, so it will go away if you close or refresh the browser tab. + + * If you want to have a persistent terminal session on this cluster, please use ${YELLOW}'tmux'${NC}. + + * There is an idle timeout if no client- or server-side changes are made to the terminal session. + Refreshing the tab would launch a new session. +" + +unset BOLD RED YELLOW NC + +source /etc/profile + +source ~/.bashrc + +# Unset HISTFILE so that the bash session does not write history to ~/.bash_history at exit. +# Users can still see command history within the terminal session. +unset HISTFILE + +cd /databricks/driver From 2df7c97e4567fb280f3190c18b237ec28b63cde3 Mon Sep 17 00:00:00 2001 From: Matt Ahrens Date: Wed, 12 Oct 2022 14:18:48 -0500 Subject: [PATCH 181/190] [DOC] Add gcp dataproc gpu limit [skip ci] (#6765) * Adding cluster creation troubleshooting to GCP getting started Signed-off-by: mattahrens * Updating user tools package name Signed-off-by: mattahrens * Fixing typos in NUM_GPUS and NUM_WORKERS explanation Signed-off-by: mattahrens Signed-off-by: mattahrens --- docs/get-started/getting-started-gcp.md | 25 ++++++++++++++++++++----- 1 file changed, 20 insertions(+), 5 deletions(-) diff --git a/docs/get-started/getting-started-gcp.md b/docs/get-started/getting-started-gcp.md index c9024092536..136f3e7173b 100644 --- a/docs/get-started/getting-started-gcp.md +++ b/docs/get-started/getting-started-gcp.md @@ -27,7 +27,7 @@ The advanced guide will walk through the steps to: * gcloud CLI is installed: https://cloud.google.com/sdk/docs/install * python 3.8+ -* `pip install spark-rapids-user-tools` +* `pip install spark_rapids_user_tools` ## Qualify CPU Workloads for GPU Acceleration @@ -178,8 +178,8 @@ The script below will initialize with the following: export REGION=[Your Preferred GCP Region] export GCS_BUCKET=[Your GCS Bucket] export CLUSTER_NAME=[Your Cluster Name] - export NUM_GPUS=4 - export NUM_WORKERS=5 + export NUM_GPUS=2 + export NUM_WORKERS=4 gcloud dataproc clusters create $CLUSTER_NAME \ --region=$REGION \ @@ -197,6 +197,10 @@ gcloud dataproc clusters create $CLUSTER_NAME \ --subnet=default ``` +Explanation of parameters: +* NUM_GPUS = number of GPUs to attach to each worker node in the cluster +* NUM_WORKERS = number of Spark worker nodes in the cluster + This takes around 10-15 minutes to complete. You can navigate to the Dataproc clusters tab in the Google Cloud Console to see the progress. @@ -216,7 +220,7 @@ If you'd like to further accelerate init time to 4-5 minutes, create a custom Da export GCS_BUCKET=[Your GCS Bucket] export CLUSTER_NAME=[Your Cluster Name] export NUM_GPUS=1 - export NUM_WORKERS=5 + export NUM_WORKERS=4 gcloud dataproc clusters create $CLUSTER_NAME \ --region=$REGION \ @@ -236,6 +240,10 @@ gcloud dataproc clusters create $CLUSTER_NAME \ --subnet=default ``` +Explanation of parameters: +* NUM_GPUS = number of GPUs to attach to each worker node in the cluster +* NUM_WORKERS = number of Spark worker nodes in the cluster + To change the MIG instance profile you can specify either the profile id or profile name via the metadata parameter `MIG_CGI`. Below is an example of using a profile name and a profile id. @@ -244,13 +252,20 @@ metadata parameter `MIG_CGI`. Below is an example of using a profile name and a ``` This may take around 10-15 minutes to complete. You can navigate to the Dataproc clusters tab in -the Google Cloud Console to see the progress. +the Google Cloud Console to see the progress. ![Dataproc Cluster](../img/GCP/dataproc-cluster.png) If you'd like to further accelerate init time to 4-5 minutes, create a custom Dataproc image using [this](#build-custom-dataproc-image-to-accelerate-cluster-init-time) guide. +### Cluster creation troubleshooting +If you encounter an error related to GPUs not being available because of your account quotas, please +go to this page for updating your quotas: [Quotas and limits](https://cloud.google.com/compute/quotas). + +If you encounter an error related to GPUs not available in the specific region or zone, you will +need to update the REGION or ZONE parameter in the cluster creation command. + ## Run PySpark or Scala Notebook on a Dataproc Cluster Accelerated by GPUs To use notebooks with a Dataproc cluster, click on the cluster name under the Dataproc cluster tab and navigate to the "Web Interfaces" tab. Under "Web Interfaces", click on the JupyterLab or From da94d4efad42bc169ce434072aa7907572add2ad Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Wed, 12 Oct 2022 19:42:26 -0500 Subject: [PATCH 182/190] Avoid any issues with scalar values returned by evalColumnar (#6767) Signed-off-by: Robert (Bobby) Evans Signed-off-by: Robert (Bobby) Evans --- .../nvidia/spark/rapids/GpuExpressions.scala | 6 +-- .../com/nvidia/spark/rapids/aggregate.scala | 37 ++++--------------- .../nvidia/spark/rapids/nullExpressions.scala | 3 +- 3 files changed, 11 insertions(+), 35 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala index e0d1b2fb135..881091716de 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala @@ -17,6 +17,7 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.{ast, BinaryOp, BinaryOperable, ColumnVector, DType, Scalar, UnaryOp} +import com.nvidia.spark.rapids.GpuExpressionsUtils.columnarEvalToColumn import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.shims.{ShimBinaryExpression, ShimExpression, ShimTernaryExpression, ShimUnaryExpression} @@ -353,10 +354,7 @@ trait GpuString2TrimExpression extends String2TrimExpression with GpuExpression override def columnarEval(batch: ColumnarBatch): Any = { val trim = GpuExpressionsUtils.getTrimString(trimStr) - withResourceIfAllowed(srcStr.columnarEval(batch)) { shouldBeColumn => - // We know the first parameter is not a Literal, because trim(Literal, Literal) would already - // have been optimized out - val column = shouldBeColumn.asInstanceOf[GpuColumnVector] + withResourceIfAllowed(columnarEvalToColumn(srcStr, batch)) { column => if (trim == null) { GpuColumnVector.fromNull(column.getRowCount.toInt, StringType) } else if (trim.isEmpty) { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala index 46853d542d7..96b7fe9d156 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala @@ -512,40 +512,17 @@ class GpuHashAggregateIterator( val opTime = metrics.opTime withResource(new NvtxWithMetrics("finalize agg", NvtxColor.DARK_GREEN, aggTime, opTime)) { _ => - val finalBatch = if (boundExpressions.boundFinalProjections.isDefined) { - withResource(batch) { _ => - val finalCvs = boundExpressions.boundFinalProjections.get.map { ref => - // aggregatedCb is made up of ColumnVectors - // and the final projections from the aggregates won't change that, - // so we can assume they will be vectors after we eval - ref.columnarEval(batch).asInstanceOf[GpuColumnVector] - } - new ColumnarBatch(finalCvs.toArray, finalCvs.head.getRowCount.toInt) - } - } else { - batch - } - - // If `resultCvs` empty, it means we don't have any `resultExpressions` for this - // aggregate. In these cases, the row count is the only important piece of information - // that this aggregate exec needs to report up, so it will return batches that have no columns - // but that do have a row count. If `resultCvs` is non-empty, the row counts match - // `finalBatch.numRows` since `columnarEvalToColumn` cannot change the number of rows. - val finalNumRows = finalBatch.numRows() + val finalBatch = boundExpressions.boundFinalProjections.map { exprs => + GpuProjectExec.projectAndClose(batch, exprs, NoopMetric) + }.getOrElse(batch) // Perform the last project to get the correct shape that Spark expects. Note this may // add things like literals that were not part of the aggregate into the batch. - val resultCvs = withResource(finalBatch) { _ => - boundExpressions.boundResultReferences.safeMap { ref => - // Result references can be virtually anything, we need to coerce - // them to be vectors since this is going into a ColumnarBatch - GpuExpressionsUtils.columnarEvalToColumn(ref, finalBatch) - } - } - closeOnExcept(resultCvs) { _ => - metrics.numOutputRows += finalNumRows + closeOnExcept(GpuProjectExec.projectAndClose(finalBatch, + boundExpressions.boundResultReferences, NoopMetric)) { ret => + metrics.numOutputRows += ret.numRows() metrics.numOutputBatches += 1 - new ColumnarBatch(resultCvs.toArray, finalNumRows) + ret } } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala index 8d60e36dad8..67f2dfe56d0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/nullExpressions.scala @@ -19,6 +19,7 @@ package com.nvidia.spark.rapids import scala.collection.mutable import ai.rapids.cudf.{ColumnVector, DType, Scalar} +import com.nvidia.spark.rapids.GpuExpressionsUtils.columnarEvalToColumn import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.shims.ShimExpression @@ -194,7 +195,7 @@ case class GpuAtLeastNNonNulls( var notNanVector: ColumnVector = null var nanAndNullVector: ColumnVector = null try { - cv = expr.columnarEval(batch).asInstanceOf[GpuColumnVector].getBase + cv = columnarEvalToColumn(expr, batch).getBase notNullVector = cv.isNotNull if (cv.getType == DType.FLOAT32 || cv.getType == DType.FLOAT64) { notNanVector = cv.isNotNan From 1f3214dce9ef304bfd0035486bd72552ef9c983d Mon Sep 17 00:00:00 2001 From: liyuan <84758614+nvliyuan@users.noreply.github.com> Date: Thu, 13 Oct 2022 10:31:54 +0800 Subject: [PATCH 183/190] remove group on array to 22.12 (#6772) Signed-off-by: liyuan Signed-off-by: liyuan --- docs/download.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/download.md b/docs/download.md index 509bd447f17..5609c7c5a9c 100644 --- a/docs/download.md +++ b/docs/download.md @@ -47,7 +47,6 @@ New functionality and performance improvements for this release include: * Databricks custom docker container * AQE support on Databricks * MultiThreaded Shuffle feature -* HashAggregate on Array support * Binary write support for parquet * Cast binary to string * Hive parquet table write support From a6137fc6aecbc812fd02f5b59b5d2f1e6266e5f4 Mon Sep 17 00:00:00 2001 From: liyuan <84758614+nvliyuan@users.noreply.github.com> Date: Thu, 13 Oct 2022 10:32:56 +0800 Subject: [PATCH 184/190] [Doc] update getting started guide for emr 6.8.0 release[skip ci] (#6754) * udpate getting started guide for emr 6.8.0 release Signed-off-by: liyuan * Update docs/get-started/getting-started-aws-emr.md Co-authored-by: Sameer Raheja Signed-off-by: liyuan Co-authored-by: Sameer Raheja --- docs/get-started/getting-started-aws-emr.md | 7 ++++--- docs/img/AWS-EMR/RAPIDS_EMR_GUI_1.png | Bin 120838 -> 115267 bytes docs/img/AWS-EMR/RAPIDS_EMR_GUI_5.png | Bin 176974 -> 608477 bytes 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/docs/get-started/getting-started-aws-emr.md b/docs/get-started/getting-started-aws-emr.md index a3a04678bdf..ad2c789f389 100644 --- a/docs/get-started/getting-started-aws-emr.md +++ b/docs/get-started/getting-started-aws-emr.md @@ -14,6 +14,7 @@ Different versions of EMR ship with different versions of Spark, RAPIDS Accelera | EMR | Spark | RAPIDS Accelerator jar | cuDF jar | xgboost4j-spark jar | --- | --- | --- | ---| --- | +| 6.8 | 3.3.0 | rapids-4-spark_2.12-22.06.0.jar | Bundled with rapids-4-spark | xgboost4j-spark_3.0-1.4.2-0.3.0.jar | | 6.7 | 3.2.1 | rapids-4-spark_2.12-22.02.0.jar | cudf-22.02.0-cuda11.jar | xgboost4j-spark_3.0-1.2.0-0.1.0.jar | | 6.6 | 3.2.0 | rapids-4-spark_2.12-22.02.0.jar | cudf-22.02.0-cuda11.jar | xgboost4j-spark_3.0-1.2.0-0.1.0.jar | | 6.5 | 3.1.2 | rapids-4-spark_2.12-0.4.1.jar | cudf-0.18.1-cuda10-1.jar | xgboost4j-spark_3.0-1.2.0-0.1.0.jar | @@ -79,8 +80,8 @@ detailed cluster configuration page. #### Step 1: Software Configuration and Steps -Select **emr-6.7.0** for the release, uncheck all the software options, and then check **Hadoop -3.2.1**, **Spark 3.2.1**, **Livy 0.7.1** and **JupyterEnterpriseGateway 2.1.0**. +Select **emr-6.8.0** for the release, uncheck all the software options, and then check **Hadoop +3.2.1**, **Spark 3.3.0**, **Livy 0.7.1** and **JupyterEnterpriseGateway 2.1.0**. In the "Edit software settings" field, copy and paste the configuration from the [EMR document](https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-spark-rapids.html). You can also @@ -136,7 +137,7 @@ default settings: "spark.plugins":"com.nvidia.spark.SQLPlugin", "spark.sql.sources.useV1SourceList":"", "spark.executor.resource.gpu.discoveryScript":"/usr/lib/spark/scripts/gpu/getGpusResources.sh", - "spark.submit.pyFiles":"/usr/lib/spark/jars/xgboost4j-spark_3.0-1.2.0-0.1.0.jar", + "spark.submit.pyFiles":"/usr/lib/spark/jars/xgboost4j-spark_3.0-1.4.2-0.3.0.jar", "spark.executor.extraLibraryPath":"/usr/local/cuda/targets/x86_64-linux/lib:/usr/local/cuda/extras/CUPTI/lib64:/usr/local/cuda/compat/lib:/usr/local/cuda/lib:/usr/local/cuda/lib64:/usr/lib/hadoop/lib/native:/usr/lib/hadoop-lzo/lib/native:/docker/usr/lib/hadoop/lib/native:/docker/usr/lib/hadoop-lzo/lib/native", "spark.rapids.sql.concurrentGpuTasks":"2", "spark.executor.resource.gpu.amount":"1", diff --git a/docs/img/AWS-EMR/RAPIDS_EMR_GUI_1.png b/docs/img/AWS-EMR/RAPIDS_EMR_GUI_1.png index e8a705d802b9e6f4429df3ddc742fe043552f520..c08ff36207ab9be0a09f3d4684ded35161059e53 100644 GIT binary patch literal 115267 zcmd?RbySq?+BXac2$Dm03W799x3octba!{BlnjC(0@5KN-Q7}xbV|2$cf)&*&vWm+ z?|ZNB|8KqTS~IL+;+nb6IF92N#}%q5FN5)jFeXjF*9@<80hPF_cJj*a&%LMLc^8y{eFB$ z3Nl2OM!yRE*@+YZ4=*A{ht5ud@7?g@J4y!Ic0gZ!D^ZheNxpt@+T3`zU6&nOt4bt7 zt+H}hwEqbumAnEg!&7k3SMcSry)Pe$?O;-AWua^^PI7sQ76i=Ey5(R9a6^<%GyXF{%g=-F38myzv4agok6W2%*%*87qF)R6@u|IYxUL z#yQIM0*}o9YfS!PLR?`^roOShamRPKM^4bKLEi5MKN)`bWB2m)eAj<;LYs;BoY{u+ zLHn5(_8k!DAFS_z(*?MDPe6 z+)2P44h|t5;jd>NRHZ-o*JBjei!W2Tr{Lhk;N&F5RovlsQXa(BcZ zW@flXD;!ME1qZnVN9V@J$M<(qi%JIawjSLV@;lY^A${086zD&1 zPpFg%^br61Gs$#RN=X%ZDI}~2G*th%DHSaI<14_w{{Q!ULNPC$<$k-QWmZA-_V-$U z(t3gUY;BpNn$@T<zc?7_X(Yc0WSHj&%vNZULRgU9|! ziu32~;mnW%-6qdeBcv(yO7p0O>%E4#E=s?hlBzBWuUaj;rg7?A4~M2@>|rDeE4k$3 zLDBo#C#mjjABnksuxOM&UZ87m-dvU!+9Ala0Rr z`zjJU#O-FdjUVt?X!e=wj-vH>l7{IUM!=%|k%Zg(<^A2Y9h*BCD#s8wrb6MfNu#}{ z^ZDb6Y2)bE+Pl+LKCxJ$H>H9b>4CHF;2%8l3QgwsyF6Om3qWJF#}c{Fe?EkRI_KES zg8Ljz$z#@`i!NiG&cmtY?y7*&UTo;dOI)StdGI4!aHM8p$rz{c=Bq8H`|BU+;*&($ ztrv4~p?_cSF+%>~G*w`n8k3$`e72)QDA20074hthp!B_Hm08vGyLAacabF51oE?^- z)X5d}+^w3pxs20sO$fum3(gnnl5aj}L!0YM;PNR5eebqEKR2HLw%3kpQta8u7cJYv z2XO1jP7#r7UySjtKl08W*ciME+{X|PtFoFHfDPwv<*2-GL*XsoPH8(?K(kyly~*w` zXVJecjZe6UoH$ju^sf$ht9Gkr>Ww0Wc?L0XI2yO|3mOled;6RW%LEft5!^0&u1<;E z9JB|e(>?O-fXMuA71!KZ3ZdNF`c<@D+<5eYtz6{tH*U>-^Yt2|S~6)?wo#F0)e98O zYC*TX9X)T>upe*%uMj20{^OQ@f@1vGa(y6WTPxABnb`?_zhPgb#(t^I+pUvKV{O^@ zdbcK`o`lDG1G`Ytxn|NZOwwZ@P1MhqCOg`rDvQFdX3ljl$*%bj?XgmPF30nF-`gwQ z?@brWVIrPWC3-EAI^O#CIcdIIvIfl?8DU3cv?hdQ&Bp_R*WFwX)rvG%p)Xp%f4=@t z*WP^IYM9XVxvWkxbJ;F;@{O<%BP$eU5Z+X8sp_e8FDR(R%|-wdWgJru`2Bcpr8zfKdG z%L#A4`~CpS*=w)PtdvAFC$ zz5@TcSD8&SE>TqeT@O>q1$@NW?#z>cuVfDhyxH`;4}+70Jj&Uy2-gsgeZU^r*G+E^ zTS#iyt7kfD$Y$LO6M!Zjk#}C|E_-*~U}6z-cChF+#`o67GO1lH@QQHND(1+OFQ^i5 z7&Ws$5EQ5N?0<)7ZT@EaSJ7>wmhS>Fl-+2+U3=a&VqWS$XHN|A?qz!0Rb5Hhla_h+ zm5;WwwGMZ$XaYhgJk6~mR=Oh3+t?awW8S`l6FBOoy`@)@Q60jV0Y7hlee(S|ZMj{; zE)Ur0>z5MHWPJe!nm$P$n%=cF^1D~g*p16JF?e8Oh8<% zY5=R}Rv*2KYl-}A+S%WNh?occC#9?}DAA$Qt$`04uJ>COAmqm56XdRwtuoIu^~;sn zh6OX6Bx|rcvhrwj)%%GK-!V`*$~qr^OR_zB{=uB+J<)QMI0_l!EO&uf*^y3UE#z0yit84^Q%NiJ*D5xFdE0& znWMt})kI69y;LL4YkImG%_>Wetw*-#WPI$@<{*}OFHq3r6z;r|Y+zuSI>%WTq<8Gw zU!TzP+szy8cG)*F+FlK2>2q7duJQJfKG|Xym<-r$ruz5Yo9@;2w2~&_*+tm zF+;dddq5nBknv$mQCXXVMg@h22I@R7C;UxW++b_Rf3Qm5bpoz|^HO%r(q!!~S=v_mq7z}#&JRHw7{FzC3 zE>m7}e$5PN8q&I$3q4s>&I~BJMYtx?vv=ohe8GbY>NfMe) zh-?>D+*G5I@Q7S2K4=+KhE6SdWJr;$m#%ugQ1f=ua;N969fqRc0FCMc|2%E6%8F-{_G z!d;bkW$#0P`Me_`S1OC9HWY7!LiG{-94;3}pBX_34Z_c5SxJx39-3dH2~wd&v*E;C zH!=fN=DFl(&7-Qy{_TbfJ?wU);Ygw8dQVs8zp(0T5e8&1=e4PqkC{mFB1Bi(jI;*Q z@VM{hT1kgq=Q_6f5OC9b62FSB=4=(mnNf>BKRAD`gLE6QQrYg#Wf#{MR2rouHZ8O~ z-=mqZylCUHt@9sQSPZe}ssCh#>wKe!NO>qWH9oI7irmO+4-Vc~{$waOqKV=V$2oV8 zx=MFAzA2hzWgxy^sPE^1*>aPYX`U|;!zsbs}C7-Kg4%{k)>p3z!1r_2}re80QPoE8<%GqT6A zU(Vi75q3G*CJ2Td>PTsb)12mYxLkNGC4$q*oF<*`+Q%cHGcRIh5-_4`ohi<9`x~5K zC_pp}(^QS2TO~4*x3+g#q9UWndmfTHAUynH&1jmZjnA?;Tlgu|6KaqDL)JoS>@(}D z{AZ>GBb=f*N-w9M*tBATI`Dn~iy*c<*86RyO$&+ z(t|psF8xRH+pw$%GJeiMD)CzR4ss`M4n(0J=ZJa@dM#WB4eLxR`kc+SkOiiHX+mskz4)XVm(k2%pz1nYe2}?MGOJx00<=4x5O2Uzh{ zfeaPoXsNy6gC7J=C$=z}wYu9{qCNIr?D-wx;YpOr9PgqSrL-m+{QS|JMeMR>EktY3(TA;MuXKJ@e+-x%LB z&39IZElwF3rtlWUL^_t~5)W{!lNeBPaDOMKWrN{<9^)tC}iv zk}m8mp@2@wuh}Ql9!mRtl-2bf$ZW^X{THt!BV~w$eH+Pa$7} zX(P$nu=nW~Mk^3w>aC`gWp+U|ew6DePBUCtn2-VWimA#Y*I{ z&eKWOH7TtRoJwjG__3k*zXhvREXwiSvpJQ=cfHm%s(5M=oqX>u&F;w$685>`>As63 z{4Kvg#nuoFb12_%N9r2{YttAhh+Jl^I6TWLq`}m{{)za=#DkznH~`sU=Bva*(Fv!% zc#b{M`^(4~%Yq`ZLH#O^;3BP>vjTc2TY(CX>;<3m6eDV?;`R@HrcflUQ$lgPOJy{Ij0{2fGxS>9xD-(;Xf zc#zw{ki)ljEzyc>FlV&^z&J0Oro@+y3cpx!qx-~7^aI8$CtCn)5qV8s?9DctiNTj; z<@#-bgUNMfpLD`7+u9dP7O6&M<$xP9?OigvV3CuuHuE7KlNDVmA^kh%#8;cYZD?$N zcSJ)O%e;tzn}elx9|7jH&EKO+X?zZb&xdH_J?5&CE7p=|~@QGl`GQ15J;urU<=bux3!+rs7&2o6EgT;dlB*SPv^> zM5XzsBWrnwJDV8-k!ZrlZ8Ns`LJ@EUXY?#X6A2bPB9flMLn(lN-Bz};n3w^p(R}4nG*VHi^fjo^ zNI49H6eM^lP+pIu%W=`4dQFxZxXSMT?F<1uge@nQ!Gja#O7O)0je_uaj!U$zwNo4+ zv-)DZov_D|u^+6Qq#yYhR>6|5^|w4HNmYX?JGORq9v59?cDfF#6u~kol>>U&&Ari_ zNPu^=>}y2+NnD9Nh7X;VWuDfUMU>?>_2r&ZN9Y&tdKWB#uK@N;Dy;SCd|9)jk9vp9`g}O@N@6!TMB^4tW z>~K zwHuJC)c*R3>F+>EG2L>1H`AXY$h%Q`K-$VYgp}8qKNwJTuIh$45Ha zg8#F^d${3)nAxm*S!$f2GcdAeUt;pb?3ZmlthvTNQzOPtjjc7ts@phux-m?P(~C@M zy?AxHy)NLkN9G|^i|b&0DsnPNwFW@>bqN>&zN%QYNV6CaQ5L=T zfQ7SN?TIZm?N7Am&Hu9|VhH?8(}h9J?em5eFk$z$hH7oqmCe*Tf1S~b_hL5(mg&NR zJEei!V|mfrLtg`D%rifdJ$G6B?jIF3``zM7T@ul@=~Ok%ux9B!X?+B_V7YtvTZ9|Ykr%YOD zU$@BNL-5z4XCQ?I^}rghX^Wh?6u0@Q{j1HKwAs)nsT-n$$^4_-R=@fn_>BM;&jq34 zHomi*sV;&+nHfOj1&luy86~b;{h~#-K9Q@`z6+MY-shLB4p}@#MnA$GpcUCKjtVgQdM#HC>0^FD%EK7R;{G<8A?twFM1u)xzt@$G3-gc=8&`YaKC4bhPq9!8qBddv; zv6a%?RX!8^B5q~R|BVR#-U-IdEp1y^Vm{+AD6@a|Hb85lyvtAo3pI zb5Anj5Zd2k97nxCWN$-@67)PVXA9sLF(0Dhb^I;MW?rC@w`|%Mzr+gIKVA_1F$t$r z{q8&_igg6l8eDBAvR@LO1Ip@3U{ig!yqEQVN)@v-pPR##NYVNJG|>few^;|@B&65o zOz+J_PsiTO0fug~*J;ooaN#xY+;1tC=dL*^t}oCjREjjYwP#t>OTX`D&fC=*w}PIz;4c>K1p7zp0O1a#cqAq0ltAkLst+MZ>e6rga2VHO)smY9+OUY zI;pEP0mW%hTCMz7XrVes!IK@sk)qp;Ju$-7L*bz*D`~#UCrp>L)&tBd8{gC*+3SpK z15yF!+y$w}5Du%&VyH<*By_&0W_~(sxnaL~M#h96{_w?bm!25LJ8zmWNn(f(pd{14 z7=0FX1Ccs{zhzM5x^Si9<1f*xjm*2owi|aq;%bA@5`LJ`3LUsTUswqH3F6l1>tw#+ z6z`*-BGYn}&SSYszLM1#3BU8Kc(4oJQyTMawqy*Fkr4er*G>~OjsT`kpb7avE#BQi-(9}{R< zFO9k-IEByo-QV6(T)ZOTK9_Xxz%}WT&)^2Y6vp!>oTIP!{dGT%Y}> z^F}?1j?Wf>hr}E+%|2fDFVqz3R}A2-jxBZ_M3mNZ1PI>h{<}7D3=#8}+y#qxX*~-k z!Ps^kjjeH!)<4S^64i;_C@IT9^O*?JU8?bdGp|oIwVN3PW9~G8n;{9TOB+Pmyv8h^ zBG2LLbE!U-)=$cna!%q{ zKLrPPE%XB#V}kBrE1l0SEg9HdOnfdU!1cphz4-yU|4gHY6I;w6pcq>QeX&oahssOw zj#FD;5`;YeNJ=I%pJJ;6L2S=&hu2Cjuo6J*lKR~zZb+}7X}~W@TLTMr`_Q!i0A_HA zj~-iMkY_|+mO?eH2a@?k=3-G8nbtKwut7A}5)&CJjy+@!e9!0HM1s#V!dQ<&WF_c5 ze>TcXa=Er55nib1D1Q#j_#(?NJW){6JUw2bw=l>-*FQxdcl4y>-T@5zRi(~If?OEj zh0DmF4mE^|1~H?zhz$4<1bS!s3KGMHyWF3{IlW14xc_)95k7A%%jcOKdq zaoV{=sT!!g5%j*GxW1=Eg&gn76*CZXDVO=!QDaGy9=4PnP!WS!*0H~`JQaWGqokyk zq}k>mg4=KMX_VQ0_H6TCEJwOEHbYDoS^CumgsCtz^S@QQtN|?w*{(~5v0TGrL9MR zB|eMN_Yz^DN%E+xCKr0BOX+r&nNz60I|R%-b_Lwq%x zDgMK ze(<&wUrfehrO+WmfDN`4W$63c{(*B1k^&I01TyhGJbMSX#Q?zBtmQJu!7~EJ-PreJ`XiU@%I`f= z3#AybvspAN54t>0w^l*=`ai2L80f|%PMY;~@PF%1MORc6bvA)}zu1g&ue9KLu8b1sxUn?Jv%%pis7tw{ zRNVrF1zy#K>V?2Bk&Jbv$UD@ybq7gwI7K&mUAx-kKg?u$DdNcGMD3~8Q%)xfNCnRj z;kkG0zsmP+9fUI4syeT*&IQNRP_54aUjz;BY{sXo{raJJ#IGs zpA(gONj2_?B;`dn{%51Yo>Wu?z~IZco$5K=Utjl^*uVPj-AH6_Yh0lg-d`o+|6&gR za)3dq3w|9|>nY9}+qwFf98f6?j~8i8 zz-s34TqRl)4RY8Z`hTXU@>3n@a1jQNm#L`qRLNA{=76kF49Pth^b8v{PhR`|Gg{|( zt$&U_$$FV*Qa>;z=N1$!w!;}xwgA@@zfR)8sb;j;Ob4Hm!}%o2BsX3E1w3|ud}3o_ zEzbvlf@%V#gP`h7j{IrV#Cr@dIQj{znPHWiU#A7GNUGPP`+cZWaSv3C2baM{4>`%$hbscINY!3a zd!?3eg)kixntddZFqT;k)^4W))mgHpa>R;rr782p+(&o62rr9egayxmrP+*93SrOM zoK(-JdT&>Ld~_nqTzQKvG30o6zTl_#X%x)|6Z1aijl2W7pzBFVw7kI2+ky8;B{;J>fFIzkUJVFBD9}0}F{04Q~5T+G#q3y4O_2Y}fYFIOiP~b$gkc z_q$KcpqaVNtq^GAb7t&EeY6H)AcavIeUC8!Kg4I4i=m; zEw-0Di<9|iHrb$VV-JR>Lp@>D3~$kH{m$%ATD0w*P2QXjfp+}PJ^(3$5cn3L=y*R{ z0?FT%P{%pSYco6UsOl3YP1_03l3W4AJZ4;ixk4hvl0CQbE_()3g=d;B*OK-CByi<2 z8*r9_a16RZcCr1*XtBG}q~>r$N*QjZz-r#VyV@=higRD|e>e-o)>T1056qaO9}$9k z8vq_3AygTTWXahcEVdQ{*?F(D9n)zvN8vo98hk0HI7u}yt&I8HU9Rt)PUsCj@!r%2 zVCd3zyQs$X0W!M1@rI%s%!vXjWRl{Zs02=creq#(gRgZP8-H%&&lG)a(xkA^di*o0i?uc4YB+r^A*?cKB-&q06?uIad#PoQ5&`lx))5UAQ zsb;t5IJF0Bl`$6s3~M0`pbfrRamzb>kIzHR;dfek!E3i^Ig#c2GpFA7u#<2PU?r!l z3jA%8NL2xU-u zAE)O#6{*Z)J3D#0Ioh=gByNx5cI4v-hau-afHgKF0hO_|SJp*woKk-v)&>C5_Wt_s zXoWKqtxEGHU5{1#E@7!q&AX+;X@H>TfYE4Ua2TX`{}DR<<_^un)`yht;$a6X`;@c@ zT=t7%ybjBfqW$UMO#e7=nA_YyvDu*Kd)XOx4Gv58hTwqBc|QMb!P@c#^#}rF!l*0a z6v2Ky#m!Ys$0at+beZXH0ZSR}P)3H#A;{>{4nW{%%SYf;9Yk+WqOCw#)NuHNv}qDO zwaW~7%@Pq@&a&^Pf?Q_yrj@8(nNa03XUXAo$LZ*u;ol6-|; z96=|IxtvJ?hmmwD;qn-ImpfShR~}lDf?)YMQ}%n`9CrCYMm+iDyFLvNZ`$)|b0JbK zV@FK_ufctSygHC-y^ZPS9M)`?o`qmh?gc-dSk!dpv2dFtd1k&SSmAuehc*1dECSP7 zeaMkD;$qsWI`LH=vVFxbl3qDGh(g!z2VztK#>FiGPg(6LWhK^x| zM_%e)rW)yahCtd);~4hI_+0_0)ti+h`D80hw)}~GtusIJ93=rp-cj-y)H9y$Si<=I zBB4jkD|}AP0iokpY__{M#``F?Nlv}bpRS3vuO7Z$3f{C%8enbaEmn#OQZJXP zDNLT1N-xBXf40uc>pQJn<9(`GbN%ad?h6JF^Ki2USu;?`p@UcY#X`qBfMv2;@G}ri zphcgB1k+7^e@7muXWSHAl-~R<9eEq+lZQ;HIH>v*t5G`km2UQ%bNcm`SUn6sEL>Th zadtiMm-f55j~ZFVtJID`dM?km6m(y;)?z2gkm|8U+|nvb<@6@heVHm$?rp zw?8uqBqJUA`bcTVNvPRCz1vNW(3i76QIcjP@W2ozh=l$COh?Tf*YS2gk#*(@uh{@u zOPIJIZspGRNF^+hh+MWWeOOZ&?re<*0g(q+ZTHu$nKSM`+@H>v#OdIKBJryt>y2iN0YWYw%jZ*-z>;=z7#`Y&&EoF@__l zKQvIz?`|d{objbfOug@eh|c7rEi`rG=sVJhFEH>Rj6jU__HCg$wfIsU>hVjJAxDE` zGK;6JYV716q0GamK`i8{z$*C^vbpFza?}P;?m?qUoJrp4Cg zvXm!O!l$8kzHyIvxO@Wh1i8G5XK!r@@uOM{-TS2ZDk$i(oD+oTS(@){is| z(lvfZb$;BDdK^0tFZ_dPr>k#$vTn@my3qDmLKT_X=JZ(p_9E7q;3Ofh2S->s{NYE& z#}|{Yq*z>*Mt6se$1IrUonFx+tR{#B-|r{Zjcg=!nhdIh0;a6*yc4@h(xM+YTGuwt zu|uUUM*PCc11=DhQ+`&@xGne`Uv}nNnZwA7JxI6uShcnKC?_bM!OP4Qb0tK z|N4A?A7t-Zp+PChXYvW0bc6YRAZM6lct)%!vXHdFuUSPQx0gA0710P3h_57mktUiQ z6w=Dz$XbfmFn;iajECT(cq!LHoRscoDaZD83!o8Ed7(6(&wJfLi27o)qX~J!DvvfO zzbPNkNCZp85lyy3t2t%O-bc1FZ?&!)`*yEoPyk_QQGVK9r{KU`|}M^%39_OHPN;cA@~#3*Y><+E-?YQrdgi>wt$Q>Xay zP=xTA`a-Lr0j_|S&=#_TxpX~+QV>&dZ5!K;WLK2zqig%p%?K^wLi8Sn$7m#_?C>^A zWdsIRfJrY&iZPnzw7Shu$a-v8U5iStMPShZvHU#6;qi%Jk2E3fq8=|dOY)%Mv(xua;`99KZ(40s%ipy*yC8`jD*IQ!5_P> z7eg}2H``3ASP*DFHNo&M-P`bCZUN27+nn7EP2RlSeAVI&`{bYDPMoi~UaG1k-%3(! z#2@B942K8{sAi>Lcdqi~HK1r{*?(A_*Vv7BT~>j7btbSu8UP^WQU-2{)?b<2Y#1+` z->miR2M<1e^6}E8MaX0p8Eq=ZG&`W$yKZpNmneDM&RO`(iyJx z+Ek6JiahVr=5~s3wH<5d$3kNz!hI_bqjn$3P{rL42Smzi+dBwy%1tT+!7uzEQ%kM0 z`4UQL81WSH2zmcQ@4?T$A`@2ar^96DSAD&7hUAUwwd+8OjRBbGcki;jAE>ZzUmP$K z;diNZ6Lx&f=cG{-LFtGX<1`C>%fUf|NKp3reW;R{fL7M%^oQ?1dk9W_5smq5I@i8V zRbkga`&GW!9!5ELIdmuVDMZJ=KDF71N5Lca*FG}a4APpc31-YbmVmCN>tm+r;mut9 zcHZs#Ensk;`m+u&_a-2cT$ZcMCS*wUa7Hp3rNSs3pChEa+^u+&7&e45DHbMM8u?H- zDwRwEjz<}pxDv*3PS#Zei-+Y`#4>$CQO*H1h4U?YFA*L6#fq=F%EbZ-;6f65v4TU* zwO~<4``qNL2pds+XnDrk)1Q14l&SAfI;gP)Z~5d+p3?r4v;nx7PL672Jo06^1B@5= zqdxxU-z*x?*6|@C;%|=DKkqW;224`no#z>Ie-*&$ycr(;7y0)W-2>1MJviRD zUL?uC=&HZ?BbrYifad;GK;U0w)4yLC2gj?p+E!ZmH(m9Y$gB+cl>1srrt807DNh8) zQ}q+J{BI#%>?uO5GC4Y>R-I#ZlAv3$7Eob;uV3fFK6Mv#W7LD(RuAOZI+#Yx{|%rv z5I}7Zj?A)eF*dHim`RC?H5W24s zqYJBqhJ{&yv?k-KRb?4F?6LXi{;-QOq8_v&ECUHUx!66AQf79dKy7e5@68hhEtuf+ zg;`@6<~>*Y`aD>utBv&LShLZ+jEz);z|3oNB)b`7v)IU`^`fNZ&JmC_l{#_@pxb0O z@@clR?8aS_=F-V@-17hm<7UfD52Icl2F0GGf95gzc z1a=*koJcL7VQ}q|B9-9Z6*eozL#w6If1`AyzIoZh0hxaF(TT4n7ei(~QwEFIVk9f0 zSU}Gw&HE@62z}tN%U(Vr?6_$M2{; zF!+%|6!UUV;;l}%C)m6h9It1c23;iHqlXH8OQHE>gZN@JO&A#4@9ZejuutpPQ@uVQ zPJp4=f=SHNHFo=$0vl8lRaR=eCOt9T^^lzQ5Ug_(STn>zn)`BSclnO-cNm+q2jn{Y z7}1Nxz&VgV6Du1r!5+fH@*g=dag|Z>4fg@uN;aG0FAcLgAhZ7R1APS*bE68PHaXwC zu~Os8^%toE9d+uR9RNBv(6>8^{COn}<50FI%ZySdfU20>m~l_8q{qNQgOBZR_-|rH zrs}iPNPo*m0k)(wAE0ROWyk3{0?k#Phnq=N%l!Ogb5w`SjI8hg5KPWMmvzpgIj%4p zbhT@~{B{kTaj?b|9VMBSz};ap2*V37gM$Ea^mQzb)hzgea$!%v4QSB~bFMS}3x=}{ z76KILvXISMz}om28v|?49zbetV7v{R!y!RuFtr`j)oYqHw!0$tH%@NAAbpLp13F_; z_Lavj0e4VqRb2O@Bq`9C3~BcW3Cu%U0VP&U#nDg5@ z*@?nEK+LZHF0vG7$(Xb&lo}7V&h#cgjT0~Pi~^|Q&3iB@EAr?P*YM^$ha<*_Qy=h@ z3(NS3(`K}ZVoO6H(?BYAI2HoC5sM@s2{(AsB=@z@I`hs)MScVG7a zl^1p2sGM3DNTVo^I##SRgGIohZ#>1;SkWJbNxb~I#f}EkZO%o_P#^mq+gFLl+Sr#W zKGgZ326zf)8ekm~6CV{oE%C-{w<@kh7nlUdL~Odq-JAj5d0v~lAU(7!VPORP5#&l3 z*2^8CcRHf5;*`)oU5`sowZ-?=8(TV(Vz(Kf!Im^%(m-oT-kFj?2L9ilBxZq_6tz$0 zx=2VY+0Kh}Evp)zr<_F^&5zTieELxx3xr2*7}a)c)m91os)w9@Vv=p7&_qm!1J!dU z9oQ8_%R+Iv%`64wsL{iI0e{47itDsRvE_LFT`KnPN?uEqm?&xMn~TFez*y8+MmLWV zuTef!BD||(U*}EWu=MyrYb%xTeZjLS87)~P z-!e4lJKdw+g$)X?ov0Gma%T3mt)$`B$4>7B+JD@EY3CZWy@52d_JEnSov3wK>7agT zWm@G0glw@(TI}(bDbSd9Y;tjkh!KEEbPIUChJJ-e{SSy1KTlFNWRND-zq5VzXj5S? z@9stc)9To`aoj0qCqK@c(GoJ;l6fEOmvsYFy><6;naMH2N4Ud)Si$t?q_tdGq^-A5 z`Unz-N0b2YfKax6_s7|k8snsaqC9>LRCp|BNyYWPgq2|Ci8|2m;&T#RXNRIEeTeVxR=_NCGyFG z;Sry+DKp=>2r#QZ_t^$0VgA{YRA$g*H`RF_Ed7yWmA2|hvi)-hT}>B2x!?te)!|fy zBPI?{(YCch$!P6BH&d?OMo^s%p8^?U9=npXCMDCXMi03Oc2lcVJa)af_)FqqN>?aK zmszUE`}Q&$5VsU!OemNNaGfZF@vE4PI~lfsn;UR_mPhGM>*YJo;KcT871r z!|1~)ITvJ;`Vn%kjjth%V%6rbl4YOv zxwEs}rBVntxi~qYe;cF}#cHy~x?#6Ev}^6=dg0zR>pGePhncXk93xjFk+LgZLOLj} z&dz|AJ7HWdK|$dc>H}Z&>N0(>^Dw)6zQveDusL0~D=p<7!^}PD);Q3ug^s#WXWzg( z3?{}3`bTWv%^7f%Dt1T&lSu!8VQKNBtQ>CM4_2Y-;{sHiBAzIPxoV;Zg9*`~*u@itUrRRtl8wQ0 zZ%(W<{$uA#_ZZ>_ubhuE)DPEzd3B?&WzY$E~H)0Rvv7!OomXEzpU$8Vmh=mLU; z8(tOH)ILl~$@k_^=H6{!f%fhl{FAl8&*J5va=J1bSl+gd`ljD6DH@L^eTFI$*`h&E zLEGLzq2zDh3^?Q)3*%1*P&69qSK?IJ@$g(pBiJ`HPhN!G)6qqCVQG-Muao_;{NUlk zlkS+!i==6EHc=6rxNQ!YrLx_!UBws|e3CPA+?hb7&;?4 zfSS|$Lzq`Rf?p3s)+?U3D}L0H){Icw-e2^`vOG}~x+*H>0A*&h85)i{u|Sk5Gli5H zrZ}m#T)a6FZ4jT7^d4V`6=;A9VmAYz6FYi8Dt)+iOCiC-T^rd@WWSPd|-X_=RzB1Mg<7DeOYpbV8Y zHZedh_B`9!4C6?jsIk+dl&kW%7DAvkH!cq<^1Hvg6FTc065l4%3cVGV5IdtLubCI8Pq57O@Wuz6>n#o&d+GQy<+}TL6!H%;dkmtpVGzC zh*O5YLo$pG7R~JQwy2FrZwV7%Q%{xo3c1mz8VXMj5f0(qz}l^CdZSRvI#U8#aQv5_ zC>r>X?20MDBUdgx~l zJ*05|E<+JYAGH2`glB7&*c_DC4i0LQM%1yOp}AHzsNrylVzW`Yf4h_u<>lH!_ERI1 zQ!sg-;>4;iDjJJ=8afyFh;itjA5(m6>2yY38;&lTT)gfuV4@WTEpCY4C&J_$4&tO_v$gP zi+X0nSb1mKp!P7nDEV_xtzM5A?XeW6uU-*sqSR!w&&(RtM_7*;vIWOp=|4463F~HEx3{$*dHw}SUD;*N9rJNl;3j?cl&KCND$_4eewT|fi9F5M z_#?h|$E{BDemesvgGcB;x!Kh@4mbWUli$BEFckt4-6yUBO2QLlm(>3P>L8a;DQUU3 zXwCiELH~*1f%1Y=Wpc;#3j=(^f4%@qK>t7gkRoHW);}?N&l@Jy!VOr*I=J}38q7CU zUWf@hH8D9{;BT;4t_Q;dVlMDEVCLt)`7^{S7iLYJ7SH6Jj(-0v_BdX%jegDJKx$no z-ZOd!t~XF)K1{=F^^1Ap+iOzPnV3pvMKDW`QM=P(AKU1>1dnS`y5sN(HDJ^7$qHsA z2c3vl-B!SZ%T?cnghp`DZ3|Q)P64*JOQLrd?J&7b<|AYh=FSGsYv#ZCutQ{+V(t=6 z|C(&r{2E*oViiFr5XyUA0m34|#*OonBHgCxw|UBRyHt|(p3d5cJvja&XJ)YLL7ayH z@2!VO;YAFrjn*&B?t2I_hF%3g<+1UDXu2~Z_dwQ;P^0&a^IA#GtPs}y6zw>hAr%%*8n}Tj*ci@)n#^w6N8|>8*QKyzn8UNi z^WB*Vav(4Jf9^bXI9%>5mWia;1EszLxUj^j z6JkQuNf`XmzXs+TUm9e;dI~T*rYyqP&vG6gQ?;KMW5NvNlt-i(8PwmmTjh_=CV zqYby=NdEhq{ZDZrN^a*L^gq7`Hkf=Nzh!DkBV5028Jp@0fXW`$bO zAoEFN!0*n~xWTkkua%FyJ1UfO#?Fz)L>gHSreH)7XkU;d7GaM=%K2#CVgH*?bL)3K zK;+4hl^20uDJAu5BLSM}LK{sL_PWAZQju`|+#`{4M)Nk|Y#uCK4*_4-KO&o-Bns~C zBFRmA*~W)h7>g@ri%rGpPxV+ZC_Djx|3TgxV4XVI&-2Fe-2yvB;UW_Fk{BcPGLA#o z+FFtUI~7GP&iXv(v$*6I=+H>2J-Y~9GUk) ztH^bP?=0wQ*pGbZ_etc{0!%DAgbskKqnys~!&JG}jL^Zc$ct?6$%C{D0o@Cw>G5hC zElO0(IF|0Jb{D{1OTy^Ch_Nr)(HSMi0jGdj2xER1&28!z>Bi5NrM!ONnz2S)ysa~Ka!Gw`+kLo*7LieI?P$t zARC*x2SfFPArj6wq2&CISt+00*Apj?R(t>WT%>r9MbuT8Vux8rzBqLMTw`=f0|4D7 zwOc^RK;Q>VVBMj{reMf4k;H8Kfrsgh6{YFD1TKTHMEh-xP5F}IW(anfl81*x*hcoB zFdr+0!(1>q4@Z=YDObBasRe^tOVgvopKsX@?JsI^{33#K3IwAUFFhoNYLyvEsc^K| zg|SrILyg0kW76?|Pq!YQg8Crc7&QE#eI5gc+Sd-+Jbr_?m3KdLtlF@yMmVG&)r zKhgxHJ_)|YJ0)?r9-N!+P=Xmewieoxz)E8TbVTutLaK>~d>NR20(yo9pqgYf1h4eG z4q6|QJJI)TsB_pk-`!m5j!~;|V9zaP!x`hGhdar#y&%4$)-ap`7iyye}k%>KfU;8?b z^Ef`IDzlXUy@aAY0dP|r2X8A7OC`Q8OX7Os0_4xye#K_co2NaToHb!)1Uj6TrRU7) zdS1*~oP151M9-pr(RK)>$?F01%W=Up$0l@lG^H{}YBXoGgs2S9R6QiHfPH(aI%$u? zOP=R7I@!{5Ed>WsCkm_E48WjrMTb^Z43-o72q!_kjJk0_85Bw|ESn(jP6NsNuedK=-;h>fljaUo=T0KKL%1}{vRHJKB3h@(QF zW~RnrPnriaQ^wg~jOP`CbmJcqWRxb?7;n`381C^8=;sBgiE9h8v}^PTrjZe4;l^BD zo}bsVasPa2F7aF$A~dMS+>mCGjJ2}!ZhI;~*PdI!E!Z=wf1wG`LBAjq*yu5>+qj;z zVc)z=kSAQQh?q1oJ6F19kZO{}K7L>?VIYpJFW`A~TifeX;A!k5W`*v(3ko;qqP=%8 zO=UTtttR3|kmc$_7+9&2o3&29lMJNv1O{1u#eb(7CHSH`mAkrZs(kNC6pFrmaNY%N zhwQ34=-_C3$w65{&N~4aq}4ARtCL#vp0;ew)>mH70&Z33V1l?!k&hzwbLyBRv9#>2 zS%cB}Kni-#N(;Rr|F_&fB=Y$ZvVA|rZN)QN9J~lxTRef%U8d23RUsmydTdr4 zX@%7&ofk-^=Qbw?`DzKX;CxzFlF5r3ppFCIizQ{i@8404%$HH#!l&?l>(n@ zK?u?MabHUDQFv4QgLW~5+2|!?Mq$`^Ny1krN|)UsKf|k;7_vd$wB$-Ht7#2;%iNjR zk?J1lZ}kg9@l9Q2`d9jy?C;e(wEy^a2rw!nb8(j2pVdk~6=Wdeb(SO0FkThC9G zQ9j8KogwI3$bM8jYhpAJJtmz*3fiF=(D}0J6St^rgH-ykj!28+JqFq#D@9ChWerd- z&&EZSrxA+5O2wIX{cPAQGT87FCMC=9$k>A;S7x#R|3}BP8LaJO_lETD1!brIqnm`T+5m6>fcd$HmO8+RBbQmt` zN&4X>8yuUrs5c)E_t~?xHh)m3KUkemE?@rxm`O{=3oWYhAs=P6hn5wQa z1xwT(wu|!k?Qc2y3gSHSu_!-U<{^>=3gKX;1M@VEZbVkN28EIu8%|2DbX;`Dfx4wTH+Ef{?Az8gR>cOPYgai%II=-tK-8 zg)3q8-G_LLFLmx+c7B3&Ah}+jP)S;P2h>yRpm> z1F{w)#O3NPBhQ!Ky?W6g7}>;FV&I#Vw5V^8@#>SfDb#4J2)c185=f+3Yc3~#CM+o- zl7_l2->}4fxV}bmr*%Gd5qa}cyTU8*94&P+4EfmVX8hSbT4fox@?sZ~|4g59jX<44 zV6c5L9?um>hER{v_~A7ao^BognLZkwngQk3K2flU4kC6El93y2t8Fio<)KfMsa-MGv#}FWEZ?Z@`q3aX!2t)A^+zq5`>HN(Ej^DxooB9VZ+^E8z&i&&j+i+(wi{=2I0`O7u*f3uPp?>54d2RD+Yno(~u(AxF`9bjW=` zd`MG!XsslfQH&?2$Q#p&h=|A8vam9bbumVYFeWApGFU9!r*{ottR`|OlZN81oL-ny ztHTkCMBwteU_hDp=qC=dCWJg0TM3I;oY4iD7?&9q=SVHEY(U`Fz-YDH($(^uI!bohXoQ~4ORuRh;N!zU@Ti; zVS#hkI5s9Og|C*|@c(qs^&pChcDLvL*JsKXV}n{&RoT6~t}3odBQ%VwISZDaRX1a9 zo6q4K42>$;?f5{8o~2B{{^D@Ob-FG-B)%#zPWVt$`(Wd%-tC6b@bigq(be%A!ArnZ zcFG(ZSIA>(3$F$J>;L$f(#|ojd8qsU$fA=3Sb|1=LLAEPzbyk_0a!Bq&@_&Q^FN=D zITi^T`GSJbTK69>7JCR81Fr5r!UFFXT$um*6Au{!l$Fo_`;m$)@O(f0mF#o=T@QgY z1VJE{eDAew`R_-5l7WoLaBc(0Ya3IWi3dvQD;&3l`NjU^7wz6p3inT0jKM@c;sY z46tI(&w?%C`?~sDsK%1?ACAi2Qa!edb^P(o)Pt28x7~3%i3r z!Q5Kt`FEqQWmk32X}+o}<-qVdwPn~>>VV;b+Vp)PuXz>cS4e{ZE#7~c;#r$e1@ot2 zi}$!4pRW#>VEgs{5h#Ea(IU$w z78w|hfZO|gm6U2@rnVIFAXvJ7M|Rra5YAuj`d>@b-Jf3zWAK{<_wE-?f7emr7O)NI zjs-y*=NN?eZw#}_w|5J;@P+Jf&3Q`~0PSaI(e}MM@UOVd&nP>4RvHNi>@MCll zpO7n#3yQjff71xX#R;kSTjS`2JoaAB-}46Hz=H>eP?iORI|C@&V53m7BE=c9EKF4t z%ri*ZK@DYa!XN2BL9l0l)_(NY9Kc&0R-r45A)3?R`*ufCD_bdN7D^^TonEk`@#Y;q ze%G=w_9^CzX-o6)`+nClm`I%s&Lx@}V(dwPd{1P~&&A@!pD&!sxRbMlDRrj7?6&YvXOm+1ULSke;PjYEZTg+`8Ep0Y_( zoOsXtB~uvfVi%7-E!89qsIeI|J5+qZ9ez$`fn)EP<-6Dq=0Ty51|b3D9i#K3olAUD z)m2>cpUn3x>r-KDmuYRnCH-5Y8hnR-LJ=al36EcjKZmy;&ARIFx!4S4Y#=bh2<0@M z67DE=8g>R~(~aH#8r75#dJJ+f9vG3=c?w8|Pgp;Jx=;)HU>*!w-sq|Ssoh)Wg|Kvw zrUB)W4wLAvpe5qkdrc_t3t+NdIWzdb1&f!&Bdfttx>}cvpi(cOYx&TD#_%;yn*Xv- zAlcvRJ&C^yA2rh!zpz!OHPiX3NJL*V2l3i|AIcW62$jh`t{f8YKkxqS>BBzl15kqv zkj_IqM{3+%_cl>4>(G`Y^F*$~12_{t73TWAOx&jW*%Yqc!7kq~MV8jZAS9~>qP|Wb zq{0S8Kc`&A5Km}TMplQjvnLfbNAH=_Q)ZSzy%wk$-b|*<#EkQ`Pg9&W?^^MZH^{UC zx5ElZ@2w6=9UC2w>+7&>RuUs|%;DP*`r@Y04ER$<=TicV0eqDMz#v$Ypr19%RA;_} zScZfk3E4*Lvab*~Cs7OYgm<)?%8KJblw2{WO+3sZ_&!v60Jw)YgW{dwA(AO(s$5ck zIl88?`i^XB;=`06-4NRLSQmSLhHx9EHNh=j73AAj5CJ638=b{>!}zdknR2? zwu~Y_VZ1P4!C9cOnq{FD0A!gK83GL)a=RCkRL{%)zQVx<%0h>v5*3NCza#Em*LUn{ zC=Iks6VN1;LTfAKTIJCtpj^97J1WOq@s9Bl1e%(b%3(OD8|lwv4F&StK@-3QbOscb z*+dys!cZsz_fck@i24K~yW>BQBwj2k1tsF`?A8bo&oFRFgi>R;!hhR_OiDHaKKj z)u^vzW`X!_5-fBRi!{9`t)=GRWIkXwDO}coIo37zC?|->1#);~?Y^$Jwg(!&^2}2bvUgV(aHQDXeFxv10O(|4*3R&^r-wBB;(PT~2)6A$< z(u&A+`T^W#vA)=s*|BjJ&@AHC>jZjkfrd4KnK~|lfEZ^j^dtb939H3(P8qtR8smGYR$KpXdX#~X50xANS@mDOL6Ybbm1*Vr$7|_@Y*u1PGWkw8 zdsI0Jlkr)EWEF>jp)D)Y26wyvWCjcX=hF)O zlwr%epVZH8p)6@>9XmbC5jQrN+7eqdH=h_l+9*3ee0?O5kRU=T5`jaBu^>dvE-4sR zxC#+hDRsW`C0H;s}g^5_MW2}lKLle{A3Z;y!+2QdK$cn(CNmPrb% zx6daC&BM?CjncEEe?9VCH0HDe_Ucb0<-I=cTorMV-Tz(ETgk4%uBiRLn03sLINy4I ziglsYu*AMiwf%{>dyMrtTV^%pbBCCQN2e-{6oTJSb=7kEQc=la9N$dMl5sMwYKzpWmAYKK$iQQ{|H$S{+mI1uIePV4I`xO+ zknouJ6Uaw#jE5~@@blEXwO^d&E1&FJ+)UP7R&E#b4L1j&oMYhjT*hiMXU@Mvwosk4 zNG+RpKwdq9ApA=j#dHf{h6Tm&;Yk3#fgnEF1a!?#tj*ekL9 zRn8dn%*DOO!>?KFPBy(&NBEDB;DKRm&M zeTULVj9eQs0y`KBUFsWcl+Ej4|9^~S{nxX%^91xZFBoZqtRy=`exVB+*?iNsMb69! zatiiiPIxh?G>3C`yW1bblqo`Q9UVWv6M>YDPFU1r$4g^>o^g;j{aqS|iTa{LMv#L7 zhd}(cRCP#B?{r((ZR#3Mo+~=j;e&?nvrEzreX3=|Y1<7o*}I`$b*2}*DEL{%0<1h~ z0^FPc95&8ylqoRJ;f-^utGb}G%%&lHlq!zJoP<_Mu~TrmN6`mKDmH4!{e;aUhPzX~ zGw3FvhZ%hOx$Uhy{hnEpyLx{mCa%klg4gm<-z)3QU>2)6Bgz^!-Bao7)eV0kGmbDM z8--lTWi-k&3Qj6LZeQ(V=0eNO64R0Ugz^+B;=%+FIczlTFqz0y{@xxxaYrD>tOqd}fiEpX%+Gwmb>T<3F_1i}!J2aP9?;x6BCTrlGlykO#VuOn z;C2_B-L1x@hxqnsWu5-}?Idl993y7U^w)#5M$=I@&tv^mA{Yp0ND@E9e8;!Y8@IgA zcaC|+gej&9$m&%k*Q;&?c9O#+7Juw2#IU|=t8AkuVRt>%pOy$8leJmFk=n-MEI&el zu3u_~ih-|nN@)jUEoep%(mjWKqiB*|H~;S6)K?s$s<`)jV0qVdZa$AIhF->U#YFvX z;uN^cZIg}mhgoX`)LeF{X z-r80eujnOQxsg<=e^dRzsEw1EF|PGmXgjyr))Nl2;g z1K>bOyam}@5RW?V`Cq`EC?Dxlv*;?mWwX`S8wH7xfKf&Oy`UL)(*nQevIFRaN3LR& z0iu*M_d|$tls(jZ-~^}hbfAgs8sxA0j~{@x;G_gT8A{3jhl@tH0SHDKDJ>=7g#8aK zNf!pBpOAN-^#5BhQcnSb(Z>G7kN-n@ZS$H0e&F` zW|lUPXu!w$*B?$}@Gc7aO`rWYT#^$T1a9P8$BO>*oikzp1hi<2^@z2(nQR_+3AK0DCk#)%%MO#l@d)9n;Ot ziLZ+gUqi)BAFrW`oc-QAmR+D1G=fI`FA&FgpZ^%OeFg3(O_kCh=5fo&EQVexJ`3{7 zG{T3fIHzN)Fw}_tp#}nW$QuYu6@MG;&Ic%!#xCF4Ab8+_>nM;Twjea*eEqO1qdAt` z#iT`ySaTS>iCLR3&KNNWBa6o9pP5#=ttRLo^OM??l@mERpkq$jjD+fM zXpqb=V5MJhYCg4g2mjFs`I4kA|BjUaHKF+eB(eU%Np|s`m)0%bdg!{U=?CQ$7^*v=|7KhlOf5A=WK>+nx^G+ z_3klHH9Q5Lq`n6uY+k$f-TSosB^TJkPLD#|?>xENe)w_PI{y$db39@FDUd4$jr0NO zqJT!A9MQ@Ht1{qd&b9_%{et?TPIGHH<_Y4b#cBA$U#z$Y3hN z(>0FiyYwJ>rGE@mutqRm;g>u1J~?b#_5^$#r&kP%ASX(j8`=Own7fgtIiFpw1)^ml z10RoHAz@s6oyF98L11di^vB~yM0QM6!$eK+v_mr_SrTz{)dWj;{KJBZLq~ z69=Cwa2^mxd(Qzc+NF#_t~~7NHn2GD?KretF|hLGK2Y=^sf0%hOgl1*PPv8K?)##n z(J_eKEY8nUV5ZV9V0hbjsAi?Y@1pgGm?A^t?7<=bK8t(Oy9N3=F&Fc_8l?4GjXm_t`yXTF#mX4=+g!nBck+o;p@Uw==uwpLK0=6;_muunjl-xC7vtBGX25e-=gIS#M@p^D-P> zJrdT@tkM^_=Lk2>mi7(2F?5Gc>28ec9;F+XV-m!K`czo8j{XaIecdr1xaz!@wlW%t z-*56wpP?a4nSIVFf3Pk9&#RElOyp`$X$Fr}O>%HRH(6|0`^!Zev-1%tB8q>`qd9`u*s_@?Ui7+oZday`S5|v@y|ub&+;c0FKV22%;^n@W?r=r z?kw_JNeDc4`jj9>POnbm^BkllFyJ!-{D4z92Rk;pEX~$ zJ?Pl#mo}o@U#VN}cgwE~I_Ow8Sn+EmDT2WjrTs@dvg?$E|12qs{i?HuErS@D0PKya z=_p$!F(SB708O+i&(xrmeP0G+lgMiqudC-kTS{I9=B5ewUX&6Tn|!;%=mzdASg8!5 zoIGYx(4ulPoIs$2yu-WCcUb!3DMBPCAzU?al%_>a{sM6#3X;JJYyAN2OHpB!`_&d# z%Pywi64U_6k{DMBV6AE%K`H-g!_N!4g^qZcZ^CjGr48StLe`LDPDFXd`bx@2iHBE# zCB7DQn&Q~Frr>p)yDu~BS0!dEl-O-6!8q-Gw4EkeZ`#nbvHp=TRzvt{gJcz?5Iw~7 zjGQiIzjm^btm1s^xExHaB(-O)6aD;-Upv$NFF6GcFr|lH8zj~qdwbK19~d(zc6sQZ zcG4}}R}%q_c#_4ihbY3f4KtCidF3?ZRbQ7*09A9zqA&GwJZg`w^L8lbauP56W4@Iw zXd*N4YPQP{GZ7e#vnQp~O;d!7^0Nnr-V02?5sflY{2Ftem+~WO&zY({#5JOVDJuJ+ z30yv!v{{o=4zVB0T*@Eyrh5t1AO1}A`^$F)61-7v?1Kwb2*VN%dAi+N)4P(#M*df@ zjozHRKp#?yy`B>x`Eh1z>bt5v*b{=h{0fWybDT2%LkKKIMT3EnOko8Pq`;8eQxEJt z#3-r!C$g4%e{}vxG?l64Qt%buZBYq+6~Cayh>Vex?977C^1(Mp;uO{!?rxoaOR8qd zIyEL^Q|P=&Cj{cD_$D--JbmwL_Q?Vwm|BJw-DQq09GY$Uxyu>Io>D zU~bl87S{WsW@$9%qYrKR>jlueZiVSF2ivAuCF4z`4iR1~L-%?R0WePhj5ECEoN0wVT=dbzd!P|WpFZC0 z3T*c>e?ol}!hl6CzA(uN=cccRT(@}s9R5u;FY4DHabc7ZDoJQ>KnHmwJp^WX*cYe zC7*p0PwSb}RjR7SIv=0p7tLi^5B?=9f$rft+~hYM9+@Qj2z{?N==E-e*OGVQF26_0 zJiOwxg{~glGcQKqJFbolWt2=9PXK^_Cpp~jNQbqKP6JP}PErj02 z$v6f}Nwp1))Skzim|^pbow7vtkJhbH(eY)~>c}QJ4LBLln|dY@V>D4B->@ydo3|8Z z9ggYU4Lrly{rK?niI!l-J8!|cNeh`xb@mStpm|B>xtGp1I~O${JQ31d!ks++a9L|I zYrYcZ;imxifKYO`(r@fwicdDY-JzM*^%J=`wiq{_>&^*EE3crUYrQxgMd^G~RN=X{ zSzhXPpMJZXWY;J|h=Eq)sIIlJT{0`U^k=>iml{~ zJpVoQ1)f=_T(#VUL2otlugq^x&4H`vu!nap< z>;}kaw5~i{@|4r|m{E-psjI1Tdh$UTmr%^}){p2v{We$m^?cONw1P6vM@%!YlsPHP zX6Ttno?3e-l%1bgc(t4!^z>^jy%Xrtoa=H{_>e)(_;g>(NPqJ#yVwWxQsx7cISWC( z&8hJlhW*9kZr`Lk9|xK}yo*v!z9~m}MZdP$6yKEpW7i(!J1X4Q;hc!WaSU&!_+~@x zuVaawo7oUxi`#zeyDi>g;YTca_Zd_o2B_EUq7dSCz zgF-yyU?Jn8>w<>BZqcCn`KZ6d4!3naL@np0#uW3mw_{{FBH?>Jp9iMTyyAA&;CnK5 zH6h@u%5%;dGP>gJH9!>kjm5yVE28)eMZ9Eo~ZA6 zBu>Koc9Zp_;tD51Q4iWzC&MgQ?Cp?k!4aA24-*7x0-wzmHs!4#o!sZFp5V}=Y0d?| zi53?5rt-jR_Qa-pO@KG)z+FLefl$8n6n++rHg`CPr7e-6yM=g8GmLnG3>DitSz5TT za~NB9)47wS);&G~>yC^}NvRk$lbd9VI@9b~p)%6I4)9JOouH-EiOYUC=iO9yaSv@S z5wo&zweSY3NfftJY@)A+A;EwA1Z&A}Ehh8Xc%3O#OyuiT@`u+^ox50y<-?efg(@SX z0t-l6ZpBhHNmE}P_8YslzR4N(=xVDc*i;*%_tQ`jmeD*$?*`KT=PF)?P{Aqsk zM_&bac$r#Dlzi&1T5kTm*6Lx1bQ#HUCl)Y|Kc_Y`L3R5^zubGh8^Npdfv%-u+=l!+ zVSx5{p!@qocY_m#9IOFFM2g%vygie*BifR@QNxmBhdUyml^%d@IP-_w8%D?;GR9VZq99chrJ8$C`yg-rDc%spxl6hP;asGjH-v3(jd7?k+7R>ze-YZCAJ-`Am3- z^Ou;NuV6smqinVmr|(}2+ESf=>AjEle5_@+Khg}Uk1o8&9~+R@BPZ9k=uyM<$v)w% zED_m{V0_Pvn90Y>r4<>net9?+CHzfb>uOo~7nGI<8_MTI2qyO7&c>$92-Mt&n)X}w@622Pb#DeF^~}K}(P%(E z&zr2NxDxa|gTDZ3dS!%H^f`dsgzmE^h0EkxB^$7R+8^EJ2T?^gg2cnF$9_MOWtApu z06SWE^Y+6{#FM86T|lB=`y&5|aSIw-ZdjWytt+GFPij}0zYUCz1cydVemde;pdVn> z1R})gRR&v%InyMCy$wjjH&7v4TWZnWb_;|Kf^s^}nyTm2#p!{2GGPxN>d*c(CzCV3!G>h6g@6l2h&n? zB*3wWcrw#)^7j9-0M4>fWsqQM-waR{2!BCI zzdRnuu+z~XmaIEr0uJhj1vjADawnHGJ!=AQ>h0iziAjmwri9}U^q^Vb4G66fI}^Y? z*9hkSMHWZ;L$<|jkZ-L8eoVpUdxw{xg-{jA+t3Sc^yEK_>sQ||oGagTOjOS!X4cJw zJ+?F!0p`p+mUup~6qg)=Xyred zB>_n52)w%)5I$^qZpbwb!I?mp!W@#wR8=}ihn5x7}M&w$}R$)Y56OwTy&e z-lD2;SHMPkQ{L^=Fr+MgOj7QVJ_mXqkCz0K43@F7rhwQnlx+Ogpb< z3F34A{QTs8$9WTQa-Xf5YqWo{(-gV){tE_98o@46hh|~qcW)42uC<63V^=Ik-hIeB zRx%Ckb+xfc;;zg^Z}44T0asGU*{k)D56YJ=!0I_&X4bI{CT`&se9&jWi0F8CR?6y@ zkxk)R$5{9V)1hX{x&{o03l^d1kVL@qmSWj@+SJNGFDQghVDep(t9H0$(NGQc=ca&gfPa!Jx0LMX6YQ@@ne3GHG94emdNn6R(PIx}N zhWnWXPYKpQ$zB-YgU!$26c%-WriT&gbb>TGJ&nsaIRL;|P?gAHV&9Ka$`!X=N7?3C z3~Z?+L*p{5?Z(*0!=V<@-pevRI{)A^Xi|(Nf(sbGoo(+NZBGe#sBI)h-leDN#wP~; z=;`WLl%3E*M!g^TX+?tYW$?XH2NcKaHmo~zXJ`<13w8)MfkJwZc`@PgdZwhe2gdjj zzjH1ZwQ;ANAXXIWp3<*is=-guky$*hw{`s&Z{#>Z%|Hml_ko_-oWusp#Bft!Q@g1$ zaWIp7szxQI02;iKlmOlLlL}6PfA4b@2`D4GX_Dxc%K6!Cn8s7e9nj$_umX#VyBi7Y zK7F5PVLR)J{C*C}!nSF3j84Fe4KIJP&5-%>ug3|jR{Cs+r`b8o5!M!Xu zllh|bdy#jS{O;kJn6Z;U!)6{!V^(=niTpk!2o$t0C|b%rRg@t!mKVMi8a7${k3h|{ z&_mVpF&f@q^J~lwy7h?RahK$B*@)>Ea>&=FXFp8a@DeV9Y?PWq2?$4jc?yw~n5kYjoE3$6>|n(f%oOc=B-!H(9mI{V4hB;n!Tj~gd z{^;I*tTnrLyK9@rtPj#4Ht)mewnmqRt>&W;H4BP1x(A~|<%!TGwukpSWY7|=N~p8t z_8#BFbT>*e8xaz2wCI*OjNK=_v44EwqxP#OU;FvTvt)5$8hTBy?iOZy^ggqfhDEUX zlCgDoip0Elop8@jXfu6Emcw%YSDtGxk1<_hadaNf+9BPkZVM93ohG=KN59xL(BokW zu>V=_Ga`=C-Agh-gVv1sUaqN6a{zqi;glsA5esyXIib3yj`Fyj;HdhT$C4_9UgoS6F- zfuC6w^4&Mbi=Hb+gkgAHrd64iY>`yJ9J&|OMpo6eHWYIfVIJw+^S3R(EG-UKS{Gs+ zjsv1cvUs9OH)(l02_N?+=N^+)et(TrQjs*c^FB7je4;(xAiHc39!w_mrWJHMQyOj{ z@8`v{eLVpJdnMQ#3I^xIiu^fzIZLx}?`j|IEWF)|t3WF>rOvBt%h-J)$n_gCerrDO zsf4UDAy{oi4)7zIu*MnKjZw^PaNJg;VBQA!rD4Lj4n|1330^jdA zN^H_w8(ZVJXY0IETc`T`P9Q6piE5k;@p9T%G1hX1UMd;#-#^yI6boY@qmRt#tDT7Q zjhBDbkB>VS&Jt<8Wr;~j^YZa?Ny?~hy3nv{5L{56{0^c&^gmI*bK8=?C=G`xC*g18 zUw-3=bp&U_d^bTBQmZ5N=^Te{PFoMlgVeH{7PPkr#fCyCyDh}PUY=BXOADOrme^bk z(vd&$H-;FEX-b1g!v-D=un2gVNirkV4ESqUB=J6V6yEg|)&V?M{g^*^cLkdSL{ey}IW>t&PC>^kGXk!sF~ z^cgQEDASZCtlos8j`Vz9TzK1U)!F(1(g;Cd(vZJ@w}>0PQ0w{jx}-`9`b2v8#_;e* zDOD<=L-UV2D2ojUOR>VLo5#7mxN8(40>)g9j13+1>RiBUdx+0kP9UO>7s6>O1fIT)}XgwpXgd2J8nW{Os&; zIfIJoGvIx21-{#?jL0dmfcE?kz#m#^Jw63@k~_=+O^S|8Svh@x3HI;REVHf~-sAeI zm@j_{M$^DQ1uyxpKb>H>S9kdOKg{TV{r?ErHWJ`vOVua{1h4gPj{vyz|L=b^9$nh| z*B9(%MC?Xc(?M1T*p>M8AAK>X{P@6jr1huaD;^NX^6Rde{wI2>@g5vjHZ7wb=<+4T z{M_Fc^m!k`dz&i`ymGNt;wnfRe zi?HxOCHM1q?8n(WYSZG-%q>q+UE`-FmPw`H_Yb6nKfd(3-1`e6U9UiuOvKkzHiZ=K z=WYB78P!1P!~tgC=+zjz%t^cdnPn_fiKeCQJiK-P>&plBtq>~xZStPeDLb2$&ToB`RnKQMFkaE_RVgu ze-b1C{ev?gqay}S%xSL4bB=0n^U*?5k`%y`z6Dj#oqi&;4?yZngL3k*gRffiEZ9CA zfhmmKZ;?j(UI1AVwXam{I8*a>A%s%-S;#P0jqt7ffNxA8s2sGCpk&n)7*0_Ae0u#B z_A#hLUV#FmdG7?Z`?x3Qlb~6GX-O4{bx(WEab&N2l+d_%U-~iMD5-!`H zb5KxBfrhjh5F-d?Adh0>a=gZE2^e+rC_vcW6}HtCbr(O>7F)Rk1uD(wAn_nplX0Sf zyV>2KX0Y%Xk@U$+iG^UY2)fXtL6EU?09{ICan-20*$lYne6<KgBj@^-HYri;X%sQWtSyu~EZ0v%Ur}Y`EdZeei=at87&U}DQFbQtRlw1T{ z!c8>5-t-Th^DR`lfm19?pDksB3IMS1vc@#+Mjg*Vs>~65eME#RI!omB{>LkT%}K{) zfXx@x1#+m;!qyW;u3BKk>!758{da~o;2W}q7)_rsGNucO!9wLp+FKxlCPR0UiL_HR7SKn^BK6G^%&NmT8}WF6#wOVeNJ&7qz41O-2y+DH#3LYMOaq_iT@mks zkdz?xM9I-ZBIsIq3zBXj%3KQ*Y2p~`5AtnX0Y(?E=R>(D04PC3a5AOhup;mk(|$Ei;Kj`Wi57CRp{(v8snOH+cH~*8||Il=#|>g$DlDP@Gr7nf~1h z8h0YSt;KMnq&8*44L{`wMz>i4-$=+k7OW=xp3)G5Ae%-IC`c?KSORqmGO1&pUI6h# z9!@w!g7hNL>*tSQ&BF?$)Z-Sl6f^*f+sCFVE$q+u0fD5|!tcxA3-!Wv>jHUtqCm-1 zxVUYiKuog`C|m|cw@v68I93jd)ErH}3cIk~)5FuV8!gOp>D+op3dTYvJ(~vW2wm_t zFv*%h_Rn(G-9iDUpfZVl=2;wQG%8F=n}$Or$PIjxI#zNkQF?70)MnvLqB4?VkQsTO ziG;F>b_JNrQqpqRWW=XZ$=(gsfeD3S4S_gRA6_{jvGNk48bWm+h|iqt=npf>lV-x6 zY?6w1QH?2$s{_}jf>a1@FuEcC(;wT(vJW6-!_KykA`nO(9uGDM+)kb|5QbOKjjwS|tg+?@WxPeD?QBi0D}1(oYkJzH!yZ#& z-ej@=MHa`K;SLl85VO>V z+GZh5=+DeqXhGd6<-z-|k}Qvhd~AGpEcyx&OB9DkkEkDsl>iBQ!2?|dcC%o92=>QI z>3Mn@an$ONsV&`$2Q^k(KY3R_tQ3Abni=8!Q(=}y@h;1k{ppA3Ixr-$J<qO|pxT+JUI9qKorCC6P78m@PvgE+f&0KeOh}JK z6E?+;&a;RWza;_mvUb5=cS+em31mB|Y5WZU)$=v%(cxyP&)!x1)54Frp#bwJ5TdM` z+Po{7rpd4VO~D)A;?ccO6TMF()GFVUaPP(eq*PD8ysw5-dWj>#mC4YiLdL4iEdDzp zl_?M>4@)4^m06!Yt!70Z*QQTpMlt6z87GzA)|@tti<%Ib5Idm))omQ4EnKeO%WH_F z8fozwV?gvLP-xly-dHS(qLFeF#w_AT4>go$l~=yW(^FfD8=;~3;y7J}XXYO)JSL$Q z#PdB1!1qST5D#%kGy63?)J|92NZ^8Xr)7P=#mgb6ltWT8wV+c zp-X9y5D)=Hx49?G`&xvjhY!>BNgRikR(9j39{6b$QtDi%%u=h?1AERcy!A znX3V&qV!YoqpCZh*(vA@CTvxT9lTeq6jgU?suc0P=nwoUsR^<<98=H)BE~9Z24A6T zsYnV$HveZ`O*$C`e1`%stBVW=_DDV(Qy&e^?(MwgTUCY3`mC^_xZVTSOR4wytT#!l zuN;WBO6}Yf{A{eJC*|R+N2h%fDuX1;*W(B}yj!U$b;45GxgS4pG359sn8GRcE*v9- zNRZ8OT?qV^pRo=2_lpqpd5QX+2NDb&vedWpKZGf=yut??b2?E;S% zU#dX!0ju(Bt1rNLNs`}ZmkeHUc8H4~JlAskeg5U^*?^C1`d^#5Q(EOH%9uO7Eqh0P@p|i;^-}ynNq5eHsIXRyq^2JikOOm+E zNA7GuzrTbu3v#JfLADFgj#J#il#BS}tpDL(w@d!oJcB=X;9Yzqt4hQ7N(Z!+V`!0%TCL@>f#Ida2qM0xM~2B|z3qH+$)(m9x0wf)h5E4rD`poG zfV@7aE8%J)r73IA7^R4n@mnnw=Ke48hp`Pvv;pD?HZn85U9o!meJ&VnC*-=HaAhN$ z$ch@2WCfZczZk&=7da}cqm0APeOyti*fGT{UNFCt_6yixlmiWXiEbYCX$|b~r(SI0 zU9mQH@@j>>BX^BAar}a+n5m!JRux407y#rsF zpY7yGwqu;7knx$U7othc*}}{N*qAgB>wRf#$`p#;ZOeSPpSnwNQU;SReo&gycqp)V z%$?xEikJgfZH>1TM-vv@!1|*So#hK&OE`EfBRT?%QS?H+Icz(wcX18Clf%@9lPSKG z0x|Uv!#PDd$mhmEuSKn{Ua2u3ahLL@8i_+z>FvjE!!V+SQgYfrdU>|Hl;e+dX-eH_|ooD=zma_;AbX7n zl`UszF?ej6k9R;cWw7=_o&Ji#tpnC{_hCKUvY)mRJ*N_L{13wO8lt6r+%x6mf=|wQ za48uDqb-TgQ|I-}^`b5jUXg85T9+OajF3owIh+^Iz{_{aE_pd{A9@j5*q@On^9EI+f%i`Ed0NNtAZfP1 z+?C9jQj*ieAh>OaOk3cyCqqQDw{tqJkL-bc2<|aH^#iF^T4TMZKX+6~r$Ug9IZN|n zK6Kf!DbiF7&u42DMLY65tZB?|qIGzpgs@ms%*;S%D{0UDx=?%SP-iNTgM$vj``y4F zu?n~+%0VSgd(C;S9Mj&$J8Ix^zrotp%Nv!eE22hprxK0dz5Q7{pRGK`vUp$TfGSFK zH@g%osxPv}gSDtdKEjRsSWv@|GYg&k99qRPclh*3r+W%@B^O@eb^O+_wR52Gsh>ms zwsy4=yGRz#VkyP&d#UqR4;VZ+n6Bq&t(x1}w4LDbpsX(Dpm?6<9575iHXKA)v``u& z4y-{WNsIMI05vQqF&ZZ=RwV`fU^ix8sB=;of~Tri(9wBFRb@6H$eQ`CE|Kia62MT# z3M;r{c`J`RZft~J3_ONVqE$=AfYazHwyn<6mL@hh>=>l;n1f8}s~XqDf1i!vvKZf3 z-;+*5sLQkylZgngC>FMvNq%Sg< z^ZPJW2FjO6o6&DLbD%*q&|1hIA=T= zjO{0vvwNf%!~wyvK6p3Pe3jT~$^!&O2%1%Mx`>?4M0+>7`PoPBz3fU!)tCZfQu zq#ulGqfv*{in|>RLwrNO;zORM>^VERz1>Hf9Q(g+P^){MUSJ4VsvtSATem#foqK74 z7O~MDG!CbMvkPf{oXVolHdYy(>K5_6H~FG2&e?oPqxrc!O|{7i|74qsS{cnT&HY8xvm1 z^kydx?PT0~I#Ny)oHv?2B_cP$@|g4J&g;Q284s4b+5!i=0O5MDFMVLKE0b-E{jdkLHx&He{<g7vtN%pZ zWFD{qu}DI3Q7=GZ|E^1UC;EKiByD+o-R&7AKR!QZEem7)@ z&Hn+CWBdb_19s@J;p6&F*KxS)>Hjlc^^eo$z<-{mxCx9v0+uUX=CNLM)&Tr(F0Ti! zmG{kAmVOyvO?m4Qu|;Je#!y2X?FFz(kdueicx5rC>{m1w7LgsBMmHDE?~~X!aQ3sas(J zije!Au0ZfWI4uHt6x)fPm;VCwH^CydRbyEOG*WJWG+-kx2cLG}ll0hKHw^^2m z#QSkSw*BKf*aYxU83&eO z4xv+^O|bt5XwE}bfz~YJLJZ*bOQ0r1ut$_U1tmiI5np-Dh`s$x6zVB$@w1lz zL1qrX*z3kV#t5(slJ?Vp#Mua@aV&v0?e+|!&W}g~04GW01+K~v$27FX^;b`8$CgkJ;Ey* zko2iSK1#2lHt{O3N4o5U@&S3}DLT)F0Qe17+bVI7HLk?We_w4Z(_Ox7VRBNub->@- zfHO$|=z`uMC{o5Cp#s~xRB3-dUC4BFMpM23&eTex1sgb=0j9SoaD|M}XdN zC#8nUNQKQ%)1|D2H`vSU3G(azm;n?Rmv`p7BE#>FL+w-`!>!?N4W^ zKChXAq?Y)xlG8dk?K$oHq;-h8(*6YChP|G^)8}&E4%p04qCQ}V{XzHG>{&2iCdEHJ zZbX-YO&nw%hHOA+?uT<0t-RM(YbY!E0YI)+zlC*k7ws_MmH&^cxQToo07&9tCO-U- zc8PWDQw8cNy+YtY*!Vd)D0x#C$io806!4h0V6Bp12VMv?R)+vP9EZ&6$tvDV1AHwo z-;%4q0ALM2&27Cv!hA?+VdI5jyKiNSF)+nprg+?$u-!<{2ta@NmaaFSg7JQ3+ZAQ< zyBE&@APRAiGu;RXb>!2EcZ%SF&!By0Lmpcx&4Yo${vl7et{7t8{`L~dD3axqWm7yu z-^H)@m=|Q~cK~tjhcWGmFlAIR1Y0H_uK;Beks!G=xQj@ZYGAL}K6skrLK=x+;BR{vl?~wx04Mtt3 zsawY|J-Hlsp8CCn&roMDpBTky4bmC61%Wqz`vRYyua`oQWH*l9AOxAaJ&7vuAXwm0 zkAix#i?F%R+G3Iky#f>}*!pWDkd&nm4Fq0T7VRF6$Qnp(7Ucppqd8U;mOJ*b{szza z5>8!gr%Pafe9E*<1lg&A-S`5ud6l5$^NSzw<>Mn4Dwo}pTQLJN8Cy|aDR8@f90DLN z*rgiKFISg47j%JL*rW10>EgF~>Ne(Iw5me2=HH(rD8par~@`T1C5 z^JN-LN}{BL50$^LgOB&V1k){pUy-ZrG+muR186haI^CW5W>;^uY+@ z+*Rf;L<2Q0ZE+3a%QtsdW=MmE9EZ~sJeXxuB+n5Kx?lmLi*U@Hq5BYBckXUet zO>5vE_ZGTM0gwuqG)W~0@PPj~i~s*c{dtf7AD&dFJ03QM!3n(Zs@1h!g_)AxRlaAq z4oK;I(@jGh{f6ZUXfG4l8hk@Rp~hKB}Dnj2ii2io_GgqWb%(?tpnvd&CC{T2<}0h6KBy>+|v! z)UaMuAJ6eUP0XzjB*z>Av2Xn2-|iBurP4s|8F}ij@@l#jFss#qy`rs9vuk2 z`^$8C^A=#~KX`|caswLEV51EP8BCd#qY2l+dcECVm1i(}@+0=L50H3nfJs$jvkW+< zPW(at+ASGd^z|hb^jw{3T^Y!^7zir!>oPWR5xP^~fZ<28^9-G_ z3{aVs7_mej5o@s2(#@j)pl5gDz7c=cIRWC4{&G%N17)_Z^1DhAP;iw1!*{_ltAUPu zn328tjfd`=6JH?{4^Rym-*6k;dhomn%<`MSJFgN-BO&!!bO3m7E~Fe=^i7UDVQ}x4 z#X32GY+R}#o1DMR2>01RcdLHG1CVo{ExFet7~zs#`Vp}3dO$CyJ-KreUPYw|NUo#F ztPuKC)nx`_^lZ@Z4@fAvaS9;O;Gdtbknj_EOT}Hu>IV{``bvk%5e?BjN#>buyLdQ?Ta_#%gK&^?+|^WT1P0n6P#)(F{sg&&5K|rG z@MhL$jNLZ08zjL5+ckO7dT*4rw_qh5-nzyG+tq?n9kZ9sWjG2x{`(UXdwjL6K;ZG3 zTV=_~JBr!!0~GGP(O`2|hwrB;(LbSjws(jG>)Uy20+ep@DeyE+>7Nx&8s8;~8-Khr z0@d#gDa`V3py(}dAK!|*-;zyYS6CYCP`CQ_s%*3z=xqPr6!qLz^R<-<4A}WK1CjTG zLKi917Q9n1ECXMJFNN(S3s{E#1OoYiucmaTZBe9uR8fUXZr}wi1aRvisIP;3J7fB- zb{jw;zX9r=Y9N;sg9+lbXerZKhQ4B12SH7ckqXpfa0MRr7%al?H$ZNWsw%w#d53I2 z z1)^8E4A=)CpykHUFng|wzK%q6fcH@W#R8RP#Wxc@JIYJgO$I+zW zG#yNImKKJ#9np01%6REyTbI(rJ>Jo|b|MeK7{4@71r^qNrkQYNB@7psRWyNvXRAHX z4?`=zx!w-HC^wHlGC9RXSXmE0l5=T)(jW7++y;ZiDlD z1H9;4OxhFh@^1q9GoQF9?BGsE2Hy^FTnRRR8GRZ_oD8UZL3G4wKV<%UZ+YuTa`{{_ zlwd>~v2zA&ci0x~TLXA_Re(W7%th}qRg}!OhFp>LSxT{ftMCi*tw8xgg&cR~ z@I6!iE;Z`NKA4d>0jVxj{@vE3nthH8Cas6*?KY8Cpkrfz3(I?8`J)?hTn7jmxs;Z0 zA*cAT7y5_u$!U}qB412_DOxh^Vs@roK=T^N`XZ+P4)%B7AfIh>ruUHBwECC=b?Ds;Otj6CeYw@G3;ht>vL;qbpmw;U+V85~$n3uJ8XMJ-^$6{U0fq)>q(ayk_qv7c`noF z*1fi*>#!YqduPXUZq|c8e9R+IMTkW-$(6Z?->OIH9Fs15Xu#RHFEiJB4FEO=!)u0J z>tQ#FO*BW4UHrS9?I=1rd`D1^1600|S@e4iG0$VYYQR^%Z5b~A&Yq#{{@IBn!joE| zdi@VI+kZ>H`yv-ucZVP`vFMN#pHTj!h7#bUKM5R)DGx&{_^O*Cdh;-a>ofGZ6)y%l%$Ls47rC`WNMdQ?h`(2C{ih( zoeOH)iDc@Tqad=3a_T;kKn`HaTSE0-fsQI_Y>q%!ER8uc{w!M>6f;=GHLDfjyBJQcFN#g(2w!8K zDIEohd9C>>fQp1XP(Kd5y-Qgxkm=e5GM2v``>~$<0(B9d=F}xhL(-O4xK883CSqljJJcDo}obIT9zp?1-B(Q|Vg`x>%T<@zt=4Mu6*SL}HA` zw@-+y@3qO9BIeFHOL6v#shU-%EUDH0hwCrhW4x)MhK>JS^lNcAM`<3#ZFR1N`I|#Q z^vo+e_WX|fpaQuX5c%2t41QCk#E6#MHbOZ(QDN%@Ro*1$KkqZF5+`>O~0<9<5oS3dI(nPeTR~o5Wm;8$G z>@e(ZWCtlEoNLbMLTIvWn6cSls%hkI831u7t{j~YKc;OB$|JZ9>cqmBXrnK=4UN2~ zxK73`PPzOPj;mQdz`zZ|qO##@l8ElEO~So%o9cPp<2COx4GOr6A%-o1;N=EVyC`~L zA5Y|VR&NAr|JA>g2!Q*r!rP^&9N^PtgUb;z!cxw$+iE!>>D8>tp-?$_vIbk2*^#~C z(Y#=XOdAnsOXvv&b_SnC*;;z71(gUhK?6kv#9G4|I(1H4o`or(Z9Vub5qA zxt7#H(R`6VGU-+SHvq>3xvTdE<&p302J=!)p^VNNd?f1yyvVaRA%`_dfzbU>Wd*w6 zYI-fAy++^DV2?D;mUx$z4F4+E`22VcqjCq`B@_7w#EP zr|%No*zNtXppUt|*P}-wtzou(T4vF}n|{r3Xk|X4MhR7na7*lEEQ~B%294KY+skUG zU-8vk5~e7pe~dd){9%r(AwRuUuBEvoL6`=SD+9US;JiQ}pJ2z_4GRGzR>zgPHaf!6 zzl~Io*qSVlm7C$zr7@nsrM29OzvT&T=HyA2dZ&;e{(NObZ!C@H?LDktJcCQ%F{hJx zB-hlYx{nwzoMvtPu@6A0qbkTcctsf$aaenLHR2$PK%#a;K+F16)zF+Ou*mFWBU z<4T=%$p=68^zKcMYFn1+yP2{IP-{VT1RJEY`UgNzEDxp9%h^QML2W5%tV~TO6p`GRX8au zFF8N_AzQbOV~d~LN9C1sE5ikG{A|>kP9P50u&evvMg*yGmtm^+jil4I6tZ3o<;!Nq zfTjA&Jj_%APzLsJG8uUO;|o1(chF9C%i}ukCtbV8=oYH#ZiGFiCYUTDWmb4wgp*F6?KrUwF-3~6@fV}4;~H5abFE_Y4JF4{-3BTC z{wnB0N^M+6(tU11ahks!f{NwWvb?`1B*`)3-iUMOz; zH<1$$rPZub-MsSeUwjo{BL=cF9=iWNGzds4Zb|vYzuuz%e1BIKP*I$)2I7Bb4BveU z5RhE!eA@p_jDJp&H$%Z$TzWog`nPHGkE5!ZfJECTOaSpGe;E)Gw^^huQq1@4w;lQRlzUiVK+3 zAM?}e<2e99X8Be9GNt|4MBlOMH^ECv2SD?oV$trU3BCDbp!wBnE>`3wCQ*9 z(@*$BK_>?F=t4V~4bXdQ%oPE=4}%?Y;a?&w^@%xw=YlBMi+0HMf{fS3gUv}1k9dFv z3;}`i%Mnopq%O|sudq9mi+)%qu-Yysph^RH&sx$ zwGI{P#y+@q)flE9_5NVH=GsUZBfiE#16ZVqJ+&m>bk^Y<5S3{;R$VB1P;*KyvDOlvLm7syOtlm{pTHcM*%+E z0hx9P#9fXM5U~k5eIf3OVsC7Q`>j?rp!hHj;Lf?}m-+(M5kX%L{twriKgcw|iIUKh z?SdutLpq4uRf2IUfxY>sMg$W|(s$vNhmB=sUZ@<&xFB~0*k$=2}<}1o%@UL zVCKg1-{9;>5wKJ(ZTLxm42mBba`jtnm;;nsK)U;&${~=kVfNU!rF?&kL7G=od)g4# z!8@5_&m`UwI1zib#_dD!e>fEi5Nq)hmo??u<9c82LVe0e|7bVx{DbGq{D5lnGC?9= zPNd6TnKpg23OxkGhfA-jxPeCTV23p2w4XrLd~Xo~Lm-pEcH%v7Wi-ZoyGzt4FhC17 zS9@38hb4!l-!&?Edj)$CxG}vC`avpF0%ymXby*ZHa%*Ql*0s0 zTYr<^SUx&{E7QWUy>awbZy+ClV`@Tvyy;rEWqqk51_^(VgQE0{Z|O-DeiABqNla?#+! z0$ebsDQ0ic1GHnm4Mvh)-T34^J(`r__vB5As7p2-Fj_@w`jG&$#3Q4Kuj~a0iVQHd zEx^0amPWZzqtMPXij9thNzZ1X(BlVTi9^=EsZh1=Gj?JfTzl6&Z3dDDf_|&d?<@xd zdG<1!P){P9>b6z`HwTU=i0~0VTenBn&XCx$p{Ul)CIhJw7wKN&b>koV%9AI%Syetq z%MKxGQ&s$NlwVtM7Y=>w>rQU z1&FkdGq|^Dpn2bxG%Fr1|H4(Bx&S8B#xNH#|0NT2_8a@iTDa1O5>od7i(ngFp2U*J zm%Pf;1(>HEza}S(3WfH60k_kneCv!r1BmGncz5_5o|N=8hkU3sOFyRR`kPXQgEFnRH3(5ObuCAe>7!Ifa(q zFbN940%BxqD3~iichT4^av3n9)Lpw^SB1s}j9KgKJq5u(9=}~QeuG@D*5qDbKG_a+ zXm6ze-m0|>PBU=x@lbAQ|))S2+!22&p~odB!> z$_p7Wbo=5wbw&?l7WCrvH6i>=_rNZswuBz;1&|$mkmC;7R1h3dl5f_YTWs8&_FYqm zkh1vk-AlaahG18m^ym&K(N&3@ZYS~5k^l0JaGL-FqO}Sj3tvg#qH%s8;rRB4?xTxy zl|R3>1{~L)lu&psXxL4*E5;#>gm>P*_)5g-90YrD^(%= z%kXjeZ{0@>F`1_6;R?$!g3IHM+&Wd#zw{9!9^7O#bXE5DqjU`gQR(878tcEk%m%QU zugob?{1c5oZK4mvTm=ovO(iFne;h?9Y*Bh1N83$azv5_rQ98}+$wz*vjo<16%NL)< zlvuqVuJ(@k-XhF%voFh6S`~lODO|y0`yn=HezT@owQB9EnlETG5?N!eA+pSV}ezIEoIg zqsh32p#nO~Qghuq7b%gEXv zaW)GTJi0lQ55muEcFszp8dlq6UNl{fLA21FU!Gj$*jexJY!Spg1@ydxdu0D>f6%I> zoi(enObyagS&dBEp1v(p^wd1ljXiy_XKa(3Am`W3U2RAzsm_N%Vryb>!Go&-oy!?- z%dg1Wx5P|eK01 z#{cwC+>y_(HMjOC6#LrkoNP2VkmH#4?17FChdyBDdqk1WfX=`vv?d*#`K|2B!rZ{l z=-`X+FX^G}A^>e68N^$6iuMe$G+vvBuYUzeI3<#k24w7OYJR6@j(lq(YnS~fSt~~y zldTwhB)(&hroNBZRH%1arR3V_^(iZl*V3g<_ArsG{uVECk~}ine`Wku9h2B*_x8v4HoIg^Li$vkV8iU14i1%eUrVq=YZEjk z8s+cn9n6#~^{xf~Y};W7SJ>Vt;WVrN&c-?U?6>3fMuw6D>chRl6WP!wlb!>=v7_2n zZ)f=8DjrF+zE z>RBOscqx5vm`EQ{@%7m_7!9RrV-zb>X9p=*P z3WUI1GG5BhKiCGyVYfHZM_>T*XIqX_k`kS)%2?nJ()X29m$ zLp*^gW8I;AO>wr!)w@j**NM2TB^g#nLaYBFC7m>whZ z(a-Q}-CMnb&$aJGfC=a(j0FSoRbTCo#>O5UVN{F?-OWbnZl7&=ym$ir!(|lw zrxtEyF68kAD4v@@ZK|diC+ha8Ijj?}4RItsyNmF8>Q_7EkGUi`HFSyl#1J^+bM2O? zf7(MUS-Nb)&D8H-HcQip&mT?oezEM#e6Dsn`AyjLrS%e^2;xSgPKaEJ=bs-=n=};u zk{sIZ9qPYv?dJz)``srorFq@fVeF~I#Ns(CZOory4Kr8xl{IvQ=1<2-{8RKLy?z*; z`hxJstTwlecSSUl?DN(WUleB>GAVL>-*N1VII?d{2*D;+emK11UAu2iXO?m{IPSSF zLKu*^$vJFre|fWJHk+7XdHFQwhlC^{pLXO15iAa|iZoG;k~Zm1oZXgZqG%GFt>TwH{NjR4aQDL$DUJ&{v|w(6h%URZ~yb84q{G9s(2#ted0#L+spf zMVf=+)1wlcXZ4xY38leEpV+i{857&Z zo1e|Y<*Z(cU;;I%+iknxlvytq^rYvg+T}xLEm=O*gyaF^FH;HnE=a=Sqtpb2`(@2C z=)})O-F2YG0e+T%UZOXIzY9UnNj>e(*dGe&bEYrJ1}&L6N*B|jFBY9QeebwHVsP9>ZLJCD!+(_hUJbfMPZQ#a#gfsU~nweqX@@Ha7YBG{|a9-I7~TV=-@e z!Uv(Obic@BG7_l^-Cj&e{0$-Q7Zm-ax3t`wmBdkN=pVI*hzt`y<{ie0(PGoaBL?F; zTkHAosUEZyn_oEfD7W}q$cg2SL+`#sjMBXld7EuXY}P%0XA`gZW}vOQXE`em=tTKY zK{&yg4$j_uBP4lI%Ocd4u-H?$d%u3qx7JvTd4p=t^!$KBt>04PBYTtEF#d?@T(1yd3QjqpC20!=?PpR)HT?_qE1$V)@#* z%pKul%GG-89-XG<{*{YsBUP7w$NLxP{|=v0U#?!;{q32WD-hwkqkK$wL)$GmY0UqMB1XCSlr8Aph;#C3SUjwus2p3uC2Y0Sz2alo+{*~IPn z3_hJff%iPKX_i_Vy>Yz+Z2+Q{qLxd?KU$C)110B$%NS{L4lOIJTv=o|c^gkw7KT&s zFn-)E;Qzh`&DO?uW~a^E0H%EK|)MDe%7_A^av!PllZ;`?+;n zLu}>RoUs)xSK&8#+HRiBQBiiXxjSzQTSG`RqlWTYpPSOAG&E_4W19qYZvC2Y+j|hm zxM94{Wx%HQ!+lO&;xI$g-f#e+a}+9&;W6RlKNd3G%aQub=A~$+fUog{Ytr&%w!*q6 zotT|e!~4O%|sQa{FrLLQrOC$Oj>-TOmBuhhB@O( zo2e95N^ASB)ag@wdXwku@xjjS_UzI~f=%4)-ZfOvuNrdtmTT3vj~ zr$tfTF(}h#wc53{{y9_6gsQcZpfwPSxs6zE!NET?)bR9dtY^q$p9*9oI>2Rs%CYZoowr<9-_mSlDvr{}LZgN@EP))1P8cCGtOzvsNEXHXPN7_O{ zX~fqR;zRO-BO|7EYk$Km7gOU%(uvWdtA#Dh2$ASM-%Vrx zjyY_;?rx@q!0!*}R8e;9kYMTcDR1#f$Sx&Tz;mKQ^R#im@=NyJsXn$}R`rG_j_w}E zsO4Y8hd)jxL#QyXGWkUd*T-xtXVNX_*w}o;PiM1nU5avK^@}@SHqJVf^+pY(OP=RB z(TylhhZ83xEx!5{fB0o1X?ocph6%pqi8WpyANLkHDsE5grtoqeN;fIR`~W=~{|Dc0 z;n`mEx=caD*5%64!sivQTvZx_v)^Is&)3|&);?r)s-er8A&;ViiBe--T3=y~Ui53Q zeGH=^5~hwM72x2de2`(F~rsRUuop?HC0I2=thaZY6m(lCW5W5F@XZv8B;X4 zp6Mv}c17yzPq+KSI!`D--x0ooqQft5yO=fgXLp_DcB!-_oM)QS#6^U~PjAr(dg*vz z6A!=fa;EOD1@zKfn{3=$l>ee=TS_G6^D_U-`PkV10JHb)E{Ht|VOE9bMH0y855CX* z413aA&MEsU>>kbmZ3K$|8K^2S(yHi(T5K0i$QbXZc8J1t(%)D6?R!LoL@#@PTZ{h% zAX1Lf{0|yw?nQr&5;DkXSuM8q5V)QF6Jme9%W{{roFR$*+~U9dPMPl30D6GnXrf@@ zZ}bWne0MjYKAKYlnREZ72VOgaBNSVe+MoZ)4!oX#nq>~FpPl>9<^JzanOP_v?VDX* z?0+}U|94QPk{!?*n@;D?|LeqCSizB+Q4Q9ArlSGL5+ETarY~Ih&s*}BKZDf35$ED` z)qnMIe1({m%1@W@{v)>tXaMV85{%xa?m(8m+m8A)gJ-D)4~>=FU-W~fjAFJ$cEa9N)G_h?09?nlmBnCb-31o8-&RoK%FLF zNuLn++;|MRm>z!v2~FUabVDwxTvbkkNk9k%3HR-R7@y>n0kq^m?Glh{iY3gb8fedi zXpg|&9QRbC&7fWMe3<@Fr5sj*rA0GG71Xxpa z-YyoXFZ%-En)-k^`w&FN@OQEORREkX1>Ftc|6vQ1nI!R$7R0Oq2<-*s_dyF~b^C}v z@XnY$1wr%?A0XR^mLre~FOI$+1LDwT(l@6*)*ozrmk{b4)w8C9qRTdc8u+_ zjW&7&`v^0>Ge?Uu!6JYcjUMQG+yIZX0whP>Vy!ZAKNZM$sOJx~d_Mzalr6OBsi#1z zO|({{&jv79B{+=_5;=ebYEbXbgk7;iUb<(h5kShTz@3={aM2&IpmB>Q01LAW0m;<8 z>@KIjl_Xcfa;UPdW9Xs?a>8VDMRS3@;Aa36?Kwwvm&qioHb8>E{@Jf(fUyCNCK~{` zj@aPXXPsM|MNg zQq)O23W_jtp8E1i#Dr=2L*Rd3RPfDxE)dpT8Tmz?5I(4e;64OK3vYTaz{4S3ae)U0 z&m2QOp8&Os+0uA@C~3JG3_Eb~M;m8X0UMG@NN~_%0WVfL1hIKR4fu+YSM@a`mY;m6 z98sl7MFU!5638)UKev);IRlo28ad#IP6L4S4ylHEz9S?Toifpk>yCBXeF24MiB=NO zTQ=0u7Xj(+6&f>!`5N$w`5gbD&@kV-^fulc&?B%@_%B}ow1@NtzYv|>gfLq>crnjo z$#L-7*@4k(vh(x=ZGjazS{Y!*Bx(y6Nv6{vm|8xX1K_L5ob|>iI04cIth<6UJ#8Fu zZ2qiDBj|}~J&6r=LxYQ!huI1l?F-Uz#z*YOGhlIiTk502VZMXn(JEK45b#YL!=84U zad!`VFGTxl9g`<$8>(4EwP+4LdzwBOauzPI08^9y8nw4F5!WD92L$v%O+E!IXB2?% z%Xg*~*RI*7x-f&?4)y+3IciL>e4ca5ZHqF7xO1`LGw)<2Mx;|+AXo$fYU32mqnB)UHbr}s==_(2-CqOGjthzpPXHT@G}&Z zg8eo3)k@t3%r5qza**whdu=oHwqS4B2WtO2Y|T*>oLnNg z3h})nC_T>N2G_wQMy}7mtGP~=HWLtdS@P#*2Y}I&vk`(oZCtEqIZ!0kE^5($2dm)f zyRssq?!t{fv@|xw-%?|XI&o^iDb<+Nkcd5uZWm*vGTS)*UH*jPQR20SG)0z_)9u_P zH3rXQoCE6Roj)_l23&qZMDO>F^|munGabbf@SnBbFE4`LH+i+<M!1jyxYe zeE9L}_v%mLdGD&nm?Qz4mvTeys;1ZlRLjD8r333>!ibAcSF=~37P~ke=@bTgk9|!> zStiX%-3qIFx7QTIHs#j)pr{hI+z4KqAz&JtvL|!jbrbH@l)Pz+ZB%qx1Vl?>>z<8Z z5qobOkZ~UcFp68se_4F;b-KV-4@PidkXq| zkx4aM%1FSba0c1a$#1I>s%UUx|vzM{8x zur07YCgX>wOoST7_LjV^fC|DGwe}20m8CX(jt=i&qKf)dbyKT=7M%v4N2H6wYLvSj4Kh`oT^iZEYq4r7_ynn;p;i zE?Ur3x5V^{`0ISZ;FX0lHg4l7TQ1i*li#-n6JKVyg=ygTS|gB-t8FtGTHC2GikFmX z;v_44ueIOlv+MGI%w%Q!@3YZ=C8M29aX!%cy6g_)(xe38Fii10m?aRta+7IVW& zbSxO4n>zDi+ZS|aFBQkXt#KSsr2I1f0#54OX4iT1Nz2`<;?76I1*kaFbTf_9L z2K4UlS1_N6b|h2LI-yz+{Q6(Ab1VSq+^bUKSZhV~ir&~mxL?a1R4|^Rd3rHuq8#%e z{w;NSY{`Yg0_&&>%Y|n9$pKk;c9NI$&2I%q6<}W2ADp&Z+>4Ip>VvBNEfwx4oSa!wv>qBwANZ+j}Y3h_n%u zQ9|v$JNm9g&YLrzJ9c^n$8~YEF{m1=lA5ZSU`2NIU%Ttl*%wwyZt9ECAwMD<^!unC zx;xo>JC2PFJ(p5p1IBg@E;xi=BA6QfZzMi;xIh%4k8%w?PhSC#P;8wKf7brK6Yn1C zHnSBDK}H;T7e9-NTm{KUS`ZmVXgOb`1z?$Z8p^nMJb=0pkWQ#^-7@v#-)w`wc?LI=W0amsY?$=S__ z{ewgX-+is{6bJl+g__8iG>{ari_nS=spes8x|#SsS>!@YzXdj(OegAfyFt`Sp(TG~ zoL)pX61dSw%T?O&w$v0z%;pW<5i#0wmeDm0EHe!C?)p+#Yl_~+A(+vtT&6y^lsc3V zT+H#3BiEA@@c8)2m=aN>%b46a_Or_#lN2%EJFT(ILx+k^(z7G&FWg!Naj9+^&e4pH zDg20t&u_vzuF6tpJdks>X$2h+Me>cq47`r~`zbzVDDMTTmsA7i$=3m}Y}0Lfiv7`R zOl}x4d<&uA$Mcu@GWgDg-))7ZS+We|C}hHrRSPIPzx0c4FP4g(j*j57F(M zAKvRk+pgo#g71anR9vEGud{xuf2_`Nog9PgG9^owZmvttu!~)16*ma=nW+;6yO?r> z)n!XmL1bS0BV(anUOE31QsO@0I0dcX<`Iob>m#zLqMFzwx0 zUc%Xqnu%mQwh_vt$jU|23_bHcTW@7+1PNA|)xP1jkdjbm`w`~E8{R$QYRcz5*c2$(V47@V zb)Z?N+}VvQBSqSC0D zEOTVbYn(IL&E8Vn4F43lsiHR-MFifysXp%1%a_^j#6>^3h73?en;-S2n37hFekusdONBi-8e zR8r==bA1YUjcxEW;=_DqD5g0#Dc5(z-Dq3t&Ce|D%|9cQTg>6K{H5139yNsHJ|;^^ zJ*hH8!*SOV*`H-cqra!fieiV$+PYOvfO_hk=)O~81x;30JpQ&I9i=5rnuCFqj?XeG zcRsWimOOXKGE5}>g{wHCWvBn1Cib0MG{vPyZ^BrO(c*@rISohEkJD(}5923OiAlvB zVB2hGDg71`uaDmqnn^|Z6pbWCZK$PM72!`gb+2cJM&=9O&l;Fd)%Z^}3@v%DZ^48X z3h1aGGzS{ou&){@BTb~<9&~VL7`!D=OE^`eI#p^Ft z@Dw#u*)S8D2dcvKr)GKOi|_V-X?3ZN{QyBWvFr%uZ)sIB7Qc6lfMCtC3R$4`ArMqhpf=J6Eltw^ALX<9vg-A;yAgM?S2uKPd z2xq?T|MzWl|Nb}U>Rg;Vwy@uL-Y4c9bBr;&EAnp&wp5Xv>dsU`>4=`9T*N$Quk$^{ zpld&8y+bnEQzbX=t1}QkR&tH5%8k(lH18YcR#vdp=7mjgaFx|mp@~IJS)6H`*M?{n}y=F4+ zlV)tR1;@9;FFXkw<3F@uCqtxPotC2WO@5Q?9{ktT@fG$K@0&X|ZqsDS*w4X5mMxVe z+hnIPPt;0uO}H+c=~t{TAPO1heV~PuwI) z*pvAIR8Ovn_k)f^nl4xla?!T zc>L2Jlg_WXJQld7fU^vOzNBo3JNbcpl>^w+;0J`PcNNiddo7m__2rdzzbMMT(h)&2 zFMcGD-Ea8Db=A0pLhSlgLq=P&S5!dxQE(~8Uv7%3qgupc(?97`wane7z%;MlmC zP`7uU)@%OhtZM3yeV}1xf$a7?VoQ5JY9$>{4_syUQ`J z%&*qXto~{%-XcntQ&+^m>G9#Lkh%8@&ROzuG{s^N&8_?Ok^vn7x(nz4XWFz$7%~A` z&zXk{fX&h|>#Tr>vsJ97!WASrt%hOhhaOUoZt7ZPxwc=-$wW}?Xqsm)lRm&REdXHl z)mreQ{Q=S=+`#R$2jtWDF4(u7MFx3i)lHHS99As*$RQP0TGp3#H!<3+u@;?W*Qo0r{ruW71!khNkC!q;oeIU6E+dKQV5_`3M7GY)JO`u(&)yovfh9qez5 zAL#*ys3e3Qk`h^593{T4L~zmkv&GiMF=$^OgY>3h*B03VgQtfI&|#(w6zW+z=1J!- zSkPv;!3!%x9l)e}4mKJ)!pXbyPeV$$nGYnkhMB@S#iR3DS?4aFq>Y6pYW3`2*V}I& z@QhF}ChO#~h0!N&LXaE*RuERrWVciRQE?uHWmp;-MEeBsweBoSXH2{;rK1EY`EEvXfX8sU7WAgeLua7wu9RughIYI# zf3lxNie6Ix8lXt%6s`pR$qs5l;P#d6_jB>K&IChngE@dp?CwMoC~g*P!1QjeWQ_EL`g-GDhyO`B$}8)pn&{ge;L1=2 zaq`@^fn)DN(Wy{!80{yQnn=xZxoDTGSuG9_hmA z5l1;bA=mf-QCX&eaoG71%Z6tZ!K-=ZC<|-}k`HYi9fy(ioMON>L)qzW4JFG5vM3A6 z$pJKRQX3^haFJ{o+IBJf;H_AA!;}{9KrWm--_jFE4Cv-7;|-3+Z+SpoH+bK zzNejmVNAwrwm>fXJlxDX8D?&vo&S?bm7RK)a$%8UvP;-j^`FpA`e>t=|3PlHy_#|6 zLxuM9rt8@y4Eo%CdtW{3l%ZLf6@e8Z#n{lk(IUhfO!6*t@_2-lco0gVZ;)^60?jMS z$&Y64=J>@FQ!f)Edt&d_RcM%DLnOR5m4M!CM_wFQ4#K8NTrP9%0ioB5X*4Gp9Y8+#kQeKYo=gQg=ze}9#KFXEoIv1<@M?)JrC zVYu0?e2a@Nq$k3eedX4&_xVDb;xwN~k3j{x=H=qoZ9&DVjW zByZHnX2act&)2=Kb^{_41n?lBSKyxd*t8lO$2LpO;>X^i@t?XQ-Am%> z#8b3vMuA9q#0r6?pR9>bJLNGqxZG#iXTW$qaSCK9Wy3Nt*Cvw1I6J9LXhq@HgM&%h z7IT@v(;2LZ5X(14ACZ=U!Ld4tXSb#J+l#Tc^H)L#R?#k!XM#A#8u--cPL1j^e`NVD ze$3q4j-%Uh9nZAEo|p`@Y_fFQZNe&nXHBk9fVUvKYf>@!yP(9&A6jZ`lFSl|^yKEi zzdQaoYatrSg?F9joksX{b2W*j150?Rd5uH1)129xZfI)U7)@{sW$GhoQJ&`}rCw{* z)@TT&FmQdhWspmq5%+`Jmu#|Y2}?PxpG$3Gv$oGmaE7T+w@^KU>94iMrDj*08>jSG z=(W#H5E(zDpL~ya{*h}26800wJFFlT5X3{A{Tx+DJEe&^73bLoH(234CNadzG{<j^;br5zF%$3&nKqtbm-^3hz{M36B^LsV+}J?9j-nB3htu>8UNMFVS=%%5Tr) z>$y-9K24HTl$i;A6NAYE9&A9K|zLmsnWi z@Av;pv74}6Ndj#~(z4U+ZXWv@+zI#spcKqjXdy;3uk?nf{o9s9u*pb&8e!j?I->H> z5u)Pek7vr{&mP8wDamm+E%Ao-svbVzc}P&0TR%O5nY5aD&)&%*{v5Lw7K(=r3e8y0 zl&3ivMUy5k6YnXnIIiKq#+&; z>?S#Du*D2bCW#|eB<98M@tL@s3|ad^qed1P;$+KWbtzbcFF4*c(0_~)Wt;Oa!O{X!96>dn2n1Q&(bGErFSuR0d17mgCbqd1d{X!ceG7oXZ6gmY-Bzjf$ zv*Z4o5t2kpdocGFJ^u;0m5cDDX+~llvlYWC=TLSQIY#k#p3Df9rW@TR=$~{M70?T* zk?rUr$3H6#Ng2}_kZk7{QHeqBd5d<3wPOAC5Q|%v^&&LBeG8OposoNgy-NP@evTw2 zXSW6(T>bMOnR`E?$_AENKb}sTeR${@kstk^Lh>r+y5sG7H~>L%oB5+hIv+=W zz4ZIrOCOa@xR>ERzrH(A$MSgtKdp`X+ZS|f+^`Eck>VGez0&n$$)}KQe%y~q9xp#Ku-Jh?Rjt6%)xHnhu z=ks#A;^GfwY{4mU1zNb$RC7%H`~U67NhT?OnOAaE*Yau5i+u^OXx74W2CC28PddmJ zQ_LZ=RR;?qbMGG~>CnG;AIqOk&V?X-Y7y~`6?8IDLmH3%fYhNM2qBLo!F8?PbJ#x4 zyWo54`m$t*trIfS6g4mca0LK+*PAg$a#znbcNL?@Jm1w1f6_*L|Lle+>iloeF)0mB@#@6HFih^?AP7o_F!jnRlzM@n$?3-BeI z??K3sy$o;R7e~4hSNH%d!qR?*VH7cL@`7TeGhLvt%Mv6LLa9zIRACxl;YC>;kZ#fa zNaDHOh^Ho5j&Z!RJ|#Rr`ntKK#GzcAZ4DutxcJTro>Pz90M^Xx)SLA{QZ*$cN#~Sm zy$vGFK13_}v=XH>dT)8P5|ZhD*s7isfXh@$!9(|@Xrojv`OC_EtKCz%9REfQpyI^L z{Z@V%NwSgVKdzATWc>+F1vT z&5nK*o&SbT0^Eqzm-!SwR7*SqRQa|`gRk*>RzPw%M0ZsT zm<2hYr9F>;Z&fi90MF(pUzlu)L%Xm?>e&XslqvgJ*-SX#6t836_VNy%369OD*LPj2 z7Gi|~mT-A48b%<#`o>-I+bHqnQB0itrCr5&o*lXGPCV*OBAtf-Xyjb|hRe@kz4n=S zP_F{`k-~*uuuL~d{Li#xs-TsLty2ZWeNmj^*`eLAbFlvp!65WJ&NeuT4{%zF41&Nm zA(8S6BJTg*%Y@;U(RHjp?lR1tHV4q6_=<8WXXV8GC9{ZV{dRwbMB+9R43EOr87b&2 zzW`83&salv8wGZVr-0AgyG8kdg_td1PRby5?=dPfd)T^9$zSlD4uB(}jdKqINS-Anj((A)V z(7Ys0GI@Du8~`WJtWOgqgsKPHzH9K-(I4uxz#+{@#LXsBVtDi!WXELyS-tYUhD*;$ z-RcW1|Jq;L_$ub{lAq~eor)Pr#{DKgz^lj0Nac@g`g1;r(Y0$6vEs}(2IEPMii~s2 zt`v>o`fZ53{{|q6P4LD&OT?0ZTaft09y3x#Ia{pfxh`2RBA(s^kLUkF?Si%3dh0s- z@T7o+Nv%5#I3zOG1jnO>Gn&z(x*6qz6~=~#V?-ogkn+!#x2wIxu#YNkeyn>&9AmZo zqKmr`U!-XVIbyLZ@wkzwlPoyyAbIT^Y0Xq^HhD-#XwBD>sEHvmZE##uV&o={rg>ak zj>ldk?M?fO8xToDsxw)INY_&-W%bH3ECa|C(41-SOL)d#JtB7RqMyD&JVbMnoTju? zVJd_3KVATyH$~M-7c=ZKn_Z?(gM;V09=bm!8h}fe@TT za$v`)l8r=kjH=K~Vv0kl!7SdU*T3Y6`cUa|S`TeD;FZyRol!xY8PnMtwlj^q-n8v69y+mY) z>VgB{1nQM?03aYG|}D>XAPbxs3bxyR>(6peD@4UNb!&Ig!zduqD;s zqf}RuqpD%L*eLYGD)~)b@wiZ-@rA3xUlDW17zX!5{Rd7(==DofD|Qlq()EFJSwep- zba#N*Cy00D)p>m(dLhNIj6#%|FW%vUL zOU0S2McgrXr!|!gWbtEqkRYq092Vf5EiP>~exSm^r-PrjtDyttlp}3?=P*V?5w(7j zP5cdyi)%=e-R;CT4?IRlob^*i5_A%i&fwWF!%5Y#*iNT5TGv^)mr8Kl{I2;4^U;{3 zXd_u|o}<`@?wxU#Q@6`*p~kco$k*C0^vRD7f4+7&bRq#JkHn~q$pHnmSm&7XT(R)c5uSU!ti`SEl`{ccIx6Q^6 z9VtnV_(@iFx#72iF_`0MfYtzk-t#5fzpL}xXhlm~;IdkaDR=)oRMPtOcXYmID3P^&&8f|H?YXNGYF=CG;Nsh=H$1VLhTit^UKQpas5=+XqrdFBi_`*uNi#zsD!V5Y4;7J zMx%;A`s4-RHGKss5}Y*(B<_3Tf22 zMH)qY_+7a_kXY&tKwcJOh2d#Yf?B10@ZiC$Lmyd`UmTt9M$P7e(6RpEXA#k#Q(R-q zzS!3G%Uq9&@Am;Z`G|`cRJaClenJIN>mzcF9EcHad>ns4jT%AXy*@co)!5n(feulS zCCb){f>*5D52)+)WkSJlphM@cL-& zs2Uw_!?^@CohwMAk*^xb_SFl0XRfOdKNe+2k|Q7c;uRIXt^o!@5)!M+7MCGL6I)VT zZGkF4npplk_JwKlM!Y5FsS*BN^q1V6_1=@+*S~VbeI*9+uGW{2HQT+c8LhGZEC9-* zNY)qVyUrc{SL0UQ*GAVhrrS3-J}3K8GA-9EaS7_& zQT_SJ>2{4EiAaEUwGTI6)&0!BSz=!r5wb2YsLjOp5!CU0Q1DGTz6>=a{-zG4-4uvb zt&or*W%4dGAI^gIHiT;>v4q-^EA3-8cMCnnB_OV`0wSR*grly>;BsYSCh)d`111U* zd4~xWTIiD^Wk$&y;t2#ECI9Z~>&RgYd5)hb^vfN6#xf4FmtR#bm|%6XTS#^@3a7&L zLc==ztWV&m1A%XCY$P00{17|6$}#I)zoyI)hcw89w#01Zg;9jF8}3jn4p)`6^@PMH z6NvRijccJaFh8%4ihbx7UCe8uPnjr(gu9}%_Ni?19^*f_ibjldBdPpadbou}FqOFy z+Ig$?bAanRrYEn=x}!g2oc!kZsmE`%zv5)OztzF)g|{XSPasiB9;d(B93>SP92*l}Fk&;8fWGGD>6{O{G)I(b!L6S9V%n!j+aJ@X&CIvKP11W5ESXQV zK22^f^%)l^chNIuVY8XR+6aDn`fwj6VRzaPXf6biSX<=BBnT{&ks?7q*qBBmPDM*@ z<|1AE{h0`u&?P*L!438RtN9;w2fKBfa6o37M2cj~t`Q1P+;fGVA%fz$dOO(V=nk zf+xR8Wd7Dv<5k^vEzyIg>Sr^gtL1a3IXHOjR+fU8pR}<&&inE@bZ8YPfpk)Y852Dc z@2S*$taR!DTQ-tKA;o^6OW3zh#=gw2&>aN^e#v$iw<=Z}z3aPO^J(^^Um#!bTyTgX z_7U21+Nc)&0Tv&lcL}Q1RAm*(H6*+pC%!{!;WHi;R*ZT$Zk3bjOv&ttI2w zQ=SHuUUYjr;3g?^KoNMnF$@OP{oslIu!H{oqn`jWOtA~jt4Oc%awbKOLUJzVt+^)Y zge?MR7x7>45%g=#*A^15N1ekaP%`8eRRQ|Y2f1|Y&}6RJ0o+m0+kP_qF6_4=%kZh< z>qx&;0nuSjFz7T5lxr4UVT)mDFf_Ug`WtiNhaU4YbzriP#BO<^p=vSJqkLUqX}UR^ z_ATB!6w?yCiwaUrmpqA@J)fDVpF4ySZtU0c|QPXCnWO0C~qYw$XhCHOiCWaf`u@(i(i9qt*VZsM&5p(dZ(C<7WspVrROtX2IM<1C=5=33r>x}(IIjzx1 z)TmF5BvYkCm~Zdn-*@YY-a&mkUH@~aZePHO;N8gO3*878L85i0;|G$aKj}$P4%BW8 zvN_wsJ9H|QHFWA~FlCGI2|Fj?J6IU?cy-&@0;JV7B#Nzk;%INo@Ogvc;858ZpA~hqfw%Zd%8i%u^M_<3ne_qNg5u6j_V?6>0R^oTIvxZm2)hRuRg4-U2n91NG zWRN8ZS(SAP!irElx>4c~43pdv<7+6Sr<4KxPqLCF+41tm!lc4 z5k3VWx2>hr;eP z^(J=1J_H@}He1+xgI46DvlYF7^3LY-cS)#dnt#jcHMG%w;(#G#A@+?8MlnT7RDSgo zV)(t2Jh7K2=tR033HZb^Et8XWH=CU3WQ{&vdETg}a?TNz)jSe`%T<*G4FUKR1vOK` zw=0Rtk6h1SrtI(B33HF++C4LsV!gx~LAvWj-zjbppgLsnu+*U=AT_9c>Z1GmUPigO*$*2&1N0Qd%Wk$B@Y_UeZ~i?1|zrG>g+3 zmC*mTo>?$kQbcH+$=dta>akZVX1RHTb5Nkhb_V-0)*^J>nOgC2UpLy3is^yl`OSEq z7wBs|4N+`PFUuk~;ViQ;Cds^-5iZKz63g?^HHD~8E$6r&=DgYM?pvcNRBza*#??oi zA6(4--etbUltSTPldtpEuXMVQ?ZMnx{q_ct#O9rAVMn&(6)>zPOwNiszlyN@PxPkq*P=m2|_`0cHDlYwue`3gD6$^E_o%r&-O%poxu?pMW z50l=HS=oFqNbIhq>uh=v*vF!yP_LLXdG=A4nem0#v2ts*mCX1Fy`M7}&Lyf`Y@keX z6Tpb_stJf|6u%xhfAlW~tuq%;nN!+I4Y_OAf{6EZwx-AzgI8rT5qlfYYnQOjXl`mA z^t-L*EAFdjroOS2*fJNI_n+wzlas(aHgMm^mAbAp0MMDx@X(m?R`<-sO8MoVfu$U9jIqmh*bPG zLY}{{m=uw7<<2HmCm{SDXvWC?Hi4*V>KHm-oQkh4Pyi0akT(0(PzD;xS0T3|LoC+K zj9FMB8SnDgR^Hq%uULEW2$~r2r9#6t&!hC_E}Xm*-%<7~wKA;lNf{?+<%zfp9u{YS z(1U9y_%Czb=ZSfj+7&|u7$Y_GwiJDM)E>H_2~Osa_0)9JZ< zaY9Ju6|PIx-3F!BL=%~66K5uti)=0dw9|Gksh)nLG%bLm1jBkWE zedVpdvbN~Z&rv_&2^1)FG8UY%zB!(g61mR(jCUnm-qj?f&h^KgdU3ynraFXmcH3nj z0=J|A<)0max?(Oti&rKe7|{5tG{&`==P(52Dh23U)8)ov0=L0|b?LpVWzkq<0`^gN z^<2WO-o0xA4LWUK6U^#%E?iYC;L-ARRQSI7CL;0dS@IRi<<4UQZq1}2C-eG_Nla#_ zbT^_WE}NlGu3;JO6O|O1Z0WU}4vx;!cRh1Xlp1$YVPf(l6E^8C_h^vwtrxels)^QB zBtu6S%w{(tORjAz#0PxAKI=|7hUsDVpds2iO(E>;%VWZ~5Wtf%p++q*s)laFJZOaRi?ku@pK7&OSJmv zc|l3JOWdBEUVL>!2r}~u!F&d7ukoC4i#U%C-Wcwc{N_bB$a36PGYIM#PrsLiwaKa! zI=Z!DmwK_YoA$FO1@qw%SGqHNzPIeT+1~lWr90Vr*qf+u>tLbNT&L*H8*V?%$vm6; zsUU=pq3+eI+WRNfx#>h?^gPw)S1seZYv|4sQ|pVb+z_|?ZmxH+{@r2Owi z3AolMK%C_}mH%RIRP(^ih3sj<4@D~QA>dl&xSKC(tELCRf~0!-RNdicxI@X}2pAh- zV|l4H38f%H_$Mwi9Jq*7D{~z_yX(0P<%qpp>Zo>VvZ~c?AHQR>tZve}0=c6X$wYT8p$VBIp$9bh~hY?cbOWmPl=yguh zt&HMrXQetod~Y!P0q+r%`tR@f_r?9Yi7c;ZQj`4zuKe_~-xeM6LX-r!Wtq=uT{?MO zuET-p(r6T;WA@L7`}?p%9p$)a4b28qe|%d^5CqQmCN!LuGx%*s_(dXeR7gKB&tb>k zLTQo-P?DfXfxEDnNq)f}k95>O{_)Sm5~Ks-0#Gtk!D`1d8L+EU@7F+DbNK0DXHdt{ z?Yq7U#K~E~WZ;(v=TwyZi&&(=*!w6$@JlZs;NRzeFU`Ps1j{1N4}>FO8o@ZNLjvqy zW9855A5U5iPwQTdFhwL-QXCtc?;dt?`1^qxiIKniZAwu^Cbz%%b52PoNq2PlBG&#F z_J6OpBSR$J;c0Eh`{w{hOA}d+UPpQd=O}YlF|T}j|BQ@mtaN{d z`|0jm|pTVY_ zLiS%-Y#^P+?~@$a*#0l-6tK6DaTok+sK2Vi#`bgUEy=$IjSQCkr91k6F#lji+q>)lzWtZ3iahLy_9wZ>fbsiW9JZxdP)T3kX03SL6Tn zj4}7drvTV^BZyb8+GApJ53LDZ%wPAnI)Vn3rTn(hWV|(cz_?JKU*P20kLkfHwJ|<0+lz zs6-3&s}lLj^W|;b$kW#6&oVU$hdIAdKHGMvsFHyvkU*ZmhEMs>JnsGLh~Zcxdn*&? zZcbpOd}n0khbNzTi>)GvAQ z3`47zqKeSXR@o~&-qOKK|7TxM_aXQt(%n>+@7%YmAVIEF z{_9Hr99?B6jvTkMDT~R9mF9q9)OG!EC;;GAA!YS8?f4Y3(5@$fi_e7PwfDh1V$cjc z(d_wi7Y3Z{iaAQ{i3E=USpB(f1mSz?%1mf=e>RYR!P~PztK3C z{+#2@>c2MpGIAIaoAQLLe}zQ|3HL}a{h#MxdO1*S^Z6D zKQF}!AC3IZ_kYEYx=3VGKHAauuS+5P+rJv4{v1CT*)!$&?+xQ`{xdoL9`sZ2i{q5O zho<@8JLVuGNQD3Yr}*zI`QJZ_+y1HmU@)B3Vf(Va1#9*!%!kW&sVc zaxC7{u{>+%_Zb%uDSiEmGkEerIl}!aSOuz)m!SsWKPn7{8FY!UV8y6>$!7Lf!K`ZL z(E?JwNoV`1@fJ~wDkIhOQJ_)pBbQQu=I6O`C6Dj zJCw6)CC$eQOWH3j06Y2Z)2zHYD1r2Whhv-}#K|d3kCZvD5Q?u#Bjxt7*83%sz%=cC z+L4K{Do7eU!3XFHRG{@k)~W(uN&=yEgbh<i5^eW(XLAqOc@8PulcOvnzL zR3d-EfaE~n$nr}biU@0riXv2Q%I#0yH8ct$d!RGdgU(+8qsdK}dvio%R2C^bh!dFS zo}Cl^8t9nfBZa3z2cRo=8DMPLh*HwKF3HgbU<6h84;x*CSq;&B>P9Wh^AeBmkbv}j zT8r)Xl-_}=b*p|t0D&m0gBUcI5_9F}9*-~X=@H*h$yjDdw#?{^kL-8exx=PRSdwWMZM0-Jz&<{MQE^oF}9%RRS%(c*EWor zV6Izlx^CNaOeNGBwSW-X88#^scHo5$SZ?1u7frcKZTyJm4XhX4<}wv*6Z-us_bruy z3X(}IFac(}P{pecc%25oR3N6~hSlx$UxZ_nnb?#sWI#PvpbEDaxXMl)qmHNyJ>4gO zC?SP((wb#XI%#)VX%oCN(-3X;vPfnE7cgi%IB%WLwhbYS5eR}6Sr-_6Ya=BMJr$&`{XH?7JU3qHa znQbfI0_A{DXpPc+&9q50LRm`G+S!X1=P~^00&PNPdS?Alx`udxme4Rrd2F#$?nbCu zd+^)9A*joydbHZKdsN=y!;YiW60a%tO_@moJ$>CKVHi;%Sa^XgfM3jUTO^aq*eo#`cy`o7F+KHr9XZhxF^r=KDrjt_6Rjx97#Cl3 zEs!|lq^vecb~Th{EG`4Hnz3;Haw_mKOC9mJXwt+2_0bB%uBdZ^)2Yl&ab*}iWb4{+ zLWC<@j-RnpLw=xoQew<=huWOlNqQg0hGUqAT$9CEWg(2yo4VbPE9-g9p>o(p)HINj zm>%6QAi0St_MIyWS|kJI8c(Y8(ozJ< znn1I5YwvYfO3PZ_1b((4DZckC;M`Na{0&=*7GUT_!?f`G#^9vPcX;_GNK zDxvUUT&H$H@j%SeX6mu$a~v#B7}?2sjMp51 z_Pu-|*BIsE1Ma4;z~aE#Zrw(yjh_?);K6k`wHOOBOnF1aN}h%~ORRgCQ?Yyi=@t-Q z$_vMVTeOZDLbo@ft{}|rQHjb3tZ^v8UwPLk-Dej6zz(DZj;gAj(drDmlJvcC^_1jV z6RF6DSfU%RncRuOC&SpjQa|yuxF|zuQt~Wum9>H*0T|wDYc{*X-2`l7uO}|bn_deh zDo1Oqg)+J#Mx=X++MJ{!Jp1HPRuV6nts7m!dQVU4cb=@jTFK3KS#fmAoh9alsS3G# zjYlR2TCjsKJPG3sM_mXSsRIx1D zQGo6BlQWvyivl8MQRbM9ym+BS#3TTlM5JpjYM#OoBToEC1-*i3st(9O?IZrR+y`Sk z&a-evMjXG8_vV#eh2&K)#^+!iA3wxIUd-y}>7BnKpN%jIn==D(L*hEL?4BsmE1N$w zpUS%+VIG3MT zlSe`(RLZ>GY$`VqKQ}RZbPg@A;VD)ahwFa>pX@W=)3!Y)aclr|2r}PFC%KP7y__vR zkD)ANnUZ9u;EurxYaHg|`+z-V6M87Npr5jDG!&d~%^uY;}J9xXGFL7wy7rn)e z%{DW^&H>dwbw)_&6)m+orB#-SEz-Nf?h}Vf~y#5__w&u?dIuhqE z$x()7_@eKcxX(;=qejtnwkn%9K6JD9m_6$bbTnyRYoB#3f32v;!{&NEoTn-9PROmw zYt`0^mY*hyfaLB@RJPQp{ZSzz+Kj!0PN@3HZ`m820bo4We^6 zgVC7n;+EI!Nnw@REI1rHf!Y=sIYric#@N}a5mkYT7)SW|8Q~qSGwcv}K&6q1{v!kF)X)L_9U?Z02Y(LWW{20ZY@c zu1&rcj|xQ;BF0G#IzO#0%RLMq^p)sB>{s~kSoiYY( zKujAz*rPh8yz!I-*lO06-iAP(7h)y@LAXV$G8?+*&9lD(vQ)sgqxhZA4C%$$4<-`Y zYuuQF(?@QzZPrY67yOa-^(v$rsdJPOv00=zDNJec`zW|Fj#Ele-6t<@+Eq^gD`J9s zp2t-%pPB-0rQn3Em4#|Fv1Xu(<;H0OhI(wlcZYU;3Qs0ssOJuvS+;MmRsNObl~@<9qbdeVDhn?l^vrCxe~!rV zL%42ViKJN-Q>@F(MEy+b;=Ojp@MuFr zkN(2jb#mX?Wlj-?{bkN>C31J|PN-Oe0j?s!ilqdP-UoWZ4MgE^Tr6$Aq#nnCl}@Ss z+8M`(!~&a5ThOReY_TCky&%&mqdd_=E0^- zmr}{%mJlyNp^vWDz4dNhoj&Ae`3rC;N4e?_47R&E+Ovg^)_dkTj|aAc_gTpZUR&S| z)Pi~bqvv*lQAP|Bsl*-Zr7Bx*5f9*H!&RfF_JhA?Rg}&+$)}{~oSVO-DK<=phEQ@b z8|vk6;84b>QVN)aFy+;3R8{5tH$_H(18yw9SArw!00J438+CT>!9gle^{S2G4W+TU zulfDr#9STTKbD`UZ2D|NRFZxuPm0Pl{xTK{W1mZ;VmwaQOIL2qlF1=4EZr`3q0$$X z#`1N^P+h~~CT2aAH;4W65V&lf9l_jF^Plhj5p)3Lr1)la+Ne-9|KV>Bd(M9{Sh`Tb z#tYb>{K*5Ze>@kUb(as{sVqXVy7Eq0_as>T~nNAiyuMDiRQ&-BP5 zs1KuH2nL#7gE0KJ$xD3xyqQKEKsT4p3LQrNex1c02*dv?hw0A0u5}48E-ilhUx3Xa ztjYhY8bDSb_1>=kIS|rXfIz*l{BQ`3{rks{sc`d}SJD6SZu5Tu_|X6I>hg9T4h$U^ z4&C|Jk6O5{R|k+pegQTuWrU;t9uu+^3_Sje37`t_qZyUY_1de~YJpz2=huXomqF8Y zr4VxTXz`glQiJdpDbT?=;IexP{G%w6@CLBgne$EQPE^x^K@H5bo3x?_9 zwj4fJPmbJi6qjl){QcBSth138ll%eca2?eBI&SY8QFP1w`2k|1(^Y@SNzkQqK9fvu;0A;hKYgK}W z?avI_b4F5;jsm~z&1&>ssUh--93-BLj(bS|3Dml;B%}(SAR4lt-aE3ldQST02@_5| z4W+&`)CBTDP04|C2fm#=t(l0fv12EGv3C|9j|#yievM(km)F29)XRjkE!}aT`!1Mp zws*%_uCBcQ_~nC&o-<*=w=u!8#hWYR*KI$7)Pi#vjN6*EeCcFxD!Gv4yG~>j$&pud zTQ24+z|kdi-+e6;y01esqpl9(gfTY>)5 zT@kaq>*wwQncJ;eDK!4Df7@4A$*;m_R3~KPR^Z=d)ss?D@J-|0`wf3XTdyL$5fgsq zFZbS5f`CkPuZy>XeR+8xju0`T>(2=19F1dNe;u%4I5OLCIvc-QFZHCB-hCU(q5SdM zoZ(DV2;4rivb~3d@&yj!F~i=1B;2mBaONyD)4UNSjVt7}`0q=a>uVa__#j;|eRlwj z(6bTN|AYABoT*3PC-)3KX&cp}Faiu~c>||np)`l6Vkw$Hg_F7nq2ti|yg7uE9 zC;g1!?2l><5i(lyG->x`UF_3gB)Gq+V-}}Ru%5m5Ud98lCX?F{KH=xLJN9X`)?hXO z8uS_cvgiwWze$9c%6Za`cl;tS6Rzu;Czh;8Ha~)cVvH#0+j~Evi33#v5j^z7FNxuAP!5RM4FR&@vo!eA;Q1-27rxc6?lL z6yrSYiH0ishTy*DiQyF()~ftbp}imehSFayd&i$nGvhh{4%eu!t^J!&sc!0hcgS(K z+?nMBJ*)`eSmfRc z5lIskb~2=e8sW%pyOS^-29bTFq^X5KpePnXrFjxFtyx#1B}V_E+|SdmFDS|ZqERR*P2m$h^?_ObmoWx`8^pzDAfZ*smd{L9Q^zG5mG^Qwr zcA}ZybIPFL=E<+PuG&s2l`b5FG{W2zQ5DnTl3=#xOe;dTII;rT2r!)>b^A^Y@}3R* zvX4k6(Q%xQE``xF$|lPB18fmVOs(u=;P{Z-IHi4i(eTZd`PaDENrcFcZ>iYbHBS_7 z!<=#9j;_6h%(wd@L)s6G@^`ss^DJCRns(%=o$?iTW4G}#A~x^+esP5OgY7ikI!{;e=_?JDELC&wTK3YeRX~bTU>+*5d*eqJ zqm=e_g6`QnbUoy@4qN(p%~=l@%ST)?`08TbcN={f)0y7OF|Z`;7DBhK(0Hzk6W3k& zz_)XnRX6{We&74i@g$oeZ9nw46#q>RF8M`TE51rpn-CXf` zN&xh*Td!Y{KT*s`5vOlSKx zZQmWA(s~=Ow+tbf2rLdu(K#2f`9O4MfP)*K1p(JalrSNcc-c9h%D;4s_3=059N+cu z^@xYjJs?(~WWbBRNMLu(>9-<%6!=bPE8}3(*FnD=der|n|R*HbLJcCcQTfnk}4I!fJoRHl?urz|! zNxa$F%hpIiL~Ob2db06@izTe`{OY`I%vOj8BB(zVsLAXQ#7De=Zyz>+IrnpziS=Suu_?1rkcK;ZTH(<>;I^+An1-I>ol=>evxNr}LEK3Idy-BPn?X0Bh4n^F9-7a9 z+^HjoukUyH8*5CR0W?pDS)i3q2X;8bo#SiA|6%W~-=g{&Jy1a;M7kR!m2RaOkPr|A zg(0Q8k&=>bkOnC!5v6lz8A=)f0qJgr?z(HNrRy}yYJ?*U3`#^j~{o(g_5q%=r zv>5ccF2KOvSS?)_q-0%TPB?84@-D{X6fb=ig%s~+C7N{^GU&YD+m-+Bc^C3uC>_d$WjYN8xPIjk- z=Oyd8mMC-Xn*(rbuqk3!`T0iXdR7C{_736(vw%^wz8|>M&lWSjbhU=keqeJqb-K&Z z2oB}%5Zdk=x6LHjoJoNR!IE16#_3)1+hE{ULXGqp{TG-fto!#Xu{Sdy+V5f}3or-H zf5FZ8x8AX_?xC02+pzwBX92(i(az8YwM$h(useLRY4ExAWLET>_Ovc{`!Lw9uev2~ z<6@D3?z$HqEgORU{q49Yw@r;-ftPCYycj~;Np&TwM|=;|TJ`6=|54gt-GYwfFUVfr zwZ@(xw6A>$Z-4nDUot2>JA;*$n-6w=(4(w3NEc`Qe3M!1ihCEU)2*7==;5~nCWWr% zg!w0kXzw-eMQv;oUz{u}4}*t8xM$rwh@u)_muT! zd!Z_c+=rA;v^ec+2FX%)z{)Wk*LBkQj+1KLL{Io+a9}cJU0H-_qdaoZpmbg5taq|* zJU+%Jm)+BCacYTD7tHxY1d1n90V0kP!EI??_U{qhACeu5PV|P8=1Z2G&S$`E(F|sb z6K>i%1B2*t2+`Zo|DGnsj|?iV+uPR-2HwOIrO^q{=JL>f!!%mJz>?M!;xdhzGnXa7 z`zBU=$oSIsxxqVHk@}nvi;z$fS{N^F6b$y9Wr>Wj44?UWff0A#_W)mQ0InOkvCI%D zNjr1FP9;g2K|jX3)iKHTDr=B(iS89SB$kf!l>zO0II^OKgPb7RNh8Ltq9Yw?q+G+` z9d8Su2`Yp^~4n5v9!?Mxo>SqzG$GQh<*3tIZnH%QYeo1MrR(vDS9|d zY?36p$TE{Ory}>JzYLcb3Ptmkr=td&Q(7La~ORc zaAq1;iz|dsoRK`pCocIbvaJ6lFeFql>gw49b>@c*QS$lp3!epHhQno@Rh=(H&fkl^ z_Tt%PIL!PwzzlZs%dg9vXYuXk8LWAB_gKHCmh0~H#U(Ueaw+sZ^?jM0*RQ6+t`wV( z;(eYXFFCxkU~|8t4OvLJi5Syz@ORT1->RjoP6wiPGw6s@$4R#X>d>S2)Nj04*s>!7 zw?5b%XMEJIYE5`QsU;d}Rra!r_xy+X%=&Y>bDxFl?l82lc^X8I?RpmZgy`8BY;WRf zpmR}eTiA14+hU;G_IPT$|LBE;)FRyk>)Z=puM6>YJQnc0srMf98I=P;=y|Gz+%k{2i@6TZOFi-THU&Ok8fklCJH{JNe zou<>+H>aRFaS-rx(VdjV^C;FpjGWkg-9KdR&ezUB2<@@>Jm%yQ4(o{gGpwvnn4=MP z9)bJM46iqW8D0`d%>1;W5JrjV!NFIhTwMsQy3SLe4P$BY(#yc)Miayn)Tc=AyYEGw z6GPst_u@6x^bgMZxW_yj6x@_2QqVVBg6v2`^>Q>DwIS4@*)6-59hxzV-`(2PbbmrPVw-R9EaIXxRDoU^q0TCH}Qs zR9G+%E$o|&^2gwPhd0CV7G{S4S-H!1|ex9gRTifX?Yi#nj z6PJ6yR{7WY?E-Ejy)01NUmP?ad;oS{x)>XRu-)47*W|SdM$+OwU#)P54{y~s6RIj{tRMQ8ZQ)WYzyl!vJ_sb^T zX-@~gfh>w!QC;)6@eV;0`}IEZ##gjm3hxTe%~NzojRjB*33xjY%+1wPAI%D@Q!RZe z+arn8h3@TNfu+!%I6D1HkFzgoYk~123S5h$y>Bsv``9H~Osr&5$)K~Gwo|+|)773_ zo5Sl2c^$idt1Q;)-|J9#)aA;`8=>b|w8Woi=NC}48dw^r4SKO>X^p$9v|O|{XTyc! z$3i*xysflTA>BM>`M&Y4*H03M`aj(H@zb|dUXYC#cMD`MGSXcIm7c5n)$o; zV5B>D;EXA}&`kCZT<40PI=+{ZkSS6N@w=DAkEK)0kNy4-ui`hNlbV+9I5MMWn7U#! zuw}CfFYOpjI=HdX7r+`6*d4+JvEPbuwu^HxCQEl=b&)d_>35A{4W)_kDAwUwF|Lqw z_wLv7hK8FoMF&}Rx9VP>pBQ(&T!L)9rq9U;2ugb)q%3UU))>02MoU2>MH+=OlcpxbeQC<6UeX`b=9X>@BiiEElc&-^M-VayCXBiaK15p@SZNvpfdl z=S!=BsUb7j9k;)weiMIIdxxI$5}W%KI+Wu{&e|szO7U?fmvyDS^d1yabpv$6d*Pk> zasFu^_M`CO^g>2!qNIczYv!E`I@d3H1oha^<&)8ca{RxgJ_z&5G1N8^rwSqx)|?3> z&}(3$(iE=xs4h2?P;Wsa)-fY+X>@s-*&wasaq11Fi%4gQJSyV%T#T^2k7E^_tUc&? zp{;LMMalzz_*yr9^&&-=G?9R0Y^w>X6jeoykI}OhG7XC< z$(PuPj%?D(zgVg7lsqM&I@+;967ghqXp1xn0{%Up&B^6xpf}G<<_?yn{;jq3#hT?b_NuF9w zeME3L-7ZCbvy$4{dIp!W&h(Lf5R-eYV{t6koH`FSOI`l^CtbgZI>D$Axe_MBe@u_NK8sclK& zyqTlp1)rINq_a{41Ft*bC@eKlIVTt1 zGu%c8c|9@5`|AZC3vlcN0^WY|M-e8;LdlO;!G(3*Gc9%fWE94Ivd;Q-Bb{=3jtWg)!C6d!g)3GX6OCbhbQ9pv%~M-~;N<5|FS1#;&Tgd@18 zC;|$&4KSv-VbULa$h15!sqOp^n25LfoiSXhu=mU^23H=KJxn&)Hxw6HrdXj9&VTuE zmv0+tMI_Y4e91-hcshzhkpQ2Y6)|V1>}!a7d;B(x<6(#QvLk~sh2*A1u{)&RtkvOV zeesNDmhpD~v)D=kZib~wP5fX5<~cmguN?9$-=ds~3rz+%Z~BP&w3mojyw5tFa(sBv z^Q6}Pj&xD@RVd`BmC80g>drEr_tf52fEjle^Xx+Gcpvqhy<8wv<)8O_ve(05_v~HP z1>%6#$Ttu*TVkLw(i3f2WSM+>W^<42y=PrI@rQNN28T`4%=Dyy^i?DIrp<3e=p>gI zROkD?a{hAD<_6a4-}J0t+SliC#{_F^-g`9KVc3e}wLMpfmppEtozpy3b1DcyF-DOw zb6=I=VDL!lbK;|QEq++j?cu9EwCqw) zd|+Pfn8s1_h!&sW7tz$2+OzJ<9@D@WGM%@Zzbu*IukidU1_Yt}9UTv`DTzqXnOLMB zy}`u6#SW-vcfuhgC6pgA&ZSCGYez?6phgT?y5qC-*KXU8r-akUPI1sH^9*-XyuPo) zf(Q+8C5pF85T0>h5Et4HHfrBnAWe?*veahqe57VbT1H0n9^IkX$Qz#{%l1uR$LJB6 z$(>_&aECu*z*k)7NLN0li%*Zx-=-J8rpr>=*dkME=z3yi9C)9jX06tMQR;m_auQi_ zJ*^8}Wn?AlV=Hj{iY-}dYb&%?E@fyO>y4c)iGn>YGclRE zi~Sq7(%NmBX5YpXnMLhYMs_Rk6`{Fr*jW8%kX7qNgdl zKasX`bXezF7xl!Jg@{Z{JheP1{X?+Pl8LrQ?A}7#SD6)zO&NM`{HT`?SJ~W(s}K4w zaeab?`UxW);h*2UBF3wIO%cI+DY>0s3yF827E(^5X2nXM-M3Y~==8XcVoF2VIK%%n zomdx4!C5Hg7cUqf;*Lf9!B*pEfk`=Ij6M;(5qNglPUOsLdKHAK^HX})vt;kQV&enc zBbr0Y=l16q!CY+R*>QwB)Ls#K8o!rG)}`dtov+U=U|x_!q8#UK$IC%BkG)8U_b&X% zcs}7`%CTs*%_aIEDcgHd#&4|YSx7{nb`bfv!iRve{!yj$%QeZ0KPV zmEngm(ZrWcbnxRtpTZKkIfmOzx!GrhzD?PWh1^Up>@#&eLjD zvC;mfam-y)M_0M8OdXbAK-X%p<3&ilWcMt~ykh>Y z%<3aI&X)!iuukk(jk&LWuR(oZ5zh_jltN&^R-{Mbj{`Iq3#Q+{epsV2XJh4(Zrt6w z$v5P}Fkkg$fXAKd>z(>gz9)(++j=vJW=^U*j?vM;-O(S({)D}Yk=_6j&m_3f((3TV zF{Z$hx{TqL<4)O@60eHT*`jLuD(eN`iTE6{NA?KE-R)__5vLrooRVp`FEciD{!<7qZ5wFHDx4-Ydwr9HGbtVug+a{=Q-EB_ zZdW=t1AGxaDntqsyh@MuU;p`%Lq815A%3$;{QuhUuXo|ge`$l!bz_&h{MQE*TYjYeRi63x;>sp^sECa;Jxjt0U2TU53XKAPo%{Hk*M*i zVl+OsMxWKq%pCg`=#o1C(QR_eDspaw*l=HdSt>)$J6HR#hRhe5MJHair=Gq#v$v)? zJ16X`KF?63wJOSA+1uha^|>BTQwJ@#@5J@iMZQ!kSkve~PM;bY>RaZT7m4jPHWPz) zw~L^mYCBKb9+|bOM+j1yi$xcMh@96?4*Ec5{EnYr4OcHI@<*y|irF;20yS+BWpL1v zwGMKPB^{W3fWCJISRvlVeeaz1$T_V9g--Pw-P7#x?t(;CY^%jA=s1GAbFtnDF4E&7 zQz4hcZ|(nQ1bq{%0gCX7MnFL~J;2^*aNk3KAZ1ilh3Bn+w3(5?6JL>wq22@`+P;3- z+bkfu(f>|*i$Nv?24ctm*a0npTi>F=Ora`W9T4VS%=7pF!OM)~-Su)R53=9=_6it7 z7Hfeqnr-m>s4dh5fX(=$Uyxd0jtW z-}yZ8io3m&JCe$#VG0q`sk7fLaY1Hrm5N!t1GbD04xLnzzD?<%!e|C03a$X*AAcQ? z)P2wcgF*D&8c^eprbIf=0GZnyVKeZ!p7U`58;Sj7xlz3s5R_Q?WzT;U11j@bkmy1gYJ5WOCXxt073zxcgfWvgxp217L?Mj zfN^IAu+f{_)j%zAUJWXlSV0bfyy$*MuMLx6iqr9fZMGrwZT3|47(mgFBg10mdSfE@ z0n@n2St};G0@&8me`)e)>^8yhaRFl|+(8(qrn3CLMpFs90V_=mINTTBSMU*Ri3MZA z+5>PZ1uB}M&=|ng)%v+rV>D5m)AWOr{~Lp07U`&%ID>l}IPdHJ1k$jC#;xx^W`I6) zH?Rfh%Md`Bdg4L6#T|?YU0tdFP1goY!!5qrC_bm(@?s7zMh4G)hCqr>-a{V~Cq%l_7cLYM~K&K|;Jne}Ws zn`jobs?KeFw;DEWR#UHk_pZBPw(M$h*5}tl#0th`{hA6@V`eT4ujTB=wC!Im#Nm32 zyX3Xdpk2!hHKNNW#kHz7!*_$pAe~fY=2z>j(^=C^q|JOoT4V8gynR2+Z}vjfd)=4z z`J7$3sl~+l7ZT?Kx?97r#^W>;#P++chR!wX?U)%;lnIw;PjA7ZZixQWx%0cTLp(%j zpq3-{&~KI+-{A7_c|0u6ZCfXKM0zEzFs;`5f6U3>R549!$M_g54)Jn~10o(^L0mwb z*~KNumSjYPP}%Rxl15sG;?jr}RrW$8!iQGqmQb*QJ3cFa7UWZdX*VJ=*Q3q5O zo_y6nbJEDA-{8T=>wWG>=e;qUV-FDMvk3(opf`IDDga>HvyFAFcn9Zz_M8PHa!vm; zgY&0PAhdsa9vNyPz8TB|M){_1E?!UY0$7A@0{KN*o-)uWe{KK%s4Z7dc`&VC!-3_1 zQ>WSyX!5^{f?RhWA9`0J)>rTGAI<2c&A0^Zmja^@Em1j2Bg%dB^M==S3WV~9XeN3M z%ke>Ei}a27W`~IJl60UD?3)1ve+03Jp~yDV-oz*NAaw0uDBJB`j{Kq%lEF<8diW?y ziw$3WeD*M7WGnRz>#ye*2YFjSvehJ`crQGFo?+$wBDfs!^kZqouWR<_fw^rZ360H! zegQ_gjPuj=kWn1-88lB%8&n(myia88$eDvWf*48NGVg`7BW3zXOq&*lf$DAMWGLJf zG4uCbou3hvsx>$1U`#c;ZUSyC{)-s42SAO(uiC+B>k*Z5{{^lcr|o>6eC(ij3go6> z_@+EVZ0~BXbspLM-crI1xIfO>w zlP424yPmUYam4t9y-$e#(wiTz~#XNBr89*}L0Xrfd8QL7a1vx}CLNMK&%b(Gf<9cyv5-pOU$m z{StO>iqIAE`7Ksj#7CKUA|8jrrdFY*JokR;3L0^nfn5WzS<3BTH}o!%yc@ zuQ#+|f}ZsAcz4tK7jP-}G5vQ9HKe;64b?Vrb-ohnU4Bjw8?GLTJoeQ8yqA-ps#GP1 z^gKnwCd6Eng-Tkd3mXyWfgB9+@MYTZDo``jhqMW1d|>AaLOC3(G*j{`u^RoFg?65> zFb^7+7dX|S1^qW8906#I+#?Mjvco0$yJm@wCw2zGJGitE{Zq^Y>*7XCDkyJJmre+N zN=+nGl_Af{cqOWV69oBZ9UCqN?pM*C13%EQEP5oZGvK>DZFrN-(FkAYg{=Zv%1)UR zuCxpcJtx<<`NPTKM7beBES_nMs$Tqlz>W!LE3jjEJ0ALf^7pf3sH;9SGUi$&C3VJo z0*|mwRwy2aZpkc}E{?`Pz2fpka*r1-LhbH#-sfssQJO|!|3#TR0qkd^g%6y9EFmjP z5UP+2hb;dChB9py{RuCbj^zcUHCg?$jTn*?b=$@bP&b$!eG87S>7Y z?-9V8|16uSCdGTP%^Jw24(-rZ<*!b6eY8iGf>xyz<6n$iMs@-m6mEnf-_}VGJSsZ@)qgG!4k^Sak@Q4l>X9*G@JOfocfmY?d+?h(X=gqeuTwNPEATrI&*X4H@Rw+yZi>zYo|3$%&o3Wr+-Ady!U zZsJ@)SqSWZmaHcP{{8uGKrV2Kh`^^*xSy<6tbk7&hQ@>unTbWJ|BG~79jy-s?H4BJ z1T5C<2eeH+1e+VJJE)pocqLHkL-epS?vu7?vbL!cAPn|oe-H+=ep zVakoLR`$kR48kJ%%Glj$2)Xz0duCNRZl%lm>a9E(=t8UN?^lr85pNa{_@nzvEf*Jbe12f`(UdoQ-aOsT*%Vhj(oh_mx%$JBeWWW# z;=Zp(^RWAj$FkEvK+g(AkMQe|#PU~%Kttn-?0J}@`P^Pv9&!ZMJEt~A+M$u#g0fjP za3>T-Gf@7)rrCCK}A%YxjGF4 zB0a=*Jw9j;MwT$HqSLj9=Kr$lR0joLH`MgL*d@zmL`nk+GuCc8QGp z{A0GT({{w;@<&3UY`;)E>43E{Z=c^f5v$6<8N+Xu;4O>Akr`@)?J* zBd0s8vhBwIoO4A{1+^`+Mk;Y*!g}vBlggM>!7mrT9&LUS z9=d;BREV{{K%xVy*WzGF#EX0~Zm269p(IF!S0Sy=_Q_2zUFByiOrgTHaZhcMjyHdQF#s}%6uCZS*}e+mB|1*t>jHw#Me?kmomG(FF9^7 zI^Lf|Q|Bzk0<{nWEhnleGEm%Y#zhyy<)W!%&$YB}3&r$8JjQDOXKkm#FVE|KW!QOh zVCpt%juKnmS|1H+nkf$3 zwjaZut!yyi$C~aAvw}yoTSdNP#zNM{G@HOwlvD-Psi*%5yCg)n1rgkj8dqa4 zw#dtf?80gTu?2@R_1*t&3Ntwh62@i@hj`i-m(YlLI3k1%JrQDMFmmZGLe=!HUR>!Q zXysoL$>*{@j^bRM&NL3(2iLpQtC$a&iZ{pT%pTt0wk_!?L>(Jn6c$$pB30yS)e-Zd zzjDb9?YOoR+Wsgsm~|nRV8`j*!TNZyGI@gK#c!kig!I2gvIbqrwVL&@C}rOrhl(yF ztIqzk#QC8ut34$0cYYN1NZ437k z?7$hU-B_bhGoMe!=g;?OsaT9ikc)2pQ~7b{ z%6eoDz@|I-+GDPU9f7Eh(L=JkMqCu*1X{gNryRM-o|GoXlVFdpSf-Th8c^C9RYg1_ z7e)o{aX>C4xD=n3U79j>K%HfB<#j%>a)fKX;)udTLobku|7aFQ=2>OK zIj$jrx$W`SNr)?$r()ggU~(}{KeWx?HB-!H=3@fKFK9Kxu5Cwf`4i%fjAzB-16ti4 znLmS1s;jww@gV!;Ddz&L%=+B(qahtm(sEEnoE~F&8@FQgBen8N4NbRbbB(8%e(9O9w|%Y)(Ik{P8tt8Rrg~;4W^fh(AA4>eP;^UUdit zV=pePR>It7W57SfiDhyfXch@-GO(wAzH^Kg3U`Jlq82yS)YOp*iwVW(dA6Q!s@C0J zIE1F$oH^AwY#Kaw#c2)Phj%xvl-V`IGv-A4x>U&aYV$>J=PWsWWSQcAoa%{OhG+ZQ zum6yIA4}65(Mo!2QuU1~JmM3Z)dbOfd9v6^4-{eN9ZCDvu%LA6cCNm3)`Te`f*h3} z*_{*ri5BrQqDDX7{v$wS8!}}wuv94QC~Te^B{g-9^CrE&(k+<>8oHyCD)85cV);zD zN@o&&ey|7CWr(ab5b%}UrT)Q+NKMS_dA==Ml>LXDlXfCpiN+}|H#KN z1W@n!ra+7PpmXKtUE!!6gei)Wrj^i~R~@XS1d-JOFWZ8u8BF9}#*eI}ndqaj&*Uzv zzofB}Io(;>TotW5#iGTBWZ|qx-d;6aj4aU3-S1HSu9^cYn5(nE$|@MkT<`m;V>N!O z-j^guyDI;;YR0**9te@!WAUoJ5|!8#)O6mwybbfEs?|Q$t=e&vphb$le~nvz+>_&Ul`Joo4?3{>HGe@!uLh;nCAiZ6M37_jTo8sMkWiY+;rQrRitvx)*8- z{Up3U9rR=CL23^lThbV*$!um1{ae?6kV^j#VILLj#J$;`#{V->{~HsP!ZekkKSFf> zjn`oN2j==u)U+lPe0JBU|9=oupv3XROE5b8b)**d zTJPKd&R0dnZyuG-XZCWAno8JyXR6L&4*-(16AiB0O-8O7|BR%6Vb=lzXp}0^`8gyy ztDHFgJqfIhpjoC3=|3vfZ*;QZ@!<@)hSQ z_h;mz$QRM^Xm@}eHKTj^&T5m$}=EVX4$PSVxz~z(o4T4K0)UWG?tW?fjVtx=p-)#mwHEt^DLXXyC zotC>ib0t9rSLct#A-{`Nva)ll=~}y`n|f zWT*9g@TELBz?ZQf$H&I+EZ(?b&3mrXxUa;?e}F_{Z-GVl8pzc#%SHgF%K|c*rt(M) zwI=i)S2xr$l4J?Lj<6Dxg2K)%0&#>KSu-31ObFl3uY#u~S7cLrX)2z?`4=zn6I|9Y zXx<~SqATb1iw{0?0pQ8hBL1DLjy{ zIOwQfZ9hPzc@7?2dU&Fvg{FRG0&y?`IQEXu0D|U`-33A@7KVPG$FtBCKL^R32lJi` zM9I!P(zV{;gsZ&cI+V5LlY0h7hLhZi+;MRC_BI2lVISB{Hw4Hv`V}mlhx7zCBE1@Z zxX)Mcg)N8a&yW$4hu6Sa;p4%=mEVqGvkB1PbxF%B8x0T5V7-`xUKwhyLB+GkX1S|- z4iflMETLi(Ab_W4GJi|#8VUO&TWj>5$4FdS)*%|U*9NzjuuV6>u_%r##$HpbN5`d3 zr7V?55w960fe|dh3GuqVc&=yGp*~qx4X8+Pis5Y_)&?@qX2|zMSj6&m;}7aS@%rKnq_P5;)2qt+G&6HLRC6D1JK&TmiGlPR70?P?5A$+a2-VF)!np zQ*L0>QF!C~tDpRBtw?ra5p@7#!i1O{nN>8X13Zdk^}y(mk50(O2wGZr{cw~v0bseP z#`Gm|62wT&f*PPVj7^#QJFgWS=?YL6U&{@*;DJsdJ#Uueu;9A&!J3J8ezTod_a+o$ zUDrP89DXE6k_XRMF~+IGMCcOf5|2O=pBoX2GX3Ryhz_`#>bQxlBmq!18eD;WTn1>0 zE_OYIE6|VPc#Jj0@sO<-lBSH6$h`~jv7s}WOZM|m2$0rbrhU;;2O4uOyk;apg`M-z zQ?iX#ZrFFKQntXKC4P6e&-Jj}09Dd}H1g33*6fUR62H~did3FAc_jTBE)Fm7Bw2VG zDSyv(Auu?uGHv)yG1NJMl$1qdZFmhdGZDD&w_#kt#v*GWx&=2n9!l&s^~KEuLykEa zY@6lQ44rE#H*NlWIa2v{P;u{oyPLd$=~TI9dyq}$7kWUE@r{fwD=CRNYREmA2#Dor zZY%#V+q}yw8I~(z-y`;7LAqJFs2ta3F7buggeS06#i+BtS`x@e=ZKk9PMUQ%ED2fE&P{yb;m&-LEpUC56em#B2O`n7rk}nXQ`@0nihk!7 zyFWgDP8*}0ksdVR`RV}A?$ML4L*3ItH$hG##Ip+>jbr(#6?01)Y{xR+r<~e( zNbpZ&^(*_np36Ye)gaj~igD4ZOKB#8nFKJ>JstdQ=9=nRKSO0@j7Hi7S(L)+AL~%+ zvEmp^VI~Y#V)@FLh?V{i%@XQ^^J>fA z@cHb&cR7#R9=5V3A zZ~wyAdoiaNh@P^P0P5*D=;3Nu@VX_OiLO9V&vK9L#iT&AMc82H zW$-AUCvc1a&^%oWHHX-^5(aoN5BM=~J$F`}}P#O$4#b!Y?ugY^_;4&8bodAttBTH2Px9io>q)68<7gsZ3%%{uYwy5ZH9jKLfzk2SOz(d8c;KaEnRKspFUf zYGyK@&=yOk6f^~lc!kC%V|d;ncQ^Mjve?nE)uZCEI@#?+xD4^*6%flkCKF-pje#bU ze)0RHM9hDcW!`Ry1A-O9?6tNM=S7f0ceiELY)bmt%BW;X9nMpbI z8{~89M>RS$LhK7cVIzZY4E_cnRqa~V54fz$O}#zSOc{ZZwkEKN5%}Q0WsmVXEd;vy zUnZnz2YwSx;1XEiuUdE7f==dOy}T$o;L0FM_-RI~bU&bv zC*hR>Zi1zdmyt24#8ro^F_|`1l&+|m#bGuJh&rue9h?dMC_~;ONnEUd{5*^#-JI9c zbSdnuIe=~-1+Mif(($nqz9eBeQN~TfhrCIPV#F_oSjqTSq<+L8f)+aGEAHP5|b^|7A59dZCA8m7r$9&&sG?4&=rnRz4DOcCa>tPb7^Phwj zf2@AZa#|aE&!ftU7V?~dOc@4DUMTZ1S8{Xg)zX6%aI+^`gj|Y~<^$=glpWnAbljLh zwf-c@H3`)O3Ct;aBMDGq_fJAXUm*k-Muh|9i!ZSJ-S008PLjkMUs2k1CSq6fPD$*} zab&V()2|)VT=ERzJh%_h>+-rJxBCbaBig!G%SK!zY3IKkeChuu+9I4U^7a=rupe;Z z200Xja@g5>spDWA4vlDemc5==vr6aoYfKp%-!aI^i*^{J!NZ_(iQGLhnwES0A|1Z< z>Iy+L;5%?etSH!ci|nDw^?SMD3Zi?gK<)jVuco``rLs<~N2@tcbqXCRfz#VQ5$ zCxl!~&+yYh9od|uAHpf-&9YXnrIs>_3cdZODez=6$92INK{$n-Y(f-P^u%X}QvMfp z(1Sg6Vzg>uF>1I-$YNDK&hxL5sh?R_GJxCOSHdK5w`qLgkMz%CM=hPkY=o5Dq z`KJhHG<7o*Y^#Gs?dho*u+H%oIdkE#brN&@VV-@DWk5NRA!0!#yYq&ifNTW|9`%-s z3-&900iv2_+f;E7BlO0$o@2nfyVx@wj$`u`3oX`H&I_WQs;UOzaihp#=77G|LYOhs zFd(!qDI*>dIWjqL46xw8#pHCA`dtN#lmK|nFBzG$B&s&~#h#CJ_n+k_yY-1ODM^+G z;b1UU>#>SUbiN$Fm1)rK{nc0uh3ML*jl%Pe{dcN5+T7N-YKGDugOH1AvN@c^BW|~1 zRrbcxfAKLR;8d!}n4{MZt^}lDBix|^Y5TnOru2;E&zT>?0hkG|~*2uwAlPET*Zf6(JsEz=bNSoM*jxwFRN; zSc($9m#o%ASx#+I5T%q{9W9`v$1W_3(#JmcaA&+fW&5~ zOg5K%@m}{R&!Zm3(uRfn(DFxP&t(lorp0s=D z)>M3~JfdCH@0^T0e9%|#`QA~|hE67pXsEk4W!2J**%q9+Ne(jsG0f~2@JdsxY*O@2 zQj>eVTsH4kk3FkAx}CnZ1_p+UzZ3m0BumW-Ej~@Az~%J^*NA&IdZ}zj>ZXm_fK&uk_%yI7H(paQ?QS6;i^Pd3)F*y5#Sm@~O|yUH%4d@JSn?Mlg4dFOj|! z_Ra2L`0Vnp#ML!NUKa5Y1M8xK>!}KljPnbS`Li^8oVu_qgh65PIQB)%F@CYMa=C1u|Gczsk%Xpab3>dkT1slAs;{YtfDY$0@O4iSm!hm2fAl8oWw z_D=KIqXoBYZ*9As4&fwYJXlBO7}_N6WNz z3`^@W@`{aa-kpnPLghE*cRty3HLkH(Z)zS|&M`s}{V%BaoWb3VyPU7AtCE{1oy}B)m7Zxlt{kThW`w8q&!H?5##dLXy&TCZ-@F#2j(8s6w-wbM-qnAMgxvIcNNz=sRdEVp`BQj=8Mo&qPAn?~q`aXV87 zCMTs|G36Sh_cOWMeK|*YbHZR2vNMI5uBFdj@Xj)6>kFwbY2_xrWk+L3@Qf-4gV2rm z$yq8+QP*|EnyYQ}ESl*}(=I=eb)f7@l)lvt3O@j$wwsEgl2m9<%zehdgHjTEX#Gwc zZNfcvuTj>`V>0tH@_4?G}jV6+rEF0E44qYK&bcQ37JnM%+OGB;6-+Db;oQOXQs#B z-47!Kkt8NWW+ujuy6|6*!>304d1&VtuMwPfeZ;S4EZTXqXeOY#eZDBQp41)UoAn`^ zU9DugTuWqd<}CTC=d_RWgHch>UJ(Z!(@pW&33ADiABZgeT^5IUkEim-$PB6(ICDmB zJBi(Jtnb+ToYI=&-FuY$c=Gbh!}}LQuWVv^kW4*dD7{Y@_1>H{>;x$y9SZz%GkJPi7rqz+nk#@>YS$|1wRF*7d0W zOae#nhx04N>nru9c)|Fk1u%yq6R{jq$u!f|=t;qCw>uvsyONEg1QUUl&;%+C)YNU> z1IM!H!p@T^d>%O~{Bc30mX0DJlu+mw`i}xRGluat9jIEGQ+hehWO$i ziQW(k;-j!k9tas7{7uEOVX{L)AT@pUjH@B!{U*6g?U`ouE?u9CW#@M$6j59J ze7adyZjF!eY^ZkARWi@NSoxAG{5uQaS6gJ@dxmsx`Ul4*nC66E4=THzDw~QqKe6m~ z_=={7^F1tNN_o-Szamy`zFgCGX)Qz*@N?3%L4M_XR85!}7W6hE$+-8$6ayC<`%988 zhN~w}bkCx;dDAtSu{6!5*hDd_+(@G6lcQpHZX<_lEteNlvl2C@s$M>%GLXq3e_Coe zUHzBIsQoc<7F79yo{dfAqUZT8k=T`-{UHMZ_t)6Pk-;Is-+n*iPUW&Or*S7%2Q#%rZ0P7t)?vV5i8kl5*|cYw{^Ks*cJS! z01d5@ye*(zmd{?FgC*Lyov%sCF)g22yFOB-Y;qqelJ={U5lx4)zCzr`+@6{1DtkzBqL30AeReHvvPgHy2Pk__w3lq0NbCvw#1r;jYIq2h56MeW-_}&;&gEpUSJQd41*dch z6hqDUz>$~1#%ARn70bqHC2Y0*YuOX(wW6ICz(~t7@2M`SfUVkCcoSmB{Di|Uw32Dj zY;|D7k7L)k*V1VR8h+3>((BAoH#AB|;ZS8yoLZp#kP!(3{EGO6e2DMJ+O0zB4m>0!Jr-9_Ph&^zTHu3z|9;c@5U4?A=#SJ+u7c z-Cr~ri2RGGd~+WqZp!-0cL57bG`OqYNf89b0(n*ON$O1n?Ftc#G5)=!AjF@GrH#2e z{GXRWo%t=;b7)qkwJj<7t2Qu9DPzZjQkBCQ$KUEb|3#X7(ZT$61Jn8SRR90rM81ij zPTjz^@Y(PGUmHnLAJ_bWXl_3IhY0yc@F?|%0O-XyP7MSEQn*Q;QRLr;O_VDt^U>w^y<2trb(s$y#0T+zM%mJ=DSSsqkrE1Kfem> zBOq_EwF}t)=ga=t`c@wB0{(!|mn|1SSA2(r;wfLr$zE}Ex6Dnf_sL@~L)a5qN?dHNGl4AvK)#^Zo8j7S_#Q80R57X2GZKZ{lCe6ewpWi#-g0M5Q0Os_F2$I^Uu1z zch7gaWdL6|ffT9|8-RUn^Ldd?z!Wc^20Q_hU!(TW=le)JR{v;ifL%w5qZ1^Kxyyt+ zIv6qn>haTLW4<bb*S|#i;KBzEUb5ODho$ z_OsQN?*ybTHGuf?A=i!JHk$vZweOCnvi;*Pk%O`$``DY3z4sZ_s{Ro+~=I`aoyK-eLnBy)^b*bdIdlo zo@r*jFXQinX!=hr`QO`V5g`DQ7=TQ>N<5c`S3&1W5OLs+5lC0yKfec*3=_cbY)R7V z$9E5Zk16ln(FN-XPPyxD&pUOz7>q?wy#v>xzj$R-xxB_RIk>{A7Pp=Hr6AS#l@)tI z{Dk*xtjOT3PZoGcE`We=zb%~PKIlF_sy|pNo&X(36ILXK6M}DCCjsN#2&3;Pa|Db` ze{>at{z3WhmVihbaAz8VFr`&sc#TpDC-wwpCgD=R4Y85v0eh}BW{#Jt*Q$-$gfaAA zLc|g$<&S@6E3u-RX*&J@+t3x7T}q5BUEFaaJH`eoovs4_pIVMwFl12XyJfOVL6g z&mfuo5(;2hva${qS2Hv6^`EP%+^R;RqkGU%!43lmd*GI`^8W+9 zhgt~;nqQFICBwd{8kWA;!_T682?uxM>G3ayIcUAO5%E_fNCB#}%vBz-_$w#>z#!`e zjmN}{=kSTq)pJ@rqh4Q})99o&qAayrm+>r}MG4s&ZrnAA>L?#wuQX+v`xm;si64Ak z;Vu(sc$*&rYrK9YCJ^30Ppe%8uY>`}!hbsvLaQ1cr#eD#gA_2_-d}>G3B(}!pc=5= z>t9!m^?)D)6=FxuO<+A)4|?;TOx;@XRzdvvPfM4^9m7DuNx7cR5KlKLG=6Uj=+<`ozA%;u^z|=j-PaafO%{-unLo}a$4jmPPdHiA!9ehKxI3`>gP!t(hEK8Xz&DI4 z?w*r`XxF-|_c#EInsV|(AID0!Y(cDiAM*2AFg4%&2f)ux>WY$gNqcLO^WG|8uljUj z!T&j*E8{@6`!t+!6R6%K`H9IeNQE)=_yN!{n&mtkZzGxlAsbsv(s*~3^pIs?RC5>8 zb2ql#?yspmAJ2dL$YSuDMx@#*Mcld*ZRn@hNW8b8&HgRdVIW6F=IgfB@TLH!oxJH3 z4P*htz1~PKfELjjdZN3eO~!=WfimY%R?+-vAmC!ZIguRL1VH3ge(Jhv)%s`)dEcAX zp=lr@*qqiF7%VWK>NwM#zebu$vXT&x#^K;_LvSzvbbuLlnE88l!EAOY=|NHp)IAfJ zF2?2N+38^NAg_0$N?#ZOGTq#T=-dad{o`d;2*Yx9oRZ&zay~{z{PIsvK9qU^>BZW7 zKYJ(MW*~r{JzCW>8_4Kpxvu3xp#JLODv3nQP#oAhPQ!4!P4MMskxl%BCU)s*dZJxq1Tdt|K-8Dd-a5cU#R2 zv@?=N%PU5B0J8R+`h{h6O#7R&2FRBEm}~FW9mP>PoC1UwIZhl93(-mk2~H7%0Fs%@ z6=eGXGHJGx?VBJQu#+1C8Zjg6EA)Nv3hCvhb0?c2UhWlkUYjZ1Ulo_*2^L6AuoR^? z3UyER^XC%h>Js%*gi)4luYiK37mBIbFgeKx8Hn?2`WR_F+2lU~yvmkEC;dFadHD<# zD77mXka0dQY8G8<=TZYqW%83YEIKaT-j2c(?R+Ma$aow!3uwD5-xs!JC`~F%;z&)$ zra_ZY)|93;LD0Dqcko3snBgUfRfwX(w%$n%q$O^$^7#baW3Fj`cJweviA{grP1#;8 zp=C%{s=v~8@+;u4`;`E=q?UD&Osu;$Fduv+(I&IC~0tUzk4!}#eZr6=DaInSi}MzGFK0Uym74emK@a@aBiU2n*1 z*jO_TND*k|FtJgM@9REzmdzGgcp=1kTpVDN{!%zvop`@$%3pi5E7r_|hj~w+xLs0=qO(pu_HMg?m;R#I3+w6GOe3wWFBdAV^3fG_v5=Ue+(Z z$Q+0ILT&lSkK~0g?N(8ERK0@Oc9JBgWS!xwLLa(>dX>q9>H(g4=-Cp|)kq@LCE2bA z&*so>H(EHKh_H?ok*>uMa9b8J&Am0gjzoYx56U$gvlExkM9tBOuuKFX_^0b7B$D6Z zf$y$akl$5K=@uXH=6DPKKBvgb;p2YM-4XkZw)Z0>*%G9&^UBo_aSpm?R7$Ylu}PZ- zafKr# z5vdAa5Lcw0*E%0U>pFc3&ryp0sOb01Z4Y!Hf`;16+p-#j@;!*=3Mn$Fc8Tm5ND_6n zgakap#s@;qc+{>YCqb38y|M`F;}DD8rBA)P4>}3QrBld5}$x8W@U-gA(w=7YgfAG(XEjq2&*&|V3K&s*i^(6U`7lWoIiiXWw6Qsjm*C6N*O&TgRBCB< z_3}ZC_}_c(U>f}3qZe3Tz3Uz#c8bwqIg#8??!P6Y(t4vu#q%nW9o}BVl$xB1niz`j1artx@T^&dyYGKO2y+g)PqfN@(U2|J;DcvSX zuq0oD3ahwN;m3@N2upw*EKN%KaPC-&TlpxYYd^6iE+6PLMP8ffy3er*OE%~sWNzgj z3du%czHMB!+>Dbq0}F~NQ`agAmpA7sl?ALDZ~>M~^)Y8M^5o%yxa840BjY`-Fhz_M{ zu^R^U+%FiVO>o^^C)@U>7=9M6n~m)POUu1GBfKmD3HmAYq=*~3p&RhEgiGqg!$uS( zO5eZVhuI8w;)}*SUvUBm?aBU!liArbD8hnRPbwqej(e0CNa`}f9{KupsCtL#5hmry zdweqVtB*8u^u0VS!2#i%>c}Fx_{H{5m*+)>n6n&P3;|QF z8_v6fmI@66X+flS#!c&&?Rd~MfhNP%I=4L3_w3=^r8FDl#Wygj3+Q}A9!6Py+$|Ot zKa;O(sM~m;EKXHB1{UTJ#~?MFs~8po4`N8w9Bf&km^QBQp_oogkY_g%cHY!HV$PE~?J{D;)EXvTtCaw6;iPaC9A2DcW4^O6T$*SS zy797ZLMyi3T*|%h9lTb_a^OYmD`ux&NZ{hC{SJ=NzMFA#HYy_1f=e4W8J8E=6A5;Y z4ZZaF>HXR1DT-xfq!~7^3lnigKU$S{vpuQwh7C{FjJDG?=nG9@-#(F9YPTV)z8)eu z>W@}N+bY+o49^QT?O_82=a_~$Ofv0l#SCm0Lz_+d=0y?J>1W=kCQG_c^7VvyJ{8yY z#ds^kW!$rV9O4Fc892B-!va>J;TE?{;LBXL@QW)ZqF&EWawtCu3cXjQQKEaMmf`#L zX@5nDizY}#jp|@v38RxZ=KgWbF7^VyPb&2szFPA8#+->*)Thw5o2_HLv^NiWdw5Ho z^p6rZf)_qTMse98PIY{1cv;b*Y7)UhJWjhtLsim53=XF-<;fH?E?Yg9 z59eu;70Dr;nRm2+^F>$K7;n_2@wH@i@sbW0mG%MN%46FLHQP!@edQP8V(1XXuV0Vt z%~2G+m+O%atj2`eEDAbcuUY#&Z}LOj9x(Po4^+Jvo_VI+TGuCj*ZJW>I)`^B9KF#o zbT0Vl+A8NUV<}1AFD&&}YeN2!a^BY@V~)wHLvELlswTn|qv}vz;AHl+gWnIw&Oh{? zU$dGO{c ziQ^|yj?d`gZ?tR~+wccoOuMG)ZW35pjkmnhVho~I$`}5uqiTif8`>82f*i~z{FU^`o zXnD~oWn#Nam29H&khlNKFvUgm?YY#)UicrZ+8D-gSMnT3N}4VP5vQkdx$84O4UwD9Yh^L|oM5Li->*Fm^MmQ9=YWQOiE#NhD{(TBZp} zUfMccDTrPvTD`WvX5LiM=H@WTyvCK!#n~g?6G6nc+^Fw@`AUg*kuaUj(&O4-OFxsM zZl(^arL@)ojWzR}YD>Jx)W`mVTq7-->< zyhGXq(G?Ar-))yZao9(bU2DF3+ViBtt)W;|Ar6Fh`rGzUk-LC~H2CoQRA-k6AHH@( zy5FqTU0m@T)u$dD+xcDju?*sE_eRfz#sUJBd}sQ5^0;p9`fZjPYzE$wD=6|aK3HD` zM(sU}ru_Lb8-?<+gwHPo5LuW_o7bFkR*h&f<8dmCC?`S56ND-}jZ_AG5A9jG5Tqr* zb{_8>6+7&l^5}5+Tqfm0y_!*3=T0ngpp}?@RJ`j7QE$(i0%Gm=FDa6(XUaD&B!zf3 ziIdv$DWX-BYq7fZqz;2Ir5zXeoh>tXjMNYO}z!s4l>%VsC(57dBK4Js06DtVXBe zt6b8^@sTD;D>K~5AaLAG|HPu>BHpslEad>1*4;gNqEAeUI!v}Pb2M8+>NS!>_me~J zoVi-4vp%k$qUTzy6~PXWNPfFmDN8t(OPYqF^)f-*^F9l|JI8$fsj<)CO+NzRzbrO> znR(dp;nJ$m`e{pF%ePojk%~9`D!g?wZ9*txG<;b^3-1g$Ua%F`UchCe0{*Cm*(0|S zgJQmj4o=~874w;WUXrIcBmQ;r zo}QXe&uVhT-0asW9b>pjoeyZ|?(|tko+}w}cvybUsi-*ilY^y$ zqqI_6s3c||ne-4l*pk$Hthh3S{fip)tEVi?hxo)I>bRM9nq^hvR8hxb3SONOpLWv_HmCbx9tqbaWt8tN#o4 z`Bm7xFC()jH^Tag{opCu?tlkXhG{UD7=X@8NvS9ys_>(ZQc_z++6@EFf+(}lKg zOj?)ODpj_qT%ND4Ot^1V#G$Q{$BQthHK0#itNS{$W{28w;oa9)nNs%I68?x`yP3)$ ztx{#!O`@}@_wrh-SA>^MH_BQ>7oDt$z@~<~X`Mu4 zmadrKj})OKjnXCA%iqT_&IKBW?L65>3NP20^_c%>Pdnuy01mi|xZ_BR@L^>is2yJ$ zQS{qMks6{c5h%<(>x{~s7l`+4zc;YIY8DGsW%+zBRlirShO{4@$t1yiCdo`xu=+e6 zQJI&3L_RiB%J5`Z&_4Hs=Rf-0SCjzQAf+UudrS2{PXfG{S3H>YTmuWe=6x@`di_7J zh=0%X|3)!B5&?i?xx-qI`FBb6YrX$VF+Gg9G$Kkns^A^X~=XU&o&U z=pf)-MD-c_Q%us=c`xwM6lKx#joW2(-O>dcLf`_(;hz1OVx?Sqh^RxLz=tTf0Pk2zHS=|F<>sJ3 zcS2g1=C5d@H&OG&?V3tc08iMU@=JmLIr#gyVEF@KcT#9yF)CV|S3098wTymK2MM%9 z?vB5mc1rv?eFFKuv{95MxdYWpJ)}rr08tC8kY@}?EDM36Y$iY$<;3TAcvxVCP2*0Eid=Snw*Xb#GqLMj0bjYFXqs?0Gp6isGa7r_A>aM z(jPaaG6cL<{AXV~g~+p$w#;27a_)*_h}Mi-|4ZztzcGjdC3*dV4nSwA3~C!sESEN~ zLj0j$mLy3;i!bvVKuY5(lx1X-WLuz;jZaLEFr-y3xhZKxOtpP%yIpr8+tb2 zZ)2jzw5no%jC4H{Xt+@g_Nc*9BF~gGp`Y5uIz6!!o%9rqrIq!oVa;~`R+t;$j?e}9 z)Qg~nBPIs&{PrswwD>R~^`Kn_+4(F49Bp(hmX88pX9564x%IN{-_(I5<4Y|>-E{=r z^v3F=5+KxY0qN8|Ms|Q>aU0xk03&O8vl}Gud!4ODbGctY_pFZJDR#HyEgQ;Nr zx;A1dzO+LBeIVVK0O-!zuz8@rT0o{#c_e50c;S!su3ICxgH52jH@Av920c1Y@QlzT z4~usoDQK^b4`@RR?t+e?@%Ca|D!v%@US|x=EC{dnHW#>31K89@v<}!`5|npRYxh8F zy!eY%H2*lrftqjGPE=#u=)v2p6ta@&iqbtUJD&;33m}!rdSSaia8Ma%4yT{Fiu(wF zQr!>hfWBfhSY~mPnVv35!4AYjQ8E!1bb?2-+PuIkMR?{4rXo&qiOa~xLwz~=_xzChw1y>O>k8pezPj*S-%g5KMxsB?x*$0&kmffITgWwflk&y$m;$*)su2U&J~e zHY73D%X_r*Nce8sR|8DTB(-)HlwCjI9SI}AFkqq|>PsJC%rZzeB%UM?58vqtjut5g zWBq|*-+h=+2pRA9t6lW7m+%u>*xax0<=-2fas*K4)^Y*pbGw_3KYdDhLoj3bR}aV@)_gKF=}+DPaD{N%c}vn6Gxu)xudxi& z?#QtLsWoZypP&!@h2p1bEUg6WJ8+U();Ivj&)in|LOpoA^{W&04`#AUe*iq^87GEb zUs(PvhzoN!CqAaY`MoH~*TTMY`uC{lZG_76GY36J11-Q;O_RH+3Eb$$xwmDOZGQJ) z?gj1Rk1_I)x+;fK$rpYb-Rr;AFD4WD1!zb7Qvn8){YVc2GEeBE*-#kil8Qmi6WWsZ zAWF*U8M(!eQOws@pp74XP=k3kE5K1I5q1i`9NtCOt21h2NUH zIA)geb)=(nsU`x{T}^zzY+)5@=NAGO`}o_q>7lgGE==RAg^2V!9aRDsF<A<(h0?N(btyG#l#aw=VZkEq)Gy+T1;3H>+I=_f+W}%OX|=nk zSlu^wt*_cV=L90-{dqcwx*ujlxs&br?k&I;)1$*T&+rOmdRXp+!onZkdUECIHc%?% zmY+=~Fub)C+AD^=el0X4)sfk#V@#7S_&v){dN-}>wRWEg{biNx6EuG-)c~_0@I4Nrsl|>bDc0CPpBxpfGOs=kOqyzbMjjB zX>|_cqKkUHb^Wx$*9u@%G9e<*RYUl)!Ae;R;K*SUexu2Z>sa>$;?;U|t*sK*IC3B9 zyI9~d_xN^6VoXO{UDVuI$eQYWP3P)+2ON}*^LLOOHf!p$U;7-&L*@52>+92zC~=by zZdt?gxIiG7A?z%9{9X&eX2v=GK*xu)D{~z+^EF?CLf{TLYvV=>!UmJz!4$m*0e?Uk zh4C&M_jy`AOW4#&>nJ|n3F!~2^!n;Yu!JOD%z-1(TWqUfzMD0TTzKLrpjAGVDDAe^QTg-qWbM>% zFTn;gbj77PWf9QpUnMfOQ}50}q;?7izDD#`M~}A8sCZO=0g(5l(QeFeT1o;*ApK0Z z*0tufUfDvMvB@HJ_7847P&(|TTq9FTwWFz=;Pv3 z$G2Q?`WGMqFQtln>Xw5Ff$m!HWEpGD8Y=I_eB=Gh7 zV4}tR>_?mQDa@0&iA=mt9>T6W=)zu+{&Sx^Q@tXwICRq3n9&NK9 zTo)2;8~Gh??GrHL_VO-b$VFT&2_^xRjLLxpA^#?1JdkDSOt z2GuLrK{}M9?`;Q|%s1c#{X)|&G9=4jlFi-K!7Zk+Z2%`Y_q7<_Ij{RAI@3N=itkq6 zSJ~01;MJQxx@d?DI0>If)7LJe6W-C~m0Fns&UT()(XFb+-k`q19+x;fhR)q2|2=)% zNkbL6&x>YgbGIacbJWC_s_BIv=Dq&(&`yS&s+VP1eQ4Mq6(`u}tJ^WZ4PNI?+Bzvq zggLT^wVps}YLqs=`ons0d60Ey@(qPF%zg?VFwsSGW8(+-a~1Z?|xoXVB6-0z2V z>d;|t!v@fEJnZy{bv}FEFoG!DCDlx=K%mLN5iNM1sLFiZh<7=cio?)Qi z$W|zg`Hx5R*XO_pQoh>@?T-GhuPNregIs8@eSeJk?_UJ>CnXyE=>PwRnCFl^hyMFS z^IG9oT*k$BK@m|4xL92}*^-s5DfO~c6@cbc0_;H{#XeNj?qDK#lK(!(;PVjO@sTf* z2Mzrt?2b_u+lvLLh|2!V3bmx zzA#Nep5DzyL!-B*VkvqI39X$^5VZHT%f?!^IFLzb9gjtELJ2 z0{@=1CiErd6qj7dioYvoh0`;ffhl=9F%X1S@6$R|(j=nj8t-Wj}ol8wCgZET0-WxyAI+*|oh$%H&_N{L* zXCR{j+fhLHvcAs~2%Me@>mY+KioKq~5=yAB81FjMix#nOSvI^0+b|5EW`dr}X@$~M_DJY)(86lS2r_CZ8 zEZ9|spE6*98)^6HSJwPfV6=>a{3jrn37(8wtrd*^+xolUgsCswjyzM526@zyYj9r# zk9vtTSTSF;_t=fkg73m?$Yk;>qwfI*S?k6Ie(27aj)vl-Rq817?n6uC1A*VO5M{i) zoff~bS29Qo?(zigjJamu5xDfFU;WD91NsD8$+#{^U?RE-Y#Vk{O_A-8!^G~%;aWY| z*^Tf6*Uy}fP;<#2v>8^Dv;MFn3B*AIw5V z0>F*K_w*Q)b;Z@V&rqDPP+>dy0Bi!sLB!?^NDd}mV>t!|QVkdk1C4-i>heBM z!3D5v?|rlewyxXLD3T^{NXF=eY|TNr(syu6*4EOtNbawI?+^Av5I0R3WI=U5?Hox+ zoi$!FpdrcRAiVUcAgSj`z+Qa!C{!kwP0ipmfn)U$6G_BHT1PFolKu#)jB+`BRsgfl z1%UYX%>6b(uP=dpA>{Iq{qD8v+XUuSKpu1lMO8!Fvj*S-TnFm64{~qHF8D1}U&0K^ zK}HxeE`FxT_;QY!?Z5_>4f2CAW*uGPqjX36siPO zvn)gL?FPRXG030RyiOq?CLUZ>sR@WzWmvmLM??T+>w2^ETW>)|-&<3X)}H|s{G;mw z0R=}5*z}L5fjN9(mr!c6-P#_|4h6rT&$UlzxROiJ!)P5(Ll>bs1D3Pz7M;VU*8$X$>|epSjWnm4blqdT5;8 zX8=i>8&-QBS024rl-7sk*#t?yF>?VE*tR8n5X6VRJk2SMtHEgkEUsUrevLWcyb|v! zja^DcLX|={OdbJx6Q-w;;Zu^c*Yd0BBPq(Q2KreGth?eQLb-acYU{mFK=^RV`|77- z;)4kGug+f;*kTOM6Dpx#)ZigmBn#WtJ0Ux-*jIxU@7d9-i5E(d_8ve;z&qQ*E^eXT z##c6_WFO{l0d96Zn875qc`(wJLEb)X5B}>NH~7MDd=76~XmPT+84Br^3C7ZPDW7Ax zYbb(M94zCA1N~^nI?RcJ8RmWYKAB6;Q;RukcZ+444>{KyV4;V1f^t zB0*=knkB)ztPWzw0Xppx0rO}BI2Vb*m2;}6ZVYw?=SvVTTXVKCA=joKUT|sJpU<&z zKLr*MC_oWT?K;;hwr}MFvGrsyRre&Gxyf6~`=WP3^HxFcNn&L43g0A3qid%B8Z)2f z+4au47(D9W^c!(#Pux?;px)%eOV%FzoD3+Gzy=pyiEv;pNkC9EdBi^O>mgIW%+zUp zn6r~59&-ssC?X4s;)Ag)jvAZrRd^pw`IGYq5?HVHLV4{>H!d$IMyBDHM}iFru*hyb z|CPMA9$zu0`aMrEdS)$|WR^NpU!jD#IL0K#-}?X_Gm-DA$iP+D$myL+v%#L_Up`~I zxb)AC0&La7=h3 z+J^Lp!o3st_(DZ7d#68Fnbe}#VsX45*_8wZi=0wd0&;IYHlj&PBonM>o%KI;=Rt1RCSrXf zkKR|9WQmX9%nuf5soimtdeg#8rE_qdH$G1-K^bcr=XY}CGj`HP9nDp3pQ@QU8P6oA zJNp1^7L=8h$`X-ROP?dOQLfRrTiZh@qN^oX)Z`;m+{F8h&bTk{OrNYzx3z993=|%2 zfhHlVN=L&*+-$rjy^One1L>$nXRU-6Y!vETV#bdmyjNh3*YM8p#vk6|#!;~l!8y63 zWToV!WdE2UJSJcF!Iu*|hFJGOKbxG$bdvr+9}c~`i~*l=*oeBzdBXaN#Dx$a{Ra_O zGj_W?+8QFHn|-$I+(G(Y^47vm;vlSp<0+h7I;!(P9Wz;C889N(^M#07KzqkOcBgvZ z|DM(TD0wn29F;s6WpFmx=#!640mr^9Iu&2|JY=a@BGBzRR<~{^8q+#Gw6VL|W||&D z6tuBJ{M^K8JhG62s>yyO(HK=azfuFXOS%o;xpa{Pp`A# z3UxWpog*c*Bu*V^eG`I{{OsenNYuf++_Tp;91jHc;h&#>t9z#A9pn(}S6gXznq` zX!|C={mm&L22w-4pe@(463Ls07crveapY9^)?Qg|7?b~VbFTMBt+??XMYQThDY88_ z+%s*N9&1V!5QOPndngV@fRZmXarx5tiPl`qI+j@lECqhE3@;mLA0J@ za#^a$cAU$ab>@U1j>?o8uA&ozjDTV$FWkwd$&_c8{_T;95skZgcccx^Zs6*^LhD-{ ziPi4sb2kUMl!uO1MC2$RS*@rQEHs!a>LXA_C8l3I-bzylW+lSt+R{_vlRnbxpd5)e z8?+9WeCJV*h8MP(*9LVOoj!5y%S;CFuH64;7_zKigE{Gd6ww_*1`;1sGE#|sQ(jeL zCezCH@we`o+1&8RaD)#(4q`K*!hYFdP|dRO-VY!f8nRS&(@nj^#`^-V7wm9fI5mxgp~`TQBqH)VMc$Z&U=j$7 z!z8r#Tp^AZd6X2iV`_JIrYT&xP+&8 zR1H5P)(ML!gIUTLaJqEtOhs_rZjq3F^h{EI5P75D@41XXYZaELkAz&o=Y`vGK}<4V zRMD(bYh;Q+1WUu)@>%Qb#je0+F7eunnah<-@gxHz3!xUQ84Z4Yhx2+ipIx`gPPhuS z`znQ9n>LM~Z3o{<#UDg)`Ws=@vUTPh3Za+fuc2%qgEQ$@J)Fk$7}@CSsj1k`KB9E_ zgQ7jZ0i#@ipBoT}y(}43>mpSo3D_YM4v8q{QREHy2`u~Y$70~6OX>;jIKAa*BU|~p zRb~?uy}1aTVO^n4>lbnoT~r^AM=bqr!(NLcQCKQAGaa^^UY5|%RCvRTYM>#TTph5m zvU#bp=LeEBL~EC@!4VpEPvw_lV>`&1W0TD-;76Qbx+G3jMg^}CdKHknt; z2WgX=Zt&II_Lyt`h-e1azBfmfH_A4DL~7iei2N!QBJ?4PI-X{qIzhjr?YZ8^?#Li< zmg)#sMS%)2Y6(i|+kGAlx4Gs?jk+S2)f8S4M;EtWm-a+hJTEJ_Cu|;veN{{XD??2* z`5%UF#CrZQQ)EfwED%$}CT~c(_{g+hN0PHVI$~?$Djae$fwM7kch&mWX5s1LDkLM- zrO;_q!ZTdS@40sjofFN8=!mFY-i*6hzqHMWvDH_#$hc>enJ61g=ATZXIwIe>b8T(f z)N{hK3t?URt%M@ot@KSTp1UwCa8jERV;>(B4)Aps`G;zdm2A(q#HbA zuGm`8EV=VlOzoY9YtMp-M`V00cP(Y7WDawV(d-db^$co!%%YVYC>*lYe+V33E|r)R zTiEb=2}bzeBkhp6tnzSURQg$kaw?yyAvQ0W_elsj)oe!<=*bI$k@Le-@LHva6<}BWi356I{fNb$6C0g0Q36p zn=bH3#gAcSzAgs#aa$*%tfGa7=;T|U$NF>U3aE&4cm5%X*O= z$KEElCquQW?9ZNk%Hk*YXQ~QxvXeP=7YhG@74^_t)RdX@UzzPkox%-Hk8l z3YCsb9r9tT;41Sa|^bWh#U$0T`BT@{~*l9v)9sZxu7q!Pb&0j@0}R_KlMEH$?Z?L1w`3 zo9(BRx7u~A_4h_276z}hk|P_pJDGruSmBkXZfU{|*PYF$t41w|qW|8HWTC)?4eT+= zhyT0e{&OMX{je;#{%qIDe?I)r0qE46SoI7Xc1R@8oH+y6P*%A4plU5p?SK5nC)g4G Y22VO`5hr~F`0N=C6|32dDQ2ApigX literal 120838 zcmeFZby!qy+ciuM-8D!z3X;;@AS$3pNViIN$IwcbfRYv}9m3ErgAl=>XH9qhC zJon@M`}_U=`Ho|A95cY4J^PCDTx*@{+^;krsSx1a!bL$rAy89Qe2jvE0bZk%V?)4i zgy!1RP*6~nUnnSOswpTiYPvXDzp%4HLE%iWFf}Dr*D!1GBWzH zX<$2|pRo(kh3I6=Ofxi`hZ4`DYd52O&}yo(m35bO(x5!2w;akq3lwl?clESR{8c@pI6XRQl6UH@t<$}iAMmkiOL=m0igUum9zG@J*uM}*I zlVXiIWt@Ui9~7HNp!dJxW_fq{h~p|ll$G)EbNC{@T>GKZzKJ%c#V(EhfRF!KRz~1H zW9tyRAfMe$JA!dMnD|Mtl_{}{g{dfCE@LSt$u>&6BX`@2c4cDW6AuVv^u^6hAfb(^ z>6oSIsqfXz&D_+@%}s4I40cATxg&z2+id#Gp;g}z46LY?o|^T;hbVWzYitzMh!-dr z;1w$PxdnbuP|$MF{`DIS+#K|Oy+$WT{_(ycWhe@YEQ*?9R8OyJ}z^eUNMzkC@Llec|0o?wrb+iz|{E0^t4?~ zeSPh6gHKj%X8QJ8y7x)Z=h~WYHPbS-t*p$9X#a4Ogv)ws5u=j))1e_7j;0}>{SRj` zqV01I&IMsd{^x-R#8}eWWx%t4^`C!}P3K@#kW0TIBS!g$3!cUW;bY#E4!-$^k4^SQ z6Ch7MlR-uOhc61d!S2-je||~rOGm+j!W1su$%+@lcISs{MWxCyxAc?P9~M0RbU$5$ z{^de;;B~3Y^=ZrcXrV0y7v-8oXi>66t$HfY_w0c4E=nH5=MUcr_uaZ^CQg?LY}m`R zhcASOi5ac@=dhBK!I-JdZmv$|j(1y#rjj(IU-YE%4tQ=*+|e$vh?5$9B)Wi+|LptA zhO*|(rRC*`!m8Pq+5X4rwWIwKJ9Z+|P9bI==dnJ>=?l>Eodz8KP}xei$tv|e_)Q@@ z`AMBhV(nv6!J|xZ%h-lLkNUe8Xy&hGQbJCE^pKX}^QM$`(S2C9dJvzo#+T1C2U@zQDMC^89FH z1s@ZN1HJeC#c=zmNudA9X1-3L03>21JFwxMRzG>C0+Ey@L%?aSAnDJ1-GP@szbw41 zyScuwpYuQY-ox7{Ijp|ad*(%$;eR~gu~UE29*yz+&sDV`9ZO1&&Hz>Gt;8Uj?#f`5IJYF9@n+v!oyzp%yCZojH!4AA2{d-w< z3ZYT^l1rXUyB>qFuU)Yo^l2qgfj2Q?1zc>2_+IXHY{iJomh3cK^c#CEU`lM3v&zpMlM}!$6&zVs9Sm20dGrl5d%XG4QId4ik^;o zfb~itNIGBWyxWAeq7bAmi9Y&P$ab=g%EWF7JLqP+b|hHIXQceu9~s&80BBl0~U>KX~(%+x-#r3Nj`Q8683(b0hfTq=*}zw zezdde99ZVl+~Ny3RK{Mpj@|wZr@x|(!B>SY;H_kO8aD_`AIaQYx=uONZf$n5#JO3i zzu|VauRodf74|*0Yw@QSxgn$zhhO4Jt1&pNgvw}teca~nBIDR__SrUW!oOJ{-gDNw z<9XCdk#T(ue@_EAv3A9oBJq;>nhvHlu1YNFUNz%U9wR*2S=v#~@|<&?+H^oF5hEcz zaD1nHKzg*n|H4#|hocpYww>?R_f#`-pYyf6t#^(M&fH*Fe@82Z1~8JbKXA(BlzcaL zGN*QPeYG{BYw}$~`mFVM+ErtvVdqZaSr|#=&T@WnLs_XsN6n5?2c4&z80BdL&eI$^ zuRXeHm!7*khpd+~L-Yee*1b?0?gfea%f4#M66`Sc69JokTF+eUumAqvsb zxBMcP5Kj}zqb}WE#tax|7aIw_r2oZn3_o_Z%0;N>={b?)Zcq%AznS1AKT1inyxt>; zt$Q~2yq8mS4LcR!vQ$I|ujR=r`BEWl@?x%6)(YcO$SMZkm2B92c9>k=4etJmn`PU?5-IWA(a%1d5j$@^!;WP?=E zQBM)AJgXWML4YIet=zbdUd_%|6UIg{yWm>E`}Z)W6L*elB1|)yq_(%P%V+QfsCNYf z<5W;WSRuo5)d&3&GdYPc%ARHZG;180=_~#1xIt753)G<^BMA|dqYxv~1%Af1lxz)r z4D6F8=`B9l?eWqN64zkY)M^AIl+y3nlLXJV5&eb|pR-~5(?cL7@H4@SXe!ivoRXLa z8KOT#NG@8jo69XKOe=3WqC+CE%R4HRR?!D!^_q{T9Cy0?=&H!}xIpx{bPJofTpP}- z9d??tv1V|X?f|iX?m!cP&fM*4x;mKK&)6y!$~o^HWXg(Q%cc%&2%)_D48G`ODPz1Z z4_Pg+gb_b;0-0Rm1Mb}IS*)z6d`*nT<~clX+Q(P1{P11wfQgp-DR-*p)cE{5ME%r?A~P-?wK&7Epd@ ztBb$2Ej84)1qJ!7IAAQ9$MFApBsU?uBrwfT-5A`xeN#;(=~o>QNK>mVnd zS9ps!C$DXk$Af-t`0_%I)z&+%*hsTtY!g|6Y8tQ4^YiI>?O;0*auTx`Uc49t=&kb{ zq;ISq=&G8(y5~46X(BH(_h@i;Db8x2`r7z;e#xBkos%YkTx@y_ufb5v0V3sFahXb|^1IPNK z;Jmg<>YqML<$h*i7mNoo<%gjR^Zot)f={B%f<_>!?VbMooEG5>spmOn%o!PR2T;ky zZ*e_!122dhI(AW{SKAr<8g{)m4|@0TF(1CY_bOA~Fz{5ov^h{jk7)s5DbE3wW~nTa zS1WfksORubXsqM%{k5W2e$|sUZ&9_|GXD_Ms>8jYi`Wr*Id>3yQq?b3uoP6XFRxkF zBr)Dlsr0`AkRsy)MizypGOHd#IBHCy1%iVK-z0Q-Elte|5y~Z@zi=fTa{Oi7U8d)O zK@-dwzNLM9GUeEqmWx7!-fHuKqmotum1j=cA^Y4|)lc;r1tSD~?=;}HVk|#8?SLkt zz)!Q&H>kpHVkk@iqi1NMy>r3l62Q}i&{-V(H2*T?XlCp=GQX~Q1~xBDuedl%!AY%^ zJl|elRgA_1QyMR)k6d2QT9bB961zNUcWyN;Uf(dM$qMTAA1`$!%`ny7`44ucnTR3K ztu>F;Ac+9jg62dljsLj(+dGGvulXxvGYkC;Y|@L;j`8-XrD9iM?;Wx)CAc->ejkr-iQn=By=B{#T1)aK3oYHB-fM z_U(1rQ_}Vz(_Xnt|B`}_BG~ZYz`)Y?AyjW%dn->b7NkEUo}D#@&HHQT>-25yds6gA zAzt6wizaLmzVy%v=xFGhx7j`0HN?SNpp&KwEv6~@;Qu$0=Hx-HJZ!OXkld#01sLlu zwrla=j|3wvZY%Hx1O)&P3|GK+bB^KQnZw)O3mEJ#3;v3YL!9ts!hgEzYX22}J4v=$ zHcUqF5I?PpNmfs85QWt3im*sHXfGs%?vgQsf*=v+47$rMyZeA~2S!!7XP8s34O!tX zT~Q#@)7fdlVz?(MSvRkS(g9|EhOU|MR}3=%$}!9K^U}fXXfF4rJ*MopY4Pzw%`6Re z_jipkr{h*0G*~-@W7CNc^%h#oh0_<=-LudlMFr=}FY|&kNvpLrn!kdJxH}hl6^F9!hTBX-b)$f|l0S{xrivIyAWH4$qbfz4u33j#*w8 zq4-S_bes@>y+XaxW&)^KgfIX_6<&EAu@=zkjiTDkY1aYw_JoC&2pXB|38nL-xW`m`0WrG1x~LV1b-n2&U$jb4)$4=73O)Hd z;+qQuntKoYPi9j0xuumMljdr244wl)w_0LENK2M}0b0jU8yM(lRO9A+#-=)n&D+0V zn1!c-CaJhSR=ktnA$=t)W6uT&GGH%spdB>bnGVu_TRCM@#b3^NXv+h|Y?$vsD{X2e!X z6S93{@UW#g)Co|^NgV_G6y^MHcmBqh%<(*^)VM=zpU?G~WPhpaH1De6g-|hNe9*|2 zRwGk_4E(748(&1qE&z>6Q`AA63l9TZkP(MnhTFBPn|kS)X&8RvGi6GieD=Rnnk)ty zh9xI9V({~Sd|D88vOC(Vv3;%$x5LZ71_t?7vmINzcgcGW?1l%A#;X76KRD4qbh`7? zTl}^dY8XAM&Bt5+^k3H4oQFBRej`4bf5&s6!>K-!4G&(5)>il%I|BIFjSq&XyH~dm zi{h{72j0S#z>TT)s|x)kV<6w=^b*JC{y)Fu7d`8&dw4{Xc^cW=72p0$MUEXARM<{$ z0+N$Q_20JuBV`8)CQ~CyG{$-_C27_pWNe3FV0<(oLue)Z&peT%gWG?NTE@flXYpv( zVO(DxXuP}kBC+qC$iVM`Y{$C>l@)Xz%6Ksh-EX@|<|5CC}n z56UqGKx4z~{zaE2>0QS4aKZ`~Y2O|dKTA2^uYf*L=I*a-5LLTfn8nrh$#yd%%aoW{xRk}ue*2s2HVeqe* z$`W;cFJwRY6Dh@t%zAzBKU+?%G;hP5^4TcddLPA82Iwc&yI1x%K+*G-;6Bik2{Bwn@aFA?3y0c;a2mU$hr*>u)Zufu zYF1KgQ035&Cwlbri9NX3E4mBtm#qxf{?e=CDP9k5c#|jM1X7ID2(!F@Q%jsJC1q-v z7uO%->WuseL&=<|b&%lTAK(@mJtB75v2oi&K)xEImJ8|le)r#Eo2iZ^JtjRn&pJ{! zp8Jl}=Dev6Hz$T?18@9oSP@e}kDi@u&&0r9>hay<8>Ev7sCx=DG`9EVni`zw@)AI_ zi&|b@7HdIi4+pwx-}&*@MtAyCFqBo%hB$QMZcgoJ5O|#Q`4_In-*0U<7a|xul!Lod zxTb(``3us6l*X3RubYE@05YEcH2=*;WWe>=N|D;++{^qz!;H;XiS5;HBI#4pcjE5f z=s?7G$WHg)Cio*~Ol=}%x1SvfEZj(a9Z;AN2*Ir-ukE^nevdID&xr_zfYtG^4Of#0 zI*+sqv88y03{(`BUZ4-s6s;etL_-%Q+lWJRgM7#B%KM@h|12QYFcf38)rk_PO(3G< zGw}Qj7`M%Nny=#lbiIxMYz1t*SA=Z;9yT#?$kTI!r#D89&mqY>ShI?PTg6F;*jpC9n!o}=Pb2tKT&4IqtOD&6Vu0V=f%y6;YZ~~Mzxb{sfNF@8iDp->9TykcJ1@K0<)Iz ze%0WyHip2<1<0kO4cMp5f*!*^+PQ+`(f8#{p9j%E-mQ=1bND3#4A}~DE1lhQO<`^zA1^2gVSDdko z&FPdUdVWg-HK2~zA?Oq)(M9BJpaPv{hKJGntmZmQ-RvL9oGp=!B^-8~S?2V~(y}1H zjoDpYoKyjsTv#UzbswPCr8<(dt`2NOnVuiLb1TX54;6;C^&yzj@BYjU=@$}Ds70M} z1Wl)bTwip%s0;Cy+N%x{&!iA8YJocS2a}Q@(wbiq6;L`OP_f3NsRcPJC(3PVVRa{O zoUqUvcfZM+22(+1QA6q7Hplq|4iVX}wZh^tB{BJUSV`qYf`he!cOEk%w^0|iDYT6< z%9+HXsdlGf<581pP;7h9bgk!(WgqKZq6o^dg9c~DKVn}p>#(TdLC z%PdHdUwu|ZG?*%yq#y$^PxAp?@L?zmxRLgePsY2(yAJa0%qu|rO+Zc$f>PJdg&tW{DB|hDabB=N(A*=}?B5*vlxn<6{pD<0Fxaosg3MGlYiwv0BlU}!k9(K2krQVrI{dr z5Agq5Sfl;5KKoHEX&jNY$@&pHSRj*?=+!~Z4b(aYZfr}iejNg&@%e)4Fq4@@x&PKe|5jXn&4mkSBYtY&8V zKo!k=i4k_8?4`r&BQ&$MpgJSSRYGscwOK~WCMVM?~2#gkTq`BO_W{o7WC1!L@M*i&(9czpyQrF4H@ z7)z}?g7{=0C`Hj<=;cqRu7b2Ho8yQ}G%xH>OkKa}t#qjlsg!;?CP;xBVzY)92mw2r zleWb%eehwz(koUROVew>n`#xqe=gETcCARih`{i-tZPp~F~bbfhV&P?1e-wRcTEv| z>2k-#J#|@xT$(w@EW}R+;y)O32y#)P_2R{OcS61Tw6Zq{+oG=3_QRG_f#oyUi}_ zBt@B}Av6*9NkX3v3a`+6FC{EI`Tj>o#=-Hjkkg{Uxm4P#+3#t5KgS;l3B0_61tqBq zI9yr(_sN`>L1K#eH;+{1lA~$G1m?`-@ttn+KD0+ydZy=*LSRl3_yNun*+@Pi%$KGa zM}fI|*_RgOvHlP!+!tYH{&ld z&MND{Ym?Es(<5zC=D|D-hSUmvH_=LUIEFG|b_Ksqd>cU`yZPzvEUl9F?L_UeX88}{ zF$MB%r(zVZ_usx$2l(BNRQ0pTlic8hG|9WH<%J-ROIS5t9jh;EkNi*khY4sfvcv@o zXz-$0%Xk-f4nBsdr~D@|Q#74n&lwDHzql9~{wJTqGbF#kOTF{AW&!RAQS%V6rvLv; zj60v7O1$5HuM}km3>3nF*M0`|KJJP!H-ss?CbKU`zF7g)xD;TH8{M3{$e_fDiScpZ zep70W0eI7y7UZVijy>{U`O_P1`C6YP>=PpD`83>l^t}>ZzHU|4(=U2VW>4~UipuK5 zyoJ5@EvG>3)QVeGRXq&shc-3Y^8(Qq8_&S&31&v4H+Rck>LO>qn-Ug{kBJ1M^k|?) ztr|61pkJPWB@gP~!{WT(_CLM?PKg=uZRy};JSv`wx(V;mvZv}?xU7M4FKXs{Yvmfst52(TqV0724mtC zfMH4y_g)+Nq*H3Hln*M}_p6e(zdw1YouZgSPR#kj2omh&FW8-S&%F7I{(<%Fn?@Jo zZ;3@bs^5)FcoXRN-Tx~#Y@orSUC03n%R&LgKEKnTLn^cp4+r4Yl*)8gaK*%cxPK=H#w~f(0Bxr|kNgt0}RE`<@ zPF!E?I0A+gK|n3g`q-IL?CNlM3t0)BD#BIFDIivTnjPj%o+2%=lHr=sFSipKSk!R7 zzH1fbe*?-xW!p2gwHsyr%{Uz7=k!R6#gylqe`dovsLFxK=_H^h73xGZ0LXs~^xdU= zVNeoBnlS7@#BmizW<_U6j9&_xL|C=Qyqo=N*{>Y&-*|ijFKKD67ql;&nhj4LD1GXA zj!#E9BJ#n`W7bah8WbnW4J;F5d1P${ZE$Z6H^z7uu3Co`a5Nt0BH8v5CvvpnlNj~k zO~vJ@<3hZbmT>|5we}Ow(Is5+l(#c1&+uK6Z4#!1(JuuDB!_|0=LqWO`FD-$#tAM5 zBz7cy0pQ0o42j*D@tm{g$aG8m?l?E=bw)@d+zZJ43n*IiKEQvI006xJ9-`{R4@S*MhbsQOiVgG^7BnlAudbd()nH6S|SlX@kQ%{p^dPK9L^3H074G{)F{_^ z8m`Z6Z89WRb3!)fLx_Z}k$wm3Dbo-F=BdlWVT~<7vrB;QArHkyd}|tO!4XVq^?@6F zyJn-najxMQ7%9l%jc&z8IjD5RFj~g=*E}B_g)gse7df`G`1|)|RT>?<#k}q)+k-EC zYv${|vDZ?osWC+ap*}J>-2i&n3}ERF$a=C&@{V8veYdWb1lVxQC4@k{)r^lu=Y7wW zMa|$#a`m=q{By6-%f1cmhaN8zQUy7!fBPM8P88WgI(lmyficP{YAFO|Z<7vdl_tr3VNU_rp=bn{w&^eCbz3^j-iuyZsue^3<+>%a2YMG~C^z zm)9URc5fjNX+5MtH=o}1oP5DidpM+C2vnUnuj!lFPTv?fI)}J4XI0Mn*w+4f&1LR0 zG&k4!ifjwenC~pHqRo7g0(?6o`KcSv$)2^B&tsMXx_17=JR2%peS*Y2bP64S%zK7Y z=_T*M`1qP}c8h$#%$skOz_veJuFL5fd}xm`q&BmG;$bf{Iy-0%)qCt3+D9R3l#f5X8ahqhVqnbM1TeKM4qiJPs1iqbZ zw)=TBB6C1IYDcOzszkBUHpiu0#dSm3+XYK*iSLB%Z%I&2BfcP{cAw-u$}~+t0{Z)? zsE|2yEW!%wzPA%>RK;7Yi!VaQh|gcki4`3KiK0x<%caMyfIsqaz~qZOv8n*^0Vrv_ z_Tk{XEo2IK=9Uc4*6w{+d0&_4WPfaC1SKDIbUWntdrvhHN5unsWIGKnt>w6hYC>uM z!Px{2F8Y<;B4E*u@7YO%GtLx&Cg9rs;Dq`zW&3-`%1Z-`QA{)~|MDoJiJ#(*nRA-y zeKtnbE}w=;mQ-KP)O;6=9MVu2boG*~d^ytnFlg{*ulg;g%F2%j9QB$SMlb0kMJNkd z*b}xPg{I1iKO)ZtDU5lKGp&Btk!q#VuI;6STLeVs!~IWwSr{mbO84MX*7{~Dy)Q)_ zguIxn`_lEBg+>OP>i61d9NJ&`0Jly_sT)f;diKw_2~q28VqkZPBD%-KxCx#uf2JOz(Z7o4i>4KF#j1$Qlv{qRXw!&vsujzhUMdhK5?kMuskK;@t zGx}4E%Nss5Z5*k6dPOfNpC|_d4g>?kzpa!alkuACw95EIvQVT#s5@ z&qYd>vw&ksfQnq9?O0Kfbp9|gBrHN+ph-4}37u&obu4?~;0>?Iwj^$F$B*2|?C*GQ z*e0z}VB3Vav$&mQjj5PKwOjA&JYenaVA82!#T)}wT8k?a%~o7?E2G~FkCZp4Z0EQs zXuTYN-F>BX0IExFH4rIT1{@*|eDq{9Xj>9)i22Wgz?vGw98LHLSh3@s4~a+F8e*l9 z3qPlTl#^7UiD>wyEGp}=iJw-`3U=Xlo&Q}>YSVI9RrW&9Cr;{EPuL=uY5r^_+w%P%bGQ`9vCie$fgsEDtHY>%e;+fp1$y&?pB3H6 zxJ8X>J$!z$;|L#^10-4p(4=uXf!a8HdPz}_Xg+qyw}WQbf(NszhybuRGIE0yzw}Ns zxxwo(j^Z`myjDT25#U^sVk#0%$RVpYO^ZpbLN)4gg@kExev}VuRo{dRm`QxI7_eg_ z9nsM;o-$Hw1Ym^hG~i+2W}Qk0Ttd<=;(M-G+k}WZCV}!mI020~3tKoE{(G;d?|X>5 zR|j&d(khzp1YWikxVgh@{rZYQEpTX!r0DK{(T7fr994YG<(`zN@3;JLocX4d(Te13 zCOh`a<4nDen{m;4J@|7tcw%}dcpFaN^TuYh?F$-en^WHTHvDP~6$D#s(v%Clu+%N{ zcBhH5;Bw>MYyJ?=)!c|0KOAmP)!w@&S6v+0aJcE*=bKip;B-af97|%}wNF1U{nr2m zWOPRZHo$kTCmcH2mnd>_V~kCP5W54Jom5374$QU<+CUzw>$KvPqxj+wImYMY^S9ur zWRK!iFCPTF7f*IBhQ165&cmR)`qCdzV(j&*Ig6V8>yCe(4a;*>FbplKxG$f(ngmmjJ z%%_Vt1;!HSR6^=1%1+T3NTKs0IRWuxI1hV?5Y>9GOG$o6IuRqh%@=#YIfy@=LGtS= zV-hVzx`5SdLA!I;t9^@E-`9(Khw+ZtRxN%POAfhLoKnlh3ELcE7<}aOLJMJ*;c`1T zfszw>CyCh7WLZyDqXKPML+r_yzlUsr2zj3cX8(4Cd&FOGAe!&Nx$u&+JTFF)TKPlj z*_32{Gg7v-b+k!Sd7MQIb=Ln&bAj4KsvviDE5+yV8xL<7sCoPW*0ELA76^ zT!wyy!%R&zT*L*+_in(&666xc&7S}e&h@!vf*J_V8ROO=!{8j<;-{2?X^~cS;xRE& z0&{HB-&}V_f$_A`)xP`=`ia1cU#y-|lV@``*DG%~W@@W?to!@Hr|ejxsClSHOx>Kq zEz2^Kii7m5!Cc9%u>?Ibb?T26?{OtbqPc8>j#sf{A&&$ky^23I5P0oic$J}L=V39#5{h88LM7JX@&VyF1YFbMvWHeWBpmW1^ zXEV@rqhL7-SFrh=;Fpf^*ltgGI61oiz9PwM;*8Dto3tC(edul=-QgEpW8QS0$aSn- z6_VyBE%E`-L(SwdoBu`tZ1Hm%LVbB3pQx8nXWMOTCr#;33C&`raHUjn5%I2?&A(g8 za5H*vRULM=>0_xYB736l_+xDgfP3{IReCh*UtODh1(#U1q%6!!6sCxn6;$3kWxs|> z+1`mZ^NZ&Nc(*AovmVPkv3*|L!bA_-fuCFW36v|hyP7BHQ=>K&QaNI0x8fJ0H<~vA znQa;c#u_fS(AFhu_T!gQJDvobG_<(m0JEO!(%JvDtUe3zsLfaDK&@55Ynl2g;vRu%g`*CwW-8hmsiM~GkGYMM* zKftyTfR*rU%zRDRs?j&H)y{vN3GbLx{Gv6WOJ2i0&Z7(~F$=oUV7F}!UO~CyWH)Y> zKSj-wkDzr?g>zPBKcG@P3h{GDj zgIE?`{?yr8%XM}o4~PIuDXakd}O9c@1c)JBl<-ld`9tC4F29aD_5QZewDTo zBv@PRYZkVpGGpqYepyq=fIe}Ys(@4|D)aNu$gxN49qC8-8}kn)@itBr?J17FlNqU7 zF>6j^xTvMFnJYwcb`jucOadRt69LjeWAPfsA4gCBczUfeWtGFx5)Jw1IU3$HHF{>o zV5Rg^3+`pKR9h*nCX)%o_eN^!c&&;l<=i9fboW<&-=MfxF{JQmulUgkE1Hr9#XRnUpi}DbeeLaQ8oI33A&1 z{eW$f#OE^l+KrU`>x}PMW&rZv_GQb=fvRz4N%j76k3r$H?*(c}>^k*6 zN1^UdIgwXwdk9n@_^S;c@Opv~s6YhYUjET}fePeuQ;^UsEx`NUd9htTit=uO@r;j7 zu6bDK!@_6P-TS|nCV+X*zUOXLaw#LzU5eL0N8~_VEC2Pe&=zQdmQg^Vs_gt`^hdig zK-S(Lr13R*W8Zr|$awc9(vC}yZ9i4{aHGyLCH1a>^>E(9s?U9IpMl1Yq^G=l`)Xty z`k6KEv@P>}+=lSMed`td9-X^|Kx1!uk2d9oy}!S2w-_VBqxq&ACg%EE57g`ujXSEo zyRAJoC^Zio`J&ar@cN?KvWrv~u-ET!cWp+rPhqlZ>w?!rRJ|s|R7x>#g+Mosk8{uP zUBvTRzcv?S1JV7w_co7@&{~e`y!L)2Z{)e5e`7^sju(I>{aczd>Dum8}A)zWPO3dq0GRc`V(T$jnJu>MLC+&sT=qNe`keIJr(c< zFt*V5WV`E|(i@P2S3JheA|`+aMxjqH4fi{9kpN1QJy0AL-ghF=@r*;w+UHIII&#>M zat`a!1gP+CT(A7V(UM7Dzct`9JMHZOyoP!mQWBE@JKIFi`+V`g*s5v}o#UXJuHCNb z0Ut*+6?k(svw}_}8DIR!2}@xsMc4QRorH(m5#ZV;z6)Vg!B3I2N)sRB{^QiMj;J9E zT8W#>10#xatT@SiRlAGa2>Sgz_DlS%)DrU?Vg;L29;3|=B57>en8x%1$K?Aas^@&t zCP@8xiSWPWBsp^2ezi9PjO%qnZulQC(9c0Jx9Z-E&i#}9UcrI!0ht>+q-?cUrC>!`r%=NQvBVfE9;Ld6{_;Ti)S_lsO?XM#Z%J>oEX-;^@oXqRDY{8KRtvmiqYb^+>+=n1t zWw2>V+w4e&(c}rL+l=7xocHD|yb-ZD$FJ&pKwrMz`xex#`gX}>9qNyCB(~v;+-c1V zNOZhn=(pE)3}h{uBAbei9&BW@%T_4aBPV z6A*OAB3NxEm9b9+6}k0w+R=xtCy&J1*wXOdU{#fdFaD|;ek6fO#4&vhq@kD<4UhhB zPxGU*={%=@+0!rdb~9(!%?S*P{<)iWy@R|= zXyAMNwADtFfU~wWtV7_Y+sbGPl4}EXVuPSdkVRIm82CD5shLWEovzg_ZOlrGNqa#0q5!D_#o}HGpX=sj z`Fz_@H?uQ+%zhs`SAHPJGuWTiB$ZF?oC&bzIGN;K!ITEAZ^#i3{eldH@hlhu`<$Fx z1FAe2zlQ4gO0p$9i)iL`fxW|i@m#s%O}b-27ZjC`e(;s`4fr;tTUB$jLVNo?yar*o z6n20R2~K7KbCdh}&#qg-cB{wEucq9F9~wnm16M``IANk~=0|n5ly9YxYo8F(b}(g? zYIJ$G+85zOk_?(4Iy?ZGany~_-~~H8&vszv0Qgum#91)+JSB%;ab!IQ{-D=D5B6z# z>F{nBHI#~;x7;$As_Ch&(K3X>i)p@{N&}P;EG`)BIb$NLa9sN z$51g04OYB(5~zyofwhW6xhizJ`TTH07*kT~hknex0x0; zEG4QNj(`9$j#MpafQ``zw2+1KqpAysqj8VnU;7?!u8IGKkCbz>(s^3$&Nub5rKbCH zk9*xVC9xUYmcmY>Xmz1~Fs{&R_|4gMTi!Wf4_^|m`HD0mI+tTJNQ-Xq@j2&_gcGV;`&y`)k zE*kzm$>B3izJPE7v}DKfKH+0&6NX6(RUe=Q;UcyE>)N-hGLSH;Zo8 zG$s0}xgAfH98%6Kv&dJWRx05ehS9 z8SVu_p`wfHFhx6_1xP4E-O;E)k%DpUqeFE<7P+?96bEeQEbXG>sCh|rl!k%+Ds!nZ z!2RK<9H1PNz

6?R{&42;T zBSIf7MiNbn-*@34aclAE0s+U*qk$I2<^fH1TZLt?8N)b&=;fiINr%MLz}TjK2M#I* z)bv;_;RLQ-dR^J18h%A6)9(JFKTqZYcfMgAP@Wz*ll&H!z8AG^YT(L57gn2r~ZyDJR+BkROWl<*G0l z$Fz#PV41A^*RudDS3Z|N7@}$QEO7Q`YkwnmFVjLt02@yXW0#Wg0kS{Lp)w{lK2?EG zto3eT3wOX^J%a{@_4%sk;P3 zcR}bD5x+`ee^eI5)6kiz&>!X&Ho~Y^4Lh}28dB$VFAZ+R+YiwSqY{|nGsT3gTR0(Y z)Z|jA@^8}QS~O)lfCg|q2T+oyO>4_kL_9lW#ggzuW;Z)g|A|hyP|(k@d83V3Rr=Q%9+(r)ds78_M&?OQ)%gQHRCW zl{e7<>LIuzZkls!wv3q#@rPytZ|?vnZYKrWnuar07HBI-WKWP3qtht9yvU z#eTc>ijFsUUOjjzI6%s2^tV!Vxw2UEegu>(Sa88DBg7cxXRgIICe^cF_G_V7GYhMF zi^s|?)DTvo|h z)Tk>H5y04Bn=-EON}wV4+KK*hE*Aha?8r2kzyR2{pLnBpHRZ5?@JYh?!k-{Z6!6hp&V= zR52krWfvX)+b)yZw-Zk6>z#5m{7ehdhJ7;fLvv9g^%6s5=dO}^PQ_Z;d#{==4H&ru z_{V!+nl-62mT-&(*A2Tv&Uy4?Mz&0Aha$c{ej)`nr6IGX!$bD5KwRGzPW{BLj}nYl zzcXQMf;vh}p$vRc1EylMWa^jpvzwAFN+Qn(O`Uoki1I?;++%e}{U|2aHc3!<0w+0{({8#IiJ`Ca}7^6?oJ}G6h{pz_94VRTE(a;vrqOQS+ z`(ioK;ZL)~ne&8xG_JuAnFc>a$nva~4JoL5{;|<96UWIa(_2R@AzezwQRUhS4td=W!KGVLLTNu$LHA*VeCcy2Q)T#!V<``!w zn5Rb?(@5?Qm1TVLr&sQk6+*)gbREI7*h!}sc4g;jvkN7(uI%2J(K1e$)Da4XqfOga zYq$`oBE!8sT6Mv|=$50h1_yvugs5v}ZqL6K{o&lQl4#Un^sTnq9sj_UWiecmWFHm# z#J}rw9|ME(_o3;71NsMi8fj+I|AD}hOCas$bk#h6NbO(eeo6%#{nJV9kIa9vs+5ue zyZv~=M(|%}A>aOgby4l!!%@)dYB~J*Vfop?$`o*ckO}J|A2k8oL_#lyTEFQQXO$G; zBF8ZYj-#4QA>wSL0fH$F#>bvAxu^Dki$}^%Fl%mS3q2lJ`K@#hEG4A7d?eps;Er}d ze970RT)O&TJ-q{ddi(0-Hv*KbA3w>1t?`SM(O)N{0x6=v>`3BydMNe3b0KHLDhlX2 z!8l~>qC_y@EdHouKU$#QkDd!C{EN)gFZ-bNa1(fpOF(P1-qrb$J)o3TfGMzxf9mY! z3o^ZYK3VYfDW^%3y@>5>@13XLw7=NSr55NxuHuJrU_INF236 z;^4hu@Tq7RKX`>~M=OBGesItos00JLd5@RxKiUE2Yqq()U(@=4?E`|}Jyad>Nj>$} z3Sj;$pA3xPPPxPRZJ(x|)ZBhb3VW=a^UK2wavbhWdCV=5ZcjVVZB|5-zJ3$f zkaRPXd-L6|ZNK8&7j&7Ww#rNzj9kTud(kJVO;oe+Xi) z*|=6#LHy6||CE8$5g|)m%>?hWvRxCG1tZ0I*T<1nQDik1SrqT*)GRiZ>gnkz2MltI zV0x{>V|9>e{bvE~qEaL**eTj=`K@hHDbn*xFqc8CYY`&z$_i8H|NrsvH8t8-NjMLxd<+3$yxH)h;%Gjh<8-9^a||2Kp# z;55sV|CX0CBXAAwg5k<07lQBPJ4K`rLbQ7{w%2}I_uxrz8@YI`GTj&jHmHi1n0F@ z2A;2RNGybsO$(L0KG~dbwWkrZZe2ems3U-@+$gwo0%t!0>AsjUkGkvkeSF0x+y8eZ z76I*2w|9Uf#U;0C!FyqldR&50(CrMp)d*`Rtlx>zwZxFU4PqUVTKx(hdN7a~;?PXS z^d)uh-70f;jmOqY@KlWqTfxg7jJ~AX`V5kBC?f+2smQhu;@7g!r(!!!U1SJ(@(bCc)B@I$amkW?? zS)`Pb5(0vBmo$h-2qN7GNQZR1*W})Nzwh|(`}_MmWAH`SSTNUIb6)d0kMlTwC$r-U zWD};d&O#=m@)kwYiB8te`ED5NfEZGKC94Ha9Vo}AmBINrau#%LDkxAEy9V09+M=WL zh6#`+XTV}oVsCC2L3dOEB#6~Xbr`C^`~~X3pQvwl(+!Jf!QMUHOq&Ql6fgLQLoLmw zzi!W5`S)XmG};Vo;9*kEjq?}sWG7VwW&nirQVC>&<`l+Mj#LID|P(#*&m{Q=xy2)4O|>Ce+y2VIN}(PzMMwcb=&M}gIy zHD4}gAic!8#r*xf5WY^7{gRwiFBzS;)iK{QR#G=%bo_RKp$?Et37BTx1rd{anro=T zC;>ui+hS4^^#F)tG|ncke8vV8RK7FawGU-{O2HV-Bsd>fmq(LB2ATSC+Z5M3L_6Ha zAtBZ18sCSSt>x0Kgzjf_G_*O8;vw#}l|hYgiUnw(E%nDJv@roW_tc+7w6C+fySwL|NX6aQDfiJ6`oD0`lo@cT(!NoOph`txr4nk9aDro3e_6|AQtP{0b+A}j%zYVaKM#I_iEY>8? zJQ04207Ey?;4xb5rULC4AHSDUqrro4Ry~fxCXwb!wjfy0JT6pX+vFe15+nAbK8-|x z&-daar7@NJ;#imOlu-$nWE%1Ab56%*6b9|8TynogE536lbH3{$1wA3v>nAWI+Zbut}rRA zTR6*B*+jS5j3s~g>Mf>|l~7izo7^(;0)P_e3kIiY8voNxgXnQAbef*kI&2O3{Qavt zmbW*#i%$B|lZoEpM{cz;Fa-E(U>I8R4 zfEKjP-y^}HX1&yYcLyIISBsz6%goB}aBX9_fbR6U-i=JqAdW!^> zHKq8(IOJ_z%bWH-+fboazMgTQ?5p*IPg$>?khm}~uF>-zj+}gbUMP1Kfl%^9FLo3? zdH4AxcIgNEn}*Rnj3*;~c`FVw$HNY(9@}|RukZ2Z&P!!Rv!YvAEs==f;&3nyGg6pY z^|XfuY8;f$d{j+spd>cKs|t7x_95l_2%2Ci;+=0QsYH&9J5mXz2u3r?zd=kG#mdn% z&SGt7(VL(#viGTOX<6K~P%i$+iOx6_`f}aFP|}w{Hmv?+_IU$#2TDUL2PKO~N?w=5 z)rrN+*{1prF?w%?@kP+AWLdTIXqy9_7b1ExIm^@L4GzHgMpGGFcOwR31z2 z>#iQDQ`Lro!vW@Ou_K{qqzA)FP-@aBp4-sUQ)xF4mj)Gq%SZDU+$K|RR=DlfzRS?r z5a1=S#Rn0>d+QG?#B{HHFYV8e>kdcvhDF*0th#$+TJ0DF5jg0_rlhhp`e<>o@c!8a zAAjedzp0*=)q$E6n7|ZH%uY13| zqs3lAA7Hi2Bdu4zdq}oG=&7kJ`Uc<#^(jB^d>G(7x*BSh7mCl*)Jo;M7r)H@J71iPlIa-j0HgUpG_uzQ!IfRl?nSVKwM}Qr~a+y~;c{tN{uBEw%epGbf zAm<1-B0GjWQH;cK5cIfu*X6|c_Z}*`Z5153->gze#YWMrT z(>A!CpL1TmjM8Q^rMZLilC?(EJhk$ZyVK4j@%DS>=2W!t?pA>AxKGmP38O zdYn;OqiGtWzvWc>*2NENX?epS%A!Hp{uWt`J}#*Kk)DZL6Xi59v!8ZAtdJiv?tNK< zO`npj(@!7qW5uqv;(Ma>$`AM)0(qZG!TBHzKlGbLv`DvZl+T&I*dx3XT)Yni1So-C zoO;`}L4(aUd(8PuYgIjOO)nAdpeNY}8k%F4A!h0f8zATF!kz7OjlheY=YvI)+Z{tz zVluhegNUo!mt05!MS*5AyT~oDZ~aNP&-A&?MJ#j8+bmWt-bfTt0y#%z8Iy4n3fuv-CxO~^q;9m_1H+FJ!5~? zYyCRY4-f1{`7%kE|I9*0?ZB2Vf zzq=P;IAZB0v**O@_jE}<>a*dgi^DtH_Y8FjI3`>UcfP))Jg)o$HN`F8 z-jpF#5UGLUW&8_JLGu<`qoN;xOvQkJ@<)5CRu^g5_>#MS7)s>}w~VioQPPXR@;?}& zFeDDja8@P_okM+60xlrF^g5%}7kq27<|vZjXL1GAS%Y?wF(`ymlBJ^k?ls&XV|qm( z(FO6$Sn+OpNSIVpr(g!&_HzW|JXa>SJeuE5T`P%XSPvSH!`%ZJ-{692lq*^v@Mo4O z0xGfxT{aAcXTZO5l^K7zQJpGC{#S?wit6O9?*1s%`TTw%jtS}@68^Uc^F39R_sD6bFImuy`}#c3D2d1D1*q6n zl)jj!8aE-FKiul#)+sInn^OHmF73yn6f>DP2{_Q|0sxfd?*y&7eU5;y&>kl#Fiuj6>U`E@m;f$DJJ4SA zde@0#D}(VST1si761+SrA5E)2A23<{^2StfJH+h4_(LX(jxeHHPy{yme|VtBJA4Ex z+D(87Dg%C$x)%_vYDE97Dl`Cr1oC;Q)(#9I*Twh$mU>p5=0UT!27HPE zD0ATU8{z@{F_W#|toR10J<($7*m97o+SUkQ6f!4(8H0)vV&OWjy-|6-nNmJhYhrQ( zP5^64^n;LS$T6~2kP}@Y;d9m*o!os3iM}=n+!NC?^lue_C#@PlOciFg!H0hdq`Ggq z-55WCtvq%ICc2wXL`6Z+003w@CuKJ=2F7@30W8jfbeNJpk+1tyvgx|x7Rz!EL14)+ zT-IzV9KHiEq*YKbz8Q&=hX4n$3Bh%R>?g;813+&+L0$546yQqnb$KTa)B#l2{f(#D zP+WIeJN_b8vmHmD#3STvp;PLQdKnE`)ut)(At<=#>cP5KLhP$Ka|4J#rT z9Q1)FjsQr|U@$`*-;%S{KO!J$s1kJ9r>X7_Yr44&N18#2_YGnbw_!P;Pzrjl=Oshr zNtY?eW`3jgs6oATyU_y8&<_~AfxzDI-BXvK*|s3DRAUwde|SVUiYqsPpfiYD8u^+i zpN^!L8M>UI6p#4dWv$bU-Ixi#hd$*1Rnc&L`Z`3etV2-nbRU>4raV{Et$_-PAgOvk z!(LMVoBA9)3$_5GQ=rbY&Qibz978#9d{fk&fCfnVs?HFOR-t)S^4-vQz-H~7lnTps z1R7(eS<9}+OAJ^h_J1z{c#G-AM=!Y`~3e?Kc|3bfOSd$qHR zSO-9#sZIUdz8FbLgkNb3oFO@thFbF0lrn7zceL8IXC&Ud9}0tVYMyXeowH*`UX1A> z#2ynAVtARVF9kO1tqscl&r6M;)!5m>ZVl@O3-f!AHF>%UJ6UB#4MX>MNlt`>x%>-Y z;4ec($AUD{O~BO^z*;kQG#$!i=_{6i6Rm^1&VdsgnDC9(I-r2kv|z$QS8K5kxY#DX zb3j<*#+V{(BYU92x>!!Lbfc9}z9z(b_h19uUUoGEK41IibWMC#n(_i?wPD(lC-dly zK*UMk?nq40e*zyx=rBPJkH0%}b0BMHZLQ*j{TE8!cH++8TXshimYK`H_P^S00k^VH ze*X9=NG44$WBaf_Dh7Zv;rzze;Wia6zvkbYRK+;^`;xuA(}pU*WV9B=dD(laa)Jb? zRNw=*UNc}%Lqln$Tt?ANQjEU){xOY1JsnhMsQet{BK zKNADafL+4IEi1<`a9xUp(+*w%>7r9RRVEzAR9AA?84QL_xDkr12-aQTxVN_&V^>RC zGY*Xy&;lKl)&?1WLcXB02KNm(T^CAPz=LjeWJ2A^2&!Sr>w22qk+x2~h}k+IjTtK3 zzU;XxiEi)4(2a$)nWjF~e>HsiT)sqMkbZy>q{E1cG(H3#n48k%1n)F?HyGmM65_m8 z;oJLaCWBh3-)2!a%nSl$?wMDVN{;+iC+CX~&9ER;N~5~T4p6xi4+_t14{hEQk5q5VL^h;Va>07mlp9m%OKZS4sAz)?g?DS?L@Il z>pdU_J_1FtAPLSWa0{qlV^nwMK1**)D%6h2(dBkFo*-w+T4(M|<^%X1sg#i~+ftppO^tl<0Gr$HmAC}cP{N2FPPyX3eZHq5HRxLMt9kSmo{!=- zSvvjwQLGIc14wLI1jZV0)_)w3;^Czm7Z;d)s8s184?-EUS~MjwCZ6R+&HIMW!Pyyy zDf+3cf^bW;juIaWeY(Gd8j*OQf-`n!Q>cj9Nz$#@^BvUL2vH}m5l;s_ON|CxQ=HvE zGqR;60DWB|s;5HCQFT9{G{c(3??#0WtP#K)5h}wIa(?_|U7H38S=Vb>UM#5YidSzq zPlEFotMt!DtF-Hnm2-c=pD`q@Ah|@}&nOKp9t}9| z{9li$?6h{)*ca`s9W^x`-89&vsM*F^x3C}*!A@Bijry< zmf3>LLqwe~NAbBo@p+QpCqEG|yd2KyzY=un!6Dgs?a0YU!hJa)`>5phBjS`jYmFFb z;k4B^+D@D-hyA6|{=yUsx~wsW3-}8grUcyu5JWdD@*kG45^gaWa9kCe{o*F~5red{ zY14CH({mMkk!Db7rMha=D#B?CkTsdgv`%_Xq899gl=lt&H8<-py_aO_eBtK^K4LEE z{B+7Yyd1^7{YNCCly1kW`9?p1oWVJRp=g|<=p_8Ny;y>@pQ%*{T+45R8M0(xdT$kP z&vYVJpX$1Ywy>}+arF(to&Vbsf$0@Ei4;^SLcxpZZCjGaLZ~d1sA_( zT;O3=R2FuKW6%;zWcOlm8j=|)p`21y0tur&*YyqzgMT{&Go5tPT65dFi)x&m`xq!! zZ--vNZUIXEZ>wv5*ih0h1I;s_SmxdCSU~_`CCrfDGpNmF*{rarr z;lV?O;SGdi+Fbt>JDzsA4@ABSI5-cvWNl+MCYsJM zVS#;nuxO$YVsEetROw}&fO*r|jl1w=m?%m6?Ivz6{?dL{&tK2rn+Vu^J2gcnYg`(q z1c=8d!M74Z@_;#Xo47I*>fKCrYrLrn$NAC#oKER+{S9;H1bLWiYalg6k5<(%$;m$GN&U_0-C^4*js=o4i-N}#9KkfX1ZS~}n<4gbYnPv%Ea z_wXGbnlF{{i)L#a?rP)XPLcb|qO{xhhJRUQ*UW0|1*^^4?$*KA2fdhNdFnxb{|RmG z;UvW9mX$E{bVU&gCYg~=FxEBfn={1?tnn&{{xYg~WAXP^%R@05fx*Q=}%xCMFfk39MLX%rPwle7JlI$tV4PJ})z)JGpv zY}++j?ZlX#z%WzHvX@jGP}@oML;|QJNU52j!3p2&%A=;ZumYecRJA+k9f(?>FJL|A z%J*@h4eknKi_Ik_;yk1i^t?+HnUSq}$Z$$JWZ8LHai4M)3*ma230k6E?|%apH_qVw zMJD-t7;oT&XQjpJkI>8q$OqZCfYL|G22x_r^9eNchyC^U=ZPA^ow$&($;KXqBkHVn zR!>4(DBg;E?aQGjrVBOBX6XJ)x~_nAwoIGFpg0$`9HDZ?M@$qdUOpInYegoE(4Ai0 zcfITBKum*rwApyqls~clAv>J9g9qLKMB42t2oSceaaK5+>Mz0M70-~Q)JZH|Lcuy7 z_B=CpcFh=ZlNih5gM6`B=u}l$`CjNC!qLU1xo5_SFt;qn!u~wdUuR~tJylxTC_de$ z(Qihnk5`_uy0TJVCy=R{6wRk&5=_a4g)t}c3j5c^n+AryaFXs3RHlJwH6f+GT`ws_ z03r3mUi*z?aKj9P>c)pcotM??G1iQm@vvsGuRouLb(y;N-L1#@=@0dT)z}0N>IN4{ z{9BlYDdbTJ2>P$~Ji%VM{*2&+9|L14+^7xg=%lBLuNKMpvUz=G{pBJlMJ^Sr-GAgN z0a7eIrElP)&5A+(jx6SJrQ^SR{){YRtRzbuoW(**;*5b149+qUlzRf1G$^HtzU_|~ zB4|;9iTQ`2oqZ#h%IFCnCe}qn-#6XxZJF!?SZ^gY3YI_OEDk?D)jyWqjW#e0<&x(S z)mkG{-KL<~a>y~Wlp+48S1Smy!&{s4&z=wqOdud1dgga@Uu|Do^Kt(LhJ}@nezv;dXP8(HQn`ru zRioW{MS{Q!vZ=@r+GAe$pxm9;;&&hD9zFGR3Ii z7?+(KaIV`d_x39LWkalwmM%XJNeH@0Tre4n(?R|+2}mPTsWUx1^h9qDj!z8RP}u4j zi5a!r=Wp#tY1zpD!HLQJ%e@ zl_KW(2Q^N98wx6=oKk_HQe(20`$pGd|Kd!V?nh43B)YrbA6bh|FGRok!ns+XUxxTQ zA~H~%MN0WmM#Wt&%L&89D6@+uH%CTj(k7=WD|ZStijpODOiml*ak3O~mf!QRyfa}| z(#cpeG@($Je4-W|EK6pLNMQf7#3nj7C)1>kWWC7c3Qkl4y9U{;hd9rlb4rzic#=nC zo1~ydMDV-2Rs|0^=<-quR1a|JN{*o7XvUWUYhlyh-{Q`Y(2+aH#<(jkMY#FjMv~FaHmn6_Qr} z^t z-|GQ^gltUvhAcuZ4CRN|OE$pQ@H`0QhlC0~ss2}fsPD2;9`C%jDI8dct$_`(3|vL} zA#ZMz3RS;Y;rf>P)*9&QH2{$Dq&qOU>M5}6XruvP&YQP2C2Zis-$VOhIl)}y0iDt; zI|#nPXV#9f$~2TE9aL2SqIusu8LRB4ggzg{Ya7?ty;IE;nt3nu@&%BT)O4CVzj3Ih zkss4Pt9fAy@=1{f#!oQaq<;6mTSCgnxf3CBA>5q!4!gZjxz1;U+u9>}pXlsd0OL^Y zn-oO4ya^Wh4MJqbH@CT;8wnUY$(~jhegGmd`IibpvOxL(1hAw`{xb*! zy}IKE#$|H;(5g2^okwE1%ra}&Ee7WvAjz$^Gt)!kyKzu|oZ&q%d%k%f zzPefY3m70QOkVGWxC0nr&zaj*)yp{^hkc;keFwIl)VisO*a$f5v~*`T5VxiP2!y_N zC;YqZ=Vw(maZAFlwx?-Nfo*EzNVD)s9_>cXmFaT?a3*g8n~<}7QNjIGFFLGZ;Nt%d zbVG4r@v!J0AYfq$p#$_Dkas|VR@%y=_^@_Cao^Q>YL z4tP3M>7>?FC@`GSyxWd{1hPxeRc_xIPO>mF4k2%#ZUVHG*lyWy*ch%1Gjju~jP;Bg zkHCeqN_`2GO<$X;pj zPUKv@44Zld0xLCE>2YH>r*zMQFM((e7f(s*B?|S*1+~N6qCbGR;i*9Mi4XrD1o#3c zfkcs}IS2s~WgrGL$)S1@S-#+ReV(ZmOUgbSOan#3UfF?gaxGBc#-k<9 zZ1N9PGsHwHbRd|OipEL&34raSpH;AW#A!OjL z;_fNg#ha(!YnRw*I#4@44yAG6OR&YrA(cl#;aXJdg%(9Ta7O-!Qiq8mC{6JjUTvne`=7c z-oW^`1<(~$KuCcYE?dyFF_0;v9+Z~fz|WaS-2?6`QS=RfS4nWWEkcj1h{*V*N-bOt zV)`p5ru{wMcQckIS{XylO~DNvS~yxd&*-p+4f zi}M+>-}`G{pG!_%-OBe&;kkPUyYQ{5k;19%k# zk=Q^SWU2}51GRh-c*=JA%&Rt5l1)4|PT@;PLBk0#m1y{%g7GtuPn!I+^Hls-m@T&3 zHZ~!xjnCO(IW(PZ#p3xb>C=8kI-vO96Xptga1N!?R=|ziRWh>84J$c5Tpx)Vmg9BG zc7XV_>25!igkFqte);MYARn=Un2?_lS0|?;$4Wp!5^+AD6bM>OkDf?~ z7Zi-hv+x89LEJB^yP8L6zbYN^>#;fUKiq>th|mJC-O%%r>F&B=CG74)ipeN&Z%xgZ zAV{3&)M8!BGqk2^B*#Mt}a)>-Q+4Q!49SkLoWdtUF*G_gIyk1TMHcbC;bIvVW7T*9Bumj7T@G99;h%G$`5OQX zMw);Qx2QkMwi^^-*{;~c&jZ%e{gQ)d?t4T;l6bwn!X?#iEzn#&6E&t+Nh zcs_{6v9c9Ks+k`L3V|Jg_@xM2Mi`6xacDGHIJ)oPYL0Mg^zk#ATIS0Hn18Qdpj{Zl zE`rc2kOShO24xCJkT5RzI)7cg@B@O$bLUXH=&nqbF=v(8%X;NsJSA6kWIqBVl6aPjeeaZ9B#iXer{(sFmlJa1nUe8ka-Zy1YtKzX8l zcJC1ejK3#*HR{Dvy1Y8uk&pF@xI@lXv%K|``B z-sy%R20O{~X>FCGMc>wjpK2aBj?`P-3G=?)K_5B|q~bvvBJlJhDSl!{)q!Deq_Hcn z2eLspneh0bEi|ELAQ+0K`iNV;QqwQK^Lh{AXv(Z}8;Y;+1m-KNi^k*IgjA{t&%FhRU zaa;dEOJq);hET)kl9c6_;``GoK}_FQrf|m`Q1iM$3uyJIqxWU@Wpdh$Qoao@7cP;7ho;U5#_nc8a3@VB~MTz!onKRu4H*%U)9Ddf7Wbz z*mfEFlJRy7Y9{Y2FBTnvj`ZG}=}$k40*k5*WpSJ&Z+&+N;vQ9QF zs`53017;f3{(ycEohd~PsvT%DQM7VE`gS>|;6R!$`qkP{TUTthQAg4CX62xVJ;9Xi zE1RMw4J^-;?BLzI-A;pEV#s*=dR#MZrixRaMWj}aLNjayOz|y03>5g5Cd?~_5hRZ`CP?}U#I;XYw`*PU z`pU2Cme14#-UzvO8u32fe-@V2{)SnVgSM+v`Dycne9sj~_X&LD=3DFu=jbM(Z;|jt z$0Fy2VQT>74^`apc9?IYbr4PEt38f2l0FhFl)Jk~F@ZiR$NL$YKCH98g@Uf9RQ#6D zGhLAjs4T>xXc$4*tC<5B@;LI(%<#!hSUBOFy2#E9Xdn#S7PKMTQ|Nq0DUbKCL&fF^ z@Jq^GzYdAQAg}h=dZx=9N~koNblV?#ehki@#>L z=pt6gPg{S4F>)A4LcT=3WjI9aA26govS%x}r+k5l#845f12}>7Uz8=?Fthr>oD$o~ zs4S*h$)bqu>%g4AC>{4#^PkK6tLYX&Qu8K^SEG?u-w-9fc)H>@)u7K z`Zgb9Q#WP>_@nbGvY_gi!z1hJ@|oRUceE?&XlR%=Ge0j`)VNfWzXw_dxLr94f0IjUj1 z-NP3w#FH8jnW%-RIzxCkIs!fWc~xr zPxDFTGF}o}&>xeA7Aa+_g8DXf8Sb>LyUm95hT6h*=acR<1j^GE3+E4D8;nUznfOhg zG22pGO)3#`d|Y8nPQbhUwH$lWhCDFK)a@{|dEVWc2{d5_?RreEpLX#XRMYwCb#?Czur9bNkB~bnE3nZYrOXQB3^ed-4ct?rgZHa z0-by7L8ws^TS_z)n=f&Qkikv+DkNQMfZ~2k^=A{Gm)69v-9LUgL*&}A@@RcMbQG9h zY4AWZwCB&b=&B~79PVwF@eE3%u4XTc?cBD2f4#dDs@~}+|78)k&2g=O%zp@<<)?26 z-9|gn-Ao>>FP@*#`>=Anadu3sb0uwTV{{$K$%~gb8D49Aa>nkg3nY!oVmy};@AQ>g zp8zz9^lxx8}N7R}mFqsn*mp3!!!LSZ#2<;9UZ*C63 zq%?!gTYHJe_3FumCXQuYSu0aiju^P!gk|c#In>_?u+?}^zHX|9Q6)f2n*>^CXuC~B zZLS}?mU6}=j=g_Op5W=XuR7MXa0l^Y$yx8^4@T{r{>~ZWvc`xp{<;87!C?;uz1TWs z82v?7q@(hpC)?R2>$&ATO3YA}#;eIE3_(nF-+7nv!gZNK4M3kB{1p#05f0(i@|r2_ z>HaQ;hoATSSbk&Lvu1sg9hQ5?a&jxD18K!_)h26!6o_O-E>iv_GytHk%a2PDskG_3 zk{sz3DKep=cZq7UA|-Te@BZk2wfaF3h5yf^dHC;^VKW7OWd3t4ApueE&a|opb*`~( zm7f739%5qFDw$?8-gYTVv>wM(8dGU=FzFPE`CC|$gI5!5S%Uu6X<;IGqFB^@){-F6 z0$=Y~#9;HqIa?qx~9! zov)n-Bg9r7c?w&+q?1e&+;Tx^!Kzx0zIn@@8b+9{XR2y;H5Jku-2pZJS6#L zO!CTqBZcUgCN!j=Ci-IfZQW^D{-x6w3u4Rr$eLKw4^>eug327J9#J0mAHO5|b?-tF z@tq!;x*_Q3u<*X?VWej85mG4qJmtLYixH+H-sG)GYWN~dp+E+{-Li1D_65k=Z6s0w zzkS*1Q7;8EJ{+PqxbOV-=o5soFm>B)zidLx73# zw?7KK@*zK;0H2ifoD-O4@|Xr?P?3n&6!#)k1s{yOSC;u$YCwB2y^YOV`=M8E%A*Xg)D(myPclG?`c zdoyISe6+HFs)n&7}{`8#6 z?|UC;1vnnY4Qc)+qmXY3`b*$&?CL70jZHr$QG-VGN}fhD`zGk-)Q}Pl9a$0iNKVhUn zy0W9;yyx%17%#0CP94i+m)d} z)`h5Kf)l-QSCCN7{s|Pfn`!U|3LX2Ldt3I>}8U7jE`&PJ(Ke38KPO4;;mEr znda|%`_jPTLfPndDRBr=m1O~~A057FWP1`Gye2vttou@b+*}?TfcBBrQ`l!Rz*IvD zjoTu{`u!b-;2IosjCeayTZE1|>#i~GXw+r*%c9;OS$k`zVCx6y_TH_M0C90GY>UL!_uBFKp)FrU5CbJ>>q0Qb?zx#G#wCiWT+byP4 zdT%Q!EJZMfXzq`zS*Rtg#;SeKuuLhxb{QoV7M$1Hq3?Rqvq$z5D{9*Fz)e zEA>3WroaA6J&y)EQn2+JR^u)FB`@kIvdMA!x=|KZS;Ascm1D8Fx-qX`w1a|0kXf3e zBYAM|!)Vx`E}u!oCDur{fz_zrrR0W1I%V`*I6_z+F4C{iIbL2EQpk|k8x8w!767dB z;%SVkQD%0{b^DJD`^OG2@d^s-l|nkj+S%z})9MIJ&n1U!fnCP-w>Da}pCkSh*fVRu zaL7N_Ag$K|O@y1YaftE?d`allIEP`leDpsk_Z)o#Ip}4@=00hErl3i|bBV5P21uuY zjyL|LV)^SbYo{>Q{2Ldmq6(u4J?8y0j9i(;2lH(+=W3jfy#N*=D0ROYiAFuR<{wD? zy65$1Gy;L3H29LPcGq<7jGzwk-Xl-|m(OES=# ztW^CS099LD>bJH06mYi8;3Jcb1+uGqVCOXx;HbldHeS z(#urtFTj*=8|fJvA)tNJ`$%9#JALn^5ihl__b>g7ELP%PKiRf;DW#r&%Wks;gX}ij zvg`)mzr2_6HZ-8%k&^9yHc23jKNuRL7Mx~7_ixkVavy;{$y3fdQUB6?NgxKlH6fPl zbH{%UkA+U&Q>gud$CmdGixoPb|ML?-3jJ16mu&Zxe`~+sfr$E(54rdM@^5->jA0lbHVnKQ^U*MI?01^}>9{3rwucjQIR%_47y_F_;-$axE7zOxu{ z3hr^^eB!#nwmqf@cyj$**_h}Z_V8$-7Nkk*e17qOTdPK;J$vl4k+ptgr5&(Pj+L3o zP~nsQ_#qEIQ1e~8B|qp3B;QL zbTi=pQ{*;zx=1x*t(W*o+fY0ZKs}d;zCBB@Y#f{T3HaW(rxW&8@|2UHW=A~r7FNLSD8aD#^ZNP1ZURA(}P+U+QBu$cFB{fYck17IDT zY0FPVSdRhc9;HETIfYH8%@;lb$GfK>`C=p*#_>#mX$vUCc0g%RPkK5fm!(HGo?uVq z^A;G7*_V}RNNM#T?LW>Ds1oWayplA!rN&zopnoEv-gN;yo;oFlBFA5~Q2PMn4R9o4 zrCL7H^UY*!JGhTX*fkODa`gs-7Q?{6!zF?ZVA+gb1fD(%klZZ?vBe3HZ%?u?V82F! zo(g0(^ad2&V&G=Qjl2R`;Lo7lBcB02VE2z`>IZ#4-9d*&4fo;i6pLboV|o$#TnT=n z5zvI`cqsAFYj?f@;693hcS9n=7hLQ6Ql?t9)?k^bq%=k_>3)Z}YYK&;SH7a z1CSu_(iO^1?1OC)`&LJqh^yfh1ObEMom0RZw*^$rYT&&Pu9${Go&Z^34fSbW=0n3C zetw&kiN6^^-rq|P}cpROgOjh z6Z%KM&{vxd(jgvwck*E@sABiQ3mb;rL;@DEK;aO+>E47!K>j~0BKplm30km8tMB`T z2QKYV9P*?iI}pdwX?SXDQ0vf56{?T(kbqoJ7wriW zD;8h(irGLjh{2&y0 z0<32@BkoN{a-GRGK$t(e8G_CYG@=Hv1PMbXFfB{|AB%rO;ywVw(-H72|L)C~gq9Bt zTHJ*2TKax4PX{=4sVX^LnR}vWR>epzlvYsvb=yJQ+Y*|p;GMD?8^IwKF8XTS4kEWC zv)1^62FE~977!Q9-F;CV1EDE*P`rH#p!T-F)ldr=s5}Wy?zfpBF~ApDcEHFePQ*X$ zvX>SMRUJNP7~Y>-wA`nF`eUrD5bk?F~{i{5?dLFamm z2rn1sTwV>}gd*3*KqmGB2CnIXhRl-S$jcvj$}U=kIu)74R3}K5gVNT@ z_j2s6@MDFrMgXefslf{!C?lToCmP{`nrbgSo(t-}nw;lsb9%!RNR)mq+wM$AeV?fy9zotbrb)YuEI0nXAqz^zNjMX}t5Gp+?<`8)cqRSStTWv_r68 z@1@em|BQ%+(=BD4Tz+6B4r08!U@Ep8^J|@O^`{?LFxpjHdkosP5(}}BHsF_emOmb0 z7dun8G!(sAYk+`KqbE zua0)+Bx~N2h{D(Os>|iQ&15r&1&asVUI%IR`Mh`YjWDIpSd?0Jz=#Rq{9r7t-{@gK zUDf6W5otLuFkn8%?crW;*;O}zAbeDmwoxQZVLsGZ7Ooi(=Oe$dHjaf^t1sUPjRVOa z8krsP_59Fm7CcCUmqI@uBE&O2dt%>zkj`foh<_w^PUFahy1}n86fN-C-}Eyl9&(fa zmB@(>wuwi>7usvkb=>}#Kyndx+6jed+m#s~>mOHlHB0i^(`KYB1W=l=mt*fT=cU(O z?0-v7zDz6YeKJ5Z$!&Cf&u1coQ&hz8t+zv zhxb%jjj1@IGUJI~ufV>^QgQI9Yru?_%w$;|BoG*kK)2jU&JtV4Sy>{bB$r=BpvZ{_ zms#ZyU|{B8wGlz5S}qfX2a;x z2x6%!AOur7@K>3J(wBt*iw^}%AS@@*h$t2U`-U>G@}=vdd6S@SNG!FZlG|0unp-& zYznrD^fA!)A?}Knir@=7i}Pe4KvpU`-Q#U}I}JjYl!{@daAfSB2Cl-Yr5gbe zmT`S=kHRv0);tK;u0*Pw)`D|4pg~+r)7JI6ej%{TpGM*?QleJd23YfgXWYNp`KkTU1Pm zS<&_m%pYyP-HkCgmVD-M=w6x-1@80%tc!cq#MseA&9=KGEj~l=sf#RsGT1CErvrIo zFwkDHDJML++gp|YI0+@VisLo0IguOaDBUAV^&c(=-twu-o(}LNu+@=R%bJRvkA&fI zURK3j-NWvmT1k9U^*HhnBlMFpA3$4UKV{C%DZHPDhaPAi|EC@>Qt*do4n$L+Zj8B z&7SEbeDL>()4Pjd^jF$y?^)L6p;m(-(-|pL?wLOZ5{l>=mSnLQEh8R()JKML^=cgX zM3`AN$!tnsssztf#2;l28kxY%n40oGVKg)W9`o<|(8GrX(7=|ykw1!uCNk`Z1&tE7O!QuU{w%(v=cFDogm+3fJi5;K$SThtIF!Q$2XPcrtwd6OmMRI}GVBzb?3 zn>$Ee8il|54LCN6QUu5d=id1h-ZCHpnf?g zVi}zFYE>W}BEf|6O#38cW<()B9s3882AE`gk zoWu0QC;#?ZIfsVO7X=8=eGberc3>894XDL}yApYfCb)GXFh-MZlj|qmfZW(3rQS(Q#Q&1X z0IriQFez<2s8&jV=~e~?h~`$yMN;U4I`yrklIJ-{SXn}*N{D|wJ_stfmymH$#4kH| zG!c8Jdq1$%qY@ex4hBY(YIFD0037BF-K%#35m66B1lD7~MT3+wvmNxk*i$$MHYR%z zYt#AYM)fPHi|gG2CbAZ=b6o5V;PySBShSyMa4)&1p7*$FFi$zr!~W@~b%=-zjTPkj znkWwt15#OhRhEJu2c3=~nh<@#y9tmkF#FPf*#$ZOXnpQ)A48=-^D{dfeg^4r|q&Y@1nSwpbhHdbPFHjt7%XwLGv2D{}kr`7`Ot3!vy%BWR^Lm zFj{jcPg@=>3GT#-|M3w<1z~UgmyfVNQK&93Y!-B%Ux<4jRe{JaPn{Cfalb>VSS^0; zaOHBod)`%JAM(9ed*shRDi_%n)v+I-6&qSpS>$K-{mfM&`<0Yuj#S;#u=q-0d9T7_ zYl_O=5Eu$AHkR%=rBw+6e4VqB$F#FTBT!H5K)D;!FR)&zGzMVj1{KdErqsNrYaQou zxx*DMY#V`5e+?1&ZDQz|`;Y?;z7`*Bi@p1i`LBF+5!Z z7=U`E#?5K*n`9%iMo`SQ#@^ry*McKp>1+Y^Qx#w*v8nXjI54X{AyMK-6=0TtQF`Mb zD!no36ukr{aewP#`ZWD?ve|8FzTnd4#uJcxvy#CpzJ8>U)eaVGA)~DpPj?3s@5?~J z|Ba_<_t8t|7x+_FQ}$5iv;v}f+?fc%CQ@5(i?hiD2ta|m&7N^k8BgfAYo-e>*4}I= z4So)ZF`$yE0iL}nP!w4~f`Y*;zyQ0fEG)pC7iDRL9Spk>zb7Mf!liJTVjp`8xel8M znT`?(Z7$W1m7R?ZXv7vCA#3m)0|Q%+)QYf4uQzj#r9-N4v~RBl18w8P(@q6 z?oHI>2G^wvv0~o6Z*dEAlJ8!8%kGP=-3D<_r|upL0QsnPKJ=-(R*0sxB|tM%c?j5P zi}lTc+mj0T&A#67b~Rab4q?IwV)INPeViJ?7ay~*08U6` zszl$n`B#zo=Za@!cIzX8L-bqE$x20dMCacZtC##${{vY=KI~?o& z{R7AC71><2Las`zj0k0uq_Wo~dq-9xln_xOD?58;M5XLKlD)FS_k4A~-=Fur z{`mdz`{#Qc-{ZLcxbC~dwO+61^Ld_+^D%z3e*XS>B_h`4KMTj^kA)+jrp!Y2!{uc8 z2zW)^=6mmA=jdGx?NH=rFTQGn+&T0SV1#)a-sAqnm5)5z!L=;%ZodcBXYDMtW_@$D+=h7QEI zjp1%@?wtP2&ZWr*ToIJ~9boVwlOcUFMjOwZ3pdzZve2$2f*{U)`jHYdTQ5xPM@npp zRX`xEf4pyq@EAis)Xqi{Zt$keM-|&^QfbcWp3aaL*$*FgWl-$<2aB(WD~z%AQ<1M` zSZ+2l4?&I2FFb0`t$SgPX8a6wfksPu@-vb_$yXV5p3OqM4t9mx{9L#;f@UO(u2DzUhL(1d0uwO#F<7o!mt^opmImj~bB&X) z)Trfg^3O=0E8TVc+?uHIJ^}i-yl2SUJ&>s+fI_zJ?21o{;m-IRAD1KvYSSVQ#da(I zr6rSM{i}B)Ad)a}tD;iwz<)57w(5|MugM(r`3MNZ1FpT`{@-qP6^Ym!_Kow6pe?7@ zp=9Rnp9J?!R$a%P`7a+bojXZzpO_8E4xNEkGK z?3h{QHhXpH#@Ba;*1g?74i8Aa7MM47`rh^x#D4K12+Y2pV01=?#UJBWj!Ph$S^dHr zA-Csn@K+T3n+Vc;io!H#4uy`hDdgfR&L@8NdvxK^xWdfP7;LNpLZ_{EoKQ?nRNo*P32IE8pP7BfM-j zSS9^-f1}4}&h)721%=1V1_ir9E*}Y(zcgJ0p_jwE1DDBHTM$)TKBt70`TT^_MKrKQ zdk{lTSZrC0DQ4?@*!-9@|LpLsKob={OKy^I%DncUZ*D$!TSq`&yF2(g6{tv0i1Kvy z(A{0fD2~t6{b1&QUz4#?wg6Rj8kVMRYAxBhtIg2?#c%2{LtoalJwZ}~;>m4o43lED zW%WKUDZX~qjI%|vjBHk6P4l96QOyM654mSnFHD;cX>tL9l;ZuHycpE@Ze1~HmCAXZ zx#Iey!B_3b9_vBibUcBvaB7?32JkPjHCKUvn?~96(G9l~q&#BSb zG^0WKb2>r)F9V8~1W+WxA&>k#Q(uqym`HD5BDcUy{A$>rxBZ^K# zY=+KSx6ZGvm~Z%6AkAU7u=r#|3w3mGKe2W|D;}Fy75?p^M16H))P!r4wj6}px;}Zf z7aYpa!-L#>Jb^e%WcWPwnw#@Sn>KE5Y98ZLU;NY%ayPG{g;2{%98IK4XPXs3lR((r zg^ktNyXh@YXHN zv_iDybt~>p_e3%B+@{XvO@CY_1xuH6D3^yz%D_V&dGGBQmbt;weUSklw}mjyTk%5` zl?<<_#cW>>=GM(<1CmjLy4TD!7w@#PT7Ksk0|AMK*y_v=xYlM<)UQ(ygz>A0h^i{h z=VQj4W$)9NeQFvHd$=G=%r@;l=M#n(Wu`7e60V@s&m!d*sQ}}LVpVa;jH(3sB~2}o zL=;1o7`1fp7I;uf@%t8B;<*%}8_;w$6%n1#uXXQHu-!5(GpoS5nD1~WG7=fsCsJeq zOArtZ9zUTg4zd;igI(+wkUO1K8#M7)mP0aQ)l7V6JdjNguejy0%NO1S7NK*;EO9s2 zY%%u9;k;O%iPQBL^aZq+GRB^C*+YFTZguu8(lA}7Mt@-^bcsuPrikCAi=#MYz<%*g z+`gt#U$ohCAiZ}vr}afv8!;<(ktjFH2b{*MDtf3P?(g&iOY8EGav`8W{+PSjR(+Kz zJ5hOB@T=1?p&mBnU28t3>`szK4xwCF3X|jIj8owqUN^@oTCZvlkFBJd21J}du`6q? zav&g0Jyn0+3JEwjNYan^5jzx{+rT*J8P&g$Fn{_tEAHy9+`Q_X!< zOf0Y>P2CmeF{aSwNaif76y&_HT9~Y74ib4gSw=#=QwThF!USUi%gTridZyQ6B7Di_imXVZL0P{NG_eu&h!=dqzx zNmRqmg`<1h&ig{zElf$<&K^+pE{bwh;5d>RG4$keT-o+`~VAj?IUFQ8FOpM29|~J0(pzPIn@v@e&`?HFC|&I#h$C zG`o}}w!zMR)UclZjR0)ak_g9!(zrN(>CuvL!S#wSy|X_db*od?GlZA6hfopJ{8B!l z>Za-EHUhM9xNylt8_=vv97{3_dN8pbs@I%!JYMRXOG&m#)_1Pl80oXtNP~uo zc1}Cnwdoo@dH1#H=m#x^EZYBbtLh2jrH|wFfyNwBQtWey;myRFUtoGJX31?!bAZV@ z`6*BEQ=sET?L7#b;MU8&>@812&F2R4!Mvpf7jND^pIvXy{5%8mfn#jqE)Ff^c={W% zw{<)SMdYXq_8hexnbHTV&1wbWh5MH-{~_-^ftOhn-?j7jZR}!@#t91S*W|v-*O`o8 zJ&L%RM?0fd8mWE$d;Yg<9p2UpK@L)UUn06@TIrt<-|3yqjf-NtD`w{kppmHv`%oFPSn0V|!ds|ZRX8<4IH9cWrt0q2 zitu(wKP&ww4!BHLRMQHIt>)zF0v~5zYr>EarEJZN8&@2rZgxGW6DP90J(W)9R{H9h z)Ej%{)L0asy5(kyK8VRnKyOxVlc~LVa||o7Ge@^JuT)tTJ~&<#t0M(3>q0W!p(T?M zmBW~!@$HRX$6C$eYgOzF^W8HmeOr0O+BPM>`*7h?{^_4m2q_3z9!LHLEnG ze`-R>`rREqOv>{6Os`2Sc@@+DhcsQ3_Cvx896`w#)Io#1-=mGo0TQh2BoVP~$}tb( zN)dGo8D4PpC9dx{7}Z39v%j%0ECHFPAw&LXu;V3rzHwV!b(T7v9PJE6eyy_y*Z-JI zTjT8t)6rRriVOdV+VLALL!cNN0wG%Mf|uGoYySrMV0ZvX!dxm8hw|jVfAW7%J{$ui zX89GIH=Rs!yc!kd5)e8lP=pQfaaRR=I$?MaA;`|Ntf@53<&uYWds3Sq3 z4DpVC0^xlNefgROMsr^>-#X2->p@^Om;%W)6c7Fzr{`qB8fn}fui<*7VFbbd&w$q7 zzZ?vA;rfrmP|ckkPhzn^{>H^4;B!1DZUC*e?IjVoTuEyu^|SB{ z)?juw0R01(TAud1kOP>|m4YHtEy41p|LNRc@LD5>!qD{5UA0$69J0AgAzkS4_gxs& z8a)MRv5?ITp8va^|MvuKM6iBw({GN*)jj9g7WHqNI-i^Hm%Vx3?}HQ~SjE1SLf0H* zBizzr-WW0fR%=uhIS7uBVCpgOy*t%j^U|j7&8O;bjliUPb8(;ta3@}U?vSwm(lUbH z8$ieQ9EQWCkTEv^_Tp8Y4#D)Ry%@8Duc*M%%JQ*N%2$7-K! zsYkd8v97>PK((qN_`5o3&rX_}1EaTku;qU~bt`;0DbNF;h0m$xx5cZZUe+? zK0kSd{rhZ&85epO+U<&mXC1)WZwUAuGm6um&=r;H@GAz_c<&=DE2B?}WheF@CLXV+ zbR|_wUteqLhoaJUxZIJS#e^x(C;iIh3@(DJfdUd6k8 z3f~EZI!Dl9^4g5I0i}vratF*Jd5q$IH?V)*1n8)LbNUja*I-&-^nWAok6=SJsv)&| z({pzwVF>^~mjyv8L~dud3%zO?Y_t_t-`A(oeE~VG9QTH_yut*N`)wW+l50)0EqB@B zGpY>TdS+|1htN_yXksYqKXOnKnU-$fG$W8k19jycKtzQ4*NfKM8fylK3yB07sNz-u zRdmjOo(%bWh;T-%eVWp>>0w<5taV$&=AX^!!--Pcc-w>deu4UvYkEj0k0`3$Mi~nu zZ~Xmw!6siy5T5x0htXTHQQM6)cb2dZtGHkNA`NuOK96;ZyoZAzn?;oH5KPD%*>O>< zkWjd2G_T#|VHhYvqa_1)^Ya&+KN@^N!`|G!bqpl{(A3P)cXVUEy51q;DFq@Qe8FoL z&;XZ$)OQfZz2+b(Pu@rr1)_nsV}+0Y+Z$HUxxL>|?MLzi0PgOJ51|M3TM57v15l)J zmEFz!$HpwP#;wIiL5TEb)(^`GFVAl`tiv5Qe^jZx)t_32N9GP~i7i-t+r2zQWX6D@ z4o22Z^_s%1D>F}#v6*WYj@LlqtGno*HdL#IT8q#1BFA6!Z3rS21~IEhNkE^>q?L>p zUKQKZ3wAWxMPgHJz$&>{mG|g0J)p1Q>aD8q)vi(H;3i$X`{b?h`(7rKWti`Mm~tlF z{O2K~^^%lMO&*sB&uD2Iu9d+(+%6h2>5P@OAeYUBK6hDC6*aR4@W&qlEPi?&K{rGC z0(_?$5nXP0Q@Ko9Q(B))k#}&JPVMEH$0_Ax>X^E_0cjD7kXpV{g0XBVx7^w!OBjI) zim}#g$GOp1t9f;6{0I>wuS%^pEU-KRZr9Ahj+e(C;4=<_ESnik(GfXvOWIc{u|{F@}uf%Ah5Mi zpn2v;y^Ty*-!?Y-^^wq+*3}YWo`@FLv5-4d)B>H*!VpU_iA&5nuGr%^r(V%Z9+ak~xJ!>QJtnUnfi5F(>hA9WPovr;enK zg^8xRkQ2*_GzIpCoKdm{%8cqPqj^XFtSy2QWU?3ie2=C}?xUY{Cd_{GoFmC6Gy>ME z&R|blLP9Z$2iV=2#!6ekBK7UHdN+V!8Zk$1j_B6I#T6d}#02oya?a-pArzp$KEj3l zy_m?JfIS8j78e8NuI1b~Cu}XvPLp2@hM_~%SL&ieTGjG5|M{Wd|F9$=^H5!SOI!@j zD%EN}V}v`2x=0=V3)r(U3A6{_NlC_9==P8D4cn39F%9SS2$U{ zxlVSal1J@{C=swD4PRvIyrJKP$u8SA%qt|n@p-7WmNY}>v7U1S?Db|-?JSL z?C-7htpZy*7gpugZYyNSS7eSmWtm7puTqm!)+V%7Po>Ek)$|O3lzg z{&_(HyB8WmEISZ&-@yUXeH3|61AkLEqb`sPGGF2QKW`%eMMB2K?PmiBd>|Oo6DC

f@KA1Dv>8NnyTO=@e#eJqX&lN=;pJ-B?0K|>ep@R1`SkIRBfZ(ARrmT1jI+4n z=?T zwbOLO)n9mkI7=fz)$q?AQFCM!WZfqIP9ETOD$%zC)KAXPJ~`3qdT-SRt*lv!#)Gh-_xyB0ID$Z0wn~bF%43nJKb~ zmLkF9y?Mq9j|U)kHWyF@I~E&<4PH&H)su6*Z=XpJv(EA)7+IXR?jk?s*bz2=1+Qxc zAy`H|taLDeabma8%nm)e1N{*dQ^lRL0%Y*1sD0=Mqo1>EwN-3p{9Nnb2ESmoS7VIq z7n3uPM5b>^_Z_P5ul02yg(Ccca|ET1v#6#?tTC??Tdl-W@nfVwhS;9W@jJiSWq-aQ zWKu}ux}3YTS`bqpnB%=?CF=%Ge6mHD6EhuMH_WyS^z`%`N(_}!3Iwaopw@HCIeYER zO~my|$f_OHg4Zr9b{TVP7fcokp!pbtip&NfP=iYVtvR6MB$A`a3YD^-jJ8q zn{QgnIzw>ea4?}#0D6JSZ`ohnVR0{)V~F@ns`}a7fg7uU7&BA#kLE|pMGH+lWZpo78yiuRwUkGP9sd1P zlW8Yt&q@|jk^8QCUYeC~DEhE=Yw#{VWBB=uMh|+$%x4#DEMN=FWofH{l8J_)xu)u* zcLOP}0c3KKD^vi)F1G$cISTrG@WDa9ih1{_h>7%;Zb}tk#)K&(>YfHL1!u`t!gzt~ zAUPoNDz+2T+K;Ge!gpXFNH$Lc`X~Rnv)VpPJguTyph$UTkS-ZZA7y`iwm>v-g$Z?25YNnrq;h?KeRq?c z2T>`!)yS-o!fbBLK6b^v;B>g6uu3J9C_Wi%7FffYC5o*#n98T^MZyRQPoHVaqcv4r z?c;bSNfw}6{I`&e7=1jZ4Rjh_dCqyfh^wJ^4-G{YgtM?bV&$5>yVUdJdk7$RV72TM zmxjfJbMVM!#xALY5oO@(pAlQ4KmU`A}@;f_2R)AStoq=Bh5m9v7N%g&PW$c;FmdAL3c#g#5 z4WXip;J#-2@)fZTFO~1{t8eo=;tuF*>>%9XuDix;ck0Sx?q57?eKNsh^iD4~;GThB zzQWeLB~$IlI$zE0`84|1m~>6mUo{CPv2bYsa;B$RrMSWrfg&99um1iY&$ zMZEDDy~$bBdFyBc5=oKvT9aXabJHGnm6x(1bJU1=z8Z(s?FaO6H~l^KP(o%OJSZNf zUagaQP1~xFl7`1BDUkh@CG%N7vHH95Z$YS&LotBW!D1fuF z_B>q#hxF)iz`K+QN1|U68}B>T^1e3oAqPS$*Px!Q*3e2Q@RJ>P+Wby4IC~mh!r&lEvOIJzVY8-+x$&9J?;pvM z@#n8-UA9niP=7$l`Qz}{J|VFI;U%1_+4%y2W$XmaweydiwU!hChiim$5U@srA?43;9s-;Dm}vf}p8u z&QT*d!$kT=$9bYpd9*R;!8kU{3m=B7uRQkrtaMRi)v zM4lGnDB>`Sm5ud|&4!YQZQ|CCL*!7^PD#oi7ahlGJwYFxbhD2%|0VfDktj3n*gxMf z$aA3bLXAkp`u5Z(HZlfP;c$1YfVo6>O}c%={f@`{Cu3uPFTJTCN5`}R<`BtAG!YA( zm5VFTjGF<#u<|1c)lVr)krC8K8~da#z)M+ErUKoFw|jtbh%i@~_OH7w+Jaj}K(*EL z&RTQ36bBEru7^9S=Xck8mEMow#7}!!8R}CFmt+}9Q-~?&!!WpIUgQEK{y0pJlxV)S z@$77OyxM4;EhCJ6EOJsydcKX=T%rd*QNw@yTEGju?^Uii69(E_VeiK~H1+J`ye8SG zdi^IT!22YBAN6iXLxmYhd>AcxipF;AU|N5 z)pcp`8_#Ad`^|Qu+LRgHJlZdWgxz%WP{i#1Yt=>)*Y4(d#H%TGJ|>LXcX&$M=@;up z6+_2YLOL%+Kc+|4dE%i@?j+-`Bk?&8I9KAoSM~CkNX<7&X_1!=*-0dj_jy~OcaC0~ zR}$D$nYj!^-)ueGlh4aRCbWEn{!VzVi8XCGY)2@d%aQx3L~8CRmWE0RhXmMXud>YHDm_4@r*fRQT~9+h2NZOhTJwxBipM8b@F^_)QbLQ}#;h=bTB zQ;eXl(w6^i)$-?+v&$1h9UV}mhsV>d6x@B6X>Hx3Sz>f+^}dPqk1-{V(I?qQ=nC`9 zfwOf!953r-V)9}HMwLA)tukXrqlK_2MwSfrgJ}2vdI4w*hW@27gN~XP1|70H*Blf$LZkj#Um)d18k*|% zXqAWd>syaQx#=3X8HvRc3{O36p z5;5-h8CV#9U#4KGE{W*LtDN(={w?;Z9B`Ni4;5MdRu}#L+|ytZoHqU3nXT|&UbYFD zH5y$Q{Oe}=$LHaRMpW}F$qNtuh6Erl%gGO*(Pa8(qJQ}bfCU7wkjJc%(%3Y3{6`s2XN^}PQ4?=VZjiJ@lDJpIqZU;Q`bTar|e2H=q6Dv<=bD0w`aHu=#ivQml&3WSK6chaD>)T>|GDgHXu zSujAdu01;Ry!n~65wucVWeRWAb@g=bzkPUW6~Jg_zGKgj2BoN|h|I8rtV)V!sq*;f zHeYwrd0w8DHqqv@NavFG)(9kYs{1MzKmQuj8wjwRNtJ@p>p$xs~++*sW>@7-bRq-7-K{PU}-cWAJJ7W`cp`JSC8+-&EbWefb1F;@dAFnoz< zqbBJ?Zie?XM{AUBQEEmRa@c;jGTzp~Jm6V*|NB-l zw|-2rDNZ>tGljq#9;CN63Y1}P`jG@7+7=}Ass82Wzd)gl6;0OZ6n`?(GStE0W<=&)9-2rH{4EUDrP07FY+eFn%k3T?r2 zGbdZTX>nQ-Qj3e}M_$hvRBK&@h< z9Omq5vwIm;2+J9ry3y+0ev#Jd@=ceK^PFQa8nx&nIhq=o{Tjb;8-PLnOODrKei?bP z?)%~54{vsR<#N$8fH5yU4OoCXx;5wWd3x}edDnBz_Z5s zS1GIL=Pp*swp)Bg6TOkoi#|kE&2Z&lKQFl59-~~^-(?w`aG0r#-s#=38i@9FbUkr27bx{kTJMmpLwx<+uA5fsw(-Cfd)AdNdy#P)bw{@bs_nZ3 z+xPCZ*gyHP<5i>U3l@k~$L^;VD!sWL|C>Dr%-w${?|5%?S6%vs=!BBF>(%La{&1Q66lu;@P&M|X4We}Nt`YW0atq&yeHf;u9 ztzoe!P|G8zhB-1lYFEw+8aRu7wj?OmCJJ@@h|2>=K{G&nY$p$3wG99+Fi`{m((E#p z<-%<$9ZBsN1I_EJoQDLt=7*q1k(lA1M`8TZ?iIcT1ZCMkkc96>@rULVGYy~CC^q8? zVm248y!rh~X&5hedGCShrahJf;HbR$kR4zg2o-mho(0XBxXSGW!#iV6sti*e+` z+TQ~;M_0kP`{hgJQospuel@)WFrS8s(hCijT&dclNb#+kt{WZ)D{$X3KfL}ZVIfA` za#8lQ!mbkO;%N2zv+Thvc1VJ?Cl{%tsKBZi_?x{2_&WsGuL>9k?~ci}Z+;Nl4!{lm zJO|V6lW!NIo$@N@&+Nl8kMvxH^ggG)P8d~we;9QTGCOhp;KALXvCG?HeG`Zvw@aY4 zYqxWyWE%#J67u+ctix`DHoWe`!_lyTyUx%u;+)VU^D^kJ1nSBtcK?{|2L(psX>!5C zr#!d9YF8|YQ>tTr)NIU&zGCH_aX!xBF-?uY)g#}rGZw3UCuT)DZGh+CJ~ZmHZ(DoJ z6P}MC88QN;{3C?qcpvSbnWbJay%RXNY_)WJY~a2%@#4aEJ7pe(XCK{kk4mv)i_W*q zcN4Fh*UvYz9y_XdYw~_nFZa6jW9uWkb1O-=j~y9UwF>8c?|ql|aXiee{ZL}?`Rt;V z^g-pYxYWj#o8PbxW%3zo-z7afEZQGhvb|V3(x6eZpkpn1``OaySnU;jPpdCpp3~0< z_>W$9Ga4*-O20FHHzu>Xles^TB4KMgdui396N|Ytq~*9`cz*csc#eK^X}*!v#F~QM znXYzlF2)J{ZAJS>QP6Kd&~BH@W*dHxW_AKhe~WZlAYENX#6l(X0MXq%cM|V$`AJRp z95DnwtuWJ(;Jy%G>e9qiaXA|a&D4=$T2u5_`jJblG#=nFb&kWi6QhC4#SxC*fWB*> zRiix|156$8i9kp)uXc8e(FGB@8QAl09v}@iSRo{c1=dOJFHSxFzQYDq`p(iJw5oLB zIAl}-PCt~Kw69_~e*%0BUO=;o0!&Gn%ZM$4TJ+U!%$n@!Df#Ntyx0hIT13lU#dd_- zJ703jD^D*tTD>sBFdM$nJ-sbG|M1bu{muD`E9k1xeO!fX5OZT1DXkh4HVf)4;V#>9 zP{kw-M6ETaN|);A8hmC_Y=3fYf$ntN48jsXT~thu`esYwrSRO8oHJ~m;RAa`V#2~I zoQ|$7=g6kZ$NknL)5e`5lff~eE2jB}NXV%;cLG9oDU_8UQK4@YNqOpr4zyUfePtg< zi@-8Zl~wsR)$+%=o>%W@bZcowW;^R=O=f+p_@(6}k?kGJlMIq}j3(oJR3Fi1-R%!0 zjk?5=+=5N@4>k!Lnhn~jOF&&$QRz^7>X^|}xRJNXh!icZJLH>D)&B@7!%4x*R;J_a z64;g^Iv=z}fsd#7qN=?-`?|2rccEwA_Rd2*5?UNZR=YyI2fH@COK~Gh+wphbDm@%| z>$zp!SXv~!mlG`PJLhXpbld2C(Iw}ZLp||*gW8W<{4zjjadSCW*7@MjPhSMZJ{W0z?kdf1U}rk!7PoZmU3&Gw$Mm39{Tib?rMp-~ye#%#8~4=8 zOc8gc|5Q*ivNY)4kbqkQdDRgbr5$b@)?W`1LJQX{il}P+VW>Z2fak9AxXJF%p&bPk z#X?yGjNA5v5W{lQ8m+YS;Veti^+JaxYNaNpjPW7?qbY5&s&N~?aSe@EQfPJF2SRiX zl0H#F?b0-?_1mp&_v5(e2qhV6VCMIVvxwbazz8D_z?zU&T>2KNrJ{y8eX?HNeY&&x zzsBuSJ}AqK1kQ#X>^px@KFw7lYXh0-&z|_FhhAkNN45EA&1;8HSIMK^DrRp76ym!t zxkP=N-K=G`Os_1|j5r&{J+ct9H;FqHJO${6?uY6^I6SWNGtG9X0f3l1OWenj>`=lh z%9$~P_R|#_ zxlb8_8vP~y#laVW*hb`y7EpHu*|}pcZ$Ix2eQ^$(kL3#9zPCE}W!LH{?x-*d8kV>7 z%O3$S$!%iMv#WHEh=9&lA@a;L9|PMHU5FRNDQwmtJaNk|81SL z(g?1yjG{lCd1<->oD6 zjEpr?8n$nZ#9At2rO|EWX%x|&ka{=z^ts~G0C;C23d1g1 zu5ykD_hJkvNzP3o6R2lvtbS~cGOv{&cieJRz5dt43|wYS7Aez1uaF^rKt9n+vpwS81?nLZQLt6qDakhJu~FW`?c_G8T#%>%2Dsu!dl9e9mni% zV~%GRhVFD5SuGV`v6CNUc$PVO+*;&X#mmmxkihW}*MShcmtF(OR`xXmM3!swi`i7l zOZOUs5^lE6_B8KX-MKurm`FiFgc#^ALH9Z%wbuS0cfQ|U>IVY<;D4lomf>QlGWyVK>5>GJL86i16N8+b+orEj7leguHSgjnJ zx0!sqMT&!u_!78IMB<#Vs{^?-^#*~%*>|-F?TLX&rL}}vmdTP|%6W6+Y?G-BrXiEb zDt>B>ohc$W{ZpFE#I4AyDt>Ync!HO>(CCZU%j3`M=m-Sj3^Q7Y+)a8ttXl0gHxt&i z9kiwc^7_g@IGcb({C&Th(PDMX=rO)E?5d8SfhLZ8fO3{|ZjI#8vs}Af$AztZlA}o* zIaZwR9^%NLiyQCFWCB%DC?rOfTSSRx;ZDO zTPw&^wfHfWJdcqj&Qj%eefE_A;UyKXL0#X9)-Yqg7kJZF&242DznbS)t;;G z3kG~mI8d0<(W;1Rqjv&DRmL8eLz8{N!6F? zQ-yjZ6g|QXLz|}m8;c1kjMuVjWT_4-&j@z2TG8=%+9PI$vAn{3s ziFkdw^)$hv_S;fiDd^+n?X-zepAXbgWLnAEUWw(o6+1C;? z9?3sp+4Po)(bcG5O!IP2YrF*QW)LaS2|CfZi&As%8ZNRWL@8M{)c>%qjSAU%?x*YHm<>Th>ff zAZ*$B%Xmp$wTx8W1Z?%iv_`YV5>;V%6H+o-T}kYhKgpBdMP*SfQwNm^I$W8wl@}ur z-5h1+m;G^ez+1^_-v349-K2fCKZp4S(@qu_-|gf7E*qPZ zwn%^M$y+Dqqa?6kPcG=*(rBys@maw3zT2nz)6PCHFxqdgu1@o+pdiA*qqucYI4G4` z!Y83T-j}yS*rBJilvUT~Drqj_>Gi97xsfi+TI%WD?dnTCmA6`T9>hLoS)w}HkxL=f z-n|}L$l0z;B~aaq$VLBn$jDM55+1Hc3%z;oglwW#V|YcwlM?W`RHaofwi9wSo1j#z zP0lsE<{)!+Fge#w?f)M2c_Tk+>Km41h=c&eqrE(;J@-redu_(g@$hIW4qPkZsmlgW zg>C2Y(k4b^3d+&u7ZUnToC{I2>KabTkE*ISC@By~a;PY5OfD%g1>|5Qas4qo>*-Zd^wAI5t`)`LNxa3I>_lwQO z$qyfEpAZKTGlPYK7DoX8$rS0*KS=R)KosIl+O65G2=7$vvW*NkZxQMv-H*fRfw+B4_#I;V-9=rt4;H z-zcchenwRZWfeoc=MtnX5GSfQvVOK4?|j3@U$+(40#zRhJk!=xTyj{53YYe)_t&_u zYhtQs;v@S|NHtU<-)bsLecD&CHk&tvSeq!o;Xw4(m%TDrJ+qtARZJ>Y?Bbmcnct1> zGg_CPZ^pMIvWX?#JuMJ#sj8><)lJbdn{xA^(3hPl^CR=-wuDw?O#Luu#Pc<`4ZNk0 zonOk>$A`2kG&b}VaoD6zya<#yx9NB}Jj9=8mQg@|wPjr$VkCEh)aKk^>C zr|F(>_>C5uKk9kqD9rtIpkc+BhtKg%kF6iagd>76w3ChZ8TPk%hU}l7-4(ieJthCE zM(MpR&-)r`2g_)+p?;oTxhI@p&L!xSpKckU3pHLZv_c?}4a4Od&x3eOzCG6_Pgm~3E z6(a4L$B#Hi@*^i6>`}C#rQPE8pAL3+^O~k^T;0qzd{*&&h+m}CYj-atL#9_z@C@C? zFtt3V#>-xwbmtK4o&dIf16PAlm26 z2-^9J=NpywxYg4CeRmH3`%Gm0)bbOv2jd6FFbj*Xzj31#v7B1GOIhD^M?FuR(JF5` zOY`~|uAE-~fgZP(As%oP&B#+ZU#8Q@A7Aoy=zUYNlyxPc_SUu(#?jpNL}FuFX%nBo zAHQvkj4kQcvp>^HpIE<)c^aHwC0-4G!IKxmw*1_6BQYG*$yKSrE~vz zt^fV?Z8|(=GG@Y@NL9Fge|^;(r*J6<&pD5H~>GLHr7!DJ1DwqQHvP z@R?6eion7i1T7WMiwRYL`OFC~;Fi7hNzu^y^VOnINvkI(gp{`9IeC?L48V1{5u6sT z+eR4PD^xJB$&7E&5e8?{RRf4r$_(udRtxW7{ksfQE5 zYdj9AU~lxw`r8nX-G#iu7Usf3D2Q1At}Af>D^J1*ViiDt-uAB?!Q8?nTRSS^*{+o| z%m6g{!IY)}WMNthQqo|_y9%1kZye1TU5ZKAlA(qb~3OwkUf#Qev_A(AZJ~JpbI@p`;b@Qc6z9hg|r9ecDsy z4xnybtpti}r>mdt_XhOe8#qk}$`>>ix{5~Fc9=Y?!O|gTv*wfBF+y@u2qn2*vVaf@ zC4IrG)zb^Wc~;WQ+w zks@I11(JI}9M--r0hR}ic*ppt*spx_15O>9Ua^V#@v51@vHfE*~8$vPRs>QIL}0fHP-6bAfh${c`T z_qE+%6)cZi9cTKgFZX@p->51kINegyUjhbj1X3Smf&5ifXCFJJC9mrIRHU0yu?dC) zqUs_eE}zl;G0r9D{`~tC-@;>Kx#I-CpZ*UR0U9_q(>G>BOEHtsCR&7_w%B+mP=nQN z$KZ4VhPQ?&dZ;Omfmk|n0tY>}G!*^LunHK3g`G5NeALEZlXbA|7h4PFvP}yQ`HGZv zP5OW#cbiWXAY%FVZcCHZiyU!=ap@|jlDN;*_Gn1hezTP%TQxwY7I^H$YWx*9zg1`L z6@kA~2nf>kTw9L|gggWb9m(;7f=;0h-I2TD0$B1ocKnavOUN$n@`LKVv3W54WTiz@ zIn4B?wzE*0cju=7&%VphQ;$~q(FLA6W870V$2VH2LS@$gO*sxDL+i(#LmK}*sgN7x zQ!MF)2THhwUFbwu2lox&=irJ)7)#+D=@0ptYinV(T1 z$3yp}n*8=dS)G(!9<_36se@_slYU`#((7EZXCA5vNz7&r#{%&FsXQa4-TCJ5GuzVZ zJAm)!@J}%n!W6Ap3vYN+pz=q$_|o!uUbm%3N55vJuPsgO`fbaSUF*M-^R@xT1#Bxl z_7>H1S!jDL&( zo&e?Gt$SE@GeJ46PC;Q#tV02jvT@5M(9INAh~x}^Jd`4(ZEJ@qQ6u;|G0v(neXrY_ z@-a~r)Wh(%pZ7t{%8TB)F@!9;4Br+<@ULllFI zi0FNo+}hmS>-If-cSp6)$oa}Q=0@TvMm|;28O(U*825}|oOw_N2f41~_Vg=dCs=B& z*6hNa_Mu?>FOldvyQJia1)o$v9HJfCnwd3YP&yG|NW|cLY89DV8KH?N)rny#uGQwemW%7;a+^4`;sW6jf9#w8BcR$R^=N z4%}_yc+BfB9tnnOy?!+Yrt6{O7cao=lMy=(p<@^Gigc1cYRV`q zBT1Xu*A-P%6~jEfC-I)4Mn|2#dBQDjH?Sf6b2>QKU(;{%i~5p8zh~CzO6~Sh`+;`5 zwldY^Ueeic6O3kv(}B9!Ox(5hGajfI*aXW}2NYET#ll(dX?2N7LptaiIYE*mvnyKM zkKJ+FO|!{YvCL^dx&<4_XF?cPYQdtaP|}1AB}heg&8^sRP0_K{p5IbWiGv7| zu;pn@mY2_zm1g!c8b(`8^;;zvpX6OO{{V#6fn@laQGuJzGQ>;ldXig|#B~~NWI@jzKE5*F--d-NbNDoX;9X*5Fc(vgfcmM7ya*0p&RcngFZTy+s znU|y53^R1!&xzj+SR>A$w449w*mFgsdFWz^_K@QX*7d4_LzVG|jQC#KJ_8C$MIpFy z+}My4RxXIYkJpt?l;D8;MwGsTsd%_zCxq&;l2M$5o$BhMuSHY1jAg# z#Fkn#G@BUrQIdw?ws(&jE?rGp#CWKtUEsnlf8uz*)uvqM8exC5*H@cv(d9w2r_!)O zfsgw_D2Pt`RkCi;lk7;)Y8SNsP|wpGEGi6c6|3;#>lM7 z_EX>5j$6fvpg}Wu$iBGgj%vQhL_l!Ah}pl->*EaoGcMrMW*^q<-{=?jpp_V3YlbaQbz4!QsUhe-y;_H z?9nVV@~HpD1|Mf0z4`O`Y=HlZxB{KX1fmo&$yYWB%z+FQ$7w=C zOokS*f`{B~%*BZ!5=SVBqK3qGuKmW=%fH+bk8xBfgC+jaoo?)%z%lYzA43#b%+3CA zUf+X>8~5G5Y#`HP5#vB*372?L-ZMqhc~K6x|MZo1&%rBJ?sU!4&vbD5T&r;U6*On& zh!yC$2uQnCD|%q>|Mgi*rd-Y`TIq#4*}3Jdx%jAo!oHuEo}r_}PINvu9;m9zENw7i zqBS}G0Z@5%<2J_}YetTlM=!DEFB53u)MLkAh(1=1siKaccVDB!PtAWwqdL@7kfFtJ zEJ?W~nN~yeL}kku6r~uSl#zQ4ed`D>3?ov+ti|rf3c- zn+Uk>UU@0C-oiL|?axIOr$79z@tsu7?3quDJtiN#;}{NnbT7-%=$oUjVqvSXG-VePeuo*M9@_OGZX_5h>&$w$%!>gIgc7|)wJB(!n{|3B<~_dnMC`@ZeE?99vF zTp=Ts>{VupC|g!2+2JDLG9#O!GE+(^BW0Gog;Yjli;$T_=zG4p-|zd*-TQ~{AMp95 zM~??yp>+V(iiSs=BM zr%@VjKs8C$&3XFLljNi+|0A(=9rEB}N}F zzkFJQ4!o{V7^Ql#J9kf#EUAR|_hS@jQkrkis#Lkqyw2YLb4n-P+}CqO==znjM^8z? zQHuW&#+Xnl;-+T6-YuzWWNbv@8$y`e6NSvZ+V!fZVVqIH%B}d4n z<<<&4%gGvy#x>4UAt4b`A(j_>WLHE~(EMl)`Bg}NtVPVeUYl2{s>Uh{nEe3WH4D>` zBpTbJs%fGTOpJCTuC-FJgt??wldcowoWlsqI5WzoC{BEDqYh?Qu6B}O|G_27vRf3l z@F>?h*(6Cu%sQ-&PVK6|%U6j?PLcE2{HaVH555rkSG_u-O_$cw*V2oU(_@sLv$C{H zRDG&$rn`^fGPd+SbNTfQhfRN$M^SQ|O{Cc7V83p&T7@!pV&U2(XPtDt|32o>@#~KF zdY`|3ChVqC>|C+fkvVazZs?E+d8U+tDbA_{4us{^vCYOFe2a#ElkLGw`;N6c_B+Wr z1hkxizce3*B$q#ttP&(-5wu;2Tn@0gH(t@KABM+E-J`qfx%Sllz_ZB%-#EL&bRcr6 z#84QzMroPe(>@iat7CBMez67dx%eAkS z%8SqJ0)QwHj*uB&sGDo+e~VLCZG9xR_}Ty4wH!jys7otA9eP|DUZT37+&u(8_?W34DvlwgNHhJVhOotS%Y#s19u->t+%TK%e zSY?-6xbdZu<^mQkp-o;cx*`o6GrExH6bX0O6$R-gi-pgjKH4PF9?@Q(TKB0_+9;te zjfIe_oHcoCFF7ICjLK{fnEqthW+p`Ar_rgbEm#v}bd!JN5N2X7sp!y9+@2l3=PasJ z6iXHKjN$1i z9p&|Icjl-HkraoYN-;-zvs+ zva+m;vHPS;@x5<+^mKB9nxQo7)ah%HsQKum%vBTjW9VjYa~t42xgD^qN(n3W7)i1bYs8Nk#om*jlJf=kV~>9$P*psR98K#zqmG^{xWOlqzP>|eI|>ax@zEo zXY0MbeZBw?oP6M|bknRuhrQsnTSyOun)GT-j?Rygz0c2vXFiX0a=tzpAj%&suh@Q> zVxRVNcW>b`Rb9sVV<_p`qqV~59QF>eXD8&IyiM$3Qj=-jOr#}zWElFoycaPrdU@b< zQj3FvQKZoC0hWV$_VIA8L+lr6?wQi_bP`On*z{NRT7Gkoy;VcjG%weEXsfRNs0k=}tN!NFUic{0*Uy!Hoyp zP!^#j*es9Lt!&5wuo=ilA{cOV)vzFC zX*N6NRkUI#!$k_vRh!;Ov<5VV7DM`dnR8sWAtC!;^9-wzZ52UO4@~}tPxK=z*KYux z9nR@9%sUL#{mH}Y0PD6~iOr#Md;b2c4e+^X9ze%6+)m##k&?uQ!iU|CaUcysnF)aawy+sZ` zqVrmR?@~cbn#N0AAcRA;fZ}# zU=#hF*HQJXB=c0^`K7(7GriLtBpGqiZXL)!2XmqU&_eD$z>O|eO=5M)IowrD&fhpQ z@lv{5;h9rwJy19?-v)iNj(!1@U)BQ9+ZFkCg{n1hAGSrZxVsMs7dKENG^IO090~)n z$sr3s2v_{Sbpy@ZR;&4h^lL_mjoYt`kAh;P9>$UTv^*e=SC7u5dp+7Rb$hv94+j4? zT!a>T-YR?9i$6&B)OoLPWCS-^ySND!S!_#Z9-AChy)bMYN^Q%%c_rotp!T$&llByo zwr@>X{#Z6jT1}KDz4K-lE^GN@4qOG8jtsjun2`1RZb=ADJ+*d?$?#}+oI-R3P@<+l z=m`9w{A>yWvo(NZdf&%+R@$o$)MALk^#II-$)jnM5(RknPCY2($#7yt?(NqMc&?%%ea{S%;y*&sRpAt)Z{N^#T)NrTt%d zhRC95nD#6+6(Un2mwNr+JBsaGI-}r7#xzUpXPD4evegZgxFf#39MHZ_<*Fx z1KR{sV^R(Hv|F{FhAVy>;LlcA0?Q-=d}FqXaw}5GU~;NQSNbk-%{Q{&Qob%;=_D3f zJPzJhaTUaJ-gAsMwh(;&<~GdjyB>CsW6%K5e=hCHVugB5;7&cf1OQ5aZjZxMi0@y~ zHvz=EUY%Bvu{>@Y91q?EufVk;017%{r5mJQ0+qzE9?&QZ_*7=b;beLwqJ`Q6z)YYa zCz6xxmQ5{>y)E<*8iz%B>deHxPXfK6x&Y1dCBxrko3WleDZ8HH4ymFBX%MXaC>^C} z7b#wQ(xxoT72JX6?@kj=5;lide7Y^V0BnJA!1rUhxZnMReD-eAFm&}aae$4I78beH zpDP_?}2Z17RfzZrV zKlrKVoZk9;d!mo%z8G1*wkw#arDR(5#D5Bv$Pt$ysK3`J5vIiHS&l2J#y{=A^ceLu})6+WDD1j#)=kZuI5hCqU+kjOlWu1T(nvz3m3B+U(0Q(&c#9f^NERNyE8zaX@`FYV?Xza&@vD_2V zCm9`9iJ%(8VW0{|`CeZ$t<$HJO&o5Ww`D-8Z@$o{Viq(3@1M$95nua>d1qE?V;out>-H<~TFIF$le18>Sz2Q=sv$hjcecFG1H zt~vt5oki2vF9zF05*~2V)FG-l9M715k1N9-Ko)Ogch7bx3P+!cPv0Gj@ZXFyh6JRt}z9wZeRU ze{FDO`i@rjrQOQCSggvBV2==BcuD1to|%8FT5b*(ylKKGavei%N=YH95mJB@JUpICiZ}?g-qahFiuo#v zlU4xdm2vQzpbR4hqwFe-Jlo}l*T@i z?hpY4?B8N*iI5?%qSltA0%;HT=h6==PzQ*$p~5hyF^n*4AAPW7FEF+RXBbQ2{ey?! zpE}=Ld^#>apkfm0GDocV8a%xnGQ9gdk5~VUo`N(h|yoduXT*VEEGJ-!l(8dAwwtG-}xkQuKU7l{&=}jbsKNJauHZ*skU5kW0{`1TETI-~pPe ztL!@1_;!ugid)7vWv*1d(^W~9gwAJba6HrD_Js*QJ=n)aLG7lZ_UbFywG<%pKhuL>gP@us$&pNhAcjx-dlx(1>1#wYehsQ3^9Fmv#VV2yWnc3dHsV4eoMoj} zv1(A>vgo7@Hb1H(t;73tYKlNbd#mKq>Q*EdLh*}LN>g&Hl*wm z?jE_x)Z4^ zJu0}c-Gf`})lE>@v9Q{)Iwt2mEMj_kEC5yc;@>n>=b`j!R zlPrBSl|m~FASd-9cYzf?JA1CS?7C;6gyb)9?AGo)w1nt=`;J=2s7lJEJU+)B=NF$a zpnXdjhq4w^G3lPNOn4HJm~G;u^CD&s48ntdp_^@KQ&#cS$BR{b;Ss%goUJjtjqlC= z;eGL=`XR=I@)Zj-xjf0&c_hjwpFMt^wx+#w8D;D^?=%*BHZRXP zCf?G`c4SbmTco`)l|dXI65UTQLgyDxj*+a{{~-O~qJ>=o>-O51ij$gM>sS_rHg`*0 z6fb@0sZsay{R(JVbooh?)<|no`#t_OZlz1OG~4zSyDx^fwmAG2TZ8nw25OHkcLht? z2{95$L@!7;n4LPZEzJ_Nkf=+G=boHVWr$B7 zY=bG@!70M-zp==fmw9n+#V2PCYhXbRytH%LnkNDU+lba&O5d;9V-d@*4EtBb=6Qzg&x6SEm}dix=8?)v^MqF1n*|Bv+ep)!KpzN@)n zzBfz%E3NL~M%Dh;3qW?CQ(8>O>kArj$M@_Rzy1y{2l=wTzP=A|g{{7KtbgJPa|n2l@y2xTmW&K<&lW;xyXrg_^RN8|vR-e% zFnA&tK{`eOPHwL)*Ny&o^`0ZBcSHKi@j=`CVo(MaREARZN$n)4$yyX_KwjH?5YiU& z0cb7vsJ&*jU345s0)=vJof1EHmVz2Uugdz13I*~0G+f=rr#$tl6@V8 zdVK?&$t*~jzj^M3>2vcK-(5E6k5A0YB;jZ9NU3Zm`Xm$j5}gB%cC+Q_JRkS|*OzpO zLJGsRx9yX(>*M4XP261J+`jm7=%Fq#Mg;s1m=!6`TKP>6G=duObZVa2)eN|y%rNkZ z4%lWQPy}KtHemukT@}eC=ub)|lsU{?dTsi0!k&qy!m#h5@2u+m$!%y*Siet`V_%<% zUF>Y-D{$D3Ksr#Ft%FaV;LXpYH9)Qp93**mLt!#KqX_DV(BV`tEs&#dEO`t& z<;L17Rv&hHAjHcU{9=}zdHHqRjmrBU#F~GA>LcpQbbs)X&ol7LF%9@>B&T``1f`>a zTchC9hyW4BF6HrHc4$Go*f0q;d??_ec$b48f^CIPU$I%}`I99YAx0_7p^TrBUPk!h z`To5UCsBu@a;)DyG_Zl90QHnW8l>0_5$vuQm=BTeKsPcrC-?D37=1nYh7WqGe3vl2 zJr!Ov->-ssz>~8#85DjQLN3ElL_+#1BT}RuK^4VIjLk%mMTFAGVFN(gQ%tq^2^uik z8aYT=jcF(YB->cRpm)e%!5KJ()PNIsS8L2RA^-)#idlv+RQlBsceADd zL8e07E3C~yO(e&E!Y!h95cXCM!g>T{KNXCl?-|?X+c4@#xA5crV*oo{x7&sF zI;n3W9YcYUyJ>bYAK|5u#85&oR0jX{oh&oND4o=*=f}vgQ?^TT7Gj)SWKCt1*(26Rujt`DXV8;;Z%s5oeMMqc69{i|3Bv=8P0!zO zTF$1TP_FvAFkJY&&n(~Uy_6poAu$$HyLLh%jkjEPd$lRVX0?17`(7xMu?d$voEZ4b zc^OEcM(cNW>IWo=c#FbUfFaA`06Ei9*Lp)h4)PMHlwj3+GG%$rr%}lc?q1x=++YhP zCA%*WBjJV_lWi=|{TC|fG*80TsIL+tZ?U!33wQ`FD#L=LL4`hP+#`Skprl#-OulF@(S7donT2>w>`h|dei>v8XZXDyE;;JRhN+3<~ zN%7ng+90gjybxD?F>xDR{&MXIY05OaYdP;Hf!1%g&V(iR>YY#~3$9M_|0qC4se}_x zkj4ihs3WG?p#R9))&6hk?y>iR?Z#Sl4)WY{KKx`7^6$jxGnWOz1hR6oEt;xV?pydR z{fox#^n_%0WXq+IbyY`^WMi>l7&e;0w`M^tdO?6DL`4w#Y@ma9qX7V$$}jk=^;#=aD1C;SJikEm-w{qM65%XFJTr{3H9w){8L8IE zP8N4`;+i|a#ErJt1ROzvF;7))rc0>tXJGjINqx&ah~X#+>DYxViJqFy{>`Zn&-9Uy zT61Ro4`9Ku^+(?8~Eh@opB4i|rGN(#|sd-6p7mP~cO3m5paG};> zJYs^=g<3A|)HX_(gL?`0PW?R;g?z(Y*7s?JHCn#hVczabHZHPJKq<<+@z^{#+ zXH-d>$zaNlQq>Ucrv4q+>~>ig7CN62kCsz2_nXx$fpB-_nAN8ES>YQqTpBSaGvs28 z#ct%6zFuK_*g#sX&`K@^r&~+EX$ct|Ck|BidC>e;WLo#JWtvbJnV2n9 zm=xf*mze=M5-E|ME7plpKf}(OC@dfV213e>h)YPoLSZ&L^sYx_Pn@kR>7z3J8fhvr z0U3cMbsY}V1@FxD`KAwhwjoHI$dSa!j`PmCm=e@!5yJm*w~#~n3L%ulykNt3>=dN~W%09qUBY>cjiTC7kvH3YCJFJNd7P zj9iK>3m&!E@e=L7t_x`#vpPJ~byE@Y{~`McfaH%Yt61@02QrdqQo(o~aLDKQAF`hn zq^cay{Oxyg_g_!zB!%dR((Zd^|M?uv!jKrQunvXp9W?mE>?S(M{%miy%l9l(9~qS5R?vE3{*V4a2T9JA1QJk9BD6kDbFknJQgYBfIUG9VYVs2{hd`NEnu_(H02LT*xd#Def8n94 z%Y#^ku80FWy|`o_A{`W(VCDS!4(grt_% zlPz6Rd_=u==ILv(=L|RYT;_HHav6Z&GkTEBtclSu42fTal{Wc@clzAMU%N$wpNovA z{wRErg*h|FdB8eHyFtY31c$pVMgiykRno*7PkJ%k6`Ee_jvxW1T@X}+d)Sex29gJ+ zfgw+B@qMt_w}FDl?XhQ@P;9P8>Rb-#9?fVaX0n8!Mch-|?nSw?tHZGvDZT)Zh2$6) zTIo5oZb6B$0pu*Ku`bhNt9QoAn%%_PKo7{J1i~4ov^x+csY9FLw@)s%K!yHDHIzJx z7pvF0%%4zG*fz4udfKN!lJ)p~=x4BeXGKz3jE2#dPv?XOXP66f(7a)aFM(!^cf2u^ z5_AqHw7Lp8Z#WbS4+RZ+?U~@&D%m3uM8mHP==3id2*q8o)-Rursdun)U@?Eez7meF z2r&0Q%!WRxxc*Lvo3DKp>SXMQpepU$M9t3~H>8+c59QmpQ+92>X+ewatf-QW4n<#t zo;(0AT@T3&H#4*!59p8eM0~j;z#4(K=+kuAXd|TF`@Vr(DA-`KmY6n1gCS6Nn>q3P zO;e$C_$qJQYh}qoka`ytMl$%Vh$+Ze8X$J#P!B6_#ejh4nLD(9jsO?2>PH;1JV;LL zKARI60U1rroc~r*kgG4`dzT-##j#jkHYf)b&ZG56n@RB1x~_q7{W{+-5jaLth?xF2 zZ4c?Y*k=B|z_a&(Ch?Yy4v!{^P$R@{kzHSse%ox)eF^97Zsc&%)a~{|0;FXBrd6e+ z;dIEsXOJPs``($lHXZiyErdX2O;}>6P2tjek(##9E&HS*&}fB=TUMsramXFk+^m9d zA$OHxUg_#q7$}Si47MKWKV0f+(Uo|4$Uo~>6=YtWD*>{G zgXz$Yb3E^kD+Q7W*GkrVKmnLiINe<>4Wbm=*>0#7aZ(dOjA{wp!6_ft4poo#!R|E!IA!ZD+f@e z(r-^l-zdvE6koAO`K`#pm5KM)03gX~N?gJrSyKzW~;bJIt&qdm&< zcMzNjBX`+}M`0pCohC}WrxahW!3{b=ccWz-F^PO7qKYzF)OT>QgYcIU$r(A6#XkXZ z;m$!MpNDlXzh-x=_BDs^7FaGFS^V})S0_k&VDNkZMAJ5JDxjX}JPs0v=ym~b{p@8c z98AM{=z2hZXApj$Xj}i`vShHtDo_VB6FIbk(2)j!+6T5Yv4`9y7JG> z?tzm9`YpQxRtM1rT#H%(jhvtSbv7na9s1>2L{Kd(guPq}!3J7VL(0>^`L<32TA za`TP)k*UzXN;$?)BF)1A(Mck#J3{5RIa1Y0Vtvf@=xfS^0pcL!GIKfq8 zu5bW%bKj?^oG}D3nAzQ1>u%oXbdH;@nrL5mC`owP%OhQe1cGHrwA1;K^(yj*(H<+m zh(5HgD<}$*_BFsJ*>3cL9igO1Zy3ASive2?7N+}z_fazTj8&FCgMuOl$_;YG>_2|K zumR%EIPssNPfAZO@c$qWXGwKqc%z=txr3iUPBqTjTNlfrclp^hB7xKKVBHx9I@ z=uB>GdwFB-wOqP}eV@{EXWtu_baRdn#|Zk-+sA+QmvdM;vpRf(Z3dx3#OhB;)?(F_ z2rSl-ehDU|#=a?5jFwbK8-Liv)HxBbm#ooL&=W%sWSAC$1zk>cU)+<>Des+-M*YPntu z@18M3&R+-RMMgm8Dnw`r6?ncsu}-V+(LW(6$@8_&%zBDlDFAm!VcCKoDRON;X}BJA z>Q{`e$f`9F74-Y=z=PM9h$l)Jx))GA^0O?Y{)o6zCH>PB#|Nof2+$RKh6@N$wZftc zdAW!97v>-f`WdI`^T|LBC?BEa9oIwRN-|PN8IPoR&6Olasm1XHfh;yWg_cmAyy_e7 z;M6pnzVXot;+^{~h{D;MXv5PzlI)R=?@&qJAwO|=G6gD%O|3p2I{LKJ)~Yu zeXcb>{r#ihB*Rq%0C`h|$~&d7a?Hd&@%Y4An_^%G=0bEE>{V2c254eSLm+GMhJdQd}~cg5rW+$Z8D~ z;f+O;UY`mMiB=2CPFb;$DNajx7JMa4NI;dajZIYlbTlg(3+dh6EI^-hK^U=(iij4# zS&R)P40l#!>d}LI_ifYNh1!0BA_vXPj0pj3RSc>6@%u>`>=Yu=gNES=)KvTnvv(zJ z6@Z#c-XsCn>ozeVUldd)jJ8T*bx7m&zjJS`lIyKkWTZ;cg74EC@-X_kE{^dSGm}V0 zhlE9&(h5XG`>d(!u>4m{vLqF3X`=Zi0fA#TLD8ggYud~*ym+~ePIXmR;gtSO?eDE= zPVAG%_m6LmAE5B7bIFoqRb!q@N>32DzyAaQ4JHe)tXBhVE&9QU)Hdn+NdPN8xnj0@ zas1vpw}Z@L+vBDUbgHei=>1ZM=;Xr6C^cxQ7*I9JU7If_6pvwgfm_3{w5ho#sX?u; zVXZZ}!o?PKd|X(_er76A=#P~Y3b6kSh}Lc%#6FClzoMsh=^&axSNkA+N@do|xFIPW zlnn}q3Y_YvN8au%?7RQy-Qp!r?(X==waf7@l2|leeDoYm_Dts#_Fkp%sPHNAYVZmc z`_ULzh+w6m)JLw_Fiw9MQ(D`fH(Yx5NJnV z5+PO(tC+kH>`2y3;7Pu`kkn0J!+G69)F*ZoTO}|jn%8P;&7Wci5J_A!w%$~dWTnoI zw2-zMU7i_ZQhJLi#=hXZ)FfRu#nWvZses#^OYmUNy4kKy1+he#9-L#cO9&d#U60Zv z9T&i@S?iDaeF`-oa3iB4FQ9G~p|S^>ekhlP*4)G2<1!VZAclvg>OZS2-ne1$+ubrc z{q@p5)cl(9(elmwc9|gtlU*(D0&aEZX2O@!4Kw|*=xy3SLv4t{C!Y#|iYfGm$614R zQT$+Wj#D+nV1v2_`x{=%e~Gx8B487@ApYRWXMdM?>=~?Cxt_4<=c$YpZT->#J-v{} z%F(uGOT5%?9bajB&e4zai5ET7ITpT}sz=ficO;Y5^yc6z_PHzHDni{XmAtPmnh^fR|3-wd0ITc!nHrd96KGzxCUGsj+ zR!inayme(foMp9%$W2e>v{GmQVZDary%qEPC-zK1nL&XvZ9Qt{QM@&6TNIf}IQIm3 z0nz;h!GS9=JVrvI)79X_y(9ixVvK*Gh7ekZe@Ojx%N5xi}lPf}-MqEZI|KbiPQ zit6SGKf37WO@~vo!qQQVQ4DtRse~2ePBK!wHa0pBF`|sSbTlD+tr-1YeW%T?!y_d< zX-QWX=RTYg?B?t~v3o0&)@{Fvs&Jyfu$EO@S5SLicN$ic{$#zZ*3GB{E$I&G#Ugox zinn7Mlr*oTBk8OoP7^jm#oN*h+&2uvcaN;+c47jgQ{LCghmvO#dWiZWNkUU5H|vfz zZol|&@o{+r7klSU9Y)oOx+Y@_)vesA&C|LRA+^TNr-1k9GyZPbt0a36B{3n>!l4a{ z^0py5t%TO4(8kK-$cQ66V%+5z$J3pW{jhkZx5Y@Qu| zfRcH&Z5P!WbZ*2yDzjK2f+S`Ch>?DBw$XAMrN?@hB4cvtor*?!-D72LC3#uy^=O;- zXN+T6-wl32+Dqlq<26<97d~vrxYg)gF85Zsssp=!zn7d2uj_>NrX7dCku_bhh-mr?Ym8l&i<+278$N!u$fLcy&;$*2G`cW|Vj-Hp|YqbJk5~ zDf#swwGdpa4CgWjUo}3|5ubKtm0p1N90{3Zn3MsFV8%=)ZT&Odr!-V(Q|+}={ma+J zZlqO{=A@*UlQ*jq)gd+(O1Y&CYr2610;7)ACfAj>UwwbQa-xJr8x>?7i_#pEYi8@? z3lFgtc<9X6_AYV{z*%~P=nI9;+fml>IZ1h+9|lGSBMHQATZb%u^adv#aH*L(G+uqc zF@8gvq9sn5VlIp>g>04k-EnR0cA?^~rW65-CNy#OP6WMAv4@LhahUAFK$8&759p^G z^HEZ~{4OR3bu;-8ts1-j?!NM(%%)Fg3KTy@y?K!Cs8w9){qkne;>EWo)fIg^Swm6X zDkB*6R&on^O;2-uB`Jq$jOf5hp?$`~>-}Ul1l7;^CBky7FJ+&z8Zr@C;r^148vKKL z^U|jl!?&_)pJY$UQd|@r-_%g8C?TCRB-Fp&ou;(YKC8+h@xbR2co+z%`LoV#D3ez2+@a* zAcdfNs^36#AoQ}~K1=GwVSkTvEu0wNbHjI&b(g~^WVGzo2mDd@W@u!*GUaVoAwJMm z(q7bBZMm9N)%R?%>)4(X*olLP`Xbi)8ix29;mrF0tW)E$Wa8j7d)t9407YYq+hRjo z&YDvYW>wuRIUTnbl5|=mVH?)(85Umnz9;Pc>yfc~*_RLEd%gb$aV9}UR1k2!8{!3T z{|5mCF$JpBe-`(9JofAQ**6d?Asv>!3;!ZU02Dc=1^}kzL({$S1%jXU=pabseaiXL zKOfQ?u`LpRSMVpd$LT%-Quc;>oBat=Rss^K&vLPEZ(fI7z4SB@=e&F&;1dsC>6x7S zeWVXW)Iw?hoo(}+3bH%C3s-qbMT$XhM;;7=aQWa@FqNoY`zCL*OZus>L(b&4_s9;X zTmFeRZ&i~$0WfG636B<#RpV9k*6lM(~a^!p9J4D^2LX6ORRCru8ejm19x9I^>5DN7=C~Gq# z%!*}zYf2*yd(ixkXVAz#gQ0-ZSd}}NghT-5B@Q)S&yc!OyFYKJE8HWJ#QGHIHa39o zqd;wP>N{`=>VZ8o*xTQ<@~=uCJiZVLe|Fq}paMERXaoa-n#sfS<$rL7;Bnd1Rn206 zC!1He*Wcu?wee68f7TA02A)xq?*OOzXvSAm{>NuhA&OFYEX4@D-w#}ZW5Nc0ks9$f zx}CLW6aUX{b5S+>J%%Dpx;bPYzeX5PeBTq{kN6zJze+EUxkRG#s29?_q9v(qs z7}n#Fr?GVP-I`liV)sifjhdA}%+9Asn}xA+wpm~DX7|a7DfrIzIF9sOs5q|Pa-YTZ zYw-Xcu7gTkaUbQcn@Gl~JeFi!0Hc3={BLxfo<j2yDF8~e6*jt&QSB@qxTaMnIsMRt7w;p zj~VBBHLAFhKbz_sNAxAb4sifE;$YDp90q)I983-ye_P&a>`puB`DZ*~v$$&d>ZznV&&oat?n_VwCF zwFk#PdaUynxC@Jc%k$})+7HwWOMJi ziI`cbnv=~A|Gk`5?(j)|uL}{bOY0=>4v`waryEFP&aH$ESYBD~RBVr><_W2aq4W$kLH^w}9kCLtYCyw%~^4_z)%zMnHljU5CIcJ}V_=(c4===ekrq$!4 zk9uDG^f8dXZCcJU0K4+Kzjq)|;QheapXG#0Xs||(M-Pr20_$Qxtu@_j`ee`tc zHyONx%u_Q~LUmfR!4Nslzc=b0Cz6P(rux;C(Ml5RPR8H2y*Xq9E`8q>77trqqf?-w z$m-n5iey$mQREk6r9B(JZ!|nqyHonn`_q6%zH;v9cG0^hxSdqL{(PUBt7Y00ZS*ZW zON+Coei&D!D(~<2c~$*O4r3;boJnhQax(wiIe}!CgQI>nDwj`yG2YLQTU!!HX5HK$(f%fQB%*j+NY zxEsRDW9+aaSQ(L-TuNyv$1+T%!92&}e0SjaN>%=>PL+0L#G(Xa65GlCXVxhghTQ@8 zB}t!wYteINe=&L0di2K(|MLxz&#BLfKbaiACV;*BZsXSdG3{VFAN{G_(_Y^BD}mWx z*zSF_PY&}j{ED(8r+SoQ#9M1K$CdNLt1LqD{Dzaj_p2`zVNU$js& zt`Spn&$2(HUk!c1*y-6@&+9kK=g%YG-z6T&5)jU;culo+-GNpxl^XnpmHQ`b0hfc$ zB7c-3GsXCW5Z#~WxJaEPBn`^XcdM1CTq2$Qba;U3X4Zr3qLzN)xV2*_LkW5!m$N(> zH$OATJ$bH%S;_jzd|;=;!kPJ{)uQdpb7LybRI73esb731Cu!vEdqCm5+$r5%%Ib9T zpQxtv0ud!AroLh5XLzfZlTAEZsJLNz%%l5{!<7!qv)Acvzgc*5sBTe zxS9jjE!z%0uXtng>9hJwY~MpYrVgZP4~ps&4g+(k1L(hN_;@m*o@ZluHQ%Ks6^NH< zFqTePc3Ux>84u<^r!BtK=nm0TJ#Y_>&akL&!?qT#VP@|{%}NtmvOBo$kbf}xwKa{o z5m8SVEc^q_J(-70{ zyUPpjg~RUCQrJ{(3DTy|7FnvnSr$H?I$|*|NrFhId$jaFtC^pKcXvh-iswweuYN@U z$ZE;vMQ1O&ZD-LbT@?tUU8EfEa8$?0E-Z0rxenwk-weH#DvF~kV6W%6`ty}>w;Oir z%gijN#>l1?W^?SPP7U>VVIZ!GFp9EG+qzs;aoijNc&GQsUFN2}J^#5&6spro3)M=_ zx_ung5XiRiaz2Ikgj7fQQtWcs+jdI=qe#F4Vjf=sa=lzn>8Hs@uGH zKyF^L#W{p#5xq7RTfF8e4vUQ-3`%s)3$AxaF^ zAmS#0A3i~njJ~tOo4;Mc=!6QtYSTk8;P#aA!5z%r*vkDL$VBib$k4k#MFMRnN@RS) z&-ldsJ-#YL&-wH_wXEYp%y>(S;Jgk&^3A&IGd+JF*Fe3=4AuyVaJ+aO@|+GnNIK6y zNzMI@=;35f-nyhf1eRBaZ0g7NJalTTVW`2 z39uVpKf?I#g?h)I35!5a02<@p6VTtB_}g^;Ps(sR9X^SWrfcMH@AB7w@t<^I4XjzR zYyJ~){r9stdBPCg=BVKOZ3O>&K?xZ8*S;2Z4&-=@pFW zq0#3a_J2Ny5W)v)OB~Ai^MacQ81R2D_Mch)e_!msGY!EKVCRs)$5G3cB2rg`ppLKt ziw7a83`Z(wm*?K)^HyEB>2ei_KjUDvfCiSy<+MvBZMUZ}ch<54;q+fdRQOPLmhU|D z_QY$yhjIUMY=ICTA&bq)Hz4^oC{e_Rfr2m&?D~Q}i%1&Idhkf*|TSXUjLj*M?!l%GMJsFXb)Ptpig-d3vcZf~Isg^j-N#XU--j_hR zZg~qNfXcy=9iSkvA}rSL0hd78fRwUOhI)t3lmi}q=Svz?6aKY+z*5HFc?OXSA##R8 zjMf4wpR+(MSuc=$g8r5bsvK-^8*CyunOBcQApm&+*ny4%cH;adK%#oNei>M^kw7uH z-xfncPzAL^{u*FVJQb)>SW6-mu8p-fsviLURD2P*r-Ts_lZDAImfI1iYX@BLc) z1jzn`uq%FX01j;ciE9}+|myEWdOxXd0g?fasa?`{%s@yR1wQum_Ed<_Lwz77FqTCWK zcfhhLkC-he4^WQ_N_0$scJlax-Z@ZH-N4ZP1P9r0V4LutkZNMIfl^sqCgBZ%cey)Y zG9i4V3|J9)4v*t-#=i_yfIt>6KZP+&qA&tc58*mAq*M%vs`TRK9hL`&Dq|C6e6_w|S~Ei)tyTodW&RH4BWgg)lhsNP?if z#exJ#f{rh~<3@qDxzifsegmenTlEjCz3Ukgf@biDu#Z4wBZz1B1U4Sh_#&VprE;zi zl5@~8{RmL?N$v(d9*$Lz>%MGLv114DehS_wDzZ}2fN68^>M+8f1Rs-2ZyqgSGq7*= zU*KzFDFimbqaTA38B(r-<410dBiyl%oBMwNSFp*0$*%qC zcZ8fY=9~hX!T~l5$=Z|;<26%Bx_6}_D5lhFLrEsJr10}VPUK1|CeS#gv}upX3hQA( zvOlhMjXm6VZ4_zRnl@ZyVs7m#$4Y&aqH=XaSSUe>U#+n5$==# zKcW2kLL9?g#Av-`6(?zJy z9%03JQyVb%+yN06boXm>N#p=?5=hu;Orv& zbu{t|bgT_Gii=%ic7PsvqUk!9l#f2MHq~|JuYagW4^iO#*(uz=Xbv7rJf`30VSx19 zw8Ic;5HLDg^d}<@0XF&fZ&EB|@>dJu*GfYXBYBPcN%ISLCr7%R*Yp$o8VFWF_<1(` zU?XMFP6cT5?-uh_yvsrw0WM|?m0ovN5Z};leJqJEJj$4|adt7a0|{y(rz>nVhgbK_ z3G*J~3+G?fX+@NC2<$I1iS5Q&@`UNoolaz-;>Vq-Wy>yw1#Exb)u{)T4Jy5XW3iVs8Y#9S{naVx9_@16RT85mDf@PF6353RjW*AfO$= zpr9@p-3c19L{U5bdjy*xHZ@#HYWZqW4H&s#2TuL>C>c3F*dh_d0~O`@jwpv5RrTJKH&Ru}A@c|9JdR?`Pq)F9g@I|K z)L9Qh;^vAd1jIs#c8uyQxg|n9>%|q-xbOmv%lsI2NvbK*Chp_hyb|hG%a2<9h267; z39&=1hB3T4*%IbaRTem7 zcRBWj($;LQ$f?kPp?r&PM;oZ58x7U4f5;e|%!t-cI;3%mYj1o<+-^KJwqQQeApJ9A z+#F@VErQYA=e{o5h7hz1^eZd-1>rcdd~W{L^2ER!O%F$^DPkT2-f`N6;oR9w*yLA2 zb(Px426r4BYtpxYbucr#gw&+7X;tpsA#ROmIS7zE7k9oQ<2y{CWe zE62OMr|18Du4V@!-cme|6{F_0=vo(p&gysm`1+Ya85%oy#yD%*G%8*N2~r#LlW$k2 zS`*LAU_WHYc|Vft4r*JfelQiW={dLt3Wo!k6H49R-KU}P@&jV8y60IjI{|0Gyu}^F zpRr|CSuUc28ja=LL30`Oc)`ynAU`8OFiB9T z%^&Ul#3;i^iL>pc3Q=QY@Oo_(i3A-L8Hiu}Ma(9pSb0$l27e!=|4k1rpn*s$8Ss4> z<64{GF?dS8Zgc-wUR-ZHQJc7zUIbB!u9AI?@VJ%EK&sob0YCqbw!2&uejna{kY@Y!*@mP& zPyYYo?k&TjT>G|fnxVV92WeD57`mhc3|cy+1!RzhAta@{!$JuuDd`RgLFrEE?&p|w zJ=b%ubzj%l_shGsZGAvuX3ja!Igexi@B44R{?qt!PWy<=f3ig;KE+`+l()d=Tl(OW z3WWxez47iQr#R93PhUWpVHhf{5k=FmDR}9iCB)%ZQXco+$q!ie_YGhBT5@>s{>QA7 zK>ja#%rep_Di(hv-r67`>3RhgfBrvhQ^5M<<1SbrP)YKDJ+>N5u|?CFcpeS+uEY(} zr;1P3y~f^bAJ9}`AtT$ko*!xMO^t?xV}TM^3X2o%WgLlaT%4o$Q&}NJDMOLtSVQ0V z5K4phcYZH|-T^}^@_8v-!BSrZt!{wpqx)`iyLYGKPipERAeQS|p|H@H#V_aQtflPO z#DZIyGK@n2?E%dQ{kLu>X%_Z}=1iI|_POYU<6xZcA05b%J}*8|j-uAC{@H#vLp0BU z1Mff9ZNbDy>>gY541EW~ga`Q`;0-_E#A|oV6veSj*xi58T#2Xo+-U4~71tA~(m(QX z(jcpRGc>CoVED}43~nn+g09ekQeRI-QBP+0XJ_XNQ$6?(y`BsYQ&C%sgj_+E{pOIc zWNCU}(0hn)rzU$X^EsJ)(;G=1>*_hJw?+6M6cBoA#FTTteI@(;9_KtV?KQ5L^Zb&e zd0*93@;lBj9KWoGwY|^pzSA)cgj->8-vWq3xn#m`~je z)Z_)u$R@u61t~JfZtcyKQW(2p6X&{e!YWM!hSWYapIhxwz#8$2sU~KbA~cpQ*(G2X zpr!tNKiC7C4S3Xh(WwL!^)qorkMN3Iy%xhou)H{AB;KB?@q`jQoJ*E!=D|%wTCYXu zPTmWa5{o30ZA8kN2dGAe9FPf;=^wzn94K(st{y_10{V+1RecI z@{P#20;GWT&nqAiRCx-ArpssM=i1Ufk%yDR=p4!dPh;RNwK@<%H1 z`~8vN&Zoa7=PCNc#+^%6;8fusV^O?~$`C;u>{Sbj+P;J2PyX{4-HG4k=c0KGRG^;R zF*OMW8~JC<9mOdaeQIZm=(uPiLI3qsyk-`B#w!T`b)|DQ0ntaB_UJH?-K~kzoH?Vi z^=y2f!8id}S@L(~SjOW~(#;3Dz_WyNF{B=-sCn1$3ulgUigNmbS&S47O|MCzBnJVJHQtA|%aTO## zMuHup$mRa)&n3vps1c>BKl_yX|E|2hzjYNPZ}XYy*h2nxVgApze^?@qX3x_q+&lm6 zi89lna^qGAJ%;?(SE+#ozWe{X2M&zj#sz}RHWnfMx>7W4QkK1B2(`3d!b+JZyEvWm z-zw33N?V_AH9&R);!k^U!DK~$BwHpy!8$d^@1o}#WRKDo$a5FivJ+6YY9YlJQQWW$G1o=%p@V5tV+EgCoai*$6tI1-@CNwS8K5K> zIKmFho2iL)nOyfHk1{6HrGFb`c|NDt&SsGhZI^s;bQ&@C>^b@Pg;CKe<(|=U)>A60 zr;O1|$d~@5@x-8}kH5Ya$*H+7w*p?Jo2KOP)98_h5&g^rvB)Z|$8TSMXIj|@i{w{G zRD1NV6&7y?;xq#VNH_aeE-YVjETm!)@UJ|8vB;$2f;FXX03-DP!}+Zl$PeJw{M0c#_q@0J4fs}n zID!-uA%}+}iA6KeJTiFBS+9Y6H~T}s!VPVwPW zooVzp{9~jU=_zmhUe?nLx1tE45(F&_jY6B;6cdnRyJmpHW9$AmTe(kaNKOs10{Ii2 zo-7J+5uUx&vLGdXcMb? zG$|(XwG%&jjXu<2OkMCBj{hPXzXb}*HZph!jv4Twu_Cq3*MZl_YIfkYaD^_UodN^K zNEQG|9QS5e%@SV>fPLfcBOpnHDWgW|Vs_l4o@ zPWOr5hv&}&ZR|L`2VmIRzao3!-c`Ij{;6A!yaRs*Dx(w7$@sj9b9)rNb_T@iP@7?K zv;*N|2asqjYJg}1mz^^z_PaT^VcG#X99Mu)j^3JWtgjg~_lPb4A^JX&SgwT%D4?PnKwNLGAN#L!J}GGZZ9v4LqwsAhcE9V6QaFja+_$M1U<0k;M|CDXE@3wpx3rcBo&z{SN=>4zK9}) zn-AmSbcAXs0rZilfYzwf8G~y(3E?i@I#_0(ZY;jEIs#-NxcVP)Sbn#h*gO2IvUNHv zQ5=% z+a)>+FA!kc7K{Xi?joW9rLA*b_2Or*a={KD+Yk*=r4COxJ|OcYk^+?E`>||ZN*jUL zjC=g$)0%X6aAV671a?DcbHM9D$MpE&$rP{i0K3X;*U|g^8x~PvWSr^&(POT^u)ELN z7Ol7OZq~PSq=(-t_Np7rii&`3Uuu`$@F}{THWQL{Ti_o+VnO>$s5tn|{v$nTiQ3+~ z*vF6BVyjJ;gcj^3md%cXU!p{KpPpm4Fl_|WZM4xb;x3$JjPs9f&Sl3v6FVtvJOmtM zg0K#L*&1+0BdfRijzZ}+1_oYba-kKT)V{)rW)U&cCy!Lqc@(`Ss))Pga=ml{U_)toeBh*a z$_t@4>wSbxV4ct~Hh$is&pC~{SBqb04*bQx;fp1ibULXW` za4S`>Kz?gzDbO5Cj=7N0#q1ud??@;-P!JMFiN-AUm`eQ~i?J!h3f%9Tax%{{5^hs} zu>KhUx4b6CHk(X=vSu{DR+_q2Z6;))29*x+Q>0EL{a_XhFMK6r-lGa2+&Aw>sa=N( zj0s9}<`rV@nufj%#gSBi841eg^UF{8aI!bdzZt@a>}kT8Z(r+Bqf<^ba;*B;oj9K4}Z-P>|5=8>VUre{Ieh%S_LKP~8lzEB*tJXTg^s1^)@It_QAbWJ=hC zihcoSfwATO7ko(O-oq^Vm)$Ik%ANRR{CaeQ%cvYI#;8Cu5*8tiRfo6cY^%v;$We#* zchX_dT%_|Uo3$#W4G5EMyIDRo6kbu8)YEb=u{@_jLzdZZ9PW-3MKVcmP>m~Vz3>pg zA)HstDSsVGH0@6Fqy;J6oh&VYO00PyIT;PwFf!T9tAPCgDQwDO1%<$tYbXWZ-3X#i zjteZ`vgqCGUiO1fFR;^j&3md}9u{L<9DYwTI+i*>f4*Lo?HsGK&CK4>U% z=_0-G)inn)Mr1*27q%p|E$y1t2-D0e=zM&yfSRCJ;$LllGFhx`Mjn014ANn+U8nI~IFgd&` z9yQb-xc(J-{B0d|y;`EN(!cVzRiS5qw@h2Rz|xZ|pX- zL&nZ;mE|Lg#S9BcLSaH_O&vaXX8vG!Ig|-5`Bq<==Co6sE~wQ2-iw3Th}ERi=WVG; zBDX^?BF&og?N(YM&duZTpSWL@dp@2v;$AjflVcV@<2%Gs$L#{~<@R=J^IWsmb~Q%X zd#dV!xq#K&j@JTm#PL)psi|c;*aNg=UKoVp%wz&s&pAzqTM2^Dz4EmJ9`<2Ka%zLp z*SH$yN%mjZ5m#yV)8CkY(uFG`vv89{>R%h~iNn1ZCL-Dqp+j!bavcafgpKidVb=&c z+nF3h`A0cbmgHA9ol~cox}*@+GG^Y>g}4hrd(4Enw+1y%+Y}PE-$>|FuY|p<D+UI%V#%V(r7}x%h9hUA-@`bQRTx|c4x%W6akho~c$YAjllk5`yn=ja zZ+jfV8YBkU8`*9Ne|dm>T% zM1mz=$!hw8ZBN#R1$k^bM0^I46lU;gI-VlYVjLs6ZTijAk=6z1-+Eg+E zESd?EIj#YZxYM>fpGqD-ifxK0x&r;bN7wj)87G|*sC`8wXc)bB$uSzbn0uoz1q%!7 zVwx^n#WL^S2oaLTz?fZ>-{-|pYgSBhxGru=JN+>;f?f=^p}ir+klT}SrBb%r<)!-y z7k%zFa6e`VEO@slUP2m?KK#v0=1EGPpC*UZID$ME+3oSEW1$~P#aM~_RNBQfy-kf} zFT>tH@!rjCUw9B9v$cDqF)5ySJm@$Iuxk=DW@NonryuWr-=o3suOygc-n}!scEwEA zEY~9WMj+AEt~Sni`phXC18z4z$P`1+{-yxtm;k39OC^(yVuH%S!=?^odr291eCMO2 zXAS$OQcWzHO{JpE`JJCzvYYs2-WPb^O* zE1aq4;4D$C)1`K!JNprd5xZh z7EN=in;YPTiNS^iOT=JdDQV&H>fpwKM3SP1;E>T6N7S|!2&RLPrti$*fd~ic zf|r@_8}{A&2sF+DsNH;$`+IMU9roF70rtrlZk$@pZ{Qj=3!DFW(jw^TaUdA6XmJ|J zcsS;txyZxML{6gw&wneT&^Q{?+yz;{;dqp|?v}`)9-WGD$fF8rIP#tp-gXA_&AnN6 z*Ns#OzHAkxh6Zl*Y#4EGVuLDLYr#_)Su;yjR0NZ8^Q!^3O{R`FKc!F0P*#cx%X7Q+ zRE#0fQIU|#J(`-Q+#j+(NNf%uutqFK@P#O4RF+((Q#aVuZwmO4yiw^_k9BJ+xwPYi zpL~bf5J#lW)4JveO_F1RyUT(l9tX`5{xcGoPQva^TUlVYm3RSC*#UJ4%D7Krz#xxwXQSH&w%w41_S>3WiVnVZR@gh=g7y_WVrnU^ktNEQg$<-EiLJ57gD0xS z93k230x_HFbuvqc@tP_ga8n7Bij51tXyK6DH4=Y=J^ML|8ydQYWz`g4!yi8qH9M3} zHMZPHF~GB1&*^yMy!)<1%E9Yhk(F6RjLP=#;tGR5Z$pe+Zam~(sfxLJvWxCwNQ+uT zDMjweDs3Rq!d+EJ2b6@Ek>p;vWx3Q{r^og|cY_6-PEQKZr`n^B3eXtI@TV6?AuK4L z#VJKO75Dlq$pQsb^)5*63_m?I@48@k_ba?pt-cz^#vrxCW)Ce*@Wq0L*uASie$&I8 z)#@eZEN6!m9U*uVk_eQMyOCuYv%Gv?wrNh)H_V0}P|Sa)ttw7 z)yRk-U3qk;fV(8$wRW{5+_WoQqq1v^$IF5uimmN;w&H&EiA_WMelJGhh^3F>Irrn+ z&IAqS?a3HZ&5Q9riJWyp~5=x{7xNFpVZ%$e@R=D zNws+_`NUJjhq_6B$t6`2{>a2S$|he5u!E9MHE`N~dBq7EdgrN|&9Lnu&|;1?`+_gj zyrmJitc&f(Hfnt_I;Z+(34+z`@iE1zU#lqzQr{Y3O5>vMg(vpardX8a zf1i4pcdx0!h0V#x8}*Yh#}d^c&RMs7;sXQYEo_nOw~yR+WQ-Hvb?c>d5*?a!tf1@199%MAXlKJNS&cS2;MVYojU zSk4x9C=BjUzjf8@x)T*5&c3tI8?kj)x`a@V{``fUts@Jejx{9mTjMgdyNgPt$01Dl zZP~z!TI;(!C8Kr-krdgGH}_$}V$9*Dc|sG9B4hESv*?N-I|f?sv#MQQSuzuSPl)01 zoNOOkq>wy_Uns(yRd0`ePX#xnLEoAmf%J*#?FV6^lcVcx{OG;QUFJR@=OhQQ%ZIiG zy;1rJ6+fSs;_uRH|K)?Xi~04b0lmbc^&>;M@ORN=_fT5h&}UtqiAbeh2?FXCK}8;E ztB6QNWgDBoOAOh~Y$pCTXw)wmtm$q%#o!_}19kQ2vKJv_d{Zz2HQ0DXT|VsCiQ4{9Ea`CQU|Mig!&ZY|A$=iQ z_`m&_GA%kxHzM^P_mq+%(|Z>}0zmJ~J$KQh}8-m2S{jOX&- z9E9EFg~?&Tmo9LvqT471URzpdZif4HA;LsRU;85#A`{b2Vm%0)G0D}GmO;1h$3wre zTCdpE=SqfULJ^+o(}jZ0>1JX)nrBXJU$LEGClWm`QWVd@?H-= zanFrvw$7>7T!PnBSrXT7+^Mi}EY#4^*vh+_I_C&)c7YS|ORV#-#FHP*aZ=*os@Z?$AG3SI%yQ=ukrq zPoI@YM`c^6UwKU~w~6?DTw21Tm|WVr4lQCgySv*?dSV#(2wA#ihb(EjOIbamR1_go z3l6!dH=S{h`g}!bS}vIG;1a)Z&0=*dX@k`1g@1*JN+c5M#go9&n$t~5I`4}eRk#|#WO{=FAkgSGR@tJm4i zG?0y;f&IN^qEuq^QU7+d%^0SA&~Igw5^WUb9~r_eBaw`0;q%%7m^rV7C%)Kq$Wj$93SM&@XbE(s~Dy+(R*pIDYCq@inD zci!bpa}dYr;MTngo+nDtxInYYGlIg`KSi4mW*zC>-G>qx%@DsPSl2VOe?en!SKY|*~9m?*QvH;s|&Oi>RSa?G8+3IO zS{M&?G!Dq8`ncoGlWr5|sWEaTt$AK>&RL{b3)8cz>az9oDlKp;YS<@#1VKQaP8Fc^ zNwsp!h1`#>y#e}pUdl+n%mK>55FmQ z3E|IFPn0vutaNY@&K3*xG1um#S7RcsU>RqaT4lNET$bjw+cQrV=XA7<(zgy2xmTx* z!tbwMQT|xIcJ{8`uRhC{6;J5r1(xGbcuLBGs4{7eyv`N+LKNRnvFaH55!<^tl6|&) zEhV!Y>(K(~mz1YvBb?C(|cdY|%RV4}5?;rBcN2hob@2x%DhIEsia zzL1r0RIci*?KaD1#4B_A4(}v{bZ>K=Btm|1^kxxq^LrV`FX~rKz^ir)no`_E!YkW) zsDK;?L27OZvp~3@LnUO~bsVdYZ78mY*%Tp+j`un-f1BmgO~Jlh{oYU{!@z@tlY4R- z%2V>FU43uG3naTb+9X0m)Hw=$hPR#Od@9rz%pch5A>8CD!2s!B9jNiw~OZRA8Zb@}|TG?MB*%{fj zXS1mC;MWwQhb!^)+7$Nm^kGhZX4I2P-%hjfaOI?D!$+S76z6=08`Y^TY`0Dr7a2P1 z-=67j3lG(j$4FMDRC*fgg%(yO?AS!>hbpIHubnhKw2vGX)2~%%(_O4v_ZI$4pqs)3 z_Q{nXPi1r2sej@nai)-|B^-w&iB!(@=HnIJ&F%ExJRjdG=ig*<0IiEj4Je$Q3$)Q0p?7^DQIX4PzGKgF3ZhiD z?->lH8$HQxuqZl4rNdUEtiEHRL&+pEsDEV(ABl}=LvU*hawRtq=eP7tM1AEL3Vy>8 zS^T|A4AW2LX<>h%?CaqslQ*aGr2fN~^Uo^E&R+*nUEr)NT|cc@RSKS1PZwWFQAm1` zGEUj*d5@Z_Dw^FderLkHU-d<#%K0SrAe221Njq2KV6*Kz8%YscDSrAUSI*Ja{#N1FUNt1rzP@`3@0KeaD1T19zXV z3g8T<6c`fv%jCI*&uyg7LKRK?*JoXe;nfCUz)Kwl;ym(sb$-&RDFl>?yo!4E`pJP-B#~bpLheyuY}W0rbW};5x5{G+JZzAQ z|MJcJZ9K0>{C-%Mh*piI769x zA2-Ft7#F`hju$pPB0qJWwb_VgH8t?;`xHFALF9YCG{b4a{V%^`9HkYo3vp*OW7YYA5mP8Z)FsU#iLR4irdZf5VJYUXdUXkA=IS+uow zJ{eOL9$B@|oXA4g1sEKppR9O-O#0u>Re3iU^gE`JTOK>|SxK`{86kMcDL zdt7t9r!57SB`c zK#}5&n-XZ{4DbeM)0wv{^2pS}n(EyBFRy4?!^f*jqHU^Dz|_1%+ef^D|9pDuf@o%E z!jHQ3;~_qV)2yf6>4D0pB!4IYb262Sf6dResAE|7L!H==|9P*OF^WksHjT8kFlq=H z#Lsp?x9EhC6m1X^wW;&S9Jq~NbxgOy+Uj)1}R zf)obII-EeQT9MPY0xt-qRK7eTR#dCLd2bjQ5@Oc50eh1jyUGMe_27hk?E>#Ad zk%hLfdZ5%k$PPOKv*T|dBk+rW5%#Ofd2ZIRyn}M33$!hy)C1}v`W^8w;`?B) ziJY{1Z5^cwSxf^2<@pgl=HaoVB(D+9-|~$pDb_y9CIa%z$2%)i#4{018C-4~q5@Jy<5L0X_BS ze-?=B`QX(~Gn0-LJ)BKZ1L^lawKD4LLXClIuAvLHUk~d?xrOrXOgVUwI&bW$6op#E z2R++dN5h&=!SvJ}sU34{#3B>R+zI7 zS(ZrKK>(On|NIPwJU8jK;9Sos9R)s#nQM^MxS84j4pNX@{*rYnll$mH0(eHF17M)$ z$2|a`n#*m-&QB}KR*^?Dqp|thMfA1q2jJkG)fIqFH@8k2n_4jSn#E?B0Sw)lJfHB# ziLtN98`Cc6_*d z_GYKuTiUaxNu>K){D4VRvj_9`&oy8TH$$ zuDjlsTs6b-Uuz?CXy%(Pj|FP>N50nCpBnXQ%vJv+HzIYe6XBaK)3%t&a=u!(POVO= zw|&VJV0~w;8fE-#o1?aJjZ{ffx15ajx&!c*RAzv+$5M+2`lWxZ;@ZP`+ zcyc5J+F7(nm|7S+0)W*1WezeAu$-A0z#< zCHNgkt^C9bfqdTUki=l`jW)eEaBoyBSuOWr9t_f3=>J0&+iM|Kk{BwWilERn8csoX zhymS1r{H7ivI10D+kFA&o|M3d@|ooW<;2B3R(f2FM51L4aS$|#>*A)(*L3(3;Kl<=Y2C_^=digl_DmOd6ms*#ToDn?DlrG9lNV9WW6d!Y2~`QoK&wf}46sxDDJTU{661~g zu8Rse(yDjJ#Sz}%Lv99;iWl3i!5oxO5o4RyO6%RDtkyq`Zyg+AUmBm{7}d$d3EKW( z_P+#A?-=en4XSHqfKTa9lGCu5k@C6B1EnUjODylbpo$B%@x!;8o3!{Br@OmT7o=*u z3qtv~vPatFM8H`5P2TJXq<1f@lq$p0M-Y#%OZ;auc4ltcW;p8kcV2%MN|88jDVVQS?jW z&r1^Mj7iUS{nxlWSJ_PJ*+jGF7#e%fcCu^yUQQ$;PQ(^@(79%Ai!M(lzRyrJCcC)x zkBvLNY?&eq^q+_)_Iov?IMcRE@M@w_?1!@d@->Uol-iufVPcbv%aT-Itzuo}X>Wx0 z?8UJ3?ropR&WH_mZJX2XmX3*w^@LZb$cL54CGHL5rz*`@VXS{Q4F(p!k?Bh(&$z}Y zOJ>y@0%;H_Zq6&tZqskZws|29rv36IbHHqsL1j>@^iUa64?Y%+rm7P}24DRAm{enX z04oHIsPH~+G$LG_b?_O)hs`1|B1d96i?8QdPM#qg*j$oi0@6XgFboO|7)1AF12*v_ z%bBc;!o1yRarAep(=NC0ioEXTk;pI%Rq_*EIItZxx`Y!GAd%iq;0uA$yCybS9zI5h zQRH|ljD-(j4A~Ny#YrpxPTU*D);sV$`SWH0RadxKOUI`+gx&$F8gC!OPFuCDJvms9 zp0@)MtM`7u{E=j?0#1lFUXFV^4{8JtTLgQ_ni-u;+FfvK_BLZ0B0)Iv9?oM95;LMg zb|GEba^Z@oF2I-|G0E7A(E#a2KlO^yJNrDqvd-3>4{0HdrN}<<07U6TzQf8aaQ^&0 zl=agaL8d0=sPfOlCS12Ud>AFMc@k+VNyt^_8LE$M&)yP|YkEOHeAk;US)(M}rAiE? zM>b5Ac3*;W>Fcw=$Jm&tD`D@6U?O*91$;!t8c(;GoaT$R>c6Sc9m{U3Qr;}>o=-E> zY@j){ueQQ{>>%{qO; zKXU%QGH!kvD-%tQspstaOzEKlahv%N6@eDxJVC%bKut>|%POSG1}LV6 zl?k#wO^*-oh79sPgQ`NV^c*x)WPA?fOxc+ria}sW@Ck?rP^F11OoxhNv71 z={xrM2fcZ~zBpbp2MQ-z%M*(sB-pKy?cpMrvv3s|sEtN4lP;3@pyD9H_ zLa&?V&soYoFB|qMA(?BQ!1SK{hH=k+OLT_du(zO5mG{q^h5_8Kmy+m8w-PPNvT*_N zD7|-bxbOPRBfgKSGE$zXC?wN$11b!3hVp;*JThc*xk?deFzhlkR#Vo(vhOHg_K1T> z5lN!#WR$gGpComr-{!ZzIOE->|KyP>V#ddZK?>D4Yvy2bh9I0UD9CZ)+?K;27P+ct z9%v}NRRJ0nUfs48G1)sm48@TnRpW{h%Kd&gcZ^1ecY#OszQ|Dz#I_5^WU-B9#ZDt88?pr>HBJoI6FzpfdCl`cFAv@J5@}#-5 zXRt+~>C%qj*U;R`E8l<@H|3{j#$ zZlWoEa^%x*x$rtB?dGs1_a1%=6)lsX? z+r^jP9k)JO7*LDsG>kdE%cbPl4T44saO38R6g3jdr57Kw7xhRN6%lLIJlgJ!{o1Qb znCj9x0s+AXW*Kc_(xD0QOAr68dNocjSjdnE|Nq(rYB_Y$wV<@nqLQzmINkhUi-j%5 z>a~)DBMme)z{hyuedX$ixD?;|sJ|8+00|l5Gs-R~!j$H0)-!ls|LRx`5u@^aP4aoc!x63Z95u_Ap=3@fN_!_k2$E zHfb5Xj_-r zH1mG#UrY!SXkaSQ(92#v1xxG6qB;AZTH;xH#omWAi+L>vRBRYFhW0~^?_ajJx*~Ip zjq#!4xdg4s>kP!9(GsGVL>W{S7-+5aK?6`nX*By-*Zoi4L|0mnA2q-@ES7_>X1pG0 z#(>xe^GQXEU4bY$#OO^RPDvC@s#bU_QJ*;cOw7Tw&L}K-;PEM>cl=5~HG{l@Wubqx z>Od7uL;z-Xi0P{%;4fsip_c0WDLHB12{IZbp)4Cfx&+!;>Lu%LxyrtuAG?H6q0_O* zeKxpy_wXT)$r_>KV#SA-rS^6Lh+TgDiDT|0iI42e^UZTgmqYBYw(r^fR!Zuw`-h)D z^##e9bEkeZE9<;`$wFsQcGV7?rVNKhj@N0YMB17g)&BhV4_@---*JpouCID+dRjr& zetTk%t30SV3^TqFQw}w35AwaRhp@Pj!pYGZ%;GiN0Wz@!b*J#boovCQY|EX;u%!w%no*(3?ImnyUTP@)i6Q10-P!Fbn(;w~S9t2c`2zbqNw8b+eHz3+|6+vQcrQe&)$9fIO5la&{UvH$Z z9BVCY1T1&Gu9d&s4i%{O;VISnI-UI@fbvP$j(!Y@9aIXIS2KaZZ|kL1GcX=ao?3P@ z^{C>`wOHSwZB22g2M=}n1lmo6F;o zDHk*RHZsD(pG{u7w}umIUu9;#kq?y#@3aUgefX9!8%MjpnG8=jh$Y9ES2gj?*}P+` z!DHH=Er^#a^TqmOcA`*qNLPF~^x3@ zNkAoHOa56lB%eHjDS(y{vvmQnORY`?E{`G>wo?BEb`H3Kgr<#Z}s`FKVT~c zd8qxvWm7Kmzkd6@Cq^Ok{plTI^nWND|NerO5bZJfz05jdu7Cf_KVHHn14xib~O`-h8&d1k^a+wWga3 z`-wbx>>98>sz)MH@$(Or0#H!*dTjpcsj~genzXgQ)LEbRoxlSrsz>rw zrzT>4u-(Y}=fSWwK)ykhoAs)8$PgN6wCef>=sUXK=Dyl!dIEk>o%KNavu~x5p^?o% z7+sOXum<6OIrSSrCYVJx1AhKHR&BASrO^U(77W)2!I@D>`#hcdUPGr6Z0!9|Auxpl)jmR!z9bozMClYIuT{ zvez6fKwjPunY3lAxjN{({DG7@>p?vlsrlwBd4+%#EI27S$0BhJ>6rbgG5^KZa6y;^ z2=H9DSCV>-S<`}za^@6>qxI>71HrI^G{Twe0uTHSh&J(=yvtG;RmC%0{tcKqtNcJg zX4T&0{R8sC+elJG`&E7evLts0unyJnrK?~N*K?2c(xzCzW9F(dh9$EEx_VrSI>Lcv z@PmgGodFh7ypjNk7W;l3jr~$WbTdOXKQU`pWC7QuW!vS?o~}j2`A@5<>MM_b4R``4 z6~VPS*-;}~Joq93C~&_It1mV`)q|d8hw7rVf3pDk7(hxa24{pNjDB$!Xj|7S9y_35 zK87k6Wa$4&LwSO5-8T^V84YXOtOS-RL4oJ_6JSNB>wK6W`-!LsVXJ$V}lB zIEubvBe}dv$y5yoQ3uX|zOe%Ym@4KS;IMuP08(X!KsLGB4L~P!0l)wG5=83yykh7? z2g8_zEyKSn8-RA@BBbJd<$SicuyVAyh>*moNTzGwm(47>#$R0Im?;1wi#6lhdN59y z0d+7-dq#P__eEXu^yqSkakQMQjXZ?YI5gMLbaG09-(@BDMVn(3E2Wo?8!v>FRsY+6sQKA~Iw8V}F8 zXU#DqkZzM|^1OOG-{(%lzb>q&MBqc|W0+wfE2dx&a_I!YJ(HW?AXK=aCB`hsrnl4!DvZ{=wkU$6}+3&0S8}XddBg)ikS3s0J!bKYi^Y24W3yu=ZcH(xBA}tmH?J{x* z!n6v?`P`7jgh>taNoT~p5HiaJ)D?79)bNIjlkF`K*>gq?Kavjb6=CVc5$eTB~lh+Jd)OgC6rFTLiU_ZX~iQ@v@Q+=J<0xgv!_jq>!qfC&v~lpMlS{ ziM7D8WAIv&rxZb_OydY{$0#UG+&+ha)Y3E<^7PsvxJ-QZLuv+F$?T-Tv}IxG&YF6Y zc5Ple>o(YV&z;^cX)~3d%AbZ~%G%lOf+5Wg@+KM~b-;-{3(7MAWd7R6dyPgqnVv&G z7yI_!FGDB}w)1>w(0~iaFklxA^8D_lQI}F#%-Y8=OT5Rub$fIA{pC}mv<2Tb0@0%( z$SFwGrR0QhL3#I$%3nIv%)EM*=&$!!rAx4b^u^TSk5w(X9RPrll3K}EuY5)>zo!T< zXz=M+K+A!DWLA(ZrE2+OhUJjy;(=KkGbGU)PZ*=#=~lf=BD2|w%w zCvP!3T~%DL$o;}@6XH?lZP1id^e<+uAE@GpO8?W=t&`8o|Jz#z-3fZ0K$F2Y?nnl2UP?kNY4b zAb6~nb}l*h9L&4wUXM3ZvS8!@l3GuQ{icM+=JHHA$xT|jEQAwIkct)nlkYo6!{ePt z!k7-+Fk~Cp%Yq8%*QdXI^#(v0-&ACjfdeIZAB>HDzl&wX6viiRhA6}Q949OHhcYha zfJD56`MW6DpjSR{B+TR?)T19Aor5xhL9jgmy#qby0DLSmDDAI-i#+z!VBvuyp6CY1 z(oVCy)e6IAaDSTtKFEDD&Or1!D`d?UF;Bq-bu4Sa0)+loQc`G~9+7h02%O#kUA&dH zgb=~fhJy&I*SwArk2%SyDsedHL4Gu+9t27I9J^Xw5lo23$>>?2Y^KQwIUEGck*=!| zX5L|F(YT4fZ)t6>s`uV}i{CkhRZGOf?=A9c;Oss8U7?KQcpJ_Z;{tV9|Ej*Tw7zZ; zbFxxI#F{v2vhFrUawUVDJQ`X=-!BdytSE-=VxUQHGRiu}DN?$E)kdvu<^G>PZi6Q) zN-_%J@0?nMV)SBUZxvcB)jsz7UV)+MK`WV+@&=B7l>Bs9GcgBy1m9UMA(RdG(Ij~T z*-fJ-mltehg@=%eJO^%|@68`Cii^4l=_kIiJDjJuql-^>^1H0a^3#{PkA@&ve0^49 zSULlHgiM3|HG3>hOu8ZAAXgHRGMwv~n&`Zd7*8R?B*?}&Qk>b=RVA+*7k zP8%(F9o|vJotAK+Nq?gct9Xiv%4h}2z*RooDH9K;`@T3yYlYKQQs%JW+sS_+AT3f| z__tn$D0;IRj27clcIdE;A}0}pgI z-q(;xC3WgFA6YP8^E{{LAyW8-oKqPFrS97kMo&-8rqa4DmZXow41PF_*Y76d|Fa^&OvZ`JL}!93e* z1w8?dU?-=5gdTBV;9@93isW=A)h~MJWTXAmy~@Zfq7@&SfY!ztkltO*VT-y3Qv(*Y zo6M#NG?KvBB5`69RarBo^95{Fo6l9_cTU}yY!8@L!Vd=7_E6*#p-{FBcKY**Bw!U=w;+SZ$-E$Oe!5qs)cd9{ix zGVnx(s5rIUl%<^{V!-MKV z>U7NfK^(o}WktSP){|&fy_?ul!y`F-T{8M%r0DrGOCA8JeIxsS+I!2WsJe!Km>6J& z92%s1=*=O%xV(rb(6MEE4?AO`u050|@c;L;;!8J_FH9qyhm=vGvM|^hzM@w9U z=Qes;MqV-aC@+>(qDn-^i38;?+nc_vSFK@dL18};%!RB?gJ%bBS8H5I0MDJn4+A3@_E0dl0{Ew{(SfFEzE*DvT$d_4_ zg*T`FW_8-4$p?p!`K5=cFtF?`)?PMk9DIF4gk+8?B?# zBIyJHs+tJP?XrvZrC!iNMqA%Vi}aNMwp6mW3PH8hy(7P3Rds_Wn;qlt=dW{d)l)mr zQ9)>ONIBz+P1WBjV;?^!pjRPZ0Kw?D`u)Kd@y4 zq`4ZL)!gHb6@5HBDIUU;Wx%auXkS({=?Y@NM&3<(Sm-S4y^i^1Io{I6B7aw8cG6z2 z>1vL##NtGUDIMqMMiPoa>w}CE?~P58oZ1z5s&sk(bHw^Lwep8?x^#Tdv3-rND!miR zbY@Mk&p9^^f}T3lp>-yuzmcRdmDN%1CGRts=>BKdTFrvn3$qlkToE&x3DjsLrun8C z#g=8n|FGq$a+FLViKTU&W<5X0Q@Sj38)Z*|VCxllZ9eDnqL=6LT+%vK_wBg*R1E_$ zqEqN!=Z(0ji?@iXN3g2xX12yf`U;x}~AzD5@`eK#-l* zvYQi42QBbbi_2S%9VCuugq?ED%$;0IOp+LOp*D%G_RWRF!BX|d$_}Z+H52XP#!5FT zdlMJO{Mo$Ix%*Ja`M6xiq!2QYY{RFR)^<_jBWbQ|ewr`(2KDYQ|hjyL$5(F|QXKIqvyDSMV%M zr|_I%ou#q#NjsK-dq!8_O@37_&k1UrFqB%o{i2WfC^00`R$X5*%Mm*n=4W}nJWY+M z^!NPbOuJL^n`TX)14>!2UO=ULpJpXFWp&1d49D6Hk-p3oX&mQ&=(S2ubXNM){reh5 z#MwIruJ@p{w3AWwh>(!Z$mk0fgx)5zTlOe6itzA8l98UKr>o1L;487)czWH zBj4C_eUaqV&rLlXX{EmJE-dV?O<{G1E`QhpCY63&*ZkOh@i_|T3|Ow?9D;b~t0W)2 zp(*G-zaoBGTwY4CS~|uQxhM1l0exqnEIw#qZLM8owHRnHvnPq<-FfL?k|t96jI+GH zX*j#h>ss9MaN`{3L5c(w&xAXYa%(KxyGc8npy~Y%ZAGsxv*RE))(&lj4b0|2P!=P9 zcan(wUfJ)>qiyj!SS+DBo1OQxT^aL?&T{SO>9?n_;LBk?9CG_jGC2PF}m zRPwslSRbVeN(dSuQbzhnB{MWx&n1 zr2yaYCfF1Unisw_lGtFm9FcQ~jvNo4t>yw%EZb zbXnE2n@emI8%fC(^|0D{0C^vC`f;$gLKZ>SDZSz1K1#l(kotYSllA zW;S_%;6m2rGkZ#MxeZXdY75fS{1Ou}n}bD3{({scaN4ONtn2ut1GHhM)SPBf<5OO*Wv0v^bZFt64)V`x z#ceCRA)q1~O23}e)aX0#Q0>-9c}ywFFRy;0eTT+9zxklAP-s%6P^DKFmK+rstn}q` z*{RBpoxvf)Mc=PI8X?Y)DOl(m$OjHE@s;>gff36v&EG2&-4@i{>rg}rf^c)S*)_Y0 z!_sn*3eNH&ipQCn<)c)Nu8MTEPke>?PwK^xmTBW_=LxI!Xx$>ukK$uTSDT)_y*FD% z@LX;272?KM_gRaov9i3fbE)qVi)QC-7MpR1Ykx{e#}(k|x*BWMRn{w+vXNd*v7OR& zP5KwEJ@g%s&6nC;&hm%;Fj~yh%a={sFp}stvGW^~Q~t=xO(ECSU@|SaBDn<}^W(!N z4qjw8r0|SIOJK#zF`+@JS4n*cx_iW@o=QCb?7o>dtbM!<;HtGa&Mp^;pXBQBF0C#) z;T7J10lwSurWASvv6*q|j+c zw!U2_g9kfSX6>HRfiE~w4$T*R`!t!q_fc1lNzsbTcvZ)lvhp42=sPf&4kkFTtkYMp z7&=CwvR&X}l4dBe)F^I|YUojoB4&soEuz~;v}jwLr941QIIOsY7+gD>L;I)cpz&?mK5<>Uklh+ zvDDin5DWC}aCn%+&vFeeg2KSi7-ydhRjhLASuYiwFEMOY+p93HJ#jUT*r6dFO{=oGy{ zn8@C!v`$26D1XfbDflIk>9%RN#4g2siM#jJYBwxf5}oMx;v|eOeBITsg4<~k0d5~j z;)6~R=^|Tav%lf-q4MtRLrx=s??^H4n%b{SV-^giy?tRXvrg0#dL)xxtn?T>+%PUM z{RG<*+Gn~d(z|Q%A!Kqo@2&HKk@Q%0K4cM6cFz9I?Yn}1P}W86NP#wPg&twjhLsMg zB2CE$`5N$IsJ%v0Vga{l+2rF}=Oy1(`JBam(>n_3LHeW(=S|Ib?z7bCN9&(ONLun2c^n>`w@8Y z?IM%SU-uX0kNfM1xS!1!)FHCBxnMxN_CXMT``s=nv4)lxT7@vVsEP5V*kBWyfQ09U z1dR`%o3iWlD^Fe$T(eyanu@b&Yg^ZW7Q7z%${;!|De5iJpUAF@9I}b6uDfN(y*lx1 z@-_EG?HX{R`uXO8nkFQdiE^1`?%MDs=Am!c%SE@{IP3!h6K$~CA zgK~&>dLl+ov&p*o_-_Q?ln!511(BH7P4?fYgFm>JyhQ5i57&n)otCf2{F@m0`*$z% z0D8qJrTK}}{}R86f>-l{m#@$&X#Zhp`0xCUgzO*kf6J#rS$}_?KMwmJWkKW(@Un(% z`2yVbRRwpGG;m<*=x3YulHXaW@0&%TvnZ=+ty*BLELG9hGDO}*ut!^~z z*4LT(TOg$QJJ7@XBewx0$0L+)Nl)-!>9pcnxW{B-6b|(M4PyOkjUUS00t&OFKJuqu z0Ci=XD|X$f$YS*qh~^KI0vK;DY&w|tQ9A_U z=vLqY=)*;~+-g7maDR3*mplN#9$t^)Chb9O7hG3UO4c*Qfh z5Ic#=ie^EXDNXtRNxQjDOR5jyxI$TiUb%2a5k|y6? z1}g4@6-&cQi@;j3LO1KExWVlCW8VA4hXtTU`wqT53(kO)-NRG&0Z=GsO3w-a5tU?; zCiNY#E~>^8x^TlXkXH51CT_(MT}s~=)W7{a1MmSqUI^ndFI4VL_)4n7^b${~m znd)8rJb>K4i<8G^7~@7VRI{KLaJ+V$Kuv{VME`tjfYuV1h}s<2nl=qzg>kFiFTgMC zxeACLZGj|Wap|#Q5K=#Z3qd?%HQ}WbZt};n&mC|?oURl}=J=&B(ru3G+J)|FFHLnu z?l9Vi`n8H5>t&r+M+2GSXohMpM*R#hzzZMYStry%T5?!sV&K|R&rbt+ggw&~q){Ms zeKD@;+dH-0kEI=D>UzQ*X;)61S-0Ll&F#;xXWAA%O6H*}025X+6`4Ky9C)e*kKqEn zd*Ff|?CV?ZaD8anStf6n`(%H6fUn*XyIUFZuc_EW2rA=~OytU)fWqVI)q)D0WQgMijkc95(tx&6R+d`Tt%(PPiw+I4pK48mmJxXp246P zS7}f8umcdUwEFJiX5^6E5V4o&J#+T#NBUxIYcEF@$q@LIpuM9(-P_M9m(ulq*4^Le zlH~F};y_y*g>5(Bv>Br-(s$~sL99&20+p%MYXv0w;MnXHDkLt?ERIFuB63fC<_OTW zuN}0QAPU_eN)o%IWvL;F$2CJ8*JLbl3hs>8{m^4wSb<-Wn=$hHcCWTRTH-$Xt~Kj~ zu7l>$1HkQs{Vs(o0B*X?q5Hvi;6LdB*MNtu6l{f6+FB* zM@b-e7sM{nL7OxncXJd(z+Gf#rDb2CGHb_Qo+O0|ZwDnrA+B5-B4%@7VhvNih?iE@ z)HKTy-6ZIIm|6D{n2wIOa2a^wDJTmM(2Emrv$nNTt%{@A6xU_)T3L-$3VjFa3X9su znO{uN=HGF_o;Wc^B;h8gh|#)AC!*@KNwrOJ(kQ40=2n$4!y?tNQ1~2QA{Ivsgs`%K zn8NmNtY9W+=cq@=)I&A#sJT+wmr8!4AU=|C$Tc zp{H@`Wmo7NW}`uU)K|dRO1|hMAUoOuL~}1kJ{s3xa~2AC$2&&Ip5KIdoU4kcW*W`$ z&Ko*RP)G3_zc0Sgc+=mVs{<`F>aPE~CaA6o*Cf+~(fY+V590FE4#hf2v7h`gCl4~C z_CM|Py!@W7xuuw6h7LhXgZPU3Bhe@Y=;ThuivmnZ^a{vcw6Rkgefbf=#kwqt_b6~R z!Oo#;0_yl!H{a2cdj<*AGpeym99-YRocqt`l0jrpnJy zR_x_xZVn=Rv8KdTc9U1}JxZdK-7i`w<_2E^ipuIhG~r4tPAaQnJ$(1>Dwt!`=*=OvSgx7 z>GQZ;#`BWbml-F8{Y+g-pe2u(xXM?((E@~tY*@+z${Ems)s~(h!LA!LQ7O9IhbyiL z5mqIT8dCP!{Da+oO~6Frkp|tcoZUqgi*jYh73!AjGA9e15Grtb_4q6Zh?%jQyJ01u zY4}(g!)avpHISFwVNYg&cS2Gq>5#Zet*Ysi>}t;#AL{Kz(c{Z*5&m78p*XbHIpGe| z;f{csLkuNNBFNzOCTPO$OMS9L~ak>Tuz7@mCHajCnk=Qj&+58K7f)iY`h2)MeHS=u)6`0N-)MzSr0UBjlGGF%RqJPk z)|2;JH8c)-9@he6=3>-eYi%PVy-FU(}p~v?bn)i?Kps=GmpC?fhqb)*U6Gv%`ciA5pQm#iE0tU!>1QvYAsN*`&_yjy!1JAl7ni(x z#z0CiKSWS+zeS6s^33D>tshFy%9ECg^nP$C1uD=FJxzIW@$I_Glts`YkXTW+f6ZpT z7t-eVnWHHLnNx~(hbgT1#c*frQuiTP3Rdw^1o$GmgOD_s0)-6R!0Cnsa@eC507SV$;8Yvu0CVXNfmBpO% z8zak(9aeNryAorEq;p=S*Tv%~?Qnu*)0A;kRkyytBRXP3q%LRAgEVc{7dO$4woP4J zbe%nry&P$gMp(F4Dia@VC$*{Z=Tebg4ZWsQjQf~=k7`_;_RJf!?W<27hw&V`exVmk z7*yN%*}miiUhD8&=T0eCfbv+Tn9Xq778`YGT2GS_2n&cUJ0#P5d697*Shy@ambTN& zqY54h@Z=KH@#dd;4-W=2Gex3&cvOQ(YcU_+wd~|*aK$KxWs9)?o@`T6D6J;VMf#3G zhmh>NmM4#Ovm;xAiegssKo7Tpq`q#D9!k~fX5uOf6F2D-3~ z9-Z?Us;GLrb>=}yIxfVfsdZ^_c^kZt5V{sxo5e#%SC>4 zu6&ESJN>F;A_MU>zw=bcO+}Wq)vuSK^|*Ufd)c;$>t0BT3)X)B*Z!thkV~Q>p)xX; zU6>UukudI#mW68Lk7;#v7RKpLscC`jLDdUN9wUePC{rbee3v~Bv28iWLwu<-uVcb% z$SIDp8TEAe?x_An5(KetO5a2riEEKuHN-ToNoO<> zxmn(hzU2u$DaDDJ9U-Xm;9W1XAGfPNXe@{e!+)8Jl;y;-VJQZ^yx(KRIf^wSlBs^) zyIIAX)XkkxEsd2Q97;20Gs}ecA0X;5h{~NP^%ZAtZto6!B!=_AsFMdK<6btD0!-Wh$;;tXlrX*;PtU;U)@|PJ)g3!8>FUy|{ybATgmuc(Z{~=`k!&UL zWv}R*+*Oxv$_89?F!;Ls%Ft4Eh1wU682C4*0mb>3jH(unDF)GwFBVxA{T0@`dj zvilj|#70>SM1!C3BBb+pL`r9Rz9diMdy`w-ljFQ-iOu3t;xGQGfv8k#(BShd>5M*d z(#oEg(7_w!rsGB&FK2}(x}&@?aqjp-mZqqBK~L8Dk9h0&?DT;{5~{qgmL_Eh2a$`+ z_>mlm@Vcfm_t3^RsL`gdgON?0s}eRzFnl>9xIAK=4@qdLiQb?vtYH$~S=Qsm>b76% zwv$F&@_J7C^bl@n810A0hv%FrsBHSNgdEyaHe8lc!s--#agNBoq${Kkd&}_hp9Jt9 zz?qCIWbaqB9A0)_xs*n%B^3t^tvV0dmOswSDRU%jqLUzb=S^=xh~?9~V(xK=3{Br^ zN~>Tav=s?UWe@*#ud{x1PJ$@ym2Sxu_jhXXcaJ{_YA{XF$D!2|If=)hW?p&mDD7Fe zZT|Q?);UneA!XR+(vjwl6d%6WkfrTfyuKO(l79{Af^dXa)|G2zlZ;WKFUVzSRqM8* zd?EDIGNI^Ys;7QM6AiR=ge10asM|}?A#$$jpG%COti8=9rdY>E3?zl1=TlQWw7AYG zOV+kbU;1T>N)uG~_{m4vtzqx{zLuepuUHe-?di=l7PZYGl5?Hfo(*(dyor)5-yEst zVP_@{ITc&&>@aN!+Y0$<>HZ}JpvhqHm_~$hc9-atrM#WOq~(HT{h&CM@ot-9ye{)j zjiYVx5?4W~jQXrjQ3B28yYtV^KiMER4umvNbN*7{3 zh~}n>AqwMX6&LYAhYu!8(e<(Y5|M~?bT#3gXF<2XN)V{d;drSsh2oL5$B}qxWU9`@ zi^N=#Vp5mP(kxGVf`^6Fg5K#eJ3kgap(Is2xy5Qmf)Mtgom3ta=$>bXa^(az)L9S? z+Sh5cqG<9ZGtOW!D`uqzAg(H;SiL zs1Z3R_~qu=0t+%%q_)m$YaCL@x9V3MvhBFmlu<4Wl??_P1PgVR?tG7-E9SWy>nHYxhX!m^1Xm9CQ;u z22dIZyUW_J`u47S3-K?gs}ikVW*uyrkm$H~lh(;FEd*0|EirG>?Hg;Vd6Yd(hR)na zrm;xgl>!-ROlj?Q(4c-^)19(Y?$fH6D)LWXSW|4WZVqyM$mJA#T~p_0S=EPMhSc<+ zQ3KZI^k`G}r~_@44bdPvqdPBNOKEsR!ulapX%QI1Tz)EhNt4{949%d(7 z@>z=Z+W5hbDXP1$@Sw+fb3qg3UO`0fMGE#lGE!(AmjPScD-Ub^X|T%pzVbRUm%30L z!QWHi?C19m+wXByUD`)S3>Kw#Gil_>%lyIc`D-l6T!mN{Tq~`#4!j>iL-6%?=f#I( zA_3p#n!CEY*=ctAtG@0*2`E}P>D`iF_Ba`;w0M5Nde_Yu-Ab4}KIv7EzddPrIIXsJ zeIr2B_V8F4b6Nh|7fS0_NlP2>Bq0XxFq+zK`qGb2I6E7N(6{9hUC;68NJJ>Q_Bpiw zYu{1v3fK6@9Cn#DA-Jw)@h9l=#0L;k($Pl9XYj*qU_Q+#PoeA89&McLxo4rU?1qux zKT!Ff4EqeOV5t6m-Vr?kSnFqShG;HdN;6cxSyw;T5Dv0Okx3~MHG#&&$$=rsR64%Xg+tWJqXy@?Ot?(!hG;(p?92b5qpurPo^z~osu0Py4aGc`> zFZR;EOg8^Iw|60#j8gJnz*}dgzYY8U*?~Rsf2tpXlj%_aO$MntpxbdWg-Np8PxaP< z!gPbH;dG0)4bIXcKP1wK0kZf?KZZ-T|NWwvZb4-pT9#UmX)}XSVk!U)1*EK<{`lj| z0yv5K^vk<9r`v-3fxQMI58{F3fnNdF2(4InZ)@t=r9gB3|9-w@#DLz)!*-wVR*bp` z0RsPIe`{l|`R2xbyg+y{s|24$U~kVdz?=6MiE%nk3;tey>yd|nk~qBd7Le=Kg4)Vk z34A%eINkH(6b`jTiVHGtmVDknxhm^#+GN1J16IkcI66Q$M<~B`g67{>+@lQ&9A{XY zd#kq4Y-)9VR2}G?;w&nwecRckYf#zCFAYc}vmeBmir&1>M5|>aYAxebD|o@cRO$-1RJzR4RM?|} zh0WI?S-A_L=hpWWHoopEZ$El<9sy#SOAB6}ooNhZP5+n4T?b_;%ux^vVFP>W)!S z_PY|TxL9ym1iXM%Vg;~Dlt4;uAQHD{-a1?iaO>R|rFV=uE`Kx^16L5$p_<(96Fd?(lEHz{PD@7@66J`86^{c1WyN1y?l0@q?LqvSB-px6WA zw-|Elf;X$<81CrK)C?f z?i{ZJ^j|E1AC<9Y>LvVVJ=ubnox?`MDG$6SK}E;(oj?9>F|M*>m>$ zSAU0W8$1=Xl6xFEw89SgB`eN>|13bpE&JxL$5n+Fwuy>x(Qch@{D2UsH{ggLANE&6 z>39QVfxX`K*ya`F1~)3x*p`6Kpj@#Cv4D8j3>{RCa@TtO5uB++?fozBg=+wL^>!S^ zozQJi5Z!UQE?(abO0kiZuK+w=CeZ0w^@t%YUr|8ZU>ER_41W}EnK6%u_bhYU!2kpu z(lw8AWd@b+I(pyYiRglyVI&69gC_tWXAD04&W258V_^^eTvE5){!53b$ji1&1^5!l z!A-m9#YZnp@|BEsQOl$^0S`ILq zN&MV=C(zPtKA&LG71nh#@>e}5O2@Wiqr(1GeC}aJaDmjCuH(L8u^hp$c_WdN=mg-i zcfia$ZGN$Uq6&|k@oQT;W45N-?}yx&a@rrufM!bqWy1Duy|emU;+W$y)3O@3TzcO# z;Fc0$BK3VH?#5e;bZdd>#IvYL9|4ZZw*A4Aw95-|HE~4jgn`O%bs8$llT3O7(oBBg z=b%In>2jK22y`YPyw@guPFDSmArrvnnfe}693%ndxN96=r!IiTbzfamJ2!pC&2MEZ5bzefpY zWUG<7R+laDt-CbU0{Bz!d8{hZwyj(-6OzYfbvHAB(LvHp_wA)YSD`ey)zPu9IO|Z+ zCC1+u@h@pAV!qS%EQ4`K3V0R6(CeYQeMEg~6~@PT-u*q?uq9CO_x_H;P;>Xdc?6e} zn)F9dkfy=eLSUmyc|itIRO4GP!+6Q1y_}CUh=;P@R?BJ$z~)klnhu6TZND+yGfmOL zcB6q0X>7k&u$`#`PzabR7^7bC_QX;o-JJ1r-RLt?iX;UQiHo!3M2KaPk{OpQW6~~& zVB7&O4{<-RAFVjcX>9iEp}2U0c;N_1{V!Cv&h#6+jW;GXfKiXNgNB5YS!g+#E5?;> zc(3uU3!i?_98cK%6e*70vRXs>_>nCYK#xhD~XF2LhNig&I}`?!srNUIfpjxX@c5$)TKj z4?XT(Go6`rT>%t%>JZPArE`k<0;G-WGnocH4E3fKf1TIn5~v_5lgAsB8Dh2z+kelb zh6?!fntt%URocx>p9D5u%TInlw1NF)j3hoI&XQaQ9@~_|cH4QFWy`C)1nHJyq1a zx!!Q%lbf9m!3=IaVaW|fGZ8QmLp}5yG@baCOVkSpv+v|wcF5D4cpY+xF(TlBNv|H-N}l?cx}+cZ)fR)%hL9gSem)?sqG%rb+%8HzG|Pv3$pr&upCYjoy0zbQ8Z3R zT{AtkbPLnw-c`t!fRFdd1tj=P1fQq1&f0Hf{ymtM->RK)C?Ln$LWbv2=R^-P!?H!mj13f@oyTO$;_H4e`}4lJmB{GWB2% z;kIn1dimmMqfB!D-1|1KCB%6%~9!|20IzUYK@|)d$|4QDJcBa2_>?Eu;m2;EV zvh&R0Ywf@PNC-Y&R!qPtZs-usiRwNFd6T`~hfuodmsdSmmVYw8D-%`=f@0)fM6Zre zA-&7~{ih5rV2Wq>_{wLAB^Bq+cj^tAN^Z<~`tLDL~H3hcEP`POMl$=#S*2y zI>FzwNU<11hHjFmEJn{^kqae>njsVsbW+B`{VufNh80&JP_4|_XYXDk^fo^{O3ms6Ll?(o+T$`#D9O~{|@!fsrvu+ z1d7;(nO`>e&&6x!lJJ};m+)KdmALz5Qo?3ycYybi@cJy;C-YmE!DZ}+UZ}~Um zqyHY$;IU0Xv|sLNg&PSBTQrVqczE<{`WH2>zzNp+)2OlpOh7E~9cU6r;1zkTvpI5s z5?MR!d}8sgC;Rc`ZlHBzpb!TFl8+)rD*uc<|19;V`O%71k)Qbe{+N0sifB8t@L~d5 zU31BvS-*HN79#B<8-e}Fypm<2mO#v;LiXudi1W=qM===up6EFK?6dA<9 zNH0c2RwAEHBsOnKS2BtYWi`|4CEpYup!xfbQ-24;s5rgL&)52&H(DL!C&FXK=4LG#39|lt=j8zmR#$^*aWhTcG&B^bq8t}y{OYaPaz4Ys zMYm~5IGr*&2{m6xt^d7hPjY$K9i()gvs4;CV!#wGwMZ6CIsbJ6c{w9jAvfgc$k~)+ zPEV#gD9Mj9MViJnSJ1SVpg_;(j@*w^JLPISKl|kfAQURxUc%to^cax7$e(VSM3CZW z<)gQ0;a8>`HZD5yEU_~OEV^#s9{Mr3N2&xyo0yyleo1MwJnD|oa5R=*EjpYJ8+ z8}MSlUZDrbp4h{WdN^|Q%sSrvKG`S3iKspU@#$~UW?q<{LoID}oNgC53|-2wO!)n) z*I+8aKN0h3U>hcO;aVxn~QQe4SI1lDDCKx9f*nz2lxcBG-O_lBnjltwbAfgr@y9q@#>h{DAnV8D0p z0n`2`Cet}hHt@v{S)Y~_5*;emnJn0m`$X-o_3O$pS%7p1jzXXp3BGo;y|*>#puV%V1X2S;V#jXmxs5Ds#P0cu@V>h@nBwy|2t5JblsHiZlq z6?g0s?0|O@9W`G$NUL{_V+)Aiw7LK`Tk;Tcw&A1ap#L=T&2skj zr){A8)bHSH$kH!6k;K`irPELHZ?EP&>xW|sz@S?J>dv%~8$OLAa*?wUIrl!U18^+G zYdI;7V3C#4F3{2A!^usXagd(C^24bnau;lnnJIyCst%}iwVx9Yf_=+=H^@6-{^35< z4<)=;+ZAc0G&m2cOdAJQ?;WkZ#kX+*Rd4b84I3w58O<{0rzY_Q7vbXiD}EXhhKmI5 z2J&Sd!Y`mE`l%Mk=5?}a2X|W^tl;*b;BJH;xZ{fW@a>G4h>weEuL=piTCYuGla?Dj z{ej%*`@zs7I5x}6L?cnKXJqSD-{)pCp>uqyx?|h(G{v1|op)gR9O!dnfpCg96!dey zg?Ik&tGIuS6lrs;5(1zs=WBzwM(1vzv7!botLwP#YQ`@|Q!m=-G8vW1PgA2FrOV`0x;+f$ST-)w$ zu&58%aQS0^(5|!|ui_y-|J~Cs79B#E^+pJOPbWxVp#>9Nn*!$|AmZJgGGNIK5@Fc2<% z*+tzqmC0~aB-ehKEr(>dueihoHXKS=(>R()c>Jto zPDn!C;*VKSg2o}f4?gHG3|k5sWe4Ea?ri3${D$=Nk8=A*>lW?T2Cc+$l81pv{6>i& zN87oE%ndotGKY2MD?7)c= z+x1|n#Ho85e%JR=z)r{>fDHSW;DxUz`+s~cIzM$E9#vf3UjP;0jdasjR2y9m{#v4e z2Z<|zEHkOnNWuHxVBWR%kRtsF;3CH9m(r%LJF&Nmh$GAS3cEmEKsP`+Y3^UnHXwEa zay(mBXz8%~I%q+TZyXc+j@P&qwAKSeh1X3JmG69pZ}s|SWZSuQ=ReZT&;jX#SF59B z5MM!;wZja!ZP$)9qKUQ3@HZsXO$C;+s=89-7cJi`Er<`N7lB9ATM7=o!eaE(XD!x_ zJE@sQOS_{-vt!{5^sYz&(;4QkZZY_qAfTTqmXi5`3U~O~k0RYsCQi$zNu{p=x~VM| zXe4ntO#t8E2nZor-FNaH_t&LU@gKa;wT}U64TTHrreAh7XTWk}^{aIY#isHi33;ca zSbD&?aWuF5=p!IKm9zr+FV4zOzwy)@g`5nSgwp`SZ{g<)Mo3a^0vg06)CM0`ZQ(ep zo|74N24K(ro6W&=#YcOqK?Us=Ax7+l^m+gxUQ60$;tFyz8;xhw@f8ZD*+I^NNY~Xy zL}9_P&IR%sDAt6$d;o8tb+XdySKuek!0Wq6for7f)=bQSXKR9}orNmZa1#j;2To4x zD3E)zw*Ppv-76E=K9Z-(6TXoWK0XSxrO6u-2#561DSJ_l3BkYD;Kh8f$)=yE*tJWe zUUJ_HhD`-r3%U?G(H!ST{f+zLt@ornwEVQqCDe%R5zSC(Z0Dq6MYzG#!GjR{eppUt-c8 z@84*WiPOcZAVtWTm3YM{m?fPq$~>q6qCM?KX`2LHeNv;?%qYKVU_6LhP5hg)_2+3Acos&n)t$IO7t&GqR)4ko> z-JFrs51WurGcy#5*tOMQ#NNE36g@-UC&@lJiglr>r)(CVnZQDdQHA2@nL#Yk-lw7G zcZ}G<5czZY+62MPmA6{gb1-K`%(v){0h)n)c_^I~Q+LpjCt<2(x{yDhzjZO+Y>e^= zlNaB>E&uyDH03udRI@hl67P~u<)&pn5|tOM4CGk;l;|=apAjkNHa(v6l6m$^YhrDzY1xAWAatkvRYBW$_Ga?loJg`4M znf9P_7*W<_9=;~+T(F>(Kf=Yy9a-Z0%tTY7+q}C|1V$=*b1nZ^zfq%?_zmfxVzeAF z3bW6{1Tjz7NKK&a>`Z4OCPx^KRA#kYh$Q>|6La2ppbCqn&JIN~W=f(g2?j8dAT^Ik ztK{7e35D_8 zvgQNd=&ArA@8*whtM}rzPVLN8?R+=y#s1XKbmUYqQCzw?ZCB#`s%7^O&6&i;ux%>g z2A&wJ&#*sanybzr6F|@4XQ3xH;s86kQ#Aw(J@vzLcCswkXo}_Oy}o z$$*18rzY_oL`NWxgOS4v2ss%he<@iIFqCAfF5$_M)amMc%Tu7(3?PV9>K#7F}xVR zrOv^GZ}Tkc6yO(uqp z@(3EYurU={W%f0)Z$JC^0qaA$`Hn`AE+9a9IVeAen!Np{Z?e032!?X(3gg|n0x~Uf z9m>(Q-<=JyUJoX7Zinh!-V7G%|K}n}R6#@6`e&0b%mk-@J{Sh(Z%2lQ8Z3lr0qUGb zw3-eYAl{{s9Q6~?)JvhR(Wj@5Bp}!_bE6^l34}-pr;s5U-&|}L=f}LLvT`G^tzf>l zO2?}&{bMuFB$))!IG|StqUW&~GIA%t!c8b-^?hX|yo!$WR-FEoSDh%oN(nb2OCxAm zS|u97kwv1^f>F)C9=Qq4dwu9DX1I4f`?ReDk_w4*$swbBc<%!;m;v4kckD;g*`{xS zC8WoK3b!crit36YYS*c|7ba;EytV*{mkSi)OjLq5Gu=B?6)oVNbvB21QGa`cPrfUo)HY5I?Yu*&6~w)A4c9{RCM2 zQJ%o36&u2yfh+nkhXk%`+HuQQbJ_}46Y_lmImU^@pMGZC^=bHRxOtlWH}d#{5Wu1& zmUm&#ht(;42#6!d4<^*bG>R{{MO!ZAr_`j;j5d4us=AGs-7=FgtF}r+g$U_H8+3D~ z;Il=mxRoc6q$5d9E6tp!-8S;g+~tb$e}g#8emUOlI4>0@1rGssVq^Mj&j4`*->`?I zZ!9c&RYq^5Qh>r6rOW4tO^);l|1EOt&?A*B!}R>hwF9MC#_yL0hiWc}7*Y(jx-a-R zU7~Y-Ao_5-?N@sIchcz!{|(r%i`U?f}MDqly!B&K4qJ0mj~EN?w_zMIF~R6@1F8uTUP=2p&E&=h(Dp zZNGY>V;b-($60udd-u-{B`tHLtQods$9_8L%x32({aD1CUCPN8JD;5l`1m8_AME54 zJ;7aRy2(b6JiY{$$q&mt7@ig_2My~qGPHxzmqy;Mio!_l$iwWQEWBHtB1C3jX&;n^ zH#S9dU4p;dDx!VJMdLiF?)*Ow{zxpB>5B@Il<4wy70*OvpRk)PGB{-9J*~%JV`2qK zua4O#R{R_rUY_29r+t}!!=vhc01-~~f4pTyyo{q&B7^ePTP=l-18BYK=GIM*+t@91 zQ5aFI%j4AhkN?7qs?~6)oBNFf+_5h_!0R%CE$F6ORh&VfZ-y9>?>lu(cDNb-GdS&O zeX}?0qyE_X%Ov0}LcSK$Os|_1QCm0PqIWy8lBWbk^TN!bFZAUHd;eO<=oRPVU|!mS zS=?bosv$V=5A#82o@n*zc;(%Pr3eZ3Y})5zgCr&d$&j7(ihn=7mnWigxP4yZ-s_V; z0TZdEq1xRqS9s2*1I1@6b48hUr^fY6d5jvtQsa-~nM#9iB+Ju;_Ya!%C&;C~2KV@& zaJ^pO@6YwmMrIP!He>C{pYgo{|@!{2>~bYjAW|qYE3pO5Qzu=>1$uQSgYv}_5T2EJu5jU_g(kD-@WU;_m8vAUi++d);@dhXMf7`oai@ND#VXzALHQQ5UZ&w>f+!Ktl{9` z%@I7j>oMZmPr$(;vT{&Rc%!DE!2HJ3?Sq4}Ee?)qbgDkSfnGmVj!ND7o50JNc?qp&l5p(>Ec^@C=5f1zn z%Su;oZ@Z*`z)fu{e6fgFpho$@xyuZ z!+<9BCsJh??=jyHlZE^?A391&!I{J(LPSXrnt}ixfw5;gA2{>FwIB7)j?Sg zr9(p~mmj*rpo7SY@|D5MogXL0sR8};ytkUwV6pbw9|q8r$AV9BVvX3;q9YBctyq=J ztm>5UYq_Yrtnt0k1Gx3KzGSsZV|{OkWz&*)6I4ex2OmFqa9~wr^@#s{3^!r7n$FiN zi)P%bkXudW3TicA{{tyN_pgjFgNXbpcg<~sEY+_+1Q-3O38_{B_A^^xL~M}P)>U(Hxw zfx2aUND)USF`SM$`(`%( z=4rqm!7e;zT6AZS?d^+=Z}m_130|`;8Bo76KeE(EfVfv)sl~p}r~hNSL2OUa8Ex71 z+zJ;-Bi0$dk0bTaOu^!;efHPV$A;XKM+uL1BJd-eaG(C2x|^*lZu1_Tr|5)HW$1}* z*RKcY;~s^BSA8Uz9vrR@7u@;nqr&kO1{dE9Ok&I9vrpWX>0|%IK%GvW?>P+^$Al!=vCtQ^O?6ROURICKcx?0dK6Aq1*?u+! zHf@jpe(;AeaH1{h;ZOGCO9*}d+tSR6lgkg`Quj)qJ@4CZ*$*P~9U?a$^knm7@uj1| zgogCzRAkluy1(}=c}V?8NpU9jo(x-Wve(|4h6x}*X~R)=z&CT#(|+f*JXmQQ{GWdX z^+9Psv(E_+&ZT5(<-es+JJ|ff%74^;Anl*85yoDHrzHR&{qx}dD*^)i0y%os`wkX> zq1tD=cEF6sA3ggU z-lSSpNl(JY1aCkn>z)o#QJ(U5yv>jCuV$a)$0a&aZ!DL3nzWrZbXIu&_JT z?!zybYWESH>0i%!BaP>dQhN(PL>0{AgPoa20F#ibu z5$9irP39I3ifD~|RwQ*!kBAkY$3u@t9;_au)tNI|!K&LVNvg$c`nt?q@m-uxlV7WC zsWz+is-~+k6gJo})I6OHr>RPSd;KR)C;)H$?0r1a!h%C*uyDCyFPNZClSO>s%# z7*KkRUzG0n%BS$5F64XTM8L#rM}`x&6Oxmn`1ts^_}mc#hUgKK?AO_?5_aCZTP+1N zVo+IrgO{9i$YazKZhjT>$E78gUW^drU*BE zXh)nwn@q$&)JYuqnUJV~Hb)@1ul*T}h)nI{(@!i6Hb`+!#Dy%YWIz%~S zm*MuZ&0^8o(t+H8rPjdCqa%MX*@4C?)2VUp@Vjlv)6N%tudV9mHZvxfc&w?SQV~CF z>T8pzm~+CEcCl z!Vfcd3Z1)&3olUk(J3GGFsdY;BVMZ4s@JGjQp>xcWxd~YhdS;oiB(PDnX(_f~?@HeQXfSHH?9A-6{+v$a_UtA&0tG%M z=T?R?9`vK;rI$cimpe(HSyd8vXl6wY-AJiP!&Fjx-)T45MPTR=$_Q$+3Nt{ZrkPkP z&7yS+bzp;k=<&fx(&fa3H+Jsg1J-rF^|z>1Z*6)N;+*_S{(9}4;WP-dw_?8DhPbW7 zY53Uk(e$3S3NW4`ZccR5@|z~Jth@TR7vQHUG{*{wFJ>e%m`oV1n7c$3pVdgCAN6KJ z24{x@7&jSTkV-$pe%?Z4ce{{ub^tmDI_5fvJ0YFZQA;sZ(S6b16*ERU72xQkR#<~JqhKsm*SQ-9YYc0(Xm}>(ajX*oRJ*7>P^|(DL*w_)sX7_s-R*ATQ3CnJ>JNp(pzh{A`7zW-SGdQ0#ou2!(t>mLgqM^I0K@KHXuk*j<|HyC9(<|Q1{UD$zvD4w5 zz7bO*%N76p#^a%kfo7CDOt%SMw+_P_^MRHz_((dwI%*HXe2{O}T9G^;+JcEK}W2mNKzbcOx$=c%MV?N1)F3-`&4cPC|rWrOeAjD#JA zXCGI78FnglN_BddDwrC#wLQgaerDO~e_j7$^oKOvkQmS6{)Ynhl~qlatW8m4XTGJY z8~lD!u?9ug(n{a&6*I3FjOtDfU@0yop*FV$wJ7JHi_-+jLdmz1l)kX3^Qnn#p{4qt zRORoce0N7?KpQ1hxwKmgo3L%mU9Kg<(-&tq2VQH4=`*E0r&+|JxnKKD`{(f3aN85r z>tpQhLyzzug5}lMqw5O1n)fvELuqaX<2$FK%Hp3f8TzIKdjCekt-E^zzuFF)_Ikz3 zI%iIJUUvPW)Hm*h!Lnmz0@mJkl~R@fOA@jlNqL?Byq#6R%wT9oE}@lee?Wi#R4%qL zIv+WdwXFwLA0#go6`$(tsPN;P%Y?9R50sX*lwm5n&79hDg0I)&V(ETlb!2I~tu6%~ z7$esEBp#vnsw zi&_0~*V#gl>3-9h=B)8}gXB{6Qg%o#_C{@Qg0zq{l_^_R6!hiByQK14rFKvH(WNY& zzzlNR8Rus49*#6IP7nL}wSbx4(TSL*W|`a{SxZgRnWbQy@8RKCuk3lEcC#6VHF?*W z$=ppf7Mzn~`t7;mzSHATccewPg8%&V+1P!mF-@G{l(Ul)*X^xK0;jjzR5-(WBK5^h zcIzIp;4riU#Uj{JoPFIO4VjY9BN;abDcD z32^R3IpE;lweHd+1Agk9#(DomYZGcSKi=Wt1TJ(HEu4}-1RaW3$r_} zNl`MZejKN3i}*Ar@$eN6uIk6w6wxzake#x!RA!@JP9J{K)~?d#i>LP^U$ML)c=CU^ zwbuZuLJW&itK`3tJ;!|<_J6tA6Fdk}UzALo8OG~49kgn2b;80XMPpvvtsuiJZV zwQc;GflC&_9oHF-MttaWo@Qh#*`@x^)^=ZnWxtEki^q{wO+!P&8v=o#;^@R^qpgnC zhD{m4#HteYeODqr*Q>Sfm?()cYr|Qn^K<_#I~mrs;{r6>ZvcX{?4CRg#i(b0kOAle zciUH=f=a{yyb+L*o79 zXNiPI0U=hgCiyJxNX)SyNGo)m(rZC|JI$cGr908{J*9`WnXUfDaH?~z$gIENYv}>; zx&uem@D)a`sF3h!{qTw+h5vBRmf?kQe#r<=@j6)U-ko=A+lgnAUCfpNVPygyMzfIq ziAf9(UTJZehq>-8wxaSBqM8a+5_}O-M;gtd)W08IEO(O1Zp++W9qmq(>S>P5%ov=F zsq_1&#M5uAy=X9^1?2@Ac<({fdD`_OV)u~Y#8y-kGpXGAM&=4*NCfs5$UI+0I*x!R zwIv^FU0*0?O7=M!T5IGD)l2}841a!Y%KaaOuu5b*rQDEQpIrPe_rLOI!(vM!9YZQi zpowg$fErVkv?}o|8~Q&0Edv8zs-1^|_@$iDRH_oqzB`+qS!E&N;0&+bg~r{kXiAh_ zE@$X0~YrFQ12??(XPaM$e>+_+D@#)EmcIb8?w8JI>S%ZU{IbKy z^=Pb>%s-jz`Pe9A;;XItf3T;jM9B@tyP&ZttglCUnL-yx=G9GG1M6Kl>%A}|BVnZX>n&g?IEH$w9e>vJS=)zuV4$0A1XFt&$3EFF&hE>a_JY!^Z zHP(xZi+c&^{~eBR96>$p@|!K%oi#rm%xNBcpmes|yazi3H zLVd3g8k^>|I*_V7BsUf?MqsGzMwq_(pJP}1A)tyQ8VDu>zXYU#*FR;jI>zs1iMj2b z?kz3B3IbPC^_!;6LtBGRXYKArm*j?3E2P9FETl6w`)88^0qfAa4X!i|3_P}PaGI|0 ztpo!>$2o9>+JM};C-~RfLTE`PQ%ZMayOI1K?6HmC4Yu3WYBSIevzYB_v!dGIx3%l) z?e8R3XKCFKg2EaO>+CLcMY>uUH8!J0v2o=?>&4|LnD}S0A+5LBaA8!Ois`1Fujy5Z zhGhQ8Ut)#YsfN**hTX+o*M%?kek`|}!w2u3FZ*D{^J#g*!yMEb!CNEcyCWz`Uc~1U zy5+?;4VLYTSq&vk++r1?rfv&gH$`3OcTZhVd6{(e>{|R^(0rnM&2Xu?1 zmOWL0cWe5Me5Z#+mqa&yU7P^u)_U@u(`w{&XP%CWlXdm@ol;lzWNHv#WqY=^Z^&I9!+p&YQN~ zal%|5F}J1d9Mq}vSy^@HcKnX?g3D4}Il0b<+DBX%bFC}mGsBcDxHQUzX?C%-4pX!{ zlf#PQzKr({f(>a6$qfKQ{whwK1^6v5N6$!jg3b;Xt#y{vQAyI)sNG>nyE50`(}|w& z*1)Fz(7mC{9h$2iVOht++@%AP2J2232O$^!--ic1UV%3Dh4b|RDh-w^qz#rS82*yQ zJsDzCgoz|-=aKHn?a7!MtxQ(7p8vNY%87~I0nG{%)(Tn$UW%+8_u&eMM=Ce7ib6v-Mt5bm^)x; z<@c)|w&dnvBeBD>+;^F$BFxJY_1T)POu!qQy`?a>p%Q(;`GOEI){_^9>MKqv<^*0( zAhhjllT=OgD+4Eb-|K`1KM)?swOEL zx8E$DO{j;gt|tX1y?izc*6@AK{~}p93$4_WEW4FgdAaG$kmqP)Q@EP0(k!7qyX&>F zBy@5UR=P7zFEd`M`>(^Uj3%s6~3Ok~aU1%L;ar2~3fNB#WHHsJ;2g6mOT^j+D)D|Stk)G|9-rr4I|mg~mdVY|>V zRD9sB%TK#2(5tQgJtt^SCO3d|(W4(58-TDICA^MW$=Zr-0XrwtchG~I(Fo=bw`Tz} z+daSSAmp#xDTiY>%kECT8=(82c2a^D;j0*IJlZO&yw(mOD%7`&YHm(94{0R-JRzoP zqXtxE{|92ao*6r`d}aPNy3$Q;?rZ}Y+Jat8vUVUYjxcn;AFAPn;T}9ZPbNBsJ&A%Vfk`GM}~A`CHJo5SGK7 zgfqn31|J){M5|>QCLdzb#E)yq)iYbL!Ogh?Q2(GL@hGK9LaMNR(~Sp)Qd%Q+igUh z+(ST=sw71Rfo}8`;DC9IM-)j+j;=+>ICViWxuG{Qk8EHB!dy&wc*ux2ur!*v9Q-Og z_R||cKc_qi#lRgTp-3VCU9?TCNz7|iq>mnkMVf-S0fbZlME)ZXg!Qfs4>@kSBLd?z)k<{?5%lLZ2vM84_mX2z zGoMph5y?+QQ*zQ4tBORCGPxGgogfLh9%ZUZI~Ou7lNnNc8a-I)&60taMbQVY%Wh}z z{?3MP_J_-){tG<}b_j@6BCG;J~dpeiVArC6N#cp0(U=f``9ibg2ViQ?U} zx0~;9^zS4sXuURFCHy5&$FexdI^TqW$<8y(-v*}qqRu7tm0X@1#qO1e^eL5eEJCPd z39VL9u7mqi%Hp&?<#Ds^U<_avSUgNBs9`&fx6=3XBAqo|834m(cEKo8;(lInO&;VW zvjW;Qk`2&b`zyT(4^|`C#grqoN$$_CVbw;`r*xaQ%CJDx$>ap6Dzp+tHR$o0KTz_0 z^GF$S=a}6IAQABt!+8MesJIaxpD7n?3B0s+`0qtdyDr&z$Wte-A|}1eShDcMPp<%* zgrA}xJn94>b?K1<@WTOalovvJx{p=OQEu)Pv!V0r`$>H=^e)Hpu75Sq!CBmm{7hDa zo+O`$B+2b(avpKvv*0Buvt_ZpU}uSF+5#H$9@A7jjg4!zV zrVA>JukDIj0Ou`7Ba-DZLaScjPYV3_C`#>Bb~ZX}S-O@N`?ImE*a=WYi%;vn)b?b= z$MPSi_=0vzQy^3{bKXKm(-as;4R9QJtJpxJ0fS30ohi0v#y1h5Y*xaxTNx-OD;W$wr_j}T2YkM3nfiTXwP zO3j1CRnF*LyYHSZ#*rlW?(&LqU9SBW`;_7Ji`?%Id(FsNUXp8P6O$|Am%*&IZpfa@ zo<6Q&gz28*o@K=Q!VgCRxk zIWHcp*!^aBnYrXzjCUigbG}$hoE~Cjm^EvUav}XN%l_reTZt$4LAW``XVFVp7se3O zV#$pvo?z9#A@BcBTH7IE?WFo|rofueYx*?;Zs;ggq@t`G*>gfxn8X+~tD zy7F3|l=yB&4*?8^^7(s>2VBJ--lzcxgb3-#EqUzSVa9BUDGjea+Z^6p?ltg!N7w#| zv_OBvYhA4ef5v6CBDRWG6}4e0ejYB;br1Y4d{iF)1oP>u#A=uiU&Rb|&~9~twK8xf zW+9x1Z8-r3NY^r?4ru(C7}#GT1)407IUHwsq%Deh|JLS*if)2Eu+jfhmtxyC=|G1z zgFxcM->(w0lKT0irnNh31Nzg1|CmAsE(FY9XW$NdQ_^zDvlE&cT7LQftC1oqLCn-r zX^=4ZCB};GhgY>r>be9SV0pM$bKV$U%6y{HL#4k$uG9No-#E&eyNg(uaL145xzf!e zV~g&vlv>LbRk0=q<3UcLI8<>l>z8;bw8r8W%-)VMDjRIi`Xdf)S|bB~1^WYNW03nr zgXNoxCy*u(hS%~kB2Yai3e`j)F|<|$NrQX?c-Qx0%hk$JDrZNIHcn%D^dydWi4l%{Z!d188n(ro_*T{ZDc`FpC&D8?Nui0+d+U6H6r zfC4$lPQ5|6PgQN72CA??ob62KD-5`d)z-wN5~C9((9S1E#>5$SoVsKw&j%eiCg!Jj zxTvqU`J4xtSTsQUgAkG^$+3kFldj8YNs*;V=fn6@zLSyR&Su8w-=ICSJ%1s^a$y85 z9i7u2W_7|ZuV#!dCU;HT`^f?S34?P<&@Oz>*|_?qYGUQG*gOM& zvuv#@olj+=vTS;8?@=4!P|S&28~=Ojw!G#8N9Qk$8_myC)_y01zd8QZn;Fo0gAchf z{Ug?`8_FaD&oO*?Ms84grgjrD=_b{DwR>!~uHw40Rjr$QtrHEdT&$;HxH?dkeX;X; z($w=`EC1iPf{euXosu&kF-mImf5lgVecfARoChK^r6AL{S5=2hFsZ19@NpW!Rfx8X z!-#(&wN5xs2)%S*oeO!SNN0bap_^oB+s#QO?+X%40Uo8Xq@h80wo+8?^|qbn!xEYN zL8jaDUKoL#UBIIMN*qk-iAr1Km)MLI$xJGX2}mKm1Yoq~cuZZB9|`7J{yi@3W8=*r zfgSR2wbZV8ntE8vp5%G6A0=C7EaL?q?*%+IPN~;S+rkbwM>CQ-&YM__?gdx;o3a?m z;cVC-(o9i{yv^b3g7k!`ayfS@V3jVN4rwz=g8o{&MDC?HL=%69s8(5=8(DH%!?yS3L zCPWY{B#fI^y6fwl$3F02jQBnh4mR?#Ia5ddMe+Oc?@CPUT*YUw0uXo9^^?t!=Goub z1mjHovsU_fTcwCP=D)x0#196Aq1Nu2GUe=C1Y;>r%)lpc1;3>9CXa1VkC3j(9Gy8QbY(vQwUv#Y4qhmj?_5>OjQTJ z++W~C>8H0vZ>4|eQ`JEhGqdhIHtq3kow+`q6>CzcUY+V2Ep}aLV$y0Lb9Tio#T=hE z?iP6UHoH|oon3@z3iFL!2&*!tyV8uJeCapM=5IZts*a|G_rw5*BgqwMCM*L9ov&-i zIS^sNw2!(R0pxuLg#F#7qh7ZhMR}*hh$r}%tNnFlv6nH@Ab$R zxIACFGaycH#P9`j{pq=OOi_g3o!%t@pzFoAhj0W&dmF_WzIr()t?6{cOxx#26 zz)JWF$2;nF*(TOH?6Bb?BTZ!^G~YJRKzpgVnc#ifymVa=hYUTB3bhAi5v#d}zMYZX z^;kQLTvEp7xwstpg)O~Eo#MOCeIg?`bsc{kMMu&W6L+biIfQhR=5O_jV*#b}DGeDm zgJ2^3|E=|AK^DeT@q~EM)OHENY^ejL95$@U==ht^9tN(VcX-J^;@)11GC#!>U^HfH?H z!{4S1H@eeQ?+HW=IX?|KRkltIu&l9d^qAv%ra=bpW!j3fZOFsBYZ}2oDuSesLQJjw zb`6oTwwCuFTzC5zn-@vxq>%-zvm#8mE*UhwWC&Bg{ulJWXT1;RG>&S3edlH+x3x7T zGk3*ju>s`QEr6f%HhdzpR{wFb1&<-4?4p}Di2kuMw!Vgviw&q&@&M8)rf=-Qnrfa- zRm|i&irA=+%MCSpue^4KJdMa@MmQ6B4p%U*ZO6IPDZ(;mmmOQk-z>@=>6~puYBxjl z;hPmIG)Pay7P)V7f1)#hs7&T#6T*DN+jZ8>p(+BP98c3zG@T6%+t&mH=!p`b!$@(} z0kOf{45g8!tV!@I;SZhFRJsU~vrkgIyf__;P7GjIl|_h*4Oh4F*YDJi-~!ibnGb_$ zB63>g2m)=Fs&_5O96#?7zl?=92EN$D7D~!`hyrk2+>p+HuyPZj5(M%PmLTkV#SB8g zICj)yYt0ljNv9uvTw6` z2dK>EKUiUhyc02tH#o269TX5nfHkSjoT`?S@`=kXfAUFUK|~N~re+scn-Y_4ew1Gk zhSazfBIWl#CZD8$1U#fMtK@%wBM0SPuG_Euk{@|Rc5tDl?OxR$meJvn9z9O;=mSsR zO9HO_s=kyMheMB1FO8Up75*MzB?H>PU60S!Q};A`hJGHJLd8A7PC?vqRAq7SH;M_? z_CY)NY2(_YzIm$8R0<*3`!xM`Z#;vO z@mH?i4^3s^dO0p=))n+}V$g;Df(#;c0^$SIfALQQTa+{ZxrkD<;x0wbP&hm%U}*TH zD8vwvzSU6Cuv6$9S0|Id1+R#n(4E_{R!v}Mw zePk8)94HerCQO6QESpC7=0@Zk&ypLuR)I(uvF`27T7$_|R(4!*0rg*ri~tx_gB{7E;esp5nnsDn0wO-Jfl`4z#?k|<&AH&1Cz zf_1tc(fcpt^=})N$0HPGjsAzE)ISa#msPwHC7~R>Jrtgq9BdWV-LU}4`5Kr1IHKIy zul=DVzG1ICO-D>5t_^MBlP<~;hl`Z(ytV;__tiLm;aAs6A~u{EA3U`~3h58~)A}YD zH&3T~pnsvus{Gb+6ci*TY;-4Qddarym8QH8i&?W>snlky1Z`lx?{fHVejv>s#!wWF z$Yef08Ejfl1gW$-962BAO}Aj6xkqN%|EAl+1YLS%(hZ_`)!kF7#wxzP`r%TZ+<+yM zM~B1p%$Y=ZTq<4k5l`6CqqA1zu)ijE9$O99&gbmp#P#v4$o-@t8SAU>KSD(^uR#f| zNX9fcauN7k7%RnjmLN93#&!Z2c-a<{QBZ;43r2#1Pdj%t6l)vl4*e9-Ur|ZDQ`!WA zTj3eu-hjZN)bA@F$ZSVUu8SoDPP`%YykI5auB>@YePmF?Y;|0kUG2Iw!dY_k&afpY zYZfL?!hA?H6mS+LcH7FCeL1y5nxEuSNfP1GCnKit*E0qZ?|e>lgut^3E53oJ4n4n1<|TcACG`|Ov*5!A74 zHc$yLV#fpl!I!Zhp1DM)?@QbjPwWn(` z1ZTm@di`J<)hx}xtq0T|E$eY7r71#K|Dy?Oh`a;zHtkR>U4(p_$}tPB)LFlRzyu7gZsQnwm}*-Vyg&UOD}Ax=M;z}x zW`>PE_*!Ec?vXa_R@)+VJbW-Ej8Hh-NI$F{#;!0A7Q4)n z_V*p&Cj^fqDv?Gt&1!z$+V8_KN~eZL)m|NMbe(MMxL0gZu)3vE%eX1Bow6F#O1%v_gY0w7f zCUmTpS2;1)*olSOu<+|}f9{A0tmDOH^i}}TrpUP|P^IW!=H(RNJ}hjGn(dil2FzBM zOqNeMA}eN^zrm#<(W(NE1@l@lEVIe0mi*OeJmWga4tVEY!>)s3`rn!4O)WC{JCT@i zaw(; zdS|j)I;xoz!LWC;w{qLIYonMJ9Tn2x*PFMSn{eahtmjkb62tywcx2|wpy>M;wJpGp zVRyy806tZjHva<*YNmpkHt91_MCJ;$miH~9TOyf?)L;lw8X7k^)N``J6egAt& z5DJkfyePbA zZSEO^#?*1;j)eWcrXNEhjePqq|4E=4yTLC$43?M<9sP;jzM3+2tMB&67({T0Ba=*? z^+&x;O^P4tW`iNgX=v{aT6t3tE;Nj4%bp_j17J4IU$<7}VZzRn>J4lQeWawKzhBMU zdmX)9C5Jn=ec{Q|kj=cjzCeV!de%n*^RL`<_^aH@wTK)!z|Yx7awr z7}ZtaO0DR)Z_kf|6dPKXm5bzv*s_D56|I@=5ED zg2_85$=)a_??2u)Pz7k4Zd=M;$38k$EzQM%F)Y^;nFvTQeC$zw@<`9;LbsDk7NR)8 zM~pf6?wsn9>yjZ+8!`J!WTN!o^#btMIc=t-r6bN>x}UKxLDwa+*ABJ0J2@^rMzFXV zipGI4q;Su#Zv{6w#=<2ou$0|KBwHcj=Zk$9=*hvI>fM;Ffr|r_S@q<#u$J=lPXD zv|cPxypWxb#KW!N3<*Zq&u%;|Z4y}$OZ*byUJg4Z71nEBK7!ut(DFW-L65DWd+Qn_ z^3d#~4Wbe{F+14Q@{mSU{?R12{`~U&DDGA}bpgLRrOJ)WwlACz9xe5l`^StL8!58h z>u+k4q`lZ(p05T}1lO7(PNT+k2M`>!75}O(+kZ|z^9212H>pMTZ2p4vNV~`obQVxVygy^&NpPi%FWW`-D{aHm&R;Rs%03JXXD|ryJoXNY{yHJ6V;Jvp7}H& zhIunn(ocxEcHbgMq)rTk;K|}3Sb!?D)vooy<(|h9!(wa2Apd4uV^rkg#bUop2-(Gi zt>s&30+I0k;2uClBQYviHu&5g+E1} zFSoHFM^RD(S8AqI`AL$=7MA3T(kB^};%IfrS?D>7zkzdPj1ZMqf9=fGU0qRqx3fPn zN5HIoUY3hY)@?=^yWtn<&APra8Nz|%2=?Z&kRv|A%_F^!FOgM%)JA3YDF}U|yF6ivxeOOP?ly7?NwaR zHDj^2&O@p>i0K?r1$y;Vp{EFzEE`RWE$~&eK*D!_r@G|GZXRUQd-iKAN8BjH*KDRE zj*`rPxBSr55+<5s$jl1Om!FUoXVK=PuQH!9inLp9Z!UL)TPJgC;v}2C&--Aj zb(Rjdg&(Ha6$EFdMDd!=4$3_8&@hDk&SD_x_Ysnkwe%D0A=gOGur?W;p7rrbR1b+4 zw$7jr99K81p)eP%)iFenS*w6##;s*^bA)C^y*Wc!RuWu}`yO!2y{6kx*c@{5J_-;# zJh^9`sd23!tDgUK-XwkIyXFKTt6Jm#d{{8vclwgi8IZ1Z=*{LxF(OXV>iwOsK69+P zSEom?=TWa(kL4PFt{rO>PklucuL^(6s~f8%c)E0c;mhcB;V6MS zT~ObaSodmz$_VZ?!1s>RN6ReNnM6Iqb~*TZ`aDK~kPiy+&f)!SN}CIgEE~7mwL@0S zy)`cMI=>k{Z|8`&KmA$w0J2NJ)CeA%&Xf69sbUosSq96RB5oEqL)eC}1iauWs44&k z>dE~$2$qrc$djZdcO&y6b0YInbSK~Ft|l$z1hc_1o5*n?{o-U=qIl80q6X86KE>AL zz#FfHUoJnhl3=uoxGq67S0df`&$WCUWuJa@s^+y)Xqypy_ z+FKEy@7SDVMqjPBm&r-Z2`UUcy1b(Hm~7&;+!L<8o^EupYx1f#oz4!Dt0Rb|hmFX? zfRx6DrkUcDd-d~j9CJ@2pw_6tATvk*B29rX2BFDn=539K(U`#_Cd|%L$)H^0k0*=u z>y9}Vzo(#16@KEjEeT4IRf!Iae?SiLR<5GPhgxEO2k{NR%1QRFey1EsO7uI8EPsuh z_{aB-6S4lScfqO>Ul%~qxF!cCpwijkIQ-Q3>9B8z%3yZ=dbe50>ueaZu2tN}N3{bI zy=7erRt6I8QsqMut-=g#zD`W%3rU;!ea@}f_Yk$yIkVVAD9o0T2u*6+*-#&e&y3yU z3%x4^qx;*{A$CYiH&MnTZShwiVHkFNtsRQhGj5qO8|e7x3a_P^;Qk2fNwLsK#215t?p> zhJS8e>ZvQbhR}D8Bm*iJueVS(Vb0F&2?m z_ti7jIjcSr)0h--cL!OkdCsxV1i^r3lA z?SsZ{JJ5=L8}{86^;xcQNNupn>v0t_?^xL#EvD6t?CgB4N^0VR`AU2JG6HUBqXY`u!1_SAJ1)+~1^{v(piuM+n ze#WdXQwT^Zp^~_%%c`yjZGVe$9y!>fNklNh_}_=byJ1<;%mI$VHGg&(21BfKTXP1n zKV`HU{x_#XW~LTMiyPmL9-kV+U`K_2tP82JaewA{iskWuuA19>iC3g`d8qPhDdIEjqeR`_^5$k~4h2tW;wnm69*5nf-W!j$1r{T)2zgI*jQ3LTU zdqWc>S?C!a>n2&N&)j7)G`tzU^hrxvIaMIRPrN;Xzt6_MWy?=oQJ#?!^x=gaH6dN- z=f+$}JxLECs=#ZRn4!85(?ny@xE?A~{AMpoOtYRo+|dPcQr-4XYHMA~|>v5^pt?+Z*h?Xt8bs#X3Jiw1i8p_W@~=AwBIkzpuos?=KE z$ecVQNaG3XuSPo$rjWHi1MU75iu@;Bm~Z$}>qXjkaOz)0chC+H4`Mhh(_bnPhSsG% zFxxN8g^L@v$!UD6yvdjSPUvA4u;Uv1D~(*(V#_O@=A<@^=vqZ~-0qK7s23=ZM>Hwx z+JkGSRd+=rYedW~ssWxC>O}4TEPReKEu>^aPi=8Df&BuFEUAkO!Rrk%e^Ysgt%7|@ zI4HX9oV(suKtoo}l6cFH@9m&7a=u1?V{g0JxF~2v>*R#YWe|a;)84D$W)AvZ4;~fW zULj@UoL16voIeq_)|c8PNx`73)8|8W>8I0v7RiM)w(F6JDYmmEn&^@TR~9xcKOjWc zGt%dsaYRiu3_aMUrHmfh`-BW}oF~6iiF<^%kGVk%$~m&=&ZS?hdI15kI@W0+yJa!) z;g6^e`FyXE9!N@{&-L3vx)^9idJo$0esC`doIU*z6rk1l$RH0gFBAYqdg!t2N5uMx+c{WE1kh%~W zBZuSRCVse0;Q7dkz%$OC|1vzK*3CQAPY7m7@vax9SMte#HMhK+ELc=gT9Oq3FBeYo zo$@&d{&L&k@))qOOAYuj;_sa2)0h-E5}O<2Sska5(n^Inn87-9kS}qDw^(t1*Oc2& zl3IJ;332Vj`cva_Y*#ByTgZQ}woSGEusxFa#kKd6oU8ZL+>iCk2$p%@Ir?;l{FJyC z)7S&dPw10^;n6*IPAYyc6tMJF-|*wh-oy94=%IZtk~9Km9<5h)~yE%_oYgn^%In(lvM3jR*GCi=`&R ztx-8-SvA4@YMGkXC<1Z(s)yN*<%kMv$Oe0*K^ikp6|}**9M$b1QFCc0duiBQ8GHJV z@QCE~%wl@sR8bPQ@hV643A%Y#230Q|JjC=Dil*`O)XBN2DwzIjV4(kbhYFg6gZ61> zhg|%Su#A0%V_h?;uveO;=fXH5BjHH-3)q*-y@9`J)zJ9$RIPP_EAw`fa$DDKuF$-l zOpC+|-9=9$pOs8ZXD~G7w%~U?2u_xF^E%4Cy~326bJ-C!j*y%H*4Bl0S@4eaPAiwL z*2IX_o2vV(XHtIg)VFTbP^`pi5;|!AqJ_*o5LA>M*S7fXwmuGio^!7!^`OXn+^{g$ znQ%4PCQq6?;7?`ii9XVVY>$j%q6C3;j*W({iWS>MH(j;~PdZ&OTb6d7@l=x1>H&Tw zVt{#-4sMrz?DogznUI= z+9JZ3q7>{w`tahXUvWWTN>m*Dhjc8bi`}+FEz0sITgY*7_)?LWf*8UW$zqMf`>Yr}GK zLidlGv$Asvyia*f783DYu{!@Y`~I5(Zcfs$HuHBz4@5nGcy3CF#{sET zU~9YXynXh$$HN*3#FHgP36w4FzOk%YQgMm)c;%VH&t4rqt}6+V&RAD7A6e;!oqYRw z0f&_H;s31FJd`q@O@%ik`|?Na3XJ7)^D^GER=cd25*@k8#dI4RurspXr<-;I6Yy8J zzD;~BO6n39Vw-m!D8?9(p@ua61oaXGOHk_RuNYF3PgxCeUATm?*OylTYGS?vJFAB$ zy~#8bJ!autFz&B#r0L_D;c`hmR1dhPwt?LLw3|x5F%ixO%m1Rvqmx@+t7RsQf_X_W z>3Af1-WAgGRs)tt-mZH{!a_TOp&R~4%hn50{WT4Js!>D79%}8SIFatdt(*X}i4j?zQImL+X zuI9IR&DqIOlN*J^ObPEu3cK~;nGqt^6ibcu-0$5c+IcMGw?$gT^ieQOI^0`o(KerO zUHgCRy=PpL%ep?iR1{n&h=n2nqzEESL8&1TK>J)d`D5vcSA%qwzu(~br%GlhBWlbW6>P6@5 zsR@Wx_D%LRD&5AuUlp^Od?D*N`{C7`0xRXbg|BGm=$npk&}J7XQI-nvLpj}|>^`lD zE0rlzqdf*`j(_E+4XW1g5Nwx^Lacp-mU+TyZK7j~@d72(8?X+Ic_$#xjO}vkA}@H? zP++R-bB@bH&2G6^JqG;raD6aKR=fbnmw5YSb+Y%>x>jt;J&^9;nbcJQT{nfyBw2cd zjBE8($&m~UJ5?J?=9I{aoX`V5J7GW9_7a0mVCT(R8UpVow^{d(QZ4`0-C_;LzE4f- z<+6~Wh=q>}iE>dpkG)>AX`qetRHAC!-pLG$=-ojpupUphv2`9+8hl=7lFIt6F)KPK z%7P6&#C#R4f__4EJ#|o|LRA@W8$cFzO&A3jAQzfNR%t5CfCaRBHU0nS3C%Ea3;0=}IL>N{uh-pQ%=8wz#{E=jJ z``Z`ZrG#BU)%h!F$<;(p#I}9Bl4W~W1IxvT+`W?nWR=b|ou7P|!4M;9%MAav{zPMe zC+v$dhxnE4u!^8$>zk{0?Nj#-N{_}F;dn1k$3xw7fhG5M%g6aeyF|?bC97_;fP91o zqrTkG(lA!1D{z5&SB(c?Z@nt-^c@e}(2I!&IOs@kY8 z=Rb~7Bwz61o$JSSWT2ouTYd@((s$$vw~H#aKqdE;`YMdw0_TEZoNLdiFEq0H^UHKn z5k;DQ*#TDws8KX$Zr?GfyREW@mIcZ3wm}V5yh?6cd^Mb3Ib;%2dBnt*HZiu{y`svm zFVtKCD6WaEitus>I#7MIbsDq#u%}hz3(4T49q^{6v^H%?jh-xX@CQo00+<`I1=9PfaS;ySlTKy3u=+{&;#8R=ADTDqYv7 z26_xYsI;pt!6zkEkDfg#{1+tN*>3e&AI|p66_Kq`l{0}XAzYdw1!9_t*{2JVf=<{A zc4uEG;Dx+WCQ7k6Uvza-rbcxKccFmk4lDgnqT6GcrQGr=Gaa_s^3A!+L*w);(fVwv zde8J8hY3Hi)LS-Re0#c6HZZFF@)B+#<*7g?&0cRxjgLALpOORj&TUSQN+03m9$uTa z7vHmG7Xz*{E*o<&p9~e2oEKmE!5s21t7$q}i6xDTe?_^TFaaB5UU)c6<03mowUUD# zF`ygiL*;yl?<9qZ6BPybI(X^@w({-FnAiBZAjt#JvF8#Z2cscqC> znv`@1J2*J!6WC^>Qmo?Xq^Xf!xM8m`MpT7E4<|NCo*gaz*8)KqwKpF?|4v`~FE+ zBeUkDk$uq8&I7T{VRMmc6JoPHeD{-WiS@Bb;Ey?4wBmvV<;G^OB=}k@@P5KcC;-a3 zqqf5gy}ZM-BX7&jipHPkRN|7Y)waGhiS3hyitJRtU8C(OEAuht1rizNm5NS|!&dKz zy|8|Wa++r9*xrjT$$D=6mStm|%Mt>?u5jox(=*ilh3GUJ(%zVrRhb-^uafgb*Ung~t=X*l%egG==Ky2SWX;f)65Dx_~yEnaIH1#bq;5Zq13SA3mSo6?5>B3ag}Jc z)z}p@?p%Oe7KgmD(NMwC9+RVG=-Dbw2f;NDRsT-vffz_(9W9r=)Po|A+&>C
}kw&)VS?QSu$`{7=xKMu~k(hJSA7pvPZ_Q#LA@V8BQ1qOf?YvW17V^nP)@`2Y; zCI+UTk(sxeAyF?D-PAK6){A*6S}KuNQMo?5-`fO3c3XjWc%NoO>9T$^rnfKl9TrZS zzu1AVs@DP(U{aWMc*<6?D_&XVDFT#ZMX``s49%S@pF9IpY=K-E$WgBB~#K%4}(m3U)iW zXZm*51pg{e7*f${P4Q3)No77z-ejJ*7MuOhF!sj zW?o=#CFyERVEN5nL^^)Zy38wozbcs*mw4sV`0AVa{FCHHTn&TVSi;pX#@;l@$8azl zSn3ybj2+t=ZLZhwglVa8RtRx^M=532LS-N|+0erwnUN!)Ch1Z`6KGu>Gy&Lm_}vxV zaD_0%w%&0fp22SRSa@FP)1@6p?#=lAKtw1vf5=651~HIrRUGr-VKF(I5~g(W2UA=) zxxSs2Zj3TQ9%xl__Vl%)GWa##_TwpRdzmkx2;)k&u|5YtY60?2`1emG=86TCN=BJ} z^2yt9P4vxD!%p)s&<*E(UC=gDTqrEe?lD>dC5C$b>?NCqJ_UfkBn~N|^1EfE4|fuD zcYj)>m35nQr@~@(AA)!IVX+PzHsqqzjdiWfn`*`4u zUhsPAL$^_WeAGS?rxj=xH;i4KT026ENqmAK5r?>Y9!c&G1;|r*S%;ZwtBIZ1 z@m%B4nNqnu#|1JPXY~GX7`*e+kK$Leb9ehpQZK2`CB_>9BCl-xH03L^A=s1Zkbni5 zH31;Q6cNj&%#hfkWjNB}W!T$f;*X7%8t&mj#J`d>DSr5iju`@ticgSxn&N{tW%FZu zfF`*4LZ?5Z+o%P%MyZ!(Df&qP_rvGFv=#lx#k-u46lF!FXa=`BG8k&*7yfY3nfeEd zHleUV(O)S*DRqd_BmTO)E$}!&$w$!)d-pzAJl^)a9*~t6P_*%&FQpEZ23d=a~|@V z!x^9VBO+W?II~3a#TwTP$0loxco(Lp2TtGuuFYh3?rJF|q@3YHAiOIM@T;}BCkd?` zTj>_pr9x0f^hR>VX0-50zRw&nF&Q-uA5PAG{)Tfq_ql^3rzGP{_VC8WED16{{mPJC z9CE$pRO2ni&UDD2oOJ70wKckA^I~s#Ry8k`^-ldz88E+=gYl9P)3$cD{H(Vxdh>_R zsLFt^(98mYU36@_VH8@fiO+2=;6u&Rvp}?wZhx zS=FMU$o2}m9lTCe^;T90%B6JIs4qbK<%qHqwf@us|2L%9W*%jR8hmX|(8Xfo4AW9l zpg09@64YHSLi+A|Y)-yK$y3}xqc-lRXGASC4&GQ}In0Q@IbO11LlRhz0 z&Q#e@8k(A?p7tT>EAiZ^*q(FLqwQp5m9LgphfP!~cC}y6wnY_;{P}}bikc)3wXjCQ zwKA1+0RnD4xmjzinJ1}Vw}izvE#q$Zj%`*CBIwu-z5>u;DUrZIZlWs`mmIpC*%0^t7#9VoBub17-T4PDL0g;j%vk;AVg*BM)b*3oN|9kF+vjSYWuM_2S7%p_Y!}jy zHZr~PymXt>q>8z5Nhia6$dW;PiCz{%2FuIGJY?RJxh>8r;Qw^3lmac1+^&vQ9%l<_ z7xj3R1C*)pUHX&AGixC9%%@P*%%iNr$+4mM3g`|g)cZo) zyUpy{-97bD4g<}$45-?*dYEueNJL~S#|y`yOh@A`&ZnKD#;N_XK;eZMFU$KP@5bX+ zyki_=^j>y>%ee()9ZzvAtx1SAGC-E284cSH&l!s@@Kp!dm0@yB> zk0~F=@lXrIZ4vfLIV=6BN`=Qn_Y`Ht@>$hjcOzg98~Ck0HrSAXKs*ut9)D{(ZMt{b zZ+dQ;5(?n_&}=8=EKK$)$j1eF6M#{YVd>`A@N zX5_VFUa?ZB(r*-{XswKN3ps@xPw59~lcBbEPw`@Y z0+Omz!^O+DT}f0coM8*1y+7+VN}kiqr1$8_Q;H*~%h?Zi6W5`%hRRHObspR(3(=53 z%9m-X3g5av>KY}NQtn>EBE}Kn= z_&3%i)67&mF4o(&pj2yyo~lj^5_C6Uq*$@Laa#qJRG;_WqCc#7Fc-JGQzhylzTE}> z-)Ede#&%^IDo;yKvFFpHly^Qj|h&AD%8XMK$BY~*-j0Jq~%xS*f$x)wu?r_Xm0sE<2pK_?H^ov1n! zYsy&ad+O8>VUIU!N40Boy>r(ky5wiS2Q|W@=x>q2|5iV)O#U20;v)TB(upS;05+vx za020i3^``b0{)N`AixA*k?sgEgE?EeZd{jfLC}(qwyh?t1faRDAM;wJ*DSkC{Q%Xr zP?T#fRZ@tpKwR^cvdhYk)9Y|)fODB^6&pjqRNm!@>*M&G^aYGR9SR-HzYxC=O<(+? z&hPy_L^wJ=85*4d$ngs)KR^`lyU}Tz{xe#-J*G`N$%{5NHV(%$EI_Rpt=t^%PjEk9 zK03@g8lrnQZ)z?6e9y`(shNw}6(Ng(DcW{s6ao3X6DhtPP~Qj6!=&B58mk8u)ceZr z84{}XEQ`=j6O?X;1u?Oq6x_OI2C=m!HuTdRk zwZtJEnD&~798zKo^d#$ik3(B}|HAfO=m!P<@Ix1Qk|U0Cz`|FknHf3qd5^PLz9ZXM z^0)g~aMqztOn|g|;|~&~mT$Sk0~v!A!1By>b6RH4MRXcH8xuu)2xNNKs_>Hh;48gX z@NGXFs#t+~?WCuK4t>vLvH1*l8N68ZEX{h_dJE^3Hdw_2F1rLb{5AU<3RluV__v%5BM|yF1H&wDichnz4Smycz>v_gt2AxK zKQPrth?!dzYUsy;T<4|%?;Tg=#di4|wVv+UD#U$|f9{{39+Ju$ds=SiC9(TH?T)Ep zx!FO=*hl@AX>*Kqo$Yh;z&!ZQiD<4Q&xuMG1g?OMFLv}DjdjIzD&EjDC#UIJ-FUxu$#2N(hR5(`eS#lwsQ~SEv0T$Xe)rg3;|4pZ0$Dhz z>VX<7T}gg-+nuL16MW+wM>u+QoB0&|lNwkKp3ptxW3#2AJChu;vC;Bl-%Xw~LZ4JV zREe&92k=t;zyz@glFrm$ue_A%sRx=oBGn>qtKxRdZupPkD-oq)#lU1>DdR$8d4cCQ z0fwUF;CnW|dowqpxKt2(ymk5X%#ixT%(b{m(%{%2_!Hl9!s%0)MIE2zM>_5MzUa5y zLl*x3EC(q#;Rbm(r>JXsGH40CTTzuVtt)-8-8`5%0#v%o`Ms-o&nActL6cf<^0oU#ByXqgB=+>M7D@Mz@9KtVEX!&z;PvX^~}&m`LEU&9a9m+QauaG^&WQSxtDl81s;*~@C$^X7$MmQ8o|8S0gFwcxb}?w-n}k=!Ho=r zhkP4aBVL<=CMr!z`_7cqL@- zmEJ2)5kz_W=)`1MPQk8a|8_D!vFomtMoD*?abu-1VqLUAv_Yyo@e=BsXH3|}RO*I& zC+TUcfun4hFwgUz^7FWHV2=;(xNKk~xsqJxi6qwq>uhMJq)^Xh_ykdaEk_jEUX zSf?xViaQV0{O~e(r|A*S-s%8KG%Tn-kP-?DS*!o)SYoT&o0EQkzrG>Nk>csqS=4H- z3EvyUspV6CV<23OK|9<#E#>9qMhcU|4CYUI%Wc-e^`OLdgZ(=4wex0{UR%NP1<&oy zpGMiO=(C$#QAlk}l}J1K2>p1V2stq68cn(p5YP@!X?U-wCQ4boX?#lzWoLVpP22L3 zjBJQ3w6D|^T|n&4O*=$S+~M0bdllGR^IU}{^+lLJWv9FqI2HKO zyCwt0miS+h`wmFz;NguJTdytgaqP-re|-W8>w3bEa2w;=b#eG&x$p;4g@Nr}>zvRq ztM7QZJ)DD#k%=v`-0tem_1hC(8Yz(8QxV2{>Ge-;YSV9kdY0>}xU=J2Xg@8=2H$QNlI>ZCz5rD^E>`d<=H{{!^sm!rT?GIU#a>0$(i z)A~{|g310%U6d}K?zxTu$ebt_gY}jnoY@+w9zC@Fi|q;h&GmyTBPiXMG@PL)2Z@@X z-+rOEptz+7P{O4vO)IVponEPQb(?#K|IPM>Fj@BWYx0Ei*@u81v*D^Fm|+u{kQy1C-?Iqz#F{7A-S}s% z(%MJeYs9;>%6yJ4IGTmH>XTO;@U~{Pc5Fyl8tY)n(Q3q$oqOuC~2V zOiO|msu1+ej`vIIw=$3Rq(^dQV_k9p4ekL~(#c~G5nIT47;`Kk*n%Mx>J#Om;MATy zpIEx!cb;{L0Ny}2z#tui&7r?n+O5$y_8`;C8`F)GG~yGDx-b^y+8k73cQ~%rSpTw@ z)JOfEXx6Q*1G0UVF1d1=xCR35SbE-c<3+lBZj={y)j8)h>cryu z?K<(i@D){)9NQ3pq?EqM-I7OwNkhs?S+5v98t#=6w zfbx*MX4xb0)dV+^tBK&k(9o1aNm~+ydX|UEX8kcV z+AV}j_1vul1!hPkNLgNfbajt6z_K^C3oX1pQ1DmXlQ-)Tu>UhVi}P?%B> z?Mvx`tFJu&vN{)0M;`;W+0jaDw&+J_6SO=t;ZkRkI!{y?@P4aOJvjXnAU@q65Rh)^S73rXt#n~& z?#q&`uqIX{&epP_Up)g5o&F8bnEpoFIo4Qp!sbY*=$M3 z)q?AglEc?_$~Tp$fYtekr_32j;n=uv)51o%$;G){!_jO#D<}8DEFgETFc*Cy2wqZO zD%qT%&$BVWJi2Nv;HL&-vHH;DOU)QuSY16L=?rJfhEZe z6l_ZFVLz*;8$u}fA-~Vcc~kbA2}QUhb^0edYwQfB@|+5FKbH^dWH@?vu~>t@p)cuW zt4B#wWn!sa@b`r~cNiGLSktJJ2KO7h7v3ny-WqJKTWjC=xnc5&b+E_$wR+554L_xG zOqt1KzhJN;^br#kQz^lsEi`l{SQW!0xYEVQph146 zfeq^dshQk`n+hm8u$gxuITk2a*ncrhhr=PW=)Gqjnx zPx#zr-}@2eVwx-vn6PfZg`+R3^qJD7M_0p)kIZ^)DNMe{?@<( zC6^ zVBK>$vx#jdykc76uCuvPQsxS0?ixui4{@~w0JcZIMJBkXfGctC%x1%^rD0oPB#UOw zT`2rv{=se%iiUEq;MM(^m}NO}%ROI$m+RwlJ{#*KHHLmiVAIAurbucwmp#pDNNLR= z88Z>pXP>JtDm(7z0yU8;D4hpXiQ$HifoUJj$>s$i z!dZ5Ol|Uu7*?FHR!7RBJ&l-jwB?4OtNb?vHcvdj;m5a1d{t$UphxfHVd3nS?t8B6* zis2cJe{Yl1Xft=O%h$u4V)jc`Mqtxu4;#xKBHxF|zc|8wu~S@qj`XgOO2--BS6Lg| zeO^?vypXv+R{t;1EYJ3Y%A^5zD@zqiEi(wi5$e&c;Wi_wt>K1a5Irx#q1`PuX*LB2 zT0TlS&DCPn^MBY z_K1A7!jN(IDSfbVPv2c9-OD!j+se76!-t*IpPE<2f^@h2(8}E+?3TL|O;M5I={`S~?mH58yk6}R9!##;JS1FR*-yIR8`AB*uL`qTu4_%5T7oDM zf<=gJQB}cBRcd_f;X`-bxKTr6s$haCL!$tWtuG<(0GH)_u&SYJK|QKp5A0D31|#(; z-m@v5xIVPrGhiFpM{VXk>*`b1Lq? zQ|pSgS`=7Jl2FsN?u;}u?AxHJ<+{w&SoPdapQpPGzfLPXF-dZtyoFp>UqWb?A6 z6LPBwAT5MCSFHYFA0*3Z5x=8*@o|I7}Ujf3)d4X{#v zm{Qh@+{W||sM5JJ#WQ7y#+ynzyzkh`1P!y*`6q-3GxoW)6e-*WrikmNOkUd71Gd04 zOo#RaN}lFdPRyW5AGEke%Xy)a#^Y&Ej&f>AgURM&Q8Y+&J#flAe!R$>^zg;*b7B}v zcJuRLftj9MUl?>MhA3yzeKAAl;=s%iolI0{JEB@0741#7y*FPPi6s_WE?HP5pfkGi`E=ME1yg2C&<3TC(Ilkm+g{Js;faMICjsze@G6}+F3(%k~T}?#khoXYgmf}yvYQ-Olb&8eSv$=~gZpF6Q(Lt>V(&=gE zH31fnqw`i{qQI`$Vt&ohShUGr%dmK+Qi*YHJpVG!E z&ZHknz9H0S+H}|9SlR==UE{2p-u3_rHLfBmSj6VG!=q&7!Bg%o8sMk|DtqQi+}O(@ zJA_Z=l0Z`jd>ka*$h^mMhz9x1JtqPZ4*F*nZu0BzjQT%g?tDs)4m;u)HF>1-hJ{{rebgyHo8cU>I^YLOD$UH zHG7CsaJw4D4zrYQ%pl{7b7k+);x@~Ifk(@EP2~8>Tp(__8bAxSq`>_91({LYAuC4!LQkjx;#e9B3G#f7Bw zIb9z3Y=yZCW?#)QUwx>cxmpCD!9mCma?=&gu9ly7SJe`2oI?(i(AyG9Xncj>!CBxA zPb{$kut}+UsLCb_rB98Wt}oO_sZCR7jt=~5m!+tNNn=^Ngr!yE#WxDxm$Me7hc~LE z{QX57sq`GZ4i#&*Gb_T@n4^7xqnt?cy0JaSX9SwpIrDgFtrcrrf&8`zeO0*s zwEkadDKBZx&KW&UvdVgq*tODCPmrN0%F;%7At?z_56MeD7ukA>g)<`4^WxTayAO!- z=jjHBP2Kz6+CDq{K3uCPi@@J^IJ;|ReKj<)4Oe%{&|@$OI7(qi4?>hI_pF}~T#92h zqrJgVmsz)rX}t!_lto+QL&XD+rRtVSAnQXc*KoXz@SZ2!{B{5PVFy8~r0Ayo_8=X* z@$J1`8BZHTY3~n7xSs_zE7`ChZqJ$J730);_9g96%!MG^AoyVB(@bL*+c-=uCoVto zDAm=j*`jp1Zj?qug>W1rczMFMttY>(CNtn-J}+hl9Q9+M8vMVrJr3W8%7y9>O01Sa zAhI^OJ$Ru!2KH?d0rS+D#a_^~x8bnO*GxR`iGgr%M)Zx|ACS+)v8MsoHjqe!Pl;^? zdz_R2pX57&oUpsumx$ou*iBK{9j{0Any@Urupn=NVQJbSHZ#cMHCNq(%I1)zWR$}; z-FHt&CRBO;TL10WU_1{GCCk4y(GCAts|#`D+~^ufJ|k9(v&eHBh0y~(b1%n;XgO$wF~F{3aq zYHY(9=6{O48BQ5d*k$9Pxb>O)m>@g;I zF;R@`1J_4nmQ%U=UT?`Rq+tchZZi|)_N4Rf61ltw1A{+Obw3&qJf~ljN_ZOMEDw6d z_uv4l*W<7M-~+cIjyTR(it!^GyLmDV;R_Jq^ZldavTH9rpSk%@_Vfxvuh%S=EsYg$ zfi@9M`!G*hN1~(GE4x~S?R8v(&r6-s(6XIthIY{@jm2$p$}eO__?L`bGc^%5)nb^% zcGOA!cLzZmt-|G|TFu%hCHK`Qp4*GvoZ^}y_a>`>j3h6A>OKpJgU+B?2OxqCpa?e8 zBxwn(Ol4+proeC}YyFYpIxNPLwb~!yyXuTee#|a%o=Mh!e-2g`Q?b+S*WX-BR-@n> zN)H!a)vEbLsEkuuIkM&mhh?O*qwzDwzlOmWZSXJA)GO+<10}0Kzx||khWg0Q1mT|| zDl-DX4myQ4;x96{9sj_Cg_!OKy@T3s#yVP4y%}Xerx%f-|9^6!?3B?e42 zOrP<;j-h)i-5Yo94fmGX=Qi-)x;I?UxoHew%Rn_uA70({WTfE8cf_$sY`W)|kvtA+ zRVSMpjz%_?#<$;$;rVcn78$pr#kZMcA26tn%|2w0KhxXzk+283>EFz1OK!(Mm?>a6 z)^^^HJlhh4+ue05K!Q-CcKRG{eNS@pPcXApEvqszOs#lqPUEuiKnKNy=KsDge-nX` zIlg9{7UQE|N_heM=blHO3Q+OfbQ@yOrSrObabw1M9a|3Mk;GD7ZHR34@>GKy?xI>h zw#p@UvL0sa>3mSBhKH5#PVV{E`tKfL7y`82?oS{moE(z4ntI-z<<=2=dm;VKU6XlW zcQO(Pivyan#JszDt`hrD=}P9M|RbGuK)P=`)RfBy-J9WJKHLyHC)ilG#Z_l3~_2YdU@cD9w9}h zlfG|H1!Kh~SQV2u;IBxjJ_4zJ)C*bZ77c`!RGAHtlmAa8|Eb1*Jfur9ZCj(e2aR*j z{zudF6pxRA5unCVL?(1^g1ORjGOMEEsI0kop#j64sS17Rf$+QCne4LB;0$53hOaW5 zxIcx3iZu~9!2hLl|ETI8-7a7cXi9-n-7@)^|GifJqbTD8+5Z+$$@@z9;eYz^-+nL* zxns##)~@S^1pN=g@!z%h@8cpLz$lGn>Uk~vZ*}zV#r_)k*Vtcw_!M2;?|Dxs3F8nI}@Kd_{KbrIJ zcR`QuIP50F#DIKk|BIH7Gcig-E?g^!{P%18M=Sqtxc_gs|Eo3p4;J))*8RU&693P- z|2ON>|1IYKwpIS~V*Z^nS1^etFm`vg%aeQ-k#Y@kx~Zi)_sV5tiaRpX|Sb7PRKo zp*rFVT`L*jq>WyMHkmdVK%Nx~rsmX2+Y0N#u9RDiXJ?p9*ktWUTo7ZD7+raU=%s&I z_>yu$UuYlGF~60X31QeJXz`ZnB$`r=XbilE#gX7ENB|$lqe`ctuJ+cq+kh z?hZ_lusTwdk-XWoqeqxK!4B+)!RUvSYD9|MwpF9&l#Av})xuT2!`fIogX;%fwAl;r zsfau4Hft?9hn;#)Rc^VqdgZs1wx)cbs;E;cY57J3BeHzJ-TQ&RSym;&JH4! zyBVvj9|mTPr4fUy_D=*%qTmspY1>u@a&?P;a}`>zIGZ6-Yy|IoqA{|@kejuCRD>-) z|E`0*;(U(j&FA06T!!0<5wD;z)chA8jx2Qzk-tTqP2gKs=14b6+WE^e>!dhR^nTg_ zWj$=o8>F6Q>U}#M@X;`W)wZ?=?pC;1zx62B3b*=gUq7EX1iv9w>F&9fiHkfgjyw@1 z6@lNsh^Nhuf~A_AR@`O_3f8}Ky>Y1RVf)F5-A-uze!Q0d7|9FJ(YYUNh6&Dz`(&Lm zZ`zTSbC*ggn&W8F?O(?HUme!yQcKU&qHJXR(o`TDqQc#2@VxKB%F231T(UI z)u@Ey%{1qb_p=ff{%W=2A54zDT+lNnn}`EPe{lj@Wpf>A`?DQ@e8YWWJ||ednmK@% zg{A4?{?GD2XPIbh`a!%A+@Uh+$syUD-~IPdXaB?78086VzQ4RhXIG9dF5QTZtKmr< zY~_zT^5Tmowcu4OWyxCsGv8{m(tOOqy*H{BqG>CGFy#fegJM2~G#N^51*+FRE7$j@ z8Cu}hP0JL!i_t*m9>ofWJ4!6a3)qe!K3OJ6AY=BohX-c0R3eNog+#73rg8hDq9b6V zdld+e+&7kXONxd&liNCJ^WoJdyc@3?Cy@0)m%cT|@2hN)xA#?cYKLz7FLB5(A*&KE zGW}`qX==|@osQezA62cLZ)ai+qyvMvTsII08$_-PBy(-qQvtVyuf$YnU(#aq`aHTs z;?fcAa@%4lf5{(LbbkMvoB4>vguA%BCGa`zL%&I)OW~s)5DUW~q`nG>9dMXQKTxr3 zGmUv}THo_=Tfw>6<;gP2hRO?35WsEhkEo08oHg%Jbd_Uvy~>g<&vf1@Q$ErOfxHxg zg&bmmbg}aG0T~Bn%a(yZSf$s``^$Xz%F0(NimUTqcvXkAoiAFRGO3&UNi zt(q>8DqSsueki;$U!6JtEd|YhC{T&@D?4fR+MW8Pq@jY_h}Iah?&$qs-l$!p)j6Y` z*)Nmqm>+V0c#<`wHckdA?nzPD@|@q~#W%4UBu)M9@90Axmg7IjF{w5CQEdJ<+Z4Vt zTXSd~^=yijodoYzMx;11ap(73DU)GhdL$ZDx3W_>)V+YXZWdF&U+PGF;!|RLIohI^ za@IajbN`%AsA@tfP>TKxAQ8cLNj*{#fKPc?D7P?p_TmD>Q2=a65}sCmb1-nR&3=!AdZ`514j}k5Amulzg zR<7(5XUFc+&p5puq45ElPY=^9xYtmM72x$a6_%+7Og)yGyq3v8n~V{3!>XRD7;C`b zQCFDz-m~J$2~%TSAz}2&o#;V^4w-hv7c}XKwWiOc!^8RBzg06vY}-5=hO(+zaUN3J zuM85X4WdOTm(a(>{)!EJXJ`^I#t+?Xs4ntf4`%Cp0co7`qHC_S8}wuI>G?_?nVbPiSAW6@pcZkri;>ne6H--{S}&uny->(Ju{UQ}@+cx%GT zm-i<`a|U|;FZ&kb{1pPQe#+Gn^!*K!u8HHvsw(RR84b>nxB~_FzgwiSK{c@n5Tmf4Z0O&8Trty`HM|Y6C-*+)uFR z%VLul}QPHD)$U5}&=Q|ZJ84Fgh)KfkqE6)4ZfYhG<^=63y{YkZ1 zPC@4(-u(+ouKwhH)qSR{uJ@%j&J@%>huhZ|IlOp=Dp1?7~{FDgT^&6#=CL$ zyJd&t-!^iZJWc8i>9&*Y2CjC+D!|kF_d!#>z;+0Ma-FWbu(ty1*=&>B*XKx?uiOD1 zV#7KwHWxNO#5moCEA*yG{B4grf37QBNlZ;A_9hG9_%XHlNTi?&yCM{HGz>r4-y-aF zqx`2J{TRDmBlvBLK|zGh9G3VBa{E0XkLKI7*Z1*_-PNedZ(DvvQ$2?e(XBf@p6;9* zElxk8#jPe_E<4wVcW^wKvQg`k{9He)>uj^AMw8^{KOd967#`mB@*M^k`g?gC{9GU3 zzPjaU{BVBrL{A16?1#u|I3Mq;20nNv48XZmF&M6C5S_@ zr+nes+B8HVUX>;NcP6$*+vM$#obK11o6gHp(itwcH>K+@GexN!*L>ZmX-anKyuD4- zn7vl-oZ!B>eh=-)%*583Jj4&qKMJXQJL}o7rDDEmsQlNyBOm-Lesyv1wb)CLM7Yu>@Zr6ipPspPoo;7*;2N%AJ7PL5}U1#k3|6%vvS2UiT z7Z8PK9+%B)-yjFgZ0y_8%T++NeE#|P_`msIiX6Y9M;AWlv+yvny}xrZy}I-dSA^Un zcK((e`&N)Ii-4YlK>n0~i^-TvwZXwx1wwh_d>~Dm&^n%6&6lG!xmvjqzKSDe9@Zm= z(D~e_vTn~-whxKdtN~??9vysDA*k$e0RuI$?`hu_TdRl$CGLF2yw#=NP`LN{v#)FO z-<8QD_lZRt=~?!fiTq>DGL|BUsqScTbc@)&uF}oS5CMgqASdx7S?k2VJ6}q_njD`~ zq;;8z?cLjFL{w_?QQv-#S!AB?PYJyzv&cypj@2K}(gwE%zqlaQ0`GB*_7-RiwMZ9f zyk^=fowPN!SFzL6h;F{8ebje^gazoCK_>hkW5%LA=Y;Lz!K_@==^yu`ECJcw7tbDNzWYphZi?8X4-k{tF75%l3Scmqs*@B) z8KM!~+aF03sA(eIr*xw{_JUL=AJEI4a#Y7?ad{gfdlQW^M30SW<=&#`=&$f60uM9| zVWT*D75ypUwSPVRxtyzOs`OZQ)>G=oY497`rBfvN!u4v~`iuiBzpBzFii)1TDRkqE zwDD-y|TDdLtYfL7AmRP4KAgxYI|J zo$}_KH~z(*wWRC+A5ZTYmvkSuf0rZAvZC@#G! zv)pM$<)AV(HO0LJ_sqSwf*VCaL_ols^SZC=|Afc9;P?G~KjS#w2UfSNKaL!phqQaj zTE85#t@B^vCnJhhtuakFxnPu^J_@lKq;MI--Ab<>K4H!#A?-@dV74qtpH<6C$k@W> zvnt}H&_$id-67mXK&$@e!-jpRAy@P`8C+=D>Pf||j;~MFa@ogI3ct!nz}? zBU3!fnd|d#v+2^J@G5s{Pixe^w1OOJbLH-CGj5e&(+Pwg>Y-h1{xDIyXTNu^m64Rm zH&8v?$XciHe>At6%!xiF@pgShNSt$X#MS(wblu!>ix+>PdN5YLw*{TQa@ zG?}e_$(4E;;pMvEl!L8q7fYM`n|RSSob+&gL~`$AEvG*R^$Glq<~ZKt+o72^5x6?D z$8l+*UYU*PJZYf~s;u6&o%p9cmNXF36MK~3&Hhgb<0kswJL$}z<$fwZk4`g~yFKE> zZ!^2s_zh=mOB4{&O{rW;8{_w!%GKojYPQh5XCMC+a?=#Le`qWuTu{TV-{}XN!`hk< z9d51cnMdEI&sh?f^4yXF??-IU6z!6OW3PYS zi=#iO`qj#Zd=!(t#Jna~HD(`(Cb{^}Z{>YfTz3cUNAMl#fHLiRrSqs5G%0tZf)ty< zct`?PpGquh?Sq*t$h77@vfca_#t3IHWZ2dKx0E&ul6e~ylwt?eKr z*UBVua#s~WkMA#~{Ao3xHs`i;yH1D5S9ELqrFNbC55=WFqM|>E+?gtdA&6pn2gIW8 zuREwcq7Y7UPmwNok>kv^v|HC25pCSWeoZg;ODCbsh5>1e`=Jf1K9jLS)i{Y4IIqov zE*GfXw#SX|H}-8XlQd@##c_H}Cxr%CZEqV0D4geOrrC z?%lSig#}*7`@BLmggXAuImsE_FDhsJTu!*?HcG93J?*YKS9v6E>_>!sEZOlem$&8w zomlrv^vcL1g@$|~mqKea?12@bxJedg8*Q=ReN9l@N72xe7NTs$Yj zt-$4lf`E7Y$ZFP@lzUBIa~`!LEwAW_)_;{xw(TcF5SG;-30~LY5*iNnS>mAUk zEFRx%wjR`Q24SaD#X#ZE(`Cv7#dXRgINFTSx{e=*`S&i2u zvmlbtGToSI&O7`fISm3XgUSEk*m|^*|5#y#TyMAh??VU_DCga6tO_cs4-z^IsipU? zeRWFv%H^x@Dc1|bB@@Aka@n9mU-vy(V<$pciDOBZEol2Z!-DRaT4D7yV7YPXLn*G! zEJoI8|8{pGofo`O-kgJZvB$#hX$SM)Fjwd0_T&&u-Zp{5UZYfx3&x#XH{RID;7*U8 zU@;y|&(|>`7!P)K`C7KMF(>T0xjtjmzStc}wnCoidOKq909mZH6gsfbiE#q%%@aBN z%dZ4&$->!eW&HG?Xe@+Y0n=~56~byG?*E5#-EF*r?rV5Mg`sH!V`@G613-U;WBSx4 zv10y^5y4o2p@aP95W%5NKe4s5{+29L{Fwp(Zx(FBV`~!9xcn&A6fQp$XCq1hO*cdj z0xd?Zhx{eDwY!wyR2Zo0=|3g=-pd2@=)G%VJ1&YGW2V<9j!GtK6ck&-PtY#@?-Wiz zWF;b|NKO>|5``6|Z505@+zSZ&*17e;Ff~Z(+mm`e((_gj^--M~At-a#T~r%%>W(-? z;To~EoWNb@uc|qQQ)nXF-D!qP@z=ox)#su0hqU(QE%5!^`AVZL_^H^1;4<~L8S=$8 z|7_ei<3t7Z9_QjVbv1crHHtzs2E_koY++#JCU?A;3$2hI5MsHW0%^)zkk4f1Uxi3A zOw&?>dYutd8(LSBmQJrXexIMRWKA#kaR=-%tApRZQ8$-Fpv>z_NfFvaA(b~VGE<(O zyt=G!_D%%$5U$EL=GG##qd?QX#m0Ze{7GE%wB@^q|Iz|>Z`(p*qn2XRYgz?L4F2)# zZ}~VqR1es)YI*lRqzY^#)T5=wP(e=8e$_4u110bkFTUut@NVAEkq0xDdo7&aUOdLD zrfILEQ!$pk(&hO{yU`S;H`L6T-ocOTuD|6tG%5ju+^7l)CQpp_to;OCHZVl zwQFp+h`sLpkrs36rs2pE)TCaF=dGhe=qJ)NcGf*v4%X1S+>IccbdG!%gwARe0sGZP ziaDpIlZqnHHG`LKlgS6VVx!d^_2t`@JU&xmtq|Jm)V_R^Q$0Cgq)k@WE8CaOSE1~_ zICf2wbnB_&LPJ8+W($ZXLHMHroqo=Npr~=Xx>KD(+r%pbN7MgiTUeXdTTB>gHL;eX z3Ni`Q)&qM|ykMNhD|>X#=;xU*J+PdpjLQyJ`Q#DDEYD!QFo_};Fj1H?8zZqG3TNKf zF&_4(u$uVgT5MzeSst;QU&Yams*-R^7s{|E(g&{;ML9%dk4<=jV?xW*d#PvcCb{|>MHlnW7uyQVSiP+-F4mt!t2DUz>9F*oOR~r5xVQZcc(f z>+58kg<1XCnYqf`SKUaLU+{13X%gMgcZq7sFam@tj7wsM!)JEaN9rLwM)R0=4|F8Z z{f3LRTFclt*lXefoIY}X3TKX{M#1T~J_WHF0$wM?lE|xsAd_l6ALTeE)|VxN`U-CZ z0H$gy&d&WQd{5YVUec_~AX-n=&M!O6s7o?WWUlOibt6{&%kefDQQJ{6xiAx18(z;v(dj0g4>p|NM*Ng zafSY>2~SdfL0XNc?S-dR#NVxhnjkCvXRBLzwN^`%CdV2LU8lF*c8l_X zI?kyfFi}=YrC`HEiDX^z4TM1eg#0y_? zg6^?;oc=>31tY?(TBLa`6>p3*P~X5OB3M7_`K8;0cA!P@nPVG4h~eR~L8tXuQ!LxZ zeSs%J*&thq%3i8E@h^`iqr;CNv38n=1L;GbMR(f$niz<4}D=9oq)#dnbuL|!``CWMX z>zOQ@#>vjJ@4Mba+e_9G(J+c_m0iVy*)<8_RTo5G^4QreP6Pi0J zA)8>7 z+kdDO%%4K<2^Q=0*yxHQ*CE#Bo_0^97e=<&FKi#e+s{-~zS)lK>yEn>p>e(`CxH1} zY6emGFa=?1O>r)~D-Q}ScKxH}?Pq!ZrJm}}b0FgS*zV8Rr@$-dRh99_f6Fe++^tyr z@fv-6R-zdrzrS3sbM%G}SsTipi(^}Kah6R}h_{4vXrVxF~t@A6sC~o6MQ-ZY(ME{+a^!d)Ur2@pL&|UegB36 z$#RW`-rGT^!iVGkWL>x{r+>6P$a%@#%3i$qUvwTkKZ>x>Guji&@@j0=v+8Fb9~Gr+ zTOR>?4p&83eu^WJ6aL@x>h?d+>r1om7p^Hkq4|}Km$M?bb^q8#rX;P$<%oAUM5(!5(f@{ZQ=v5?~2dj64=Z0l%Gj?pe^dwu_j%9?$* zP-1tvpFWM~BRErxYD!IR`9*+2(z!T9c-U2Xy{(>u4_wl>eWaW*#^uQ)pX$(JnJ=x9 zX)7}y4jn3AJQ@B2PrvfjeQgKN*O-2n?y-X{X&jVaZo1iP^tzzzE?)oI$VNGE-}Eca zX6}tqE9TGDxN+^&ZQQex(t>_%8RnIMIowgSU+WvS)Vu!HFCo>XsvU4=H1C}>>f~D| zmybt>iYIP-3-Y)LFUd$Tmg)G|$_;A!Fu@z-3-!!TA@M;c-IoZu<`xit!(Cs;;Op;P zvvWP(q(j~?t*;&)yN9VHJLR9p#>z*~^i%WoxnX3{^^pgV61YJvht`SwaeE38|3N{U zYR>Ig)=U|6^v?57SKp$xZbgjFXN|;y*L457e(f%WbGwl}Q+a7o#Zh_C4kL#&LdZ61 zWU7)U(9Y?x?CpujGsXsEy3@Xnc)Ra90t0JYh`{D*YMg?@H}`c6Q}N?+4N?;wNQ)a~s-o%UucsFxBC& zzi#D{m!BtQuY7R7goiCeXn!3f+q$~!Q_>pW9+=M_q`q3%yj{yS05ddv`vjrJ`rJ#Y zxsB>frg^H6_kyC=zwaodLHD#>W3 zxuRX5jn~NSuMS=f?8e9Mx)MGBUQVrkF{Zxk(mV|8>ERaP zoi72MADAHuCBZ)xctqjqs^YmhS*;?ozzR}Z z3~dXF{6X@bvRMw0AW&Xv_>@siYC9Tf%AB>-Kj>3__Lo>?xOG3YmwWZ!tAMicQH=FC zw(aqPQF;1%%G_I^!LHUID1|1K8liX&wV50c8x_@GV(a@kjU#CYKdaX5vRj^~5{6#m z;bSbgiJiujTQ!8y`MwsBE;WHe8@EE&ad!LdWp9R=4tMQ3nu zwn8mu7ZY3)N#=-Ba!f0oyCCRZj|V33*&)KwLG~DHob$u0<5S*Cc(BV(7o+(MKxxdd zUG&^`eKUr3NR2jLV#P=PJ)Q~R#!Ta@#?)EtS`DR!Mba<`Jhh>|*XEub7Ts}&^Kxh1 z7~+%9T`wez)%eXB`dxi1uQNZQRHDIZa^$QYVLBb$`#d#+M+qFA)iW+oL}cxx#O+*$ zuTNRX*A6o47N0@OZbdL%CetmXJ!3Z1XPzeq+}+`nnQaMxqjU!YOT!e=@&aJKraAaPKuf^AE}#I`kwk2ehTRl3OB(x4Fk761)q0qy9^&Ibzar;d=*s`K3B#OyR@-@FT@_zD+ns9 z{lqE8a{r)xqv7=44ugk;%og?0K2d!MXp`I`3-r1FD@OJ*qo#P}BlwX&D=Ud~o1`ss z_xWFEBz|xa=uS|>LP|B7a<_KZg}LHU-RrMGYI6@pHm0_G-;__Ia-lUR4JE-fH{Tis z{7j8ZXJfNpYv{y6C?rGJ5es`63Q;96Un5y;>+Y&pYvjjg)2#6nrNENVf#)|#?`R2H z9YEPu3q}9{;i_)?(~Es$`f5v-Or&>4>4`Exz1hG3;9IO1PBI z&=2grCIqtbMt?S!^DVE{)W!UFp%UAiVTpW>-_;PL;^k^2GF8NxE^71VXphNW8=iAI z4*uKI`5};L8Mqds<`RdimmJwVwKHlm7L9NO2mC^@JpNi7Z+4;JzC}Ei^qQNxzPK{p z3Rr-<<2@l%*<_8425p&t3_lXE>C zP)o5bgUil;Ij6Bsx`p^?cP#^8iwvURfCXuHVxwk?x&4$h!0HBU`9WkPl9`fKSCq0% z`2dvWmR#|t{_x%rxYgPp)K1b$6>;JlY{PzhxCTS;LSF@(7UN27HQ8}q=Kk6(-*R=C zwdf!`3}bs{pbu8G&#?MiK7q2-^Dkf4cnFz!>M5F>?QKVO@YT#lR2vo_gBM{jla_NWHDZksJ7_kFzMI@e;Df&x~&YO zWy`l320LzqfAwp(RCzeG@pK3WTYXT*&o;)zX#}=q#bv;YsR7c>v>@ec{oc^^7C`)fzjH|$c&r^}l6bTTK`HFn7QO+< za#2@7&W4Hv?Fm2k5fn=Visl0;$uRnI*5<=l1U}&g<+4To-$Bnu3GJT3pLHCjBM$UNTxn+Pyt@YxHo+_YkSS_iM{*6n-5SAGsK)Q2%htVbR ziWTz*!85fxUHaQAte;v)U(N(#$_U&8-JAT2onrA55jMCE_VnhmX%~WS(SsjxE`rb` zH2tHY6aFeu78?uO9oZXr>t7K9)moD-AAPMyH`RqtiI< zi<0K9>MQZzxkmeXP-%gTuiZ$;)Dcl(n&pKukC+|al`o9#xIE6<3U)i(emyrzVV>FT zj~|eq;|Lu#Zd07yR4;fJemAR1At%`(JB&lm?vb;q1svgv=E$9K@H9Qz^0^6sJRPfq z^pXV9t9Off^oHl?OYzJ7zJb)nvjbvq)N{tN(>TNv+DAwE=~Jgd_v#2Vwbc>82QZ!d zp$EzEF5X(VD;6rxk^(`>aJiNt5!FDsg|JFIGtrUfw=3F1=mkzWp9oi*UxYE=u*F;u zfh_ryVPFk++kFx6+{5fmPZ&UVv@np~*K@^2Z=4&Y4_nAGRn1P;ccEvm%M)l0J2pPv z3}QF%^Xs$Gi5_()d19H(_1Pe%#s~M5)wz!_sOoM%hHz!~e&UE=uqtPVUq!F2>;+ft zN#^!Lxik``QAULG>ih0Ewxv(Ah9(c*jCJuC*2JHp@IG}XL96;(fk-qiiTY~2Kg*eO zIaz{Q=cHfN@+f7{mYRZ}dp7ygNg_T;7j&&so_V-Cti~Ej|4+=@D2smc|%6FB$FcfT71>9|3g+q4vb);`RL$CBHt= zjX7S5yxvYWH*2jW4T~ykUSdW%cdrc8c84*(FNQK4k5juaczT8_;Q|&CZ)RQZ#tL-& zd%S6pcxWLlXf-?E-u^9T+#74NHQ^J}li%HRtMR(AB1f=3pk_CLlS{bS#w`IdCQFJ4M3HLEfBY!$;Y6Dfi z_d$yN@BoxLB+;`+oyafV&nbu@y#pd*qQBJ9HB7ZT zzTD>*cpMr$Y*5W=yzh;mUV2jVmk#G=hF>Px(Hg7h$}yd#XX=MWUGv9D3&T)Xg;VZ? z&2XsORTuIFSBi**uBrRnfdOu8;{0z$Ob@|zYq5O#TdUN3<15lihPb>Qibajx$wku3Qy0wKIH~}mK{HStAhEmpC4cs{E`QL98eCs&1mH?lHhB7w z26g>#J<)4TZ}zsr_Eu2gM{KI-9#Q`I$@RCcR>8CBk9!Qj>yFXL{-1$Yr|(5AMz{=p z%PfqyiER7}X^yOJ#;koZ-pmTfcSAZ2H1qx26ZF)>7^}&E`l?f=uO@R4HWgGXa?>pb zSNP3W2ItMOBO&&~tsqbP37&7^KSeF;kLV+mrzK3K8_jGu*r{1wW=l=hKamh_p2SI&Ig|7 zSVef;Y9ED2rHJaiI8&`llght(;=M&Btusm)@)e}R+RZ1}VZrm&UJLhjM49LI8ay4| z=+tI|V;S`C`nsg0qn)Njv0ImrKSj8?8koG+UUW?ES|lLfc+AsAGRp`?s`8rqJ@vm+ z?0*FQ(*}vmUKbWJA3SXx>Sp{)f&V&DoIiNpS~EWRj%2*^@zytAuiXM|AIiKuB-GJ( zkX4ed?<@LtO7r81`F~V}UkgZteht5V{d{GL=xd?FvBJ8bk{k=r{w%V3KKExQj43?Zb)B6>66uwgbBgEEL48}DRVI}pYovskjuTSOOW!s90jtG4#{Rc$^((L7Vo~FJj>CExm0?wj?y#-80$(dqzVyA- zD#i5io^e6jcNqIGWz@^Kg>C?bBVO@}A`!A`Qdj04MEG?=mIZ4{a7tQr@BXZw6(Kx` z+M!j%?XD`%{n7-KlsfJoAx1m9SU+)m<^ZYBkaOwi<(H*xxL^FzIEtv55sLOOyNz9b z(hv9T^OyCi8QcuXK+#GImi#;6f%_w4=^t&|&(z-pX*G&3vTc!rAWjKfYI^vrAfjYx zt_t#s*>6O4OQb$us=ol8-W*Ub^zV7`=;evs_OjEsb^M(tsm{S8&wgOpA|v1Em2MMQ z(0D*EsHC@(1O#p9K9~l2d)&dwM!jX*y7q1?fw6>rtFrO|y?V(JG757~Yy4wUS2V54 zMn~5DjIifG`4hACeoV)9?dp(F^7v_Nl`H+)3xvI));=sT0QZZVu7I9zR^(DUUlN3x zJOfKj{k|)4m2`ULQ|v`t7GD<&=C-Xi0Y^Lp%4b7y>>K)CButs&wFc{8^~VXfCPe0z zY}TFKH8bi_ZdFSSIlH2wuCv5Tm5-;~&-R;+sZWkYg}<;Sx?49^7M;8L2A$Mr#Fk8? z{y>lY@$Bu?C2)oyIvZo6R*!9IC&eKTgCq~YqjC|wpLVqFFI8C7XnrQ%YL;3(ec7r( zPZiZWo&X@oPBy9>T-vm#yvJT~Y>rDNH2Xs4Y^tutIgb9(ItJekKk^E`$Dgv^E~@H= zehk0A_GW!6>jYvf1fr|UrSEN5jp~4ph&YZWRxyv2)xB#CieBFgE{=(!pFiqgx;X@+ zElWv0`(=zWt8Os~dU)V~asu;CVRG#+FuOfANnRu{Xgr(2Dy7aePJ_u;=c%94BmPh_ ziS(PAJ_r1DPl?se;>kWb#y+A>H95?8GW|0@nFk_b(qjaWljgXhlNP;!?IHfmjTHSo zI&80GtKOFf7MW}n)wM%+3>0%5ylwhnu8Bs0)@tG;aBO@#a(k}I^Lhf@`tpls7zaD_ zqH;lB8tU;{=5&IwOfxL#Em*4W@z_6=N^tz-0ttMH_tF0$=b4f@qaF8K#IGeiob;d= z;M!<_ZZ~>2aaN|zLa)+w3Ttq&YHXt73S|9gEH4$anQQ74a^%h)rR@mpT`7-TVH}(cE5* z$ubhnc?)h$POjQfohGJ&Us${1JL7=M#T{EPExu9Y1WN1oQS#lB^1YS%vtfR8;h82K zpT}^)H@p807F;p4Ng^Wg)h@_O8oEq>i6MSk`h!Z2WBkCXHpt!Qx5&PTgW4X`Mhx5EO@S=xK9OiKm%7qOF@I82A#9In zF?j{;1l7zCoJE_;82bF62i+X<2lYxu+u09fI%_>@M@lzH2CnL(?vqa13sWlP+pVaP zHLJ}kBtq5e8dX2sH+T*ZxNNS=KQT2xUSvBxZ0{2GLl)cUfa=@_6ivOmxht2jBD>E}0U+GxOXWc)Jr zirJcc?OX-Sm3d)%z(}>2+>qMU4}=qKS!c;TE>^0ThM1|AkH`IaccltSvjlxV!ubqE ziL@NUTIKL^c4}kM_`KjsxBBR7*u=S>1@p9x$*n3{0Z3&C*8RVddiq(ui<$8w_{gE;QCO+$=??sAs%j!W9H?4LXZTqfXIH_u6dm9`0=(w_Y_7eAlJmLv$N#F)!W75 zJNVGvadW(!o4~!mk~|pAtl;+1`jp)cQIA3MsOh~e{+?;U%tYCYPZawq%@E)wfV(}0 z-l^tK9O73MU2h*g$gXWTso`*J)`ss7tD)8Nuk*U*H?EWJKi1bOb9Uc;w+1hr8} z;2raH?+d}rd?9;B-QRB^(@FTL;=8MZ;;758IIWd$I6r#NOc|+P_Nx#@4`yXXUzyK| z(=83?O53b6Faun~$a0@~JDkV*tq-a1r34q3GvaNODM0ZzTOEc6IEqr{pHt=&TOoQ1 zO>+{}B|}27E&=QNhn#Wv;fb*Qx`VG&4gisU)9S72aWgigaEZrhOTLDPBuuEP__s3ox@;oJG zTQ_n5cMlPb5nT~zq4bEQjEhFEI~sb%z!e9=b$wpnuiHPF?#*jwc3br2F?@Qt`6gQA7>y{ zW0Km|muK$z6><3bi>07bM`L4PT3NBg@2E?cwf&Iy7yU-Y4eTHn=c;M4hIWM=AbR5g zh@X>nuUw^Hl<3gLWgT18LsZj?Pkk!E7qK^Pb0eh2O3aaEPEndIR%kE3X?NP4%{=O7 zSkhkd5@BovTykB>vOetou8jOjJKKcYb|_7&p=U0@^Fj6e&J8@PFqk6&lAF2~0#<&} z=bEQDRS`^T({m2lgkWdC{;X;4djkjL(8D*)3sIfA!o2%VhJ zE&1@HCZdJm6N?$o_I&$`{f0ohLx;TZQo=!=&RNF*(v1gwcbYJZyFH~~|5;_M#kPcf zp7307SKR1%Thb3Q)){`N+GFDlP_fFad3NY3oX)W%SnU->xVbh-RnNkpUr`XQ`FH$%~W#u)Xg({L_Kb-Z9YEVz3+ zn=@)LarFxrvXcD85rU`}({G<1>3V%Qm9bTlR%~w%ns}oZ+j~41XcZ|+4eOfw7~?7) zOFwU`f=`xG`nEJ|c|EQo6w>i^>1nbLILBLtP?Gsb}=bDRvr5J4I$X~hp z8cPC4LNN!!L#JN!0{8uMcyQhF*n_*Xr|UdzW-Y9QqEFNVFUq98)k}*$$RGiqGlIb_KDWRgjcO@j=aBV%0;f%S#ZtyZ zVu?`uP)Fw17E?w8h!hxJ%$ zWi`@os-dCi=Iw1@IXzd)i(A$6rU%?K`>RXf-|24-V>zzt4IM~{&Hdt3WTte_WTUq3 zWo*6M`NWrue?+9Lin3=$K7fL`g-YqX`U)RA0f$DUYeT~HZRihp!4K@W_qkz{y z={@sT_GkK6d|I#Gu(nyiR+;Em4Rc%hB^z9;FTZN&2ad)bkoUQ*k@BbHJ(l(B@FCNb5@npN+LjZqw+w z=e6yn1&hUix|=gh<$%wSfujnaaO|-Q$-bxus8~)&-`QQy+Wwr~q>bM@hO(Nqv$3d# zQ|MW8sNvGM)p|xw`4U8en22x@a!MK4`qK6)IEyPVDnRQ&tL&+Lr%`d7U&eX}!RZ0n znJy*j9o?5u8VJ034C*mv>s8Cq4&20WnvuRm-hP(~J?#Y~l{C?EbI-QJT#0sjE`GaW zkJ+muf7c%CBO9)NDWobHdbir>M7})2~lv}}!QVjDQK?d$H5Nj;u z(>X(YJoc~5K(?QP_*S1oGXT+IsPP8MfyBlJVJwjI=>}Z8EcV})9BlWLx6(UTajMHw|tl7zn0Ydwc$*@R5>NL09vjG4rPE;QdP&FsvoVl z3U)AWz1lCQb+7xC&V*mZ(1+UT2o<+ayrHlG=5<4&pY$j9`?O7UPPA%*h8=u+&aqY@ z9Gcs5WMQb)c2ffGIxdgBX|Xy-_SnBSx{q5dQbwM%-apdz4k`<*q?$?*FQ^doFI|Sm zKw?W>aVDg9JP9ceVfRtV=-WEyBD|Pc#;uRG)7S1`S8aRh&}Df`)x(P?*M*N~&xdr` zuKjT6j~$I&Uzb&)J3XELPh@s^&5jQh%~$YgKkhHuJS ztY-pz7P`z|V!}N&2-jO3S_G}b!1+lsGK2W46Pip|yOlBMHszJg>f0l_qDG9W3nu|n zPMvER#7p{MhL)`^HRA&MO>$SS;u^KH5(1|9f;w)GImCpYLoffU={i2@idx1av|iSn zZFQr{rm+4g{G#;*pb%?Y>6NiNSY?4f@Gf@9MrHKhE$*uON2OTexZT1Z8oL!r#n9`q zvm!3cqt0S3;t^~%Y*)>DIE3k5)tp-e5kE%Qx`n!#skGiJnB00-Wxs8G>eOyAjlkam z%ADPL%3g$f+?rUevm)K@3g#9#&L=gjo{yl7Ct&k7LtD-z7v9q63%DBr8>`#m@{g(E zx-H@rgK-ap<# zl0n6-Vg37&d0|=eZ@pJeYK{u;iHZid`*Jh9xhKfvVIaum?LUQC$F zI})hqGW_;hf$jd2UsGX;UIoVvA1b+jNvIO^h5r2K`{Dq*=h||<-=q^3Hh7ect9_!0tblo_r&v!?UcYwmND6rRy zQkLB4iCY4^Hoot1Hzc-NF41v``L3M3iQ8l2Z2Xqkq3#vojhsmR`iD=nzF9VyJRu0# zX|6rgsW|E+rYv1vAE`vg9A>+paHQGQH~PMk+Q~%u8Hj)BjjL*JwB<3}_sR&P=`aVz z&y#xb5vVa$nJs72nv#qSHq8RPZBgqk=j4$?Nc}t9e*b`Lzg2r#B-Wd=!-GhBenvWr zQXH1-IsKBu^Jmgg4R2=#{Hq@32mIcsU}4ZKj0`;JH16Pe1RWNL%z$A}TlDdUiYLBvQa?VN4e4}%@9K(t;?`fj}(KxzQ)m1FdTH8%5~!(ja(mX*){zUURy?}8!l zP73%F)>L|kuW`~f4z48po-9cVOUn&@$PNDJT#ic8B?9aZree@P+@l70_d_r$V;7Wq z&*(HC_mvu0etv(x;iPqz0is1ycS}^gW=W(p%LIQM7X1+cLH%b&LDrnw&C_!nl|pxJ z+*@_h98aXztDxe|Pm$+zsJ4N4u=_%*;|I8%$XfH4S1AR*<&HjbYA$86HPKs|wMUDn zulA_my*^fl+~8de{FYZS;ngF6&JZ4MnHX-omFM9Iw!dUEK2o^r@NaTLM9&W$64<)^ zk3Q{FYJUvbVF0`^_NVB76dl^g;yv_kIhq_;qE;pfvual%UdEM->~24MWxG1h(OFls z9y6o@ELv|u(vdMc(z!61j&*tbMW|L@k2Y5VZHS_lb6!#_Tl5e(YP2+y+Ox;$nc$VL zwmNA%cOQUA5BN(3ZbrEK*CyDHj7oO5xBAB$&a~Sp4EkG}Lv$R$>V!Gq==liK8kv{zQGROuGe?u3ETuu+xR%@~bVj-(6?w+X2= zA!C5BOvc<*Zr=*3^#~rL-<0fA@f%A_=Lflwr!20DCw_67{YX9N{wd2;_uM&ay5mFX z<^$(FyMO+wp+>u`0(agYVLc%q9Pe9lnn-D5-a4vE9~`&&+-Am3$lC4sjI1g9vlf7S z4d`BZU=1@VtHf0FHrjgMMBnbH2n=-oW16OvMJ-T?<2IllHpo{bp~rgyoKq;I^`uUj zxZ{ao(TSWl`m?IGlIbfO_oUocx`8L!a-f@GKzuYOnT8%;s+|OrlRrb(Dyx7)1YqT% z8OH^~u~9)nyX~^i>%AnP`)nuvFVN0eLDla)FwYq2QOaTg7SW*jQif9q^9aw{dPU!g zyhVYR;4DjkjXO|4?WBt2?yd5Qn%8M2ds>zx@vj~)pCH!lv ztGAMGwGQ(P)a4gFIy~`$X#bCI+h~&m*Bu)iQ za+ijXaN2nMrmO?bjAJSy)u2N5YEefx3DP!m*@O-(b)+CA^L4~jcBl3q7MzPz!TknD;|HhjB=OFqG^a3a)Ai5K=h59yW=NPmeUV{wCDYMg#HRT zhvp(QT&k1u-tDAUuOgT?Ye8#CmjyH zR=m)g94@X}0m0t-Pfc|3uxMgp6`1;TFv4-i?x>WQeo7{;p-oT>VSkRHp^FhIlY8`r zdFWmg%+W%C-RVlr)Cz;p?-HB6WMsg_*PV#a->}zhUwzpXYeeRR*Djfi@U~4ll_NV*m zqXT6fjdhQA>~xJ+%OxI(9d^ye91aI-Px<5I>iJ6Q77pk9BoL@e|_C-&l83f<5G&R<>aAuBK0h^bZPHLz*-vwFl-!_NC&jlBtlLPYGb84wC;}oO6^e2;DFx|Ix zz84n>^4=d-t~=T4V+6yxl>LOV9e<1?U5#WfOaE#x&~|M!(aU#+=~2wjPPlPO+HjLU zuQ0Yw?*Nm=no51!j`#;f7_;QrKb!&v;=+wI*6}ssQO`EmSNcxQ7y>Dz4{!Z369Eeg z0PA`+>4yYNxq{GZ3lwRe9#HcWM4S4Ba5C}keBD9^4(T=CLetxNjGD~0JS?)EeF9I6 z)Gsn3X-EyDzeJX4!6=>mkys zI*vYT>u&!M$Jh*sW3PNp8?7xg4CIDJ7109gvpn(`wHXdR&CVc>+e* z<2UG14rV?6E_gHs5fHxRk|tdd-pX$1UXXR7c3Xzq=cj5<@S4VDwnSd6nra3sDa`Zt z8YRXW>nAZoHp#(dDjxw6S)`1hn8f*KBddv*2&xIS`pcLqXX!T|uDVAE5vKZo?uN4h z0Jp=%{;b-rm(E@NV{`ehjlV6Pe42Cs-zQ6XneI@CdCj5rAHhbO5}iGBEF%;GuEjgF%#>Rs#YyJHgJ}cQ&q^zQqZ$WFMm)V%Md>0( zwZi2&(kXOvN;nl~@&EC3=Fw30;s1Y1B`GRTmd28V zBxGrfb&^W5RmzfOFeI6=4+g^%MIMZVkYy@cmW+KLvJGP!J7bL*+t|m9F=P9AzQ5o1 zch3FqedgTfp7Z`(_jSFm*Slx1i&-pGzZfrU{uQ4jS+`^N{4h>L(4-5!`~BZkl4V>-7n7{Xu_-wg5HSFwmHE@39h#{ou|fit4J?|vdL zHMT59lZ^<~|0+-fSxWF3q zG3rOzj;go9{jmcxV@aRe4_Y=~OtCE^ZMXh1X?60s*_4HsV!`kHY008l9b_GFfJlJ{+Kr%{$2I#5%bo?(qt#EoBmP} zzYTtHz+KFzvc(RRb^6h+bVGB|`)a;ND$*eG>28lS3WZh)$7ck~`b0VvIC+`h#XKYwXQNqUeCgw69KBiW2N<3G;< zHFibv$n-i@qfQwU2<`V&b81gXDfQVGh&YBbwX=jb)r+nz74T`MwE`B7TP_HY%^RJ0 zGSd4@3bYNTe8BD4Ek1m3;7vpX$M77l-&#El4Qsd(OMhv}rk$1sev1zN{T16RyGR?b zUw*R+Wl0Bra3A2$`2B>0$R5y@3(ogT5nW2CXWL7JWEbU*xHD(7V_}zhie$P3ZniYo zEq~R#lSh#(q+Ng>bZ!0-7*h7Cr(QQDWMbIaqUwWE$np=^&iKRzs<~-r%futqYQ^yr z>-hz{a}8`<^xJ`T>mMQhyNFMy*;(!5HXaY@hSD31=6FJq&32Se7It{rb~i7&L3%3s zFWi7p%v5lsp7j20t=Y4UuXc#evgEc|mmdN^ardoCYC9hrF?IJ)7xqA5_kR1j48->t zWBWtUkA2!Lg{{+$RdF?|?sV~&5vZxrFZ)0xbFbB_ERhNF+z-#F^NxwaEgwY84bRj$ zk)=fU+KM>aJg^_M882p@gC`M6SBmNg zp8~f*SJ(IdWQWgB@j=44_l4a4jH;AyD;x!Olw)>ZzcNXWl4jW|;y6w4F7*!Wpos%_ z0CX&Vk8$NTbMv=1jkkSWk)Gj;rYz_g!a{az-&nP>bZG_e{mVR>RA$P>O&1 zwrS8duZ){N!g(l5xfGxoPKHsi2y9=w%luf+Q@q`kezq5^1QR~ z1b=pC`w>moBUg2_YbIaMEB%N>tXA}@7&qsIeB@mH;i#Ok85rb4c$iaqg7(4|p;qJx zeWo#-fnsFO-SAYlYIe~FTwd+4TeaiWOaDURi0)k6wqJ#UJ;h;mKOos))Ui6KG8ulP zvD=NM^k+M5jYFDo(H~NwK-}n-FP3~fO04l{z6AexB1n8_^n%A(v;8Z^{W6wIpE3wv z%K1+0ww%o<4%_%~%0RGFmq`Oi{HwRhr@Gm?&!rcmHlp~)-95;`0La>mIowg^m7M7? zrG1J~xOH%~jL^<$_WV(uIQ=uIy^_hMSqZ`Wlxm3JmSK1{Ll8 zBO3G0zj<*GBam&K>r6g1t){TIAb|I8Ljcl+q&qicR@4FJNwvJ{(^sU2V<26-T_GK& zD}Hx*qQ0Y22;;GPn|%tEj`Gd9bLjo~(fr0wj;5ff6$A{TZxc1=Zs8JZ%^9iM$!V@p zff?sZ5g?h87(-xltqSmJy`hG~y`3i;Y;JSt_fs0aYBo%~MD^MgzXM@U1=jL8J>D+U zdO?Ppq0*VSIJAKn`@XLg3p{U~gb}9RDk^+s{yNp?>;Gd_>e&5z@3oHU179_P_h&Uv zi=t0QpIJE9YIx@~N=h+QSEm^F*Gsig47_pmMNw$-+un+E_u5_01nT^z+s+FyL+|S3 z-`j6yv9Q-qpRfG-`|Q6IKF4bL8r7a$n~AZadk<}Oy2|Un;Sr39_!IK9SVkQLC9)zF zv2zL;DTh^dB}dN89jeri`7v!(f(6 zrzzO%Lf;1w9#658qIb`3U*>|IU28R&i3tqt}NMiXxwc_}P~JUEB%*&-EDCqQN+& z=bmPyJk4E;_>h{+ZYJJk8V)d; zx~!@tu7KZO4ru?t>awN9(2fJTM5$FVECbuMFZJWz#;SK;C&3((+P2x&~z@zoyE;IT~F|jbtpfB6a^gHBmbr|(2c&}VdUOqm3194$y zvxt4qb}=!(ffh$gN=tN}G}k^jE@{3h+*}OE{>*_ccur7UA9kNG$~Zouwf>f(e$NbR z{ctDm&@6C=dfS@O&nmrv*-4723VK8W9hJYS|J4u;Ut)sX;_a(EFAKCK-C^9%zRRDy z*+p$P*ZvUFCR@50LCso`+0e5zCM#wh%wp}|nD*AJfZhRoW;5>IC!g%? zk5GJXY!@3ve_{P|Yx&OhQ)>y0!`ag~SHE(rQBQ+C$@UIECgH(r=GK5_EeLC@StjpT zIAk(edYB^&X>$Lv@$yhb%A(I@ImOsc{L60vf3>2!uN6;C7el-~16PEK84q=w9%)_4 zDe8Z@te4s*Q~M%GKy~TKt*OsZnv`1&9bhB62AaEbHh!hrsfE+)%><67wrYO`GR7-E z8wF1fc6k<(bD*(=9JbjD8!7uaX(7Ge?cBB&yxSI1&1>a(&URDCr+X#=H^2?KgZ3-V z7DIF{W;{qj^q|z&>)(c;O*BA`8()L=&)YtE837&5A*Yx6f7hDN&8r%u{5J5KJoJs$ zI996(eLk}riYrPo8oboKdpU4%vw(pD8Lkzub7QHYoYwYl+5T%SW($*~U|aPhVosqE zuE4_&lH!t2if)AvK!6R$C3Je*{{}=EF*Cbj# z;{S!7$Tay7*a3C0`^}%m6?!_@_RmiO`p={3ji5!1gCtI=j8IXh(|5>S*P*|a&Rj>I z7K8p(n);D)>A}lu^Yy@a>R9ks>NK5A4=M~f_RjiQv+{q{LKdNx)!Gxb7PJ)Bf5abD z|5@~NqXw!RwCUApJ*WtIOVCd4T=21jE8F(#<$K(7av%HRwQTx-1&ON2ct0QP$ZMg$ zGdA>xZRXwh#_ZAv66l+g-TOxtm2&EjmdHSboo@n{bV6B@-}#PtZ~5I{l&?WWMk`Gk zjB0HhSPqLW37V?%LsbP*i`eZ!==PQI;^WS7#m_kb+3JmVio%{&V!LOXG{E-vuPF%M zdZ`7SKoA?&N4Na3=y@S01a4EdrXXXhzH>r=hkUp8hBC`QfPC<j7Q5DP{g!Iy(@~yMh`CnRxad}XWu@XzME9?YAps%7tW8_vDsIJ8 zh+AMES$sBm>@$dnKjIIOjg9z=Gsb=BS*aO{{-)beAsf2aORf})FRhFVgrX+=;T6Yg z)uD|SIg>dBbz3=U|MslTienBtq45|)s_K_A7hWROo-e3dp;dV99AYFHo7S`6@Bf!> zy~yCrsufTV#E8u41a}(kd}%o*BR%YCZ%V$X)|(Djc>br)o;BDRyT%NUUBXbw_63WL zcDHuI69eoYh-JGnTm7M4#7PDTQU%W4Io?%2ocjmdCY{F}0z)&E-4ZhLjl8awbXl-q*btvPEj_nPmHyV|UU^fBc1h9t zxJ=6M@578wyIcRfK)Y=qxl+1UWG)J{^tBW;tLapR8_aa&2+Tj@MNJ4B)WyU*bYoF# zt_SaaDR9vA_MiU>+te5B+kyurB>C-XZC`6R&op`dbq@KIlr-F8{&AR8jZ{yCBrJvZ zj1Ml?Y%j7cIv1@B6nkG-zdgo~hykk1nn%f{uBMhlR%`_x5JZLdWw0U51&ZCK=ZD-1 zQ(yV2CCrYhwURKlYy+CgaVNPCg z?AUQrw?>OuZ|X7EiPj1N7_#b`EBAX|$kRr``6Lwb6^>bk0v>HB=38SqUL^j839D|d z!F%n+u{xgLbx0qMx>4O=-eB#bASC~4_UB;{b?m$nH!ueU0q{nu=rWBy)L z@5(De^A77mNj|82&X*M}eFrbnkU>UXQ+o}rytUy)5_VbG&^eu*z$sSSBSYN}x z_QR}&G>-N#s7T#j-o=$}^)ko*cZRT-<5Us-`QBQ?M#~`~Q&UaP`bqEqI<6@Hg(>ZP zmu*V7GxLz%R6ff-oy(Q2UwNGBA#TV#(09jQJ<;WOe8pa za~e^vM*v}MV)4+?;kB=XPg*G+e7QDdKSP=Ra^ME?acw|b=FV6j=KR^n#^swx@Sq7# z+JU=j5hjd+P48d8dEr<>O~aUQl4g`+>KBm}q1gF?$_`Vn&m~kzWihxn?m2@yqoObS z{9dI;)8T9vwvP)<=!(PFSA9j1aFvTcoGC)_BrV99AZS5w#{(y(wMc|75)- zr`fL1c_?g!Hq?OL`*7u`!wS@uZ;L8jF2fOIH2*j>y&6!^J|QY*bM$7XFN~LQIqJ`U>KG*hyoy|$+IvC(ZnGaM^d256PARgyO)9Ti(ktm0x&%V3b z$UpyN0^7%1#RBzh9Dn`tKc(SSy>V4I9Xprnd)u(Neti8xdTEGV#m&e@kG~rKoP*Tn z@2>-*mLk&UWDRo(cP9lEY76m| zKdMgDWMhM?hpOkRpWFChi^Z`MBjM7eq6~0qA>=moFQNp~3S_~UJnFq|8Ky06TY#f#O&Hj1Uyy`btExm#!zb|6rIh-!dO{jl}vZ|tt1Ukb_+kixmekc;>z z5?xJ|QvfUBBI3CfHis7dnHW98*|xgK`u4E5RgQ&Fs)K2s*a4nB&l=eCrCh;jN=&HL zE`@EE&vp3n$cjytI-#a_83oe82wE3p->VeD9Jx2%vb!!~x9>OQ5%EuO3|Bb1FmOE! zgd6ZHF|^v7f4aemdL-6A4a_;qj`6+T1q#C*woI2#ja6EL{bQ?hj-%hx6b(}Dk9^0c zDK@$)NtoL(X^Lz*%0PoT-!+v}UHdX6aMtVs>kknf-wA0bShcjR> zYue9`scUrdd?R0SGoQ8R-UwZ$U2IV zbQQK9a#)oMcQexUYf83@{(yP6lmKxSn~bUlW(V<{HZ?c z1MTHmOXT|4mc@H?TMGVsb3nDfD`FYh6lm$afP9uJ+_@o{A;I^v{tu3lz4e*ju$p3; zLHH24{_?z(oA9Xco8+%|8uz9)G8!UVzjy1riy>r*YklQH)g^;G$1STh z54x94;tXONzro7^5r3l$$BF`rZ?Y(_o(eb_<))}e2U>;G9q+k$ecwDB5dO~gx1JNgk zD89CT%-7XG#fQNf3mNZtgY2G|NTaNSd6Hv;C=!E2;w#O-g$c3$EKs=*CACvS*;fsO zg>-@0xnO|mqyCpjI>Db$XxsDgfoB(JL38IW%j+iC%WFDo@VPL_=Vgv7+k>h$cD81` z&$|*Ie&_MWseDW=7yrjJ6FfhWRI$~j>S_}YAjQR(X#yYYru+eECFY96*!TL(3E%Pk zn0X{P1a$=B*H*srsCD>gW#p#H9mbo82p8$qHCaO_ly*w`@wZs(dyBO&$<%=}tI@h& zg#r6`k4vJ8hn656Tklw%5ivGeNnBj6tjUegwZp*D zabG$5O~nwq3vchUn?kS|b+0FA9ROSX)Uw*b0+WF>1P_GmpiU^Cr;P00B`URrGIDb>f~gd%BV=tuN`e--F`Max7phHP)S{%e z)35YG!O)B?c67e5p~~xV{8wV#2@_w~f&I2--D`nJsF?Qoo}?>2qn`|K6V;_Aka;TSYH7<`ov@<=nEXQ{AmPz$< zMl0fZH-u2e9sA*Ol1ou_g}N2w`NPWu>`R zUTbXOq;jpUZ?%hox1tJHO#-(0;R$NdnhUD1!~=9wpyIZ+%Tio}gSCAG{b_@G4RB#e z4>nsn-UV{Liq)%}u+9~8RfXHTxeUevA5Yl9gP6B{ozD*zWe^u-2g**Y$>CJilSXP* z`|I0x6ByZL!C(laG>^r8-r$+`BzT$6-+tw-;T2J*vU%GR*YriYT!ssR|W!z^>h&VEg#HDGe$cCT!wmFr$C8+=?wt znkede?p#XxSNCv<83j3J{5CfcW~UEEgxE7qHyBM4;><-z z)TmWDVGx<+&mM@exlPL3Hwd^1mFb3`&?DPZ6^OuKC+{=tEU;6W=tOY6?Lap1KQ;b# zsz~acK*$r1yDK$GuaNJ)CV$;Tmf-t0QuHGP%DHs-mHEy1D3UE`BIXc#5PpMANR;M|qP)r+!U3Z^u!>MowsORfrBl~_!j%HYq0 zpQr!U7%AeSxLU0d7b5C)?Q7_7x+i0aZ@mYN}CQY`|O->z%9rtT?Y!=pwDu(|Sy+;eNHDs=&TU zV4tp9|A~>S#YDNYt1S_^Y0j{JMC6vU zr=r0FWqfd>GnPn5oLssy>`~z7Hqo(2-NTwGg+x`aafbTNodu}tbfiyi?xE}LGJRu= zX#4YxvS>`hL9RU5isY2`n5}8+|8qrfNV0u*UvpGp&dVUqmVz4sl>tVX*ebz~yX=yfh z_<&&ifLP)ctOKfLwSlL|8a);s&&%aapTeMLYfFf`Lfmg(Nsa#mpa4PxXV` z{pA`Y=wRG&CR%z<$>jH|Jp-t9iE)28%&cyk6EyZ2*w_&m_N#B!_5;F@l%W0EW|4yn z)l>Zzp4gCEWU=gwAMej7X&Sqv@#0G@l4$qVW@$QbIOs7%+x=GiRr;#3j23#&fzZr|0!ceG0$S&F+VM`qa~U&@R|mg>!u{O;X=ux zO+=23sc|nQH<*l=Yxib1?VgYD3?}r_A<%z7jDD;LUKi0Mz5#He7ND0DJHZRs4{FurS<8hrCl}Gw5Ql-6j{Sx(R=@8#?w{+PU#NEBSXKv#*#rL}oK}aucdz4z5cY#!Q~RnzXc!$IdY);K zeq~c@BS<1Z23qfn*-|;shrHxD9QT~Yl-sa>?L)6=1mq{(=|TE!YOGX^2yO)q#>xx2 z^0MPl-giffPnZf@HJbu*j22ht34sqZ-V!C(VXrJZh1OoM$MW_m*s{TBq_^>IH;cH@ zVrsM0-Al@(4MA)OxzYd|nr!?fp14tOu7rF4UH{d;K)dUI05qxa;4hP>+M`CVQ^XEu z#oR*1FkA^--y=g$S4Sr3i#mUF?@Jx)aDSd)7snu&1k)(85LTy339VVR#!^*BzX1wP z{g!-9%^G^8UHwZ2iLuV{%&^^Vz)`8(prf_-L0o*xjowq{foG-6#tAv`w?qBnLf*J2 zHb3pS;hhv{ZFo^?gx1oHH=I$rg`SXel*cWH6ekHDWQP-E{EbhsY!vML&=pR&I8mT+w@v?x;z>P} z#JsB*^c*gR(VI7_K80sLDf2PG)7=r%K-ew6^z%-GmIpQkfI8ou^}!Ij4x=@dPxt5v zz@zq-dBDLa2?cwdw-HW86cv{xn}S#NoekqTm`Hwy`({+CsJB{>c$Y#9LMt1^Ei*eqr(g z%%m&3%-jp{I?${9{zcr5Hs#thV#|OsEJlu+$(wzFu7k1#_b<#1xyW=gjMVCOQnMhp z$VwSDFNny)GM`DHmWE%nrc&_L5E2o%ohpf1x~P@LfO@N~t=J@#KxUZoo7xVe?5q;& z8kp1c;_?d0L={@PS1-Ht9USk}>&VmEplQ(IZ-`G8<0x&d8%bKq3sOI~s^T5h;eEKq z?MGGD7Q-rDQzfS`*9HPRtUtSTsxlSjqc_A4G+={nvq95)&xl>M#s`GPj}5f98#BoH z4a3HLy$obVrGwS9$XF>aT;^8|Si{a#(8YNr$opgjFpl|Y20%@@7+fPpgtd=<=t(x8 z{c#v1O#NAuT)pEaxMxgxAIHoKJ1EoD=Jl4?5G8g`bzE~Dy%_k^tkYAgmz%{`FRdhP z@m1h6pXGS!ZLhK4AhDfwW(GvSqieUkYi|T}eAmP<_Eu4F6r$u~HV}a6C;3xSSaqI{ z?8=fiT`b9hVlnBv2B}-U$+_$IDW8#XdotUSM|z`CRGAQ}rn!JZV{s?{WI-=DGI2ldIcL4;p*#HPpET9LM%NGu^#YSwiNs+AvJqzEht!q2M`gQ ztx$G%MTE?`N1F`#eRskUx~20eh4#D`xk0F1y;pVoJ769>*thBR4Pe+C6@kF~iS0DGGcbEOf#mPDgF?ayH*%;6I*Uu530w1Y?SzL3T+CyuV#xD`RYy`~w zqHn^|4@ysy^K{e4z97$9dCOtd&#o_j{zngH%Z_%5#Kbxe#IoY$7t3=GedAb<#wEPG zjY9@F7Y+!@_NykAje(*Q9lKdY+wApPd8owx+2myI0iwlh_rjgKQcl5=xL`vj?90`Z ze$!;{(ISJ%(jg&oUb%eLYs$GpQ&|w-u>R()BNJk zV_;MB-KGa-0WQ|=fXyz-O+F2Hoc=d1KIzbhMGw0Wyt`jjtiGkR){O#%WtO>*tzw?% zL8--^U6l+z?)!XiVRF^VGKpqZuO2LO50s$b8QNJJETm1;Nw|u;x6)GEBT&EGt8%fx zxd*;xmHK-zi7kUU=e&B)QF=LW^DjdJ@-b`=*sYl+{7|}0&FdMuInb6qdL;L_r0L6a z5d+D{sS0#@ys`SUp*A0*u14tx`qb30@4YO(_!;fC{b?V|Iv6!n?^lZ&+su?skpLrt zCnlzWa{nrRn{hfr&QrWB&HWb2ab~q&$f}ZjRl;y7&F<1-imzQJH80(v%kf{RH|)Ks zArdxp_Y59hO6ajj(Ec8DzN1aZo7Hhe6=MBRVX-xa2(+D93nL|~#MOOy=eK%UqVNvU z;+P}6Vzcpe73?b>!z+*-!je$HXg_WJEaz}fRBHVGp>V(ABBEA4+hn|7o6g}h&@vXs zl*v62*#BtN)+$FUoguQZ1Sn{biD8#=@pyQ(mMM6h+PJXc35^VvTf*X{oSjN|=Timx zqSgfCJM00xMX)$)PazH85)-m{8p&Q!X5dKbWb$s{e6rWgx4dg#fhtmhreirOD}wl) za=yJjH0-feX$Y<3ORw_U##ftSPGoufMa#jS&6($dNWy74L_!* zvsWXa@wkYX20KG(J#EHTG|s?v)ej{))f?}e%3jnehOblSSG5t(Lm2&>fL4CGFAz7QwTgSS;FcLaVstT?CLq?)rvZ1q*_Bw zrlQ4oRD)^<sT{RFe6JS2LR)1om%O@%2f&63v0< zoBZ`XS_IZy*Q#8Z!jQGJaP@}jSC_d$nK8rG&-aRZwu^L$1AeBORs&dIj{+3h%T`Tw zu!2onr^p4UuEbIw@>{h(7Qwpyei#TS+XalB3dRFg&XW_E2v~Eqtg+Z@-H{*rO0f6v zIA?SyR?`B|82yVDun;8)dn!6#BBKOP?tA1DXF{UA2~D-eHaEgyCp7fi&vph5a|#1HKw=F%c&t$m7Vry z>)Kc~?u0f1(LN8G52);&Q}ne%U;4sjvG+$lh|Mr zt(vA5GuuT2VRNCpYIU2Rd^H!+%A@>BeHWiPij5B>XWwQMuslAQ@wGM1qdd!`c5*CO zn~}Tkkl^Fk(Qe$~AT~Y*a`h}6Ou+4T~& z01goXDKp*kMT4!$uvMo5*b4{{GjKS48gEM+)0wlRPwnA`OcbY9q!e|i|( zwXQz4B4&%IfF{wi<;Ffryo38oTXdzWYS(@R_~*t~#4^MEx`1oVU-SZd4=jn#W6OGB zQ858Y!ArVgUl$On=8(TS{QnJOidv00)4{G>;_5#m4*w+SYATy3@st0qT(D*F5u?v4F@;%%Y%Bb?}T9*}1+vJt9g_d#cjU!2|XmGWVKp})M!9YbV%8S>x zYr+PC-Z!vb*C6ssE^-S;>(htPBY!RYTu>O_sSTI>Qdj&*%rwFkxcGa>0$$jE#sv!h z{)^6myj<@tU&_N+uX%hnqHG}vZxHahD9?R~JYostKD%s1&BX##%Q9*|qRqt>QvX2k z{UH13Ho4#ns26<@QuNl))rqf{^)_g9XJ`c@(yvbOu@z{yS9tvW0U?)Y#iZF2p!@vT zsl-HNxmWZIUYWSqw5xzdr7pe+tSv*9mRBKwYV;qj4NK5M(Y1%c$S{w|J{)rKiCB?b|+(;En zTumJ_fE}ZLY|9s^N!bVx3v3s{ei`YaxG5h^j|M?u3Zr#9(x}Ug$i5x_pH;wiCxON? zx}&QAGUgOFOV`}){Ze#tu!SjCg z2RqeQ_%kPMlm4%cLQanQRiMy7)jUPZ;(rvIlv&Z4oi`Nu1b$@SwO7pB);XR8`0`5% zON4Sswnqpwv?WJ|az)fTrrsm0%=fS)(
9lF}QP=g^OTLYuRDS>(TbyB<_X*zV5i zzHwKaPcG^X%?M;TlqkoWe!I299K!e{WuNxxRd)*A;%0s>J&)Io{@50gt4FhPbl4pS z`>&OQOx=;$N#@sPt{XiTnv*+88P$$dmT6Udrx`)WZo^=_5!m$Nm;7#Yb3Z5aaZ;=x zk*XGMv+mjd$iI{Wm}`?>q3AnC6i}eec1#=oA$_?P3yjLVIY}DD6#4#c3>ppxa=bZi zu;&sK^uC8_DX5)Vx*@_EKlT|&SClNm!p{fv>u3TTqdG6q#l2c(4ZaRt;$YZ0JKkzn zX4%MwDT}j9q){JHR8yYP517((Oo$zLPd`8~}<^;y*tj$X*`*vYKVCh@> zviyY%f9QKv9{FCr>{-t@#^~{#PYuN6~8Nr=jcsVP-Pxgx@XjF1o8ENQM?jbdb zC(1dD1^eC|77Z)U*R@-TA_g{=oZ>_2v=SpkyfG$I0k`~UL|(Jw8EELR3Fl!4dc}lqBh+bo{u^O4_j>EO%MK)Uea}j?;u=6 zu)W^fA%6ARc5S&>jgV_aNKr)r9X6fZnR5!&>g^Ow{>R_9B4(-`y#zv!t|k^a$7FVz zG-gG3o>PU--P(hr1Se*$MI{; ziIZa+pM~eSh5U-Gt}>TgyL0QxcOR!VJT#e~{f6esgHShEpHY$rm0R9I&_k4sEcCW> z{j?eJnEM3OBZSN0avF!14375DdwZs;mEzeLoa^(%kB}zjx;_0P~h)suF$}!u& zc~U>EToCj*7FCPa!_j)}!yK1%LmTd+7lq6_wk9nq^1u&9kc%IvkHTplrXj1=gR1aY zh4kzy?F?equ&zzpB1#r(gv^xF&%8kB+B382(;=;lKhzdy+bM^byp`o#@9fxrWOX%a z(p6TeTtW!1gvhQ6$c;&Oh*4Y(059OWSC0QeNuo*Jew!po$9>@IcH6BHQCKnxDdvQV ztqZ;_x6mIcBFOoEyHGb#H3UNqbj*i@?qpzjNmVMa2Ph>Ii?5K+c}1`Z^|bnxqR?tqA2Akze7vgj>cBSj4WQmr z|ZfzX%eB+t% zs(}`MD|NLnzSUkql+%9Qk6$__Aa!w_%FF@SECk&4bdI1yG2Y;_BBDW?-}7*O|B;rb z7-3)w)JPU@v4~Dx!+J$72Jk^BRjKmUu}^Kv#U1IlHBrK=Bmt=*GmsmLN*yjY-(5bw zK=LP!E)hx;j)4?FQbQfX@E7&CSba!OYNo``n)Pnec-vTtDWx z>5db$EYlkw9>T#X&l=qi9)@7}5KycD-uLS^DEyL}ldiJs`@X z*&yX)B~@R+g&?%DT(h4+z9{7L%-m)9wg_rySSQ3OXJ4@~h?bk4-JWos4Qf8l<#qQl z%Tb{abu$R6XNbc(HI#Go7nL9cAGk8i3x5r>KfpzcUHfC%8EJ%eNxY&>ALOwY$2m)I zE^mF~#Cd(vhlp>@Dvn?@)inkd9MSvDuUP1MKCN5>x@3~3`bP=911&!>H5Z7p(-Q>| zHH(Kc`m9+nYgks3;)m^~@bWT4OGP@*@*%G21~be?`?UvcDe4RFH0@>FGJJ!vM)gt4 zHAL26!;C?{l}L;H&buoOE~c6T@&nZ-fgtIldfcO4$F^`qug4pGwxRUuc{^zrtF_0Y z{SVcVt#Nw964ANS#LV2iIL@gdRp51BE_bfsKH-L7yK4c5hnk{d^;vKWS=~U zNy;okHqRWc%@{8ZxnHiIZ7exl`79G<>{C~Gm}x8=FRLE)w(wEHGV866MKmNvy((iS zvOUo#_h8Z2uj*TLX708$;&_~9TK5LR(vsNL+y)e8#sVd+G>*d=cTt~BCmvf4^h=U+ zmDR|5B%DgG58~6B?KfpL(BlY$0le{Klr*@$Lx1M-hS=dCgPL)BLoy<$GcF@NZDhqK zBM0_QlW3|s|400G450H;+I*Zg>Qy6vaXxlQ!^!$UPr4nsDy@_@AF@Ci^lj`8E{GoOmfMPw`SVS=1 zmoMvzN#&>AtV@e_QK7WfI8`i`_I^wphgx#djdahnYT?p;S~WZT%JvS=f8^oq^2!Cw zrp`ym<#NkbPRAgD&L3f(;s;I2nk_>(rD6SVdbf1cjXGO5j*nI7;?7o8QiwMjD-Q=cn< z!N1v}DdJ$ASN1_)s>^yiHF8)^)UUn<6PTC8))fd8+-hn!>X;42_hz?2w1ZlbMUhfs zXm$fj6&nj?7ge_6dRCbKQKe5-}D&#HRaEv&5 zq@iC~A3L;Ny0A8jQ#nG5_SSVb-g!KcG3+-q@f}z3eaA=c#>U@Ar7t1-Yk3E4XDTje z{7tt^^ZFzKVY_$60e6x-J8`8-MOjC6CKqFyPzrU*8~P$p<<8c45pExh2;Bvo|$>(GxM35m&N`oT3pQv)bQE4_KELMClT|l^4aA4{e?epFDsWT z75XLyQn!h4Hfqii)v0VZGyE4fIC-O7Q#){}EpF1u_0x1g-yqNPl#|bI zCr7C&MYqhly^^p)i=Th~W&X14Y3LI%YDYe`vxq?*>+5%Xfq>Z_U(*gVuG2~J)#Taux|( z@=3SiST@gdw=gdMcbAHhJbWwPPnHM?Z=d7Ob{iqc0HJ5tN~^nb5|Gc=*YeXYq+M#f z(OhISkxOYP>6qHc?WAUwE32(9H@XMZpvwXguiuXatRYZ5V5u>S;nsPL_vRtS{QH|}p?HJ71 zrT9tZ?}V^3pYdY#)SR1yV#3%NLd)kq^{#Z@*cZbX%0+Y0>7NO~Hi34})c}7Wyo@z} zxMsV5#>~Do^luS{boJO*Qyc(kkt_0c5-O+#1x++PgRk3oSYN0{C3WuJ8})*fp9vyc z5~uC^cxBlKSAOw`mYU;2*_rKFA>qNDPi$ON7-QGwKuoLhje1X_9S0L%>yD(Fw6T%W zgynDkIx@TOWe5#nVe9XX`bBH!X8^dw*JfK~Y0F<`Y*Jg!>+aFQhayqaqUCb|!n^vY zk;k&o;n5+0rHh$!r7h`2G6>OR-(5OXU^WOdIlCpIaXg>+L_3fbwiY<_hVCAbTM~;R zujMs+Sfg1Q$U(CHo+@utw7ud3*DpM3yq-FmW)QU=Q``EWfahWw(Gb?%4;>CaVu5zG zAqrl4!?F#bRV0SBP ziRM9WrG-trtPByllZMLPNmIXT24ysF-0%xrP4e|v%+cQYLSJ~eO8wdAJFiR8&Dzmx zUQN}^PLMp>%KsVf<#5(xywQYQ*2TS&-48XE@8Jq) zIw6Y)LFv3ozInr8V_$mxxmo{?iL$IShbYv=S>cd{J?g*6S zYkhJ=#?1}#^7}{eqV__lUtVXk&!&-tcJc*`i)fopiCYubTn1}aC>t+p`ZipSFlEDCz0Tv2aqnf5ZkHn?lr&4`lpi_4v&F~9Bd}(x~AGyZ+2dQQ{zRWhq z)J4pPgcR?G3483ZLGin3hJO+yBOu%f@8@u$N_f2ACENvFh-bg&>_&KR$HPgeyh7v)3FQlfQa45x#<|C!x zQ#WuPurUpeT%~j@i9)Dej)mV^;(UJ+uh!_~$u=wNdUYU*?x3gd5Er1Q%6EX3mE!H| zV2>}(G=qS-5R^-kUk1vB6S*S4IKMfrxtiL;Ihc) z^ecI(t$cljd3lm%R$Dn3KS);SQ70W7BF}e9ktCMXrsrRg6|Ahu|5>*w3LF`BJ9{C^ zNP}*QG@oF^FDoN;hh=9fQyYw{GFLq6^iraH>col2 z>MOlnyFXggkmuzXg=U9+qrDb>=)gU(%6Gn+YxX?=QpOMM zfX}f^?%ubCNv#5BD4raiDcpHj%&JL1c5u%Af;Ntp-qmvzsnW_#&GaCQjdJk~ySp8X z;0q6U&g{=edw(#WaY9632S{PmPg=NB$$Y_|*E~|zO8RiL@bN;?b1RV2K8twT%v&l9pox4)Cg8nm7g*eK&i4@jGQ%4<|{1j(TS1qZVTv&mWJ0`ZmJvmst-i& zAaPOzo`ex;EYY)vLuhb~lB3eSBXzb%Yl_fILAg8Rc0N9_d4`nGW|Ru{y^#P; z(82xG#v~P=p2;*F&)UhQ#GI<=(Hdh{KS|E%tB|lL*=CmNR>!7?DQ{A*beOuVq?6&A z9`kOb^_sPJ2r~metAmfiP=`v zsAkfabY`MX7Q#`_-?pJ8NHjxqy+AN!ASg#V&ugao_ob5Pl5~+3#kLc`Z=_zelGnRo zhoYxCo@Jc4)8$pXE`Fxzap@DL_*`ani{5Z!2FA?+*owc!{OUyGt_b+@L9)j z(6Pj$j&&!4#}yvMViLgSSKTZe?-Z~3eYoXx-ucl}m6GwUu04})*oT|0a=5ErsRvEC zAI9&xby04-30XP4K8QPwpuOAvc9=MDqQp>UH#(MjA^0c*C7?`noaLOCkm=(H(Nt54 zjxFE!ZLu5!X(1CczkTLWoBm9Q z$t`4N(!rykR9ahd@GKOVQ^@NW=GmlPX;PyrdIz2MI(qlE&on|%Wkfd(uHLT9m*#d+ z*H1m|)cj^*wt3SNp6LcTzd3MLGWD)WZ)dw|XmcWA6GToA_0b<~SsKb{|ASo|9;>dw z6Fpabk{=5iy1GB4AZa+5zU)00sqPm0*a#y(Xt+s%3GeDv<>oB4U~JDi{eWcX3m2;d z9(9SQjp?A-Y9uE*@LG*e{YatYU=KV_62g*kxg)+!z1GA-NYR|=*i35o!0Sga%`9D; zK*Fxo`)VXub#%OBCsz{#!Op`2^+=+Nmo2h@S~5IcH116MkY58x3Sw8%MRyi)4x zM)*az!SmPqqgks{S4JL;C@h(+#DK2>O2+SK&NFn9Dkiehdc$V)cFbUJbK~-j z2#5^g4Iy$2NZn?(+SF30XC}AWnJkMgQDGJsT8EY)l|%;Lmq`W7Yo$m&`~r5#F1o*> znq@4uF;7If!=E<@XEmeTX;MJHY@;28)(!c&<1v11Bt6Yog*ohYbw&fyspco6{xR0? zmGbcQ5v{7jF3rXwKf-6E&$yT?HR?v}A}uT%FZPhDR6IWim-Hwyny$w>vDM{GFPF zh&*DA@MSnRz1rwi`r03%uIg5`PgFv`!ru11Nehv5$Tf6Ky@yqdEb1~44@fyQ7vYo& z(PTjhW;%`<*U^s_&$4dJXFYX3CB z{|nT1;qqdomuORn@LKay=5s>Rl`nWI_*u0nliuU1p=F3;1aQN=D;1204ij?jcNSuqHMc{u z!#|Ino9GL-OLv>%w@uytf^n1u4$8{SbzzC^{*@h@A*r(r+@^j_Gxd4R8s$82voZ`} zp_X0!=N>GSxbB4sO(iHH)>4%CRVJ#onANOvMRAhLU-ywzVcqbA0Sj?Pi>}GU&Olij zyAujgjX^fWsI!XR`+oE`->VMfd-^TA7$*Q7P=lp3FZEfa?qNwi!vT4*&rqeZELE{E zB%uxH)@C%qwhZaj$Q=gjg6r%#QKYQo_7cTkA?3Z@hr*5xzb`1Vz#Y7zX{sF~GU?}sZPzAeutWs9!a0!;H#V^!wDw=3dJPE)avij@8I zb_=_-o7vjc+19j?$El$ccLG|XpdP)*`Ih3iVlu>KR;Ymf_ofIyfes1| zwtKe|q=f~cS^8?Tb3@;u-(nLgRpGt9u-FAe&rXDAS1&xzx~+?m%j^wT$}nZ9MHbDI zw*#Q@P$0Y5ts|lVUvmwDJ%EC5wu>I^-T8!E5rRqN3#xP9213fY;>q- zGM^d>CXy~p8#r2iP+h%KSWI!{68}7WP4XKb;qoBwaAGW_kteBTzCsyL;KS~Qko@sj z>xR)=#5sl}{GAPzXt|T$3erq#Cj_+Fw96A+e=m)z9rNmR_hQa$x2zekic4GM4jeQ0yCTvzHfo4dKSQ%!I);q)a z&Qmc3hG$1_IK@HRu=c?BzVYa2#r#N;AteT#Om&)5!CsC;B`LGy{D z$Y))X4UfpIHWPH_RIa+s-TmTO`+mAR!sl@E^7?0yJ7Q|P&&6%;VihrqSmz49NHCG4 z<`k8*)GiOlR@%RjKYC|>eW+A~oIuf?0C~nhi)QuFgsvm4z;O_nMCnS#Mld7>o`3ir ze@t~RBN_L2{JnA1fG?xy^;J9Ed)yZ$o3%J!ncWp)x|8XJ1jUy{Zb(p4%+NIbq~xwv zU?-AxhMLVZp6i@xVlK52+1Ig;Y{M=gE^G^Z8?5Pq#+T)Z`aaZXLj>HnFc7awBKhKF zYl`Bh3B3=)@K5$`4>;MaeRb~D(0HXr8pCF8S27)N#&h}!&j;JsEMHoWZnnK1wWFq@ z)c19iQuES+Mw4*}JW~j$utB9!cW1}vjplDs;ATzt3rG~}l-W;EGAOlGZ^xG12>h`v=3U2vUhsxrtjW^!JOhC(8kiHtw#U-gtoRhg{c7e| z1FKT>-_BND{HsVMu2%ZQ zQkZ~MH)6V_T$)ViYY$XDol;(2_P7_Z8d z|8CEzx$Vz>6M62yziJsH0@+YqL|g7&bItLB2ea}evdtc%*JYlnWp@GRYJ3WQLrw;A zWp&=lBhLgCp^80)S+m7S??hHE&K@jt#s+Ma77tO~X7v-gX}x`Nm;x_#&vuCtn6TLLz4JVOL2)TpeW18zgxv~%Ma&t zJ}0tb@LlSlLz=spol7!Cb2NnT`_`OqDeFN=o& z88zKK?{dP}2fnU1^d7F}$Y!ntcNqBdHjXgW&Db8F=}tkYc4A3;7pc~ZrXZDMQ)!#I zotzo(XZzHFefh_Qss9ik-h^Mv_iXfTG5-+H0fAEa&&o)D*){5lKCgACZ)JCTbh#TX z);_J}H`lmWf0EnmRaWYu+sJXowX&_gJ;eCV!DRgMgIwG*8*5^_uz9Cw$060rR37!h zhov&AhGp$wNk&SU3%)}u10Pl^tI_X#FVmZMq#_NWd7tQ=CU$k7!o@k~>kHM4WZC#Y z-E_;#skl)hieNXXhAjBNh7kifKWe!b*uPW70J zF^2JDY**wR%6wlyxiyL+;k+LmrnCt^$x17ar+XONE=0@XIyNOOv$Xu(l92afVe8H8 z5z`gtn-9i!0=^JvmVXiVrvw(7QX1gtPc1lvWG>sgPo5*fe+iX! z=shC;P+%=BHfkpSeB~-j^yS=b|0kFyn8Se)TxI)4H=}HtD%LVj^+#y<8ZCX@tkSqz z6_-6ck!19X$lN&S3e~z-n02jbAD%YIMBST_-se*uLnblrY1?yeMVC9>FUHZ^bR@J zyn6Z+0FIgwHnrRQYA&jG?5hs<*tzva0n-%bf-Y2@4MNcf$~t>)lbfXF^1c($fA}XC zn_5z*@p|j)pY20Fat<7SzBTNe?c2)(tnS|KJ*yL7!?AR@WItE3k2Dx^ZMV+*>B_)& z&Qi&4kB1s2#@>n?--|;xs-9K20tdhSuqN`Ct8*&2GAq?zS;hXzJ3(>iZicFjPSXt4 zKyzS9izI*#>@J;Ce(7D7#sdymZ1}21<6djo6e)0blxwTAobqdDCol4^PZWHi=Gx1Q zeP0a#t{CK!Bcrden`f0;c~7VWZUqGhe5>YHMRSM>Mc|J`e)|1;vBQ(hhMvMLPgS?u z#_aF#k#x758GL4S#rvH}UZ^b3Sl*s;%w1cfXwROKtbrAgX zMT`H3pZgP3hjCUal1^KHQ~xWO`#=4~YuqAWcKA$ImA;I^}CiPX8LYjA?T)RZuDs z*=c@HFEXNa#nh1WsbKi-v(-=Pm;S?J|C{>=;x8W%MyPtia8r$pUkP>!yJ%u(md#Au zH$hXAiPcXWc)192b{p?v&2g0X06bL#kDz=tuX*aZpiA+`#Xbcoe`yO!2yaOnO_Y7lSg8WR<^6~yH#fXmG=q88-<|(WHYtD z?_I%-Nh^3qp=_`+DHzI}YW;Q7v1{nE+rRc(8p_)2U=RSj$uu za=VHcK+(m}bO9)}U|dgqQy_MamK}>()F3jte)XJ*jooJv_fu*26QJ}UXd=iM%pRW! zAkv#uO!i2aJ?9ZeiZf=*86eJD&^4W4w>VrA-#KC6$OAhBU=AIwdLWSRKfd(r*A0>o z*xfNl!`@*!ElZ`wg|c14uEtk??ym;#8;WZv-TO6Cc9uQ3aM2O5(`8NE8HkMxSatxY zJR4KZU#OzEjq>_BRn~?ZtFY(I?mwqs_ZisOI4H9L3P!H^6a6U*>5(SF5%U|ra(>U6 zt5%y*cxvGTe+zVcXH4gUxP5Ts7?)h7*1VvoymK%dhvLZD?gF|utD#}qcRsf)#i8YyLAiuNO2 zNbKh6_JzNo_y1uZZQ^Rbk0mw)Iz+c<%?RrYE~lu7T-nQ_xY|)akSFO;U4)UHVKWm^ zUVdU_;frmRMEEk<*$y}o)%h13|0f|a54?t!ZW8*+yl%q&n327H0gn`=JDmf&^&)4Zb+eAC;oc@pp7Ad@(1z8l@2#=7P@_TpPJG{@Rtr!Qvg~uUj`OJpa=kedJUyn6!G<(W>%?sw?wIgujV6 z#+MnuR<8CzR4T#r4Uuod;vZ>Kl$ zm>$z`SmL2Woi22zC)T%mMzn1IRb{2&S`q?xd258d_fcQHPFAk(2doE)S9kUTzZJ*{ zm-BY&L~tw^=4Jpb(aR3v8XKf?+Bw!ZHn1;k&?UX!aO~(M+<%tP$)g;DJ`N^0{W)P} z+$Sk|(@|0|T#`k`?vZDi1rGrjtc?}1v@j|7VB;gxYLdte5v{H?hR*Lu!Izet^U1e* z>4HMK?|$ID82%*3s5Ob^GM`oa{+?{A@%8CBfjF&FreJa}!lqR&+@I3AzH%bii}B{P zrYa^y5=bmLuU!)-+Z`BtJ)lJrT4|6iZ^W(-S(`uv%16k|O|6|3Kni3?+fUqP7tc;K zU0}4v+#gRRPrj8jte_nBGIw_Q?zET>+mzr_K^`Kjx}4YW%xc;DmvI>PFPymqhCOpp zA6;h=IIxJ;Z^Fv>5&S2V-8u@p5+;BP>j3&@65W|DhGX~GHGw3s3-y;vyO_r}T-OPW zgh@4fKBzWIVV26bF1-#sydm8}ih_%o0?6l0!B8&z5r)@nY~iOC>;#)E!2x|m>DPtm zq68`;@`G|;{ggvT{4}g)yB0j$(+hw`zWU}y^Xn>m2|cFuaA{G^X}%eaBXCc;XUU22 zIqxvkbk0;?2vJV`btoxfApvyGet6N?RGvKZQ5oO$%c{lxn*9-Lg1SLFnXQ2H(uHmU zf^rKwuCjE}4AJt0mT-lUx4g@*P(_m3tOTY7dngLITWti0!hxl*id%Fa!5zprKfTP21XlY&?m*c^E8I{1BL!emP6N#Fa2%L>wY zo-JPd1HK%=x&e2(@=u1dlrwMcNd3#f^P%bIc<$E956G8_3SD>^{e8xCI ze9XfYLga&$E@&-Rtx7Gi0Y4q&gFd~-2YJCcIBPH?hM#OgL+_b1Lh5;iq|y0z~e182;a!!!_XSo9u=voFJGBDvwlMQkAs+S zWhgKHFp*QdsSLOItrh5~9i+ZHC**fzG1F*sqg~5u#&>G>4q#7~Lbs2T{dOFB7&kOS z`JNx(Mwxm`GTm^M>g8T+Y?d4s!mYn{(GDY_?yh0>cLDel`!)x%_@bT-o(t+Q4R zT3rqn!EA-uhuIrQB&->rJ!hJyF%4086DtNYCfaA+D5YSwIY#iHPe?RnI2H^7Yy?kf zi9O~Jp9KYqR2Fr@F}tH>yBx#IYQa9@yA3uVfBkVqCpnA4V_V-+7uv+=hU8_ts?}az zi-qR;o6Of(T(p9cbg zt`&;SF9#k?ME_9I6HyU->A`DZV-2&4va+ytx2m>&+va>?UJVdZcyuz7+9HFze+pFf9Y`CBofyM@T&+ zJs*{FBTq4=>x~pfd?E?O6M)JrEcZ2jQ`AzjE*@9JActTh7XFEp{45=E>(gWgy z&?Z}B@kmbtk{wIU>ZjT01#4U~5vS%xpDMpCC|aWXS9`Dj!)f@X|Ft!MTJZ+~@YF7$ z2j3vi_CxzMW`Rvj;4dJ^PW!&|eNR++m;kM!f`j&YN zb6873h4w9b;dgP8aMg>Ek-AoaN8K8PZoW~f=h6^Hl$?Rkq?%>8HXC&zWHR)b$ep~B zws@@k=Enp-7h%}t<~UO$jrN(3&WrUx89vamp`Y8=)KnrpOT`4~QJX{zOI6Ky9|5b3 zYJg{hD_hbb%#cN?Z=Tc5kJ5YJr2}=C*1gdHtg~Z}ncr5un93ox#YT@Dg!=Vijpg|K zDIPMIlClg@C77_8%Vyn3-VA?1h4KUrr>&ei9ay9|)8r!S?*OB8R& z+$(PRh3lSsBEApbNcnNK`JHPQce1wp-lechPu1?rud75o+E$`L1v zNh_vR!Mj+(^d++Vmi(%`muB&461bp663Elk64S+joe=yuYq(b7Id@Juf`40ZQm}hb z`dF&2Zdi#|QTJVdQ_R!{v$wEK%uZ6jDL>SkWF8#Xsi+0#@hO_CQ;XoTfo@k2jCtJ1 zvvohur~)_@BDbxdedtWwc@o{0sL!Y z@qL=<{uQ?ew|Jq)_PNAg1QB20U*TU6q>`oSvM5!c8%l$D4AKGssLPsz)p#MiF7z{W z=)&{|yf(7S!5!Oxn2T16ryJ;Si?Cu^n5ie zb;l6<+=F}Z)oUiZ>;a5em)l^4q3$?4MG-LC5m!Jr{)h!eZg{s6#G4|oGCti`4>3>k zYs(h_olsj;^9;oyB2SEe=N4_j>{XDmi^vbsTkxz=ys^(m1={^LAVfHlsn0Zw zh(y!UW2<4O@9+xWiT8Z3&4(0Sln4W<42%j<+(%C?e1i#lLiQO1c13DEU@OzuE#nhc z@^L7iix~>aXf-uSu3gBxv|ti1fhT=|+NY_L6MEUe^fbk#4G+YeW5HVLIIV3456!t^ z=BQkwiRq~gSlh<r^YU$8PVY&bA$rOqoM{H|y+EBG3a&=~!2 z3fT3z@46+@!@~Wk$8(S4heKio4ZZ74(VpqrAYsdl8FKR7=U)?)ex|04yvoCUk`!$g z8gG#n2wIQZ5CUuH=?m@vpD44k5|LGNQtNLl=L&UA=Swcf!?o z2?=>km&hIbMw$sta&@{yz!cFQf}tVbq0kq`5+KDfn}EQ#YeB#lL@hPu73Q%G`=Rw~ z;>pmA_m64B^-JA-=Z+CM_mA4rRq4`n1J4z0)ZQ^nWip?x2m>XtcyayUxCM(N#QtGc zf_k4Vb5;ns*g6V{q5IImFqw@Xmp1R%`>j{46K{xbP78lhN#*ox4I#dR;G(2EPeaGY zcG#uvdg_Br2VNz9Aqf3T=+1`558{j%m-nP_j0#S4$06^i@P!iLET+-o+*w_mv{I&7 z0!*YQt#>$gj&OVhO^G1%Fk@%?l%V*T^MfH{obXdctdD=W5WA+Q1%|Aj9# zPI0Gj;msk-<-6+0INbn1nmvAn^8P za>Jt}<=RKFeL8+!2ZMhk!5{q4Dhgb)`z(JLN{Nm%nzNj=x#GUUWRcRKZYH z04n#pgtu`Awl*R$C{L?DVsBn_4t(vHqM813F-sn*xeSk!Yk!$y)~xbOKK>9dXV+1( zNE=0}x`OFzY(=+Mdz-BDx+7WFgc@{W50b)^b~S zvswQProoi95FQ&W?)RL+%`me7CF>a`_@AMt(qGWw(`JU_jDm&m8}@z zo#hV^E=%_50Fo17Vp9#pg`Ds~RpzGs{-%^6*#5X}Jcs`bPsxF|-`4eeqH$hRm}%%8 zA2d>b_vt~G_c=z$`G>Fs{7w%3gKB|FG6b>_Vrt}5?<=S=)xc&!egqOlQ#WowtWmq8 zL^S0W^bmgB@}T_}hLDMHbe1%njr63?3{{ohTH^~?(2gB9Xph_i?o)uVA`sAKlL3qA z^4$LASeEFJ7YJwj27h_e%6H(+X(M#6vx&+N{hSrp>?1Q&#G-S@E%?gAhKOS~D8&hH z{uZF)AA0jE1mp1O$pI%1r-yDQmXmJg97+nJHgoh0e;&BrKF>>5{mn*0-sgvH({*`2 z&uc@69Xwn;o)kXqfotjM`ng}|CVuXzw@#J+$T}$}nn`s#9l_DutK!fy*;qiIeR3<$ z_>$lU5gzW8>!5s#Lc|fYJtmvjL}+r(;2klsp1wjn zJXHbWr(@_HKe@R)-wW?!JI_zG63Q9{gy){ft-O&6JJ{YN;^bP)j5#7460kw| z6~O&KJG*ZtBg&X4;TA+ao?WjhRNM-DU97E^%)z?+QBHIrQgmO?dKQSdbiul2Jz^gb zx02MUK^C9BoFN$zae1@jyX0n*EV`X-xU82)?NtZJxl2FcM!|UX!h>$}X_`~-4&cv^ z(vAlI7lI;iS#Wo_<)N|;+Am&PipQPDn)hU=P3S`&yHG=6JNXXzCV6#&)rIU1+b%(W zL8ORMicnjjgqOu{Ba@HJxKkUgB}RS|AO#hJdm@7ZyS_BrB&cSnzA2b0i8i;)6Uc~t zldg3}77KAuC%>0HhI4c(=4%pU;sTorADxRp3hs}tOASts%bbh&-9*%HMrKk=w<`%H z(4yk8a4yw(};m4R7UZoFT z6W^dgHys7uZ~YWx>VU>O8gLd*9efxv@W7MRgwjO4*y~1iw3b96luG4>v1-g%Fu+RyXpeT|8`eem$R^G9T+ICx*Q{n61-W*>^mOE%e7t0K4) z)V9(0`*Gtn+h@~AH3JY|C>Fi7Iv1NNC6Kf~^bow&+fYDLzqC^r3EiO%f1q3uJDY~y zbb8hyU?2YF38@%_;PIC_R=6RtSzj?@?F+k1nwPnGehkCTTSR2f4y`d2i6!PA9(G+K zRYI)(envdt4Wue5>PB5?5ghUdPKN&`@dv-7zKH(Ih!;lgNfKtYj}5yMUm%azxL2PA z10D!U<%=bkcKX*TJk*T>95*TeoTd%WYn}*Vm2!sexEl%YdUEM5TdCmuUL6Q5^^lA> z4OrP+O8m(F%znQjPSGsO;fwaITASa11po^=Oki?!%{$eNjM!H~W^xt+;V!sx&AuQLAbTio4(pAm0IT~wYe6e3Mk^or_!s<(3T^Dd~y z@GKmAXF=@KUr<~eGR^*&)a?VC$Tsy03h?6CO@~WyKf3zx{7wjvA#GV;`{`1*g=a%7 zI?OdWCm<}shoNg_0FStjk$`XkFPwEl zS&4W>5|PLN*c8Qx;`&&wG1C}~P zZu%ENshQy(o8N_r9oq98+OKV4A4XjFy-?ow4VRm(?5DKw2zfV=hPSK|t=Hp8gnqP2 zKm;pkH>j6KCA_Q>3Z~C<>ny8a1|*IRb$);$AiKinJv#ht`niMpk#?fl*uRG`eq(R* z?zI<4KZf;4%oVAz!?m0-{9uzyYZsztr&?vXn+`dw_HIybr>&o--+8MR<4ZwTTkggF zX7$@rfB}aU>ny_L#K}A!SoUr;z;kHez00_Fb1~c&y5*ESFj@li&dPsj<=b1{zWj>{ z`CQyXzFu4GV2-HC(CU58?!FtJ`*(7DIglCoA3v;&9LXD&`D2SyLfHQ>&tl2nal)Sw zhm-&gM!kIp7PunN2OtOjUUas~aCTiPk)1X(*QRb*Zp7JI5h{zm!vAov_lhT z-zv!J`u<_rD-OB?A>WIhrFMU%sRX<1jM2^}XEzrUmMZyom2rNs(mZJd^4=YSE7P%T z_Px|v?h?jZ+<5g%cG_9LC9TERIWw>w5!)W(8MMJf`hIUHIhgjK_4-EHRvVZq@O`WrHsL z6Mq;;^k8Ft8%OWL*c)8TdE$WR!Naa10G~&hWXGD z<+mH)#OEf-EskoC8@qfr7HsG4p_(V_R&(u264U5QuQi3iOeNSCcDc#wlN z^kbK_=yJSb#ehExBY}!HJYT)y{xFg9S^R8j=kE9sjbcc(>6F8O_WW>Ru7Lc{5G*RS z)D$|Dw*>9TcB_|hQ>if-eVT7OkMTxN>n_`_Af93dxa^r71Dz^FaF?3fXMZccMMejZ zI_v!%_JM1Oa+y}&zo8P)Rp~R)E66hbH0NiU6ZR%Ko75frO>;$Rrdx@X`c(GsOm#KR zuJ%Te0}o%`e)Q_2!STm<4bk)K$9YcXH>MpHIgtF~UTu7>&9xMq(!)1%XkO4pvFuwsXK&26urCp6GS>=p zAMyp(!Cx1*ij@6JH53s&pve4Umfa=Ao_v>5dc6U;@G1CBmS?l5Z&-590(XAR-pDkA zlxF{nRWZXYlJ-<9H0ti8z8wGWPdIV62Fm~i^j_OXFDktb&SXB3YXv~V-Jr&G7PNB# zHTg<)hU1xmVD#l5}duW!9&Q?ep6qMEkT@7`SVfYURrz8 zO}@=0Ky2k+%~}9F7rJDoxzw*ViTEe|?KrodZ~g*6^GZTuvBfp)48pZrYf@E>yIYs>lFA|{svo{fbGurrlw zp}spt{Bq2c2!K7kKK1^jq3_He>F^ysW3A~xbeKMyPX9UpSsx|EWplZGQA9b&9Ha4Yq$o4$AztoHF0IXaJxQS@^Fiw*+?e_p9 zu;|aC4A@b(;nf<4E3!*1B7LY_?!6(rRms(OMHhWg$~4|#1X*-M!~H1SsH9yWNpZIC zVf^K*g5&LD-RC_hv6LS9hW83cKefoN_ui#bvDAagBw?tF_dt|6OtD)`NLWKDp7WgB zd`FPtlcJsOvdzqW_95%*K;Xk1OaaE=s9U0(EczM%JbL8&zV800Ni94K?>^VUD zmCxBkBHe3%L?KR|g$dMB=Uokwy(H|xOu#GP_2gPjqr>X2g1&U0?;dw8a5V`N(lUK? z>UMbIN7ZLy+?tI%=if}?Gyfjc__f883x~TM7B0mog2&?{VzN=)b_++Y2yNho?9N}~ zKlJn7a)O1&?n)VuopH!N$_f-){{cjMEbw}X`MpAuc}os&k~y2+yKCvLPYBWMcX|^9 zvHMI?@SBfoGk4viP>loj>`1aU&y>LZUp%VmUuXV$=zhuNkmEf%S!g{SJ*A_(vSeMQ zh>Vi~Gg-h^VgF02XZt`-22hO}BMC(le&`dO7}ueg7|xLGiG2zE>*6JV(!Lvz8xL!ER;B~7V3BUDp=eYuj``f^RZ`BZ`` zYF5Q=!~s6R-g-!Cm9sVQD1z#is@FXwhs{+c)m&YGN2zbzqZ}tP11M$n6}4OO{|V}{ zNdSr3lfc_sDI+93(N>|D%V6FF3CRL}qa&4^X&gSS8ZYGzB>rZ6>*B}3`~1Pi1sC-M z-m2+D9Z!_5JymT$32Bi~{mwl}V@y=7h3|z&PqS!zD616B8SiQ z7sp(|+(9+$!+uLmWZd5`@lUEk zvGxlEi32a@&qu$MnKZ=MNA+1rIltX-QSAFHI{tcI8{2a~hEwxp*Np0=x1djaBS*up z;kSAYm1N)S=3C%@cA=8|zu0@vu%_0oYg7>hEIGAXLd=y}))4=* zqXj*qUSThlS`F8)E}_EbgEL?jU+<`F^|M&bNo={gm!+!eHs{^#J0;z`CYB1Ppm6h1 z#Gu9a>Z-Dm#QL(tu*L4u&IwCq)`Vw^g~GD3CMz&jtI)TSAK0MeH(6x&%Crc^l)FnSg^_g^9 z=gPNzmcD4FoF@@;G62nehSiqE>**&eor)(Zf>JM~SJ?d-Z|WPJfPU=XSRb-jNh039 zz5;X-<4`>qbI>_ZV`ceVIr;L6H1z~dA7S`P?g`OU(s?*DyG{GHNU`T%SB zNsTKF?yibx)`0a?r)^fh9`C|?bfV?BX=db-Iqxhv3|wcGZPd5gT*wk@;uw_@(moQP7Cg(7lF zO6{4})`IdU&V)=R{lOWWg)39fT#k=hhq;FiI~fVL_$yx)v}US0k%`b0uE zu>x24yn8nC&empmL>LgH#gss~q~xcpF-Pl31I~rp3edendo*2>lqcAV1kA&T3r*{j zlc(uSka$e<#@9^hd5Jj1y_()&?|oX<)Lx5l|KEiJ=D8ejWt3fW4!|eyLIN+EzbJjBh_`X$@L)}7(uxJ+ZjOZBKySA77ROH z`ct-z42#(jtzFhS;)un`@3s0;=zMl1X%;}%#?N%LG%w&XO)A|K5YOYoiV3X9QdBk8 zg(CB~T5BK`H!LRUr_&#nJITFuks2i!^G!)*wGCT`enyRiZ}Nf7SFeC-YNC!Yjz^;I znH--5GuU&b#IZvI{SFa=XKMdGpU}A9UcUEU1TpT$nIh$SvS#dSDw)R@L$4k8A2rH< zXQJObbw>V4`1?}u2CD_^o!ER_?iKCtFD70VzBT&x&gJ`+Dj&+g&GH*67Gp{)w;aDW zQ{5yfpfY;jQ(+gbalKzj`=@PAX#t!7@ zA|bR+HuN73n=bvLdSPH~+;%SToGUYDqpkc%&W z)84IWl|T!{*~4di_i~BqE}~U){%AYRzJ!lX<9L z<}I&Nu%Z(0q4PXf#$tdR2HwSDZHekc?yDQA4T%erAkvdVFHfqe^L`x@v z$9Y5QE4YAK?ZO3_z!$6yrh!i5UsXu`@kb<#oGyeXK!tU_g!6$ldN|h?sRN>uOJxnw zV&Q>%4pqS^h~EUO#KsAPBHitO1k&GMTGfHomGd#HklB+79U;JW$cNJJf6Q||W?5Yj znvkt`S_E|K*oO6aeWAmJ&$2b?h>gr2oiGe|ng7xs9%JxMl+jif>m%H&in^_(&b^F< z;1`T1YgP+4HF&g4&{GkD7d$=Cajfo=TpgDC_xr7NWsSojm_H}fe=V%3et4Cew8|Um&uS1*B^A8rCJ;o}s7w<1FXs>5ir&)!eu)*Isd6&A2 zYZA{<(%-{S&nM~I69oEf+6))0>l_j45?2bp?)~k-#6ZH<1L38<;z$&kj&OJg8JyEt z2^~ZZ!|Tpdkjh-R{$q2D`R6#I@#IXWIlda-6^x>0H4ho=b)UUgS^a>K<$l5Tzt8!< zaXxjB%juU+Cf}vSiDksc3Ss=| zqe`5Pnm&N8;}otuCx!VprZas_QRi4*&j=^wNuTFFrTKvN+Y{U%GO?H9`u7iaf;F!Z z!SJZ!Z!oKRK6d5uaNIn5Kjw%d2FaIm8Nd_;?)+< z$*~rDt>vQ&^zY?cb0Jgw2T}!5W;Ee}i#+ou-i{xg^};zMD3?VE2IU|k*Zm-R>ft(){5!Z)3;+DJ_vTNxbC$^V%f^B`U#xoi(Zfv^2WKpDx}L*E-)s~n6&l-ftzYwuqrUvikefUiO6*e)4HP{V{oJZggwX3?brE3@mLL^iJ8e;Bby z8M6BCRe05M{;ABYDT2KW&SKuWI&9XjAt8Z>5ugl z^w;PQjBFAsKke@!hU}9pmg7Igp}~x0LpJq?4>x3EE{FT?y%6#gw7^lToQ&0Tg-z$$ zVvxDPc@U2oWoDa>R5R)+6r*P4u-V#MXc3n2P}L=__pxVT4P`Sdl;FPWrXlPh`%V~$ z->4RA4#R2WXZ-(t0S`bex%K97Cl!R|_sD4D#}#>`(k zCd|@@yLe1w=x@>I>nyUb4|P8~CfPU3RIb>DE?OdOG*O|!YhG#EZkxY46_b#c9vw-Q z?W@x{NBnB0GJeU^jy|)Aw6hBv9F!eo5bgg4)5hms;q^^q@C~+jaMU&8r2HACuUUly z;-x|O-LcaK@t^;`-h0bxe*URu$MYU4ld~AEsb!egBU|3+37iegtB`$4D=IE7M+XCE zL+SlU2Lg~v82;ODSWV7*W2-Hjd&WA^tdEm~5-x5onnyuMIzZSS+Bt{Ar^SCBbD zEVo6;pjAgu3*(XJ-1vtDXRYUx6I;Q9BtwJ?r9FqTqRSX>ma_xsmfBiZc&BjKe*|8E!lJyi(GZWju@pmjx0 z3pd;=yi@=Fe;n=p&G^PLi9@?V@88qi-Y;IZfBv_(nv?uTOXX&dZ?9JETF9vU-(JQ1 zFb+Ft<=9lWPhY|>-B|nR;LA6l@!)R{@y@*S7I#I!(_8-w{r-Qu0~49xsheWlSDVya zeV^Q0d#1ekZ!c{GF#rFu!AJamw+;Rgm99C3u*}_u_H3mS^ZpKO_~%9c%WtMb0rE33 z(aQb5V(H)FjsNjs%LNv*u8ikJum5J7{4Y2A*Og8h%)cD@zxVvRz4T8)=l@;X|F4bw z|7#Bk^nMz=_OCb1o0WAU;(&o=9&S{?s~!S&-mN62=+VyZao^SUJW2#Jz~ZTX4<4ra z6N4kb$45s#2yTnx6DR435fiI-sw~?c+47$W=t-w)!@8Z{dJ@mHYr5AK!C}1*#m;mF zqWS`TLv5GcSLWjD`-2Cq;OXsfb$@!qTuuOgP5gT?wEd$FY4T-;dluM|(-+sR`P`mJ zgWKV2&!PI38d9hmjIB|x2-lvWo$9ir!7z9NBXRo2egStGqCXFvb%go!hBS;43VxeY6^~<3ibnj;7Y#)UuU z2(#c7qX``Ll@*h83dFt<@g;ufjbFO0abMPvKqI=BpR!fj5hPcg#V_oMQ1RO)DB|$%$ z%?=`%nY9N#+uG2)_^FtG*v_}x=~FK)`p86Game&lfgYQ3o`{RhbmbRj5fS_A$Y_z# zHJ|X6OOwpj?ZWPBD~B@-cUTYYb$y&;V?xf(T5JIEr(f-s7Sp-pFDXkl=1N&1_C(o; zJ)770vKV{v+*#YEXK3>WNp%RVi8wq)yNueTJx>Mi1pn9)lTmS(^vMmn;UCw}nTf1+ zgHzAh5-`U`M}GKn5(~)$v9h6*M=$4Whh7f5o?%;vpl@$ai!Vcrghs9Y3>%hB{jZY~Z#v0Vb5|97U3cAYZ`#xn&1lSzt{;WP5q zNiW}P`Si)$a4yt;;uqFEVrV4kZ6q2vo8j_eTxHC=y<(I`8L>#tZgG@+pnuU6C}(C8 z#lK0AjA5~?K_!bs$zLdlIhDf4w>dD(8A0axx*=46!(0zRn_(?|ed!wuAg~5gE|sL< zP5WvH`Sng8K`?!+{9-3`rG5@4$kbuZ8EBBt4AT}dVo0X>X&#pl)FP#}0DS*KIX#H! zuOpy{rjeYA_L+M}d)Dpz6^qfpBv~OJP&lS!Fc@A>Uo6I}caG(k`he`M2tpAGO_U$l z*H(6F=^qq*!SXm+n_0WSou6rjlPSLO)2g;50lrS9gsvmnRPqVrJ`JzU*%Ps86>)j- zWBGE`ni}kGG|@3+x%xe%(MPK(JHW2?9Q@*>L!rA;O~yJ6W9Wa6VH+{xO%gKiuXiZ~ zQ~D?5Jo`7_Cv3K_jt}*M(bb-RJ{M-D6ux|Tc#vc{yZ~h#Ze6-o_AWJ@Ot-!;tv*_F zHmPI@)&m(PH#2MR7g}YVSP`b`Le@aj`z5&x^%FR#cEOY6;jVjfhbf6pqH(7Us>&~s zE7I?t82U)NHHMQ}dMYw(A25y>=5(j{)=#<~+vf!G7qq5Z%sswzvAhRc-*Ee}@Y$zk zlO)fjb7vC>#7Tfp1G@IE9;HSG7H~=4_jU7)q3Hb)PshKsON6ahnr^C(Vv=8>y<9_Qq9e&G1Zq~DU}bnP)_AEZ|i&HN@@ z8q&8)9*Ix5{vOq%-L4(k9e`fXxq*DCw?O(ik+JiVG3Tgms@ z@q<2XHyu^?7p

L%TL^&7YCO2sIG%jAfR8CmT!{=rFe4Z62*N5vS!5Jm#`e#rZD=6+x%mjS#uK8<7s|abNY-<7k!SaASKDO{^uL#G<`uBri{62 zc)hjH2XV|cjG|`wsUD~?xm*lzGDux9!^hh?hW6ne8zj=ZTVRgHC4Dt4qeMc4@i&zAr|rn;IW{& z99WOPjrJ&QsNt`XOxk;Z6_d;bp-r{J$nrcpJmR?jD-G|rZN5A! zn)+Cx?3(7Rjh)?j1#YeqoR;sd*6@-DzY6trdWYG?$0w<|pU)Obqt!3;kj#ijMdmwX zk;3YS#&tq7S(Dn&?%J(b{fJk?f;oa`v&;b=c)N&MW#q{D!`DpBvecReq(me3sP8QM zTzSg-OUB-w9yfaQ`l5?SUnyp7|G`}*#q?}^vpumghf{iA06%9?;Tt5nWc%-zu%$zW z85>izBC7u(vFveU!RT77y<`=?tH%wLU5(;lYq z_T<%BbDo6#5k;T2FQtwB3FY*QbF;Quk|!stRQPS7yt{;JFZ=HNcoDWIL;7`_CwAt& z+nJ^5h}6*dc8CY?M@0_7Ma$Y{DYk)Vd~G`QqP>?}YEaA{Sc6RtmZ8VXxQNhjXfX!&(Xgm_c0d0R!BIlPWv78GiGQl2u-$2emmDIj)wop0B>#$dx) zl;#yalCSNqm(!&|?V|B`z#BQ1X02m9W+&)_Nt-6(Lu`|qg^;X~d#3&&*jV|k%7moF zyJ9^#8_`+l4VWo;1ixK{3XlIUUYKmolcjr(@Wz@ghxn-mHDje1I+B<$p&xAI|md`PJaE=!)$mC8q0bdB5$O`cD=<&*e)%am;Kvo+XsP|gKHL$IrT+^*vLxg)=};W^9Wf03{n zUVR&*W_T4%{->RV@Y%zyb-)wGhXE&8m(W$U013n~hp>^=BAM-pkkvEO&do0X;_Qc0 zZjDh5ZvhoPy9oI_S;Kltevmyf0^~Hh^OB&0n3{Wtnv^k?4$$W@WdD>4#wdUn)9R9g zA#;Us2w9BkGDtEqDE7Ga+I@i#$rj8Vi)>@4IRzRQq?9Q8T5vZvWJN9YQc7@^L%)i1 zPwE5UGpCL;Af5Xj&=rs=jL)vv_)UO6kOj%&5$a&{AN241M>wu!U5nD)98bQgiF;+% zFZu3>A|7GBm(Fi26Gz&{R_TDq5#vBE@t0cd(k+res`r& zpzuvVobjg)yEQ-rG<%=sh9s}VDmUk5V^$IFbZnX!4+&VoqEVW%u&DSE0<%w1n>l4S^?8wy!EXBUgnS^f4A5Gc7OjI51uqL@Pr#1HY zsbd;=?4xjNOlkS!d4pkv&2tvzfZ^Sc=~Xty(-r;Vj>(hs7S9;9jpGpkW8C@?Z-4g5 zZ7h4ep>E127A8?G59pf#4~3RJaw7h^ZK8FU+a_A4I_zKD=G@_!6DeQ>hjEt#n44c( zme%HQGqGhnZI$ORP}=CBxF+q#M0m&HSjDYPa2A0QL&{nZ^qp!|M z7%s+iz85sbs>WD7S+Lq>;5L^blq-=Zo0gQJ4|H#@gDZO--!1foBJTl6FRKj(dd4&g zv};gOOW*eScB}EGH%)j#;w3;)3ADf@lp1K&dS1o%iW6CJR|#eOs)P4Qd|_F&q8hGl zNM+Jjv|@7hOJ8$?@fYZkU6fKHhMyz^nFYQ#RvkDh4 zaSl(%UI-xkw7Z;!o3k8}3P})oLA$HH2rbjrK8!brFAcDxebvtPY+SC9*jwlvtNBj^ zqh43s(Wwz_9*R?;rYu3Jg^K<5+H(c`!^D)WFlvtCh`mW%4px}-$EtmyT-o~sGEqH? z05u0p5;kNxe`t@!7b5yMJ~*O&KKxQ9uW@H|bj}*8;)AFmp+Zrs>A^VVGAcQ(<_t1| z7;8aj&BY|eEMZh)$OO;kecl7tG}^2IlC*t}WKhW)S_(Gt1!;_nyZ+oR(p90r?% zaW@B?Q#T96V8yFnHqJUGH1_Y@lXsN@8C)o_pemkv_6BMSQ;Wc~)NHt9muxx@eABL@ zruk2;MuyKTLL-=6Us|U&ISxNQqbI;!xO!*pbXKL1i^}!Br(@Pph1AWTUazRwbx+Zo zUf&+o`UHRZvrO-iVWQ62W<-&H6^SL>;(Xyu3mc)sjfXjogQ0m>#w{H*>z}t633FrS z;sOstrggnn(}i&0*)>j+EM8M8|~j1TL~7tSlLNs-*R`b7B{r}a)~ za7Mb`N2!f@%Y$=7)7A9YDZS84nf(0yMLAF`p{5Y?9!A5!pP)dktM0Z_oq~$j;Qf_|w=N_%rMiJ@L&m1X^@Kq5L8(Ho!9CFkL z+YJuB7}wX{5B6JW=vQpZ-rt!C#aoKy-U2SjbeJSA^%MKpnsZ=q5OeUvm;dY*_nc-a z>%%IejRb6ex|NhAV2z=79ZMP&wnnNyTX&${vZZsqOXYj|4TKjOY(X!=rS}nH@Uq%D zK2V$w=BcLvHxjXpsY{G;!H)2pBp?eLV=MW5ct;83xkk(rsNC|-A8)Q(g}$sa*96Yq zp}E7uV?e;u2x)R7P49vvs)Lqt3UifmQ%U)aKNQSBC{hG$5#1Pdii#&qHdi=@GL3Aa z%l9}~b$y_{RCGq1I&eEmx=y{cKMUU5UOZ42D%d4ROr)vGIPeL4Zl?{6jBUP#U_&)1i)>8WhSxBPQJ6=7oS?ePo<|=tJoP%pAZ#rsCncau$aAYN5bQJJF(#_` zCv~(P_}xp8Gu3nVUslLGEEsKFY|&3%Xu)%C@wc+q>(*r=IeczZSPWPq{8+jC_*%75 zOy^>&83`V49>ft=0T}PiS$1yN_n+FV$H>O6N;NZwu*Kn{uI%)pBsPhYNXfvv5 z1qe~cS=8_Z)23c~_4`x4r=ffvutgMLUC6wFUCq=H16O57R>yE_F8y=Ro3nNiL{cQc z@rzZGYH1mYW+95A z+D*n?qUcHFN4H-lRTHh_`QtVT5x7;P~osovJ*=Dvs)P_JY0E-yHtFL zRT1Ukm2UQ3?GTE{`(y-Lg2b^(e@t)84Xh`0nn7#sD5nS@V=PNAQ@Wq+D9@2@t!o$F zV=%3GQXZBFtUm}_NGdnhS61*a+Jl!`n-GJr?Ozz4^$w0U;Whpsr$RCvCfn4SImA3N zL>%l{^Df?Fxan24jE5sP@do~PVT94zB3A+V*as~#C?zdyINJGbg&C6PS+-SZRj+kc zEG!0F?FXTDsHQp8d~ur{H^I5U32uz}Vfy!NPi)YywjW+gat%k2J1#io+nJZb>m=C%7-yG=b<%D-j&iQe zGuMYmvfg;|@rqzZKZL<6$1b4HNz&8j-Qspd2raRmiX{F_3!7R}K~od@L?oVF`B2UW zc6NqIJ&ihWuvA3DKWAA`lc(X#C}%QF@s@G4f~3KEr3<2=eTJcS zupYtwy~<|{%4Y*6lU9*Ka-vj7HteBS&Q!iwn4py{z98}~xn_aBHep9@&n?l5?qhe57&C`vj=D+87sXS5M4yt{ znueKl0eg4~DTS0lFaMarJ13ffGBL@79Gdo6_iIMzTD~P!1G%+$ut+OL`dnIt`j=BK zxAyz)s*$_rq-2V(hi+?;(J8+S!AVJ(2V)4$6kFjFQsrPtv*RmtAVKiiA3tT&5!Z1pSy5SLS3_#-7|F%aYwbhl zxX2kA;RFPf^#;dTOAVTb`A%zquUY6XRTpxWSR1n-OQ4I#XP9JH%Ol8F)?I%!s9-w+%EWGo|}d5ksfJS&m=24TtzAAE7@7 z^{X!Q3xZ1?d1jTDxU4@bzns(Hwlo+{CUnJ6cixWI%&k;SqW_XjnNgl>)KXn$-D7s5 z439O6Wt6=Sd#9k_BeFubxgd$-(WrmGubAyHp)c;%Zt1-7Gt5t*Q1Qn^}&tfz|nrzG;Q&nMllFD7Uini-Xfv4k%jt>h_BE9Sy7_1tp4U z55@M=wmfVsiy=EDrK-aXGtc~bv{6c&bOGCIP(WH%>hxsFvwG+EAmy6P52e2PC~%4V z(l;Wv;cn0aW4RQ^u6k<%jdFdMINc0MNUE$Za#ciO&92;y4xPM(8GL-icEUCd5=Sk) zds3LpCA1UuE?lv3;|RmmQdpq?Jl7-O4t5_2!gAqv?rA%EqxRRgVX5OIy__{UD3_q{ z5)E03?}}50id}$>_Cj;{pV*-{A8RIaM#H~50Iro)GEL8Bl+bKr;u7^^Z_?Td*(O%;|~~&>CP=8ixqMpb%vf?iO}GUU(b6YQJe) z4q392$fd$;XsbP{7ie>JDDMGnp&f&P*|@@0r*h8yL?Jy5-B%tt*Ds6s_Jm}2U!G2? zK(O)Se#)VKXlC{(pRiLgJ{kl9l5zR7{OY*!d}3VXk-(f&vyzd(`^``U*FcQ@3#e149@ z!(qP~04Ln$Tyim&pvU}Hnzko=d=U+0&C;@n!Lf9S{p5R6Lu;08S+S~ynm=WZp;$vO z9G>d+tLR+=%{Fsm<;mTNZ5(e1CK!5|9D~HjmywyVy@MvRvdaXw)uC5?PVbJml4X%2 z=~*||K>o4e*b9`YXTiDLZRTMJBnAMVEj*=1j@mMjazQbL9$K|J(%p%({X6I4L#}HmzF}HDX zs8+PAR*=a19JoC%wNBZ4w7)i@p1XK%3w0`BE?&*MJrSM5OY}T{my14-1S%N@_Vk5j z3s9y6?D1;OqMIIL@1A)$WzsU_;Xj7X4kc~E8PGR%6M@FT$}YYpI8g0G`5R#s2WVjV zUsxjE!W>=A$JG;uh7106(j*)?{Iv1y_-qsNN9$>ns?LV^#viO_Sy-r7vYPmz>^e9D zR>?ZkIJDHoGfDCx@}bUNtW0^(y?7HP2?)R_;vkw!ltC$zXOYs0V`mbbt^hLFjWMU6 zyXs+p72Mp?_1r=+cc$n4>|0+8ULBM021s&T!AYg?R46e;SXVW&_S;j|WOv%_GYix5 z(qNR?I1obrT*@Ux0eH(I)sAWT&^{?R(Fl7Lcppu04VSz|{$P~utO3Y0+iU+u?(un$ z>a&LNFXZt&6Fmur1Xep#fD0{A=E$N(%LBCs*$V}wpr-Q1({l;m!4p%rSut zk~pZ(=t&*)3`|~+4fRfuvxpS5m5y@y1JJeWafxr=MscX3+Bsn{Rj~$> z96r>cuC%?_A5(zR5*-@pKG+2cpN-YEG_nWVq%DqYq*coUUX0&`-A#jxt`%w+?IllP zBRm#PhhC4cS+k5*970+#KF%rc0?V8Wyyg<9-5K>&Tk%(s?@Ul(_I+Y9aF8SIj8%wf zXid*x`;VYyuLQp`FaI9-e!;Nn1lWi$`LDt!W+!jG{7GUlTlGJ|n76DoK<*onfmg)E zFL84rc%pcX`N7b5T|03j?$d}9A@YDgc12OQWj`DC$0@1^`ISq+SHk+!il7&VbXA$d za#m^t9D{v)I(0pY=nfsE7FAvF)s>_MTXlBYxB z-NNSOvP+$}h*jgIAK!bjs-tZW0Am}`ieg<;elhPB1&3{7a-zPdg@17LliH_t26fgJS*JA=mlMtLHnoXXqZ|LJn9aV&((mqs zYS{=*cih^!sYQK3_G;d2wb&&h-Mx&@K$VS!j4yGw(=H7CFt_)>Btzni@5GUzERLj* z@kX4 z;lR&xpmH^<*i|#w0+-%CszVT}61pHDm~rGw3VY;&1EcVU6RHUDTOP6Lc1h9+URd{5UD#CiVy}rIm~(Y0g$(6JK!Ew(WF?bR(?<}!;?<_FYNUgwY{Bge)Ml)?35^qU`>a51)`8GvM3|1u+NtEc{A(H5#fl3=>)Oq zM}BrJ(heByb=^99tIwVTb6LF;A@}8)yCyxc4L5q#KcAy_3NKhpM>jYW=H>Jf*N3Sk zR`G?K2C|I8|ELhnRS#361Y7)?x?P4_p%NgS6canCT}s0kq`a)u-nY@er@A5V8bvDc z+h*lHWn0Y=6Zihn`10A%YSjX+tnEc0Cm$Ee5`;!$VPzvtpUTU?;H@HjxQYNmdPK2j zDlROs(z5S`7mmBL?kR>^#VZz*FxYA?JlCY^@WmCL1%l(|8U<55=+n(@{$v@>D(x*? z?hT)~h%NGVf%6?+i;5bQkgWf@ns2wozUZ3+T-RB`y>;{5c0>~e%oTcxDYeSH@IlNS zPrUemG~jk%Z9N)uzYt$?%A?xSaGVdO=4os7$gbIV57m3N^-a!@WVf>oa3)=78Jg-p z(QleElPc7zR59d3W4c0VLc_yGF}qpE6{=2mcrN}*?UC9VRxsT$Gc>xHy>9U6dv=>A zHj}j7ur4`QuWk*^_V>lP=KP5?sb>8?_k!I5O8)tolr;A{1iRsk6p(#F>O4TPK(|9j zsMss-cFJi<(+9%3FMu2zC-vhQ{QEqB80qcFh=aN#-$a60rCFrh+7etka zVzaQAq6W^5op}L^RRa&>l#fwg2f{>*x+KcZhk|Mtbql*2zY0Hki#q`72t8iQKGwAk z!Y--)b@NH5im^80*REm`x&gRZ7U(jEdQ2cMEYk7l?WV~?oP0*Vt6tIrzh zApzY@L9@XVLfm52YF@d9|+(zL>no?Dz*o|?b0>vXI@)5-u<$JV~bHLqwbXD zfcSj$V0!cZTTXR#OyrRG}cm}6l)VI0?3g~=xQ5^v7KzwQAv z8GEnkvoFn}?j?PGqbYUfSi9SFH0E^8Lm1q`=gfLjXvB1^V3d$g$Jw=5p;35>f&R{KMVT4x)Rw(E)h9T#KV-@1;>+sJ zJLsUFS+G4O*DUd{kYfm=`bL`fC&eX0G2*ZO%>Ky%&s|1BMNpDgjTx-0^8J5m`KN%~ z-X8Lt{6aey?cvf7xVI657~aA(mGgDDp6bu;zMy2*7lY|~@MOjPrryg{G-IU0>6pW1 z?VI&yD^)#_s^>trNymMn3Hn`ybvS_|^TP!eA65`G8b-bgs_(whVwCP#yvoyd2OHP1pA)I zRjqVW4EH)4?LqPtYV0x;lun!}43Zt?NFykADZyz_lf|WM3iT|h9w~~dW zJC2$k4Y7mw`(8Z>k%zYqB^b<_UdRo~8nO-REi*~;CEpgB8-J*=q!31W!gf%&U%EB= z*X{d?6|#5g+YOr^sDF{+r=GC0_Z+3@OG-xy^eq=^GZu^}FTIl`y#s0i$)<-^PlQ6> zOgT!pa-WNFvqVWQ&!G)oO5Gx`v-k5%KM&NMAM=t(Z)i{EHhu+s5%|kET>V_$tt~a7XjS+(_{?S9~4}UfxK(V<|NIMOK(c zf1OXfd?d7`3F#Z+2Q;>&u^ zt`)gIyI0I8>>Kk$RnJb=Ja8H-fD;&po*2fHc!tMF>^{SbexGUP&{!bSt~O4>E+E9F z(A`7Y0`8Z9X71mWv-WV={VfNP;h&xu+E6X}vcn)O8sdAZT%mD9;*EfmG{+S&Qslw* z!Tu!T$PN%k>q`p11U*W1h*dSx{N&-e*jJl%dvBm(8+snp#}&Tz44}NOOw}hrA%y7E zI#wanZIT^qcA+xVxpu$lENryR6V>Iwn7EW%KV07ZCC+1+*A|NLN6BPiuQMrCgyMrC zr%NyVz|NV1IUq*-1f}=q!<+9XZZTTTZ`>1A|wf%N~0~;f^ zw8pJ_jANg8iLWnt@8`4b&=1y)5RJsreeV%7{C5SLgrJI2VhZTjrvwl4 zY5eWZp!<#MeRP=PH43=Qx1+cowX*eSKYtSa=SUppNKeB4`2(d=x55^~!;-Y+Pk^8+=c#(Ea2<+G416tiTCwhjKs*S+V$IX_Zc$i17&4fhf8$=ZRn;<6K zSPdR$ZG5X+7!bXzqB!)-cQU~D34-^$P3AR@^gb@Q*)h903F-c{^BQfO|5$F+VP|7xO zUN1Y{`iA`$!kDRr_06^?3ff+ffP6;=wxb3*R8eUbBq^V2>$Y_I)`TjFZ!1O?#v~zM z)l)(T_JAnD{cPY^e_UcI)R3{D;b(x&I1oHC$0UdJ_+7mqtDvk%$6pCOzu(qUM^jfg zcG@Lob#Nr^KyU>7+Vuzt0xCgu7Zcui+*QsZjwUs2IxX9<9oMG7n$6 zwq@A+$|L(OpYB=am3BzGeKaKZ!B))Oyrzq!2w!aDfc^Oe~E0WTCo~P{PBt@;*&;dKIUz)j*O% z1#gzI)x1v+PaHiQIi+ke7 zxz95htH@K-nB*XvczVa(@B01p1}w%BD>-f(lQ$UgTwc5o=Lf+qYF z=M&bjn@ljEiQl}|CH}*dH+&$+lAs$XF&0QarSny)cEqLUCgJ=${fp0;2yj59@wFSh z+2&!?2?N2Iq^;aHdalq$Og2KN-~UXKnZ6cfpwc0n(N&HQ_BkokVL$XTc&`G?Zo+&U z@pE>`F%QK{E90q+{nTniEZ|K?VCX7Z09qz?051iDu=waG0GW zzu3-O#ig?_cRlT)=k?0N^gyP?6`uD4;hCi>9_+_-H!%8@i9GE)QQ?03P~7zwQet5f zMS|@?Ulkd5y&&rc)6rt0Cn>4%_q!|3$e!#jul}kK`EDx(!5%vl*FbOUo@BFl08Oec zL)R-6hW|2i1RNg8?`iP+oT{jG=Rf5MIZsp#Ddb=qQ)zRbamrac3fbmv{JEN!`O|}b zje#~}Qw3CQVmxG11C2+1`PH8qxFZSSvvrmTuj&mX zP(?yILNXODwM3Sl@|l%N z(!96_=G-l5Cp*xReF>RQ|s;+ zJ4`#G-y*;02y)-#L%A<)JPljgDef7y_NK4dQ7>@w|B=~ zWUYJhI2gY|hGLr;8>N>13VsJ~MCZF4mq9Pk56V~2;x?<00~u(j95v1ZF8R%v@Og#3nJ&R1WF@li za*_;7l9tH+*sTqAgPF%90UUPZP+!%*t304t!{}4Z2_nTy?hkz@|3jqMVk}c}PKK0P zXIL_qX$_xlu%kHhR;tPI*RzmOm6)2rO%|HFo)s4J~a)?mD9x0r(N7& z&2`Th#LDFqlQh)&YpVoR{cOn>*PzLQdYV}TV=+pfif8CWw#CZZ*@df381W5GyRAi) z$wHsNSzP$~#|lS4GjUFN3#x*8qgE56jGc_0ZU&)hISCR&?eV8(A9bLePqxz9=HtmUgv?z8p4t$DUDUQE8B3W4EFdG65BQ;o@3gT^mKD& z@AV<6Gsl(DOTB8JaxYUfFUc0Bakvg?r3PL!u59vgHyaJRIHBuLs34CpO z+ zIw^_uLwWdS@q0%o&m>0_vYx7}HFT=1qf9LH9!Jy*!TVHy`N{C`+eUt&e!vi z#bB_mbzN(&S^o1kS?en3?a2&0!$<>2b7t((>6t@1idH$?g6ig~Oz*nfC(TCkqX}nh zK0P3{o^-FPvT&ALsDCuX~6vi0XuHU8=nJ~;9-fL=+F`5D^2ENAl!n-J+sxU@AMpNN$9 z!dyE2v19DG^gsH?`|nWJ81BD6Fb1{XDnhsO6ul#-<@Pz#q9iSg>rPAtD$1|AdBtuV z&Fs~Z?0VaD=jF!MiVoX@)?WD*9UzoW&ql>q4%Z>as>Zdnf@LNod>;r`y5kcBXy3Z+aK6n%Kw7 zLt?)Af6--OBS@auGo-cua=X`+e1EjBQ`uw~k?LH$-tN8=-28l~xH=KEj!Uk{l0wG4 z6H~uCnjmbOi4DZ_kGYa&D%JVhUnykP`g_jNaK=?65vDEi=9#0EL2YkGxr+MfU5Fpp zWWLtS(N=m38hvJN#r<5259k-<4oasQy>`Ce9~%E|Z=?Q`-k@orPq$oo?9U%z*_f{s zoo|fw8zQY|-NVtS&?EF)lxjKemS}f+nmjlDvv}1ca7iYv>&`g=I=We!)7$5o-d%4s<-K}oDpvbPdFG&& z&HWIDV4u8}^B1LURa6QEq4Rr3M%JJue`Eg1NrE<9=o)i0;rZ7&L+S4z(YCorgY{ct z)E^#8trl8<)7EWNM^DQh@hFo-GAlMMzOyZ(J&bZ zQHi&kMTwE&uvGl9LhAaDm`#p;?)&xaH8Oz4@J8g40bb|61qd zSmU19CvW4py&e!xpAiI&l<5F-@5GfIep9-cuCmwmQ4yU;+W+`6^Q5nFU^LSw-yL^; z$$Z$kYNJ4=PcpL*N{XIyKEFR_^Q$D@@P!I!!?e8f-M~tZs$#!&=dz>v@Ks^_a>~Z) zyFmD^e!L9R=n2zYi08O+=5Btm^XX93G7vrp8YPcOL*?~@3S%bz(I?#bhDi|lnocsrttI9l#64KW_F4--T>b9i(aoHP%L~S#NWR-*y@>n|O$c!1%Z@*5~yL387Q$~W3ERP8KgOORw z4O$M5BN(Yg_j|81znB-B?r72mTg!B-a>WBzS#O)rTyp)H-o;0m8_*$8QYa1)O}!$f z_o&*wA8>p)KknAf^A4O7r_Ve8eqK&xyl5u=fGxvq-};w{bT{!;PLvc_xe4zTz$XYQ9m5C2`J2e$id61R!+WIzfCVrzNbu zE)DpH2;fYk@twYU`e1>+IBLbR6{kGuvsYf-KfwCj>z8V5rxg$vptfgOAg1#@MY(8X z&y~kBcc7?o?;SM-_^7Tjvm>^swf)Ar0V+4Pw&O9#^94qir-UayHm*S2ddNiedXdM* zX0>k-X)k-(7=M{HAk?gBeE$3fE2u>8bfvR;J#J$zl}mDCFU$a98qYYXZ+I78c_oOk zRrGY(o>c*fe+opFX``H69n!ftXniLKXf3}o`r5Vo;*<9Y?I#UK11D{#9j)RhmYHD} zk6YpRPn`{!7Q1RzIP<#ESI9C+)^(~&kSws!Uutikof@3fxMX@pq*srkH`1tIRkoDxlV zJR>)+qgc@E>y3o~@{ZJ;RA3tqjy!pET?-vI_wqBK*Ky2oN~dxlv<0uF*Khw!XaLt( z(7&k>WTwPyDl?frB9oTz-uTbR#>0#L5(gdPws1>Wrvtf>^K|FZ%@cA*Dw5ngO1Nr+ zx@nX=-eh#uAc>=Ytc`{z(wSP=j*W@ABrb_Qt(RRU#-rVEo^$Bc-KW!r&*vCM+f>xI zJHwAFkzvf_Ov}dJBH|y;(3zyQ$(P!)U<&U1yT|*^`r4Iafq9{^O*@L>S8IMU{cfHb zMaL71lVAjX$g*xer8*vLy^*#r6h7Xki;W%{Z~EaF*&L?fHzvyC40kbGy3$rV}AALDybVD`n$*S|4<&wyyb+n^Xt$3 z)!(30S@jQ<1&Wc9biJTVv&Q~k%3BmCH68e+vR}%dO``-WSHJy*>q<4Ds~=hf>Y0J; zExMJ-s{_c5rJ&T?ZSRG;e*69WHCg6zgQZ}g2vS!8>hC;!##kF2C(W5z?`BahSiUOe zGVxD^!JkiKpV~`uBS38oH=ndVgXsiR4dwyots2FT`V{$#4(E(&emj~Lk*d(*`})!e zk)_I*6JuPfP!Pe`?&e0tyQ$x;4ZWx3;6_v3c_IE?ZmT9-%_8I&G*;x?K)7c3M>kE(SewGMrM8;;-}>bKqhz^>;kbWJ&f_-|2tZ`}b+F;iUXM z?TJfu;Auo`;Jj47C^ROg>+b*l-iKD>Nn9Mi0;3MMWeHxU3kRz#{1CO9sxgUlr#>XT zEy9slAjf|f_Q(=)f^@L6#P*!f@oD+k+Ch0w%3&aSL4e58Io@xgMAw*C86J!XwDqqf zWlm!Hj+5;x{xp%^lGsGx|Zs?;9!#dLsq@ZJqx`(fOYj6+v{CB}A(!05bJ} zxK(BS@4RJGHuF{+ii~)4#CLGJOIRcJM#2V=m~Y9@`iy9hxVMw^l;?jF0{y4EMp2#B zPMPl8S^cLw_)pgrkF4 zt@wM-pBFv5z)H5#-I=Wc|96Y~d)*!v>B+T8F2j2UH~w4Q z;s3no|9?CG-nsur3-SMHJF5u`bq@gKosSa=CQB89@Z{!Z$!T&p+^(`&FK3Ow%!V*< za9Vi}&Cf>ztu4t&n`j20DrlRkwWXlNou~iSX^$g3Bp6jVI%DgN=ZVN%^hX~56%!_$^y=OK4U5SgM4x>Nx)4{^ho3#}EjDTq^7MeTI%1f2WL){Ohn$PIyg~$Gw#gWF*>faC$y$CcHp?9jF1BE8x7=Pdl5xO zQ=@79d6itfQM6UOd3%7k7fNYwLr=J~gVKP(-p1=nf})g|ITRfFrJxi#)G&g zlsHw2;CkSUbr5Nl@(+f)}vHdKGI)M8k>1mIGl-3Kbmnr{sn0hq`%a!EwRMr+b+Mx z?iW7FkUOVu_V=7kIK*zg-JwyS7p|knt#lU1 zPr#8oeF~j53K2%DTLY=%#rihg#L{CjWQ}*fY;5NIY3DP!1IJtkauxd_1gRc z--3%j0gxcoJS6~7Gt}?qwZDL!J8g73T{=BFva3vM$l40XW0z><5 zX*;0KA5yD^m|8D$oPs z&Op4s9%UO@jkDULj92(MiQX?PGeWEyW8Duv$xp3kt{qG~iC8B?xshdOkfM&m=doK; zN6k$~7&67Y;%e+vS`Dqr#22jMjmzCVq$2MDAHwREwoms#ykMme+7)$6=nb63@q+oj#zxN z_N_H00?_k2ch!#-I5xO6j(5yN_*#OFAnH)7dj7PpXeH(TpE4$-k`r?7uok^raL)%R+z6S_=(l9~h!#xG zIXWy^^|O==Nm7?NO3#QNMeYAdBtj_~35jxTij$E)TtV+)*-WDyCldq6O4w?CUieu? zo?Gp3mF%5blA%GoSf5)mP9O==FBo}DijEf(bFX3*nPYa!{3G3+OMl*03 zNblo~Q&*(k_r_Z#E6(ojQW@XEm95JAB63QCcO1&VFONW^IeHThYe+;I~l~39JfrHsdqcBTg(0jRX#aoeZ1aG%a4ND9=`)G zkv5Jj!0*I*-8=ZFWsSIk_(GbSR1+hv!4`$?!o-*B_rAKPOHaRv7zI+^$)aNmzWRXL z`UzcI<17|V#^PiWd0Dn`XZ$ZiPZ4l$v~ATuwxIPOp51>Y+Gr*8|uEtLdMiWRLSG@15^ zXQ$830{PP5Td*r@Chyg?@zER}bOX;@drtt^icb(7o_anruL-fRsLIzP*fkPK!;d7) zmANCZqX1# za6wa--8qWa5x%czUxpx~ZZ#LT)3b1`x(0Z_t!-Ec=HA+F+=mjcFAXN&*}|FGh?!Ff z-N|_%dv_kc-hO)Fus*VqliwlfW|?#+B|YGiD+gi_pZ%0|I%IR_!SWkP;ryqncioy_ zHk7%KAv9Z+u(y*mRHC28T$XRGEDy_zm`;MPUKZN=VNFT-8`9@bf6H4qZ(>2A@v!gu zL&5Pe9o{k9{2AfQ!8bOtCg~{)O=UEi9LrA0A#WWr)B<38Tl}$B9a8T-2B+&anBa+f zXY+P`M0_ZGnZ2`<6AMgiibO5(?7r@D?oEt`_LL(WA{_0@I-rP zY?zr5h_w6qbLsfPnfLO+<%lnOX3}}H2i$lv^5(X1g9VBT3@ux$d#k1o&l#UnZ&i=|oaC!`PYBGgdnA&8A9r5l_X&`_v#ujinM(7TymlK&gkP!>m zcWcgs9(jTcLdLnWjD0nje0s0%CIZE>{hTy8PcH{KbVesB#X*)NOJ{pRWyXk~z@>`v z?uj!MuAPh7#3}^ywSVub$H}Ex{jJLc0Fk%(D-z_ zm|!<)T@cb}+|fFAFV1CT;6{Dh9{hFyyK%JIVHfUJanrNIeLD|gsXfMxT$f1_50z}R zk|?%<6I_CoPOW-TI;>@P|+-3L-fTmfeR03SK()YO5J?F|8oG~8@N~K#i#$D|SZ1Dg2@?dWX zNGhY~ri>P$r5jC~k=BQ{Z+A)rKeU|dElroH-<482p?+sF_wzcyD;G;jN| zrm0%?1e)15RyKHWM^<&Mvpy#w`s=WBM|jOJtUD#Xk~f#NnLTLyl>*yd^GUDdp%*76 z)@28vvfVBKrJsla?a=c{0r{$%C-aE0J{xh(Qy1O_*8apEH8RmQP!cXKP3fXZW?P=dz^bcZD}20~Z+w^OUy(*LY5gYY z;4EwmmDIhIa&U9%+>A;-n}D@@+`Ga7uc>wTF0PWN16`qb=Q0_g!@;yzAhD|dJfW59 z0id-)n;-0epR)Gk<5%M2*9ZsH>o3H}em3&;ngx5&0-cK`F>2H!QoD2WS1h|Y{Rce_ zY+f-?d2H9DJ@0CXN>M zYE7f()oOY6z9HEVgAoA`IYp;jAZ%{Hy4lEM<~)$J%*p4^ZJ;%z?b3aFKUkp>n_bbM zu)Jk&OV2H5IR1wEQZK5i`k7@a6@Qda?O6Ow(c0Vt_da`mw+YWrWm`kpMlc}Z)5j-)Ully1?yO%hseVfp2Hp`kT6HYbWUu(Z|4?g40L z9#y$I%c=vD9&75l8nNX7w6&L^npnO)pQtX!hug(99xTdvGr96B7wnUfXC=9`P-Z3G zR=>yeagdjc&Jl5y9Kl>vyER{yO)a3ZyrqoQrP|e-;(m~|AHI?9*f`R)^_lrgSiorn z2e!$~(Y=O0=B8E3gPXWIsdw&i(*10(_wKW{*0SoQ34r)@OOk#x-q|(|{go@iyO@cT z3XWmNzUtk5CXuD~7EB6|Q2;LaB38K1cXWNhL--Ve#>-_)&CC|2kHYbOVMpfa12zTLG{;1{mS5K6L z`E<0w@bax-G5A(`D^*Y=Fapp?xWX`_U)MIWe1`*>dSrKfgY_WmYkV7)sO?G!z%!6^ z{B~*Esfdb5$@g`y5%?DIjIB>bT;n_uW3zXej(B7Sr$4D(+yWT6{+M~)85{~TXBNx; zy3Eg%t*vf0EYho^zPE8jeTfDE-%8fy zK52W_`BZeyL+PUW~wvpIf)drTA?Z8Fcb(<4jO&DwEe)k5Hra5C0_X6ACr*}Ff~396ZBQzw60b^{y!vPsokbdZCE zb-S)dQsgbFW}>6prrxK7YbFZl%qGv@W&AG|z%$P#G4h!26~P+XxDYM9aB3&MrPDK6 zG}}vMvFidhkBkt_`Lrv`^hlZ%N~Jw&n{OZ6Tkl5muCoUxf1p$j;n6J6csLu)@4h^h zE7oN~?HY2mel1>t@9r8XIHxwm4nRT-HH8LIygA|!@_+Ef$Q8xewl)>Q5Pg}*aJfu1!ZmZNRKko{a&N9QS+5=yvncXJ7-Q3uD&6)s* zpXY(wO`4}L$6=1qEKT=yS*{grQcv`qn;ci0=m68-gOgo27@XDw;TqLJi=kY{o^ zUR0?nT;`NIGB!eBzmHd!rvhf5ViI4-oOIhl?o{pO1HINC`7S~YkIAlEz#(rfQhy=w zox$)MnaZNixlMyI$2Vut@dhH$yu)uP569mEtM8hW4h%8{O|5bQy?vgmkkNcg2J%p6 zTNOz7#>)c(E=X<0Lahu%1iC~YHxe9+cFhfCgL8GRD|G9+o)Jcz)#*kj9r<)y(S81v zHcU5>vAhApRme`ehjB`}L1eZPbovXP=gI#19nk6&TRj#S-SH#WqP!1puDYOnVvXo5 z7?}3&QSswT$-Vdwcl=e8EcJZ+EA(Ly@di>BbX#>zc_xg1ajr-)5!};wnN+BawIyvu z5H!s_(oOFt@aT=111_i>)jW9QcuD4v(y0Dxd}VtpaYg!qW1vq@DYV)%T-1l^5ZWZ>;H9BQ^@6O2J`vQEYUt^kyHy|8H zTrgjt+@cVdFS~HC)mTh&E}qfeF>Ma*@3#5b#JEI=m{1Qpu@URkjBqLy+Ah`RQX|ZU zn&P5&DO}`5PIJ|dZ;mcg#qW}~!(!{7CPpNbWdRjG0~Kxna@8?=`g#0>CfZL##;Hex zs40~rwBO!>(s<4Oy0)zx;VtmiW}*1x0M@@0{n$6jjCu1&zdjZJ6n@Psef=gkU!SJK z%>z-|j_F0lEpg08kaujw({|#s``MXj5z|{scjrDVhpO8e1;IRA*l?x;P@Zk zLFPQ0NkJ30>&E`J|IxusZbto5;rACy$yUfE^2Lk04EpUIq1 z(_nL!dy|Kf)od4!PluA&SgwcEQ~wAtyjiNz7HTn8>OooTz-e|J))wy&+%`?-EvR%C zGO4p|uWEC1zoY4y)0Wj3jCD*+U{$BQO;iLbg>!DuEUn=6mFvXX7InO# zk#tpxlK{Er;EEd|$AUP>&bJj2w-9qX+*N{*s2WT^l>?}iH7U0(>kvGt_z0!RSZK;b z9E~>GV_A!xxPeI)4jQHpjiw~dH9;%$TiynBk)wB(qy2?!pwALby!SKW_X;hHM15$1 zRwnFabc25|VzLxw^QC%j00R$S&)(2>9GO=9WDy`^q$n>s!Os2UvF!A$H*T#+;>V*D zx48o9nU&6UfJNv>&+9UMZsRfdXh7q^P_Y{ahmWjE%-2$WU!C&|W8>6QLSu@Cj&|-O zR>s=Ah^HcYx_dwH+}4%o!GgnHMFn8XDBRAH8+Td3IoGIMYZz9dq+3>Udbzc|> zsweqf631_JCPmB`TOQ6=3>x2ZCBWykJwKZP0q+iPMm4jDZPyTh@GG9@PZtwliuK!| z_K$pq78;41sO=Z&{JQdhk;6x;SVjQMxT{6shslXHlR&g;*TBHMJ-)jWmXcYtwR!kv z5>IOLcUsdWhJi}w+bf8Mr%tW*R&$(uA)Kc}V?4G3Z(FG>yP9;vD0*`S7F5YBUmA}2 ztFA+HWl2%f3XAMr&x5J%kG~?`fdm^4zIA*74B`hx$7G*)h?nC~#hzbc;jUA6M0^*o!7T=dDQmHK-f-SpobHhrq+uf#f zhDBSY@ucQRH$WeFF1~M=k#wcii+?Cl(LP0XVoJ-;LqUF7moH3exeF#|vfd{INsDb8 z-LiMfF)oL^+Htt!{=SuCW1`seX*`G0I_Z;p>7f0#(0a5bEY||dvA>*?k2&GN4otgq z&B~}Yb?>1EMN}%rs(!||TYG8tXS1jEz_S}_VRmbkF)I6-^4J&mJKmwAXm7@X;MCgi zQrqq4ol6_(7N%`)=KV;Q^Sm>O5ttz##&{>R3c}s^hupLMZmergJU@5CdNSFJH2ya; zqQSXjc~R|6&s7gM=1aTek*II!eo{Z8)TKXnOs&hTap~${q9uV$6!+O5)FzM3O{Yyv zx*Q@_PW_D1NHgt)ag+G$kIXESgfB%Hn&5p(b$TV&k>M2ONvx!%JskPP%t=Ld1bUBDk%WmGa9|TR`;HtT6hF zE(&)VUe{TT-#jkSejC(z^4`KFU0HO0tTcHl<&?``U-KK)rjev>ALwIMmZ;Lr z(|ysMAd;Qy&h=>!Ky*FQ;AXUMvHlg0ShqL!p{Lj5J<=P>kaC?Y<9BTdT6LX_2r7>i zCyH-j>hC^p)=c;J%*e1=P$Xgzg#cswm+C*Feh<4D`i*QN4lg!nPet^9Ar-gUS#D!>UQ8;!Xc% z&S}=b&!M6yd^)CkCAw)vMTR0E4lksBw>f%e{bCT|K_~JuzjI0ha;T|O_GdRVl4O4S zB>)jqy<~QZw%UXAY@2$7C(Aehbn&sUP(EMD$tK2U`yHX}d|raf@2PtEce6oCH}G$e z?8*Y6=x{K+C%fbh7_2ozY?fabULNkS9_nVWM4Nhu6-Xsq}aIBD^I! zbvKK|If}A+mKHxP|HvsoCzxW+sVSr1vF=4XVpq>V`NAr_dy8v|1kvJ?B==UTb@<`= zXAl}|X+5d8$Suk6A9*`RO)5O&`Ixze!mMaebOd>j)(NDgoifGrXTajx*ioq9yIFyO zdjJpV6I)DC7UC|SJFn~5u2DE*2bi|}&y1SPSw<~7JI~9Ux?hr#QlU9zR(DoZx&eF3 z?|yT<%{`l2z!2u{f}J=F})VYAmpOB*6M|fZoxrcs|UJnH?{d)y|=tpzJH7YsyZ2o zc2ic<_cqv;<~t?Nnhf4IDOck$)S^Vfl0P@~m~CN|`E2frCGj3{j!%`b2R8_RRQB-U zD`w%3Q)A0 zN@9n+==+=-#9g>0KKCU-h(yat&(>{MUm)JBuTGG}LKpm1O~%5jnEA<&jQ;Lhz%7}X z@vfJ&sx5BoV2d$GKkZm~+OXALAXr^{b8tVq?0Ha6UvzO{{n7zcQY|6zw@qD96P<4* zTk3abh#(FZcq9X;_m@%c6FQPTSOKLbBTPx382JosBNpT^6Y8j4-O=%%mi=p_scJcJ zY84ydO47f>J;ciSQ`ZTwZ^)8KQbJwC+o*1irg*0Jc#J*v*OZ(ei`I#dY% z3!d0oFQ=1p(++^l0kxpTnOmKp#j@`oVe&Ph*QLythfPK1E{MUB0iaz&q-@TH>QT?O zqB&dx;GF}yb2`~<`ICxvmklw5t}Pj1igS3E(xr=A2Y`tPQg*9JMP3R#?1}+5A6qAZ zezr`WRoVzcjpMUo?{ab0*nkKZ=0UC_F!fAp3fhhp;! zrglaLyQ+e1(;_T02dqV|9>tB<2uus%#4`O}M(^}8N{5Q1V>?r2N4Hi{EEcipQN!aA zC|*l^yQpN59#i;6mbSIBKqHbecmtGeRkavbJ72Jh67?pVZN)2P4 zU?V0(L6+1ISY0(`uW6WPpJguCK17w{0$Lw&CDiGQzI=5)d5GaGyXbyVq5>E((qEq= zY2E!hU?H6=on))xA$-a;^GU6DmfF62f;}Tv#@^L9cuvs2-eA`XBz#)RNE*HVSEZ=` z&q~n%sBy7`{6&F=ztGUl61fJINT?xIibuOq`>nlt;UIx@5b1CMcX_W zT#bcyQv|24Vc=H@-*plW&bm78J;@hJ;3&NCjXGNI9NlM{S4no5n@Kj0v_L};Id(vx?QuW!|QnCTL2Ium5r#ltqh2y(| z9fR@p35@5LVK#9=;QS#SEn}ODX~OT9HgJvjDBTTw6w)bqWYgxjW&q)q$3B}o)Vt!tUY?#%nJdxm$V zRf$`z$yRrlnpis^=TBhvtP78Lj7Zbv)1QRo%$If2X6m=UiFxzpmC({NDK|5~$6pDD zomj>HLeuZQWv`s8?UR{+=$SYr?>jZM!rtt{*3q4x+Db9AH&(QP(hoQp+o#=kZSN>R znkjF1+c>B?Fn8Us7PHem((ahWnq>+x_0%>u=J3{AFzGgpCn=J(wwrc*S+r5A^1UYq zl4zGo*aelFOuf$XosbBLBPe|O-jWfN`=r-s%ns$+5**jZ%~Ik0n{UPTv9~)$p-MEn z*JKu@#Q|M=EsWg%#{HHzO*8NKuXw`d_6x8e6KCC);QUk`Q)XOo`IpGcZEyGIIXvSP zkG@~!V~^g}Y^&wCOb(5RTxcgg0zV4?JEO&U$be^?gmG*JZ6{}<<~5LsJH+egobhpY zu*qEB_D8}Cj)l+I_wxPGha`PRq!Ld#>A3Oq__|w9C=|`T{&r3{?rg-7dV8M`Bm4t! zyT`Q)+T>Xj%6#TC2z`Cp>D|J&aMPwIvsa7mkVp9~t3a6|crIc-6Vu+dwtYc*apwE! zH8J8cWh^Zp%;tM0W(!Zi#v3VHcWe?-^6LXSelVqDrsmu zu4_!253tu;e>V3H6$hsdv;z{4p595*ULUY~CE;*orRVy3QsX_$c{Spz%eJqAlqV}x zkAJ=?{sYYprFg>erOBYV4C*bI`3YR3?bBfgFBLx&2P4B=q(9oj%h-?4`lcoqYIa-u zXKmI?r*ND#oLD;0a;blL-{mFXekW6Rzagu_Fj~H&<|i%2m7_tiILif|O75Q}T^ZXX z&%Vn0nGmr3M@N0__ML5K*fTl8>7IKL{MQvD2&L3Ihfe>BN*H__IoWz%^r(qbeb(*% z`{Aw{m&D`1v+7mN4S&NxW!cUi)_Bk58%G}M!tqtF+y*$$O5gvAA8l@DF+g>023S(R zYU-z>o2UMxQrMBYrBWtXMhxW}a;x*zQ4*v5xfEYY`6%jOR67IGG4X}Q!!5DM@5KTS zq(#iUB{Ud8A!?eLO%KjVkWXiGW7A_<-NCPFm@U?mt(*)byU*TrLivh#bGe53xoZzb zk?b5!BDOCTR2fo5?oi{Mu2DYS>&lO&|+h0-KRMM)>(e!+!l z*ODQ_g!s2jJnl~V!b7x^3$GR1(gLpci<#$GQF@l%gAK(Wg={@Mjq5H@s~2e=og*`$ zBM7f=UI3sO2sgah(*|j$Mf6tqM%Sr){TXQA79|GvE3gxDGLg1)xbZI^Xi!o>4rtAJ zqy2QC#N01`ny-)@Bvk#_(nzb>f6 zrP@3)r8{|YaB6wz;LYL$(P-RqFl7;3hnQ!J9XZghxSKcl;I?i}-dn!C2z!lc?)d%%)Gdk*B0QcVN7v z`&5drbDGTaA`umdLDQdL8#|2YKx)cDZ~cm;H>KQ7%P}UzUX1fjc`aX3*4HBY@;>^w zp=`0#f_P5c*su7?H&uK8RR&`VPT`}3GpN%9g#zXd%m~czA!FibS_Bz@_q7o4}Of@s`i6jV{ zb2Xl0#s0F+goSbYPrP#RV%#-+W)@x7-q@Yj^vLSwc$jhu$KeH^*~kvFS+P!`6z|Q3 z14AXwlR|!iOt1i?R5agA)OQF+YL^n9gxPeoU^#8!;|cpqE&?B0MeI0xw-8C>p!)a!x~Cx zNX?xvJt^06W_W@K_7Rd+Qr@|jOPeqpUhj3CBYrR{2ZXK&Xf*%P8C`Ri`{t0x?+QQb z!&*t?^2^xSPV|wy&qklmZXw|-;}u9xc%`Q;Q*%6ifTNGsJgZ&DvaH>AaA(n)lr+Hk zyccXP{rQy5KDr^zej#@2gOeEW0|=_vi?22cmOB;^KYkH5LlDZFf#nGKq_E?xsuJaP zE~s}`UXQPec{afQkGDCFoG9pJUfCqS-@aNJ8rMIUDsfcP`9k7M*_*Q*VrRuE(z3d1 zo(88jjdAm^a8f>Yr8d>O@R6UQP~lDegkUPmOM20CT?@Q`3`OO+VeZFCmnb1OX|Npp zPA-BW=Zqie9aA5!IWPdH)@j;c&8!l_PFq*I1+t71Hpl(DRW%E;qye{wQaOV^J0$g? zm6HVIWt+y8rey22<5BsJ{0;NENt&`02s5nJC1AdWZZntUzGeg^a*t(UIFp)VqFdAZP64@k)hlH zG}s$&sKD})V<8LQ%1`pR*J7rzpAA4b-76VXk`Pgi5p+LlMwr{G1Q9!f;I|GH_G=1H zzCi8kY-}2foS zE`Y`_r(95WkpBb)+%fWOEUW3PK5f=wxdpB9x4b&?tiIsjEZgzkcu+2=#%QX*g-^aQ zZL?412)g;=)|KTn3n<_+j$}MJEDSmQQ?m5uTg-WZ;)If4JEkLZb~q*sIIquYSNjxy zZl--qpv`~Y055QQ4ct9sz24&PDM=%Vc?2&(U~O;ib{2c}nr>@6fyXJFJcJ9$$Rpu3 zx{Y>^(tP2r3m0tK+}X;uE7}6)-M63XOQof%-60X45j+_ykD4QnwoGiY zjp`Hb)cHer>USNdIE%Nco&!Mmg4lk-yZvp3T{=!5Lg_rL(*7aE{n@s$UwK(Rr=$a; zXs@;wq^|q^6bEt0eEYg|L?e+c123gZ0jmF=k6p7`W*a@81W)!Yb`d^_bsm|gG}_H@ z=vY0G*ng1H&=UEz#doU3I#!0O2XI+@ce(1P?h@sp20j-P1Cn9oS&l4vw47)#jxxG$ zzxiI${`JQ9!|~}`GMRlcCv4kKU}AnR#fWnwL>gmacXL->)X;8YVZs}p$#443xkc@^ z>l1JaU9TJ$)fMi{cYZ>wN2^ziw<4!!?V=mL80mOJ9Q1>4)jCHvcA=D~)_-xTZzsmz zSE}f8JZT#<#caQ!m)m)R*;3suiBm`R`qVApM~Bub{`$mb(P_*cOh3T?Db`sJexY%L zBF>%yrP1|PxR{xq}l047RRs>EWkDl?3l{My`rcG}Q7fU8htG&Yfkk zq5-UqGlaDE5Y7$WqNt!wrskR?=;{w{pw{d`tau@!9?LntIQ+HC&$y`$QJgnbOcYd= zGdNvo^7QDM&WFoyaOzu&`d=NcM<87y)MM6-`4Ou3uI6y6p=mTLY*I2hC<8Za=`D!E z$qfcuOzvY#Pp4*i;!7@OOp4^&pxH5Os^ly*Rxb7V07q`95`W!^4v@sKBoZw`$FpY2 z(mLhKA{e)#8~kR?;||qHkF-aBh2@wOw=(3(yiQP({E@=%A1TzmoNt6_Rtzm@tZH*O zyIsc*#~x2!;jd~~{nk==P=D~v;xUWjp6#nrGM*L64EPe&kNYJ^F_=e2*AYLoTEOHcRSr$mQ%lXY<)tF)AH^Nts5 zJ*$0Ft~=a8OHw;U$lmMg2wu=~t)Y(nC{NZ`>Z+@H&JYXLWTmkNLs?T&Z7(zeApoB~g@y0&2B79aYb6dOEBfT`fQc+v zvg?J9Xcu;*a7FW+n+ZQ{rhQ^EKO9AMs6rIL=gsWf4XZv_%DXn1#W~aC{&Wn+#fYYD z?2(yCc7Mvv&$biI61}P&B!2>4XepM2M;=y08MPZ}8zb+)^MS5=u@P;k$xeP_a14j# zp{)=A!qhgK!Mdxk2s-)O!iG(y ze#ghp&{O8g(Nw!%*ssqD%4M+(v^Mt-hWsE#LAf(LzZU8PyrnbzHn?au79I_z^-zQw z>_=4jYdWS7*#j$hpNxl_4O`j}quuSqt@rKq?|D7;zB=~7dWjpJmyj;Wr5k5FYgafC z!faL6j?OsBPpRA=sygX=eDrJ$)){Y~hL)b}Mx9rx|L}C^;kFjXzQ5jmRD>7}BQ~l? zPJzoF#?4?}HrwGta?HhbedSZ$QKC}YfH+J!lodH5o7477`bohTJEFsGA8|xX{@A%C zJ(ipMFzyhozC~q>U#s-5S4uzmTjqFHSFM$HynI8=lK^y4+ z`5pNlWNx}^ecbulp-xzz|CHIXS<;Us1_sJ0VG;L$LoHC1^$_6aM67^H0PhM!c~rDo zk!f=u7Prnq3HtyZOS+#b_CS8GYDKwbXY1%pal7#LVq!hM<5N(NC|b<3N!@ts@>Ce> z{kI||U8!wcN?mN1hrS<>1-Cx?XHs%2YvPoI9t*-E(q1#6L7`&FE20Rm37*uycUe~* zwC<{%reG)n*?Sjj&@`f@tSTw)&+4||W8Ux>=N2%r$*SlOz9U;V)EKL}1-)qM`z+|W zx+#8UXG*Z7nAc5P%{SX^-2swvII2|Hih*>zaIx15gvKkcKYSP8iSCo?yllMWjaX?l zvu-zXirsVcn+ukb?G3x97EV-NxM zk4uM9cIrZbNgtLh06l|P{&CgFyO(jny+YWUA8VMu(%jXa%$LNwIU{W)DrY%k!@$62 z_>y-6a=`fq;L$zy2Pg3}a1nLDB%B+=)EEY8E7p%HPMO(X3sRa0c`da{+)`5ef(uX%KnKduSbA z23=S7l^+G|a&7oW@k~w5$SMuKT$9;0g`UWhIC@Gh`^PWjv1p@t;nxW!W4G+}EbWVA z93%ZZD3e`#$g?4>Ft?v;#?olp-IPx?u-38GMJ>*}9bovC*lhM^cA@5xB`)j&n6FgY zvA6Y$k)cHgPS!g2bjW(oUjLz^`IP0%(|9Sh=2jl5Vr}1gN4{LSLs6N+SmX+sT+E?j zG3E}@nD$ChomPJT5AhsZus}%=8}o%<;SW<RIRy-5j)^+LFOY6a|-gQVyeoa5f?`A8xz(7Pz}>`)e<6UwycO zqFqXo;uxE4-t@Hm7(~*u;mV!l?kSO}6qc(v6`b$$-nj++H11s(#mhS-JYgUk7qQU`H?={5gm}iiJjqj83&w9 zg~m|XPa@j=X-=24v1XtSw33$99R$_{AzQ#uX5of;Iv)$nGFYtxDJ=m~2WImC>KaQX zf!wIQ8fUk3u9eKcoSNBi?HAx?&N=tUDb)l%avR;20)fr373zt+4Aw>o@a?dJFi`>X zvgHJo<(-F?*ka*NvuU4Qpgdq210-9c#EJdUy%?pA*2$IDCB=m(V_2C5oT}+N19J*Jk>(W&xs(tmmM7@cH{1)#6H(z3`(ZmHN_9ZJk%EcWq+QE;Sr_J7U9VB^ zwsOYwHpA~GbNL+sc|ymt!G zy;Y}_y5tS5L?T*Eb9yRfubf_AVdoBf2k>6F1bAVEOghB_<6j9C z0EOGvM$$<{gT?Nag4EWcoTIIlkPAGnxw)QC#Nuu(o3)UBK5M&>qgei@WX))!CR87? z8!o$(oM46ac)>jJ~G`=*_m0IJ&T%PSK7nG z84%nR-Qy*FUP0R|kMvjbd*-OkILeqWWE6~G>gPGf%R*&#dcK8T(k zMJ9zv*L^~FSw1(uF7e*uUZO*d5^HZBR3}(L^AM9Z27eL1npHEv^Rj9~eW1tH*mOXa zbW;+Ws|1YC@v$Fe-<&EETvu#K@C&Af@%|E@!Z~g?T`uRcml!gJIj6i}nuhUfre*rW zG$OX%iV7{yi?L5j2>r@v;X1h+vUpch6vAwzGr-yM;|(w3tk91+`H=Yew|UaO80B+c zMkLD2jRh%BwI!p#%=|k$SVEFXu>5ioBJlToP$^ zea`>L%?vxDA{$M9y5S6yG*DdE?r`1?7bR}XLC%g_SxYtdOlaqFjkBK7Alx!Is$wR0 zqld4Dy*XuwPp^Q)v#?N3{}APMrTrgeboHnHij!_&YN@z{l? z&X=|8_+<Z}=wWD=IkI$Z8LW(Hc0N<==d1}o~~t?=HN+nN^m(aFstT^L|K*S%G&WY?xi__SXo~(i1Dk=w53gi(dIK^Z!X~I zON0vlO-T~((W=)!QmpJ+p#|3tGg_KCkHjW8LXwiYLe490vm9PuO_CRFi2`;EPbvv5 zui>YOny^c7Jb#xD>^Av*N5Eo>5#sl7IWe8oNo6L;dY>oXbQ?uS?-X;M;D3L~RJ)O6 z3Dar5sOn(Q#+02_%2F`YYZ7zCRvSS{qd8d}gzH~Bd=vQKfTO8>{-ef;d=-dqPe+r# z3BMLZHui33;Km?Jr4{fNy)35N#k*69=21Kw?9!wLawF{YV}jUnp%RoHwsORvIGj29qPxsZ1roc|FFACUdWpX>~Fd3lemdaUF&ACkf_-0&hox3G|U^O zbo5p>tMIc2|ICxW_f+ALD3z~&;;rmAZ`_rTOXJ-^TXUpEqr(;3VlbgLw~~cDQzJRl z8#5xaLhp0rb!p^e-BF|YfCd0?r-w8T>W7WuPLFVUE@|J*h)ifTxY@DH3(+4(vz&yR zHa(9E@nr~@dY^OZu|-%OHGCQl_WQ!qs)=u)TXlP)Ed@gkyfo z&=LL=J6}ay?R#gCSwjDEtyGF2fsTr3wcuD&M3ejy1ro)4d*m|H)iI%Ia8A~#JFrnU z!L}()-n3pXghxBSQ0iq@eLMZTxo=!jOhI9OoU$oxfqlG=9lo)eWE3B)CthY!cWyM9x$Qu?+989dPlS^>w^*I$wMb5{r*LFq8aN90(1=+DS&8WPwi4W_^o0DaORoflq zlbSh5B{gq|BBWB9b+4zaT&4T^P6IY&-lC>A(el35?{s+YjdFJyP*Mo6j*@GWz> z**-V+$@?Vz@8VJFvQXxxg$Yp`af>HL0otjJbMP#OR~H$#FVE{QI8C0@9Hu5vfUze% z6PR6mslcvcOC^yu8gV!byAP#l+TI>NuV+oNnyJXMSO3`0v*%I2aX7Tv_(J9aS*Ht% z5Is3J3tbpnTG|mlei#jQZ;KARZ^?Un=v|4zmSSHK!H=-HSVs#4bS`KO zCmxc8nmWRUxJ+~tp+ZbT&16HpsxTD*^~myQzFP(){pPp8rNB<-M5!PSiM1&wJ;Uxj ztx6`w40#V7Wt1$=%HsRCJ9Pe;0USNAz1s!UY(IL}(f*Z3*ma;!(G zVgpBv@;X1X3CP&JHsz_6`ZXqdESSTD86SY)F9Hy$6ZeMAYd;u-EZzj^ zB91y=c#0PLV8)aD*9jj=^Itm<+Ot&@efM1E3jA`9W|stF(>Cm?Yr4FBy#`)3mXiPposv4J>}G; z4Q{cG^H9wsizr#24+_XDACLa}mhTDXi!&{p<$RE7^ne`eoZLL4#|Dbd(uRH*cNa{2 zab*@aZGAsX2p?*Kr4|F?cZAyt-$ztD49dOA7GiTzD~1t<`m*m3$)$;;6~U@kc7uf$ z(k}{Stn;(us=mUCXstk(>xI!uAe%DT>QWE;d$W$z2BM@?RZhOgk;NuKUz1}E`(JKX zoJa+#=x{!rlO4c+hL9ig-YDg!B^qU7@cisl3D(!rHJ7%fTOLo&GkH zTE$%JVm+(0(ljQU?t?+X##SLWh*eB+($B>`SN z!{TE839MNHPYBs6<8bcHP1A1Yeb*T{^VZPPc?hPDGC5->lQRr%+~GkGoDBIq}! zO{flV7ck@%_?`4gj;t=b_f8vzA0tY%UDSn_whriu1Kxh^U~M>L10&?Y=YV94jiBnT zhV{=`o(h%khaQ?B$SZXM_ZeZO{KIyHPALYuUVSEeLQhBf^^39bVb^s(j9qnMEQOB?5BBdRo@0I8LU3`H z15Gl_W^3EDau3kq{fq@$2I+d|N9AF-xp4lCZaddXwxFjH-JxvKyQ#GpQCZCqzl7Ny zoopw36r^gW1ZuY)oU_!qT^S=-O{W5cw?&^>raNRIW|{rG9$f{l4G-9E?6PtYGux?^ zQxQ#Kio+DGPhM!0KdJpYgQatdb`UYa{$~$8O`CdQPL6|sQjf1;@l6}x{;ga0MBO4jM8!zZdES=$6fy35tb7s5V1ytre`Up^1 z>6;43@oJ`%N6=DE$%)v;i9T!c_IXWhpP$Fcg}lwj^B^hueGC^tHDl!5ObN3q#b27? zm>_vCE6S80jE-``zYx@1xQIRo?zBk$&hDXlpMDmgK^Qp~7|dlU@oq);1 zCy~N}Y`^@#|Mtl;@BVW#x*!j=Mop#2?e;{2$qOsT ziwR1x8UW*Xrj2n$<9HqAren4WaO2k8QAu!oRnB&$bWfY(L-;PO8YuI1ni%I`M>lco zgg}`5TsNb-?QkZCE?%ECMs#9}T3xl0Z!4quCDP;oLQEdZN!=`7wpaon*6GUYLmU$C z+&ouM-Z8ArxHL(gx&N1oJA-1zU$16}B?2e0i$7IO~6MqUO}6ON)O zjU1)W%6O)o9y-)6cr-!%kjIc{MtsQq28NFS=Hl>fKPNz4RY4eR|L$<-kaJwcIJS5N zC>R%MQ~#m!WDEgp!e8G}*^>3TIJd2zNeUAzcwm9Vv_<6ptK(G{WlFV9rXpE=s3Gg= z(3nh%?Vt&=r3paUjh(zOXK63tJA7aw6gk_Vt1GXmc223v>e@oN#*@_{)|OdXMYo=q z7n*xXfCfiM*=bGcA??q7WcgEXsA7y-$a!S7TH|34jzf1ocD-dypICN>P?8^*Dnvba zojd|jQArOe@z_yq$dC@O7s|U8TaW9SvCBnvF-VUdj58l@zm3Ju@C{nhQA3M?ymH&Da}=T5UPrM zl!IfOC6WGSxEM{+#k+-9ar&*|C{ErxFPITj(Vh{fWh!g<9U4no5t?(a!=<$@y`~lu zcQKP*y}DtBf7Plt5+%U`ishkv*ji{34m_{^5zEF!Q-)y;BHjYWO6aqaSvNT=qib(G z_;Je$#%z2(E^K5ZG1vJjMvKWE(@_vzT0Y%08^u0zj@+szcjf)LSFEo`U&z+$k;>Kd zWZ=_ftR@=_{iL&=tU^lXVeNO4Ss-6hVh^sgDSz~ zR0}x+#vp^6wp??ZNxr8nsy|fVLLP?@-seZs<&1!5wpTAzBhqpGehsJx<05Je_;mWm zW>n*tyzZ9r&<;})bv*J)ImWGsns;t3-|O8^xU?aB($H)b@x6gm*wW+CVr1J<_0?_F z<6{I6HObFUV9J4dC6d>57S9TGKjYN$l&fp)Ccf44L&M{%r3=bPs$VEq8+_X|6{szA zEGx_09GqBvfr70elj?1cY(4>(Vko?NRZ9=aZmDYfAy?HRY36!wftJRamJyC~_wcQc z2Hve+O)Xr$?x&!XHvdherv9q`vVi)p^mtQ0S?NuqN(-f@35xrPKuX716HY|^@G5|9 zVnjRhO!m^Rm4Y+ftfa(=?Fjbcge&F?60vSg-(=Ap0*Ow$S2W|D2)4L|ST7gjVkZB$ z^BI4KnY(qUVP>=2F#eF2;9z8oLdbxh{bWSh^1bQ}<- zugj;mhiF*@Q(S6H)D4B2&#ID3i9Td$&9n2Dz4BdP;^d{r@%@m_9BFw4q0w_kx=&9K1R3t{Qs`vbU zJ;Mx-_qu2pE^;9-VfZRUSgGjqBI%<#`@q+TbpI4Sc6Zy$1E{>a!1tEC9z1e$Bxmvl!T%}?p-rKkLR2*_P`?%AZ1`XYw4p9;?W8a=<6`k%6GEJ3!;fLin1QHp<6t+i)I|WR2*pYdvkKINV3jXMImTPz{4`jLgg>^s*p;9K#x|JWY*D z_+^?G)H<8#jkplV_NwH`o!fWCLQT!4n`mBv!nxeK?M{#B0y(*$&M#lIo*5*Fp}nPg zFXHcg+0~6~PUX-xk$cySmZEtOuHM3su1|PxV%=6jZLjzKDk=41Yjl(p%R2s&oNhEM zW!jJBoG<9fi4otMK3k2O*6ttEJKy;RMpIxAx}Sn z@2#g935*8nfm|GOQ>CASR_R?fwhi6YpRzuhDsQ*Ao4`G_ZUXU9-{T4neP_g<)%jvn zqC#OJlM0RS{2`mY5?~suDOx}%MMs^%Rk9-Gq5*v9&oVI1b)?Y3r(+Oz#D^(1y=AqC#Ey2$Z4@Ss~~ z*~a`wnzVnx;WZ!aWs|1zF^vXHCtx;3%VHxV9lg8mGn`MsAkCH^U&AjvaYl#D9bwV?#8>jAqTS{|?NgJ)njs0q604YI2R|e_L=ZBt)%k_As8m!WbAcMHF6uyf zxjoMPXp*zGT9ht*ynsV~FvxesU<*oyiSJ3j3yNcTZBjZ<@_a~$Dn zG#+hf?x@&zZp$?!waM~}*MLsLU3gloX7Tc*h1FF_dm4?sSGd>tUm=-FKRjHAH5FW| z!GUi+55^z`%)Y%%a)h;t1aVmq*_?dwTY*+u{0L$w&1B|^pE2E*$O#T7Nz#Xdq5;8C z>cmK{=5)tHt_H8!{DesM8T?BuCQ4W-FD>VIyBicwC+1(GF^jgh3Zqjk(vu;gE&kLZ z+b2FOAWz-0hF4$lb0a>I-Nm86PY4Fq*G4%3jCEC;fri{}qbA4WTPTIO5#eYC(|yE} z(WtS2F-T#tKocDaR3H0{0G)UHA^~g`X@quEsUY_TmilpBLfgNyj5B{jGQMv7qr`Rb z5;f|^W8(Dt8?)tQdQLeO8BXQXFR2a7FKP|Q$K7~PK>AUtNq>{J#!n3PT;if3hg9fH z6p~O7YIfqhag(lKK_X1-0&TC6_ouY&tcjjzJ+e+k_=p=ZBE>)4=9`x_wWn@h;_j^E3ml!xtY@&kB}>7UKUZS&B=UYSX54Yj8hJVwJ<8YF)dlRmjW zQ`Us_mJk$^a)svDEM(8jtYy_PJ|W5~dn>*zUVGoMF@IMvUvzu@{H~`^a~6O3PqNLx zLL2WjJxB837HjI@Jn2_mNk|$N5~<&O+yxy#Zrw%$snXCH>@hmIvx1w5p9$i zNFGuTC2dYBQ^p<>_|y{JMpoos zstglvLJGQjzVb@s1Og!Yr}HnEx!?T!k5bmPs8*81&xyFQeG95A$AFQBrjS0&ArWn4 zbNTkvb<{j}@`%Z)*Yi13NwCxYV?&6{RDTv2WiI{2u`YtlZiU1aMFL^sEH^5qUwwi9 zbgNK3+F&)S)Ro9`(-!f&Dd|S39gJI}^=WuGuAc*)-6g42Uobg+Y}OlW?WlYmg1;Sp zd5QRD<#3!g{dntaGGRYmz~lVCa%_v-kkRehToa2%kAx`c-))gmF|GqD)3v+vx3|oZ zq+0ZK78Ks+GnR?M0IWVt$b@b)E-DVID^mJ(JxWLpbxKl$qy4MH(+2_n2<-3cT->HD zh?wUL>GD8MbNi)DpsouE9q}G6uwKD!_OS{PYhTtj_9ng>P=9|}>3 zMX{L?blITv6tEwqHm@O`vIvm3IAfe!vvGeMoI%<6v6W!X^D7%@ zQy#B6us8a`|0h<}axcB<2}who;s>v21ns;N%FlKATd4N375Yc_KUDmv&-82{%cV*w zIrXoBM}&HloQK|pj%sfO(h9VR=0E*kd7UAd{VXWp5d})#6@q3Tq|LV}w#csL3+T66 zKHxL%Jpk2u_8l7>*$AROKU5lx z&)BYe?tTzTErqt`4dy=*+zLc~IqW1tJN7uFrfUDnoGBMDyY86P{#NxJ2X{uZDgGrZ9s*~1!7 z0SZso?g!ikx&<@Ps9HEmB7b17ewx_({&DcDatKwd&0SrW^tZA3kB#PU!$b#tgFdx& zx8l_%6*pM(rzB&nMHJ52|kR76n0y;Oy(j8(%m4GvXzk9U7Vr>OrV zr`Kx7u8GQ-Bb3xo|0I}{`G$MBX~WH02TD^-jaqUTcyi6*^A}$Nu)I|Gw{1ZM0)tomoh*>dw1=rj@DFH6%+b zbca}X2L$o0y4@+9XW{XEkE{TvQQ`7Nw3(wwp?UNTV^DefDAwECB60VR5|^hr|Cbw z$)%@Wt-|t2?8#qIrhm-ofBg7Azws;TXF)YgPQRVOnz?z@NolaU*Tw+>1a{qOf{ z{t|wA{&;^qipM$&gsipdGg6xM-QS^dSl1n8gD20cw!jzG>trp97IN1>Na>ujp(h@Q z-JCK~mVMymM6*3Bjn5BtJ?}r`Ir=$NGV-&C^qM&jZGy=mY8-7$8p*CHYtRgcJ^rs} z`)!%IXIr$QaJRFBM+~fp1n%z@%Q`Q-K_sKCek^HTcV4t^ZGQkkJu9?xWQcpIXhNFktcl39-}-@-r{K?cEtt4yQocmyc&SHZV&%!=9HC~WAqE)`;%9wZQlAM zE#H#Q9EVV8x+Y}0>re^r9w(HH_uRAmgYCN}>iIJrMbsua_AwGq=@WM@xY9iJmK$^t zU?eORFy zOy%ps_opVBYY#g&#^LVIN6V^&fMJ?RDo?e+!`alnQca}}r_5E}`cY9jq0 zh9vCa|C;KdZRwLBgYjePB3CwJX0|)QQvKC+Jt{GH@(=%bMZAeM;tzy*@=q>4XAclU z?5F{r5Rqfc%Y9e8C`KMZW0<$=HxG0_uaz7EPqu+hmA*^qS@l>dMXC&-6N(KcUS3@mhyVM1N7i)`)$k5Ij@b=iFNC-ngyD8PO{7 zE8Bh4<5@JUHOM&V>wj(2o>d$i^Ks9*NGGI<6V(mRKI5}f5N;MpTUzlp_q|iv42UadV{WBy7BR2>HRAnSy3jEk^mmmX3ji$ zxy&DSxn+VsTI)-iQ2S9MwSm{DO`7M^7&ZVEc*NMuzwrg74OJEQd%JWi&jQbyG=WN51qeFMBRwES#8ROSfv zvwTumXq06>JJYfQf~(P?2#g(5+m%IbBn_MPNCXVFP~i(ldsN!{HE!Y)qB4EUk~2K` zs@xuI#D~^jtHfSIqELdQnAv(C;`z>uWFNfDHf)5h?GG;`TH%s*xq{7bPG0~NT#|e3 z6`T|M&4Rg(;BK~g_WA6(q@n=$Hzp@VcNyvYmJ8a`=V1ADM|~hzL8&U$0?7Yz&zkBh zanLBugpf)@RF9|~ApQ2gZ^1np^f!f<|IEiq_sa6F?8%O-6Lt5j8*bjs+cF%A;DE5}e|LLHiyj`yQru-9C96i&d99 zEhyi)xRg|Urp2CtcEYrL-|D}2nd`Jdw_-kYLIGeh{@m``m^K|i9pmvw9BO&+%|noV z=fVlKoIH5HOBUhP_wa*c{_z-?GPV*H$;9S;0V`iX6QaTcsKZ|=kNOT9m<^bR``IQo z4gL{*UJ*Yi?2E`ap!RMsouiH!msI}CBHFYf&*-qCG(YHh`|DZrCO@Gas>b?-QO{bh z(qm`Q3JIm^tH40#kllSDM90ISn9o5m4MuHi5dcl)$ttFzTJ65Q?gfsyfM=3S{|ct< z3h{&H#WI^&^Xz{L?)LBG?i6ocA-+<)KKwzXZiVJrqi+cGXnJ0g8YM@<X=L)N^}S@x+di0Af%-s^p%W8o%}=~aZ9DsVs0m6#aguPyNRvrpQLC>mkRgiZ*|l) zu{8~~HRjee$L#*WF7&mWpqao~flkqn$HR^(2lL=rrbHFUliQ1YFR@mjgMc{kc)BJ& zI=OJ5ibe~R3Yr(!oH;q@@&FD1o~2+L0Cr;- z@(B$_TGIk;`IAiZ{TlI!=MF9ns}2|d)bXv8R4?S~7_IYC_xGM}TGZYkL;OlGcFkO` zj^i1oq@sg?;3^IKG`L%%9JXoJtakJ_w5%qNAz!Jy--K#`@G#OSUDXFD$qa$8-3d`k zm4gYLY?mahR06pVnCEh9>hp6JqG*1rDx6mIAh#yVG+lRGDm0J%Uhc`P0wOpU#q69M z&!6N7Tqygx@<0&&K3Y>Rd>8j(IJ!+zpZ5-=d32QEd{|6Cqoz5BC1Uo4ERxyNH`W>02V<76hX7K8Q# zf9C#wTU@BfP~L~RTWv3Se`-I}aTO~`<*Gc@NvDKd9NT@+IaAwo7qd_#thd(&KzIh7 zva4!`4m$O4p=arC$+pJ2eE+?!MJ2`WyvN!WFiGqM-{cRCalfCqG)gkBV8XiqhUFg0{PTa`qDpQUw1< zi4ud3m>|igNM0HT4-e#m#uF=SHXobaoTtQXR5JOuA)1f(DohTv;?55MkJN!T$9Oij zDT5vepB)c;S*13m5xY385lVS1;PalcLf7KFTk5*Gf{A1yy$o{5I6_Jwepgx5wK{Mi zxKe!Z(Vz0Y%mWwaiUpPsFBu`x8hHId3Au27%4Ck(jEkMYGT%WX3ac836C7Ues1eZA zqO#mw^J_){c|^Bu06)ZRG3f$NAa-n_fUS^gIwIYN*jne`JykK!}ji65bG7*BJ1 zw;wmnP(uqx>Oa@0r17^2JNN0`Ptngb%~(qP!n0^9rTgi4E<+z`Byp^7@OGby)1bz-Dj}=v_IkKOYi010bUToFZY)z&Dxiiz1 zZ|Am1mOjz{BAP;h)hB*cm>NRw)CUjGso|Sr>@o+uG zBbN-ss{A$lW}E-hl>R19J&=Q3%+Dd-!7*ej{Y=>uYMC2^QMr+mWeE zH^4sx1|^$z)G^(=G6Z$WvBq`utL-T4PdAGRJ(+DGaM(z?A|reb)u2~uy+5;ByppJX zzuo~n#uLOs{nWEeo6G}0RXoh)5%|>$i+2%1&rzFNjOC)&p2#)jA3kPP#;DXVw|96! zPANGpu(dosb!qEL>BsV>kFrTF6^Vm%>)Hi8*GvMl(tkH&#KvVq4*SR2neBh_5MVl@ z&;_Z>Up?Dl9nnOb9=Ji?On}4bX*($2Y~UB|NrV^k)`&GX(pN5S3U&6x$ZPsf-hiB=?Pw)~yE@*m{26P({j?rXWltJWYIHbRpSnKX zaI?-g3p&^?)iM~7zZRIdILSf6*l&@RTtcXwZ@_`JL#lJN_5O34lGAwD+$z;Ug^t}m zyh1DMMKAhPncY1+_gWm=uk&U3M*DrLPHIZmSk5W^{)F;1^`RPc{N=|gFL2)97IB_C zGY;wFK1PcoW#tTK1Y|c7m>x6<@S6=xcnUi(H1(!rL(s3HLKe zL;O1r?|7*3{#$94@$G5uPkVHF6qMck^nL^u=bFnz2xAwMhbG@N6#$Fn^&r_`oIqRYiKDdN#G?Usy&TqBKru%(fpImpBUD- zTl_t>OV`W{Tf~rG2}(3hm$+3hix#6e3Gr{xv{bP|{Zz$UKp20x<{Xu{wjnI5J{z*o z$&Lf<-tm3D6|`)I#J{7XmXG^xYgO$0a40cQ{W?OwjLUNA4(83tz|K8DIRvI!1Z>p= zoQ!!8T4IODs;;BDbo1m3_WeGW>jPwSNQ1xDcXhZLE|o>8;fv$or?UsNouQ9Hd03qn zmg6L$-R%f`s|@~Qxfi_?Dow;kmZHjJp<_WQ{EW^M@|h3K^SMLqNOG9S=Cbx$rSkidTpu@FLfl}&{#ES0}|!DsHle16`yQU>i|em%-F)?z8oFbS3ImXGv+ ztegOJVhK+P#X#yn`r2OJVNA$?IKJh>NSar7#w0Mi=twm`Q zr=8}bRE`RNfvBOh^@Pxze11vnK%aA2=)`$;I;Tz}cPn|V9MtS(I4ME9%WF{7Biz4( z)4{u1S;gI7+{;bn=A*gB-Vgn~cZaVl-DZ!>spgO1XFc{2@vXTxW2pR))Y3#eBj_|b z*nUVb;Vy&h4d1&VsRSTwmx0;Kay$fh;H`siSU0MPcAHt|uXgjCniXd&zeZc+DzNA0 z2w39ZwixMHC_M}$0e?TD@!`I&%mE?&$P|Cv+etIZIFAW1#Clzxc?*A@3ac!Fn77{m z;=01L`maMUTG|{XiN`-j8D|aR&>H-(^VKQoW5e4&fKzzCK_iF-K}DTb3yW_5V6T_! z?siV{&CiGh+&5#M3G%3!XX3^h>lU~2=c|F}2R|+$wLL$Eu&eUN)4kD{-qY9lRta%a zI*+O`B~})pb-)D1`Ak{YO=;Hp?~m!s8XiD)Xl3wXY4Qvw02rEXEX%g6`;DqYXHuxx z(iO_nnZja7))K31cY&91mZEV2SJZ~)PLD{M(Q2Bhp+xtKn<4lg72EMnClLa3$9}y0 zFE`rc;!12>x>SOQi?`atzNheXJA^+B?BL1n0ZOF8KX{N`Tm-VT2_b9k&FJ1y&?A%< zF+XeT?^4ZwM=e`bbZ+E|*K!rV^y`eb``CF?|4W>H*VG~ufH~*Z^)!0fs?)+w8Xj{v zavrd#O}=uR+QPx}GRJC{L(Hp~{@oGhy5ahhnGtO}RQA_MSW{#(b(`Grg1QyzPr zpB7D2@5sb_qOz@o>>T#GmWU%lkMR9;G79u9^+G*j2#LmA8rKoyGoRfSZT8%ugXg7e zK6?OiU3(SLX94SaGB?P%urmTAPA#uW_};1yt`Z5v(#*fp*~A?mZ57~$1H)J+Cl*@WN3Vg# zb@(4)F6PW6%ijdsBY51K2W-|;Vl@_1KDTFo+R1fpjZ$AWWPtCXJ7juJx%k8yLAqM>^+T-pI9ftmUvnYssLa1Xz2DMUu?J5?u9iUvE;NW z!qCVUxyU4!K#@4;ICQ6wp-vvSr8=YCpKHjze=!qF`V;PZW*kMMKB^l{5qd3gYaV9w zRK4>2u8PIOo>Q#+(+Vvl?^oLcx;=Mzd?QWvW=$-k)8`v|`b!jmSr+PMp_KP#uqwT{ zbX<%^bxX8RkaPRM9#p<$T=8H`UgNPR9hg+C7gQ+n?PCbkOE#ir`>WgN+*3m|AE-Oa z=jfdJLR3?ILHmTps|4PY^)@oQ>~21kY16kaAK%V+aD`aP#V>%HSGUnl8$=0=(IDJT z3cjm()PDXr#Mn&pY(q=yET&K^{L+tG!lS1gQ|~;y|KZMYUx4Aq)Q_;RahID;H_qxX zXsO4riTnOgdb4%>xj&-6%iY9hSmi)-_g6z%j?aL6zWh_g?p!A~za^R>SD@fh3^YZy zvC`g0sFycglU3Re7twnKATas*oT|fs$v_)}8}C?gQ{Re7o^6_@pH)i|>7rG4tKnbo_;y=GBCX-Jb()bTQU;bQjBRIN5_H&MlTh_z794YEpk!tKBpqN z^M1l-r!3BWB_3hJ08G0{?iNBPg0dB=9x6>=6e;cWAuc9pyf^7Hxjm|34KfP6#B{8@ zQN9u`H2z)6vHem~?G2DWHb47H!UuYDnX!hD4=Jly{@4e~D=m}?Ukc&`LTfF!qNgYqn zWHH+5NI16id9w15FU^tsUY6x#iV^w~D^;+w`lBM5-wlKnoE*`Ob{qK|yi;wuVb(lh zUEtxVdT>A)X59s)1fi}rjVHOA{#Q~4tt{M{w1mV*5LIB;hdULB_hG8XN|3mgVUzi6 z*;&fayb6rkfpvFk^jtQk zcR9phJKppLB*gc1&=#=6mQ46i$!Dbz_2At5j|{_R<~q0#l~H-*Rbjd*T>M@r(?O-5 ze{0fUq{Lo76aO{5DDX^Y>kgCGYNBO|Ap0WC0*#5BS%$zW2R7om&>=QD8^^Wmt9N3@ z8P`k>HTl-g_hQrSMysq~^(EhME?B4fh=gQzFH+MoIG=cdKfC~ji=7OVfHFe9OkVzk z5`lFlpl&U4R-U&A`&#=c@r$2ZHjL=R?+^nQ^)GdYN%qU21o>y-O21w;hbMd}5Evq| zRya2d*DQ2}pP*i7!`mS1T;-TucI&yevsT^<{F?U{qt<-{+51Mh=A<8tDe-OVGhP$e zsd^R*Z_MOjUqxmrRO%)ryc8*FW3+iDf1C?#rbfVfv~?YEoCSSv49xnT73BOXM))~L z*tw9nyx&K=%K38Z6`S~dEO4!K;F>ze*Bc?bU&(W1Q!_>15Lgu|Q=1osqv;Gd5s`py zzjb$l)tD$ZW+D}dk-%!S?8ZyMZ;$F~f_E+>H8rBy)Kf?1JaUuS-hBT2rb2LQ0I5^G zXaQd=#`EYo6qDqg{Ogh)eIQiYQow4653`dyc-=rWyYcB$rw&(GtVyT$%PFm`HFY6A&kYsCuMUG=d5QBs z7TozLLTL(G5uv=P?mn}2nIw1ks-sLXZ$;t<;8{!H?Av){S<~2CcGcS!Ho-*k(ttHi z$_O0^#HwQRyPnjOd;F@&Au3@PwVC;o&S?qs1hoH5F~3ck7#Phhgz6FBfBy+_I^Wkr zI%=aRiTzlV9IWesd!EL=q2w*KM3E~f1&@N;wA&SDSH^nM{0tgm)K?A&7VGEzD@7%y z#~9Pce)VGW-sbMRm%pFap4UaD=y*e$|`Sc89`vmg_>q*D&NQU28`+8AdsUtgOKNhV-sDts&b`@2pRJ zg@)Fz_Hv5L6%Q4NMG+EvMGN@@YpG-v?%!P&>|3^xwGAB=HNkr-Hp~{N`hXI^&IMF@ zrn=JQw4cGgs6_SEHlFVt1HoLpq!-Z&%@~oMSEsldJOf(eG`*kl2h93S*s$W3?x`ex zi7an0-NJiDoWM#Z>0c&%n85H^HEj*$6*=l_1QttjDX`xbmtRs4zdT1J*OcxyL~25n zW5ojk$L3HU1y?HVw0&Fi-`MYe(&%n`!?k}#g6nFUPe|2ys@{^>pXkkYG|py=+mz(G~LH($f-Kos`=+Hly=;BBO0C)-t6%+c`7rdc%4<8slTqjXGu6Y2V2)BBDl# z6=`M6#}VU#_Gu-DakrZ~+imX?v?cKOtc7Z~KBJ9ZC{-OOsk?*S|;23iD3QU zu&>HRSh?`h2KKAQF=nta5rOTOJpQt6DVviVLKu5ROlJ;^(7KP4*j@`Kjj$py`!<{gqjxtOS%F7{^7hV z$W$mPM-IwaxktFAGBtCRR$A_rdZoEH;-DmRi>BrvN7{5}=El8pm$(P0xB(X;>W}OG z9ryD*_j7;$^f>SjJq|vcoag)fdTG?{&x8Ukow4rt&;Y}i2lM_5&$;`wsM1Es&AH2?e*HcCr#OZI?A^EA zD(t9|O_ADqiMrXB4yLk{GP=Bim=9WOR#c=IzY9mTfUNT$4@K4q>SZNP8uxCRauBaw zeQTnA5^Lh75f{~*ceOKI{2bs~1|NCw_5kq|jP^(i!vaDhRy&(wvKU_SEFEy9vVLYn zOBVp3r1FZEtfGC8eFF`U6`vsWVp>0F7;-PusUSht&ubiQMEUs-P=rF-oOX{wzUwR7C0#AvCNePnfBxX9ZT)`FRSB~kxXk81eqz%Q_7b_d>+lRKa! ziPDlHd89eAO8UPib^UbDzYU*JHKdbO$f~kU|8iXeTREUnC8!)_s z7h4lGh{f!UMKu_dE!Y-WR!a!H!Ky5Oihh+fX9%M zos!YK;@*uW7tmM`x~Ybh(QS-g&zPx_2=0;wu!L{KH&*f0MqLn?kCjkq+}*PbFZ+-v z>&;d!0nE2v)-C5v99Ae#6 zJsF6k@tpkG$0cv%9}V})a#|$EWD{F3ffMJElvi--SN<2Q4E6zt@zr)kX%6pafD3O&kB`krhO z+QDOJe@|=gKA860CHFHFy7TdSvgXl;K&E@J&(%Y+v|?RXeX#lurwff%U}}5V0TF6I zvB%-Xtlo*?Qz%jFCP(X+8$L948-K?7+bW<6rF)h<*L4?d<5n-CA2P+4Cwx4rAK)BNtoC55^U$y|laKF_!GK2pR^i%N}9h$KQUhK7Eg|0r%%^gWBhO>jIgW+6^ zt~K`!-NQ#VCk5bFY?n99#WGww^(RsSFk1He)A5QE+*4iJ&yNOf)KNe(RJ9bct41VR zt?giUh_0`(rdJ&HtS4b$M<)Q7z$V+vEb6WGSl*j0o zc-@xQH&bF|`G|$kfLl;M11PBG#&WigS6{ZZYCy4dP?3>JE3vQ$KDwiymXG)k)aP)% z>n`rf=So{AFm$(L=j6aLR$aRDU19%iy=-xaS(uc_8Oa zIm5HasYIen>*Juo3uiNv&U`g2saG7ID8xJ?_KV^OP_ zMqdzG;qEV^t(rHf87A3*UNqrACX5PJXFEcLd^o z_^1+XZ+7npS^IdOCdnJC@W0|0z~@%ZKJ*{FnUlu}1qs$69TP;3Uc|1M(qOk^kD9HA zpQ}pzb2>Y@$}D*QJt>hhIc-w}+pV5C{!_y}!YIRn(D{najP>yeggB35s$+Oys00Nj zzh$4O$L@hUwee5?KazAYKgIEP=D&*mhj~;d0_!LrUE04)kfa9E9wNlc0Ry61Ah_|j zS0?>|L=&0&lO5JHYXXh4DaG@3y82eV?;D;^23B)S#XB!T-TQsp9h$=-MY8=4E%vw~ z3*NOQ(v*=bn@M&&C`Aze(+-d|4@SHma5pW05w|KV{mi%%oFZ~EAoSx0VvS|XhyZzU ze`;&{ebVdto*C~>K~!mi4AlR^(rOV8?Mh+(?PvUocsKxz0zrgUVH%+bMID|Kc}EXoN!`Eh#ZfwM}mhZBl-fw~WOn zG=Pi8t?kCkrlY;9Gs2BPcC~@1Ygv;yvNGZ`?VvWEdFw*7aIx5#>Wf^t` zAlrYr*c}TdSOlzF4YI@_eW}}V@ja}GBpyIX_Xo~O>%G+obbuk~VDy55+R>l<>l-8$Q=PMQ=X?gtP?WFcvE7l1ivZvIIwgt5?q}EGk8wV5Z{qDvyW^cxXvAv}=jIjNR8y*jHq=WS*0o=GFW{EwW@$D-a9m%EF+4iOA|IMeK_*J+&e3_^lwjM7bm@Ns-cH z%ae2b8!~HBe@|68s>DGI2Q5dpvSO3PV&cnzN97T12d?-W5uP>F-spUH6p z!-syYT(D?9?3QpB4@ghMY%Sll%}Be(+bUe(v^B(9JS630(WB ztQa&~Lw#0}N(8_WVo;Oi^?uGPs`-3@scu4Dx*be_JYFc+!^lv5SFzIP5KFcPw@vAlBVzc2SeoBA(&=MA@pogUWj z?sHyO$POoajArw_#=xb6adnL$i?q?NzrUF0B+T7UpI{Lrpgh#8Hy|CmBR|=M>x5;q zp1|u`w3*u8@yNM&tO+NQgLv9mS1tYkSo!VYb7W8%iczrA>xQ;(~=qp9qrnsK|pk242&0;eyA(|1D2G zk=dK)J26JQhR4_<`H@mi^ZunVwr>)Q?vtonv?1+bGe1JHD0Vz}Wx z_ir*Qp-qx##-n~7eMgVOC}p4n=;EWA4`xoL02*Z`nJU)?(JUC~Pp$NNBb4(6>RGa= zc|AsxU~b7JaemuDWJqv|q)w5S2ZW&5o=zVQN4V^)a!9~rimnPpMhEo3{;;Z36n29o zHluR)V{jh|jx0`Lpjg-1bSr{<$Zq!UnAc^QJN%?%eJj&G6GPoQ>Ixw6%#o+N8DZH> zov+FFyT^JT0lRK&c`v-+2*c;;^=7(>jF?yz37X9ST#vS##?f82=b9$yfLwQ>=ttMD(#%uP2j!2za zYUD3MRb&Itk|Ce2uP>|cIQurX5##YIS%V4VjmJEDo87q0`PlR-F`or&$;z4VJi;^N z*3V>~8K=ijvTmE(&JDumTcQO?3r}tEEkRG;YL?$ty9%D^DgqHc`k$rm7{g06c`Lho zQrY0;vP|nnr*I?H_?JK|)q@(RHP>UjgIP)cw%g2!bW`-ZV>0m?8G+nkF}K#7&Gs7l z45u6C$o=lP9L82J#u)w>U^kOz?YiX}qB~3HtImtlSRYm2T_IQ%3#uiSvkUdo!#tAW zuc1Jb{ZNC+D}_jcid0|bT*cU36S8w6=?g?A_p7Bk%F}^1`4t+5`7-7#=u_mNQ^yjBwV*q?>2Ic*T&!!z^JbU|iNsm4+VyJs)t_4hAPfYB>c04t zLl_{qmCdOpo+mC&y`~pX;a&g7(Z|tC{;XK$r*LM(REVj0nd*wjT?Qz1UvV)l1tC;8 zEC9-W^9BG$K_syLE3|Y~bJgkm;yu1N*?Q6_;ED9~$x^S5H`%!=J7rj*(ACN+ zlN$<oJVS|o`-h-6CL!*f*qO( zZ%v<{PdW5z18Mouuxne+nB#eD~&;u56|pFe7%Lz z!8t|)3+zPS53H=-Jl@y*7ZSH>6lX;vl)bNZDH-Xak3%>xLh)eIYpSFn00!vwLo^QS#^h)uI!(9pk6Yzera<8GGl)gkbGLX!j|&c zj6b34@?@o5f1_RDky};_wq>Hh;!KV!)KzSWi$0<_{qGuS>6tt0(ey#STAbKk1dsQH z>Q{l)3k}Qqe#v{018(_x&Xln2-^$?f6gQ!LS~Pp8_ndUMdQCZBr%EFieYJz9ao24v zrG7=xs#RNkP-ZADVaHdF+cP?=KKJ8n8~ahTCu8V9X+AMw=R)ibF*9&A&)oYOuS!*p z(VBh(b%Yc7Lpv;))5d|+SlCi6KznDx(c9p2SPrHUc#!2+_ebs)dqt|FAS z&w2ZOe5Wvc&~un2bLT6y(yU@U;+E~iWYP87DTC|>mVd;Df7bHowE?r@fAiKM#Pn@8 zI1~LiSP!~0%O42^RPh9_Bz_xph45RE&6cVvS?887q2}-v_Iv_iZ-pUs zIn;?94Hx-=C87KV`C9_P)ueBIHB5wJ7HOlj9QC9z{drzo_gdY7F@4>Vm%rRjQcgAQR zay`S=EKyQ43(iM=v!=m_7q??kBy!F^b!uUt7MIAU%(dsBDI(` z@wnQ-lD=QwiF8g{NF|l|j{*)TPQsAOTZ*5dCRRleS)wi#|E9e1MDU&xC^0fMyfW%S zmjA%(H7$vWsl$!?V!V9_BX@Dc%V4JX`013d0LQZw1v#$---TavP2ZXX_j%i#c{m;& z4yh!G8E^R2mLodAuYE#c5zDSF`dDVjxRlUkIA*CPO@zj{lozStz+GDi8p%Jco;JxkLj zwUiG30;3Xv?#fN66WH|~M{Rd)eg|cmzB{di+H}=BCK(cos4bLswCPVC2Z@d&f1#KaR*PCQJ+%Q|YQq$!sU`~ z>_yEt@VtV0;`+>0KtWsu`2%$9SZ&6a-D+KTN_RoBSERp4##T9H*S7bR(l|U zh4apN2w?Fq1bdPtJB8JD42-GGH}vVgnNFc)ln9a>Nh6Us*5I?w6SC6yR|-MN^+FM8NII9Y@>MN1r2`n!4aK(GS>Wxr$sZV|-86 z>9!vK15owEbq&f5w^mqL-ikyJQKVq+D!g$Q(+aI>H1L$A=9fyB>Br8`ceeaevoks& zJC=5M9t~7kOShT?tKH%ew7p$!kGuJF^t#QL?{&I|_eQ3s@Ye2$xj(*E&B}W^T9Oe~ zFS9+fURabr+|^N@;QyvEBJ|xu^Khz?)dp5l0S>DDoA8n;?67ZSITTg_1oNQ$k5*#(fl5dBB{J$9t#nhAZ-7g_C z({Y&ZZjN4&Ilrzm6gr25xW#rTuNoW0D{#L7TpJG?I_xVZXTM z_X5{3Y(s(@QOM)09)OgRq3&pA zCt-GGPB)g^FK@+Dp&0yb-(F&<=DO_mS#!c!r#UH}>tDoUMMi8{3r2L<(waPrYqAtBB9THQnvJbH=bezAH58y2|NJ@~^Yu z8b;pj>>N?rL9NQR8Ma>yxoT&OC|9ws6||~E-z~{bY(Hj?RdG)QZ%pL(K}T?B^4#bZ5o!= zk}g}NDu4Ds%CIF6lYwIRWvPwkWW3q7_({7Ual!MWUX}_zqQWU=#!)+#XrSd+agRrp zC&$r*5a=qEui0O@aZ{k<=nNlO#kyILAl7sY62jg3C8c6q&cFRCyu!#oM`L8)@z~1q znF&d?;5zesQ;Kk|uB_a)ne2_KhZQG=Fa@dr{))UP>2eY3=RnKDh`eeS?YH!H`5(T^ zA zb4!^ooh&&Zexxhc3`e$WLWAZa=M6!Qm&4wMug2PG9~T&RIBkru1TrdhLL!4r-R-87 zD^RHhjUEGZ4?<$&Wlcwizo}EL_XHVlt`#>xmu?!^nLGo~$RW%^{W{iimTsmV$$2Nc8`Ou$qd52v?4 zi=&H2U+&5Vxj4FKa7<)Z3`uHL2?j_t9p65=4R8)v3Y)Dc#42w6F(yYB1-#X&l@pcB zaoq-0jh{ydtmdqW@1BwM`-o2uLD}Y3cH=*Wq&-x=+$o)`%e;d7UZ)AS9_Wyg@uUgGI=W}<4AbxH5 zOJv6tcLm2QFx76C0M0ozza1k{!!nfvfEsGN*6B(d%8u;C+oq}48_7M>v1QF)Xym|o zL%d(S2!x7dnZEMM1%+W#26jgLd+Q;(@SA=ai=N577SC#7tip^c z5i_XR?PK#~KZz1??5MZXqwlD`lhIABK@Q+AhNcMbSGyZhtREpU8nHdnq0vI$ej3e- zVE&eGd<+|L^ZUtpM7}#VQNGtM@4!QNrPW7)>gj|9%_|L|6zD9NFwC(bV^_PUQu!#A zP3-W+jsFkc>VG%w1td<>^0-%Nat=HUD6q+89UZ+)3w$bdf%U>kQo5(Q5#kebX+i-G znI8WR z`d?wx!l{`DJH*^wlq;xMMm*q7_{udwaC~gD6{XF6hYV%3!udLVe(kvrCDR9WM&?7Z zr_@tTOnJsNmcba?1(YT2t;f@{zLtV`8^#io`r$v9IYpW?$+raxkz5{|CiKs_=LvZ- z_NDedh!bd*ccZFCxiOrgjc+=>P-o(uoy@t<%%6sO{NXe5;Nf<- zoS?Je*)x!#HT@>`js6$ybzvo8$?-Oad2vv_);=>mruELv%7uEpSVq3Aq0o6}|BeCQ z4w>lnd(*C&2?{U__v5LTwi#iw5{CEPap8lH1gQMx3Dr^FkVl+uHCknx9uygS&10=n z)wJ_)!JMECFEYvZ^|4Vu5mBUm3b#Cbfc2rnlm>r8zpb*8ujEO-%Pl-gp3YZ3?2Nsr zes|#c`7GZ!4a znx<97!M#w^(*1PQmbSJ``A4<9k7l}@gCDK-I z+n(Wqiw$_+BmQ|QP%_TPCs2YJw&6-o=aHdHj)fYzU!I|R_>*i&7Phe^sK(vW6gB&J z`1V=Y?po6o^Ivfb{h;{r*>+co>S zoFCjAL44iqOYyc7(Ma9NcD)Y5b!q1=hsNfaF74lZKek7I%0wn3oGZ5HP+;c8=60yH!u+i=Q;3Zh zw*37;ufS4*L*@FB`lmQnm{e*}+(f9~Q0lUr8BtE(o*-`TE(=Si|9~oFd~lVd61EJD z%e^aF9o+*rjb;|^dAvRIe&t2gB|0}RnMd%rLL{HDrz9@QJ zMSES-8I*U^xjMei@W^_x{V~mfQ6kzXJlp3ROncq)JWrS5FxlpER}ayir&UD&QrR`l zf5sc0?Ao5(l*;|mnV!$AMwcGn&elC>yWW-{r4saf9uLiV`XTDzOB;`PY|s#1*5(P| zWI-$j{uylSaL$P>zKsuXviIFA{Iu@=!bMdsjK!0&FU#v~4uy2-4hlh<*c>CfsajW# z-y^qz!DG~GK#08Z7&A4hGld38F-l@t8GC`nut(&f%4@ZXC)zTwb6KMJqFj7Jtxwm_ zdq5muVD-GM1OLeB%b`QIDR)oQ5pTiORyKO#x|>7{_+X(fVAa~G5JGWQaowit z1_+CR1*&pLeit}Mt}j4iBW<9z4^?JcLk4i-vx&moK^^cc-R+3uvIA*7++_wdI-gT& z@n54H<>8jNZWI?%x`z`Q&xgI{7IHuBwPB|nn*x*3`dTVi6nCiZ(NT$rxs5Oq7%=I6 z(>-KMck}RmsBZPB*aep=;|QE&Zd1kQ*ew-YWp~yEHSm1z7wBC43Pp>(T27dOQ2e}e z6Sd-@%ID6TnuW8?{MJF94OjOxo$~NWVd|( zB1OTR?bdrXtosdDS+xD)Vtzj>4+r01R1vA;1W{ zq?vh03dOVY``Ns+yQK`L;lO}Q`?KZ&b$tQAsij`?Ub2BTUsL?orK_|K$y&Lm(UDZw z(t=V3K7qt}3<^>&Bw^wQlNHuaJ^vcSdk%^VrxjJ^g=Z0*rG}Ktp#1K5V8h6Xg`U$@ zp2-J)RgyX1{PLzx56^wiXeB!LUB*2?0_K1m=6F-}gy<$aZ&U{dEU(FX{ z!Mc!t+hhG&memKSgqXTgnqpaSy2h&2%?FqJ0+(?T5H7}Lfox7;K8_*sOoYA9zy(Fu zjKQGiL({kmFu6>3=eaiaOpbeifc@255)~<&iGc7i5K2zV>Z`hQ3%XW?-|=_|MaT#r zizSPWR8+Y}>4b}42N*-N_E*)voQBuWU1i<17;ta3>jw#lpDp{qW1%K#fX-%v7{_qt zFKb>#=a4kQFR%ncj8sNsC#f^UK4M3aBK^7EILRN+5WEzI$K)8FDX!Al%Z;gZVY^qZ zlMy$9k6yuUjz=)#|bKI@OR-#MuZw2 zlDcec-G>WM!S7N|7FNd1SWod8(WC)%2dfsPuXAaZa^|NILf?64o^=6b;!5DMs$Wh| z>0U^|IyGxAa1gm(G$;CSK_LmWg}N60O=3Q(O`)Tle^*p|{ucxBP^BOlpGX~f70&o% z>n?))Qu7akECLRx+YZWbDQ%>#lH))WHiKP)V`&Tmg#2+v1))MN5~{M-Nu;zgs~MU0 z*BX~JZ73+^*dQX)eZ&`zY!PLDCRePuGgwA?ZR5@!w%4*ZxZBSnfo}UN{p#(Rq}%zP zumx3mk0B(OIPwyI!}5WN)1C&L@*0Z2+ezz8{0-b$7806(ayV>LO(w5v*>RZFSE8#R zO5^7R>P6}hM6SuRFo#Do$wt#bPfWbxzkOX7-TNx!XV-A!v*lsOK#kG28S0J85G0FF#F_4tkOhoz1xRdjv86W%OiYIu~ zKd=tWFMO*H0DIP=*@X5*N~_O#Dh$a!qlZ|e2$lv*Y}LMkZrpW#mx?XX7rL2Vdsn{Szd z^YO;R=EY!hG-D=I&-QCdeRpfe$LYYayy1a46JhOz-yLT%B)`hIRHcwV{T|K}@p!k< z91*wb=t)dPZ#MWgcPfT#=X!h@uvpU6nkqocF1x)=C?1&Bd#N@kUS$?DV-MbnbKJQD z?=f_%-4_7MHuhWH*eDij0ZD#=%UU9%ePQVX-SRjYwukCcY%oyL6)BQUc~QEc7+Z-m@Z!gK%U z?PW5-tLaF( zX?l;!?`rf?12g(UPWe-YlvCN>j5@?D+`2;XB$id9ozE+WC8Kq&IW|U64IrrWzmhi! zQH}df&8!TLDW^S<3_uZbdQoK)5Qf!zQtaXt5Fd_7)=@Ai7o2rzfn;xkjMNt>i?R|N zjR0!FeWKF6&WEo!HJHsHOEvpIzi2&``K%d6+6Bg(yRh?}Gw?;xicF=!OB1ew&+|=o zhc0DSU1GSxgD4zZ#JrJm4noXtduq_ds&VK6UiQ~Yib(IK#)$45aqZCrHs)AqNKiA2 zh`W^F{bleuyE{O753)SoHr23~+B9Y6xjMESv|Rapr(hr{we=?*FL>Ggo^?WN^~cYQ zhX0$X_w7!^Kuh>Mf&7)Gx4_GoXNdiHag0}?K;fMk_tv;ajWZMY_g=RM22PVt9--kAC4pOeAyf5Z+J z%_X3at)NAIaSGVJ15T`?njAS-<1ej?Azv;i_lOJ4Px$g$RFR$8gXpzvwGO98+Vp;u z-EML)K|CCzy~c(dbT|kjZ0eJZY`o>mzklv8_ILMzHgKA$L+G-NX975&$*SQx$LDj6G z75S0j3OL&=iWLI&y;uuoEW564P3&*^QZd=4++|$SX3C|VNzDq~q7=nd92)g{MPRiJ zK_wGoflSO03lQEdqI+81TRUWA;GSFy&*81mk-I|lzub(f^K}oDC!=o0ZKF}No^(Lf z^pU8CN1)KQUx%XR3zS?3IJAZ7E-L-4GPTvNHRuc_k6@khoxH{wXc-9hC_T9-Q_5F2_ok+ z$BE4BOk&%jvxQiu;iIa*)tBqSnE<41IMb;N8jRO^>@Yz716~o?VK1{cpJ2DaD4b9@ z8p#3Ctd}QPp02=nzXFZ)C-tmt8bG^-Y){vw^OrVz5M z$~SK)OJFY1dn(Oi1Pvxsj#K~4bofZ>DR)TI9XvY}gGo^~VJk75Wnd+uBan$?+6RWX zj#Sp#Z+D3#a;%-F%T;ym;<@LcPo(e$iIhqH2hzTDvtZHY6w&s%h;>dTi;lfJ=S+|FYX* z=+ugmAD2AROVV6e*Y60053!g>p2~P{wZ^(5vN?X~eGe`tp2UiWFjhL${a-YZZWPDSn_+XiBzPn3)iS#8>>knplo z#5pkKb;t2ewKvbsfuBA@1(D~0Idzf%?{ld>D}ij;Cz!{AIgY7fzTpR>&|-97hf5W0 znxR3B8LuZVuxp2{689;3eCgSO-LSLUx=Ep~(h$In;6oT9^6rY0w z!Wl293%$Zc3Msr1v;+-@hpxlh!oP~^hcWb(yP2aV*mZQ=j_Xwpz^b`&*o|T>`G~et z?GkMXqu&J>dsn=O;8^ z^Vxd5^gKG=4Y+=BZ#`)PX?JYX!rr%X*G6gXBc~h6JtKFn`|sR(rbbT^5P-fQ(Iycr zxO2uFDA1*gq_({jMu$4eHnws6aCQdWx2bP7nr(hL?^p>yk{*Xq=Rz^3&6P+oR5%7tf_OQ|{*RwvSlM8gxT@pRD|HqUMaFzwWa8 zd(P%Z({z51H}RE{8E$oQHv?SPh*20y%I08f*qrvkV5h8lJ9?S)5en*GAH8ide7l;T z-YJ6oNOGy+J1X`RD?ohhP{X%y8F%oggz92llk6WtwF4G+pkB^RUa0RrJDMA2y6ZWg zdS^owCEji*{D&nUkGds*b*8?*#`}RYDBB4FX8jmBWX#+FB>7FLd9zxJ-{=K!VQf%k zY;V|#5aRMZeAygx|Is$dbw9*?S1J6}0Bh6_T;GQ;0=cQ)9;Z5hHb);bXRpJq1&6PF z{|RCp`PsPp-F^_}nyjO`#aRRRV93R&UT`TrSSNGX20SYGZUX_MurDy`2r<6FsnDNp zrcH(4*y32YtSuG>#_xIzeW-O41SXpt`RT!JkE!WE_w{SazZAOVK8#!s#=k~*6#$LJ zt;=%UQsBIXvm5(J2^VZ(TIJ|kGE)# zRl()gF%ee|i%dXg4i$f46h2>483h$zug)`ikAJ^DqI~#AxojgRIEl59^lzv5boL`> zO1|Ki#FzMlYWF+814J>)q<^i9Cj-5nk@Uwsc5kK?|GyE;|KrAv(v6U9M=ym^^9~9S zj~a_;z*#o|$9qQS+>@vrY4m?aE!moB1G z^xWAx7{&@8mLEJ3qP~q(<(j1mbu8$4>9YPEdF81@D_szC4P!>6ke;mP7=sUpBeR<{7=6Hta<{jk0ysY?~9pUL+(X8r}ySl*l zWvaU+3k4K%kdocWj%pw_%lYOak?44pRa94vJ8PtIaEpKhR-{CyG*`|}4=L_qK z9+a3&Z`}D9S>iJjV^-mbegzL^_Ilf&Yi9MRIN4CK45#AIz3Mk(-VZHe3%;cPzL)h^ zVpeHOL7?r;sQgQb#y95=Kg_m$8EWGZs0#G>H*4=UV6X6dUT?xqP7tQ+vy8_xMi;cC zNNB@ORzHmP>L>C-7)8EX&v)NV8$&VXcRd-$!xm(8oqXYd_ zntQm$_USILC8I?`O~+)?Ntgi_}h3!2$-oTS!U)XcrH5 z*}r_lW0g1SYX=V7C=}pHm%;2|nz^7UopS?fL$?acsreL`B_5UlV>PD_C(T)(^p~2Rjxj!I&Kd#o=8xWtS=X(ZIIzd0qP@D+owf$T7ZPUOM5L4%QbKJJ~0S$bWDogGPW?DO`rF57%QLu^@h zp<8sFno5ig^VeCAGQE3(oHy=DK?f#+J9r%PE$X#WbL-L4lMjjy-e`3zDkKZO`#I&) zdkb4nstlxW+t?74wffR_Fmjv$ZqNw?{j#$|X-EIyPh@)noa+9pJ~s28o3^%@iC z<~z_tW(@Axgof=#XX@~V=pt#~UmVhb*Y{MWq-bD4l`?Q`0W^$H17#$~%abmqc__-9E9{Q!7 z!kbLdj7Pi|ISTRa~%@%+`t+?=e#{TeF5C!Ja$tEG9P zo_7E|i-XzJ4ZfMhc1Wh`sh|hg+u@H}M1GNmf3RiN&gE-JQP5g^A4zPmRdMKg*V&O# z6Xl*@&q19@(P`g-idZd1H_DUtDLpJh;M1%+TEX>xnY}tKx$7L*d9gqb@?wDUj@~WV z4r=Heuz_fHyfZ2ypXalGUzk9$Od?DLtNLekwdW4vY@K-5?Mnvaa|W|=QP_GAog$@w zv~vcs(G!;4Ue2JQ)w359bBEf}W1jiFYOL6){x&me0thRM>6$Kc1ie{m^_tq~e6m!zm6mQZCT`c};FvZld zXdPsiJ&+BHBgo*`WSySa9QobAp2=!1#$^)kv=y%pPQQ=W~*kR)|eoX{`efDS;;qeLtPS zlaqAmiTjzYm~O+^A#BaXb^Fz4)B3$$4?sn$1&5@8s0RA*UyA%ok)hIMIh@F|wRO|n zDqzj)S-W^}b#!`8Fu$GT(1G~woZk2jdi9Rk<%M`(^)_v~{I2-kN1Lz++7DR5yISU? z=>E(B?#`-}QmYBsDwLoNe?NX4Mts~eCPycT9 zG6VSYZWGW-)jf0x>>}jC5Dw*2N@(AAjgOaISZUn^=+hJSa6&o(JpbOH#R(=7;1PJ zXWXF2TC~1AU8dxi>3q{!`TF5of`hiV@>!j8Z05yLHQd!N_ZF>v(f3I&z>8muCey zi+aOq>htPmMnn2>4? z&yEW2;nz^id;rZIlz-zg$ljpiQjsw?2M*iE@#PHeyMK{@KzK8)f`J2WM_#Xf?2Dfv zQaHlBEy#=NIcVCH=Kh3Wh?IDctDZPgJ|N!{OVEa z$nH4VMX?aeCtFGO$54byYtS2HzBvpC-c=*qHdPh`7;IIAZ}cfv@AZ4mW;Ps8eAYUh z>#D!{Mri7*a)2@5zl6bRvaK~vXoGpWzkR(pfy55#`FL{cMe+<~ka_6&|FdzZoIA~o zPn<*CKyt$0&$g5La>62+BrG|fcDbI_zM^UC+PewQOl!}yU3n2xd{fD$k zYNy2GnX6l)vPP=@+lO2QN!Zd%l{3%E(j?IJxfqkE(1oiMbpp6IT?I`g&SeqUj%S$V zXHuwfKRGhIN?@jQR;&&sKyH<+b$S52oKj0+bEz<3z63r~AOSxZ>^A;a6qJndARFaRdHmo4DUTT0|C=ud@xVr{ObglaxwPD zuqy}x|12-^_;NJ&KTPcR3BuJf=wVaH9bn04sODBj_nn!B{TX${${3XN)b;Q`*0g=O z@^0dGtA)l57+3xL*RWO%%5QH3*WYgbLpz=l`;UebtP@1rQQKzefXE`BEK&bLK}lB6 zUEK8zM?(ppFDA&vc@wScTj}!=;fx`DuLs37DeVI!Ak*>*BI^b|%r-nC5Wj+$pz;$1 zskYUo9cOO&p;ky1?yCa zM{Mc*YNoluF!TVTB25&e zmw+Heg7i*82aygUC?y~wDoU452t|rgrT0MS1PDC@0$(`KcR%NN%KJO-`{z!wv#;zu znVD;5&02}p0EJTeDy(;>`osOQzHaK*$-T!&r4f~Mm1i~uvjg;Lhnzf){nr9w@z8)f z9952`PP?p&0(S(CHELzfgJZ%i-u=8lr3`_LkZ6-U-DS*fK%q3pJUacn92uix&M=(4K%3e8_ zB8V;vRX#T39M?a5UoNy^N~{~o#HY;?^BAM+NhAguFWjUU*D!Q;g0R8YCkiLq?VNB8 z!nwS{T#5dzjQ4Lzrjhl;ZW=5+Kv4Q8doE{QZQT8frTvdeh!Bv*y!4nDwEzx;0W4LT z+aU48u4VapmTaDHhX2Z2?ks_Z7w@B67o5x|%BY!DMD#y{ZVoYBEDg4g>QkYD?0rvIxCBgiu1|T53oq0>=Lp!tO|a!w5%B%Y~IgU z_%hgU?DW=b;%uRnli`U|r_>okaLn+fQ;gD+XLF*U(~qEUbpswXMhOj*_)J?^XIKJm zuw2e;LGIJ;Sa8B=XYT2WyMfL@EAzN`!T=p-$9~ECOh0DMe6nZhtId5wV;SgW)iz&9 zCvLKdFljT;r5v!vz+j*da~m}pniDV2wt`w*r}dq3?@)DRoy|nV!iLsgzQFnd1@MuF zHQ$=8ORC-yKea2afERf!Njs)wciH$Ap*5&v|Lfpoe@;veA=v=>)Ht(Em*FQ!|IFOA zuj%D(?`J>6j~U(LJZxwJnH5eiE6D;gYPtg}*{((1j@Kw|#q-X(in`hfswhwLI))d) zI(bBonL1QX(JF|;;n%I@)!}U@DaYn$LYq-#q~QkgBN_WU_}C|F^){jsi2?*a@uq2v zl~DSv47y6oIS4b(AwKp7qOF_}0--Snd<@Wc@DIvj==dqTyu-;SSFxt$75``-JQDp> z(rMX^my%GxqfEuk*Hj_%!{JJCCP>3~iXp$r5tMwel}^qbgl%rV2*K=mUvbDBdZ_;9<(wIq&_S9v=IDQ@N76MrDV*?Re$GrMkEL zn0pa&e62?(NAJ&0cmmc_0;JV$k|Zgqca%xQAN784N)1q#U2F~B+&$&(P3|!D=-pTA zS9x&NO)8^;TDXD7jw@Q1whzL4aQ7oR?#O2TvLc7#Kr=dbJ$oLZsij!n>IfQwT`X%^~4DAj!xhNCW8*w-Q`+<1|d%NrMX9rf`r3W-1!az=VTY# z_v1@s(+u9CBwt*2x!&<{Z*al1r>iOb&w_mWT9Sq@r{6-H@|#2=_;OdNf>eakf2y!@DB>;rl<* zyPG~rF@Qf>`oT0R*DD4=d%sIUP5MV&A>l``XL1RP;X<6h*^IuE zfhEdgRKXaB03ucuYjr%kGPFdt!JFARln6Hr(kZ*EeQM+K{B)RO-E>nKoTe`bWK|GG zLzk#$7CVpgL1-b9XQ0}csmvX&qwy%BeHbYG+R1C`QbJzj<}e-0?#YJy7_|#36ULhw zb+`x8YiGW9c~Q|kW{)m}!vd1}j6)zk6~fWtaLvygp1u4oM(2veGnp?rB4l+hJ`R8L zaK=?xt62Hhh1i_nOu6mw& zUqi5iQuC1i_p6P!mE1o19aZ~nOY{{4uBxeme-#jrqaw#?9LLvVzlr(9j8ctg7b~NM zwtl!OQjJ%AJLH9q_fI5p=9PWxtA{HTKHDxG#f`omvD&I0wq2v%ss;NRB}U3-)>@cH zc+b3k2wXC0+Zo+qD-UvdU|xe?D~hH#fUuZ%{Q9_M%a^&N!o15Ra}krhxexATm>9sD;@& zAoAM!7IXc9g)SQrjRlfTJ3mFEB@dsqj##-(#ahn2 zXb_~WVQW#^E&fcEMNU3e?3WK0EAMI_6{#|Pu)9bL{#E9;SmdA*m7+aJc zX_NzLA_fVdUrD9A6nng!Jwqq&K$?(dBTXLkxK(cfcPBxz0Vfh!HlFTr67lX-y=kbD zi!|LTU=?#P>BKraOY%zlsJ1w>jCt1WMKOSB5J+#`7D7ON;Cc@SYe>w;~PPR)9?L0YNZ&& zioB=}9j93K4WGOGWg)J3qGmNQ(S_5-8PC1WefnTLPjW!$woCK{C6|Pxgr|f~KWpiW z*;YiZxz;|{XAc)F2j9kAn7)6wL@K1b8rW$g=Cu(-*dOgHu3P(}d?rH-X$x6Lu725g zwMula)6w4~fjOB->(t<$d-b`eX7_G=E<+C`pz||KxKcg|5Ucn9v?2KamCy!XA=TO} z#rW~D!P^|s{PyV5ahu^j$t)uqbJNQvp%*5h`@_bR93v8AuJehauNn`vrhbFdmpcJW zkiF9W1gpVq4WyoT18zRH^&&M#1t-4jMS|REonj2%EA69mc_N_vyCZ*C?|k483*iAewWPS>U3? ze|Tj7{`dS(JxJ~eIbOL)J+iv(pXDO{8v(dpBXXiCL573=U$0F(`_Y{E_W!Kqzq#Vy zA5H~p=!b3>SRXT^cGB(v$#)w zIcv@h%B1h-OTX3oCyK6vEEINg*>5G!;_v9(e}&oob)PgMLUhHOj(S|}&!>z6IlK~H zrhWU;^N}-%NRKkvI%oRFd=YY6Ef?G^530K%^q;@-f8RkmsiKJK)w{-%onIBBR)H=D z`1O*$nV{Vs8-s)|D%em4;+_^aKOPO*%NrL+x42~ySS2z(v`g%mT-xiu=bHKBm(*O&%W+IpyKZHCM+?PEU>G}I5=lDtk1pRUQ)lq-_!vB5m z-x+TfkmxVlm)`OaRx~Q$L|wNEI055O0RRWN6BAvbTu4$aaq$kL;9vs8!NI|2G0nIa zbTY?Kve5T2%OS_#zKWPR9M3TX%Mm4?j+@esc6YuM6~8+tI<6qb3Hvb}TiYJs>aa05 zTlr`qwE}*+S6Ff0C9C))Q4jGU3axqB_5yi<=QPNFEtLMsxjEW}Xe0g$n~>wC!nMTp zhyl?`2V=IaAK#p-{Ym$^be1`FjJ^ocT&`{+Y+!ed09Jl;n3LT;o`l_AMb|vupNmEE zK*G85DJ?O?$R52 zAilU{tVdcM=W)mK-D+jx(t-SU< zucMvkSvoP?1~pS!W(EccQ=TDGc8QOUp)!5DD$XAMhn8p-_qpHPLrt0eAJ+Z9md|$| z*{8G^D~(99HAs+L3OcBObHTp+Tmo=BgQRT#;N$>9dbhicy|VxU;QF$g?dhHnl$t{p zx_<@SIcq0Uj7d*G&&?C{+4S`F#XT*bQcJu?5+K~ zAQB-J%V+8dk{dcrVbbCiD>ENCpPh?Qj-wM*Kv`!iiKmV@{e*axwW>LL%M?mI80q>h z{P?8%B3~B(YN2>?$@rgKtvZTO(Sz==l~=GP-!ZSuq8g=74N% z@VY#Yxx;#Jhf-`Lr)v)G$x>etEF9~#~e$4sIt^1ym{(&`(c{7tk z^oKLnm)qk{M`QKzqFZ~96H-T(B~sQQ{%@)lF|)PHPof=SH$@%(!7Ke`UYtDSOquQo zR2XfPWWU%eiom6HnWwj1i37#h?^$h0)yeIn5ZHbZm`(-0Ch}(o+|g$`ez#KWV`ZGxzqjU~ug~X{+%95v`daF#6)9U?Q4;v|kb9(V7VV1L z16b-_-CvUL{aN~~QOt0=WE3*SeKYLvvEv7CZUIti9Y+RI`iW>%$blQvm?t8Bur<~W??1PaZRs(1N4sJd$l{0L+;!601VB1&kFsOofTkIb|*b!&J8RNocl4j z)i3|f7LacfL;PsqQ)M@*QVyTm&fhu-Saul{EorzFB)ohsz{%)(JX8p}eG->e!T#va zNBTBVl~`>$Qh>oD*uu`0Ng0}Aowu{4n`=wiF^7|{9}w)J4bI=D}crKIs- zqO_|+=oa+OphR${+0N9wEf9lSCyPCvJfT1yXQ)I_V#lVZxnR=l{ySh|-blvnry>HA z55QK${Jf(Sw@s=6H}(c$?k)q}##NJM^^8h-$a-Tj3#I-dgw}f@J!P^@nS^fXzDq8q ziv23*4j~)5z{fKVAnRx@J`_N5)2=)3$KrJf%i5BSGMt+gH#KHI^AKlYu*MipB2NmQxkVEN^YB@&yqspQA&R(%zAy`cg`sMZMo8q|# z)gA6rN@X-R(ErvY2vX_TqR^u;P{F>cOb;eK0HxpB=R*6YlSRpBmF~LvSEBammOkT* z3{)HLjg5cgs&c!G#0f5Gy+UV!fYYO8hr`+6vu=85>=(8R@)uyE2j|U>*<;=5&j!|N z)9>^8TtFw-)-4zNva#;2n$5cPsC&;YxyhW^7yBYe=LWOjxHh4vLv$ac_}M{{H0qe> zw=>V@Ysk32)E`VZJDf|bmhuK{3jlbC5h@^*hm$D23a6*71-2r$J>kbwd9K}|@vVYk z9o&Hs6d5S%epMI)I51-(MoCu)K;km>G3#8FwStl-$c>79j2#;(8DvAnhVixSeLT7U z*y+Cch$M)ZH>iVCXHTo-Tdysy3jH_|4L!4&B@zT(hJ%&QXMAcKKa$*>it4Yl2jO3NB28i=T#F7)4>zCqUm0Kk0%uREOH*@3B1t`^`n# zTt0|aIhiZ1!K~B$bmte7GPACF^*d0y(wc~A!_DI&cs!@(<$2Jj7+717m_tD za?0I32P*LX+Je8%Gao#TM6J^RACKpfsdGeIyOf=lGA}2i9i>A{Xf!!Hen~b3 zeS%wM_KK0OOntoIV##s3ovM;!>DL^oKDi&%v?@2Y4|_KDu>oelfhRHu-h|Zjzp-f5 z4_=8CMnt5k#-9!@X%bTC9%IWd_eR|y6vcB)T(1La=jCG_7mUvOL6bDj?7AP!zLrFC zvVuFl&#cIU?|5>r>xCr5_g9oreS|F3#ASA{j_&-cd@(U<67O=k@O{j^?q?Wy^c!56 z@6OBFmysVWcD^cziozs%ml)>ynH9HL?c8Vney_Cfk%&^~YcFfNX&6V*`f2#fo~vw$;=E=z4h&2YG$?0Lpt~ICs7tR`qH*tc z1=Tho<{jOoy+v&fprZTO$|5iD!B_N)26xSA#5}J%sA;c2l8*kflCA?c=_EOORodyN zjR14lVdm_|taFh;M-`yVng&!&=R!N+F~nkMR`5|i!-((mLk1#gK!5a7A)1Zi<-xC6 z*srFe_RLnjoK-%E%+GY_f%c(}I0K)`6({K;U3u86?wYqBT%!xhCj3@upF#5fNICzK z5h7+6a^!6YNM$l^`Sil`$5THl8UU1lR?S-7nz`j;e1l(NMp8{kygcmZ7b4G5q-ImVtR)++MQLUC3Y?n>~@pC}Kg;=4iOJK*+y!Jb}J@Q9gSLq|@fa6&q9l$)!DS*L_y@li zHnT#i^3p$rTZL2LwhK98J0i{lgV*}3eiKvc(fx^ab?8d_<6j?NW3>u=8wYSsa57+M9qVGwr`bA?Fm%ejUNWcM??+XK|=>-mMNC5#PK`D6ntos^xH ze06o0vh7Lz1bxP0us(bq>dc=8o(!;WcbD=_BCuwg238f8t#hl6?U65Mb)PhdF z&0T@}tFe2B?eyL)_jvC4aA@yzLm_epVLV5}iN*x@Niwq@E;KU&+X1+Zv|N#6V`8`)OQY^meK^kXAE`o%YubTJs8R0o(Pie5*{; z582t-bA`m&JLYHy$*zx_vAw0&(&H)YeiXirgo2ScG_lVz=ge-ZT4k*T3leb-r-N1S zx;$nBBXZq)ZjWB7@%(B1G<2lTSw1!0^OIG>$stihSXCQ}K&Zkh*#w>iLCLIOOtdal z7c{8P*xa^*Tj^vkWmM2*)|_icU#cTs@~G?VT*&nkPoKP)($+orPvpg5KlI*2LnNhl zvhu-4`mGZ-=~~N*G#4g^Oo{RLWIznK(XlKTxNNOrP;Y9)%~1Lz2b2~~2Q`z@JLW44 z2Z>h|4PI|h=g0$i_1A2-b42@GAlsmyEgsKN4LJzaI0{Omv3Dx5YZdM8zYgD7c*cym z#}Q6cv%>vQJDceLNNtY^6N9h>ucZ!sZnQd2yrAAlWP78_#?^Y>?Nq^M(GNkUao~V( zJ>b)VE#eI4hzOxH`*^D<*ijU6eM5cn&7S-oeIg~dpLJ`=_KuP$xaLL5hdURkb=J`J`LL=#olLD=T@x#BZ*b7edLNUq^g@t6>Vxc3Zi$)LL6EyBrtxff$|NX#wvOZGRZr0k{Z`T4K4nKxb4uT1EBD_`Nr zwkmwD=-A8n*)z~v%jvYY2H)sUwga)KX3$KzvR^zrGpB387~mu20AG9GHG_KG$K0pn zqLm<_b4p99#!E#W%F9R|rpie5zXuAWC$sgDqBx&Zt}BAfLU+#lLkAUe>~sq1#fBj%Y)i$yZ%CvF%h~UkFotrWBjU*43aU| z7;m5)%6T7%K5`7RCcTy`GJelIHBCz_nHLo>tYIu=n4+d@pTGtFP?X~-$V)m$S`rp( zl=2B#TZLz}W>I1#_w9}7uzT?62^`;A7;+;=Ks|${Vd`^-}xlD$HQjB#=N41g8GFU=EQ_Ubg(V;y2we`=P-$xaVfFAA0InnZ6xk;n~{o$BuFe-1t|E=8nxR2n31q;ToF*kIKv!hM5L z##|=-DhivlE>mQFD#{gq;31^GjcdQ%&^W|2HRI&do$%bk7A)0V2+Hcnhi%6h zyV!hua4}V<6|~}CtB+=xPO}=9P&mL}0hCYl+q)h_dVhNzwron#cN*YI1I+R!w~a8;xEq zP`-A#Tw&u)?MW>bk0wz^IYFVuA%bb2WO*F}%05Vy2xQ=Qool!1!oWOA6!iqzM|NL= z<~Ee3b4fV%re+~#fhQmT)z<%J__0)2$&A8G#Zjeci)@B=iwVxjhAEzrn&MA=)z-zm zh1OQC54UPowjEvvt7h&f2W>9`8tH?3^mn~>k9SyC06c5?&Vzbd|L_+ILmu4aO@6oW zxti78QgV+pB5aj(B`hl}(dgmXBpAQFI^i*}edQYbeb`J(5ME-`0(5DlaFq#cb>O|sys zle&5PTBJ~;m|^92{pgUYw3lP`o`N-zH>U)sSp@OVoS7(4rW>NkI)OXJ{pQm&HRbd!hz0_Mttg`K$2WP1z`+#r*jULpH zr})T{ngo(c5ch6Ese7dJ(~6dqy6@x~$jw)90{m3OX#apB+#Nrh%tWLS;G=a{|q;zgb|Q&oRAs z=A~Ik3|EqM#VhPqj=dtqb9!*MK_TVD*xBAu?~dVi0J z0s8JmzcK9v14;=$nJN>3_T7RM`#81szU7?2;#shpk6s>wuWdeTxWY-upW2cGHatMd zRVO(n5;vCSnMx70g5&~1rU~y6+bvMT!{|oX_WV02t54^f@i5p**y(UEaaVrqN2+lv z`HBRrLj3Teg}~NI;!IbFpz`=op=W~p#<=(qq%@+eO+To}A*ehoyNpWLg+ESSkPks* zm3}AY7gYe$|HGsI{oo#X_f}6~PyUfoH=$>?=jf;%cM+#`q}9Fs%h)Et1?D>Q;zh4( zM3g*Eok4lXY3K2~R?$Ts1sKNfj87bqw8JOi;jwhKD2rrhdIQ6y5>Kx0J~R0H_Y-4D zl@O_B_jpc#`Q=f;3#5sHmq~p|%LJ*6ASRBXLCh4XmqInv9(hxo4aq*18kbtdnyX$` z4%L^@5hQoly+p31Yo>Fb{<|Gpp!ea1<;lWB3if@`$Ev(WDG|fUBAw6Y`I26OtSH!Z z-LR1iCu;!9%w5*DwjxbdGeJ>7aa=q4xuAGhkzYupu8JdV(!4sj^m_CM&Qv*%w=X{z z>5;gMTCZoJZZwb=Y_BV1S>yv`_Cg&@sOsj`RZSVwF|0%P)xkJ7SUFYqZ^|6#s?%-!2jhUs)$lAVZ0U_vfxozwKX7pRUmp z5Ruck@Ao@9J`1E0p||0{S|>lW=}PU8B3>D3JEv;jK!nstL+c#kNH*dAn5$gCAHwA+ zFw{NCJtCC_znwQX+-p@q$3nI^*_aDI_%M(5fJ);Y*I3nl2YzeuH2kI+uo9(B2?X9O zD&^zoh8SYa8-{DQxvl)#DozzjHY`|2vn%on=()cOXBIiA#y^I9flwOlJv_s_Ld5go z!}F9U2aNO+x0@S3ODLCme-g?`Ab7`a| z-c_kcoxw5D`<}fQU_$v$2ll$}wCIGBo*?mldCaaeTm&^fJ3-dN))9rG9A?7*Hsc9H zCQ@MmSm%tb%<9QqODuIpcxLsn%Zt7e>rGd*=$tEx);+%O6}b2I>7`v52Ry7%zSds> z9J0U->n&jN{OQ^{yOcPO*$$(GeSiH&I<*n2%H&y9vbnT0^{Xi+OeauHMGdKS%DBLS zVZOlfh~-fUGMBfiD{S{%e0Oz$jG3I7P0b|qu3B^b7NUMoYCOqtmk81-qg0@f{0_hx zmyo80m6C3T#re6Q&3fuwu71SKV9`#g_St+JZBR$b$j^vs@%Z z&?#)>6-=}y9w39=uAe0o>mBs>+neR`@VI>->o6n_v7zIG&h^n;3Twob*s+`Z=9*L` z>ep`{Cu?x85@>r3+ViHJ_H~A0@GK|!UC&V@%HI^!jI{EAa4o%zLpy5fpR9znt@#;n|wa6 z5NT>y(YGY`H$_cas9`g>$p=a|81i{`MDYxz7T@Wi|J6XLVUPTru33knZnK_rK+r4G?Y@-9C zdPe1K$??6lO;~Hb!a5Aq!%hE8d~e`VsWAEiqB8_=c;R%K?B1YUOuLq@hPAO8x->_O473noJm(p0E#BPWqC6im^Pcq_K==JQ9 z8m?W5aOoA1^3A4eE@z7SX~iF&m&ZNcdAAAwRB>&f;6QL~aH2HCo7r1t z|8XRO)>l)O6pOp7VU5s|r?DoUF&EEpNRFXNy-9ek$Gwx67-o2S~QV2h3DcY;FbH!*w9zYKxb}`jiPbL(XuVSnVkQSs@GC#Z1N$gqkrMqTGNm@`etXheosIL^iP zAj$5xw_X*RioMsG1Ho5#%LkWsq|hch<~y}gLTiGt6dKDk&I%sc+?^qMhtw0j`Jp^e zj;3{aFyYxB`_sP+o0sYmbf%}og}x^p^03FZS?Fnk3!+!3jp0$SSVu{7U+$+ZSrwxr zG2H9Q3PbmVGvriG)4Q{&DX7uF$P^sj%u~KYi%mAV)Bn{6H&=ZlSR5XL_@_jO{yY&< zsFQQfqVh6oZ@Q2l0#EmsTxNBXxh6Hv7{_YzI>SKo3TYdf`iI#SRw@QI&mI2oIfeAOH#xHmvx3^_ca~H+Xl@A)dxpvUewPp& zBbvO@5y4b)+dnnrg7&g#p0zM& zwg-j*4$KZzmqdEg>E4sL9+v1rn+Cq?4gxr&l9zAvGl&gz%S*GF zXLHyQD&K3&UY=(0WwJh=VkT-!ekIuN!itI*V%XDir8Jvh`7wZfNYMtRQ5L)1&5@*E zS}IJiEXjtsPI~L+E9z%bVTo0KMk|xIy)41SrIbKxgC;lLOizgOXHrU_V@WYVh9Ggz zHphA48b@{+L+};*AA)cZ+sxZh%`uIwA=wq89%6=C_O{$7geZ#I6;=1%cKY}-uU?-$ z=S!d7s~ifL`&^p!r%@i1#BbWzMuCVd-wp2N%_Qx>gQph0!NZa!hL==A(-8B#1qFqZm?mGuke?%{;E6x&X|&QEYg*F!$V_CVEi)+8m-&%Lp8;`O8WevV7HeJJ!(Z;T z!S1@?{&luxWc%y-7C zw$86Qk2*L22^c?lQ+Z#|Bb6sHEdwpTnzi0;e|WuiOsIC~c1|E$5{Yun@S5U8v4$F@ zM7zpE6`;joHa~BdYP)wIGcaBQsD4@iaZ&c8jsGKq{p(XPwLM1u#55H| zQ*KmWurQLK(hPL{e5As20X)-ikANpD@}K-*^+zqF{d!VcR{W1Z^+4ttW_}jAujEXr zv?Q9ES9-ag2M@TWJR-5(*S$OBDTWmasmr-?u!8PcGAXN`D-cu^WFc)~ZSxiDyuoha zU3uHaea60A#@{k~m}Q4{%1o{06?xck*e@+z1i6}k4pKYU$Aa9A;2myVOrE4p=N#%o zGOHEwo_irdJ~kH|_LfIT8seK(7Qs26BBjeo}|b6XsavPuoH zef*WS-@jbLfS;1Tmngu182Sv`NEay{{qRmaPaRR7S|%=k-<-$Lc+wrb7FXScuvF*Z z>KHy7oB8hk?bpPbgz=ZA=d!2DdgeQ?8w`2qmR%CG+f%C7hPk z`)sX=V&s;&k>Lm0X{LTo6)CVOUj`+t>Y4U6>X)0gXe)uN2k)g2SB|F`fQRotP!=s? z5MDm>-TEh^x6A9IU8H;xuXwt&U&QH%E>zF$_647M&PrC2^L6o^t~rSpI3_xwm?iB_ z30uLli?Sn{z?*{X`KUc&{d)f{gY5)S6m}{dyo5FV5_g=hD93JKeM`8BpQ)xMp}5Rj z=@ezawehp}&02kN-s2egiQnO6dQ831Jh!uX6lb{X*5o_5`A13(sKsIEU($~PZ)sZF z>YeBB_>3+aV|uCHDy;~E^z7uoMj8mW|3Y%8?$B<;w;>S~ufqxFlKlnF7^Ck~z|;R! zrdXxInqy_L6j-$zFh%$uPKuFyge-JFmhBs*Y+hM06_qN@Cz?_k8N6jGHHnYf1M@VR znj*g38@$3v;-7@2YTxLSm<8@pi0FEe%W%<7&;}(Od+1(ZnN7J${@nTYmC-LO%3eBI zTJ3d+iN-e0W@C8M6{UL_z$tF-%#IGOD-JPU6afqcm96a?5qwB=q>J?R?$N_9tKUFya_ zyJP^G{7p+CUeX`q99Gh+|6_mR%@^5?tDsdVlU{>@vEO?Y;pGe~OW$61)`K;~w>AnH zmP7H(Dq{tLc1}WHYN}P7E3Um_`)5sVaHeex@5G>N&XT_bDn`Oa)qY1?>UT#? z?k}sMg6R2zH90*vuw$2P2gRyu`=ty}_0s_k8#!vx9*=T<##i7xx~+Q=ZzBm}SRGd_ zTyW>I_bO7c4R8^i^L=Bn=K#qYZ9Ha6jqg+kcuCK6KGLkhmWTCLmIWDs<^OHEhYGqI z!jON$zMOxOEdP&94;1FyJ5|JPebfg`u|9xqH68{aD+Jj{r%2<&;?UQnL^SUzHfw|` zyHH%EI50AQ2yJ5L=ccTo@SSsgyA z7A4K4%jVr48tgUlUazvfNFMp*I+CA$;qre z2Z7NimVwLGx*xZ_4=^*&Cewn*_zsB0oGmZ&-!ts<++nM@Q^V1*kYaSUBAfC3$F8>t z?0lVSl#oe+2I#QJHL70lN|*9B5m2?K(0bjQT|BG(>qjP-YOG`akKmvt8KqV`u+&(C3Om&R{#_*(pMJ$P>6VKLUya#SZY zMj}71%+VYUZ2zosx|-NJR{~Hei##jHq4jguTcMm?cyrf-$mipuEXNHvn4cNsiL(Lv zH?j1%*Doe)s|xP*-xdszXnDyMaL~_O9~yqTI?evl7avZcv(n<-Cm=I4y1CuKqmr%j znI17dx?a*W)A4oDFKbx=CK{9KlB9hn=@h`m_HbRRNe^^n7kuC+SjMIBkec-0SDgNF zA51Ktjwz7?L0S#|7jml5N1+d)ab&yrPu5?n!ji*Y0Z+6psgjkT^`QxU-}X&% z%}_?X03JW)skE28V=(|VuV!aW3d2s@)(Ye&)z2X!Zz(EV_EaB^4&3FSgC-c)-QXOM z$m)M@?DqiIZ-p`^hSvU8bBYFvciWFV3-=B)?kUw^wT2?$1*%+@dAa8GlXKn~#;u zp`WM-#yxaSPc-%~RlRVl}gR=9H;Vd4QLJDQ%BrY)B{xs2@iu zWKB}7O?E8kyz3+_(0Q2XO!{x*@odBthZ|A$McxOk&^`XrR&Hg|bH4)NHt0Pe= zw~Tn~KD(#PWa<6)c9xP+;Xei+&JmTn5!(7imi6U#v(RDjP2y|=_XketYbb|%l);N< zKSfLSkRKMPG?gZ_9Y5K{yVrN2X(DJ?6xfijF}x4HGN|3gjhhc}@}Gp49rjFtWcOQ#ODUlQpR>*bF{mA(-hu@%mQa&Juunw&UM zyAKR80E~{x+9)BqPpjK+wIDr8-36jQR>%5u?9OyLp(XBYEyu>!h*`nsWDGl@Vzmxj zI*n9$2hoD#Lf)R{5IaUWchxT*w1PH<-x`j=k3+;ZlYcu1#FlFBm=whJir&;9s&Cfq zI0-fkncQ(*DZkct!Mm%N$9%j3DuAu=wf@x`=de}=FhS?pg)!h^>yuE4YxGTC5sCge zvK812r9=j~I#-)+?#`zt1BL5ev99bXR}vF!R33I;8MGg4znC#Uu6+MtpHd&N@50_V z3xT!gX8d;9I?#KYO566$Jf|{hHs{6OGD6xO9VxLN=^DVF9`GM3@&Du(sYo-Ui4}Ap z!*Rt(g=!%(DVUl% zSp`Fet!Ojr4DlL4_pstsCqq0$DQ=_d=FGl;J2?U!;wuoN$-b*u)>1?(x2Uo=_Wdro zPf6qKCS45p0MFR@h8MZHCnO*8&8VNliD0P8`DUVC*7_5VmlhmjpSNnjflKbH8YsYA zPQ*k#9KWEPVrjFSBE;`+>Hi~*Fd+jvZ`_j$9he-39A~+d3kS;8KJ+=12=t&kz*_be zPw$o*8~si^?R@Z}W0R}ALKmlK{|lF|Xq$2zM>i-hhNXgQU2P*fc8BNxGFlRj1Sp0a z$!Y+MC2x&1_W|E#km)td$DTkKU8M$9-VMGAw47(G_j1*~<;lb0i`5bin5AXC zhsOWF+y(FJxJeQ%h{1r6OXpCR2bNnORnAqO@QW6g?+=%qOo~AJ3M@Hl*k`on6RCIB z4LgNy7N&%(go{0sw6n?I(>dFNfR`<$XT~|5jnOH~1$ev2YxQ{=%P4Ilk5n*MMvn-d zzX>)bGJzRs)Z|%EO?`->;ydI<3`FK5uyoq!51>B_cYm}q|2P0Lgz_&zxDOn@Ign_O zD3h2qJ2XEDH4QaTQ&1ar3az8Dph=;TGrIhQ#jz6LmBq?ljK?Q`eH{8gO+D0_*wuO5 zd`C^UK~AXCvEZUoy;T=2vEoOZl+OtaxTQz&Djl8YXZp*k)RZX|wKf{v#0|n7QYx3m zJ2dOD)8x!o_>J(GQfOON%+n}Y3w7N^y?tn{6ZJdsV9(=>aLAxS#5D(^s!5SyJHU+Q z*RYMbNbYQudpR=gk{eOXy*`kuNY2qY_>;gWDsM&e=j8__hJO4fD3Y@jF!?^k@%N7E zMpbD9#n>=79%LguQV61WyJ4wrVAFvYhkXTx88^sRTZGP@7oc3~_wEUL^aGSz${?wm zY-V>NtwEN~t@P6z3EMh}6RrKOC-a%vEj!JWJeZ>QxxzLMV#^*UBi=4XgT)i|B?6C8 zu|2OFkp{f8w+~1-s#@&dXuV95xe#~|8FigHR9tH&Bz~8PY%J35Q#5zLCo~CV&;y7` zZ}D95H=yW7w0H{1=|b>aB;cWtHoGl$33MAcmyU=t#ZX_AdM8#iOcrR9zfjdk1gsUR z7)An4%9L#@VH?BJ2UEoSn@d*C=!d#QbI{2It6vFc`| zQo-T8w|HQ=-@(}4qU>Zebs&-_ZmimQZ49gSaCz*NXO1RB<7wQ@0h*^qe0S8dCyei~ zDhJwsik)FU(|r3%&o3aV-eGm8xPJ30jL2!|16g^SjWqW_5y99~E>W7WUrxiN3*_E_r4?+}H9mH9y$0KQFoTS<7b za+b80#*B$>O;*8=&y&KW%VRjrc>iwHJVRetotA*FTzg>^wu1pe{&Sx%L-dAuj7$No z_dYEHE>&apAgrFW%6G7mGr8Sb=++>zP^g zdkv5B_|HAy_E;;~bdh;e?u;Mh&K}-V2b7M$^^1m((Hp5evjU<=HdJ;AhI~*~!e&A-|lt<=e!*6r%EI~kIj+F-pE!EDLv|i#gWjSe8s}FeC zl8M2!$Z{ojEg>8Ndq?jV+70%W?HXoDDwidfLzTX0fH#F2=Jn zf@0&rwZK{VJ0BogN*0C^fp#o%#Xc)v^5ajpW7vYSM?W$F`g7q<6s~qRvNl#b{JkpG zW2g2Y9P0*fyHR?srn&c)!+@9VThI5LhNI6%ef+k60FH&+`q@qZPuO??ETfW*TjTW( zV@$i_Y*_+m1v!p#=EKNh-&wt&A0+S_xQs z{OfGzSnip*hwUk3{~u%T9o1ylt$RNrQUx@KfRtD$kJ3fy5F64#LArF58bc>Q=q1tx zq?aI|sG;}XJ4omqLMQalLulX4yU!VCpS_>&?DsDQgE5kDXRW#By5=>1)5y%D1w*Lg zOMZu0Eh|bqH;LDighsgW{S3rFhLlJ4WVtce)D0BwQDxs-e5+xfHqJChZh!nn8K3cHUN&Zu?FS-aMLFyq<0im%+XKdvE^x0iX_B#_@dkmtlvCHf|- z$v1E4>1!jFYg}s!&G!9#QvKf``m2i4yL0740gn+{aNc)P-~X*%|35F`w*M;Bu})!< z4X+7ZyMB$4jhc2ugjt6nC+9D-YfrD;<6eh8fpcCxCpA91eVPNE@!@9eyHSxhTk7*r z7iw0a^y}LW{uX)>SD1q`)`l_NIfZSXGyb8c@#E6&F!hYK$k!H>z0U(X+Ot|O$i%;9NYc${ zmRpXG?wHrFe^+w~!5)-QBQm8cQD4nQV4HXi2~C*vaq{rCH74@7HCuf*KkkPM?vb4i z&GbAPm;O}dQ2&=BaQ)AT{rn&}gV^fWZ@6J)c0RhD0It<~(XL1MRVGV49oz8}0Abu& zIFzkMy>Ip&mJ9bMPDXs-MeYo3Pbj7h6k{Vs(Kxw>(vc@d7_X5~Y~R0i_BI@c0r!<& zi%r;TCj`zptf0?c;|23&Osv1(K(+6zvb63H z=y`4CTpucr{2nbWGYTmmHl+V~q`-v=MPu*{MIgQ9;ci8u5!yiWWQmyLjxI#z{y-OZ zV)X1X{EIvFR|&RbCjB!Vo+eMxn58_7wJ7Gebj>YIMxFcLjLMr796}bGOhGSz+KtV# z1`BPD#VMc-!t;PU>$02KtGD!P>^)=Wd#LM@i5`bWpie>?bON#m$wQTCh2&j&}%e>+J_Oudm36%<-VzDof|+u!y+ zHap~4vHjULsCz?f9rzMfa$cs;{{L^X|N55%Mqgo&$TGC#+ktAfhOK4(o|DUK({3r6 z5giM013tG}&Y9B$4`GQr`8+!dt4|#yEcVjqiwlE=vU69Ru#gn1`VY8{K?PuS8+$we zr0~t$db@v=(Er;%27Gy{4+_lhx2)>vpBS%q+eyTlc3*iYWbpux2C5D;pLIt+ek)J# zFYZqivv8PkLz5;73qX7F72B+8`SMa82(_09O=rggFVlM&fyE^&f~@~@;qbp{0skqA zk1>(z@i(u=s>gNOh#A1l#Psb2ilfD~lyNztV;PAcXlbGb7A3Jh5T9lZ$!)e86_MWg za3^!YG4?Ak;#)BESUi+cT#|pR9|9ewcW$d$1{hAhQ2&NJXtNa)&A%twQjk2KeL|8= z4}K(Mk(k9w<6){g3{kOKXG@8Lh8B*{E8(3QS&d;N+jeMZW;OP=#H1zbNG_zxwuM9Q zAdB@_JUKHjR?H7OJRW=u6?q^`gw)RW9qBu_ro9+R^=~X0D!xOW|IGsUmjOtJ>WYGR=(rd|PX18JcfBP& z;6+w$kU(ZmSH8Tt$w`rNa=8(wM(sJS|I_3DKPDnlUrJKaa6VbS(7YW@o2u|l)< z0>_nJFR6ZW{d3;&wxa04*Q&d!JZs*7$CC{_+J3;6sXq!-4cjWLA-Y_vzcT;8A>$u; zPWcf#exp`Jxj|}9tFr*+gH1I>tbcV2Tkkzqy53WuI{eU}Z#7NYqw16P;I<1?<1obA zMWd|PU@5WfC?mI&M_Rz**yGck@D!l0$N1#_8`m>$$n)}ic!wpPq%|g}ikI0I@*=1s0nvyBLah^JLmajh(-GqexgRX7R4V z>!NnRn>)Gl_W6qrfT=aP6IH=+a}m%*SY$3v_Xc@ZQl8_G;8-T0YJQ27vFn#*q`9Q! zU~{%Ilm{Hs_-*0SRKel};w~FH?aLie52Wosrfj@>+!T_s(OEfs>rlZ6h1l)H&mqAs z?sf^vJ>M{wtb^uwBd>8Gzfl0};y%N%Tmc(j*EaGT-t6DL)X67lPRsqKb6#J>K(iM1 zzowWkDZhxp0)fN!F-e;L40BlP3|&}lY)f@#&i9kM1>bMzOtZ|zf$sJE50461AUez> zNIEdPq4(Lr957Xq_0msp#PEUR7eM=BleRa1e8u#`P`N5Jm;)qvevI*^%?B7nc%W2v z)RJtxPNcuQo(1U>Z0u6 z)6DUCppXIb`&4@{a)nO&y}c|_Hm*yzr(40L_t+F0B}`#UPRg-V15ga4?KMq{H7BK-IdNIQ z1H(T6POIaap{&>b-|Nh;!OhRd9K;i?!>|lcp4P5#HgZskN9*>+-tT<9`}q}%g8ZT; zX-`hs<6L={zSpLov*=`Z{mt*7qbg`0@RXh|fG=?EAjcfqh5SX%pYL7-?7_bXniMC6 zF2+e{X%GD2s72>h{mv&k03~Bs@GJV=g%hB%X7EvLJ`u5?=Am0jCeu6Gb*12EtpATW zX*X3+Pz$T4PVCRE*N zE>C=IAaMrj5g1X0nnEhoTssX^ML|(02k43v<*hX|cT~I480~MRuL-*8MQi*;B2!UcpH-5J0VR$gk3cYD>FQZ7x<-gL~ z@}T}7V&Z=}e|y4x0w{P~D3ld?3(D^(=Z=_^(kn2Z9Bpr-cjS`_{6h8?GQzVMD4Dg2 zc37a!?dbF&#fO%9)8q!h_ex*44Lq}){hHeTwa7KF06{|$yKpjCt9zz>Qdnyl`9q@gF{z&UXdLbA z3%cQQo2{z|G3_d27QqZ^iC5e8ikNl<^^4^{R+`iZZH2XaFT@D7 z!L=7WYD3fCE@Zi+={{A>u*!L&Wo5C=Istm=`Dn1X-?5wOQ(h*w?hqX3df9x}3?#2Q z&;RFaK=b}xJI?WE^TEUK5bi%gtfW6Uo>7=9%o2d$HABx!I>*DLHLciHI3I$M`?ljo zZRa`aXG3xt{j7wUytl7reXBZ%1y)hWEl-gI{df}>Bjo^~v}gfFqP7tq86^4!Ddn}T zS@ljXKQBLcuY!|K-jJfC-EE2hQ~6%3NZhk$E{J!9X>B%&nj^2|Y$57w|i#5TC?-kgnz zQ%D#{`vc70kgkOxG`)%Dnda`eZ%BzlILFn*&cnqRwL_xJ!)uVyRd-JKf{8R$;wov{ zmKve>`R?B&ZNSYB=TM;UpvKkIfAxJ%z&jL;nSk9_on?9$RZf5P#rcV(G_g-qxg^uu zsn4H*C2q>;GTs9q5~We{|5f>my$j5Z<8mW&Ba`pSQPWmTI4%mmnBfjnJ3yd!^3s^w zID*sKtjrQy)F>rFB-#*Wu`7P|0|_u&NjTmZtRD_dd(fuanV80qCe+rpaNe@X#*sYpnS{Y*%VN#20eI7BEKltON#t7n58NozP<#kPxIk?fM}^(ue} z!0d9OB9E4S6ydJhvlGRtpDnDL{qgv_x(i)Ac^klC&u~xs*}G`JSERw$;Pe$5#;;xn z--~IijO8%;qVKa4E;@r#xb5fX7hoy>KI-QawO^$43Lug%v7gmzGFDO+*>-VA<$Zas z$MQu*ly(vV=?dD5IRVsgijRQO=}q~VVDVt$6~YB1U+u2W1IY-$?~?F-!7vhg_U^FI`1Y;b|tsL@Rn6gD|^ybw2_I~D*Q z{?bjsGPjXC#bFmSoTnn}EPG+PX?$(23txmf#Go}hz2E9ul3wx&wO1^=nVyOoTER|V zT$S!jJ!})i&>r*)FT1%gxA%#ZT`G75pMk(^N(lss{~=e=0)*sPpP2M>dNry37+TUY z4&`?lLB?^<5*uOzs41!ZOhF=aGZwh9|JE8@kBhgloF_J;?N=01-sCAqD#NKA<1 zzh%=^Z@jU)LFTofJ*%pY%%XST*Y2Q#pkj&`ru|Ybt5w{fc#v={bw$o|F&M!&B+H> zPjPyWZ%(84POuksdwCbuuu|4`+a@aU{jS?Mb&pZjg-JISkLbTzXE+XV`Y&#KiUq~X z%>B8I>+rl+NaOM!`L)}PFT7?Of`^Ld`!Xhoqp2s$@ZLKy2BL<)7RD+LlCh^hB5kzB zcBTZiSn8!1!xYjS+Z0y9H8Ijh?=Ln1uDhR%UYfKBMjCSs!`~@6<9KQN)OiJ@>xBV; z@WQA-Pb#o@uv-`1TXV}JVT^UM%<-X=b&)JT-HVe=bzI@1-Fu1q?JlDjm;OeVm10}m z(9x!3SN3hT*B|7Z^I@*c@e5$Gw$}pXhayZH6SX#DrNz)qSy#z$>cK^6*|s0p80@%e zseR_K^-F@P!81dzp>+u8&2`*ckjJ? zV>FwJQchPio~9CAZkCH=-K3=PJ0#gzJ|{6GIf+)}u#s>18T*d3fpqCw`Fm}n0wsNg zs}T%mN)B`R3$U>bb+}Hetb<>`)mFD6>NU!l9e;`;%3D+ux;I}xv#)&ddY9^nB2#~m z_jizIoQ=_FX)MKu7}hyqRUj&Xb{_RBg5f_{h<)UVq<7Hr@4k6EDAoZDQxpuiEXY1i z=RSGjj*=Ft-jwCTgS_!B3yvrfAk zXMJRMpT}!jpUC}yjo5O#XI7Q#Tzqu8dw(&;abwVV^IDEUCWP`$UNywzMOjf)QTy(3 zESthd(+B&{P{erKYi4In8Z4(1tl9tC%vydvJ6*!v#UmeC<8{oYgkG~8g=5l~(+1m~ zm{qIkF{8sZdRgMbux%0;p6$DBF8c;Y@9~ItLmG1RIOgh_a;BA(qm;e(ikITmO`tU2 z0_wd(LS`JpDv)W|1?73jRme`oew)n8|J;TDz?6nA@V#N(X1TxZQ=t}bm0LK93-%nR zuOQO6mkml_mR)rnReBL~;nfcsD-OR-IsmZK@T*DERDZugf5(Q3zd|^BUL?$10>G$6AC3#~La6!D_+YA&{W@vhyNTo-KjlT}yQUP5rUWkN%a> zZypLPi+RH~c#gXYNGEL>_F(qQ7F9dIVdaWtLj7L6Q8MhSaU4dfX2NA9|Kg9B5E6E{ z5Gk{zW1UGSHEPWRABn3ES0)mc1iZ%!gP5hi3dHBl`h4s(^nLeajLoCfv*U^(p`#Ry{+$K@Bl!gk z;_{24vv3FBknC$m9ARVE#%GlbMIJoQ53xa{f3W!ullZ2=bJ(EbK}`23lm0Uj^6~Ir z2Z$BLPz6jXX2W9YkbmCo!OcFV74Ze}&9M*(jMSd8TGLL>-u1pVv8=Shjy+vap+)e~ zEwdjjtidg5scp{|5(@u_S$q&QOIwL^N%L(h-`TUdvn|d@+RIG}Qh=aJY8#72T1tuz zrJ1zUuaB_bAakL+B2YG2neda%n%*{sq4i~sVMDF0ha0xX6!gsE>DvZ+wgewIK1m+_ z_h5GC=PIu2`!8uF6vZVeDqL&&;J=Erajl_hoOVN`7aMkh;9)mHOTC8=TjGskw7c(K z^v5DE{@X|KLJgt$j8hXSdjsoU!jf-D9sil%VLxq)oSP-Ct!My0#?^aNSNt73k$LYu4yVY9d zG*-Af_gwLleU#@Nzf<=LW~pBoc{D1W-lnw-_jEt~2+}82C>=7e<+>4o2SslV|rI35b!ohqMiAQ z;NKFK#dA%LhBavKnm8H;vO|1hbI!>XHS@EQodB{#(ooKt#%z33ve_76@pfXzb^Wq= zCIO;@8nufgfASlo@6XqR>Jr?0cZB4uHWcYt`jMaUHs`8Zs$S|h|tV-{;a;d}r@#9tOk zh3W(0_T6Gj1Za}~Op zPk$PCE)-_o@+u z9)@<+AmUPwmF|1LH8<_O_x!d1&?yd)wEV=b8RK{q&&k1;x*j;v{rGrVe}CosF#Ip_ z-f}>)DUjT4qd)r-W4xNixr7+h>TuhJ$RRDT1Y7v#Sw1qdPD%tZ9qmvDMK2*nZFT0u zG1{9mrDhe&MYYYTqVFG19SkZVJz!OcijS^J$^fo^Jlf6BeWiQuEG}j6H?63ln~Bkf zb}GsLqbvUBB1G98lI25$2;yV(Uxbd+D=erxVC1u!`7yV~$1;P~X8Eb=}k zoE7d_;aTtvT#>74pBFlAmwql%$}3W-lNvI*d|(rGwpG2&ep}BWplo92djs|1-t8;T z&~ZcHyLEsLDcz`;amz@5j`t_7Jd4p&`Sx=Kg*y|+w?3Ow^HAcGcdzb{&qcDyv(Gnc zeC*^gepvVt#F$F09#?lw5slAZmGedf*oG==mJ|+vPE5?!7J`jN!J#_>dI!(cnLRl2 z{UA;Rw{aacYbzP(0l~NPs;k4^QWOo?vcO1c*Oz*KJ{6=(8O^INkW5SA5xyg1w)BY zseP&B1LZ+f;UEDzC7hm#6_oXy|tO= z)@`~vel<>2nakSrB^}op`w;sSmmrtU_ih!FQz_g~cPQf{1vbNsTLO=`P#d4q%3dTQ zG440f%C<*=@pYSCC`?nm;pf^OJ=)Q;51=g>2ctwM;BG`;;Nx#UI5s=u2e5m05@FLg!yGBKfLeX_^*m#N=OKR9b_v?=DYEtN{jCH*G{puYZ zMy+IB@f$KeNFuH&emlCZ&*0&VE!zbyiovs7rsGE!^DXvefZh zHcd^+sA$Sq*Atd^09;%m&}y#p^{G7%faXc-d?n&t!_AC5!!gpeMtb1!QO%ng`05G8 z;Z3>|BC1y9enAlS54b#*`oym@j$c2ac01Y2wZt21RN$PLa>zRVh`4p6-)uL%U3~y5 zmC;EvpC9P|HAQ_LT}R zkGo78wfN);x93`qR*j{Ob;KEyu^EQ^HSbQ=1Kov0O&&Mut-d1KDlC5E4(_sM2YsG} z2~)=2n!Tc=7V)9Q-B{8BwEQj6!R5wJMs!kw{!=*)FzO}rWp>upKPyW5BD6c(=a0x9PuV*V z#Ekq5$N5Q(W@%AAuThVMl9;rAxWwJ1$f3D%e#4Qti@}e+ zd*i9>hA{j~NmAx!^1?1yi)DqNVjD54x8J#9%|hPSm!|@5C9fl75FW8=+Z%Z359it( zBC55%6efkdB7H8k-@Rhe8h2UTaZ(S4s79yH=Iilya=9}_{P~NuOEE>R0lC( zDgW7o+E&&;#8;G%j={F0a{egd50A@`V6Tdk#=coKwDDeow%7V@eE6pO`|Pa;H7dp9 z_6F^mbY&k-Y-;Jo#~(yW4M2S^YB&5+#e)Q}_mn$(k9zrAX~yEtdf7IG@p18nD@7k- zXh(E=i=XGUsh;HOxV#tiOqmI(`6Qc0tSvk*pQ;McGfP^D6#Bw9a`GA5=1?f<;oR0b zw!!@I`hk?ND)SMv4ZaXr}h)4UBqsh6|LW zpo9I(;6IV}Fzn)Jn>ogsH!gdx0BBv&Eri=|TnQ(u!TUt~5vtN$18~g5?$CK7Hs^ky z!bSG|IiIpJd=_jowrTJwy;_W5dn>g#UW@2@2??w@xC-{PL4W0^wn0>B9WIsEQLngd zpq&B*T(TqWAw<4OpJ3oVWy)gy1EpuxO8w=&$GEtJ-F2nSt{)x25D%xe$5tAIK`fK- zZli2<;dzQipTL>hl|jRMgHni$$oQgr`}mg_GfBT4Mf@X~5`Si|j%OkgC8!mz$DN!! zl8)dp9yqT@>k3n`p~*bt@q8ekqn%wqdiy<1+D9?aG9?$!`Q(?rpOoA9GJ8xuAUKS| znHPW4q`?QlHT$|%BS9)(&*t64>$qEV4p$24u?Me`MImBRHK$wg%@#%|rmf23qBwKM zRXdED0L!O}U1Up2Dj#4$i2$}VmKKGuui;)tIPXu+g_aFOh@cR|ti0Ub?;}Q?Mpg9<|naR;^oR?m5R`R%%sCu*A5OfaUXAC}_wFEIpH^626V*htg-Y zAv5Y@R7Ntx$qB5|hMV;;#Fp5ZZay_j_*w3@!RGt_3k&$alF1w}$?|xJ=doa1X0MvM zSNQI)gz{vRw*7lYH|HMC(qArKnPwXC5I?m>EwB44U85R%7a8?aX#)PS&Lx+0EZ{6U7c44nhJHKl+ShB(I>+=6{zx z)onY9HAAGE4a$uR&fsqUSbHaVn8VH-`#UpjZy4BJKK?ecjC^Cis$H=&{B)^Nq0n&A zqku#BTO&3ZBjQ?F>14>@1>Zte&v9)bMc=5_2=z?aV{aC8u4BIFp|*%+ir|k)Ru*J< zBv3U_Tu4-hN==53rk%AhWA2z~g+toh&AzMzS*X_1!}*!=Z(-a#KSO#RZP2Mm@$&LI zB@M2*N4WLQN{dL+d`pqkzVzx;W_0rXjcgnfHN5tX21kZy&1wj(VpLnH4eL6Oe`@5d z@GY%zLMflp!qa1T7Iq&QM;_X!r;o)yz3I>!T8i{n2<${2uDLZ-Y?&;u)Q6D!?paFY z=aJt?BJ&P@MBG=;fx|04eh1NsU2%2p$cPXjz9<)=6Zp=hS~Sl<_ky1PjM#l6Ig3g5 zwbZdR-r8a9`D)7D+IopXP?USMfI*5=)a0w1p3lz|3;G3WN;vD|-RTt^ECr6|0qP4Y zFfQi%+?!;}xpvZ~S9oCZ&gNOnTLXGcohZh7Cpir9r#k)3(bCZ}kdz{9A|d@`q>%Bd zL<7o8dgyXmM_zqXOpa}9%cyhFmNmg>1XuAIj2kll1y3yF9tAof$Br|E1(?j6lV5Qp zj0ObP?W|4?bNLcl+7&mbM{FCZuMzPDtJ$n91QFd<^4c$yWd&NJMxaK}mgd;fpc-WQ zAl(bP*dK9TY7Z{38LL8dw2df~y}Hj=x#OkdB8kZcGjC;eUD8G<*pc za}lV7v1)%yj-3r8WA5mU=&VgTq4eS!SQOf7(A^=iPMQ0f(B3=t_f&t>vn3gqiwf6q z4RNhCS?2AH&{VgZhoIl#C*!r9TR25ehv8)@&oxX%kkMgi`<+m)olv9wov3pqOyHj0 z5eRB{Hmco>#X^->z%u<1xr24dJf^McrQJE7mUvHS?%VoI_V$T-N*) zISRZU%oBqsIsMADchuZ mL*egD-%<(S8n6Z5dmXOqM<`Ebdo^si2ZDxaNrb)tT zO3r2`739;RUhQpT>X{9#(L1Mm&1F)3efNK7VmS;~9L4saUQ~OV zR;^NfZCEo}4EAjuyW`%jfTq5q^B!W9aP`l2X`%RXe-c;5V-TKuma6QEnp>#j%!n590+ z-8SOA%!5`nH7yAaa@NkQP4j+kWqdO@ls2W?;=7SZ=FiY0JvTY*T4k^5*Wvl_qg|7D z+tgXQi^At(e+#DY<`uiK=-(5qor{tZ)EGj%0qjP1MTA;Tpf6su(JlU9)atU#3I7a%|ca06dKC8=! zE1a&-#JIIHTz$MOmJjc>vgmnS$Fn8@<1IRxTpqudW|9Na-lNU-bi{r9$JO; zX9V;Q)7ik`=2>8{5HoG5akcM^8y)#Rr*J};JB8aS!_c80I)d^mprx{|B~rkjC5S1| z4j38$!w{*Hk)QW0@0=dwO1pcZZSKFNqxbcQ(RQvGlFSznm@=$Uc~SG(BLO}OhyOyu zZ_JH%i&iR=Ne`9mfwo&`W`abW`+YT3PVpK7L?Q=cnldXNkHx^77`>Wt(jyZeu)7`N zrwN*`QrbKg=(TV*JXlXDgTJOHgO7-M;g!kv{n8YwgV78SWBE^;L%sr@&3ck6wRgW< z%U=|X%B*hE@F?MA?dS0t3#SyFwxkv)ax>BSJ!F2#H1yaGy$)~G=MkK!{ma&!PjT`wqq;W}kEKn>pr&n2r5j_8``+vLGr&!vT%~8*Ias6K>T_ zTY_A$-sX0sLT+M)SyO|7%()`60iX z$w=`qV_9&lkDNftEhjYiD8$`1F0ZejvFcE%mV;4%XI#I3srG`E?R*-fm&3Ie69+G{ z`yR1h{|1MW}?Whc=q zHH9v~)XD-<%hr11p`~mGbR}7==oKB$Hfz!Ca?-TX35y~__(|w_XAaCu$9ehI{cvQY zeg^GCDB^uzjMIJyT~W;t$TY{v(M*36c^%{-bN2hd^%fW_z_t1$$+I!4xnTqzKr9+jp+%K8HO@cn<_&FAFugR0TqdAwSrtI7cI4PbMdCI0? zSmd>n%Cmt$36XP=I#JKLVnI{0#W~nO zw!~UfxNV+QJ`IEi!X!1+j50g_z1tY=D=mi6{ty{I&&KAY?JsILcAa>r#PAqrQFNw;D@o zgQCX)E(Y=yn*P?k5B9I**cf!h40%M^%JfNUM@p~`?gIbBTs`DCBW8L1G1u@IOnInR z=NA*Xfd*E%U&n%XsF&#jOiBZr5E}P~=?9z)r-p44T3|K|n*(4R%+r)h^>^)P9~A7f z{1NAcznYw&fbTrHQg;9IA_@I~yAIPLlwHEm-DdmS4nPe(A`e6QKCkC$p15-=Z7vn2 zTqX5JiepFb;br}z$c{tr%igS;&~`z0U=)dj;RG56GQPk1{1*+gjf0vho%V-` zmhVPPXL2G3cv48otiQ#|f!ytw5@0k;#@-d`A_fu5+2%!9j_Bzy$5lRPAGbD}4O(E6 zirPu+p>Ynpz{5)CcRHO-P541x{(y6T`_;t#mn)XVECSRnREY^s1WzllO9Ey=rxb-kqEoRsvA%|f4okc-Rrqjtv|7IJNREYU4 zo-z3!m3jRo13iDB4D7D{ub6pOb!ggG3OTOmsw_X#_goHb>u&V+lBwy_>Mqm^N5o3P zEa^WhB^T!$+^duupuOV_4VDB!bJ88(F6b8Z#PbO)Pe-zp8KvSVrAEt!?LSqmHKjd! zz}!X!ft3cZph*op$7kYh6=&wB6+5oUiIus)H?>PYM25kuPO>-k*Z8dDO(RHA z^=#JI-!SwnD^x+GU?4x%3*oRntrl_pVmlvvcnBXrnr9pEp*t<=OT6nkxsT;v&?XN3 zQZsSJ?;8DsXD#Q>6-U_l{W?~euT%wUpv6e~k|Q~Fl8sZbQ|432*oJhKYb6AA>p+|H zYb!7-G_OrNwgom5`StE;r;Wh@HQBP^}$-^mf)#pb&Y$ugnY3rZIPWw<3Ote z0z2U~;VFJ00pg45;_CP@qrP{pd#X8mgU0DNnjk(a^C$m_0=Wn%7~gYrh4GwzbM?in z-7h~2=+MeuVZSgqsYyFTPA48rX*AJCUuhyI8ZCBf#e__|C_NZbXMHwD{RyGEbfivx zVY_l!qAgO|=xY;~F>)#`-=Ul6$me4z@Dqc4aW7Upp6JTkr*j3Yx-y*+uRD8G309r{ zj8)u>lRrdxEG6@4rC25DWIyf_Ywy&QzQiuWORsH52XrXE?tH1`j!%XRNnbar zX!eK1vi9y3^YQZL`O(S&F{o5MfZ?<8JGh<#Frr#|>71R>-4Q$)Lq2l;Cbm9`FQai( zvE8ouNN+hjK23q zAk%N2tMObS)6h#ru?ClXBT+s9m-LwlC(WUJU5ZpntmM~UWmW8|>?DR6Dq~VQHRhR1 z-xjZpy_98vsub-b(>j$@v{b^N1ibN1#^$$-T7|@_yoEnA0jl-H=lb9fGdjVQ&JEEz zY2}?zFBO+zS6Z$-kH6NEHx3Plq@Ng!W@FjwxQ-6lYq%J+*wxATR!T+1h5CO|`jK-T zA8k&La=#@Y8FM;xE4&Y)U$Xh`2(d~R5&oEn^|RF4Nmb;1 z>VK4C91wvl@4R6UYxWn$T+p5~Qtca`c8J$1izgwuBDu#s$}XbLM3#$%jrjGZAzn+Y zVL;?h9WIieLoR{f_N%d&l4oi>fibR$B2?=`-lUiKlE@vRG@-~@SR@iWv@m@6aQ2`u8|xn(Yo#+%O(Ofa}134 z>}-!a{qb_ahvVV?4gi$M^j?Tw-|(pI?kscvqYdR`41S_*Sma%i!C5J<#hm|DI9DxL zOs07#Ntq9%nd4-}nPj6k4>aRBgb#n znR$4IH$HXpf^4>Ge}ymcRknplM0;|K)!4+MMh+WVt38sb^(R{~#-)p_yCqqP2DzJ> z@mrgyh^h?EF;-{^meq5l>(TR3r70^^tVG@VyY0QyIGu$-Z4hj$`e1i1k7cslagyo< z$Wg{d;W?~K(+&~6Zi7C1;!YUe4vyK(N9zszsYbDGddlA&P*sm=NjJzn^PX@3 z30!hCVcdrGHkO&b`A6M~?aH1T3ziN%nbTBmX4s2q%vV2WSRoh^z5=(hYJw_(f}lZQ zAy}2n4bHg^YO#5-*L~g<{?@A2B-e^5`iujTQlF=)Y{bwF75znns_ZXW6(c8z|WqOBy}(m*PKaL_8v<(^V}qwhCX|oo1uL;nrNNPVnVX0rmJ8m2%T5ta7;G-TVfSdL z>VM=zVLn35I;N~bKbTi;nMK&QC0Nuv&hv-}rWeyT3p=8+XIzL=H>+Doc5l0fdAJ{L zzvjV3*sO6`W8$gRu*^4FNL`Wi3X3syNrMs$EHNt3OUIGCsj96p_5C+j5h}DQQrAzp z9!5$Z>Yx%5o+K`w0>Z|*7vN4==QglDvsL!PfYJ4uXT;X(3e{kB`ms)x&O@6r!&>LSQf2c-R~q@2i#G=*J&<9=t1(C_6EQsN>v6|}$o=5k$&y1L^VwXK`*=T?xkE&k zYcurRm;#2{_k+A253ldIiJ?_Yrt{bD@7{7O7XEYa{s+7bd>7e!{kZoL` zm79GkKjs>r>NSzJAgvI_>r*ydC6=1Ht7orV$|&fViXg3pC$=jQ7bBhrPiNwJ7wzBE z+dG+5zkiET>Gw5zGW&l2cdgL%EiLN#u*U2vyL9~{mPtEQ?(+FhuaBE*A~|&B9C5G- zYauP?Ld70;gccwo-w4<8=L*d*{GMj6qnPHz4#OsfbHGw*w!ASHc7m?)S!n3Hj z0**#`zuphf@98Z0dhgvV)r0;)UIO}eBmXk;6&Hj2@u!m+TiMm>opmgXrR#iu)>6(L zgwC(3IHpI)Kyrn%#%hj_*?HIJo#v#>s!J&&60euc!6Liaoax0^-O`oZK{R_qHbJ7Gw9t;s*gfhYdw_txbsQ|U^q-6UMU?P#APBQe+!3ZckZn< z`(ndULG|(2cyKFh^!N>-9N%TpA@`dGLUX!-0w7&^4>pfG2{FbLN-}IjKNi$S&fk5U zSo#Zg)DsEB2P)|Xe{aBn9^$&v{R976z&652zFw1x8nL%J7g!&uvirR`g=SUT!^`2E zOF|4V!LJxy-~i*Ymi7gHPHqYLvU|I#cNCX$c>1SxR1ZH=X8wrTViUbpQD-KC^gb6Z zqo%G}=4s9+(a?!n0@`Xe&qsC(s|`0VARb4@=Gsx4o?fDlzdv$AvdVM%3W9$4ezM$~ znv}ljx>b{u@YzlJVj)!<)ACBW#VmuZ?+-NP6?H(3E2j}>dyd*|B~P)h+Oh9IJSOWK z#jeX;vGqNkYa3J8;sGW>0chQ5sZ|8#AllviZz%7tQE?*E`V*muMa}X$L8BZmv$mSL zo!6o$n&soK=yz0KNxna;CprJf8g3zWo<``wSH@tUE*^+4ZI@y=J6$Qg^ATK2j+7`# znRd?^xayMQw8dea=SdQu^WR7DbQc4B7v^p!tOw=I#gxvSD2&eDw(sug&M|H~vEK1m z`Zc?k=A&zhz9f#;*Lh!DL)<)=+w86r$U1M3m%;B**B>3S?a7XULEgM?F3$f_-ftZi z^5#VpGUztN+lVUE-8hL(=@eRdMpGwP9c4cQW_$C(tI{Hp&AVoCha6vwk+Ct#@r*h` zcG+C`@8?()X+wAA2Em0jZZX;(!dbkNR@grpUY+2G zC`WE5t--u!9U)I)u|4x(y=rlX;@`BJn&cNMsuc=kXMN)dUs3sIzq}D2*f)DvK7DwM zvd@gPg4QUcC!H2$s1otb;WdYA7-c8>DVis!ua46d<39q=OIAgbcQV!Xs`_7##=F1H zi+BQg#!8I*G48!FD0@+=uqi?JvIc?%L_dKAq>nC?D*~M)pp0 zB=Ls5X93E{xHux%aMgH_KR?<*_+j!PVq|e8+{{WCFiKn&!c$WrsjzlgJ9Kkt8H9Qsp0y@K_kWZLS&iukI~-jf7gyft+6OLmA7 z_o^j%`pUeIeOo3T!RX8f>$lQscT=(~<-40mKG$sQBPx0Ioi5v3^REohb_cc0Lgiab z%WHf}@~sl(JM21U%)hn`U{Cj+NyiOORP%9R@0N_{9FY`gkp&lA>(!ZNA8^_DRpdjsU$!??aod zC}~IEsDkvo=T`3|R5(uIK(u))WgkD5?lO6;C=~a4BAffjbAmNMq+0S#{bKZTv z^Zw$>;vyGot^2;`m~)OXhoohY7;bdw^Qlg3ZBS5{%%v+0&1VXx5J((xC(_hVA^O}HifLzAWa8!AGX_1;fUMzPGMDe-VDASq`|$08~X@5iACBM z`5aa|PEACfEtZ$6vg|G&OH{g>!s#`kJ4q2Zy>Mlfp?HmnbmyM&L1Q+L<@itXMaBE2 zMpa{`4FYp$QIkny3-oMfe~{xSD16Md8SdhD)W~ zQ6)%%u(ow*up^J(o$v&osJNawZYRw62jq7fM&`Ua4`koHR&w{!*7tGfd09$t(CVG3go!`m zzx5VO!48UHVVi2afa7hw){1ikjMb0kmrW-$>NIfVEU4K9-a0Ks_k|VAcAEY`z*z`c zbHG2*li$OFz%P_%>yd&N6W09v1$_IUjkQ{q!p>9m6J&c^$DEI_7l>DP7VFWjXjsMV zS36XQ=>D+Rq^Jp;q;*rib_!w}JQ4mD)=6j|ZJa+I)eZK0*e1+D`~mfVpK~}^baA!K zR9~6qidpYOEGw=* z8#R7yXlFRn8f-blPa3#c%;6CZnunC!5k4tmsI`5tvC=WJQRmfPT-ZRw+laewfA30k zF$+$cAH9XJGMXeft@TQmE{3M|G)l2FJZRdptQw7#VP~S@sQG!7Xd}_>jyYHZ9Q8(K1UE4)Md4jWb&^rG8u5BwIh$3-MK!L(m!`EuIW-q z#FD&iF=CXs*q^;I8JxkyEMc+`lKuNRO~LN3GdqC%3Ogkb#oNV{C5}Dd%-hF#u>v2Z%#YC-1b^ik%{gjCq>xzWOhU4=?V&vwjOp`X>F8@TP z%5J8lZs(SnJVbo=m!-46^%?H;3+Vwy;?(mr#dF4fUHmNf%kIv$;fbqu(~;eMIz)O{ z4Igr~Ik#i2Q!s8|60?((u+u-Jl%ddVwtt}h^Vg$>9roC^>PFp$Zhq^t{z+E-rYzkx z+uy}9K3-k3!@8J!-&b~=dxM@6t^U?y39^Rq=)##NezivXpIel0O=l-f)ukPR`vVch zr)9;n`<@4EbUU?aXi~@PBKe80Gp3ntEYIM6i&sx7+-LUl>5o}K@-}5KQ%~ncg3T(# zV;o;6u~Xaj04DRVchX;s>Wlq#)XPgq{?Rw@!9)7dcZVl?Hv>he-6YI-l7^{rG4wlQQYN3xKVFpPcUfBh! zldQGd{T(yzqw5)ccL?8-NpiNcrQP_h%ED2Mjl~E5GCx`1ntj5;h%dUTJn0~%dh;tR z-suLG#oUn;Tr2;o2-ee9O)6{H93iJ+9R9?cRfND4AH0if?7==8^kd~LCeEFYzX?fN zLH~pqsBnj#sRogDh8k>R*Y|=jG77uOtI|J@#y0U*^V%wnI67t6!Xu2BUX>2Px=o&_ z2SzJ6R;1kcQGEX??&>|##E;6BqPPRZl^D(hn6r3;&nEj_v!jo-Ta%QrWEqJZ$Ak!P&g;2t$H4~^GLK?l#qbk{C2-?4 zZO`Hf+b;^E=;YYZpJz#9=uLyefa^S$U(ugy8`O_~{pm))QxTXufBeT(-tJ2)8p$`y*dIG4+5|dwJ+n_( zER+BuoZi7@N@DC(@Vpd}qcc|(qU!esaRFQWS$Y4$!vDu!7uTekGQ>l$IJ@7^ZXPb- zbjP~9xeoVYPJG;dNE*cSO{G9}2`2(AV>pcvC1z(0a>DnGQjTSYGA8oHNFvp<#mUU12Z5@2ENu)#E^0aN2 zjguI0o)5#>=aCN;`Tgj1qi5K`$)rgT(iPwJGpt3On?f&saJFViIjLu6{`7H>N~vy~ zovmcwmxjF#Qm@MX1X?0;RV?#*fdAMl;(eR%$8YI|JeNoQXRscBbb(}CbEJAe=G%Q)ryI65nh2~?5vJ@YI1g>>HW z22=m_M~1@U+bP&#>lKG)CpngZs_;#7N zy>D<+Kt!Nt`w_6Q#;q=ltcks^iuwL6l1p`;=F`KofKBBh-JO+}Xd)@Ap`ydDXNRb; zmw;aOJs4YovUn@4{y5k_$)}4AFZNR&>n@yywG*(Z#Nl`0A#TfJ7}7q@v-_t>?rK|> zZUpe;G(mIxtz11uCt>({Q`G*UOOvyt_w1(sadI(Hh7%KFJ|T`?&2LJrA=i-5TlHF= z(=}Qw(H)g`=XDII=9uw-&PHdR&3l+Uzo@av7M@L|` zw&IEmQa~_wg~Tc4)?Q9}z>!HcuWE6JZEts(@y=h#92vO@k#oBL^~2co`~t#Y<_xdY zW@X-K8_}lphM*W-+{4ES^Q%{UHK&Ts+_t>#ikjoxwE|pjasbop_3_<*tarA(*EJO- zf9Wun@ys>iUaGD9{)?z}PmxL3qsj6w_1Ykt4^P6_K5C<%nET_5e~fm|kJLx0NXWLf zi;SXeGl<>zM`!mM);o9CY+pNEG5ndYTt4F4czRE{Y>`0mdX`U`jMJfmX*_Qzb(Ow)fL2(p zIX(Avwm6Np=)ytz-Tb9?AfZ0Fos<4Ix|Bal5x{yJYli}xC&WRIrW_X#&%*jZrsiZ% z4SGkk&)L2yL$&|Ire28{2lpRBI172eq>Zd)mznta!mW;vgl~B{X`F` zt4c2`u-;V*0gzzRCi*GFKXWJ8oNi^0>YXgZH;-17QwjF@D*f+7cMC>O%vl{Xb{f_# zwdIQ+;BsLm2#U{yk;v6=tq}{!2kf$lZDEd4W)r84a%8W)Z_|xVcIP%m|Bp9+PC2~L z^IthEKSMO?ot68w<1<*GVUaoqzU|iBhYAC7oeGtc8sHGPyw-Q4$%k4gDxs#nok>dN zjrF8gLmRsern%cB?N~tDb1wr^^R}o*9F7y4IupCQ*SU^ZBoWo!;Lnr%xt?~b|;=gC3 zPSjj4G4*`ZS;!|`c9l{^n|_)yrI2e+=Jv&1=(bKYiDF^h+UXKIv%BHOdt%6iE-W@2 z_j%e^d2*b1lH~ZHS3uplugXifpx#gVQ0mF20{^vZisN!_y$4^1GAnFV5GnR`ih*YfIlJ5a8jlIWvtr+k5HXSu3b8im9GS_ybVIq30Lyy$%kG_*sX} zhckb2EAxOKP3yYws@HK8v>7P$MM%*tsskipzkfDiVwTJVhqPqAp`J>fP>YW;TGE9F zPv_a}fLmG2&_r8Nej#ADOUnie^X;}LC!?RboR;iCV+`jH_m+nwW9FJY=TWL}XofDY ztjijxOr-(jBz|kggWoQR;rG%xIFN;XoXghnYH>9Uhc)qA z3+t=terTL!J;3AiQ_Dgm=~ufN5kG|?50%>3`qffMC-7#*?_^zuEer9Vr8+$~(f7K1 zB$T?dPdYz=S)gG`e^oHV3?tU|xiMvt3SMum{PZBdv^37*)%QA<=DbrW-7U-?-WW_O2S8b7$N z=;)nS9*wV|NlRyYe3#uCbp5BM z6?DL}iaKg{cY3u>*~G{?{^0zQJcqVh_*DJm#1#tt%@|`?hpF>_J)Qrva#Pu>yq;o^~&jm^oCN;tRE#N0nQbaxX4&ar2^rq_?`FeM|@%UKuv zSmsBiebKAM&mX6&3MXrcCx@ObEiLY5z5bqY-~YW!=JJi^g7msmk8uw{sZ>;m@gBI} zLZBVwwPTD}YvY=aSLy%^K;EOWc~0g)`i37Y3ncJd10I7z@&qoi%lOy{iU%+|4FdsA zWXCnGU8EwzS(XNkczAe(v^cP}nRyQM<>r$?XWnwux!zReNfd=epw?R^!TS|LJSzZ> zb1kbl^`^zvKn~7mbyb>3Nj70b=aK*-g9%!x+?)}nV(IYWrSrXYpGdaMiTEfsx@B;M z-0OwA<>~%QbQFKagZVy^yzN z)36@PXgAqsXb8n#6{e>^NFJ~q;#aW?R2gqX`i3GbCDzF!=p3WyI41Io7f0YLQp~2X zm7kEkOT4329KPhmcmqDMJ|=f*;I&3wUp(B=z)r0hm{(tB>8|D$BHqP71%8zoo(aq} z4dpLxRQAkNWM^DjQfN#~hfH>30T`aRj*g2`rr?*!OV)CRj$gjMJJ*UixX8rZ1C|(#AAj)nE)BfV*?aFViPgm+s*SlwpGyAJP8} zGyJ_Oz3R^=R^Mb<7^#mvSC0$d33`NM=%!5sON>UjzQTq7N%gSbge=T^>r+;p`cjo0 zo%;9fZ;W$Rd8^^g>WYk9y)ukzRBm~EOhr>Z@)1?(FQi;Kl`A5P2cV6%r4rG4+SOYMrqR5=WZ6q=ae7goz$0}oZX0D)_?sF{t0sE0tsZm3UU-QD0bF@? zr!A-B+8-ykr?d52-)ME2o(6j@)T0%dMBLJ=U!vZxWyi1N47Ef3r;dl|FdVA-m->Y& zJ$t;IFW4nC{w!{L=b^=BpoyG=_}F<)=;v;KW3lXV8RfdzVZ~7AH}Fw5d5DmYJZ^wx zo0~(`#pZNw-zBLykQ@*xu^VFj`FEU-z7`D0=S1m)a-x)8}671fOmo{r=?CGe_pMt19kB zi|o6J0yqzT7T1mfl7(d{suYk7Cwq1x?w3`j*c)c2X@r*UtCa%3DS5p`FQxZCGZJa% zf6&X)UhJMr(MedC6y}0)R(FpO>`t0d@ea9SZLeRKPCXJ>^rTL9yXkg%x^&lbVXDWg zVS+WAMYh-hMX%gMe|Cls2-TgG^(b`IAL5kD`BG)B{a--se~wM&BnoaEnneO)?bm7= zCJ)-Zfa6WBPI4{#3k<(A;wjd4^}@k>uGHP=Ny1gy_YT!($v-8yI=%YUP{5uVM-kc( zNvIw}A|4=(_T&FV>#eLyLgIL>Yw-R1pUHt#^&%uwIrsoIacA{T5PhZ46>j}NJ~gxL zPtk%?6~}Ls6l4_giQm0`8#MA{n_E!8NkA5h0J30nbRqfp4r5 zZAy~xf$16qOSBRiuD}@s(^Q4?NE_ey^)+S$AS}<07 z_e5E4Fr1RLT%LFIOK`9_?~rlZGBmfS`kbQh^X1Pez``{;aa&ugAa9ovrK`<=(i8Q# zXU^_e*Jt_DEH}o$lq}SD<7D7uzbW zX&HBC6KJ%5RhaZeaH8w)=W0Rc#lc@J^RHV(SvT5MD50`Ww%MH`6IgBv#urn_JJK32 z>4+G)%VnGo*|GE%??P;&bxB3RI|aPoF0Sw2`e(Fhdk5&MC9}(i3;63Wj5SI-td**M zl7Hzgl@H#ohs`%4X41;Fqb-(zD5g#FL7H^3i9tq=n?Ia;lk93S?M;p$X6?<&o#g7!hySiLor7Vvvv3MjMZVROvfQ`EM$MMS>E01j4YzDg`DVl*KRR&XV$;$t(4_} zxpL~{%x12rMBg2z+QhV$FF<9$xBP z?2WXA<`CZ4B7i59!XQY@*pQXNuX?q@A>lcjrjL$sjQgdtqR>Jx~e zfm$k7<;#U5TVH@wT6|xmsiU?DlAYlNKw&OD=|yL?IaKsorQiO}#$sVX?QCP?%?t{& z%?PuvBShtuqmybr7b5Ws;amv=o9&$iB5p67nsOT|*LtRG^4O!v+tKoHal2r;n!#Xc zYZ(ne8s#E{`D*EV%qL580+ceC|2e_`&slgcpuJeNSI{SQt|qj`4}HkQ-$&Cuk7g?R zyLh%W^dL=z;Qy4^AoNuD&0Faw%BU&=BSkmhlGok!(6FFG?*6U>xITc9CeALnzOfbH zy1-NevY7i-Sr+jJiHq{S3z|=3BAmp!uJj3AJH41sVU^)@u)uJcvUTpO<;rlp&W;dX zJE4{-%K~x^Xag`=V38&P*vF!v7k@W5Q2Jcdx>KWr*gPBp`TN;d*8>;{0)y8osoL^F zH9B{-6&+Wy1C~EwJLbna3!1`Gq5fm{xB#LRLr(77hLq?%!^tD(3*9yddtqO<(OJ|5 zh9|>(_UQTJ*JMRmCG6P+{KsrXv-K{pDo|q47>`CGqmGwZ+-qo}5;oheodM-0virty zf#F>9aip157%>KW4&8=+y)zK;zI;2DOfx#tuw0iW5e|iXYDi&p^RI7=#<~=Sdp8Ve zbgqshrp@cbR+Xh+P$pzSmS21s_$l}5ep4PAvI4q4V8n*!-&ZU?i>t;wJvhz^yLk`p zK9;+FJIn5V^$T$oT8ewXX3C9W-kRVI2}Xk#YT+Tj4#hwB^iKpZU&G_;M%{g@zdvwh zxF+z6anaL((}BITPC27p?k5H%?OR>=qRT3zw%oo)wc19ACpTDU*G6QUl2XC%7p~=; zQrGXJPN8z?m5Dm!qfj-z&9P~4;gqVX{3?h!Mvvgm5Mnd(0a{E&)+?-9E|&tr+9<>) z*rWR9b^z)v&G>I*mTtSZit*s8kI1dXsfjIT_a!yq+T=48#t4KX?Ie1%Xz2j_MT(mW zZ!7u@0_ERgNVCP~e4H*QRbx$2>%N)#&uyDOT!+6;#(5)bIi&BhT7DmN;ZvrJprV1q z+3J%7g01T|&ZFl_pG~hfZ=L&>e-*kcwuQIKy^IyhrRACZO*&h0)qn;nh+Es%-?u=F zM?1f0LyRD){Ez~qcQpKM%v%9Q^7zz1o5k1KssO;1rq_+E-gzl-FA2TQMeUe8-j!!) zGsqx;w>T(uIq0qg_j^U05&&iP!v)?rEZn3}Znh&AzaBq(6BQ_Cwt@@N9lT+!oTvBI zf)|#8=cB7MRIHGn{N0s&&=FfZ$Y~Y*>aCB%L>ApCOVVBBUR2zovqmNGUB8_Bmb!$y zKA8k;^_I`0e0F$pQf09|mt}U?xkhl2A3v@KC8WYVa(6N>r_MpU9ueF9Pc;w8YDFnI zYRVnb$);@7os*;7^@*H~HE^-KhmB-o#Se|ZH4d^R`gMDMrxoCsSMk}=6?@keEsjky zzEf#)$IfdlH@KQ%J3f72qTLr(!-@F$_U+T(vy=V}qh=9oi_1|HTZRNm&756BZ};sK zGQZ1zY4bF5>Epg_oqc=zTKjU5pm*iFl=_+Re)Cab#a`PcIoa5(hK5n9v-&>K;_UiV zS5B{4Mf#ocyWU8hlbOEsy?17#K`uF)6_eqq2cl{poYuHm@tzy9OA8umX{CD)CNAEN>CiAIU@{@81gmj!tM8;nLBREvl6?*#r*ryvZ3w^&G# zcM;$m#}VQqn&K|IR?xNha{0Qw#c~XD5`Jr(!)&s5%Hd+)*?t|Wgn~LIu8wQK&uOZW zr3>{v#%KHU4J|W3PWqV0IMp6ou*~K7p@n?^f6U?kbFsaMjN)2?AfBm;oN}+y6g@Yl zT5PW{$y<^pO&g)V{I;6G>p?$o%Us3(KAIhb^(}hKrNXVNpAKK0EZppLh#LVRL zy-SP$>cm?V>iy1Dj;Wu`@^q@gsI8>-0rNRigf?0pANopH0&3Ev(jN^Vr~ik z0{bPgY4_jAKQTuRWse`pClyv~3;F2UoM>_9=E@LsB7sL6#U~k07Q6ydsmi!p5DGx3Iwt9UO;kT~27HxmE{g!fG3O{gNj=l&u*v&tE z{@n7nkRZdiBD>anG$9rGHEfH{Z?xyz_3a`0!G^Eg`><+LwO^85Pl_!|{ju^Q$%hnOTy6HGQLw9` z&i+nQ_oHHaVQ&hy3gO8(@rZ*S^-RGzQpKYxJ$lCriE3^vfTycna`CUaD>S&TfI2j$ zwcF23)u!jQb+fhb0SYm$E>xAHO0Y?z{_qZ&wB$3w;yY?8vkx8eV(mNT9FHNCdqb6X zZ=cB_3r`ldKB7KucosK(t>u{EjQ+}7)@v2=n3r&On_)mIS;U!3?Em*6lMzQIK&Ty( zRT#tbfL=(Ga(6if>R?mh5$&NC&9YpNOr~ zPEv+2mZ!z}Yp@X4{7{njB_!tUreq;gGDyUmljnYuo&}nplyp_s(=X1vM$2 z5{Sp%M4_+f^@!}6pWh#trhQsaQG|7_}i7t;RKV4=BuR~#H>Nw_|#^FU*w#$)}d=jqXQ)tAdccw7(KkXbfzjV%=F z?Nbz8z$j<_r#(UbqPb=LG;DHiCZ@CNPZd~@N<`0^a2=lPJY73txA)d9^^xlH)GT4i zKSkpZii`=RCJ*()iHJ=F@!ntMvbCY38*UhGu4NHTs#c2i3v?pO=F<$(uT$aUM8sk3 z#`oYvNh$T#WUbEyE&q(%G!^ysf8$AyzRc~hyCm1wc8d&!tjH~3a_aB(@x z_+&omKA3^=z3_VUSS_sPNRnd~$}d&|$l_6^BdOdo!Bm#DC6dau+MY&K+h!L|;3lLr z)@_HAG?C0$W}cw+)2YO8oA`2eaxG<1z3)ls^Y3xrnYzczthG%fULI~tPj79#GGS0P zu{n{Kw6=CH`x|@QBG(FRZ&`_ElAm{p@nygQPiM8`6--X(r<{=UCZ#zP0R)QvldPS_%mFWE?`>xvvnZu4fgWntvgY zq%?N-3Z-luu`!dG)g}SWmjfMBr_i?9XBB#C>;t>&fjAkbsj-kRXIEoasgma}_sdS> zDJ?qP`8#`~`85AjlB*jB=}nX;y7b16?3LZVAN5 ztHE$44Lj(q{^0lu_fp5BPD0K-J*J z%av4XR~%;>8^Wbw&r|3xCF#ML2jjQBcNTsyTWg?AwTtov>2OCEcDyZj2{(t_2s?`# zQ#G#Z$3EnL#so7G8Daa`eDSY1H232=Ie^EGG`;wLotS^W)AlS?m5bo3oCV{EO?Yq$OJS#YdLEHtUUe|`esbHl85cnv`|XQo^5CL1ENcSt7W@fj7F zE~}6o&}??UlP8IYJmKqXkEJj}(Wzd!$ixbE8!7<%lV$8pB9RQWd!eMQAZBGe`Pc5CVpCAx`Na8J64`ycHk3uI%B8>zm6a=Poy9fEAKbTUdIg*N zC~&9luO*-Rh)fqw{h2D<^pEAZ|8`hfJU$s4o0tSe)hzBV%sNBcMPdhk{mHY~MLpet`$5;|T zpO(D8ItqIWTv9hg0q%yFM?88ub$%(+*K<>vhb|dMB=euSZxmO2ZNSXb{cL0nEz)90 zL3za|f@+j|`VHQ^D=M#@$$^4D{#mW}0={Rd2FQoz%XTLGx4rg{5x`iSstDdRgCx65 z&~*~XJunkxvc<g;^A52aag;N$q_?FlsC8uzJ*Id#d-H(IfZ zJo&Wb8kBrP6-x_gY{?iDiM_qYAf4+;u2d_iF;A3QYF671H4RxHO-T5J=In`t9GHTB2jVd<%qp*=o9nD}#0Vmv*z|nY3 zFoa^*Bm&k8Z$#9b985Z+spFNN>nZOIFb^QEHolA;CEdN{x%rN|+Nt`W01b7E9a||* z5Oe6#HuSF4G$*Y};u~1L2m4{K{U}p+!_~dEhbj#hOL8<2?`XICNNb(E{Q1Ru%)=V{ zbXgN;^mwxVUcT0GZp!HZ9Ptoy8Zq#(Y0W3Q$+Gia#NnZAiro$9*_daT@-?DC!p^t~ zULvyACsSP>pD42vPGaQIRX%>q#dsn$$HLL;IQ6x3(N$$VZ`P~ln9B%;tT3EifjTbh zu3)t5hNx%Nyvw>d;UmxLUxfN0r?gU}hHXyveB!PiL)MoPkvl>MpB?^AXTJ%H6J;CT z9j3qd$0hX-7cH7EZ(+@)I3YxJ@9{cM=W24CcABhmTz^&*bi_Rq2v++EybO3K;?K{k zBY*;$tN{QhFc0R>Aa-R`Uocl%3J6SJkzF)K@!Crjvd2cxde_e#1kSnFIgVvKRbaPZ zus9bsNUP{&QLO7wj_yD2_4`k!mUHZ`Z^a|U1n_Vve`f=E_$J+rLHhl_6j5Ybj8JBV z;7tIaz4CB>6u#-VUjUa*5T(V*DXW6_x+pt=`+2c_ot?R&JA+qN7sarYYgf($F^E9` z!}>45yLYX4OTaw0l*y^k+QH`R?ApMSJzhtU2MRjoPu@!=bw2LYm=17bMy}uUQ5PE3 zW|grOvQ5m3r#nXep%twVWD$m64ZHQD{a72qiT_-FF~%-x?b)Zs)Tc_4~~~Hak0Nht*2j_DVs+FB5(inN3b)%^p$i zM1_So&=M;b$%Rp*ToDt+FZ?@NmZz|76T<8%PPWI{qe8QST_v}%!Zn%Rf_MF=2C+bN zC8lEB3$0y;4mi?8Gk%^&uinPk`Bq)oRf#YftykG+yX`xFY7letxemYhzEIVFR>Hgi z8UD3rx{f^fpY2vX^?l zdc!G|{nnFvWT~s>iJC63C67Z2bhNdid~s0+E5f&(>a=3f=UsKAW{VB=aaHvHg$TU0 zFwlN1-5klM86hr6L^>L_TcQC;KaafHcKI3`4>253Dn4nUK{Awu&r`+O?95g^k0r4 z#;9BuY~zv!JRKvwv;*|AJQkFX$)qF%Z%=OZM}jmb-cvbfIJ6 zitOAsYW*$jWXnD&=os}T4DtIJtd92?iUq}hdiRGO4|ztb|2}AU4R~`5{1(NelxLeM z%d2-BTRcaTMs&Rbg-&m#q&p_j_5l90dn+F7nNIg%(ypRW_oOCwX#KB zfCbQ-t7k=nDd*B*a{LKv&(Tnxb10P2Vl&HLI#6gHcD4)Su;PXBT>r&|qUEa4I$`Xd ztMlv63ER8P@v38&f&c0=W{Yr{(%4sE(Fzmj)#2^M1h~wcrCdgNNv4u6nZD9gsiL3I z$TaX4G7}iHM0at3x+Kgm1Jtxcbz!hK%>0g_pIi=n`5yEW%Gi&zi>da@$vr&gT`$pt ze;Q&P1Cj=09IXQVPo*9&73wuh9uUYj;H0W%-HL~Yno#7hgPn6#q<`;_Sq0Dj?)Stf z##AjL;TeVwFQI(2af^ec;;YN9#*k|2aFEPjy<6h(o1)X)EB?!|20;~O+<>T^B3j2( z-MaCFqmK8b9pH5-?y&ipRi(*N{o-RrJDYlca(zbhca+OF+4WIOWXw`gWcZc&TS!Kv z8AydiZCJ|!Z?N*Y?CZ}E`% z?2t5i)H@|r8jCY=HcaTpa27~RV9zQ!{VCAB9{3S$bAg!(xRo);dY<{r5#u01edBBT$h`m@+LX^<>amydqL$rpoNyXqpRuk=}*tJOQ_g`el=J@>}Gzt!mx0p zhg|U)kQsYZQ-SRD^z0tG`FZfbnlZsKY@3b0^5NLK|(b z2DS`GYCj$L!8%~$IC{u?XFKw&t=#99usI=Dx8cXh@T|)T(YsdYwkEO#<`~nbfd7yFUyC7Z&d_r7~|d2mQ}_U26QWzs@Xw~qzqZUnf-{8LY zbZ;|tRY9a{kQMy{@|-FAG%AzMt|81YO-Wjy5-Ju>+DV|h_>4mw$eb<;8_$&$?w_)(kw)RyHz`MrB@0~lJ zyw@^c9J9=XEw{soD@JtZDPFPh?bx$$r1AN^@jZIhoy}D&B`afhK5c*=dBK>1KENL( z+WP}!lObEU7J7qEf_}0sgR;{=(5xcF-wjB&Y+{+>zm`W!DGAT_Jhde@k@kyCh-uMZ zFz^a#Qp1WTGP8bLX3|3#qd!7$#iKnYwBSfehZQcP-H{nT#kN`>Q^B(xv9rMy9GWyN zcHH6Bd9xo*GMIgtyZzju&QJJnRfq^TIZ=J0gft*``Tr3M{~s5z6xZjPy~QkC4`-$~ zaQoZYNGy};c~$Cus*>MVWR1Up{CEPvD{1s|008yF-%2IlTV6qaHB$qicr3*edYxzR z=Rsl)Kg2NL1G4b@B!JZd#M3h2{*puOTg2;AD(NwB{Mf?&`Nn?STi&A^pFiwcaFm?O zqZDcp1O&}bblaAE*olyDhv-MftuOC$sI@{YLOS^OfJ^}v0NNO(`~W4lSUoO8N0!s# zQ4VbBs(oNE&*?AN(QOqD%uyk7A|GC+;;sN;d}hdvY^N{8Ufp{%Q($@AVFP1=j#J$( z0j&*aBBJezE{sveHN%lvVG0Dx%+j)qX+V8N0_60eKzrD+-AaQM+-xO!)Fp~>5_NO^ zv5n=@w~be7U!q|rU49d?B%H8yQ=QDP^dlldX5*+BzO2nR|Ey|V#?KD_iHWXbEVJ_F z_bQitxfqgE8`24d*#ja)gQ(*NtKQu`9y%K8MYwvF`v?RTEtUU2 ziRFLiEs3QRSYXKqFn*4?ix4bDA#!p0d)cjwM6VJI_S}4D09@$^{2U;uMg(}HpjfJ! zs}K_!{2nY0tlvQ!rGKNDKI)s=4;K>(?m1cjWgQRK360vGh6gYTC4-`fm;5bx1ndW1 z+CJ|C?C5A=anwq*S1nD>AnAXWWAomtV?!A8iL6% zD!tZ8X&{7MA!e)i?T=a_$h4*?O~3W6PZK+0lijk*>HY@2gz+5CNb|bXD+P`#qvrdd zNP*9@UuAI;Z0$t_!?rip9}5-HUqZtO`haI_5*gKXkybm!crzA$sxF z$AA26;*BWSyJf6Tfb0kM*q~wCssjm_VpYiCL;UgXE-eBSe>8+44OdXMpnNM>bTZ+N zr=w6NApHvEH=ReGsd~`i=>pois^RyAe#9~H8)2nyzYeAvA-il}eY6v$VPZ2{wsa;< zc@|Z}hZQHYbo}KvZ%LEeRxfL_k|px{942+a$i-UfN2T+h0BB`;@%AzA&4$B?{Y=b` zf}3cq2g~(=A7ht^OMhrD1FsSEtVWUp$n?|eb$=iJu*c-*4So4Q^MAJ7=hzMr@WG0CG<}j+<%ktuhS+|BF~*SptOjgT)pa`Gwx%-#8^mk!y6Z+WU_rcZ4|^V zL)O}g$s0hka)?6mifc~uen8oh%R1}_BVrO7H+Ea&Yt>0VTC0Sv#vr)Ic7FsSUeTkl zHqbA(q0u^O7Rx~)gE$?sT6|s+Ez`Is=tDk0=!^8*NIJ2n4+zO5dZCs27xeakd zgAl)-$sbX50Fs~i%f%-w>(;nI5&=5tVG1XP6s$C{DD~kw+rCt%GZK+b00;(fsnVQk zdj*PCiDu3}N)0P&cJ6lE#U1w*W6oTdQGTDHZt~aTJ#Ss`kwv^ca z;XvElye`Ey7KxUKsZVO+3ig-4Y)74tYa$NX1=}MoKfJOERYpA-Cp!V^VNC9;Cws-! zH%Rh=m|crhogMzM=y`Jzc$Uai0&HTH&1fF$%i1J7(@~H$XzJ1q6l*#y+YeOvB`NMy zw4T%CR~hzk_2w^$dF9C%m0o;7?H;5DWmYkK10z;pwnQm(LN;@(t)BI(Lox?chXwp6 z0O$oAyO|@954itAQCqE-J2>j_zfK$kuXaNP4 zl$qFTd7Et;Gm#;QMdR?=tA5^>Ic5d!ojGXA$%MAf#9UjuZ*`BZEy0D?-v4;fmA~Cg zksB{?@bz-x|E>A%{Rj~D?3xgRD2fVji9rx`9d!{h0fe9~|2SnH!-71;1A^8a$W&7B ze8wlMGRS#=0a+59NJmzb9*lz}0gM1y?ko^nGFqrKIxVQSnHq6fU0r!?J5*Gqvp~gt z3y6xoNNv#}5EI;^#DlI%Z%+Rs;C`$j@8OF-7+WlN<$?O`7RoNx^FW)KdA9+s045&B zxp!4%J+qeW5z@iH)?djU`;%*xXL%=51}rIi`TPZd%V{Di1tup8t}{8)r`eBsT_Y`roK@q$GLcV9EG;KR=pDlZJpvxNb;Ma3B5A0B<3}m;n9a_o z8WAc*Y>N)A_MQ=UEa?6Bs5(N;_e_IZiw?e$)H~CexCnE**|IN2ZVwkb7LhXy@9fEH zXL#tIx+VMLzw}sXmyDCi6z+1llkEDxF3g2opTl+cyyZ}ojVW(n^RWE={<`yTALknw zyOz&8LFbo->VW7e_E2ezrT=)0aio|1lvjI|6=Mk>ZJ?-A`1em^+qHYXbFGcf*GfvI zHoP(K{A4TL#N2Ax6s99bvvqSZ?x3#cj(b{e8Tb@m$WwWRK)r28GjC695RRR%)Hjlb{x*g>uJgUO8Xqa&aG=?E8IGHZXZ?H$sG z-IK8?;p|JaDHUEyeFgK6+26Z|SmNg(ye7HfD^7Jz^&u&)b5^4=Rq4Zw>Y8uy){Z1MA7k(*Wx@q2U9@%N)3Nx(AaB5ijC)Coaonq?P||1 zYnFig?fYaG9i@P8`lr1a;9;~f0c_+M=1)Rx<~fCeY|Jkd_b~r^7C?CZ55|)UN%Azp z!hMdWI72c2BW*(z+eOMW{s8GeDyJQg^U)!>H!O=$dy?TA(M0x5&ZvwPUA#)PM2sC# zm>YsIoS+do52>On(LQ0@GcN^BrsdN4#D&|NR&R48&)0>^@WSZG68(dnXEN*5&E$O7 zM3+TBY#Y~5YYL+MxOeM~xKwugp91o6r^aC%OBF67+5YvV6qE!_7_xE~vmhihA!CNoTq z)Q6WEsNKK7E9)oA{7OtJpZ_if<3s>*WF@oxqQ>Ubrpy=f8NkPv)tfRU&!;<&q#rtv z)tXt4utIvSgO9hmvR6H8=UJBAnoLShYn)m>2mDdOO;4nU2pTXZ)^LA9Vct`a1tiZ( zfC{RgNb?1toFv^r$Q-6XcGCl%QvY)~$gEG(Yu;Trqsafh$o^!#---F*{_^TT>bCph zNuS<H!;*%p`; zSphz3VcEJMe(wn&&Tm2cjeHyT`r=Tk4#EL;NYsjQcpDy#iG5zT^@$Kq*LODGCZ_JZ=n@UFqK^`HuE-=~C?a<6A>*z%P zcH51wGD`NAUV6RW)MC9Eb@J&i>^szbC+Vf%BU}Y_@FsQxBLSc*xt%P*NTjJJ0g2Eb{ zSdFOK{O$;+GjKyK!T!J(v(j0T_Ja4xW}msPv^9rssHxdD zl<~z?cgVy)qE0~VZ@1?AKFwof9qs)XXPsUq(Nnl>X-$~DJL-(v-L2cJzB^gdsLtSk zoNp_EEy6rZF*Q*UDDVo5f%Lh*A;`a$WzwFZ2%~_YI7EsaFy=-8XA=YmN}uW%De~~U z?pj|GP<(}eu;q&bp@1MC*0Umf4}PSk8I7{15ZQxWasMKFB{|vd|h|3nqUc%VMf>>^{@PUR)H6@n&vYwj@HO#`zcDMundqb8}xQz-V%)O3DB4JKK zA^c{J@zPk5far5k_{ICG$RASEOGOztV1`q9?3d zX=y!2^dbf^9k<=>tSZ?>0{rRHpl=%am^kJ(>jmxdYly>~PSIhdPKc5x6#wZ zlzy#xl2(>B7|j)91F?};|KQMxrDleDAFYAM&V0oXA>Smbluw%oFdnQF7W0|QDOYSrw0;wk9 z4FLqBh=nB8e#&L<=Oz#yR4GeFb@0lwqxxT)0LWVkCrAc!=~&}*K;Q^f$ufo5R^l^y zol#H3BqR;MGr?!+#*yL?AjAiu!=~sFF60%Wf zkOK1+MtQhx=o^hLaQt;0)P;s;`rT+0UvNO!i{Baybl+S?0|UrIjO+zH2dwwZ;ytlb zTy9Rr6QWl+%Y!|6&VP7Yu8R7|hfEJ)O>O$2zd27yT2(?QjPV<|(ND4#sVC4LVZc9E zmLO!6;$86uTq0#s|JeaW+mfXa54j6H#!1MeIs{8f`g$v7x7X=C(cTR^Pc^+ijpBL! zQ4vc>6P!A}Z$caXdfi-Rd%ChPICsQ@c9S*Xui$Z1c;A#MG*i1LsVO3*uZ_Lo{s7*P zI+_rjm0$i{Qs}(u)^0O=Rh`aXXpAP*+GjnK)IA==wh^=%@kNVNyX?GoKHm<8VLJ$< zM$K5rx0b+EOBk%#+#?tx`n1V*`dW;=IccJ(D6ndhit~mqp$<#+_0zRZDbqyK&v09a zAad^2+)jl8Po$BpSLR~2Xbti}NzNG2!iR*OtM*W)M#64(p}tq-!s|#oqc4c2v9G5I z%J`61aHH80M(}p2|1~ON+f&amQ1@tP?11qns4J$O?afKk&mT{AjGyf@9m$;r=n%bV zKR+L3J+R25ahl-90|+!G4g*A!u=iO*cX00hX0-lm@%!ipl>eE75>6corE$0j`f44N zQWqTuU4Op$u5)N%z{T*%1r)eC+Q=SWRXKAWDIel@4gKbQ5#wzJ^>gLLUq*yMiB@MJ zl8az$bmrgfJy?+_sjK+8bXlPqqq0T=Nr0BCE*)ocyN9NwIjRh^HhF6{F<|EYp*QU#i%(~ai@9D zv)Qq}Rg;O0uXIGw3tX}rkxmonQP)T}Rf|DOQvms5q5Y@K?920qFgv_`TJ`VLT&uNN z#rsH-eC6x1{#vspR(_jzayuWybv7jsFH>>vE|JY%h_I2E)L>?rw9W?sx5*fA#MuJ1 z)h@O76o})F!Rk}c&+B39!UAFgEY*=|9)$geIerA9H~kvP0i-DOh+%3sj7EI%xE}W# zvYciIPj9bNSCsSbWPl%@yfY~ZDvlxi_9V5SHn|M{6(W9PXYXrPspTKiLZo=R5cQ<& zlh+BG#UC405cuF+f3K~Z#FN+fVv9^Z($lXSs zIN%99Cl;*JT~$)Y4`XW8K1D2lV>)8KV@v;m@=kl^)r2nWZDyf!jKSp3?(z2OEkXpw ziXL4$ua6xRxIjsS!oySOzh4IUI)j}eQAXL#?Zc)>8z#RCL64(Ib2d=RbjPj#1yoQ> zWu-XhD!Q(EQZD9616;tO5pN+2Q~IXd;wuKFprI-H&Tttd2^I;tA`~*F>9TqRCPESz zU@i7s+c36f5LEmV>6Hik*@5h1606tb3ic?R6sf@};SPgNx5EoaWxqPhb(dRa2|s+z zTC64%i<3bgNj0V}&>#ao8-~Q|Q^z(Qx94MQ=Qmz}DM zq3gW2!)LF4E(WjLcxKJGkW-T!=D~{3XtvpKzax1m--j2u#%pZQ2!I%-wiiQ~yQbr_ zMS4MG4-4wbc)|+*)|vck`8!s|C-K8eYV^ohtCw;{yPemOu5?FYmhYpZ*@LME^3=vs zC|rB25Sl>h9wXUT?S{$kRzm$1FRgC_geWNLPJ6uLGPjbjJ|LOmHc#bT9gJ2l7&rq2 z;|eQx=bWGr(7RPsrgt76Og6=9@&s-$bB~#Jit}}N@kCGHXLEu{FxCjholyy%6mjn% z)QAYEuS?|0jSCH%_!}_`ED6ZR|6km?@{3i0)&sNUZKHspMtp^VQXSQ@!(JzOKNv8;MykG7`r7=?znz$;@36V!c09CqT@faD%k z2y-=l(?ZM`mg58<$CSG+8RmB5=IHpGXZR;rapEyEcV8JjgfpDf=y4JL2)3*WGtuC; zDl_IIkP7eTcq?Ph%sU%w!g~Dp8ZBg^UXM9-uBAnByAsAn4~-a9onyXau9$mOKk{9Y z-2m5qlH3-{wrRbnmRQx;M1&88d>~}`lA`ti3B6~6QQlc|V=uplQZ}aKZ>fArt*9-w z&}yF>Vs@w9z6<5(cOFljNk&QpwFpNgJ|-KzU5?P&yto%N3vQxc6&-A;@RqN$USl9M zaR!#xFA`=mCB(_)kG95nD%M4&MyPQlHA8yA|rr6n$;*U@L(JDmCta z$ixRCX2Y(7KA36wbr~dHItdN94adW$xL;)H2fAc{^5TX`#DH~_K;quJ|8`L!d86rK5$Sl4tNJ z7O;(C-^ebK`~?f2rxEipT3%RWWhefxgNIa3H$$z#5!?l#)Zn=VQr&)}`H#j*;uZc9 z?cKt8A+>LC)XI@X~8+p%+SZW zt(e^utuK{xvD)bq67jLo36vd1gkFln-rJ^r7_a zsdF51np?_W<>;DwUNJ_}oA+lF9ciJfqhk8M=XxSsk@e=saC}jN&00L^(ehTHx3^kP zS-jd>x1`81*mXl0O&tlp3*29LPy}Feamii|V~P-!f1Abmpt;K4Bu}HfG}2M1^M)Vm zt%_~1r1Yg0*bL14(A0tbvV#@2P8$o1%Gz38zrnO;L90hS8i<=~i_NH>D9InddLgbJ znSqm(_vK%*z<)>s6&e$V(ggQ;Nuots)7*N-4C_~Fr$0IxUUXW@D4i6oX?* zvRe&}uKE*HUWGD0_TeDrK0W-pkd-On}_=Qppb%xf53yrnEg4sKt z3uWC^IlI-G)snw9{C*D$N*ACLYA=i&p#3CYJe|CKHm}+YqXJvwL0%X>HvLML=_|U` z4y>Sy{bu|c4%8gNa!?xL*C6U39|tOaO;-(0G^K%b5;VCVd{-;rdT1!Hs{%~j{G}GX zc8KW4-&!lx5zO#(I{%rMp6CV@gC?6W`p2U>5tbVou$`d06hnryRo|^g?JN%lmJ!S7 zT`&`TJ<<&v*U{v{^F{v%D6rdoheF@@VE+LOEtd8{*IK)Hb%VFSP{`h7#mkgrY-b8B zCq&!qZh~s{x`@bAXp(esVVPlTu&!?XBPse51f*s)QIQQQt2r!tg; z)Y~Q(oKtz?mtEco3yga@tcnYm#BmbZuN3DIH?9t3Q{>}YL#S;+BqiC!Tx0qt>aJcW z(`h2DUewW&wI}#SPU}d_&7{_=%z6)DKg)8EDkh6yEKTA9!JsLwu!jG1dEfW0zbd25 z=oiSZ*d9gFg@UEQD;lOZU0#2)5%xdacVgk@{VFwoFs)TlFrcidqbLSY_7w7gAc|2W zDY{&RH|L&)k4MQja0pXf7L*s~{~nZ42$@Fa>o(+%=;K3-a%R?1E$ypV8bhwFVN~5G zDLa8q4#=|Y6EiG=bi6Q6HV4GdNms{b4m(rjn0HVch6TKuZAr7UnhL%T0>?7fXtks; zs~veL6NU1t##Q~s7ffi9Rb&blD3rL$5BAZ|QGXQ$eY6GQek6Q>MLHwg6I~4oX7s4AO$MZ5e zk@!HBy-%5b*1R+z`Zib*GNEqNvcub06ly4FXBGLz5j~B12@O@ofBukpt?J1S6A%?v zeX&e*$D652#zvxr@GPO4$~?s8wxnd|lbG%Y&vY@lxX zd-C23O)2{Vb?2d?1pOi5{_4B+tFhhk*)4$Vjz6N~ z8)kHcFdvb0G{7$?@OFi~n%Wbu5wJ?i`fBdvTK-Gbb4bH$8y4=y!CV1l-d+_9N~78~ z=r*s~-hY-UxL7`jH91Cmdla@R+RJ@OtNS+*?2iV(PetVNXOCn%{1u2w6&rh*e_?dc z9rhyiT@;dBf`T)3_cX2$Dgp8=v}(Zo1f&ko1+^sa0P137NsN%6rX%$mP7QZAdjit$ zsxE}QZ#~YCf{Su?#|)WOX%iprE;Q@yFZNI&B)NMBSyQLtY`U@D`jKx{#p;?!Vy>*c z5Nc&=nssyeRrU&+%J}e+cKTiDl_MuKK&zN?s1?eBmt6ZaOy8Qjk&ga%5GI6H5l_Cg z9-BMGKcDc8cx2=)gL=sNbGi-H!}aS9RW$3gr~L zr&GOHJ3onQWkIM>A1a?(gL$5q3BRYS6qNFe{1~mNSZ2t@Flw(lL!kkYTM#e-Jn_#9 zCv<=u6z7eFpuj+frSp3p`0r3-pt=<;FLhEPlmJ<(KLhiJW16*1Q5Fc5OxnLq1HywO zR-`B%9+>+VE)%j~(-?Jf)H6hKC723TzHM#>NUJ^D2IGf*>IV78qFp2LN-#8|%+pLW zA(~!*h|s~bmeJ#!E#!!@|UIO2r+F-XB~drr2^y+{Dn)Pnv;X6Qd2acoiMb@;6*tWBC??){TO6yUgQo$Yv*F!35UB;y7s?fYk({+gPrt;I$)Ho!xTV z?mdyOeZnz~J+vaD_x93l-lT=f@j!OA&L3AjD^#nS%b$(`taBC9jY>cG>Ly|DBn|w8%yDedf(GFbJq2-wI0wL0 z5IQGXaxp`A76;w5=n^ECLia2*6Vc7t9Qw!c)I$0SX&g3=DULnXP?k}aCzoA*Nvrmi_n zIIxFN@=4KBiPfH1xTq7{v@^B(a6#-N%k8z-gCb9*JW}C zPC~t})5K6Wcd&1>n-8`3>aVPa7t=z8H$$EZO`&AmVv9sCOf`6W2VSC)FZFwXbo)YJ z(1~EPunNkm7=wc~p(`BRZyG1a@D`T(?(e1-PkJokr<4Zvy z^N$9^XQPKpG>N_07Rz2nFD%(SHtW_vxDGm4qh|pHrO_OqjM4G&&x2=#@;*7#(z$UiSF^Zq9#T z>3;^&e@L9Xy8f_y+r2S&|6@^(@=$CckVKn?n~EzWe4b{d!Q4O86wISm6&|45fSN&B zqg_yokAM2QQvz8aY8v9Bowajo(d~yypAwwsTr=5y9FbUR_xxZv30sJSL))_G5w=7b zRSE%VTN`z^k$-ZjWpAh+yi6!U1tly?FcnJ;GZ)+xd>Y7X=+6=6UiS7QrPsk*clQX` z8=B-e@$C(EfruN%V!=^)k>{z4g47D;(3flUuY)@{Q&XXD$P!)8KkJf*0d z2598tC6(`*Vs@ZUi2{BTdD0$@hn3rbhSr9e#iZ zF!|m#bVdiHR@RsLWGn?(+~n9yR}C!t!A4zVLwqvS8ImS=pZ=#J@xPy@e}}^&j0+?* zL5zuPy^+wa=(WmM1Q@UA?R$eMc*z-tx4s#rHZt+p24>_pa*^p_Atz3`A#K6=qj45c zj}M;Zynp_tHNBw@ZgX5_ zi4;5_b5-QhS^#TEA7KXKD{QJ%%@>ZM*XLWDKsYEqLJq~_M0Y895v^g4B8T$kMnS8M ziu8_zFBO@l_)=OuVS(W{M(r$ef!%;qjjv_X_F|%61H&AM5;uAUsIQhNmY=-o2$Q_dfA_my}jlG}kTy>9K{V#{< ze?4f)+!$Wx=^JVmTIHF4w(k~-pV2G@cD%}bpR|b-N{^-h!*sP>5mCD&4d-HC^ zg-z|CE}1Z@-+f;E{uNaP5wjvPvOVh=itsl+nI3|*agplw6ck;J?N=+$*+~fip2z6>|9qQyvh2)m zk1r1?yEnBLJ0_RrSHpbhVL+Ct#H#E(ja0R$Z9YJ^XSo)(-!&F6Cd(t?5jTYI{h%yD zI$uynn-LwXZlPGA@FFvrZSBeWoawWr_&!C|I-Ed#vfRS=+IzQq4TYjd}Axc_&8PSXf$kAzZKV)T>rQeTh)=D4rg%V{knD8>83?oVVJ6YPl-Vv zeh&eXtbP0X|ESo_XdKRJY!aGvCm&^o|CP-=mGTL)MT-V>pvXfR7J$$I(^q7Rn2!b@ zrcB7Ch+Uv7vwg(~|04%oK$Qm!s6Ou|`q+2&sX%JaI`P#8q|uY z1aQ2cWV1b~Ld`7sjz)5JTQ+8ekgy%|WxNzo@tPEqNKXhKb597Cxvf}N>w#FqZlr+Y&8p-BrU$l2Ip~$qs@QI7q2r&xQKpQJF<~c2$ zGzaqj>(4+DgrxI(Za97U)Rs2Mwv~)8BKa!eA)%Cb60-PE{D@fBA0d;I9$qxM z$)oJ*@`+w6hMmmSYHiXCDY-ueZ<8$AVu&7kuEoDh4QA}NhP+DOs?4kq(@+So?Dqy^ zz8BL;#FbxP$lZ|)UONRJEjR5z$G%Ll;1N6e2=s8j(6qyJ*iXGPaxPP_Tbt@IyRArT0 zj5}>&-gU%?5SU`Q(7S~kA;+w_25zTacXKq%Sj|?K{}$V}m@kcmWKBMmmqt@Zz2*Zd z=++Cmm9xXkghgV62*t!(9z!AEN29rZM+@m1v#j5*u;jEYq?G|SbeIZX1V#9D z;8k;}2+G%vb#JKm>dNZHCiLEZOwK3#fpv_U$WKy;k9z$J*sbd01t}rJRz+t8BU>W9BPFzx74GHPdzOnqF#&NoxNg$rO!5|XNOlq-(S+xKlAk3Hpl9_ zxC#qX{?tDP`$X(^eYZX9Uqjp)*M9I6W0JT<>w7odGwYThe=d-n$Lf5tNwQ2&>WhcF z%+G)AE%njGZpekjl81a%1sJTw2`ULBX(g?We3G zGp-ALyER<%<=5zZzI>zIAIaH{BJXl%hvK&~^6kJ^| z>{#JmN;0H+AWm0Ai>fxSvvoo?%Uws%Y|gE`QH(atJ26*Eun3Nqx4?M*F77c5h74zEt5B zk3n_u$(LDP>&>|xm4kx~PWWs3_d{O7tgF}VOz)lMk9VXcxn<7ZOK)9yIazUB4`I#i zO)s5G+^uh2*XIBFq|#tN$bDljq>eV9PJr$6UmjIfbh;ijR(4|WTjXxg#U-Hety&Hd zh%ai*Ovg6EwuWiK^zvYDC-gJOo4z5zlaq48!L(z03y1H^P{Bh<=dJuoCv>6voDg`y zyaoK|`x0FQT>xnzt2iCQ**;h9_tY)R^WA1g7=6AMPhM>^)@RDkPxYzjQjwbRPxl(8 z6SWs6iCw4X*ND^UwG3>vf-MZ1%DtJ1x}(mv-H2PDVuIW6ZmK$hFW5%I+<|v;Sf4}T zJJWMr29DRuHr4ZW=XUdLcc$=?61Nay8|!rp62Gf{ajIHRjfR)q-*z|W@OwP#JKnLh zFi*prE|+`jNKH)OesbmZ_;8zD40;H?unFQUxGrVwn+5M_d&F1n=87Y!Ox`Ow-MaC@ zr_BXQZZHRI&yg4Goi!F~KkH|H?p}Cd#1&pvPZpu}c5;z$^w5&J<(e3ek{--f8!Tfe z;0brjn!Hcin%l+Th955_l3d9}Eeh(;bW= z|7Bf~3m+?s8VJ`)kiA=0!uGvCzH*XKW@DtnQdaP-h`~wRY>hK7ZIF^VOdDGZ=72}_ zdPKRGWf7MbQ5+sp=kJC}CQ1qqqidqwrmd%4rH$7Mb@7wUo#dOVcHF}739YqM(!QWN zah$IC?!V9q*@Yvn&j;5g?sesJV>=KuLr z8rXV{K3Z2(Xkm5yhNEY4vXIiDNC!DtUi($?K|FOJU!JzZ!dSj6-{J>}ahurzVI{E^ zhpWC`+$cFp!cNtw)o$&^p_7N%1obl$f%dD`Q$xyz3ehN;5T+dm<(o~G!LYmG zrb>2IDak&8IRTN>W~vvY!-?Inp-yK_X<9cXC8smLGFpqi z>$Y|Y#ZuN4TChT3xN#XX(9m%=q zA5RwKd-ll2((k&qhz?uwhx$d?M91gJ7;;mv`6uV4q4opGac~Sq%YA7 zSq?+`2O>FyE&`w?1!vFq-I-Y{xub3#4+7cifmf0HKb7|mvRPB0a zgP)0p6uMn0$C-VO^M2q^HY%!)XVhYWNId%d!;xM4wkM`#wN#$ZbE?pM8Jsnh^IV7n zzDM`?_4L|eXcTna4Ey~)RWj35Tx84J4qpysBpYY_^MSqMkc`XDcaa`zt*Km4Dk6YK%%Y(1l#q z-+9jMI_C@7A+G_?m1xGYHAQ{tx&OYls#d6kt9d+&6ZrS>oHY_EU$pltVLnn7-$b!wm3aC%bB*W$@llUpIrl*xmQ-#oXvYT;F#>z zoz3m~U_CXxu&zQxQ^&4hO*8f^J_KX?z&>BUSNBA{M1LV|1tCW#Gx*WJ>0ZKPPV-OHy@3boCkG8Gj=11T- z!Mx>kMSjge8-)P83i&S~y(7zOcB{7z&2pqCoU&j4A3pQ`WBrhL;DR>v`oSN_Ar^ot z%22RI09<;BDv46VrVT5{|TEF zgoU&uB+;hnx8NiB=Ue>sMlTt4iB5asxU49-DZ%VKn#(yz6S>ObQ3e5!cyMb_GmquO zKnT-hfn8#qofe}3A0pDkuYpM}+47cr3-?GlZYqIPxxURi0#u7*`oyAedn!hUyWnxA zuTTH8@zqMViw2|aXlrEW#_CAG>;S#2v|=2b7qF%3-8fIZXd0S6%CzA1TSwp{#iKnr z&l~&q$^80-CRCWsEBq-On=wa1dpVCZlEK^03_jE+Gey~ypKTUi<=+14bw~nBOPvTB z12<=NkAIObQX!Ws%)mxF157-Z%Y`vJDDb4Ip-0e#g9^M5CPkBA?3FmRS zyE)UKX8Jr(==oezV0}KOWvVegg$!fQZ$QG-mUOab--gDChvF^LB5Lz}k}$Fv zzcY23?FiQxj|LmjWgW~s8P0@zO3JS^Cpvd0^m=$de%}w4X?Y`zRWqSO#4)d-z3536a<9I3+ed=;?E)mnNZB&`G^gM`+6Z`@+m==$)Bas#PQQJN zB-1(<)}5*+Otg(c>~Q?f-z>bLNw@YAJRW!}o44X2I}&rV?Jb#tPI!KclxmwT5de2D zm2i^1RQ1goupi8M9QA?}9b_cmNUnaEV9ZjDZyoK_5`9g_mvAfQAwubZbknLFjaANE zJyOQ$3!&s6+I1AgbDT#SW$oO~AnOfANNk=?yK+=pcJJa3Zq%Zyaob%_m|6`H*j?YP zzuFxWZ{dd+g|CI)$~=0jTY*HayNy0*xz7zRe1>oEMakh`oW1Mr2dmJ0OpUjjyYL!6 z`a~M);{8A`B+4FU?{#-I&-;7I`?}p*lVDUVmP;oWq5k)-Z_ES2I8DYhm~a36B>Z(S z8d1@)pUR}FmoFk6%WQzXpcq-h=f)WT;86=|3!^Wpk)LcQ`lm_P{ohzM8pun!$uN6s zlQ+&nG^%I~Dd~2)6n#l(-4svo4Z9t^B+3!ljdNR!5Z{#d9!)E%Rra9?DULH#mZR#n zC_&m|s-Jw;YL5s%YT-raw~Qb0y1iJ_c+{tMkSUDu0j^4W-Z=J5BqTq`r_|rwHR30* zv1n>Ohx9U@X9Qk$)C#(hX>eN(=2cHno?fSiyl%lL{l4kNNLQjKTKg^3U146mH(glt z^;x)+d}00eQ)a>`^D*yitH21p&YU;C{JV~ynNcZmLe=ZU8uCog#rvj#94&7*D=|z+&^Anzpx{O*GsIoG(4s2Y$32EADIV zxEO%{5w5%FkoAeE(K&yfzhC-CDE})CKcEre({K8v?4)LzftK4!ASojaI+W#0=-~~s zQA7@e@x=pW$YLg}YxhAo4<6~reo8=p?mcQIh7VlZ{Mdk_ae<&;p*0d@tT7eH z4Oqyr%7c^;rGh?xCh{|*-?5;VmtV_cxaoby4uSZ7LMPrW-VDs$$>gD7Qz)!2&~AF{ zel`{r-Euo&5L)g&TUv?(apSp+Fu?=FRoL&jxA7xgX}o8Xy_0>CUnmb{kH#HWy76e6 zu>o!V-hlTBpCPKsoVStO>i5!rM(lrqXSS0B%z6QG=jSrtY*dHHd$=9P_EY zX=13C+6)!G%#vM7m?`Wu0ZveR?EfzL%HKlf3slCX^`#vM!zRVnojA=dxK|VT8^E9* zC4niX;R~O2`Vwp_$-B#lC7q~<#UQ|8`vlBqmiIKG&--I z_$bOm2Rb0+S?qp#+d>#ikh`xX*rzhxqoO@M{!U8SpSB|FP=BCplhH+y$u8oWs75uC z<62(1XM5it|2ndAd!n?vo&g`cqM}TBDVfh!@tY%BfC&Qgdsm1kmhdnio0vdP*@e|vP`8`I~p8f{1zSw9eS67%^sl>Gr zu6k~r$mnTnD&bz-EMYg1ldt(91owoUsaTeIRXgUquioW6JJ@!pA+K$P{6*X;-8IqM zlMUMTAHmiAlWhQ#+ruO(VVIbG-L6hy#;$(oKl;pZsQ&kK#|TZ$3>noB82 zc$r(Xmh2wy$z%Tc8hO)Nu)4<@^5|LEk8g5=@@0oYf$lXaGzppqVHVuCR#n{DnoqD^eog^B~eNm=nntx7KD7%}ITUnn+Usu#)fa)1vX zokM7r>PP)vpY7~8S%NuQhU+&YBD&cxzu_9e5uABQN`x7J=3wcsdH!~#2`s$mf$SWwemQS*!d~Ybq%JqJT5<~B z)7L5B#4fz zV4JfRxwXq^b&RaoRvgCiwHFeNF1R643~#-!!?G_&-=))0p|^4J*4{&}nl;WEBx=yG zobs9d8h$x>lFep!arcEoki)IU(iFd6_AFS~AESO~zv?S|&F@ZbaBXE6->z-^C}YZi z7H%)}&%HH<>htwZ+JYTINd2$S_0Bqtk9 z1dL4q+1vIplyoYg0`!3qg-YW5z^aECKVn-LYH+mtJy9PcM88e*X-LQaX{2lY*?ckwI>#lYjvd)ysayE36}9JJtF-)LgD;+m z8B2aKS5dm3IW{TJ91~}OmKzciW=t0+6it-ZlTE(An(J6VpC9#%?j)BVywMJ#6#2{U#cfLdq>I_0koRH zq!twYpQW;s>6e-ZZfM^hY{+?hk~@@|?xY9ZP!-{9pgt>2D; zTUqgWNGDiBfnmW8q`?t9jpPQ7W(G#19L5O7dZLScy_xZEJhKv)RAHfOSazxN*G4h2 z{g2OJd>8AJFsX~cFBRn*?wh&6;I5^%acqZg#}1T#mkx@?a3HazV;Y>df2I}xl1h}8 zZod873*i1&jWDQ`e8d(=Dt|=x@RQ<8Ld%z90R-jh&75$5WmkrX5=$cMSae>p+^6ml z**xvxQfi&(ML5$4(CH+ip;NtK@)en^t-;r?QV|=VV%c{I=iEMhXL>Jj`oH<>9F+b} zmVHEwB)EPhD8w-a?j91js3J!-6vv-~1XY<20%=;;Y?IwKvmaW0xE(dDa6=#hCuCJM z-iW!!>hf0ItVGCFbXtw))^3p!b2wpR9)CfaM3x;O-@L%W7u^@&5=*I~yxlsv@=MA^ z--n2LG~h|U@Rg;;4{`4San%L7!ozEZCf!i)a`JE0vf+n-vTb++y|OVoP&icNjwLZjINuJVZkZDz|gd+?S|O z9~5Wlz{kbA7DKCUb#@@J%h&ejuhkDWVZ4x^vLYgYt zWS)IOKSK{>;lsk4ipePwRip2AaUBnS--P}+pLu&}%3NpJ^!U1rtREqY?jJ1$oWreD@$f;VqPd$|1yJ~8! z#Z(|7s^7`*^}-YXsE_@b5S2q{ni9)L)DO7-8I}HdA^s7Bp1f&H&y=zT7JzJJZ{NQI z*~_9QXeKkTy};S7^*4+73A)>{SQ=T$?L{y3SGth7O_<5=+>km6=`Zg7>8}w{wk_3{ zkirO2z`tt227Qv9A_B%8Q)QM&d=yRL43Q;Uo3PD$4{QN-+m*%#xLJja`T^|%q^v1k zKrj8{ZC}|n$TvZPK;5$1KxP8fBBSL%LDGaz&a*bR7q5a&IqkF%{vnYZ-bI80%GVXe z8HrIsGRZXsbvxy8!MG8(JUpE8-Y*8cClo_tFCg1`+<1;GU(k{*38a2$e6 zr~@Z}%Y9yKA3yZYzCQoKaUiDVR5U0t%R52%0JHrFg>=q; z?oCY~cqtC|sr)wOmelfnlY*(w33;OKApDz8<~O+3RE)_>KH&$(F^f|>P;c5?My*AV zPG;4o&$jp_xXUMbz>>clZq> zm;XmbosG>VsD7O-C?)hCAv6Iht7zj&7LGPB1tc$9`~F?aW00>_Woi>2zfH-Q;d6JM z3fX{CtEN$goKC$pnW7SdA?KlY+y+-C(}sCCI>13;oexcbleiTg9P5|yU=L6@t@W>i zpU6JZGl6nmDJ|9?vDZ9if-)eutYQKK33b>P97DVK{ngNX&-bJipR^{n}olY3Kr9=p&|92=ty_59Q>q+?o54N?(1 z-Qp%Yzjj0th?e5<1x)yzJ=J&0KzW8Z*8&X|AL9${P`qlKfe?&V)|Xz4(Rc>bglDY~Sx9 zzK<{93A1WoRl2heJ<{;>qr-JD@TsAjQPvY!l~At z>qug(!Bu|m$+FZPxUs^0+(EUbntFySS>xmpdyU7jN9oIuim_-Sg^>E)+Yg0TQ4^9^ z#Bw*^sCIVAq96Z#*Abw`0FjlQ=b2+NZ2xhOank9AI}fGjTMqZd-~fMu>SV>oA=RB~ zz~839nyoB%$B{kT+MG}R0EnlTtKgYbh^j(?Y_hQOU-uja4Vh9u35j-enz1W!$ zq7T^=xGr!HpT8`gFMmnw?$ItzMu%thkyWkg_5;Iq&g@pztAcUIdVje`s)1NoqKvl! z{U)NMi7nYSEML-YbF7Nl0QeNGBX<5@wkIi>PO_1%Bo5w;jW&I_~8;|x_wS~5#=??Bf6!ZK6wGaNsf_zL~?FJu@rE*jIpb}%V* zU@s#&r@toaDuELlciehGr!}zGQPU}r{wXK_j=Jpk@sp{91?~e`7@c@~v!VV}%dLD1 zZK#wNYbHTeEaVBvn{QJ36hFc`O(Oa{2(O0az_T7r-_P~<1_kqmi|s*`R;me<>7sJ%q-&Ra>7+t&f>NhGV!N;cT`L2k;*G;7eAEIA?r(BLO&M3f8Wi0{>Mwsi4f znfgrX19<0n!}jO6-5hJG0tOkgoD~Og*b?xS4>*>I!thd!P`{Y?lng}QIbr6MMj^J&H{LT zwAb&zp^N)=Ut#(e77k*DS{XeTIU|c6?mo(YhL?~`ix0lu#~=j$jv{UiNHYmvD-!0< zXK&5R7ebpbt|U6$T%u@8wrE~cMC8`Fln3^{0FU$+F5f)kGn&3(Fy2{wo7|G_ZR8?$6%KA06Coe`Fsj=7R+&`k$B5|6}aEqng^*Jy1n9kS!$; z5R~3kz<@|mAXG(~C`vD(h)4&K9unytlp-i4f*{g~^cqS4=_(zh1rVh}C`o8k= z%sRVplBPgn^ZG4OM%gi8oU&8{kyt16*j$Fu%(kQI!;yi^*I*6Pv>Bn6@f&Y?n*22H zw&*U=glHefL0*NW5>>vm?ul;==G16dCbq4PYx8JP%HaiXSgOLjw6CQ-Eh+BKk46a< zj7v)>-V@p&x$>E7Bcl_!g+613To9!o|HN6nj82xsw=pRxrTWvJTV*{ccf##P#KVo3 zf7F*a!YKloWHxU%T^(RpYoW(VR5MZgQ;Mpwtfs}Js2`=KT6*^?`jR^PPxnG!H7i>vK0ElO6T7-|@uMch)t{d)SbnjnT8qd<9Im8LO4A?mbrByMrOC#%b#d)5+f{*)6 z`y3OCr>#&$WZ--8I%f|&!lUbmr7#V1>83ac9OvX=-Z=4!(>c0D+zMA^_fmel+_@Aw zF(kL6;d{I+)13@gt-neC7h0VGxPx^#n>3b97*)_T|N z9Vmt<$q2}_P^`}Cq$w#-He6iYH*s0W(ec^%;|h?WG;5l62RZ2*(BPO*lV)Fi6Cre? z%mfGcDNW>5fvggVLN&$9d>!sg_4-dIgtOP13hS+w7!(IuU60?~_cHS=>+#;*03=P? zwV9Q^Kh~R|tgV3qelzRic4+6so#l?^C?4rm=Qe>0X`uC6UgqYAmnc~c^QE=LrTI<2 zaj*W$t=(G#RW`eCnX55|8aLN0M8lRu=zTjP*?yjm9c)USi!!bB!F&{uU7k>Z_HLoXJr(Ig zxye}vA)jxXCf)6B%#I->CKwf>MZ*~^$camoft*js!DM+EfzJN=Q9(ZXCg3daGRL~> zI!^#vZt6ZaW(fHgO%_pQyw|T*RZhel$BfJV(yBSxPQkAJAhRIlW#sKNpU$Tm{W?`v z#d8c498vFi12T(ML^NW%YaJ;0*WR>M`L}s|EAQfRtg1Lb*pHP8FBZb!LG12vi5_HC zg{8t7L;v^<`+%5uHb(Bb547yd?9-}kb!Ij_KYS9PZHIr`2x_z8{J6>mUj*G>yW?Nu zLF(P+9HExj$cI25rHWOv8|ZVl>y&-6|29Qg4BCGyEBIKP(Q$h$r#^Cd>+#x+$$k;n zR$l<6qm!zt)LaGfYa*-xP&?*A?J{#bhlqNFYdTSbP3cj&|LA%op3_#FtuUDF#n?am zDn3N`5pAr;o7B#38wUgddZ)PwAiE`}xI97k)T9n7Rnua%EHj-K1dXD-{(J~^XE#&d z=TiB(P@POo+fm<6pCM@{yH3I8ir7Zu?=@^iwBxGW>jDLHc(-I8i^YQCZaUBR`8=;}v)MGic3EwplXODI zE>qDoZ(PK{_<&Akzc#f!j^sINF}lva_UlbfEYLn0G8`X?lijp4DL{%UDvqZVU~4Tn zR%)*|%=EZS3EQf&6CyA7HW~&*nWvGzA7pB*UmW0d|EPLU8~0U~MP`(we;h|x^!_U+ zD7t+L0ckgg9{7Jv63fm6zC}p+I$HZSJ*Ifxq~Q!H73y4+Y;ka9(&tP7Jx#2r*|O?^O~6WBHAC5R!Xtcy|cD>;{faG6m9LarKU!prF3Mzn4-v%8*u7SB(m%+0pEO^CppkhE^PombqtCIU9B1*tFd9J?cUGbD5+s~DWGCc*; z1rN^n-|(9?W;CKGX0KZCT@KzdH3}+PDmE(7w!lxyA^gV1KK3PiW=rK4_Y4}P{BX2zK<87f-crYwPPysvwsTX> zc%ie8;v?S!jr?_Pe~qSzuw6Q5_96dTL)(hzzVloMp@@tH_5OdIWdc)9ajNUTOdvTz zkCwEAusWC66)!%FO~j;V=y^E`UQGNdES0O<_j-nTuYlDlCYSf=Y#D z%{rMYakRvddNslVadjnDX8P%9y3B)E=j5f}2BH0A!%__&#OdgGZ=3_NP27zBa+f1< z!aD&=$e5Rz87j8OK6ea3roI^6*VLYCKt5cPE=DdvM@s6P5T6tSmY+L{plMraP!xaA zI(~yUE+#&kX7Nd}9K_uBFxK1qn^w|Il0oPNM(;>xVp#PGzSTUPX>VO8 z?QlX1y|Szj2P1bN@IbaCNr-MFo;v4oEo`9#1IB1(54X4>!n0jz8vzZ|Rz zfA-IwESSPE8OTK)D;M$mg8Y^*EoUDFXI@%md&*XG$!V-3_z8(NA?e|CvrB4He&COq zRdINnU1D<${3pZi^|mb)=K@q&7K)8SPWoMOOW}6O#o1-KZA19r#t`PMJqLXEz$yb# z6nJYU$KIV`UMjWpAZz`cgpTH+xOit!EFd<`dhTeyLj*N!xHsi^-1@oa%vf-6zJ^=kH=+BE zMeR)lp#npj>%(Jx{*d{bHVPX?F|XQx`P6D?OI;8gDP!zSHA4cj5B3KrL{V-d~=BF*r zc{7eA6hP*ln9h+_B$hPjdlv+xL+6s*x1T{QI*vPja3hIxq52YIL-gc7vI4ujvjgYk zXpiULAIc`1lb{{fqa9S-`)Il6)IZwBKG|XYFmzLq)5mL8Od$ zb3^ECPEyGJt9eaIAMYQTn3`O0_%(i|snmZgplhRjI%ORIxgU##GoS>H zl7LZ~4N>0B{Gh&H1_q%V5zgGP)3^w$7jhL`Q~Bw!^YiP7w9_w-%yUjZ{d+XlA^Ys8 z9KX^>)>l**DcFzBvLojT`m<2;btM0W4bwgA(S8HFXNue73Pmov7#oo=r(q8>#XZE= z60$@fc#G>S*W^08uXEc{7IFtrU27a}<3jHWFDZD-&yNwld+4Cpt5U8fT%U-ZQInN@ zdTz^SdTXX!>tHS?mKc$kqf0trBTX4*y8olv9wlhQMGOEa9HQQCT<_X%pqXo|6pCDb zm|uG(7zZ{Iq@V&FKWl&2ZuyZ)j_Ten>*ZsLE=$`~+L(xQ4Ijn7c*tv08B#sjqq3y( zP2r1FH=_!r3a9cql&HCW_meJJ;_Oi?k2$uvSBoQ9ckP!?Q_8aC4Zn+5;Ei|h_y4FS3pH{Sv6cVU*#4h=Ua8hB zJNP8#G1K!bfsLd6nHiy~74y!V>rz%OqCbxA=;W|JyhyE^_EFlU2s1BG-{0pw<1KXFtTrucP2!_=46@|&p{lY1*rNYvFH6J!tz!Q0!`@#nY>d3oHY z1Yqt|2dQtKFL5ize(l`IiE2k2M0HJ%StzK1@h!pN&%T|qo7{@d!JvDT%!vD5>0>N% zdp(%K4NP<(d06tgE zRnv%MoefDMKI3Jb`xrn&pAU6R<~Drv{X0rf^kLI$Ajz(A1oDxh=w|>;C&&E6D9t z0!Vn^R)DQufxnve;b0Y|$hv8O9(r5UeWoVfeKdSm=lf9Tt3z_u3c{)M4^#MR*hC|}1 z)|U0vYXfI(8{wTVHW1~kDL#w!se2p;=$XIrKJoJaNb^moK?H+i)QgsZ(~b<)3Y83% zeZtwWz&9y^4Dy>w?9XY>Nxr=PQ=Ep7qD=*B%Sk``>1?#}V#V40vR(&{(RYJqx@WtU zKELSY89!g35nMjJX;VDDL$OqfI&wVdxf=X1n1DW7wObR63S2R=ydI|uw^ltN_P~$7 zQDt`nSqg-lf(_aVcyi7xc+l@=-i*&HAds72xkrEf@q$&H3J`a(xP0m#OIJttsqn-{ zW0nhCiha4Yt+XA~FY~Jras58G2Laz)UEL^}ICcU6z{bfRQWR>+LSJFa+z0Y|l&}_} zgjeQJ^@f^S0Anzr^~dg9`fpv+|5*-0zO#uRD*ITm5+AoNKe!jL+$Bf56v-RY*I`qe z)o8yVx7-lG$ucbGb0X8<}yObfe7rJaSX=DJCD6@Ae;zDiC%N+RlRa+vQ(w(ksfX*OB#rG{3O0BFX{jW*% z?(WweXrMOf?VE?Y+&1N2BM@L(d36`JH_HtU?93w~|Wn_HPKf zf0cUwGwm*7r09?ode)dI{`xx1M3e7{_%8>(DdJ2`zK~C>v*R}dg-VKO33Qjejr8s< zSEt45Psh!jK0O^WX@m~*4!3D3Hu$BKi|4kj-j;RNgHT|7o5%A?kbazy)DP3BMhW)4 zcIVZnMOWynbYcjY=~1@#-Sc%4CByML4uoQB?n$m=X`zmb5DD7O0jZ`=YKGo%^vWM+ z+holLg}T#bp9~bREeao}ZUP_O%H{7YBJJ<|z?u_d15aE#6l)0~Bs}Z36u^2duyQ4M zQ%!v67(>jKBladb{LeO7FLPkdhvAC{#zjXOz7?>_hXduXH zg6TtISi4ZK`+0lrv9XJr7e`vWy+Fo0)B(ZH&dtKkvBC0)PURye_yX^jmw9^azcS+F zCk6tt;q)sdhl6`7rN%Q859=K~Hs-@3vC^LP@fFFph`%yt%U6<#x$y*U4`ls-0G?Tt zxb((&Ki^}QcAom)1SRVL`$qkay|b~kv@P*H^y+&ov6yohFr$O8qAo8@u}`o1QhzAs zWc8+XN3R%*z}1N4Rx>5hkXkzxyr@%!Rh7@4X8R~y|bw;QDOqX_pBI$axey zS;LpH%H>BXmoc>k{DOfjdZ^Xjod7hLC3{=>41IStug*H&5;U4#Gh~dM5vIPXH_Ml%()X&6qmit z*+qv-*v70*NSxM$|HqTX@R=C`nejKp=c;Y~@-nl>sakQLmH^BMU7n&rgfTlfo?mN zcS(#Y$EhE-=<&I=f*rSh5i$i4c7{b`1o+JW)%48RnxE{$nTDk+?RM9C$sb9KgmSr|&R!Y|5k7sZH+JK&=HM*cV zlW|lcM_MDLC{kUa;$NlxLvfeSW~B=Gu6zp)H5&U7|E!W^zDUPKvUO`yKPQeQ-Bz=@ z&6Lf-M=kPpi0AXSe5x1=>&r_uR9<)j#rx&_z`_jmMiVFxIAm`xn(OPc}|^= zb8o$UL=IM?#HT-64C9y#^*z3lDI1w(edU9GQv5_Zy2=Whe^wFg-mXkq6oC0e&bJ4( z$olKcz$OfOJw?6bNv>(Hcc>S4l+&7iwbR0vw3bI)zuVv*SMnue@mtp2@6cjvpN08u zAwQR*h|#=hO(7r`=1`Y@-%H&(vD7cSnYqR%UaveFKx6$0NjwvJsw>|3&@16wwm@-rw z`5$00zYtY<$@}YhA(ZppRMyfTJk8l5soGkr?E8}+a+j&M*-Y8BCOHa$Sz9pqu;dY~ zL}t=!#}u)u#_&&(3=C@#65I}LUPkfqi0Txxwq|6yK@>MULc;0rp$tlJbVvct%|;IO z7{8Q>_9|j99k*?&FoUV4Mb{4OLM9Ob#ZGlHGk13KVbxaJ2X%c-$(c^`H{R?1HT1jn6xf{y`S-c8)^jDs$V3DSEG}DM6184>nxc0 zWDj{rlISXNl-~H6gKwdy_t^}<93k~MZ?gV1e=pYrkV3}JZ-|vbAT~9yYU--6n}qmv z%Nv`#uXzl3CIJ7=%Z!V}CMoyUt@+NhKEK+mqL!$X%OTXe8II>RFWt{CrIc2aM&Q-y z4wIy2cN2QyZ`j-z=RWrQ$!BnDu2ebMkF<@0=-$A@k&9HU@BhkuH2}`n;<8;RW#v62 zuwL2|>=pmXci-=nY7xNszi<>-qMDO7$dsK;M(e;3~# zjgnpqmw@kpoVPm_k1+vDb%#|6u~lC1p)&|QPQszPrj+ddX2sf5X~2zUQDZ;aIGL0p_o3t-W@nL~_F*&0?muTbdH)=QeH)=*|Z? zi|UWpHBI>M>SGJY!lc>>q;@$fpQKH2EKV=2dXBcqN8hv~_?tZIQ^-xf(GOqK8|{A5 zBTwukeR}9QV7T$_XG8d^0alDV#kzXkHTkkk--&O==k~vvN=E#rOnH7f78$#g7s{%1 zvJF1s7}sxD95t0-$VG#D`7E*y%_>e=z2c10;EUic4mq!)Q^u?uo}S{o#xeOkCDdu5 z@H(Y9m2n7_+LdlBU^Sy7tNOE9HyBC+g+<&TexMBuPgWb_f5v!DE*U)=$c+%z><;eB}bAv8q*^59p~l$8jAd10J8v+}5$%ol=pm{mEZ~xsiZUJ~yd1 zjxaBv3wmSL8Ctqp^I>Y0q{)%JR#1Ww!zSxjvqcp)&a6(ZNKUBKn*wX6Z@L}G^N~ys zuVDDvr5AcL5%HYfJK&KvAn@soz;&_iPT=@PtQxwv1}*T8k+=%u$QKjspp&1b6N$4= z*rH*_sFQt^GmFqZ1Mh_q@-1Haxn%#25AriaM~{I5(@RnB!jQUdf_eY`Z2#vgvlOSO zr+1%~X+-1CFunkxB8eeJZNkZj?G6cLy01|l$R6Daenu`Z4{hQMj+c^II*3Z72+Z8u z*C(}gMk3-9j#Eu~n~;SADrOcVj2bM6nvYTi#K$}u={*L;P+6%2eCYi1az7zWNtrf; zmXZU(Q^(_n$&S>&ji8VC99+{W#RnQ47v^%$FbX!`9g=B^RH2a)Ac+8vP0HsWT|va9 zX9YHVpjPIpvCE7!yws>wQm}%MQDo0ek#}I5%|f-Siwyq#bhp`%5SJRS?V&C&^Nju! zV(uXx;?CegX@d8nXV@jE$~iEgE12G|41SZnw>wesFj9ji5J9C~|B+pF&FW!h>fJ`f z7jIlYG`-9sQq9O_vc}#9()`qM*Ya^&+<-ObC;dB$gQJyi8pxNeVuw*gdk>^h31TGE zsII!A@RPOJtJnhuhSA+C$nnDxzaHJAmnh=0%(u|vFFNqykBBbVU$ygpz779}#*YFj zi`#q$U?ZWkDnXO3*v4e(NZmsvZ6)x$Ov~B8m_An$NLhg5%%nJ@F<2i=k>o6b=-mxU z^6&qboX7%C&6Z(7d&jdWXpNLW3tZ>4}p;v7`G@|&IJbP9>^M$ zvZyS~OG>;MSs2Z;uL4MIxb6jQKw$SHVBZv|_Rq+~ol~CP=~O_bx|w>U&D*7V81g+!T|w zAkwILW63UmfX1W&UOh63!?3`Mzk3@bF?yv{KcXJhnl(wJ*<*bK)pS%6!?An<1}#oZ9bc{S!T+m+X^<7aPQv<{pT z_KScxv@`Pk!Oj~**kj^t6Ym;;rl~z{7ILRJ2oXgP(q7<}730R|Ag^e*Vi6&VI2KA3 zJWIX26JB{}HuVe6S9(Ybr|P3zbp2ikgA>FS3?5#-g(wq~&Nq~YJ$Sx3+hSnYu9=#p zKnyAdm`jX3n!+$hq>=4iZ3B1Sq*mP0haGB)OKZ`dm0Nm8Uw%wPi+Szb)bKyGevW*k z`6Wn#H({?u#nkvKH5%|rYaqARK$qX!xa%x-o|PO8xT6Ulymz=`I?gmn8(?5qSza^V zW-?wE?+~ASpvD=SgxuN2ud`8vH$|mJw zaB6cm_xvL~rAQ#y5_}z;*>JQsIdyWpaUu-|U3akMXd>cR<|67q3O}?cX$3zs>Ix!{ z0UWGj;U~)n$XjrgKl#qHv(KyTShfR|ipV~(@1FDg;UF?P8XPLF?mwGF7okTJO%qw6 z-fju!i?x?d@o1{F;Z_I5;FLwPOE+~7@>$QTFb46CzJ5D)f5gE3qk6+KFa5Ux8gpEH z;ogWaQO!q1wDOw6R&qLrUQ@Tf85CZ)};hU9)2p);P_b^ik_B3bPJ_olrT)*P>YmeoegR(1AIA*EFn zT2-?L_%|sQe)){NY`|iE$1rYzKWQ@wfk6#GOI9al^OWW_&ouk7t%GWgVyn$<_);>X zG_B$8rtf8RFW=kzVf6@E8VK>%_fw9zL4hyeK+kHXkc+#r>?O*mJf?}5;|Qb-q=859 zFQ?wm!9lk5pY==m0^#sbJ<+dZ-Ff&=DOZ&3G;OV}T{GB}EYb3XF}|1O8eQ=g>?B%>_saD?^o4qr!kO$Du|KhGb;F83u9w8W zSOeTMoslNCU3Nu9WPw`s4h^^gAovz93-X9~R@5**y%636Wq395P*-6X2MdAqo8AnM z-BOS$5yAc!a@pVZs3nH-wc;|1OSM zY3SQ}mUv5d-zic&%)NpToNPbzfP%5}ki9A#u*O5{c0)qOHq9@0mBu>9iNp>-^tRsn z7&Nu7le8DI5$@1E+%4(R3r7QATo-LG0px zd&&IQn6UaGVUb^2?g)5DWK%oQ+$f?E7}3PB)yP8QWh)Y6pQ>FNi3UB0h{bk8BqHtJ z%H}M8;wEDpM@zgwim=FVs(b%j+W(Ii;-@No zsdtg#&DTLjfo*vn5$Ck=X@@f7y6^ZHDU2{-;-L|CCVXx@E<8$ZW)w_?v0&CR)#u8y z530tmW`z*CnFPW6pKbX2%S=GZXXnnk-v!<5)|D2(v!y-4pNBEZKQgnPjRr76h`r=+ z^QMs~fo^cHHsoDV$e8kB<)1GHSTGuHycXHaXwcvt1h3Gi$aC+bOlc`^61&2M1F8Jg zgu9_h3%qyvioUtX&bA1#4>t|F?r?T$=?p6pb#G*64Cgk+?TfUQ4VM$F()1;%n0olpRvZHO!x#OwsfD)rqZdzF&)U- zRi=%?QH>z}FfUCGRVSrp%SBE;7He=}95`h&cI;gUuYOYqof>3$Cym>Q$!v4(6i5b< zU!AJ225~Id#y<@bp9eq@0fgwW+uuVjQcB=1QC?kTG+|CG@E6=ih&|{#^>^=xLLSezn_BW_jjvxi(n9Zb$#lG_4n%-!9zl6oD=G_qZuTSUxgi&Xl6(+uIl znUKQ}S~?SvFaDThx?~TZJPoV}OEmW(?*+b(yd$+MCG~Yj2m95;6ccC%`JIo2&s?aG z(^o^B65G6AZpeXYsWR-_M;8 z6i)`nz<2F!e8s=ePmH^+sWt~EA8VNHeD@fT3S>MmxWHFbnC$JQ`&(81SI6a_2>0J! zgaS9C9Gqpw5O|x3Cr132?t;}e63~s*!6~a*O3{l<18K&MRH0R#I^It_)4W4GqCg=+ zWrYo4r-aTYn?`^Z4B?;<8or_1A*!M+RCD)1_G7P8?zY)gnen^vq`OULDlXdwmxTsw zb*YbKTToE*(TaTAOjc5Z!I<6;IS4pKq2YzenqvYkt&A`*77U;5;C2UiC2Wi~#8`$0 zVFWcXB*as8#0$}(WfrYb18xJqAQmNVFLzAe#JXRVh-eDj(o6Ii|HftT`T_M<8fE5_ zHHldnMml<*?|`t4&Sllqg?+umraME{>_P!W0BPk{&w6M*p!n1(oV!Df6crjiDqGnz zk4+{=^i>Rqcjaa5=&uzNZ{pIxTHIn*ai3(ziJCV;f}i1Jh*u0nPCj-vW~F8i5Io6QV-AZw0n?!t0l z$0+E4Tf>Jnogfd!qT@>wQc1u)q_z2jIbAC?n0hNhHXhJ`*=Dk`j1v2*Ui?92Gu!!x z)*gST8e6hT$ZKVSr2#K0f3oPDAjWA8_zYVv-TdgSKq5 zEtz{2b1fyvcdiqVd>ses#UJz7RHKC7X4ikqNB$*|p6%v86*3w95lQ3&f$tj`x8Y!O zpshfCX$L{JRtAx-ntMqqU?rTjRARSk`u(66Ot)+98S%sq%SBMp$hwUA9!qLc!YJuh z%Y~DFwqjWBCqW-j@>1~z*U>00*p?d{xJ^m%(FMiTTI96ADX1wZvpH1$z=B+GyphR1 zfhUtCQzLPtd>2n}TSVh(Pagdq>8Ff))FJEvt|=tnquV^@Oi)BHF+^-DG6l7m=y=$i z21gb$7{V^__a?m}UVgH;Q28hARv2OR@{0UcHY;GSLXKN-ck>lzVf$0>PNr9krZ}U+M^IDpwd3vN`*)h!H@Wr?qH^_R ziRuc6ym5HwTq1G-_ElQ2+g+3a9$}5crz78$Xd=qHR^H;8#sr+ROQTffW@i(PeK6{@ zYU%m`N*7_XeTK-c0WVBaxDH-$|9wbG)Ag7v^UsRE2lxN>TAU#ewIW>1x^Mp4H)cVP z#!Q2xcuWPEO*LA4P4zP$BMaCT%n3fMaq+x_ zo_p0lJ0aV22wr>+$WRyiC7>VycCQ zrg;8OJY-%PM$^hRDfW_I($3$9mS>~AAmxvOE;S&o=GPEz(;k+SbbQ`(bsjEO&qsAi zZIz1|ker7{R$07}v+w0yYGxznzraQikm%T{<@45}CCj`y1=5P12 z_-aOH@jAb4kK*F6@qV(;F3MJLv|y$lbv4dQ;^cojy?+-A`wthJnE?R2_SwWiUL{$} zXEVoo0u&S!G(q^ue!c(G0-(g7!GahZ_^*{QD)pv}@i1G3i?_HJUryFQ2 zQ}v~w%J0@!@S(eXLGX3zKxcIA-ds4h7wY6-Mh-1=s&e%9N^81mIDMtAvoR@$$O1i% zjxOD66QECHtH2T?WRh4s+T}zgXWX|6y^>hH&3=^UWkxf;_-JTq=>Ldgc$z?JNOg2t z(M?%CHu|i-TJa`FjC70nN6j74kWRhH=UH{E9!J)b-V&Yy{|5Q^58wJ!-5^(3{3PZ$ zcOS|X(k#fYl(cnfAFD5OEg2%X%oFsI=B8$^RVahVYK-B9iwE6zoYiOq_bkIdW;^LZ zgfEK>PfN^2-4K1~IaU}PU~g2CVKj%rJ#J3|bRpMNGGH1v4p+gvw2D#OtTeV^BBiub z3DqHY`sJ#Ey2Yohj@vmH*5)h_ACd*X^U~C@@#6d`Ot zvAh8z&IY4vezfb`38pfciI(6PJd$~q!tD&#S!W`+oeJL!%|lexE>qrOdT?)R%l~rF ztAQshEa;+d{pG1|-z{*@cmq^8Gl_kJPkjL{D}c(@1`N;^U?I2)(*1&*B688*rW*Ak zwuNdp$*%uCeE5%BvUyR9M=~hIbyDb4w1A})knQ{C%K7d5ylrBVfj`YlU#c`qM0e!{ zIe2Cc@8~KR(DQZV59=1i;2WIZvb8T2!;W?a*Kte227`^Zlhak6rt_mmn_8wlRn_1A zzz|5{_;h3KdhZRFLJhjx#^nonETtDr)*N0R#zOyFtokj9?KwS4wrg^S0AbCjs{kWO zV}Ebbb>MM@&~EXbo^_J3$G5!TqSp`64^6~QXrN`OLeEkSumc)%`@4ql($|ixw)+LB z`hw!+wgF@9F;dbnpzS6Vy)O0Of_uh*exEfTm+9JY&J&4s;%~^5|NY;}Zct1+65sQ0 zP&@5X-<=L;GbO+OI49hm@iI!MuI+*Rn8eK#xl+mJHZ1!aQBx_WG#bOP&XDB&S+g$ zC8|8KWm&aH*Sy|e=t^8;*bc*U*j6@fS7Ba;g6|+vpYslh3{6Q}oz8pP22JEP#pBJM zT~dhZOX#Ca*X2@KA*n?9=sAger@xaXnoUj>%S;@KiI@K0o5615Odnb9L|gId6YEnO}GGicMUQV7kM>;vm+=LeTmJT#vO#x;<46e&StXaI}qVz zeAIZbUO%;jMop`UPq0!uR{^!{h)d!`k7m2U7R()wV=&0c_nN`jsGlnK33Xq3CQ$Yu zk9_)oc*!@EOv|m;ydN|qs;uU61!(c@)h)g@eTyQ1!V3cOlI_qnoemwU2dZ!mn+HcK z0r+x8hAmqIGGT1v@HhS<@KonA0V;U;xP*Tyw*5=q^N+U%fl9?M2|!Tu9*`$!?{~1f z-uOz1`QvNo3X{aSxpQZ6Ks)gJ8o)U`O@rTlY%LH@F-dLH#WSHJ_b)n8&{?OmyVPWn z{BmhLUs`X=)l&2?xMBf_yRL__T0BMb8kQvZMVdQJ+*Lei<39e<2)L<=oyUBEBDbPl z05vIR!8CgjGWZABTt@_JH;^e*auE(y4D*-Yu@hN}&_!jYrLCNb46c7jb8$F2)Z1Um z!v4FS7IrQVo;Bi4NJbSQt!3?{9pqJ}$bmrIUII?+q!%pJjnDbZ}rrvLg zZ2Y_wzPHPL1P@>R3Hb3TBpy%s%+pPE=qO-!4f}FXhNT2n=uvYWx7mQT$tc5KwbS%a z#?4003sWm2<^lTqz9&pS0A|-)jgx!n#G=ZLDo4Ad8FFEL;tX;y;HzYM`_9yx!2ovn z)XCz+XkX**k%yZNcBh6l@w^y|Li83c?;k<>3dIun1Mgg+=gIE2uj~8*dNI(ym&vEk6&VL4~fOo|)_|#ays09Op zU9)Wv?8P`l8__(EP~Kagn#8FZAZpVSi6Dp5a4D4FyI#9AVV9AfQn3%*rpXXZ++~}r zgsOn04DY)4>hV=xCAvC{){JXlTgHJhN#^1t@g;zTSzOd~L%aKzE;({Q+xXQlz=vsy z00=gH%3qpo30|A3k{?X)YrkGB?;hK9tH}ag{7JauRX6Lg(UqkUQ&;ExKmyL(U~|0A z)3HY}#`CS~&A-oY%I^XL1s11d=@;wN0yBHTX#=7x6DNWZXF~`nd>QM)NGx%=XZ;bB z>1}(3?Ml`92UC}oRXd4*(dxzT)3p}q9#f)qqvoi#-KjYCDsOh_fe&T&N5d=Rv&7YV zA&(hS?R{ zGp_E_K=n6Oirg*i|5Lu3X-D91M6_lT0U)OSSv9fW_;UWaA`iKWn+-LN>fA51U?v>Z zY6ed?FZ01GrD@Bh!*%QJnI3&^z z4@3P-CG4bGcYfVgHS|t9U*xHzT01A#i={OLr@53rg zc;i<&KFYPHdX4;hWp&%;-c!RN@*Qv07@V|1BtjHStDNllsF%!y zBksTU3E6B}7T9LI2TlZQ3_ZEcvP4v(IK;s%70 zl~iwbGNDMy&v8_!p@-<31%5Pg#1F_(Qck)oa0FgEyx;G-HL2wk0Vma2>>8i_V2x+p zOHD3*V)^Q16P^Z60sq)cF_}sj7ag%v~{RlqpUy3;3_V1ky&MrMr z-ZDJm+0@3G&^@@PxNOpL$@=QwCg9f;Db#MN&m4X7KQ4yW^OvH3PAg)rqizQ{V?WK7 zSfqNKWUkb~IqNWR04=mg4Q2f2PBXhe8KFt@46*%LQ-p~T^q|!yUSZ_)R04@Rgz8h! zk`O}6k;0iz`Pi&8An1&z=v#a*c;c?SI_Ga-%)9+2tz2ctEAMUUB5=CY+|M+^15&eM z+*_2qcR@}OP7yH8bk#8xarXc$xsXMt$VEj~?Y{dV8m;cPRyq1sao1khn)#S%sm(+RJ zn3fcrpPFPia7q-K@|tuOLld+#oEm`vW-KRfI2KiO{kJEK zKYC4~JOAcZ0OULOUe$bVY?(I>Mq5m|wx@6OBSxt-SnZ|T|2Dmu{Up4(_Qqq%lp8hK zNc2+V9gg+@GN9+xCGaS){})k2hCDkD^$tEBTV3WMd0#6p$AnzIaMFx@m~ zsWBS41*3W8G&WsF-2;_TX4s+*ybtmq zW?4kiA&QkA0hFWsc+Nc9KL^mInuL-+-i;x%OS47(0kw@7 z#0BNclT+EopTU`~S1RNfu?tn2oukY=kg z1h(oZ*UG1HRL<9HPp?A^$Kr)o`9i*Jlnn|IEpzGGA7n%Q-9@x&`2VGv>^;@)Hh5=C zTNCk&b{WxHKu1CL_$ymF57|6HTYU2M{nc(IrmL_Py3l$d5fEy%908>tG@XbnSzPv5 zN26*F7Q4!)B3+wS5Z{)auoHb`#}eSYz+}~9SeMK^DzWsLs6`GDywlHi`ES20VhG1e zd3A|-2VKQTxlz}dblB)<@(B#Ri)1|c-(KXbrr?)pvAyhb*3l+{9}F(=esY&*UYvd0 z@OTmvpk94cx2E$P-+-uMvSD*g{@m^kdbE&C!Uxo7P@=7S=6hfBG z)5?+kmfVo6VD)nNX_0|y3|`?%;B8PqgjW&58j?01?@!xe*0R%Enj(BQJ6#vMeDbZD zweK!>z*p$Q86c3!9R#GMkNfo1tyitccpkWpPdLwfIu82zg4hf}EV8jq5l@L(686Xm) zJER1pLqY~fi_$fEbThhJM)&AZzi~hJeLwMezOUc&*LaQXwe9`BuIoI{<2;Vzd~KF1 zP|3o|grRe%6#O3h-TR5c-F(XwAN$T+Nh#l-?tpYt>%M90f<3JT9lrbbIZoc27-Er4 ztG!l8esq&6$K#*p=pL_@ONPLCc>rqDp1tX0`3+gV*~iW6**1~cbYpsp``~WSi$5N= z?qak(NTGZfYxIdru*#l93QHBk*fc+|AUgj>EUFL`wQdyoIr}9~8=m8{z~o!LmN&4= zT~&uf3Gu3jKV8gi-{?ZgWyl=yh76mmZx4;D?V|BD+ia@r7}L(kO3`SOc+$t~=y_0c zw-OKeM?DvPW4_mGzEuO3`S_f;c-S#!86WuM0kEqbFij{f0LOAg@_JcP2fvL2gnB9FnB9g`GX3Qmhu=eMEi|9Ejp5oyVb4>1p=I!!W# zFSuGB@>}S;>f&kQ7U23Nhv)@FA394{oL|@C!c*kEkR#9lSG=r2dfrX|k zsy=$9FF#r`nrhUQMM1yHXOsLftz0)cR+c~bfuEdq8$))uBB!7>z1yOnEOTrp z?cuG%a~Y#k5}3IwT1FQcr>-j(Mc*Ln7DJhk#db~#Qe!w9OeTj^OeBzPn_FJ&#gKFEw>BreU)LsO08vT8O-P>j3s z0AH^yw^LLn=@458AkX4n8)Pa>swGrKTfdql9r7)OSgto`cTWz6MkRkM3OCxHt7zM# z(C_WeJs+c+F0ZNc-CLcx)2&wekNb3cyL@~pQ^>uLN_V)H~-KLk&nm3e@XTO=y}6ID4HQlLmRM*|HJSb|TIgCO!k z)XDe!?rF=8(mJ<3Rv_!5oR(5lB?8C*kY@fF0`iZ#c!dynSk%rl@$d6$Nwh8#f|cb3 zH+z9vAr&z;+!b5Y9j>r>XpzABSkZ1uKY_ldJIOUp>I!~cx;qtU(Q#l zyRTb_%J6jU(tKhUB)?*Tg=_+3PGCU)Fd$heUZ{ek=qjz(@y-uCWppdd#FZ=adFht1 zXHaU0(xx=eE4P=7roXPT8AyYqLq)xAmQE~duFp~W|8|q!;zzr)H)t*VTAaSF^- zbxCcmkk|9^?_h`1@rVP0uPS0raVqXBf%i|%(}l0N3~~wPJ<30AfI}N8nnzzjuV2#X zsYz0OsFTRj?asN$VD}!g>a`F*AH{g;tP@>+XA_zoE=>M)qN@Jr9k8@lMiUjQK%L@+ zNHj`0)&=#+m{sRkXX2Up#=cQwnBsb24vUon;@3}WKjBQjn(j=tM&VKTb0jDTc_@{1{|<6S(d)@^|9$mfG8cbS=Ir#6_76! z6q+da8nTh1ngW?14wbA^SLum$X%-scDO=FV>AifF-YD^l#x|6+VYGn+j*P}vwxrY+ zx`>Y~F)Btd7mN2GWbdtQkL+G4rwlfFEDH8qvF;dAkLdrT*x?Y8@{f989+*wyuXGa&f9sHWJZ&v5;XZSBPjEzcTv4?$i9@gWr} z{mxnV*P1W56|tg|gbn?q#y=t;j)W66^f#X$rcO%eFUnYm9$&1>nrR9gi@0+Z`5O~tm+u3mo~HvfpGK`%%hWlJ!~p9VO_rJ}R?I>GZk zy!g=m$RxxpB{;x3sykz`sOt4!Z)*C1+t=p*EogM=j4Km6kCnjgyY5+czyCMrxp-V8 znst#TLs7`3?2%%J9^*GbVYGlYx{-A$rLy85YDwGcl{_9*w``aXXDW+9J^|Zs9jAzz|I#sglRhb>GOSm$5(Fd{do6KfN!M z)h>x@dTaRO2JNj{=tCNit;caxwep7nQ#s5?9*jSJG+vAfnxDQINxvTLc=~3e*syZP zW)zL%dPqUK6)4M^Mp~Y02gRRiFj_kdRYwJju2|F|N^$!eX_r|CRS+i~Ry<_oV4?5* z(0Pwv$H75ros02-q*{{b%AwF3`RfMF5`Pn?p1PlaLlG-8Dm?F&)lk`EPf2S-gH`xYd#>$7fPIdPr$ZSvl|9i9 zPbNrruZtdL5hYT)T7yEi=)Gu5Chx?@1(jip4n8_Q$#fiPoBgUZp_m#|IK zoqRdnRqWV>jum-kp%V~Met-H){~-4)K-zOG7jUGh52ALkSompk_iQ}jns~7xw36~r z-=+?DWO|}6nGVRWKLJ2@(u@Dk9$|jl29Y)iZNMX7C$*rHBgF|(@W+eBI1fq^QL+v1 z!Z(Q+bn`T`5 zj`B|Chh+inV)(ffH?+&qr{TyxBorIyJA;6RSS-bi7^uOSGlInNgDK9q*t7{My5uOr zOnW$Z^YIUwUnXF4p#v(WgL_mgbZ>_o;uleYj=VBmcn@D{k>~T z!|i)a9nPOh%+(F*ka_wXm<$3R!y5VYZTTtSo4-f-iUP|UT_BO0iHi`S`xeKd#40Wr z>=T%tv1PuK*Myv z@Hg~3%=eg{{9uWzLg~^AQ}Vev&58aw6=knP9|8+g1fJWgkxN*N^vD4 zGU)zLgk7q^jDMsbQCnZ8I&Y~slIrHtUi4y8YJD4Nsws9|9gy&9h{wQmyuZ`TN=`E@ zxhl5G4$q-YUenmyNTZRs9EbAl-)#}@gw+U5_y?WSHR!P*TPbpV-dJ~b&`%C?;0bk^ z@AtV|z-TYVa@`|qS6+=WSxDx0Rwk=zUaW1p{xi7#QKt>cp_5$iXB1~PCT$15Q_3L> zJ}Fq@=th+RjxZAxtp|Sir6q!H3Tp>&Gmsv z8a;|LmpfDEMa5a*_W%MT4C3=et@^28$an5CH_Ty^bnItgTAR_rlojHVg8hccuyOd5 zqs7ecl6L*FA-_ha3hlm^KTBVjGYv6QC38LVB%0w0Zcv-Pvhnm~-fpYN& z7a(=|0`dI`XKzQ-PP4IiulS`_N&7PiUvgpT5Ck{)Fhw=`rFwL|w7q(XQ_F#Q8qSgB zbGGTbjBlg6KK{AU8!f)nr31Z}Y)Dm^Iu1o07t6@olwh2#WV)UN2q1EZ3y!@LGM>7} zXj&u2^HBZ;pcCN&qZpU(68JGkbRC%5k1y2SZ5YCQhKYVh$h>nX6P|2Yx-EfJt)d-h zUoHAS2c+iZ1+BquuJ-y*;vakR3_4(GJ@7{Jz`srN@o5C`LX5tLU)w>pw-r*6(J?&M zQ_ioHsjL`IP~+gMLSLh$M~p1gdgLgJ{A?R4mX$%di=G&oG}Jw~Q$~4}Q8HDzFsXkT zm#)-XhqO!7XC6D-36}~e#VPqoO>AKNp^D1xE`5WmmFq;+8I~rAv#i2`>?mD&o5#~C zSCcxb7`G>XpOfhHVqaahX>v4a#BEMwu)fzk&EIRe{@??A8PV$=TWZL*{C?Rkg+IsM z89s&(q<`)rtRciEEGr~rYX#x2>PMhG6%Ow$zcCBjCru{pvg`(1QH3XsYds4k z-|RaHPrpj&k^!R4b&O3-Y39PM;;eZk0pZQY%u)8PwjtzD(V+?Pj{1OWCqd02umWPQ z7mbW$=)rU1gG$evP4JYF9HzU%kUsoL1=(6Upzp66Widt=A<-1nx*Kxyix>9MrOjvZ zTV(|65XfM%6jNkms=f>IMIs`0)RSX%p@dm|8kC?U5D9!X!Su^2=;|cyM9%ffKXnK&t7!6aB_LOrP2FN2`CFiq48jfoK=!F{Snl#!cI| zU)YtS?O;)lNFCg)6pMo85N<@Sq#CaKdXnlJBO_8WOO65b7$J~)RM2a1$j>+D?G&qA zKbZ_U(w&cHG~o7OizPv&K}PtWKm+YddUh^-cYn&ff_<(0ZmXK0hhn3Au=I;}<8xpP zexLTeUC>C;2^U36;5-F(x9yOIZ;wTWo^N;7JS)k{?@S5p_@P|XaeBMX3YRE)ZoQE6 zM;w0H8EaL^d>VQop^O`~y|#cxRH*j`i>NN6v!2-YLwbr`NB-xOOB<%MIScW}8+@KZ zIQXI*PZ`9FI^i)GjT=pTK5tT3gfCo`dtjAHX|!9-^9zXF6fe-|o}C3vhcBF^w@K|{ zIwPfjubs?ml2dy)ZHmZxG#!5p9>>xR*w*nx#%R}dX_a_70eMAHA0bJ~4qiM30^S zGs1{>5uxIo(Ha3Pt1jD^{tj>YK>K0VqL2Oem(uzLq^PZ}DbT#Y za@WY#LfM$h+PZiiGV)k)x`bLT4s9cqt8A)nYHrqN7QPVS+RYKQ5bGYS)N|NT(NP`p z0OT|~nKrw6*J&utCH`@_OP8X{N~P%~*f#7__LXjGc(bGOM;_y}yX+p+TO-i)^?c%p zt$vw!{)9wBeQ(-}mC-jA?tK{RY1O?rl!t1E9Nw^bwM~Vp3c9MKtv%F~UI#wVDdX4k z$kE{WRZBoRh-Dk6qln2&R>5vW@}CR&&tF~n5*JaXOOpqJ0w z(NNNig^PGF7f4&aGRczvJ#}Xc5KWXYeCF_3lnbHL7w+0rTMT<7ja_Od4B>K5 zec(R6y4EtoZ+Y)_rr%tyod{EN$4?gtYd>JCYZt85wLbZrvsTpp%!cKONYwXe5SHj5 zAs&WQ3$u048WbucQ(gYU+K-r?BAb$BwW%pHO~xQB+vx~00jyyyi+M=ea@WK*aqXTo zt9v+3yac+4X{955Y|kEmB4zCwW{?BjFny)PFZ)t8N(>Gw+m4l@V~U0f!+cNF8e8vCqaLH(?LT3yb1bv|1-JB*iSGt6Zk>+RR5 zu)a+bF~q|bdQLQ4-6e;Hj)4MOn8$JI{^U~|chtSDO5cl76f7xdtgPoqgjIA{qx5fD z;kim~DDF~hD(Z=qNjPAo3uHX*%=Wm3{UR}rRY4Xsv4&s^2I}>}b<7TkwJUSzCZZ%H zmbfwr=|(Z)>gNs64dL86(~hj3n(>Wxit8nY>kR#3zLC6bHHm$=O8MN+3x!Sni+?Wn zTM%)Ggbyyf`poB{@sJ3dw)sqhRW1-4eRK~(dPUe_ULAoQi$bsn(})ZkK$`Hq=V{M& z`|N8ED7J z3fz$hFC05XbFQiy#HX2F!8OB29(VRzqII)73tuGWx@>{iPtTGr*6~F>mYV8w)BEK5 z@}i4K3XU){iBjVe9U@10^LUR_QO4iXm#iC^&S&(RA^xca&}mrRr8{JshM@CYt#~;(Og~1-f#`5Ze76}_JUcvBTL(=tOsk$AVv>b=d<~$SwV@z-Y{nI$qt`F|lH0^8%$AdP zj!SQClKFym7uCA3IVK7Egh#z@xtGKr)(<{DS-9+N_o{7DTVP3Aw?H=O%!HPotiMop ziI;xp+=?S~GzJ`|yyALhq0>in9CSUxi;^}xUipksv!06Iz5Vo}<@u8Ed3-s0+)1=5 zQpw~jd+tM+!E{hfy|t3Qbs+ubZ_r!l$%e8<65?g4<0g_KPlzrrz-I`DkNH$>0@M2# z+r459Jn`W-1pjz?xY4H%ICkm-wG!b=tmsBh=c(n%K8RRWa-p(=L|J76O4a*}#n!QIi$-Go`kMy*FeueHgO)5{(G{?K9laSt(SsCqRoz9)^d>?1VEO~@b zg7Dt=I7d+L0bnX#FztEdcpA!qtAMKyQTHCt<%pfd`k0b#y+Ebj`nyVB4yqd+mKZ?K z7*Rv|bnxQ~mfn{6GX|o~!Om6AeL)pJ9EUIGp+H@ZQkHI@FK;|J-n92C0}z~MMx%UW z;)t4|{b~RJKL-coWf$uchgcQiX>qL8`g#7n9RGy>jT)HkuI4L_e|A6VGDDiGnW?Qq zyJLif2vACq!x$#W6-SMcl$I&-IKS>BtoMK8r?N6e%oJ(6v(6U6u3tgX1GEFFdRk>k zU&Fu@+2X+te!cVNOC$bYWnSYuBs(m@P2MMTJ6<&)aY=i033{{j!-^J)HCM^FtK_&I1q)IDn))XVon&M+O zHV%rHjYd2Ew4)GejUN3>`jy^uyOd2ZGt^^)_^=I|&6-(#6VtF-hu|@#`>^s#Q1BsQ zZ-%L1G|vYWH}Ch`Z&8 z+}CY0zA5Dg&u=1ywY&Obj7XD|n#WRVXmR#W*?x&vu|_p_FBHX8bjKIMG$r-n{RDPw zzseB1iM?BdOZV7Hn1ZiA3+q*^KpLZAzWTr3G*GM=d;i*H_nzDO_p+@CiHk^rYaKyh zS2ir+G_7H=mWm(sz}6pnv%?PxIl`mFk@8_7p$HQ=6H_V;F*+q-K&395nSxB^OhJwv zRgKg(Ac65`Meksxf(RhT`Ox3v#Vp2fu~dB@T! zsACbqH~%=y{f@Ux`T>-`5Jb=PGy0r>ESBE~UZ}R4bF$xKlkYX*7zuRp3p>YFJR-eZ z&m~43{w(lxn@#I9w%k_Xb>m8i*MtGm+&>E)V-)mbq7M6lHsw8OING;g*J{VPofh-SxMJn}BDnjDaP@I})ZBKR6@P{7V(|>7W{ZZYM`JwAmKOs3) z(HDw89WuYhHFJ49M(Pt;^wQ0p2&qC+(zkBgTx{5QZ+HHL$P%QR{HKma7t7z$EJH26 ztVKl#E-=Tn%b#=kObe`-B*IdmcdRf5Gc_d5&T}XV;#KFz2l5JJB*Q|dTny1r7ycEz zeYxt>W$Pp4OKo4LKF3MsVnU*{b_&~W)0l+DD-cFpxI8|R1Ev?Nf()1YUGn{>t+V!Q zh20L=hBu+hcu9~_QjmclHNa|jX_Ecd(M7Q#&w=nNV}OiP*rXkF5Y^^w5l@p8)0we`VfcHcmxej27b;c1j}*$I<&?b=Zw4V9))K4FmUccxQ|ED z3a4b>Eeg|de`H~JY%Fg$2!#!%?! zEUKnMbvF`N>G2Vp8snKz8J99E$^>`sZ0jqDxkqqHpQfpn@`!@ut~?%HUyN@yPr#0m z%F{HvzRqShfb@xtjKGs zG+H?4;jIG}AimokCMvyk1g(j%&j)c8atp9-%w0b2-04_!%E9q=AmIA)#YJG6O*=dy zV!1nC*`~SeK(S+cpMNg{Q!fT+6u~I}(h=UC6!ep^&{kpAv@7{C}_vLXpq8{Z~v*{8BfNX{t?HTL}i$@zu1!W?ZX(WS(LT~OroEMM&T;*+#p75?)uX5+otc96nK@e@Y1kx%2IXM-;Rg=wD;6E%3s%Hf)_ft z2h3PSTPHfYqFubJNK(Suj><$V14yU1WQo|rrn^LrF3?iuZ;ccs+#0&;iUj0qdV1{a z1vX)iMF0zX>3zSv(iq-b2sE9e$!qJkLTyi@EqqJ4dBrIUI87GQ9^*h0 zx|uDR*D3`QjWWtoy4n*j3-eLYBj5VNyfH(IfP!0Yo|WL7+ex$~4%-O-lwzCP;c4Al z|M#M5m&%YX$@|JzK0VjkKOm-Xi-w++<8Kb#i5aCut&m= zihoX=Y3r|Qtr99!OZl@0^H=b*0agZ%)s}5m;=51!`_p`{&nf!lW~K)d1mvkY4v=w} zrR`^^{JgY}wZpn%({=_x0jtilIeX{)P5^?j%u*LdjNs8>o36q zKcdoNJ&dZgm?@E@-Xt;(ySO;Vpt;tSh^g-I`a|9uB(IX8J1q+(T_3-Z`So`?WQ?4y zcfItJSR8tI%EVyj-YfoesYnIyE*XXu{9Lccn8mR z^V2Iy!c)^8Cf>Rq|0nO&#Jc`PY0>c710Nx-siGi-K>W6cDKO&p%$&7)m)HlHKv0Zc6C62Fp4a=sJBw~el|*tL z?ZixR&|y+2V;1-*tnMh09qJ?5G*jJH1LW+yn=50Ee?ehRPUB$ai<*OCIuo*Y8YwC7 z@PKeOH+rAx$=!f=wCiQ7{L0^6Q3M=gq}m|Ha)Hq3J*64N=~C*vA^t49DNk|@YJ7w? zu>;}Mv2eJ$uZ`B70imwv)ajg+A5z06m?@8iV;n`~CK<%c#6VgcW@05LV#WAnNA@SW zGn-?;35oR{4WF2!oVz#v4$ghX~K|Nx* zJ`PkWG`_K~B(6%$#|63MVBVq}riU{C)8*^>X02H_(c^xXt|ZfGpX+#PA|U~ugmojH zns=GOHl_SmFU>aiUSG7Dsrw$1%)D0nU8$w5wbF|dwxVr$k=3Kp@pTg#-IsN={BmVe z>tDYkIxyJ%BIPiFlSvp2cS@NGo-ZdICe-Yw!IRw-XjwL}P&~9k%&Us+tZ=M$>9W6n zU&({JiM)pQUv?*-aMjQbM0acP9rY#8i_Zaj%im3j!c1< zV#3pcYxusMBT`{V%#2p_@a=F5GalwBW@s%pqlK6c{unm_^zB$=o)<+3saUiqt9y4+ zS!KYh8?8~afF-ISMF&S**d$=7t55LZ7aRYX7a4P9+A9K zr|}(pO4)GR4ctX53#I6Vgg&4tE8x6s^uug>Ws$h~vjJU}zjcmm}Ui!2zs zM&{J)BJ!tpFSl2A%fSP3k_MK(#qTf1l>hZ!{tEr%9?G9x3UyBeK|Y3W%yK9gET8KA zdS{zqe{AAxc0FqWbeK$tYdv4`3j1mgY3vvnerZ;I^&e@%AMJ>J^%u8*(h*r_urRBL zj>tzvJ%E_vJ%EdYK(BjieoLr>5ab2T@5>?%F+gR0-Lty=nZ7sAbo2fMSZxKd%z67< zn898bbD<{+(qVF*^)O_ z3FWfM5G1DLJ1xPq@R@jpsr4{BR#Z58uA~W3!{QHdF2NyejnMKcQY8`-BNM6FT%WZv z-m8A1Q6(OMLe`k5%buA@|Hhp1>%j?lIm_C`RR-7j*9=T#xYzc35Y72gXF|7vT=j5p zYN@3t!rl1?aqcFmeMkhSMO>n{SskBRu*+oHat~@JX~8-M*i$NTJZW)(7~zG#iM)76 z5j&LvZk#HSkyXc`>)LyiWqJ}PTA8H+8QBZbzt*@ZtWGazqp;2HoAvT4dnykoFHODg zubf*-xHL)~#djTNn!50B7~xqyvyt9MT}uITA)>~ZY@+3H+ByTUe$R}og7D8h4x4xX z)};IME{@;#qe+P<{2CAak?OSXrkkQ35{DSmG_;AVy(I|(SOMZ9KW^n!tF%m;b4&8m zv_!EsOMRlwNNHi$#+LPkOr8SsKf?Q&0;3%d?SVXi9VR(Op#wl)lw+$5T)CU4#r4Cu z*wcsS4ERNnu;X#vIkZpRT-SZ(Fp=Q(rSl|GMG`h3dPtP4&jEqVYrAbN>FXB)FDRr$ zsg!F(wvxYa1ZYJMy@~Dm6u(%lBF5Cm$i$e$7|1w)U{8spmzeINp0*>(Xg$0@lNLQ$ z-mo6rJK#_!x;ZZP zwQ(51ZpJpo4h@WKEqfY98>-@UJd<}8KxH%|r8j>?-6WTCRz~)txzNd{6VJ(GoVG}a+hGOJ5mJt z7{Z&#qi^lbtACU$!9M>19K^+>@fX4CESZUq5-Wf;V3tx<_12l2_OuH#gdcKX6(e#c zIDL=#-pfb}V=_naJ(By;RMA)h9wGH1&$`Hw*?qXUM`UA|+Q$462u&P^peho45?XOr z3EweHa%tQl1!+usf`hxV@=xVyaH-W~Syg#cyB>RlH^4|aNJQduFn?qEhgFe2iKdBm zjt(3RyR!rqXzpA{p*fGWTMWvJ7lm;YsW^87d#Ne?+uLmf~Gy18vsDmN`8{WGPS^V#q;t zY7a3viYmF_Z58itCvna!Z9f(kK+hY?Ez)s;Js<>w*;tTu)(-1*G`_aHHdP$E4aor> z{bqcWUe^acITKH`=S0fvrXyjgl}#^>)y@`6?G%k>8qq~AC7E`t)gn4^-juy&f(aSp zkWYj-f~3>x-lMxCD6h&$aL5M{baO6~xTDr4g~epNjq&bK)yBc@yuSKe)~U@_m2yO} z#^HPLzlMpwh)I93ng04;2?v_Nr~TmD_rXBNK+NFQwcGKJkx75BXZ-q%+iGH2>D&Pz zdf&@~T{No*3{Ne!+wKcm4hdLu646E<_cE!PneCPKB|I5Af2{&7Ib7=y3&Oh{`gKqj z72tuitFJU{F=Wkp00AgRTagoP+=*GvNi_I4AhP4I&jMq7aLfLmyX7D;moPGk?1{8( zKEC?P3%~@Y1$Q_+(2Lj3w-vd(BuicQ&K|h`?J11>@`j zN3<;1!L!k$3>xrTV)^P_|2U#h$dz|kQ zSC`h^`uOWhTK65dM3UC%sh9Lh1ChfKbiU_fiG{xEK!FwpZ8=s5tI~=y>&JoHw55k7 zGNv17@Bi>T+Escp1JhGy3*A4c)9}oOaO-x|VS47)SrB!uEw~V5GB7RAlDESah95 zf$D*#**S~Lec|a_(lSYP;uAd%NALbdGjnr=Dt|meYvecRaa)4daQ#265d@>!bYR4k+P?psY+wv8>R1g@yF*AuK7KUxLf&ye|tUUt#=q16QEuiJ`2%6wogYDgX8;@0V!!o327`|4qO+&OR<%#$Y+-c45;Vg_*${C){&5rC>v1r` zo}Q=YypSHm;!!GB_4Rx-k85h9tti^>s9bzohOTFWS7db|M97jMvU&cwTrP`Ty}#(IupPVIT`*tmY>}^QSg1pn&!JYIZJL6Vo)9m;vQKT1 zI@h(AwCoArAK1=0-#y!y+8tRIoa@`x6P5skIF;)SfwRJQ^x~6K!Y=wJ=?b?SQ`P6l zabDd>1bD6-Av$Ecs4M4JrV9`Mr*(78Y$qr3_zPVb!AA(+{EyWK%eV;0GQqbn6?L0x z(?!tl%cW;a9Lpq2To~NUk@b|B+lujGG%Gt6t)#7?W#24UFBMOBy2(?oJQQbK14lOZ zG3QR`#3r)b3Zs;Ld4D$%lRh1f`>qSM**KYlLTAV>j*~ist+#X8bL$RD8M-jG6qMnE z>ZIL0J`%aS%ZF^fp6B2QjuQHnIu_dRuHkY=p=VBA9*W!G!!wWtjDDmelrgAHu+?C? z3_XS6bk}<>t-;mRvtdF`kL+%Y#8FgTBSddbG{`;|$^yCtaO z^j8&{&5U0II57)?y3ARkh`iu3WUQZf7f}zp_N!E(^W(VqYS!J}<8+~;bkPbGisQI9 zhB;_gm;mt*R%CNjBQ6~x*6<>W)ge#6S$m3hZ&3RRq%NsQW^vKjcqF|Z>&hfPUCn`_ zm_I@ayRYnQHGf8QH&V?|od^R;SazB*j6(`~S_ne#vBj1#^KLx#mQx1NnLQTW*sBm0 zEfsVg8e+3F)EDyp4m7L$HNTBh6^NUa+DM;|S%W%$suEmb;Q+e%s7crJBT6x})aUEs zuL1|eYl+ccTs7<2Ts5OP1rEsRt}2K`S^8E@dbDf&(J&5s?akqh56ts8bx-GH2b~wm z5m|4Y$VjJjyA_4EADYEK^Rz4ylW?rL*hB>Ft|uNSl+`?)o2wIY9-yx&zmQCuS&*m( z^8v}49UR0n6$?k!#yDy<{#!cypXpM%&~3n!y0Q1N)3D>PpSN?F9F7sxGjnVT6vP6o zDT<^#;%E0UydNCpSoKuXTGK=w^+a>?9;f=kqz=N4GXGf^T&8{8K&BQwO7RF$NR9Et&g;5*5yf$3;*hrx-4(a`I zOmlDQ#`9ITQ(3R7RkX3hE`xzEjE3LB)a&JkR8i4R;kLXCp?mymKUqBI?zp~>E@kYq z{=xyEDJBzf!OCBJOY$&=CAX!f-?)(i50I(SH*^iVnt@evaHlzTVJh5PT30b?GAVnE zo&eMIxt{NJP5Y+qwp&#k^09&m;5^}{N-9WMKt9$$e8$F|p6A-dCufJ7t@CB4x~2m) z?}w4kpQONeM7x%1J#*L9&Aw$e$E}~|e+{L#ntbHli6S`KS>6#Bhd36Bq+dPLo9I8& z6%(R53I(9P^G>{;`MBIR*ljRe?9_9uUo_wL%3XjnC5cP(L<8cxk`*_1bG*mk!rMR& zs$rb4CJ=zb8P^oSkGR#~C4GVi`IB*Z=B*S$PQd};hbh5(+t(MxC(gv%Gop~nYo7oA zM3@817VqwsV0;s|pUIIdz*ody#B!Jv1k-Y`<)VEIDTI{hF7K@LE=|7ble(^U;>_Gb ztQEH83oslDIlu|Y9NBoZU$eNaSwx>)wkYENWb8oCnb|gsGT19nR&3#ce57 zutj~KiX&f|WYDF-xjy#iqi72eW6m5^o9H+mcNaVaKly0qiHC1et0Tmj){vj3VQ|H6dhNv+XpAJnqi$dj8MEOV_t>{?Bg$jlX znx&kds$H}N==l^2(ZA^y=m88_xq!n|g0QJ^GVTF7uSci8N<^)(s6~njsere4zliRA zsb%-yR?q(rPS+pGs6RWTe?MsP;y&QFz?QIYAd2c6(DqMiwf|kXdSZS(rlNF)81wRa z)O&#Lpv%Gukq$anBM0u-I|ZF>oM=;?al0ZyphW3B2$GV5+X;7*X4c9(YiGC1W%E6I z*YD9fU@;$U*z&4+UfK$2jf=A`P2$)wzL8f|)#<&6wk?kssGqIjIk57|4M}j6fG=0R z1HGD@>nM>#U+Ii`EN0)cYRr`abGtQ;5MN^_>Fdwz2x2b}%r&Rei%>i3JaoCy~#L#t5CAje{FAEuMIE)l8mQiU3rhEb5@=^eJPMMOvP#<|fBW+UqKJxd9$2~f^Hi+}1Xtplg1BC=7nsJ*O<{~AV`PocM?%-o? zF&Ge_Nf^1hPPD9>SI?&@$G%Ad$f0Q7_!#(2d|bHzDXGviMaBx(uXDjBwU+Y~Ds(7+ z?~N`;W;<}Vr$wi(pq(zRawp&&Ah8KN`A|4Gsob$w+SP4u)pqYhDdN(R=9S*he{XU9 zEqVOc6r@1#Hbo5+%?3oE6?nEWxMS8#cDgK|9+dKT8E-vj+O?T}R%y`ww8n=G_w3m~ zmCekpQKl>9a(Qq{00EnW@;^F68u)%2-KDl34z;far|L}Ag#|qwnaPz^Y^|Mt!u|$E z|1~%MKNVIQNi|Dm{_Hws+;|-D@teOYueZ42A)kJHEH5vg3py+K-PfmhQ80!!f0$t} zlr6|C`-AgY!i=^0TV10(akZ`{gw|bfIEf>E`5?K8asJ=5gnw_9|J*<0wl{o)=z64t{?u`l19r= zmqoKsoiRhA*oQ&s7!>iYbmqPvU%t7fVyHJiz4bhyYXR}oZx)BfdnaTy9WCQ>UsY<( z-`ajR{E>(nb1utH4VbwKQhIUy$hLP<<|I)7Q%uce$2xSja9}xDqq1K^LpQK-emYFo z!OD}re`%y7g266f&tqG(j5p~UZU1EPW@9)lzJ@~>am$Lsy4?oSOMK18YX?j_GpJ-8 z?)hoUHCeeK#^d^puPlxwxRkA2vOy$cyMJ>)clP0!*FX2h|KpZIx$j!Q4L9enww+GL zPAO$ujMfrDclYP}=jL}(Y75nu=h}N6=b(8Ob;VirwYOYq*S*P63HJ}3&;18D%%1#% ze-y@qhP&FMwSSASv*7HWhdIt|B)^%f!B3n4%jB{Nx#;E!xrhm=IHj)~ z0nunYi|tipX*<g6ve#i z2x$B5F;yK%8#dxEw3N@uJ@l^@&9(Pq|@KcT)AA+?W}__h&_+pzi_eLOsny= zIBoN@nPWM%w&MNHyjp*?v9Vjoct$JvTueHI&XW{wS4x#^_$ExiEG-X5&#~&AO ztwq7zW7Q;3v2N1ach|&qb2ewSM0z+TmmQ66jV#aU&#>txRfs{ApW(3OadkT7d}V3k z4Daq)l6WZMSnD|WMe*6|m)484%EMtH#G6yqvTcb@P9vDjL8H3<`C`;n&E1MTCr$#} zo6?h%SGx;!k0A@>SwEng?r;cfw&A8dnS$r7>5OocD~=18NGzR-gok1yc+ujSLreGI z&iaH!U0epuaK{N_@vS!4F=D+_*V%jahkl{Z{u-~rZsXammn(A@so(*PC89|`r7qP^ zmnD^8De_tSc=SEVGo&KPgCeJkz1asMDpj6CI;0PH+9XIludFAPJ^vLWBmhKi-R&Wo-~uGwlxxGx-p(}`=c zi-N_hzj&U$#B1(xY!O6l?F~RZdof+XmREp~-SW5jq8k4p@O^NXgf9^iwI3%0vl)LX zgthF=HSeYs&STw)!fB3UsD~$Qa68s_$ zpe+RBq#(fzY1_anmt4RTfXiX-j`8BoW$Mx=Z@Yzr-5Q*|Z4qn$Sto@(t3fa>)L8>Q zj#kw^ROMa`_`o-2e~%?tv4hnu9xdkRK1g1zkm;GCCm!`yCB`|_$)=EEf%MYOF?c&y zk$#EXUYI_5p}6FBEgpig=k1g0c@?KVmuf5sncE6VI_%HKpM#-i+n;uHV#*?~T*157 z6YQrhQFGA3m_ccnHisD^UL`7h>pDHLF??$+JXyyTkybe8do$K38#!HKxvu~T>M1@c zrK$Wimauoh`0Q#2T{5K@lN+Yh5AKBeAtWU$^E{8{Z6d5c>5Z$f=~?gemgM;QuKe0{ zB6)JXw4^_mHKcO7`^^@V=5sxUil92u=(^O7{xZw5H&vN`Q5j-FYH9X-xMZGvz+f}Y z+s>j~1q<390?7PX&9-nq_mW`&IFG#AGfoU)kbg^~L1l@ZyrKg3o$;pe9Jn(O5E`L7 zA;FKs+1Z9@7*KKbbw8D^(@b_d1DUdLwtK__?)xWRjKmCXL8G?6T#YkG+%8>1!-l=} ztUMObkBffkXbLn(JjCN@toyJ~($H;B8v1;D2W`*}lPS{;3sL%IJoDb-KY~b!2Cm(F zKOF8t$!lPZg?-4?7JhKx)@?MBYrEVrmON1S)kpHwCHgfyU2;Lc)+rxg0x&+-w?#SR zXI}>4Q|h?GR?;DmZ8kO&B|#rYksCd7-C*6{(D*dFPla2?A3uJS0e#BZ9B@P}&ek|8 z@pK+7A00W(+(0gvk74gUtL)Al>fcgdc?s?`7V65RPoLtE3IF0P_o*(F-u=2+sArqT zG!drTV?&$>6v~=>0jsw{dK~m2h5b{kDph=_v3++|tH@?$n`P;pJ#*i<^MZXxC4-s< z?qFm?Zbf(-O88OWGtdPPgSUlcbCzZZY@KY*Bi(kUMVua%xbgHKY2-BSXt)Pfpf_2- z;dQyu+s5mI(>XCIt`|L|cz3Cd%LfSCd+$TOW^UTa`&`d0#z_LjvMyPcq!-AELz`(X zBo{)raqWsk-mtvII_b(6^a!qja&-MYRpmy4JOPKu4-7z=3WHdcDX;yNsom75MldsVNyzNo z*8U;lFjB`!NrzCY&+~iqbn%^%vQ$g^!lX6tH)MA{R<2Ok%djT8N=nXnKWb~@tx6-V z)J@*Pg%yS#^$NR)jU;B9lDdssLKRSBT7B$Tm-5QEVP&WTK?Cn1)cWXA#1ud z`}9Uzt49zR5%eq6qU~08(SucXs=y2LtLd$dgD-8p8TkYSN4(KzR%;xfUQtrBw@te87Zg67 z`C!yqP=wkbsB8r-_g8r3f7xxZ z0)c}vZo9lvYx>c!vW~x)t#hn|sW7hlhA=Wu{o0nxjseWd%-loHZVWPB)xd$C zDt|nUi<9B}2iIW0=uz>^J{4?tGiC5F)(V1;N{U#K)cxNU#`}MZoLmaOoNxSTc;?|7 z)WT{x01bmt>b+*$!bZNdV9wHuU{bS?5ttKM3mdYl)cmFwp1iofHW|_P#P)JChxQ%r zx#i*WI8V;6LF$Z+%eNb>dM$vBFxdj#7aYSDB%~y?T9NnuQ-OXgijOfDlMWlf$TBVD zH3{#m4fJ%g4B7H5oYn+A3wCopAMRImnXb27dUUrdHif+?JxVjYm@P}|g>SY;!YLtE zf-Ad80t2UZJkM)h2l4ZiKNR2Uf)-<%=4K8Jyp9&n&I*)+{2-0}nH#Fx&FF z;$EwY&pj;}PjW2&|I48|78;=QGUS}l5O*WsfbOP-+TFM>ZXmDy6Sqxj@9EnwQYdx!B$bJ$A0ULzpIz4ko;j6eKA$nPt)FO zr%Tk<0@-c&9rfoh>^Q1{fKjym_}+jxZ3b}Yxd8P%CxnVT33`IPe{fQDn!yicukN~@ z5OB05I08Xo46E&Ndb1*bZO--R)oV>dF6k%Q@6(M)0=QqWb2hBAP|*D$#CAsec=w=K zG+qM*KI-F|BRFvMjJb*;Vxv`P)S_37VEEG-wPmL?JbOu9&mSBnPSfKbq8iQU($D|g z@ZXd_ZbkwDh$W$~C}NI5%>6S_HXaYll+HD7YDoI9$@x_MB@Sy^ANL)}keegmB4;r# zl}PI`&dq+Oe|27$M#*2-#eirn{%nH-{w!xKU7IfO_5C*1ck(7M&mS?x3t5Cwm<^T~ zHtIE&2woJ4JC7o3PSO8J3{d`R07Z7ZJ?SKAy zm6}y{axZ%q!+BG;-*R;{NC|^*V*iCPZdUz$v+JIYU1fzWy^=f^AGtH~MgD4u8V!G2 zbP8}pCrI>o6(BVg=uUk&{j3ya5G}gzFw$CN22w=&W_UxlQ$F+EZ>>%~8wjdE-dB)! z>6>MWF$=vNY_)<$@k#p`GI=Z855#v>42AdnS;3wEi_-Xc=U=f)WC{82ml2A4=}T4k z6^Bw^_R>$klNQDe6-cQ4QWGF5wPB<>jf#%?sCUi8w&eja6$%BSr6mQ5(JJ3lf~aZL zwufI4Mljq*&F6knY%M|do)b;xZT03q zE*Akc)m^JDN>p1QVPOivFKS`*xcvo`FKYxGPAWT=sQJ159fTe=DC+bj=qdfgo~Nnr zQAQ(8#9XLzQ*|3yNe4wTCQqDZX5w4@{mgIJlGlPkBIGJd{Wqm}@@y8#j~%;JA3EH9 z-Z$81S?arUqT_PT?DKYY6$RCz2F!rXFUef5RdFp-ptKGWJyi7hpI_a-y`2o(=0PqE z{wFdWx8w{wTQ-NzgbW~zb(3(KgV3NqJCDJQTreX|w7JgY#I6rzieayvndJ1NNxLRF zda-g>^i<;OL!0V44;~ks-?x_ENz-XmjoJ(qY%_m7F>ezcUhC+xQeC z%!qfqL~+LX!|m>jF)AG;i+>xJn zZ0?!$_)Y)Or>r#w*B520H`h8DgA+HT5w@EAFw58ZxKUJB1=bi^+a>XLA9gnIP-ujb zlewKbP_P-c4ePJ|j+*={;p?^#Mc)8#j#Y8cDybR-PwL))=O1t5vt^lqOy}rqW7Kxb zrGHJM=eZjW@^?IJWq0!8-Z$SU98aV3+_vqxCAy}@?_tE;%!0Mt&ZiW8TTLwX{~OZ% zKiY^=fw#gqI7vxY2{PzO!r&nX`Eu-xv_1FpCgc%#z1 zlw&Bg>8N2lY=Csd3uOgYY}Of-AW_TSK4UCe0hMeY6_hkgw)>>iZ>OCdHAor=pIxg0 z-mt3nX#y5gU_4&=<9B^?!s2@y=Kq1Q6&uTi-fZ^JC%-9KRe{w`_T6tv8@^=Xx)wj^T{S)aG@CGw& zSs5#PT{B8QrP4?y9=o#lfN8$o=-m5qHE+mPPW48^uky{O0Xh=Zhj^f3c>^wbWuE%wuns zeLnZn`%(?_5nr(!q1G$U! zPlk2u+UIKH(|IE@UEc*3|7u)E59Fg-%SWNZk9sSgQYS`>amU#o9rtm~7o~ur|4DMU z4XOUm!tPejAc10V&M<8y2V|pG3r^@IAmuRaCh5JD3txytJDX?O!&lh6gOnR%@iLC@+wlp(@biu98ZSKj ztm8kTh~6ROZ{@kxI+zu+tUWoDH*l8S`JHPY?#Ul!Hq>Vg0}?u<;p*93eSyk-=`!Er zzog41Sy@KJt!D-Ixl?{K=fYny5UodYpfnQzv*L5QC7iF?&WiR&q`Ys=md@u6ogQ-W zHu>+o<9g0Qku{v%{uRGUEYk?3m-l0+Aon?C%SpRoqGdM<*W?c;%7khhuN-bZA_>dm z{5%BnwezfM1YtiZxM0N&?o-|sf`3q&j$w=0+`Ax|2*OEvhH>7=E&XIg8F_Igim^Y< z-^UA&xRr+9bL?JEYoTtUd2wxZk`0!rfW53_@)qZCY+~4@9>~~5hOyekQeHwvV(|Jh zRIWi(F6->+oh|)r_$#4H;+1j5V6lS-^(${q5Sm4oT~Q-VuN)XBt!o-*!L4&3*0Y26 z8>gdadc~ajKRa2%Y7;AumN%@jwK*F1{js0cowAsxTH$b((q1b!@YQE0!+Tqw#?9NU zGdA{GKP9Cy2iVH~qD$FoZ0pTii%y@+1;)0~kE(s?Pv+OE6N+g{%Vh4%_1Q5)cWaSs1+`;@ zzAo-Ix+xr>^swJ+}95CjqFgbY|yeD^7{^m{p28^6z2;VPvs<23;Qxh0p7mGVj zEN@?#c30v(+rFl_ahyHRraaoA^o3qz6g$grJR;x~WO0kb9x!oGKkh=rZOd|8vej@w zCVS!E8#wVdT>P_EvVJP2?z}$g+m#<hI2=$^Hz?Eu!nSJ| zRC(+wE8=A6RMu!Tb{RE$IBi=hF^dV)kv?K5Z62QeOhO|~^DNpWS~e$))hB}7$)R)Sa;vL>uxUT6R}N;*A&10p$6u{UwSJUb zrh7{!cJD2cOoxn-vkUGNr{xS7%op2Ru&`_lqhk5Y$rbk-S6n;)*@CKz2`#P9ms){Q zHadLOr)e6KzH4P6l*~U)NRnvjiJ@9m$`z?zyfXGX?53g}5Vce}>p!0Uk$UVjdi{@{ z-zRfIXS4p#awvv{uuX05`~1zbP`k=RnQwh*ba(aHc0~&UmSTK5!+v3zZNeR%PmX%- zMU_CYedgo6SZWRIoVW`RUsRVoZ49_P8$>Jz`A zZ|A=q-{7XDhEPW%n!B5Y_)^30NX1nx{Z zNXS{B5WCWG)xlTS3cvl~O=#-;y!$_Q!S_g=QBmN}Rv_(y5M^GqrjFcwjS>9P8-|eY zD3WK3O%D~7pV8wE93}a-hMnBFR2fMM>8CnxttLcqx|sv_RTlf;AZ958JLCr{L#l&g zhFZz>I)1TH2*Tmm0U~hloY#U0XKe9NJVJLuDg??x4vS(w<)4Z=H$689QV&eZ0@#l4&uUD#Sk=wqhQO zru6<2bdBT3X#wkuDVss@awPrHF=oKz)x&eVZ|$?Oa5$^Q1a*?qbv@%^fj&1Ekc)An zsGIIdU&ME=LSni%Zpt%~-R1j4;uhq@y%Y?kB(xh!el!W@W6!D)mlz6Kf`7W69Ep967x^HSauRtx&L$$y;8O($ z1r*CE=}Pa=wm{ZQt(@K>)5rDP$p#|>9x zHE$``q-i-Weuv^C7f+Xx)~Bn3z6YIrnWdlmTT(iwnB!zuAIvW*Av&MM!U5kD59ZTl zr?yXjsMuGVpHvbanf6~Q_)%A}PL!(pcz)1pPO2^8{^~Wx*o720^62-qedHHK;~Gpc zsYQeiSbE9=$ZRVqLeeS1)%>5VpTw8{SXVKz&L8Gnkcj_QchkAzg z@YfYjJuK-rZA@;0)J>hR(pcuqmsO7zN<*SHEClOejhb9NbsmJavce;aeaj?8DZQxeW|RTvL8F&9&2z*s1<%Oa(QX5nNWuSeUF<&5AXnSRfquAO zC;32lA1yx*ZNPDidDKb_R}Cs&?d>8XhGdA0cx@56%6xVGL74QXG-l=dFWEFSomIZ9 zBrWyTQ_CtM0uo)v(k@_yYB}8l)%g*FgRK)^jzPq|7^3^h|rsH0JIPM zAV$5ytn7S1cD`eh=IPnu(VhsoMSwaLsikjy+=YZQ8xDIddID$)-*ncN>W(|A1h6bPCrx6ZDH9Asv6v` z&luy4|KaISpkst6L+75+%QyVEXMs*~3W!>;h{aka)beCZ$H<>17|YmDq>LQ?psSC% zqt%&fkK#Ti&;Dl;JNpxLp=gY`3ynPCAbK!kLQTQr6B88)Ae8L11~kgi{(j5q=|^>W z9fn=3r{jm^y{xtnS}dPNr+qX6Fi__>jwi?hBK^kr*4l!8k#U@nvAS8XO>?Ji_aAnK zQA^FuIrgclkU0IG?r;=C#lS^}Qm14^cE0CtA873H^X;!Zq?NK58XCG*Q>jFTD*+?GkUd(vcs8Qo7>3!l1QI(o?{6WDg zj`saUhIJjFQcLO2okn0Z>iC3JBfR^C^Y{W*th8|nd+dnMr4(yt7A2kVkMk`aySiEo zQQFR%kmopZYr4N1(xElciheQtdHoF1<^y7*Y9)J1wO|+PlG}OSAn502)JCXqzma$@;n#Lyn+pBv>v50H~pZ0trEYUIu9P~D1T+8u_rX4>0yuH z=ka`9Z4I8z02mW!@7l7Q3G7kuWfKVSq1fC5ceooz;JrXr2_Z#at2H8+iE2UYS(9Ju=RHikouU}eJZiLu@K-PaJ!mMSU!}TJ54q2Z z^n`{*bQcI0QpJ*|Dl24?=3SvVxB_og+%nh{@}WwFT+wu6U`?#Q@NF-q!m^EPL;bhz zM~1#`PgZ7o9=eCM`ST{$&fB=pHvN`XPoDi`J{c?EY>MIefK75bovLdK?aW3Ve(3}{ ziCG{*>u~8IH4~SN;a8cZQ_NDV-ZS-K3`(KK7Y>-MgzXz#+pyQRZAzIEz}q zmlzt~4AM91SFvcB8aN`R5=QQqugf>%?JFtb{=>>WqzyBODhYF3Zt@ST?GHUu;U4gp zbr<5DQGA7!9Oin_Hj=xq0;jV?_z(WL>WaU@&aod=XXg>!XwLlPhIsaKdQu9tm*@yXSv&9<|2x5w!gvLwY z=F&3=_i$%2*j7S&KInrF8?Bbz(^y{)=1)=tl=RF*Z(q#b$Bf(9d)}Md8#>%#J~&l- zBesE#N4NXkcd_P`gzmt?1L)p2pg&gW9Cdi*wX!n4`|UTYFZ%qc%e%?Sd;<)a;ryMJ zeRS@m`Bkg63%*ga4n1R#4U4+AMDKz26i+;a-Ha`$)q}FiCC04#CcsG#c3+c7O6gJi zP5JSTjH)`_dlG|xSm}zEnWc^Y+=1Ho+w%X2(&#?AjdZ4ZiF||9ruyY3?W*f)BGU^J zXlQp@HaQib_6G8UXhC|#zWGVnZ&>KoOI$VOPT#pse&yw$UeA>sa9U)QcH5W^Jz#h4 zauIVOLp%#$Y%d?$?7Qr@dF7yCNoBkXcUw&M12lj)+f6|)=(-h2D3GQuHet#-IE)4- zz1ZzM0aBr(SeLXRj8obypkLl6z5;jqC` zE*Hu#t);A$_VVx3w;|UP=&dmNBZ`^8Fn>%uY$=364~%sw#0Aqgo_yrfaIJ8`bUwdS zHWfZt>eFeJo&J4}T}f)avgD#QpboW@Ab)^O2txUlkTe~4X>2w`8u zn#hz{zO)fPaW$N(#5bMk-FbPHE$3dzGO9SqFymq{XJHalLW~J9mC@Tgjc*uD*AS_# zAO0J@_`UQb%3qEE!sp6?zY5H0U*iGtjuHiQx)*4KKaJK|*fiFsbJ*~ELhz&0VionL z73pH+kE_2XUZ~;h3}N+}8^mZpDrdIssQbXDu|2~ZtZ=zOz7KRT^f;qucy2%1AwHWq z;v}XC<8J)6^jaSYk8z3XZcc34d52-d&>U589dV^>a7n)4-QDD>Q340wmlhn<=u?vP z=}egGfX{7dR&*}p+C#WfueFkDri2@HqQHMyn>UhNek(G=B?gL^SqJzB6Z;BLr{&VJ zd9nN$z>@@bY7PeP&8IObdp5=cTiTMuQWX`0ghUTd90}Vw?UBq~V?2MqRZumKC~b3X zQmOxv8&ymqWocrr|M-W&(T3UxEjk$N9Q8k|>9p_IuLLMm1{4RpxmyMf6)u1@t|TI5 zlKzh;y8n0b=u}Coiw*a$)W?PY?d{(j*OYYJawMk<>f@ft3*x64Uh9svsZgZm40%eQ zMa>5^>q=b$%01E&2TFm8*$~X~bRjOrr0t6FG%-(*-MmL_^rSVl`HWjDvnrpL+?t(4 z(+Un36-t%>d~AFNn2S($(X?Igy+F#?pbaTbN~d>Ue{oH>*#U)A7#0_0X}TUzvvAXm zD*g%Z;*=)iY^$W6pBnJx2Zh4F@V94<2>fi))Z#E@XLH&br&ol(!l+Iw@_MgViYtnw z*g}X(I2i@T8fF}!~BU+E40{70o{0a(2|yzJ%5IUt;9@?<$m zU9dMuH`ACe5I&|Z$kC|j!Ev*}Jo-9X9!z0I^iwZ5hrO9yPG$XB2Mrxrp48<=;7Pc{p$b*KRDbXn>OH?QMHC$O;Oho&f{;)s-_12~NQZ@;E zBEV6zNiQ?dMk}C>3a6^FKkiX|<0|L02V)*J#9ThJd*Q9&tIeZ(5&WmWY%pZ!zw+Md z+)~<)lVVD*a9sulQ8HQ3WHa5k6leD}^l7}=^kf-Zyv$sQimo+V=Drgo z-z={L>XEls8>qf6?RC{Q<)DL6?qD?NrQMU-fjg9F9@R0Fkz4d^hU->QFssfsFrL=K zKDn^z^+PSJCd;dw{1o)4tQ zxzxi@W=-$oCSD)HWod|Wgg#nmrcNLd zZedT?EBWbS{SLH|CC?!NTxPe_L`bU|Oh{lyuFu9%D@;a3DT|cmz$pUcmHCOoJAaQ3 z*66ybpnOI7RPYPmKN(=9j+m>V04n0ij`S5)kFs~A{)sfXC!(^T%j(;<4Tg@(UVN$2 zC+0Poa+1TF!kY5stxZBF@^bwhMpKas@Y@HkQZtEVK9p{-V1CRHFr+5|6taIQv>`+_ zT@HP1$iBVgniD=~A606Ck><`DL+q`2H|0eqLtCj;^IA9Or;k^FS421*|8lhM2; z>;l_*feE%!hbF{jX2b&<#OnsaWjgX>-TdLNb~L8AYxq%JT+##C3%jOwJ(&$3zB~wf zs{bQTGPhy{>L=$#!Leq^S6_;R(V0BA0ps8TQwcy~_wy2eTKXtD@=kn&rnf)3TC6-*y#9Ksw%s{V zUaVG=gfo+STk*-7WuhV1-I4bAlAvOGIl(N|jm>j6v_(u*G&5Q^>-k$k6WFP$lNyVw zW7~c5B;3~TR={(>`Qtvt!jcz{zAz+D2^!x;X_cf4)t;3iPgIV$x^{Dm39H~xJo`x= z>BnZ1T}xZ=i_Jd~!q)dVBgRobPb!RFYPNj!gYchLc%)!{-F#A};U`M*K! zq5wiVH&Nkz@NH^6C$-Lfx}*ygQzE|>W0fW(AxHa!rq#v{6nIhjvYbPV@%by(enMoT zYT&Nq1vwWT;O>e{IR_%FB`l4f7i<}%9`K!ulVV+tv{qfVPez@*Pqo$0#0t@}oLTBW z%6PT|oWrO1ANF$x5m6s0`_??W6t8%#N~-jEik>_D2OM-)F>CC%q6GzRNK_q2gCV1` z4kih@Pb*7f5B_f~LNSbArWLkRa>m3~!DtA79B4o^Zm#I1l#$g4m++)slPBM=mb;7* z;0|I8Dr!|>A_nEKqvFQS??wz;PBX=rT`$ELj>^l-N2V zpC?V#Nx%q`gp1bX-8u1c^&q8eF6X(sw22MaEnKKl!Ml;Wc8uDgx%uar>o6_qNMjeHptygqjJ zo_)%9>8sFS0Cahd8@YY&a_2m1*TG_>6$V(6?QDs5jpmsu>lgI>%hpE_kdEFBn@?>g2wYh1F^&2GH%n8Q$6LjlY+Txan0Y2r5GH54nhZM*z7EK$v^I)y)^?E}e*A8Mf?f8g` zs!T_B!Fo&{%aWqRmPeVNw2}@;TQ1Ay8v9TG;gUc&U&)<{vh(^pV+lB|={^T1_R0#n6qQRS0Mbb(tcAPrTFs@xlZAtg{Rjj7B)Ws`jS(X# zQ;n-@Xr9s-&T5JkASkCNlRYNwVp+r%4#1@kq&EatCrcvJQm#IrS`yWalBWR1Pe6PP zQNBU(+;P9C1>M`pJ+`uq2VS=zoQLN{PiXs}fK`P_Hp$i~-B zfyuq?!My*tRQOX7wiclTXK1<2S#%u_nBbC+pB_Ng}BGw9u*d=VmPUD9q)QzC~(%ICP{-4A5 zyB~e8zVck*u@T&`4sd7RvrJYt>h=-Aa8Md+Npq7(Go>yS9ZDd&`)v!*N<8rtAXb;pfU?1_M z{A6JwOtUBHR@1AU5Z&!Qqm5OyvEfdRb(ig_=L4Dhh<%iG3(0nIdDl$K{kC3J>I%K6 z)7afOQcvp^UpYVc#NH)yvF5R%k4^G^UHuPpj8S@6q2D zqzI?h3+xAWc4Npwk@DLFgQIdyLtqvszIa&$42&h^pm%)b%OJ9Y4Cvi`jB8>shc1f(P0TiGLh`^N5pKp1}* zY9gq#=Jo#=Z0Vmb4(b^4RaA{XaYWm>4>R#;Th%W)6vt)afT-dzKiK1`%QFbrF)>@e zifoyQ>QxLcnSNTDK4bew%`S!f!x7CUP)&7|4nOUV+q5y#Y#6HHq@l%Uu*tlh^tJZh zi5~CFn=QI&r)16D4|h$;w8-mQkpZ=XnnA${ zf-gFCKg@nmDNNDVUSi%>+KPp)M`9^!y3;q5yay5;-+LR>U9Co*hdIIXZ!v3Fm0o`1 zJ#EIXYqttbQ}E&p znrQcEVynDX`bgQLDAXCUyCkzVXo_e$>TaF*h@WYjRVIivIot^wpYPUrAhYv_O=W^+ z_ft7i8^-{%{bWCMxX)?d1TG?ZowZ-*G}^MISnyPj-t?~NtJb5?A)&_%<9FPbJeA3X z4fq>2TxsDk6wYR}Tl*iJD-A*Ko3$P~#~%jC#~^)ZsTRMsf{%&^U+%oeuP7&JNy6{?iDlnco34)OM1Y^if_p6v)L0u)D7NM|uaDr!O{k)GzeEp+X) zUu|%6-1r4xeU)8(Hm3lSU>4AZ9yap#0JX5;$#mQz0;7yJC-yfQU*lEn;*pYw1)CXo zJ^G>^Ez()X6M8ftA@!I0!&87Vl|gzK9sy@c9<&n`V0QZ`hVMG2}(y}9U=fC&2yJzH5j9dQIK=n_+8h_ z`*?A7^r??39a7sZp|%Yh2VjWR!mv@(5hO_6$RhHSIHNDw~mU2ct z8CVKU1@(e{z=cgk!$cFsBAo$>hTDIX{;>FJw=y}Su~DbuwJ#W$mu%mn_6 z7(ohDLp~;lgFnr1}jb0wc}< z+g&8;*)DU9mi)p%&wDVUUu+vwdc9+whegD5g%6D3JX85+E`H1V_(D5KkAoOk0X6zo zANioltu?49pT**m9g9-?$>Cg~+VlPWni(Hg<>}5|mo&_eeMV1gd4s!e4du>(X^>@e ztE`<&wZ8me?U|t@!(ZG^V(M99?}yD%+VT;@o-UUmu3A}_zxj*`)ej%oUug=rh+mnnbwN)uD;FMk zw+8b|^x8VicTV{cUkM7w8}|h@Uy)9a&DLLPQnMUu&w3b=OK#B|D_sV{KVIB@Jop(^ z+0%(%Q@3~2@Vn8UrxK0gcD1?m*W2l|s(6wTwcaR>e!+h7X{hhK&1{3SV&Ai88lLcjUVT#=+^s@ASp97iC-oBu>>mR(sIzvtlJR$ewuWD`y zpL1-%b*Dvedc&(uvcIEX*QWu85$5}`6F=5`?i!*?E#tn;s@Y&{`|YUT#C_Ub~<3<31kECS^dq@?n4 zyvrA|_i}dOG9v7O?GAGmc5t^JyBUT4>Yy8ImElqO$jokwou$CMhWbPHG>*Mgwcv02 z0WVqfddkBTM(A(#1XK8Hxm1;|(3ti!_I4+^w8Y`VM6$8zI&4!$AJb17=KyLaQPf2x z#_wIsR~UV~n`<_|(WN$hz+c2zt(QIfp2cLelxvWGI_II(g+j=(yc(MpQ)NNo{H%dj*mHtL|;9te=(U0Ykk)X)JkWCI$RI540A0I z{xbg0$C$f~<2gjhdMpco)E++cz|t{0@w7KL}ci z*wxu{>Jik5J&%H2tuR@&h)o?;qVRT*MQc8ZUNKVG%~1LDau8v#JoHb^O3U+QC6*Ya z-j05xNqr9jD((Y3S_nb$`QqbiMq+A>=@(;>>n#v(#whE1ObdnXsa*HY>e z_*)Jqq@ybSglIkTERk(SlGy9k3Z zcBDV_78utxr)KsP&%k6FxT7ETu(kjf2q0K<*{%J7u3?XX!}e%=qAxzNB{GGKFgy=_ z$+(hKJ`+&iG^6*EL_Ey08Nwr*2-BTgH4{Za0$BlnFd&_&0r99aD4%$rxQ^_Ha zf~i9JeoYp?9MKyuJ1XbTL8}9&hmGg9qrco{HXUcoIbgW*DKXrPPAi^Zrjo)Ynsc;- zZRr+7El${!Um49WG#gzXogL~g*vG^;4%%%4i)6DvX3wr7C_8|==JGa^-q(5Nr~;E5 zoO^5HYr{!9gM!l2q4&^)#wXQt+9d-Pf)DLnSY2C2)zXU^*cdCDn?_`?L3q#9>5Ms- zPV4uEp0?u!O(@XDA&pbG+%=)8ge3hm)b6vPU-jrXTR4F_5ZJBUv9=;7yOU|?T7Sja z>UDCuq%3kg(u+}C43dC-rb_=F8Vz}NStOL97DSui2l-h5(#c{0lOIp5DP3CpI{!x8 zT6e_W!C`S}+x$am%1_LX8tfS#c@-f1SDG68)odleXcR3hU`&GQ0>y!jY-Bdfc3*N2 zaIlI}!(I;+vo&lf49OtIPHIi%Z~B7XOR~Yemk#xcz9A&63XvldvQOED93LN1wT6%u zk-z2!M;Oaur*|^x3gAkZNak82osO|@YHOPZZ}>&;A5!*sUCrQ1S*f>=o|>35;~o#r zmXkulVI}#b?!qOcwnJ>KTB|!bc(Slx=9*BG&P9nE(LB5^c@^Bn-|#j~ZdTgNOS|zH z7mS! zbsCc1_#}zH&VXz3z=v&A4oDl$4a6EMyqo9@*u->S%De_-YmB9x15yXB^UvIuxvn=b zkkuTOy5)}SIEogiVgP3CS*A<^=>r_MqUM<;b?Y z&U|4Kf@xtK^Wb^S>i730Sig4+ROa4K*<~B@HO4rTXu$4oNATeF;clhot6&(lxZMR37!9ieI#qa=%TgdzbLA567Z`%Z19Bu*9geuhS$_Sq{Houl0|Q#E&l^Js45 z+8IntMvnohwr!?pEVDi|wNrcyhRG%3L1$_?QD2v{BY6#bx9s_F4PV~Fpen(D1UXf!`HSj{{Eph-PY&zT;ujvE0c?X2SZ=CApA#uDtw%(ZOTjz7L&jGH4P$^)55KU z3jLR}T&G9ejiJBGqYegiokU?6>uQz5=>5so=`oc8b6qyU_#dCmVJl5X6AZahWp>j; z&Rws^IB_NVu&9bCeCDeT2^4wg{K>^RLrsyWY$=STws2^chK3~kbB}|R0rq<;o#v;F zg$epqzwJ1Rp87zM0a1sd;3*B{0`xc=fAAMZUg)P8XI_}9r6kfi_s5=Zvv)f2A{-tI z&GHV9<)-fS?!6BUP=D?wWPkH+?q~DCS2S~rOa^ORmX}u+QKS12htIeF8cx)t+NF{< zb~1ce($K6zFS`;L5_IF6HIwZBN#y^}r-jYd+MOJBs3}L#m=1(W*CZrq&B3&Q{w~QKd$k&u+*3YP_D_o{AB>?B!F)Ru{GeP}f!uwkg<;3><`lha!2R=LLIX2DRoMixyH zxvhn!8I9GlUbVO209xM3d z`LJ-5@m~%28nZIxyFYB4xi+aFq5^;f=g2}eSmIlC&$}yf@#fBR%M0U8JGJl4YBl)) z09ci%D7j*yPQ)fxoy@1NT^KJ=aVb8}($gPrZwPy`ExGSMG1y?Y^81jb5^aQPHM27} zH+LFG@1URi4o2-FoqOQduj(%CR0JwYdO?b*=J{9&RE=#4e%yWh6r1V6ZQ%0rbm#+q z=n!3Nc~RcG0QQZUEZ29yo}&d0iMiWFO}(= zL~I1vu^)Z))0+Y2YL?ca?rMkZonr-|*3Q34UUe69haZTk$LQ*27ax3-h%%xZNf`P# zh3u19dnW&rnpr6S@;CD~hcPwVOV&W%Chz(!zq+ht1CPz?!xsXyyY}X+vq1;2$A_cl zr-wD9L|n)HS2}j>yD4;(De~7J!V@7uOXj~djRi*IWK}`$6YDLh4UXk2!hXY{YA^hb z2kP1Gfz6GY{OG-oUFpiy?U_fkXbZ#G_PmDH#b9zXC9;G{SRlZyF^w|s4c)lDVb}jMj2^@7|19+9+Z@~*>>HU zVEx14YQP#@%H^UADqMF*C7ssKrQS;Z()M_+^>6W9W1qPQGl0gfMjhzb=Yuf4Pu<0SbT z8OvPyzTaa$yU5XBkMw8daACjIO6%kAR6@UCF6$3gs(kCzn`|xM8RTdnORd#UapCBOiqE=ROPbtEfQWpK13YQG|?!PA`Ncj#yQ2h z{@jy?T|d5${HS4OseZP?{UL)o?}vNM@zT}${na|lFndhDz_8+er+GJy7T!qPTeK`A-=iH0eThOVq#vI~^;r78$DXXxm` z9~OiT!;j}&pLLI^P1VujFZ8{)G98XUwC4fZz8+Yd#dOrF-1)5ds|jIpBI$BO+#FVB zug?6W-l%riQ)Z*l@%3@VD`vtD2s`jRHvly@UXd;Q z40|YJ)=;nMujGpK6U-XD!5{?(7M%K%U%FeH2(l5GMuZt$^7)krEsc1LhrCDa`{BlD zM?*x`zJBdn$?v2#6VmHC@SMk4kbS=XTYi*aqmg*iRl_PGk#w`f=gLggdZQH{GSAk* zS0e#4u{K(Xjyq znzZ>WGmoY(`AIV3X3ZYOP$E^{e^*#-+s(TH36lG#g{E%^yfGxH;?gw|qDy<$d=x^L zpgrruz-HV;leR$ir_hk6ajKT(x4xyBObolhtsVLOO6PXh&IF_NBbC#9AFbg+9lX2| zGq(@1Vw{-mDGlW>u8)5HiehOu)v@B~eVrkNo=(z-j=bL|2ljSM{UT+yoTQPISe@7%6g3f#6Y$ld~}a_zQgr1vqj^c>&FnWru-V*M%>69 z2}9N%ECOM(|K^7mdVWR1-hFiQ{mZR32PpM%=MS;{9Mpy_u8?Y>!2r5W#B|yGmWXYM z%TR$1Kz7Jadw;RAFZcMQm~Dev=-tTU8XQ9KwC*5g!iFncH;^%uZl=4YEOO4u*RLQd z3UM><*mAyilRl$Eui;MXrzS9?lIh*>XODzZb7DK>MB9vBQoUzCpIT z@AqS+5G78|(a`B$XJkLJb<4Zm8>cGo2Cmx^D}9ao-erva@Ot{sYb3RX!xz7ksp7a6 z5qS^0=G}GtXq{kOkC#c~apYP1dl%xb^~|C*Z2C=>v`2>x5Gnil*E^vT<#UYfe}D&n z0%|=i(bROot*2~E_V!7Iy>tBf7pA^u%@Cn6yB!A9>R9usm;IYajRJWlt}jC=-b}vJ=%q>KM!=74Y`L?h|Lm6{ZgX5;8A0qPsAJ9)lZ>6Px19BO;GK%z8N*3J43rMelW!vlN z>bLrm{75oilVB+SG}4i|I*+~fH#jL{drK<|?cYEedO{t9L8-{gLo@HUwRat%!P@g> z(`W~rqjH@I2E4ZUxo6joSjp7`is2@??K(=3_n_fn+we@2u%sGbVp0A`7Y?Yz8$;xp4=OJ<=b2* z@rSIIrO&XRIPkM&GwO@=`c{}wj#I|tihU*6Ke3g=EUp7>z6guOopsS0dl!F4YSY9(~U4=x*YCVN?528;s@QI4_y%`$`u07~jT&snzEu_@^ zyzSSTfZ@x@*sexd0jw%|!i9FAJ1Vi>?$o!?&#pPKnLH{gSD;%xI()OHcf+w82S%#{ z#D(FiC1D-q)F(`K!#nJx8dF1{g@6%?HqR_I!u{Y#~f9q zSGGnl9iHkoZ?fvL5tg2DjEi=Co38-ynVNiyS}D=dkWnW}S9CsM-g7%*;K>B|k0syves<|8i%>X-lJnauvzO&(wFdp; zo$=xB2Zavux76>F+Qk*5=O)%(noCzPb+?sQp{y?~KE~hBd1&p%A?Pc3o)z*nx>&Xt zNGg4t*_Z53(w#X#8dcEjJMB#=Nn`R7#mo$}%2Gu8l}RlH;=zUO_=99a@Vs2&mstK# z<(}v27f2DnYATG2QbpxLfG)xMys=F7oQWxLPaDFcX!6D0(^cKQxNmJ{*|9fjxxzXi z;g9jr7x=86`q|vd-YCxh5_NB@`YIElecwi}!#JEvUbPAF`mIOyy+J3&JeOR3cR8(G zxRCIV7`42~P5%%cHra=~rtj-M09?7jj*mL_H9Ni6)83vZm%I15A{|1o>K!ZdEjcgb zDWB_fFu_$<;y($i$}=Wo-4k(TW2%cG-sh_N`uqbJ%2UGO^p9!PaeQ!)R+P<}DrH>tf|h#%-)K!pGi7b^hUDP-*w?`~ zgF9niJ9B1d3=)NnJ`1HN-%{7aEjM0zv&f)&)a5zr?bhJ%w7z3ge*Qytv%O_d3oNo_ zZ(}!B#SJ~+t#8ARFqmuv5t>Wd(3NDAAXI}n7mV86Kf#Z0LzjANvP4}VMRp|eh86x6 z5gRGqb%o6`nzcViaN)(yK%=|cxK`VkuhI~R2Ck7B{yN>SRY>=~^w4uEj9I0dHSO-|`~r%%U_Y<J1Pm1Ine;~oYXC1Qb z!E>13?m>{}xSW~c)KQDr40AiQ3?E=%f_@1(-a$HzO?jo@XLDI&?)5aoKNYM8TAGNX zo4`!A(`fu8@+|q;0%gL(uQ?^GJ>{cBw>R>hp>^%0MFyvb;g@7x=)Q?q3-zk6j`_>; zdCfi@&xP7&91*bG*P660y;9Mzp-p(krdr1PvFjes_$gO?B6yItm?(#wU+_-dkg&l( zm#>28m9O@i^E1htb}r`5TNgZ(u~%;Bf)&eu0I(ep+S^B0}1hge9Cy~Zx0 z;|W~TBF8rQDJ9lBWAfobc9!adH!4BaoI1O z+FiD0@{C1=GNtTd*hG1`>pW+*H>a*3c!uq^-sboUftrGvI~wB;0~2snd!XBIg=mMW zZ8hrJ^DH4Lp)vI1=BLqf>NySP8W!JGbL6I9i;=8*g~G=ExF0?E5Lf((gS0lGQIO73 zCSFD>@rw|<>9u`NT)-|_{P(=_;@bcP#R?U3eP6M*)+O%+9<^zspq9sh&Bgn**xmY< zMghTs3qAWB^@F`5lYX|lgT4IBP92O)`UG{xyk6Y$O@DS11~E5%b4~&MWqp6n$-_;* zViB8`-KE1<-KgG(JDmHWH$#e#s6%@(T9(^VlkV2_)A2M^Z#L7dWas4llu_u5>zOf3 z@2b=E`ZONQ{P_G?%PNm3-2o^%#}<39BXV!m19H=CTboz?%4y-*q-WF7dQG#*eq$vowljS`_dWXO42|JHK*xATromBlw-Q z;C$Wvrnm$x>=zC8rU0fn>aM0nPX*!UtU^Q`WV;HuE@)0^s(|d;Hg&cQJ$V{DQV&@v z&amk{;^F+-RCH^8I8Jn_nDdEF$fI+({gkm$rOS(zgE*kVtmXVA81mu#s8onLQ(I@0 z+G$W=$5VMR{)^*JnS3hK%&x|49JtK5@K||$P2G8W)BcmJ9&bv<7wwBz9$YWJTp3U# zS=W;X*qO9DdZ5zjNsQJ?PB@FCUXraiJd~d4p0pZgnxE&>n((QTJqyLwd3g?Wy%h&q z2q;$Z^=(>2iITE%-v^o79em#0P72^`Khg5eT_W)uQW6cwYHbzTVCP~&M6MMo73h4) z=GiVqT7^2z-5ebA`k0-P8M2>33fb*${glq#7w;2Atr%<2Lw6W~2@}>BUioX=s5+bW z&v{<(D&V%cF93_%xjMFcHP zSKrA3$M+X2ch(+>`0oTL=_`mRJPcFx5`U{(m*g?(udn-6{X_Ej!h$~#&x&73z*suU zY*R?v)YUte-p62D^Vjm0d0Ii%eh0Ay55zxp4Z8b1d-S9|PGVWjxX7IcC1@X;A!L_~ zD))TQP_{WzF?8K6WYlIfq|line#CI?1AHC<7*4>DdS%U#zkoBh?62#aSpCbb$bbf%F_oTefiUI3h zqGaQgyEVh=$Pr!>&Qn^vwyPb`d@V+46@ zn7+nSKG=U#4C=R5C_cZw=``6Xmm8<(f4gA+y*od}`VISiHrZD@YgcTQ{e=baz51y- zILsql+32p>MMSMR5H7N*6r2qFLg`ZsBHP#g+x=O~Byn(DD`-5{ffUydF#8pF zEn)6sy*xf)k#ylRz1qSs+$ey5pnfZoY8vieRRSrQU#T9e$R-i{LHy~><)*viWd}PW zvua%EhwUq{{Eo3$@WxNedHZ4A+yvmWy(+Dtlpds~{u!>2&5p5>wJI6JEcNXvpr^ma zY!&g5&(9BR2eOc}RCzeJPkN{@U(FpY{GQi{CC8Yg?t+BpLzTQimbDWvf@E}8qlrqP z!xj9S5c?-#$h;4&bIF2ZHCX)`2QF#J2HF`DRY>!CXyDq2G;y!hkE`AWnWw=Ny~@>0 z>*c8d<2<&SgjRfa235)i@9@t#gc~0Z2jvEAJ%`XL->M;Yi{pdM{I#ccZ{R8^S@wLy ztyTVmpJ!PyO&vEngCuT(eb3WPVAHHa0Q- z+TY<`f;S8l#f~lC$NL&_QWI~+G^7s_vVIrSdZROU?K9I_4JzQcUeHGwG-{K)u?cOl z0m#Ljk3FGY&Pus|Cx)Rg*cA|tux5JzP~FUU%0Mk?z)*q1urWtGiPWYzWvfLF+j#$es1`Z9V@6H4m&{hgXxL>S$LVCVl*%qTIy*ux`o_BtoB-=Vn}jpca&XI3JI3M;*9;?zQc(g&_Jy)AOXJvA zJ1LhIyL-_GlVVM|VE^3xhhc}%OH%4ifz5rN$i0uLn2J~(tFje4_x6fQbJ86$DgYLQ zc>U0mNhqK6*2SUI4E!cim@hD(UpjT%+|i0dPwd=&twBx+FNfDe9p|HeeZg>B3qe zAM{IZFZR{58mfEFMEcG36p@n?0X#nHFW8+20{e^wJ9c#*aIRbql7TX6Br8l`cxf{~ z(b8o^OV-|hnzYvWsBcEYPuiR6IsV=XQN70?JKcEzq_-p7;$IWT(TD@hjwz;{$rE1R zT$f?#j9+WYu{^0M*doSz2*^^>*HKJ6@mtXHjio$0r^IAX_a*tY>!0r+*~rPLwak2X zZ`42x;{@Q}(u04+*8lCA4mrsa6~)B!jR&(84vE#`T{laFUy8OAdT%RrXR|2!Wlo~-~7rd5wc@g+iX}d<6QqvZ}mosc({cEIK-Xiqi1Wd@kR=@^R}I4 z5)7K7V-R*f9cslKM!^*v6u@52V$#U!k8YWxftwnDAbMZD6vSm0muCNFa)cD11OJ9^PN0>eqS{#x3lMh`YH9_)GrV~V-5pw zSULiTI#~=^N%I!ApE7%m=;!A11WSb(&&SOsg@PNWvi+LX1iEhG1k&1j(WSy-l(k9iLXQ=T8HOe3= zI!Dn;YRG9Cv{HLYYo+1+6aL%Nr7T3pCx=etE7n%3D`sT~%Fh^8p1D}d3^2J@*c!$W z6`!GIT&fzYv){QL=8<^aPIm=xRz{TYOw5DePu?`2^5eeC-*J$+T7lR|7z8m4CD~(> zcwv5dD4CmeNxNl#jhf4&b053V8UbwFy+fi7>`LdH86-b>S zbu~myh|pEF{AFB;vKZ6JOKjTekZZ@^GelXJE?UZf{dF+PYsun5Y3EX3quP&Kt>SuQ zWp4Gcu4e27Qx!{s%e?a$Ew8afx8&@EnRm9QeHy5;SFdYXe$)E>`$mHuqknMGHT;x}5i-fEp_t=8Y_c^YME{=Th-`**t~M!~m- zRm=vc-X&`McY)dXg!}rmd{L!g*tlo3e%$A5 zYuB6AK(m9thv-u%50fqoOkZ38nWX}A@oF2 z?ap6e>QN}T{8N8MY4gCjuMxf|+I!(8+&PpL$0kUxwmdBOZV(^6*A*LK`f*iouys1= zg_8_Nu12*N&acC}L9Ny?QR(nU!0f9(5?>4k`ka!`BSPyRQctU*GHUh6iqadpDVc#o z)Z|tn7k#i+4i<2{rf9`oWhJkrtdn8i!J)9!s1HmQ8*+aQnkiiQi<2BJ_T@8=;N|3? zQEg_h{-i@HGe9EF4q&SwiSL&62nq6XDd_Ln89bY8rjH_0z#~o>K5wr5fo`Z!q+BfsT)nVyS`Fbjo^p^Bl)Uo|s4!tL1Q*x6VmJRN;@ zIY6FU=m3}T{^J`*Wt)0Yy_aD!x>IpHUGAX<#x6+tDM)keDfzsaF0U;QaA(eR-;8@M zBq_tc(Idfd-w;0@6u;2bs=xnE1*J!_YOcC9`956^B!(I6Hs&|1p%$^2t(h0@p<{j) z9j03|om0g0dZ1bDSYK9zuetKDTQ>fTtj-xgGcdr+x3>NI(!Jx<5>Ks=%3ZaXjI0S2 zPUyAVu&kNUmPu1WlgeecW-Cn7d2WO(E=&r=kmwXEO{NA6QS*LypEEbh$ad z`NV)CKe&uKmc{zj>k13S_LnclyEb!L*m)b)%2l7PvKwIW(>Y|N%+0|}X2uGRyK`3d zi5T>%P!4+qgOb05+KeR}A=@=nX992GuIdwOkBe6yv2{ve^4HC>-7K-3w&O)l6k57% zLrSE3bXp9e@8)b6kg9jd$1hK{PwFA2q=B$sbqqF2yldU%yvHRLexNP)mWuF{r??ph zuG|@TGFYj+4^&?v3Ux-xA`SNMKf>yr!3umLspuWw50zIJSQR1uHarhJoDH$Jnmd(r zEPVXiu}imyAvApVOB2x!5_vh}(VkSJJ#a4X$dP4!Bj3cgX~cJ1g-Xvh@it*A!I|9P zd{ife$q5PuX|JZq-a#g)p_*QG=;HeqWF{@q69NSbE2ceCfp# zQntUXDI1QX{k)W<3v`c#TvAXBY!Qjn05>OVsw4M_(_OQgGX~QF5?@O$y^j))@Ui0B z5DUIQ<`4R?5N+w8Y=K=%HdqZyyr!1%Zf#?}khZ=E>eDEqFrIHLyJe9$ABNSy9z5$S ztp^dRa)ErDguYEgYt@5|tap*$HSW9@vsS$2CckPqv4K&S6rtO-avcPNd0oG#}SQXQ;e+pT^0wlTdbZ?bw z+XkR^kb$LgsLQiYpYzEpr%`AK*uy2~*hrd{)NYK(4JLinYYCAP9k!><7!mDz* z&)eTC^i*4&e6(Sc|3b^k0m;X?sV>Z7yj*Xt6IHRjRKeRRhzl{XMq2w#QltE)T*C#V z=ZDX4_3C8YBquaBWDl4L2>XQkQavHp$^{jDlqXa_iq){)9X#yz3#}`(k5}~YcdJ{t zBHKqD$p`NeZxq(8Z&j>uwE9Wpye&&i!!3Zm$dK%k=jvlhB}a%tgqPj6gzfKSb*8uR ztol*avnfZvapGoynAwqFijLCvww%UXKS6LKHh)Ag>i7t$%9l%{w=lmN_MkxB2)5D! zq;F1xB$=#+NetKcYM6|aTPxKDSNY9!c@1M!)X&p%1>K&Ejap+wzX*(e(cBeldt6gD zJ1}jhB!@#r{h?z{+X_H%AG~I$*=^*xy~1YC`o4c``tDTHQ{1v^=rb+l7_4c9#+4Sf zfZ8{o)M<^(T7GXthhqgok42BKu?ju5jUiknMr*&i;0CM4M(iNnVJ`B=H}@pO!NQ1| z5^Tx=frCp&Az4W3>Ta_a-d>4(JzYEUMU2#VY5q#ejaP2B9NpHYYj}|U2Yo4FBOq>@ zh$L*$u4Dt$e`b36$^%h{7Y1RqDTUxjY_i<;9A_S7+-|%Sc+gbvhF_BmVRr{SYz9@WgxbF&)sAe=XV@yu;bpexMJYz% zvr_U00Twq!Av7{3Mb?%J-9ggL%|BQ1HdvEnTQC(_vU)}(?}sfh#It;$l*QteXH{1puvs)+r#psGZ%xpT3vdAMvw$ zGYvN`CdSFp4C-NGp4}We7ryg21G07G7{ei=v{lK-s$UMW3Z)(ko8Pz|rLbig2Dsn) z{f*U;H=H@f_ngZXy5+gW93bvOOh|t0iiTuV!Xx!L&on+5>fu=Q*agtA^-j-Isc*f< zcEQxa8)R1J@8Z8l_Fwv@Ve$%J0rKE2PCbP!RW+SbE&k?NU{BaTvz^4mV4-gMRoYyH zlkMWS<|&P6ZB`k+9MzqB#>MVELOo|QDW21=x9P1<${#t0&-Z^v0zY2{tCcq7?=E^9 z+JxINfYptCYe|XTEEzxh&!lZ$!t(U|9lMTQ~pXQ2u7`zBtL*kQwf9el%3!Y-@dm%@RbAQnPiK>*WQM;l2DEHvTN-W&IqY{M_QJ)TX*YJ}-Cx&z;UqG24 z?6^-P#0o8(APjGb@%&?JCKM5%R>fxq8~SuuJepH?O>}{L|8v*|JzO@SQ^Q^s}>k zGWK0{y(0zOD)>>t>N;R+n-s2Ym6l?B*z1@0Hs<~$q1gnpt0_EVy&b9WanO-g`FvjCAO*&GW_U-f zuz88lU`N4vUKBe(R(8#|L){&T%Kn;-#x|Q-U9c&9%1eist!3W^g(0Ye;R_ zls(Rc*?WlGf;G)mqS!Ct`Nsu!JPqJe5Kx#vUmtoptS}(3Wpaz>Jyr-_}e6lgK5p(#cv%jZ!gLN3-i9)!7tU_=YKm)v~Ci%&osJ*ca zoKiek`j{`uY|rff;GDh}@w&O?WG62!uQbOS4(px)a&vQcFAmw(B<6pB$EZekJkrxP9PSHE6X=m_M-#Kr$}sY zkSdPB-4CG_OJ^SZ26jg{RNyUO?rp>VyS7mHdN+_Cw#k`C8oW~=>0>Qv@^>r%>`#;y zBj5g$jsq3KGZ~hMt8(md)8Qt!&c>c;+_u&={%Ze>OGWdhD>LrytCfUk3I*SgyVK+D z59_~=nUYf>NTLeV@47Wqx7E!kBF-%&>J=tmjZ6uhn3tEH;*NBaPyYBs2CN%ebRKWh z3VDA4rZbL>z`p9hjX{*dWOO&~{}#!eJH41wq*@tCVv}P}xnqwsk(@yK(;KuN7-dmr zgzOSif_0{pU>gW{lWPk9$}EU7Gbl~Tzc7>r7qPZ~5B5>+&Xt14g)YB`4DQOKGb*fr zQpdI_iZccI$sy{RvvyCvEo?ZN2a(FvZlliPta-9Y;1n;cwo1}<{4fMuJvIb_xcMb+ z%RPsB^CG=KR)<25GMtrXQ8~U4;WWEK;N%J`d>Yammw#nDE0@~Ghke;O+oxavPo7uO z$sK0e5-;R+!xaI6P$7D%gOaB&WYiMYP7p2lDeDxv92qZe>hqfN`VRkV;wQ!9MB+lT ztMmk#bC~iL%}b4CNB+l_+uG<)#)SW&I(n*nHI+CtjLe7&rc*Pk-bl=xm2HhJCo?{t=!`af95z$S_`RR) z;!;FpWAY)Aw_e||M@kLwxodscSF-i-_r~SZ{)^WJ85rct5V_o&3TZNuhG&tA8WKB? z#6HJ|A4a;yG|-k#`FUA7=o#F~tH}G1_n9`2fJ2toBQ$}-*O=MvcK4of5@Wcf?7TUtz@7fdbw}SRoe%6L-X}hA%G#niTj#Gr`;Fj)Wr+~T|M~tUY#+2n~mixoJB6)Rq7LQK1j1|Gh ziqMT1sUmtI1-P*rQnl+<{(;vn6LF)Lz~bKO+o&Mz#+u2nnKIM`S(9B%e<4NB06YX6 z3xBALWYQY0=%2BwIs6JM_Mdya^oAefPyA8BpGhH|fsITz1MB8XS&qs4lN{ocsJO0) z@HW-dk(5eh2W}RCW!S@W_XclvxG6l5oD{*6(Ea<^m8s2* z{HDpVHU$4z`w{jJx;bzDaYy9nT5$`4R85kpB%@!Az`ck&U{E6Wh0o_Gfr*$)N)_A! zak7;vwIYvfFtydY*V?|Ug^3gYRrrD5<3+{JJhpZWQ#s7u%u8!N^?nwbT-X|BHjiQub0pSV}h`s#n;jI>IGTX=} z(t4OLZsOk#IOzPT)^8pwf9n+tym?z?# z;61xj=!fbeeF8;i$Wl&9uhh8&zy45?ise$$qduL~OxH`MWd!hZ{ya1fpO8B(TBUv^ z`92_iG@>Y!=alC*)s@RUbb1S5gQjErzsM*`-k3-7;Tlu$u|u`%_FNEB8e~{bGU<$i zuWCI4KVci}2DF+X6+vrx_b&g46&B(LYu=1%Z)-5&!c8z7aD=5L?SeoNmscv=Ax^4c zM&agJbl8ZqPb|EQ3Tx0FkiA@4)7%-~@S)`yiYZ!26995%ps3U6sKFHak%2N@X+ge9 z-muG?rgw%+xLF}|*_neUiH>p1oW%G9a2=FetuvV3#!T<|fk8&kd)v3S4QTQ80k|jC zxVMB&#^wfAL+bHbg`tYY3nBJ@#85uKqnuByvXK|9Sa{fXl}$VCjLypE4&gn_B zDCXW?MR(VuG749lhJZYKl`@?>UAV5Yg{wY;t0N5sYcRjE*ZRLdRid)#h^V5`zhGZ+Sl!n3I=Ug zEf|l(-}`?jTV*jm9z{H_X|seusvMrQL&RduOA% z9&@3BTl?S|tH>3jsUU|K`;0^31W&a&O>ONLfGH5dX4}*`7_z}_g=3tLZJsbkSH#H5 za^8U`U^)k#s<5MWVIL?i4cOx)2vO-j%wD?v3)esW?Aq7lxnzk?sU5@DN3IXp4s(q? zM3WCTpWwP*x+=tM-BCR2B+eiaBlGo)*7db=2i`M9GiXgNo-cXHJZb9EkbqJK=+ zCJ)>Ya4580Y{hK#ox^Q<9J113s$Df>%|9({@{EjIMwi?{rkGem=t_mfsI#2UDtqkF zD&v|mcr7r&Z{kuwkbjztW|l5tvkeWY5|$Z#A?LM3TG0QNyiplyF{239vSY^g7x>|j zT45PoT~PiPV*VQ}^Hyf{F`lQ*!yYJUKLO$#QxskK(d-`mx|WPBrO5zUs@ZM|e$s#_ zb)}9=_JJ1rGf~AKUeC=b#pr2oqBBcSf8p!<)Tql+wXemLW@_N{1|e6`RaI^MDU7FC z9wNSM@1F@8%I+Newhh2FrG`~E@b;0EzA~6ES6J{Q&CsKm%6KSn*MtFFARU^1!KQo`C@OGzt0=i4I6B>T0rZ$rOJ?-({DC| z56>WGobt@eLh+v<$1BiLh%4t(>ee>Qwo0+G(7)##)A`AV=%;0>W~h^F=oh+_7lR=t zywlGMfk09gr~f=e>QNd$y0KcW3+eStMG9m9UIT1}C_S9Xser<3Fm-W^jtvt$_?RM= z)m;dF(0dIA`ykG-B!op~e zuyC-I|%$8k1Q}4 zC^z3lKo9!Ha93c6y#uFcd}i{`W7BI_P6gH#ue&y&T=a>lYs@Ttp(29A9QZ)(+zMOOZ8Iiy05W;c*HzW=`S@<`Xx*%hoiC6<~ zo|k^qxiXJ}euY$@T~dv1a@&`W2Kj3K-Y@M9MsC($X&pY?X3BYz&)TlaX*hgF4qex} zsZukH^GrmK8g!wrgAdWe=s&g=U9n0!8HdrjYy!|MFq+2PTVDYnyWeaAbk%}z8j z)QZO-Ch+~@2x{26LhI(nhrL!1rjW9+Bj0yHBBT?pS9(L&oD)U!x4<)fr>mw%~{`_=q^t!ayg4aE7VV_m0QRmh*>`^1Xtd5eg!Wo78 z3VaH2DtfqmgHo+rvP)(UF%|V=DHvAg3ax5R()m} z55@>RwBr&!BxTR9w9=4Y`iR+$P;mp|w#mXh?bdn=5C$%LL~hcX_UcEc;Bha?u8Y;+ zg@?}BxgquC0w6pq5Ra_s@vpBloN0JrBCOk#F9H8TWptx9nR3PXw!;qw(z~|$OG-So z-5<`J-+0Fh=y<=S3Tbei9h)oT52M#IN`6(IP)n03Yw^!$ZhMq|E9<~x#d{-wzkd6U zvkJYm{_Oy59=75t6>1F~B)_vJw&8MHn!)vE#mYf>f8;3G399_=%&tkZTkAm#ki zV`i}Px6#*;+av7l?fjJp@8$S=L^qVj zOcAud8&3Le>O2z^kpzyRVw>Sn8iyeSf5a1OqOTE5@u+;yHymBxyoqmCme(bqQl`Hk zWhp%#&Mr{Wc1CWn4bl`}qf9}1Tg5nq7vQf&xh25<#3 z3nI{#FsG~K$e@uUvltz-ZmPm4F?ie=BV90Wl)CRx&R~qGkiT=hQHOF$={#~jCjz|6 zBF>Iy^*9RrK9MW#F0&XLyTdoU?@|?B5j0La`flxHfH=15ba>ZeabWl%MqUNCbwY35 z6HLxVVG1puu2AJMosP?gF;1J8Vnt4;9ad*<8*Ib9{jr&tqm%JYR=`_i1b=W4VU5)H zIKz)^T)u|7)ct0cWKA8v-H?65=*yu8+}A(6lk&Zw5>p?OdFfv4zUet@K1p}lW;mo~ zw_t=CL)++!e{N;C9M4Y{S)p1Za#3U5t$~zz70?*-WG;s=+gO|{+3z}Ngt@}Po}m?aQJhW zq0id6R54S_$8JIib#ddpEmxjxcht=<5ki%C{7YV_SZ?4W-ks#h&#K@m)F&LaJUbKW zw0Q`5e4VB~-|?|>FV3n4xN*>1QGw5#4Rb5cfHe<&tIj~=V*9@hU%t%DtzlaH<(ZJ5 zzxv%}xzEO7L$RrRXo#1kHwsdS;!N>&~#XCGeVU9_;S=C<+ctgs@loi!1zm@g>P zY&{qg4=K|$fMKdaMW3!Z{ZqyAuf!GRbDmpXOGM8vVEI<3MwzviG50izLX84DNMutZ z-z;-_WN|=tL}6eD;OBV*t1e3Pfw9MVp7d$)&hp;p;P(7Zs&#&QJTEBl^@^!T5VnD< z#Dx58luOFHJMPiuEHoK-`r?t-)n>J@O9Wow&OG6cw4zvTLgWuH1B{E$x)4BMoCoG}V~XV1YYpfg*j7xGM7!SFRxVSDDL$o>CFB5oS?2}6oLMt2 zuxK=Q#wlN0ar47!N?1bXevx$e)AqV%GTMC*g_+VDhj`@4qkzeG4jsa@W=3GW5Q0Zt zV#j2N*hA|}Z=6it&bGVc;w1w@F%R3ZHvEP8dE3qQLyqVi)r+>~jh7zej#z1_jv!CP z5cVo>cm%T&S1Bwy+NO31NL}sSV2ztQp9&QWJ-k?n(c-9O9-1=e8DXa19!WOe^7g_EP5HEEh+dL*ZHV#019; zib}>lLxj|2-P+qD0n#71s*tL?CJRX>Yx#{BM>pVD7*M{ZngCKImd+9)L8Jvp{+0V5^V+Rk{ta*bR+;wCc_~gOPv|cjs8)frC!fsonC8s^ zpR7%S499G>6#U-9x(`8MB*+lOVW_!>eOw4zd@~HyVBM%G$XlbwV1r1KyK{8V8KsX3 z1#MR9%=vA4VIG;d?Qt@;KA(=icWC?!lNjo{x4!lqJ#&0;rUJe79Nm<^TSj>`KjNoE zp8bIC-Cn?Km(H`XRv%-7J%7%#!#k{?$-z*o3yt~0X-TLVj$PLEiJM%hAobCp=HiYH z>(8kvIwUX|21H!YaEt`HJHbk0xk=YbcYmO)Fe&E3xlrZ_|0saqAuF!K5b7=H z*}D3TeKF(5@{3FXmLz|n;QEJK(eP?3+OO2{rC9__E|-gCKK|b zL$t%b}+hg%D9XlE9vSkCy^fQ%>%&**L&Wq_< z|Go)@o;lrj!E7EtrGuwBr6BgV0vT$>?#M!GFveeG8f|$X9GNR2U-Q?h zuzy|#0=3zvV|8LpVqYvsF2)jvI5wbmLZD0WD*6a`o`)y16P-|8d87l ztq=lY6sLtxSx zNQa@K+sX9$oUc1BH6|1rne33G0|ZURJ!d{BOyjoB->^mQ{wmKA{e=5FD5&+N6FOn? zSz4PFDMI`_A(e~i#F6w|T?516!jo@Q=fAIdQyag49Bc>A{XG}o#8RAq%0OS7fQXI>VW8Xfa$>n6+_SjS93f3P2yk~6}Dz*CNL{wi~s2{D|*QpZw3rNB=SO^RA$0v6aKY-3LcaH5>lc z4wJ9dGY%!{4ODXx$se-|%eQ?tOq`0ph1kgbjq)iGR@>3G(+%N&R*irAKmP;X<3D?p z6!0XZIXwM;Hl_cd<-ek_R`eyK&(!Ihd9{BEvHgW}|M{a07ky@y;XwU=WKaKNq5StZ zX6g5$6kJaZZ~f2Y{e|q>MEcC3^5;+g&m8*r@Y?6RyC)p~hNJf1llPZVGfnzT2jhCl zKlx(*bDsQ}hf~7z@A>w(hLHa=wUR>gnYSdKUj9>l%>Ra6uD^<<_5PSq`9D*e#YUgW zl_|;eU(5erIP>?%ziu`E8#@2(ouOa*aH3Auzsd4{zsCPG`swXy!QrPIHvWHemj0hs zpu?}3nj%C0n=kmMcWm@dvaNnRceDLh`Q!iT`gDsvQ?ew@;eXGWe}D5|;`u+V#(#wB{}RvtX*c_q_WZME|C3($+t9zX=l{!-KmFzZKicyX{O+M@i9e0r zM9z_)j^>7=9AXN2fuVuNrd>Xx3Qn+a8g)hS=+SvFh3hObUWP_CH+o>;hWr?(e7~vkk~Y$d z?wSVw_T$|lCkAF7z@-zAk zg+m}FfakXU$M)|LOpYK6uih(%IKsof9UuQ$nLK&(2RSzY6BaOs=2r>!ox?Ps@?HgT z9TL~_0~T_iZ>CDNIYRV5DIS7XPf52fr>W$Qd8PcIehJIQq=l#z`i!Xr3?mYwoqnip zDU{){3r{+|;QQZnwUqLUOfZel!ouCHHj?BQu24|STSH!5PE}L^7Rq50%8L8&h=8fY zpjUl$=EFU2IkwZTp!zCEqJK3PKfTfXz8sukAp6$+{QMD+-X)@e4$Q~WkZlbKGW*f@ zbMOW!ucKu$(# zs$~BSkH%05I5Ab?!zcx`7UY>*>omf^n09x-#b~%xsH+J#;df?e)Lon=yQc?ka^o+ntBW3Q_BXb_$;F z(bLuIj4^XgY?dts%K3ahX2F!EG5Zu1x!E=Ps^npbXEn|VCoUYq<;kn^h*tDRgG~yl z=ndj{MW(^}c!|?KStq&%K_$(cVo|=iHFS__vIky8gjxlyWX_C12n}CT@K6jUoJJ|Z zg#1kWvgWH#8Y@WwDw4)@WjgPy9w$2O)nF)F0Zu3jpOJNo&gsAVDgI$1>(U(P;W*bK#Ry{7WruR zO^SdZO-g8rV(8Kns`TDbnuv;k^xk^{(xo>kq4!WjNZ=Ruz5ATI-*=yLf9L*}@=3mH z%~~^S=9yW~=mF??18@ZGugY*M;=R%>ar%r}_fx|S$qy^v7U}O!X7ZL_J=frujQI8& zay&R=pez5-4|_Tl{)SMWp9i8sAR+#0WstBJaE*d&)$omqq(?;Cg| zOXw_jKim+I;n_oUNK%K>=+O#zEF3iMlqQ20`holuMhcB<9d$k+aCe2_Wp3ME%?{wEuF>k2piVE5)#69<$kB7!ud@B_H8W*Ph2^fQW zRPdh>D^YFu3DAr1o6Yk!e(F=;$%@`LG7g%uX zzlSK~N(Rm5k9`;su}FtF>ns&-w#i%DRrbCwFW`R4xZr=eXB(WZcVNd4zxSB!>;sJl zrZVP^dWtYNch)}umH+cpU3$m!=!@7>4jAq7G_)<;LuDqxXpmq!0R3UTX2`jd^m8yA z5C9>GnD?Wl{2=NU?>(x@RFaIn+jfxhrtX=j+*+(6=}pPZR~~fQ(reM(FrXT9_R&)Y zg1-P8o_~M<{3)q`dsm0!^l;At|JWo_P8}cJ)(F9_WvHxxZj4V2!*XJQ6WY+Y6PT;1 z>-7fkOi3F;yWu^NVfR@}?)0hLWlPni*&=$Xhn-Cf7xGiFNydXA#>w9M1@C-0!RdfFo-hM>`3ObM_S_tyC2H z@D`r(>D)7sjn%(_Km|8(Kur-lFwt5+^w{FNUG7UNFK!xoMYbdM$d&Qeumw|V{C**) z_wGgkx)~py&>2m2Pj016LlLo3LnP7rt(W`H1Z)VoLz5XMz6g9Xl5g>96jw;yl^?@# z+ss65Dwj_NKgL>o_oV%bn`DkSKTvf$Bh|m&?B|T=zizgr=bTnoFnOA1F%b_rJ{}70 zk6}9s4TAQj7Q)6Huh!pHo$vf^)4%=~V}}cFBE$;Nc+a9X1^GQ8e)wA3Q>!S)Be+{x z4A;@2DaBK83s82JO1)l#TdIGO_Oe%fw$6Fc`*T!Aw`5LfrM5&LuV6d`_+}!Q1(#Tb zyIoX1lOcS_c62ni|2_qi^2VgWZ;axHz>!VUB3S=H}Zi}aea_nZ43vPW74sq%9us2s0rczSFdy;qYbi+;y(WPe=^>c@?^ ze+(|UtLGn2=_8^wc7(wtZx;>MW`-I?s#xAF%q(<1INP?(3GAL!VBTz+Y&q;C&yRB~ zb`=f)SNPzn1>tp9Wo$c*x}Gv;9qqCjm)h|>+h5|f7Rpvd`}*lW<}fpis<$+PddGY* z5G_aJy4<$9+u_k?h2iMU>ulXAYSuNuG5Y#s*Gmr-PPetnHw*y`qa9t)SH;F%6* zHnz)mP*3;VS0K-`aCT@wiSW7e{(`U16qLDc6)?BVt>}s1EzE>W3}ZhpX7Zh9C~TKm zf?YsEmi=;lQ_wq>P{4MM{&r(USL?PdJL~k!`Ha6-_Hg9|>)bp41!v5?Q+a8#>8KgY zufqyh@U+vAe(TS}Sfz8KQ%R0-Eq30sS*q)OrrkY1=eg;CS&{7$=RzEpnJy}A@fG&* z1-b;-)RMAL`*Z9HMzg=0QJcRfeBS=$z^RP>q0@AsbD`Nefuskhy-b*I}Q8`tOE7e~PKO^?=I<4)3I`#-(| zyg0qt?VEGfQ$`FjK7j24wHKx+h=#@n|{h&b$+}0M;v&?9PmJ z`Z(i(<8))s9r?xK1lbb>U0g2E3xKyWUtLzhy>vys*Y(k?;auE&TaaUTx~Cq9{yo@O z=YFLQZ|+d>rO}i<(yxN;LgVauG~WNg&!al@YaYm-hMTeeU>W{%oaZvF1Fe>7Dz7M! zv?|6n4Q+qNsy?f{e)m}$AZ7Y#KS%okW5n0hG3bo|+UOI2rzgj*>FyM`)2QvfXLZNh zvKGnN-66SyEF4ehKxxa_EP|WGM|;c&oBnjn^$XDB{_+>LwXJh>9y$}EpW`+Gf?l5# zD6${;#ZjZ598X>;USABWD`v#mUf9}Q{yFy`RjKmmMuizHCQfdjxhzv6Zm(`v+Qw|u zc{jSL_N`ouJl4W&saSqOeFjZfYI6055F>e_i6uPFz!7luQxD)N?N4))y|hyC2d+Iy zO@V*@9=pJ)AN?9Qo8NPG3U$A*$GmKM^N)WJv~`yurDY4lhtqWsklP2H6XSCM!&jqmdvYR&4}kA_$6}&&mv^KY z{UZxHet6#&QPaU)fJXqG)OEZOj_-O7Vsy_mRCn==`QXu_t-=&oQ27+<{iE$taY;hQ&4ZF%;L=!i{9LCm1}=;4SuBufRs6*1V}@fgg8;(_m` zf!uM5{8q0qN)|wuX`_rZv^Dztoas)$zl!$}&GZNfuXw=+Y9#$5rS1f-FU7%(Bs52N z%JoSFRmW+}?F^JYC0I}o5xu@eoklCX!CO5ZV$VC!<yWhY+yK5lm0gYDT@Rtv&c zo7)-g9N)U)bny9!Ur0op$f6un-1P*w4b5=)HSPlKQOY-K@v!ci-BDY5*s{*#=EZi5j#iv=0J#d@iUk*-R?I?#pE@ z!-s;zW&)!*)*C!Ikr7jux-|N#J>B)$)%3F#)+corm8sK3aePlnWdmNuvz->ROYV*mggu9BTt_JgJ|X3iToXN15bFYt3$SQP zxd&5^#NQL@1q<7DeGn1juS1YE$->LFNDPj<9~$6MoV^Pq-JKy2**>~A-86ZzYeMwc z{&Yj+`PKhYs{homaxR+o8HX;pLjnkuXnWfshj`joGKJvGH*)08#j7N2H>tb4T$r&d z)~lVNdwiYJ;Z;j%QSSZ}+%h-dM=jExJzuJM^6PJKs2;3DN;8XSfTWk+!{cjGV_C27 z<%E?n>2_6A!<$Z{ZI|D`dMsS!%}2eZMYX#rH@DcF!gEJchfWfCgEF_=o27NVBz_Tx zLvav?-1QiB9H>5~&SF!})K$xTlG<6&rxO_kXBoDyjJk(%Dy}EDKtA;hq?PFfRm8tp zk$U&iR-8iOwE-k!VQVmFmq=*3)1`(CdZBj#gvKcAq9dBVE32WR$|~e$tc;P7=ig%B ziJ92+9VXanGNj6X(=IN_kd_s3nd$IW>jLYH3O(B3(9uRzEsoPA)8hvhU;Ble87! zQDg!`h4N>br(g6F9$R2#oRn7*^;U-nTs9zpGelEkZ{2W2Lq*gG+C1{kAANIRvrQnl z@gVNZet1AshJwlN2As9fZ_CAjxznFG z$#>VOL#zq?-gEe?{t7RW{szb#p6&QR`BTU4-iK(t9>GiRqpI)**)_2rSGVQUyNfEJMMv5@L%Qm?;X%yLb z165HO`)*&qGb+6i_N8tj6k{{aCO?879o!>?+~9g;afl)P&lBw*%4fB}y>!*$Eie=q zK6}`~G#8(LNK4(4F4!~Yl4Rmz zx#E$0FTjx7P0ROc$rSw~BU;|6@Lmr82kWc?22tUmViEuE&gduG?eH)RdHfjlZw*mf z2hobp<)RDWe%J(ckP&0&Ma`w*xYg>?8K)gsx{^fo9z*+?X5+7 zQ6c&Zk)f{hX{WEQcqaGYqDxv-CXswmUayC*>VX$7(95K3krgo6hn~FZWp^>7lctZy zuzAFy%jI$I_58~iQPf z^?omA@Mb9#{6f5+O7riQZJDP2GrcJNkV!DKi{-S^yv?P3tHgxm^hR^3Cpuy;V&0QxBv7*`Z}&b` zfNBUnnkIaOA=6ZRvi4;qOTmHd6jJvl+T2JY{1b!Q69tKcDd|-ey+p@K&KkR8EgIiq zWl6Nruh_saP$j9#M2Ii1ffyNwDzMKW=oYHOfaX1f49Kn?Hlt;A(`YCeM{I`H2EhBc zGkEm$JLs@a*tkSlnTfxh)J#z~B_%3O7u(8t`2N;ewSK#rlZ^`PGzP;$>9x1Cb@wL9 z@N}HT&cf#(Qpwz^lvWT1o5~u=qmv?`LV@zlCvUbY`;9jp~_*AbRWG0g_4FhRZcXY{`--4=^+{0>19nOImABBzS_tNe+oy`H-H+hb zPSZ!ei~;NEtFJCo{y32TdDhVoO5Vhmj=l7T;8DAQ;&qpqh+j0n@Aat9ynIJ^beTWPq{SVjZj#V*mvrMU$dpDyqEIU{*;MwAWMZ1?A)lz|pi@f< z#GQ2{jU~MPJWhCuts(SXA21MFj>FP=k~Us~3*+L1Tm=~YeR0NSc0|Z3V(#Rje-E=Y z3~nDw@H_^5w=K}CN7cpKq=KBqI^^K^##MLQQkSJ2$I2=Y6jI_?vK}jrV*KG?$s8ZE zLgn&aJm9lm564fY>~Pl$`Ro)8BkEtIUnrNpn5}&HvL~ z|08Wyrl3CE7O7Gl11-yLBP;wZC6Yvi_v)X+0)rlrt%SxHJ=4|0qor?W&MB5XBccEN z>h$9uytPGCrP$vZF~#$+7B9-5P!>k+c4zwWH_+8ldVbT!K8NF6V+p;i23;^d-D?`R zRjTDbc@eojR=RnlKhtHLIrsb3N(mwejcU*>4ML7_p5GB)KlM8YT+UsUx6F2Ex8wDx zbk{Ka;EZ32WRMm#B^?b-wMuPgge-r~mh5e*;3XNCsqC6;npuAPLsKu~OL}eaol|nb zGts2CJ#=l;HU!vmT5}LC_rkGym`!14s6K~$d{e=8EWI`ME1!kOjL=a2Ai4w3R!RxP!c1z>kdh2+PRU6hP75AdlD zix{#PkJJ@x9&%rCT8Ehjb74%{iVZir;iyGLOqb#b(u(W#3E&zDSkT!e<0s-Y}<~vg)9u|J!_1k`jcNHv;$1_|ebS_OdxGt7D_jgGN;8v!xzQ(eO+(L>M*n@z8 zWhVPlp05Ck69R(Fw`p&8b#VuE&XS4|^w6+%l_{;XhGCrkLB0Sedm{HORXZ=={|?~( zL;qrf@N$-vBwHgkmZXf%PI<_I9U<ESBuVg2dKJo-4nInRd8cX?-p&E|pkot*HoE#nTB=*z84sLO<= zXCV3S0_utf@R@h|9bL;oJ4tPH^xZEv)i})c<&zR3RL?Wy zbA<9pQ z(G?l-kdH0hhGh1w1oW-PO^eu4+XvOu<`>m1jc*@1FTuSo?q%`Hv801K(ua@b#5KJu zi!aD^b9Ij#28oc$C*>nXkHd!}swu-}6F>Nqh^_3%UG##fS#h!14A07GF{>?SzTr}| z@Xnm;#Z{f~i(N<+~C^ zov7+34nf9G_N(HC*)b*D+TN_h26$zcGXdnjznBQt_kUJDO`ak>e1-`$4;26~fjua# z=6+4o1YI+J3r)q+i28PfP@X*XGKpbQ(1>d?D}VlY-h9qD@F$xXRj#lcs#Kv(QYs74 zww5bqF$A=Oe@tFaf0Xo=)4W%Uj$3+}SC))Y+Fgpsg^khXUN;=ez+m*)roA=REz`}z@kAsrzohADrhTKx$6~j3% zFu<|Z*L_28mN6cdX`M$|D#G&F`}UBQnm84N!4oDGM_E@i9g#t%6XivP6f6xrk5bwy zPI7P=LL*g-E3rW(tuy#i-w5LPom4m=v?41M=+diV>=*g0n$fAqWC67%&;4zvtd6BC z!$!=8pW`YE2jHL$g{(%hDuhx;vmSDtS>DkhHj@;uZLB|VNLg9j)qKU)wDN-P2bnJ7 zh(O{9{;LXwsz%mF5ybRcB4RE>D`HGtb-eG!9cP9E{Od0R4=?(P`xKE=EKwjxcIkyvJ9@f`0bgMN$9Qs$%shJaVj z$3lNsnAm?P3Lglurk_^#ug6z zboPnGx`u1w2?w2|a4IhsbDl1*At-K#74~Y^H?x@wIVV_*~%~vSFE?z|AGgv zvlRa=_$58293A=eQ)Dcvz*FoKQv=Z)Mxyw)yCtgqX`3PSJ`5499Pr>9XM`)SC&^?& z4V|`?x%;b!V!xyo2F#SS&-77u<%z6%HKqK{^1Ad@**&(Xg{BChhie3mL_!KY9sQ_n zG?iSg<7@{-D~w%qmxl&@s7!sjBzPU1IOsvD7QNj44P(YG@mjGT_$B`>N-xt0Di*(-m`%;BJ zLVBE^*zj_ABMeacfAF~rc7-s%6*EINGp*l7h?s*uA)P>il|Jv!!_s%1!W4abbpi9w zJ8+^N57M217}a?{pelBrCk2V-7)9P(kwgo%MqC@Q{dC|fzYC0c$$I4B_WEs>IsjUI zEH{6vfS20oHAr=C#iz$(Pe=!!AXh8LLF!Ezhjz?$2mteG&KH$`H&S)AKn}Qke>FZC zU4r17!fpBKWnz7dII9W_p^$92qG?62HpETTIp& zyGoA#9M3L8ESQbU0RK@$DtbFE-8XQTt1=ux_Ia(!R?>q|OfW)RN3BMwR^9}_A1`f* z3%wkFtO5M7ghg0QN3r9@8-ofthA|p-{hAli~~dws1Y-o$BO75xDaT= z{_gD5H|VLlLN1(qjwF~gh4TWR?W7zTuxF)Xq#>k0u01^shgt4^q3)4x)jIsX!~{M0 zszVex9-8VNOro6KAHXNdk`5?)DQ>x?`mHh&fo~?M@JPy6yVKnqR-~(FW^H$XUdjm6 ziZ$N22(8`if?FXjyda-q0F`pJL_+PE*^?XlG-ufa8u)L*f1W_%C4EW7{dUrB z(eBqjo&-rPdDe&cU9VV6Q;O6YH9OL|)EC&1_3;=1O2T(rqg4}Wb%i*-0(_VL_P{WvkW#@&@lj`9l=`y6scB}X zm-hhj7d7<*Dwcv}TfSWM?5Hk)?iP53w%nCX67|J*ENu%kutWuZU&$AMeG`R{Btg2d=5mU48H6z@K_rZdec;yCefIe%;y*5v&SRj47VOh zFbVdiJDQwCn%q+VJ`v7NievbB`%7 z@lV9J9M_s^ZsmcXhj22t*mHG$r97P6j_KSmv)W>Mhf01Lyk}smhwK_s=z@5eb|*ej zFrhA?{&ZQ?F`wB)sy278UcIr!p3se=;IAG|_s(E}p|MYkMtWq9M!d|2ilp8sj{BGt zl+$YmC0;R{hMUOR^8rjm-iO!oyhj zo;98T)M!*^p)$PvJ{zjCn@y|hcxw&qzP$A@`R^E{$|K=r9M9qwq>H-vhi8!U^Uj+J zLF5zA0N0cirmv?zng>nBZb5DbBxZ#~-v)s-gtyw|h}9OeF2BK_9bVXHxFp4#c3QC z0@AhuDn8$fnK>&g$k{HFFIwe=*VV43?36WKb;dg{1+rFNJm^8~R?MR)8*ZU*q*QLBW0cdJ3~Ji*-*N@zePGbP453huuz}ZM2^rHFr?*FKC4dTSJc`f z+neLdA!dx6am|!f#YNY=pAFk!MQ9Im#iVF(*H!e=sPba}z2w>Cd|2}9ya2mI|0HQb zjou(V?pSo`NyFgo9Q=~lJ7;n6>D%rY6zbezvD1M`CovKnjE@$%&Unub;hldC z*R>v^!v7HLgKsiRmO+ZC2#w{i%zIudJg-R;d29D|53Ad+K9kj7AfoOVfpVoVE=D`f zct`JMG-ywrg{cx`4NM6VNd`anOFUUEF5^o#Q2n(fh_HHx7r=QF`zCmrL zZPCG$8-2K=p3rI3J%oo(PV)6Bjsaqp+;e1XLl8!@OQUkvhlqW1g-P50%}NtqA0l4y zH-Us*l`Oe@-{Wr$4Gp7;R|rMVqev679FofRDKC$LyQ=F#B$dxu8%gN6reI3teQ>52 z4Nr+*W=Ui=mfkyNtb2|y^BwN`$Lp@SoVixXgFaUj%0Y4p;+C6z0a;~|U|#ROsja7= zjy7EN0Q&KgZd+Vcjg|58w}9zmzXRX&W@Q)|XZx6-#$kSl9hu``O`hUb_<=EJ-+8Pa z3|opX?D9GHpgfQz3*fyJG9w|TIH$~!&Zq_p;RfA+;jb*8~)*IXH zH?4;1TN^ks#th@w6eN6cd9BOSz~K*4zZdqW5)Z`!*os#!an-$Yjn`L~e&wW2WV!3f zE)t8V(Nih&#bJ%3%!yn+J2Ho_ghBs`VwStowkx?3S1?1!1E6~~QzB8Z6g38b#TS&n zC0Fl1yh#;NJcm2@fMf?91u^k;#^##xx$I2wzoc3}?~AeEQQylkRlFJrvs@A0#%1TO zUCOw0i@oFA8Qcpq2-u3tlA5HyV;_X()PLp;!PQt+A`btc6g!VRG5euRmYWt>9Le$i zk;tTT4^K6{Ry*mIt|*chllVI>dMDwyq;B!WkA8;id;lv|@9~O?}-jWth)adp4?##a-R8xPIEW>S{z777aPbG8iDI3I?prtgx*F*I3n?L zt3tKvDh=-*OhdAAt$B$M-*a~pLk1*S2s+c^;?+{@%^*HjtqxOSLy$h<>_;CxdcjSE zzn9nQ&#S~aZ#S7<5ffP*yYs2-Ym_AZeRP8GlCj{shhfN*XqDT<8YhSIzq;w%Z*2&7 z!~h-a-G7kH)YNon=jFF<8c9$7_UWTa?Otxuk9 zZxBwvk3Z5dktu7>Tc-z1iss(tSFm`m9s;kgv3TbcB?`0H0zRNneBebGhk8LYZ7#L7 zE7#_dU3se-luhb6@$h{PCI~c=D%wFTIXeppGrM4f^p=))46)0}QK98WN>$en-LMT_ zX^Q_*>hZskLhx(CL!K;VK;mY+?oze)3uIJY@HT=Mb(ce4$oW3UZ*wdIyyv)^x|`00 zVFTIqf=>lPC{;xYf^^8^9clV!j9i6-!wPG5L|PUQ}H={ zqKXS$Tz*;JK6ql-c*|*a=G$Fm@C<jGur`Cdz;;Ga zN%Lgy4d+~VhH@gP>q22BL*lj_$s*-;T~T8#7n7OU@XCDVWYn*qb%Jzmrw1Cu-VC{t zfT97q_~P0cP_O88uH#H96r$jP9w%z5%9Pn|md1<|$5Wbgp*#KgPhC&uq1jw`7m$q1ynCGnl=QbSk(!%+$h5Bqo{qZ?bzr z$oQ(ARX=qrRLu=&!kF;rIW0RcP2X8RyAmO<3lMKP=>E$4I~Rt;N1}adG%C6H#9iFe z&w^-IEk+1@lsgmNMJmi*JWK%<&*R!-W;e$rdm*;8Soer@5)2$qMNwY@f`ZA!xH?zW zchk&@vK256%H(7r30>}ENxx9}$RA8FkQD+-~?N_sa%%d8D$h=qL6JwIQw z!BE^ZN_7}~cR+`M6j6l8{~7Mvh-2b>-ZM=R=s< zbh6oD)Ne@QVtdN)^;uop-W9RB``+1{^Fix!`bRMu@AoB*&-i%4YvL=rR*^SJcy>rMkD0QQ&iOsE2wx$5R)|} z044*}T7`7FHe6)|PCEFRpvnIg^h>9{(QNRUSciKkFn>w7zV|Uv8SOExmno@pSCxwj zhEP}7pF1b8-^2$IDUh>S9PWE3ej_3|B0QnS2EgFl-U2P{&eFuEZPSYh>Hn_2aoc$Q zoA*ao*Wr~nu=bALLI7|OrGR+jZxvRxq#DQ3fZGy$H3(f&jzAuM`m~8ap zDsK3hmsY-T4Sci4L~Tv25VdnGcSZv!|6GbnL~&nBo-YZcg%69q%CGq-6{>rY&J4&L zRod+UayLnvV!s;*D}U%IGxr*J=>Ch1hIqN(f0HD{cY^&oIJLq;V1RZ?YHtwn_pF{I zzVWz4OL&9i>dn5#r4UBUBM$jnYPS;iLWAk>31&H;kfgLnbK+G-y6l+Ie2Y4W$aKdi zdh$H>Aj3VFhGU&Pg468DlDCaci`MODC!YY3^tZ%}hzK?VG27}%@toufn-vKneEC?E z&*GLQV>hmJLF%!l@lhfGDN-4c!Cm$)2w)0HA6Riud_p~Q%4NmRsS@5fVvoJ$bcoCb zp5)(evARj_HtwLbO=h0xW1joVV{Xzhd)K!_2a9V@9l?%ui*dXNYVB2XR>F%k1{08X z5WxkP3-intMBgfEz^nzpoUfJa8#|H~xsLGVdu4^K^4lGnZ+X($(?ESYK#GSU4GJhS z;oY~|H}Gs7;Qohq=27hPgtSZF0&KUGIfJxUCnsexVlN+>7Iavf7a$M%w`^w{w{ad} znkY5FC(JZk!jdY8xii|CbQjkwom?`lWq_Xfj#xN5U`2Or4AX>h_>TC2gEz`UU5#X^ zFNm<4{1MNYNi09apLh<~qmiI1#K$6bK-dkoK|bv3E`f_mg$o9X>J4YeY}R{PMTxd8 z<#bc5DTwrSs+S#r@L$3zNlAQZ)gJdl3M3JPMc3^6l7;(TxI79b+)OFZVR}fHMJQDm zzUv+1+83xBH5eiAJz6(XW?9ij)-bEf!$mUJl^?pIPxq)udOyP*Ex7+AXXVXl*G+Q< z_~+bX6g9N(?(d+*s5z6Zj2M&xM_tlm5Zj0eIM|Efpo_)T#&!e*jG%(?<$CtCY28E{ zQjmm?1E4NoS8?o?a_)*>93tOvvyZVhDs|vzN#85s%1&C(6$v{U5Lbvdi}^%}bh}W3LXSlE4@+>J1NfFP2RNw`i?9#kK`OgzP>tpTMNT9q5*tdB#b$?SIL>dKPyxGNFgR7vC^Q&d}wq8taLmRDPQmdd^R+v;))s*f-822{5xh-U2(flhN1dXKQA<t}t@`Xl zUM67}ncZe|986C=L8C1+NFmV(sv{UosM;EEl$`sH1?yJe|25J3NFFzG2nAG6zLj$cmNWDQ z&KQh`zOP19LdfgT{3kImOe$N|hPU)5Iq84(r|_xO3X<_g7XYpLibuTYvzn-`=6-Lp zPjbDz{FZ+WASd(eSf82dve4B*5m+~Ee7tHehM?UHoGy|6D-`H>hpcrVNm%98Pw>%HKl31o{yKNKP7dAM|>7=GOB)Q?|t|N z#}H`84`1R+AXD})q*9dNSoE-{O8W3cxwTLsz|)^GK$4oV%QwP8f%vap{uOO|dX#;F z5d^Od^nIz=-#f&3;OCr6+BV!|h29&$VcSLPXmh)sHs^T-iTF+PBC^&<>J?q@&n0y3 zS;aO$Ho6Tqj;XS>5p6lq zPmfecP3Y#p1{_*SJE52%9wA1yoAE;BCc|G-7 zdxvK%uUGJrah0sBo^(}3ge>BewRCk7Ow&y#`~=#AeCP;bL+#gG4NR5ESE#!n|D!&E zrrZeiqc;-FnQca93cQR*xZ9CuJE`@#3jY??eikY7QjqZ3VIGOxN(9?r4I#dYpFS;y zMn$R7kz-P*S0p$zgEG36@HUMCFIgWQjUFLQg5C`-J1u#)TcoUKA&|lyu7`BR)GDK` z_>YH@L4lz$vkB=r+mqspZ`a5indCq>xFonD(fQ$l$rXvuH^xKmQt;iL9&1@WWB;J? z%KCXdD@a0TS$RX#>WD=I{cPCi25&Br^@-}u8{uSj9W*+y1PW{I`pxhkE@k0ZRR>zj z+EF#VM_lQ?;Nq4qc=d1cIiRt3Grs27JY;E1q3kPE_(0=dc?UdXIb^utZ?ikBXW91s z+4LhZWwA$G_Oza-*1Vf}bFcnPLHL1wwUnN zUZ$ly>a*m-ZyYl})d|@Ne@v?ctWG&Kyu5oo30O390HQwt?6~CsG~>mA0#NHsIYr6R zMsz5F;wRDFtDHye@tDT>$p_!0$p=$re+SA+>5J;L3hHHLZ|Ld$mq$HZ$&FE+4uJ8FW4U z7P&ZL2C})ou5H1Dig2Bwx>;_0|9IhH_baLt0F%hf-^`3SZGF)KO0_IOB#CM!ibK<9fH!ACDFGMh0b2*Up5~trOl>IDMINVK zi&#^HjMEs0hC#E@7U=pEq&}>02^J_HB5mjNBgsPBJ)-EdRCy*JL<41whLDtt6Opkg zuU8ic2s6r+fjkmaItXpI`@^35iCHQEY;S{T@Z(k`a?Z9{iC>{P(-HVA+exHh+pNVL zL&@|kHOAF)M{9|r75Q-A!)P6_J~m{%+PsBZEDqTV>tR2lgM{*QVKVB0PdvAs>jF%w! z2b+F-2PwiV2Lk%!2E7-`KC)Y{++XA+Zx|>zx_fk)%5h=ccOJ9c%t^)CUJS7e*q*Ka z>OMHtNjxMb#sMrUSryv5<*!b4^t^?O4V^aftyLVA(zwP7`{9p6lrETCVl~w-(e8FY{;cn>(Uy4#x-6m{S#iq$;*iWJFc5wd=`f4%T>OxF!GZkf{ZCp!;&W zcbwlf*Ey3wJ4+aml+b?6`K`Kkt}qN$nchkK*g)r8HN3?E=z0TtOTF9pkQsRcSmCWb z0r;t2yW%vvG|}bz&UjoXN9TNH3u||23-(wx1P*#Z6T>NBg(cGY{tVvF<)#_zjdd|L zY|ap;*8o(q2DFbj^uBpida-SnP~S7x+p==CdUso$gjr{LixfV9(MJoe(sy9+d|avM z4>e2}CiK3Km6|%-S|g{DXGV(y-b0xcLZ2eO9%+zlpq?>vAlx4q=5qqt4mi^egA52Z+2eZ*Kv@Aj99gg1oatt17Hg3id7#R>ru zx7MGO48v;g736N`3fomu5&$Evgt3Ng#VrB?Q1XCEuANokg#s4$X==aC6J|Rj>A>aZs)jMhvvq~{e`R9%@miuA z`3|d6FPQ55Q z`Ac*a+}?@1-D7WU{z^7VkE=0F(8v?KI2vi6sJG#$#lSld0_h({PlWO5L)C4HZN8Da%i)~ z@BGZNMbymgDc`uZTnb3@x#H}0KKjKL!Qq>iPg?%Eb^f|8B>xJ({9~J^awT+Ur7nBr z)qrG7X4ZfE9kd+Iew6n+ivg$ASPW^>7{MEG><)gJ`Reg!#?^m9FM*9G;~IZ0^#A8Q zw*M%D{5OL!xh656_?>$>h-PxfYLUQ_1$5)601ETxylEc?@v; z@MLQ&-xeNs>OYhvGdAa|nnnn?{PFR=icoIIfA^4oH%ZQN1=@OUdUxKYC!b~f&UQbV zsCGVed@KH_<$7A-Z{;@s9C>k2;Do>E26gbHtYy`0=CASp{Nk_nL6|8>=*ZA0$&HEw zD1G$t`-I@z4-r)UQWG&{P9-n#>aQ^V zKOYwMkDRPBp3zZ*rK;g&>PbyfZR)M(LYw3Me9&)>|JBd_n?2C}lcu-$$E$W-?tupP zjaRRIwmyPqtKXy$W^09esgeDg&B9qeN{jo6WFrwzz`uWBImbT){p#U+yQ~-4uO<~l zUN-smr1EC}4bA#*2KFDzs}=o-&iz|^lTmNQVuWh2j?u_WL|XK~KSlEY=P&-OS-YAV z?hcZc$-(YS6*M)AugEcCjP9MVR`y@9hW~i*f3?ztL7e4he=H3?%m|*Ca%>@-6XK1sDGd7;lSHspASvscvKdqS?VdKl6u6_rLHEb6ZqtDLm-Fg4~TX`)B1jn%Q50kZrLL&0V z3UozfmnKWh)a5*PnfK-!s+rYLYB)2~o_$Rg= zwER!hQG^}`sz=GtJ^hhU_54|Zq!3(qa=oay^tL0;6n3?tBjXP5LbXIqd`b*8X-q`$ zAyJAG%?JZX9|U=!dG}%2rdi|IwPvT4OsxVG{Vo7%SdvoZI9EHmztHq&-8Y30;NK8r zfSKjXmo-^$OB>cZYf$K&GQi$ko%6fZ-o(+fy*YJs$|29_#(1GYKdrst&PRItA?STH zdZ1zSL)G5LxmKafkBZ<{-?7&_>yqk{j@pa=A5&i$)&$?Tt%3rV$^fM#Mk5_cmvo6( zlp~asPN@+BQqmwHBP0Zr7&&@$ca9z~8b;T~VDGNudG6sgY-Cnk zQtDKtyQ>>|ekeI#7CHAaO1;L!L19uGiSibhK6lIFlSxE2-`PX65}Um^L!mdR&(R+L z70Z+>Qfyslq7{5xgTE(JG_QU?-~DrXm8iNZO?^;qpsx);#aejiwa_k3_P79^>-^*h ztbv}KpIjK69GUoYXNf&*H*jK-f5-6HFRSLqL9*HK+{lL-8TXBsald~)+Wr05`pXa{ zpGV2w@`-Pkd-F#+ssBRnKuBwsuB1@hgH+4+64t7ZQ=hyxk${*+tp=4bjD~(^kd3_O zbR+wiIq+lCYsi_mLlHE5+C!+hcb7{-qH1@{Z5PB9;BmaxCm+{e$YJD-rocD6_4~fh z{VHP%0c>b|_4t!9lExS~uG#Oi97LjKYQ%Y4J$IYd2zZPL?jFY}W1UGeQia zjyRYTor9xy65jX5h%3k%D)`H=XZ>?z>S3)#0!E41&ClOD;3i8B|H6uq=ENzW3FBje zia|VsV+wRJ?2!2{AJZ|sZ(L2GnB5h>#8pe#?w-t3|GZ&*rA0`ESrx2$mkBo_4#p^a zGqu7G4Y_h*hB%DfWd9F`>-&x-zfnQ zm!)+wN?1?ObPn$P6Al0}8`a$IGcKy5te)!l2>7;0$~IBc(4mB$o)?+XU@UL=7Cz(Q z^$fiD1rhUau-?t=g;CVaVrRCrznuqma{HCxNBz3e`~xGa4@D-Fyp`=Cz#smz$xS>H z?6*1yj&w!bs-#64{RD6y>9yl34%+msN&Q8+IF$Eq+r3NBuCOi+@j-S~`^QP^QndQ( z31k}e+66c5i_FjuZI5_r+vFFx#gFO^PxOQ*`jmpp4h9BKbP;)Jb2@xg>d}qMu=A~HwFUPI z(tkw8%r`RgQ6hK`o6A1#Z29%cIslOQnapBSO#;0% z4J*xKmSN1qjez=#nW>-R(ooaY$L`WADRow*n@&umHm zf?d^N6uL^Xjr{Z=1*Bg8demnGwQytkF98+XQ0yzA*eZ&K?%uoY&ljcq#Vs~GD`@#}&i zYIKwe*M|Sw)P1yI`5MA3?NQXD8OGOuRs61JMp#dLnd?ldnfCot7i-uPL`2dcZ%MgZ zN0_)8nLS#2YFFh^J|zN0Lb{pc+S%X;pYQy~q3ZFGwY$|cMsy%;z!p{C zD|68F3`_9vOPz2S+G+1&e0<^F{5;zP>_?kqMe-$MzQ*1lP>tQX&NNG*gkE@C^8eoJ z3pMkUP%00!FL;h^ReI$n97Z`K%536_};nLM$rg`~c8)pbTr@HjPUtfS- zb_jY%P)kZ05n2E;kGB!&G~c-s9n zo;$hrnQ8CH_yVt&2gW9#f%3k^a?cf(^kpR{?nvwB#Ltdp30xoyHM{HaFq1s|=9NUv z_5RpH0nFMC8DOX9tZ4zs`J{Q;!|!4O#STbzZd^&f8O?gd$D>ck{5g4)*Aj3zwbkVn zS4|vJN{B(4>&(d0DMR#~&HB{ODdU-t*}C7-Rm(o~DNzxDv<&M4kp#0HtGo9vh+vql}+ay>VOzgmOK(-tqi7DwcI&1LwnH=pCr z=EK#2fv#@e*!InUKX!o}a|;Vpq|VNoURM|UjEnx-^XJtpTNstJ4uQyhA!mOP-}<4c zxDRHv#(_gy5^;;7zK5~);L4%8faI<7HxLwlb2d)h42Hp;94tQDWn{(BRjHXnN%r1rn2xB9ne_=>FdqnY?$Sri99gKe9B*kn=AvjegTDD zZ5Qk|*Y4Z(c!@=Oa*Nmu@h8OoXBki;f5(hq6}9y*rN)(tj5XPVP3;&3#MkxQPf7Bw z#8R0qpv+yFB2^^M2(CJd7+A{H{7y{^b_yzeumrcb8jQTx6U)3Sq9u|{=$N7RIak#A z?-oM!90gO(4_b5*v-hwGpAQKh3HA7*L1q8qv+r-d9|_WoPz4sz6<+(sxZlXJNmUvX zOWW8UnODHGqNi5Ds~YBs$?m%l7yJD`LUv>xvpSuWwW7B<6d|uIcKk}sS z<>4duFd&2G@F(^P6?g2v*{X|#r?R_&=~(X;LjCuekxCnM;AbAofLdQ-L=XiM>r{R( zap`)P72e*PzItzABWz@YYjo(Z3hmDsCkjFu%k|P{rW>BbPe1kZsVlgu@_+h%Hjx}Y z9;IZ)e#?6Z*C?0&1Y4qVpIKD)TT5rUOS910n4|lX`L<`u;}tO_{cLmRNZpO^K2_tQ z8HKxTref+PiR^z4{x!;MzQOLAc-;oNM*@EWigWN76)L?b(ZyYg(Q6ykVCZXu$b3@ymL(|VhQhd+8)%ACn1mf3w`yeCUVh{`dg_$ zLI*kvL7rU()yR_dwza0P4R)DP6sb* zqWd|FZIz7Y zFRCfG8pHlZ;qMGRI$7`B#^3?Yj(p38=-(}O_Hq59erFC4A}TSCd0Awl4LPH%_?-J9 z?8aBzUo-uPm-l3?BV~Ej_vAxyA>Ae+L~pKXcG>_n=Xcgsdws(9Hglg$4y!OuCGR99 z)N|O5!b`Xr))Yc-O;de_e@2Ky*ODBnYDz6RBfGyuKY#|hc$}JoX4g9wxz!|UIq3OJ zqyly3={`X|tT>=Bh=C)=KKNy~lWJpVUvwPMwOt=_$~xh8c< zOJk2^VO95gu-O%aQ2zNC@ux0~4ls;XkNFO61`;f_lgolZDh+ zkXSHw)J%|P+GSy?siM^2uc5jA(XT}vsXGFZ9YSWf^bS`EKc~^_oh}`uuOR7Hh#@h8 z$Uf-SPi#PpsBz|Z#pgM>NRko{E3nPBSS5kxu z!d{_k#D!`}P*n0ar?n&Vf1lW>)f#*01U4sgA26^=*$B88+h)tmDWhdBsu9|-WORPpLn> zP`l_k-+=iv^P%U<^qgB%UyEUrcZKJw(Owd4%mHGk;R$v}*1 z2)9FlPHl?3L9KgSs4gA!1s{C0eiNY&S!GM@`>96t^a9DuA^0 z>#3==)xFmzn+VQemyji<<7i-%^)4S!2Z+|OazU3vU~{c%ja@FszNPJd*4`nZCcJf@ z9#n2UvJDID)7IZ?GVR^0WS6?0Q;+ZLmpWvDF!)?o0Hpkxb>EKHmadf>ObT5u=aV-| z3VjNNW8Xqa@4ULTFR>@6-xf-gSHBKwkPj zeNVv_LD}u$uywP!t&o;hvzB4)BE#6`(*eNpU;j@Z;Pwv!K<8q^qgm?)DDTm{ot-NE z+11^--CB>)+Jo+IK|Xe`9-d8g`*d5 zAF%aK1P1Fai{bwVGYOee0CU?PhQB=Yl1ur)E~52zn81LV`Ev23e~cYtUzJb_~1PzdzA>R0xHA{2j{PqkDna)({@cb~o*_RE^3+>j@)jrp&65<7)|0 zFYZMundaphGIiV6!5#W;S-#(#VCPonp0ju5q6UmMi>oIFO|dL{If9rvCH+GwEOg#% zfs0mz@%<$*U-jKoT>0r&UjvY%#%hNcx=O?Wglj=~JPWpbT3H2X?tK8LtVks$Rt?0) zcd{FqyYHe-e^y<8k<#Dr&g1-$plBXTzz(j3N&T8SxSS-y+MS_??l5fP=p4)Djo@Y3 zt1O)jzhQZeZRMRbCUvItC#~NC7rQL7-+k2VFv(=r|edH_R&Hp|nYT!KOv1r_h zpR;AH=-APQkiH^8<|H>cJKddsu+IfGAW!h;#qVHYyRRl_%VeOoczkt$;Hk*#&YgFO zs+SD1lb;y8lhEFfXH+ZwOJz1|PeBKt&xTl0PyO_$QYuyofEJUy2Hi9BiKdr}&sD+D z!m2yp*Pgg(@M6{u`^Jh70>1>vS5MK5pzQnt&b!sk&g7c+ZK2}FJA9h-F0W%HelFPu zY@faHn4K`4<;si`1XgpTL*~VLd7n$f(CGfN*lgh*h@J$TBS#$yHZeYZVH;R($E=Ad z+3H|wV|mBRI#Z*&CEtB2s#d=I?fX*RcUmiN7JF8|jaBSUevE<`9QeONgR`oDJ&0&(-kl2&2hxz zS8!r>IA28u@pC-ogKDqEuR~KC2m0>LCtDv7#bL?2Q%!OFgsxn# z8M~CXaUj{>`j5NYx+pvUae3`}dx6@ytA@!P5QEZRxP?Esn2Le;>P(hpHu?^Jg?rm~ z1y+yM8s3s8Fqd0)Y^07zt|J~UcZg^A@ms%0`Tc!L?NxR1r)?E=>G&@Hx?I3@f-7R5 zCa&IkMX!gCT#-bfoIY+lUVFhpR$g2)z@oA*U-VC=xH%cDYc<1zsk_dvZ04VN$649H zg9iZVwjP_iG&|pe^cV})zA@2H((0B->SbnEBeH=O?dp^#!dU^)BY~&)Z5jX;N~I)M^_bKckAJ87VuiYXjUbvzzRUMjCE#$%>5IUcfdEg1<=s^z|>d~nRly*Aep z7;r`oopxoUR}$xt(Rlh|ZlSy(l9T;T@M%Iv+NrWBqgT~mKxIM@Kl4|a&9=IjrBmUs zn`(pvBOHHy5Enq8nVy>6mD)6nQO>ecwvi-kpiJ2LOyd_tP)TOyli4??g=MuCJBgu; z`l4>k1rUaNh?-93O=j?3#jB+l_vO|gIBp-}44);lrqpv&IGcQDVqN*+t|rohnWi#s zV5JGGE03}4xrgCa=5%Q$L@k(@`Ht@XOUUQ@%ktH)&*vsLfYJ$dOz-*QX+fshdorb` zYr@@jD^KeO!A&bqXK!QFv^F+2uri}D&98anR?1>99o9ISSAG!ZLw0z_=GKPnB8Gus zbb!u}=BC`u^iK48Vqrz<%?kIY5PT)i&r22vUB~X!WvSd%yZ1sC&W;fvSzJ77)RO!I zm4@kQ;XsTMekOAiAweLIJ9sQSCZ;E#`%f#`l|1gSgGmPU1G(KJ7AlAx3$ItY_hqY7 zz53#n72jDJ9xGTM{|-S$mpCh?DFVZmY2v)<$bp^*%wJT zy)_x*U#bFN8y7f>i*roC%6TJfHCZK3eNM|DvE?$_@(G!T-+tirXXIxC`^F5VIFfXz z)-9hC6g1t8i9zmans)Jq%Be(&U1}>T<-tX&M9b$*@(T`bX*qc>e~1j?jJWB^JGQ&P zn>x<6RwFr01}ekLMF`NDSXCVN0!x-r!| zQwK@dd~jA1YY>zcoUquYan(o1P#*c4(M&|Cq)51cna_m=$ZAicYBO~aDUYwb zI~A8re~q+J*gv^Gf+Z$%P^Wn>PPyDn1Z5A;A6?4sqovn;S;@X5+lDh%X!RH8=7&2l zp(IaoAu0mr3bUrEuu{&`g{6%}l$RMYvC(sg`$uMrX?X@10L#ULNH!k< zHj}V$+mpiCD$HcF0MA~+^pcH*_KZ=6#LCb?lY+- zGY`eF+<~7ERqxWqp9 z)F5yvsP^mhSfdo?v6Y8}y$eI8v2Z zFd+DZX?#?tis8=`e?%!n^qZ^ULX$S`#oxufdfSVy^ABa;qN*5P@KtCylzUCYy-au~ zlmGA!M-}9S#g+lnHjvw>x{j1~eUtIXZcrBio<*3!hf_;xl zkoCw)dn@qP49X>grR9a?C_Kl(SXu@!68hG7}slm1pEZ8w82bCGJEjN_7wr7?TSa+#9FOo z!Id2evU`I|rv)vZI~6>EHsF&4U50rZ6rF+6L;v0l#Furz&N%3`5*S%CX4-Q23kvHd z_+~fobuXIGMcZbI=m}U&D05yt03!9d9lK}ON$m0NqoZb`FX`vho!~k{_W!s z^zF-~V$Np6b<|5;j`H+xom9|~gso9v5x8L?3;Lm^gUA#x(kG~?&K4Zr!2BLe0d>AM z5@`$ZjFhuP6v!1`H?rll6S z6<888WiV)aJOfe;j;8v+krAIc%w5fCt1 zH3Xe)d}ct_Ykh9mS9SWpsR~l zH_`91jw2}!w3>Rvp);0^tkiWc%zA!Hp-<+}=NBibX3=VR5^li$`&OqQwxOVOt}OtI zQcmpo@!==xb;sIoad-pOVy`vsomnjjwZVXz-N(Wy;V()sZdI%rQW) z{`Q8xS^`C(oI<%BsecLx2J;D=sDb#h#j8lWuPvsy9QlI8Hdkcg&Rs7hTb_@6^qHsu zsUz*v(8&2j-gJdyD+Hz6QO|{f%-Ci~viG@ZOQ}gM=vcEVJlTU|Ex{m|vbox%=+s`{ z-T+bZiOG~bk^my*K|@}V;pIi5ErgNOs@#O3B>Fe}o2-R)XO-xPpOQh1;nIymQV5`a zY3$mtY63Ad%w1;3$0#;`Q9nNnVcclF#Ce&Brhc4&+CL2JGylRQH92=U6`~^%DYhrz z6LoN1iix+6t9RS6M?{H_q%-cR>S{`ZB%IOmd=+*yEs$^b%;aYhMTTe*xA}z5)>66h z>}C8pC=&0Ai+hGATgsEY-N64m2i`M-DLsK?JIU{T;9b$kWdr>L;}VCqe*@FXtL!fv z(KR+ybhLc+S;pzRedgl5_5Xu@h6<$1W0Eq0F%|}cL-;Ol3BiLp$)K6L%yEsY4dzsH ztN2xZN&cnp2ik+K2hip^YWWfc+;oRc1%pL|{SZ}P+G9l9&pzQlrxf|V?}1Yonz*@_q5E<`MBoca;f)UbZlO_Skbtgwy<5Lln#zuER} zQ`oLg4_)vG=`rV9e_yG0x7IkD(f#4OEj6|CZ=A@`M=fuR`7VRbllB()vhb+3i~aNE z_m$s;py`2lSXvVAB38Z z!Cx^M2B@G06Z7sp-GuYC!01^8(wKCGsgT9EDUzv?f6GHk?xfQk|8C_Wzq&7_wO(fW4fC{+9tYI>sblgJXjjEwvInK4YdOzemRg_KX;)X6 zM|5%)Z~oXj(WFJ=U-n7s05x?+s;R%4?rAcj@m13!{*Iv!`78%pDmUNvD6bBmr#x`W zrsgU`E{W7vl{#5keP@3NIGNym%-oHfV}5^uFcq7x3L#*lV%tl$Zlc_2F;XJe=%N1j0R_|hH3oh z$x8^3>eFSOR1Wzp$FruhH((~~i6Aq*V{(~=n!;vy6=v8i3Qyi-uDLpzbWvkJD&c1> zZxl5$I`LOZU)HB^2-u)7t>2G%13C*cE)(*L^E>5g`S$RJC8(3{gyUV>=FU(228=qC z!~JEWlHPU~()fa-uz7xM#eD;&BIMQ(=IGcbrN`l{F(Z=QtKGeclpjxwh&`?MuxNy( z@!jXVr3WO$J5BV!#W#%`H_U!c>ko?p=DwS6v_C}qJ;KPUC6>W6@rU0Q6H+8koV3+G zSvJ|I~ZdP>*bEZdCll zIVJ8>eK!Vk;544ECE>0-v>Sqm6s~_^6c$ZVOR!eS3}3*NI!TCrn+*tpr&7oLY}!Wq zg*`&LFKI+f?i4)YCaNP!50<>gbZXGZ8gf$3%UYZlwjif5f} zA9Lv-4j$Z5m%pUNXTE1f`15lq7;zp#V?|@2ePUeJWTyV^mVEwnn z83Q0nm7!8^Fc-@NmAJ_qu_fUiQFS6lHb_a zPb^!DZ5Fmz8C0{Po!yO**oxqg&L?E|cM{@cBjN)}utAe-a7*Rx#cymDz4vA*0FL$< zkGMCHHjWO*V~Q~zE&QE>B(v6S>yNT0GnSGyq(y3-+$T~x!L0txyv6xYJT`mdxJ(Inic zRy8mzzc6O7+{-UVsn%7hcWq{r+n+nz9pzm`Qh)h!b}UArX&{!M!3BTWvYfC9YCv`Zq6U@|BQfW)tB$eqjRhG_zhWksUWNIcAO&d*d;Ijw)V*W2V9CX65JtQOI}}y=n;Ibd;=%>lp$n*!yG&MbcbwiF_V9Afe?YpUZ&w>68qX&JN~Bj=FYN{D)u^J^l8FQ&dz8I3@6BmN3zs1@ ze5m`_(Ry$7Z7a;<5DglV-nE><8XQN5Mv!w!xWZ)(Z+JLTeLiF{1oY}W|DL~h1p3Q+ zs(#rLMEL6M%4B|7T#DbZ7PuT8B+4ZByamchzA&zTcIX05AAdP+;PbiTRJ^=OG$Q{i zN=Fc@<&=1uM;yv&SAIqf>~*b_bAZ_qU)h~_f=%v6f8>c&sf=$9UpS~)Bl0&A zP!AVe;gV1swL)$lriP(3g}M!x*BvnUmDvPBjdERv_91$ zG$Cu=SwY?v{bKU{ZixEd4kCkML1}^BLFQ~Dr3wu zCj*+H*@7grkz`bnc2*BO<2j~Owlw(&1;_ynXQxy?rZs*{CcTFg?9<-i!@NlFE5I*? zi+f%96(7`RrG2cSAfux)7`zWHXJbclK-TWw@a7EJBgG|<;R$*=9P8x-*8_$U`ANST zstmXT`f}*(p@RGO1LTwg@9hG8o}D{_C_waWouRbW1e5( zEoUu}2Z~KmPZGFYKEuEVq%m`#=TCDBMjtT??a(04-e&(a}nS$hN^|@Pa0K(^Lu8O&; z?2q9LP&D*cBYd{!?1%0V)8)Rc{33M1W{&su5Y+mtQ8P{BY2tGWJppMWOu-&1Eji>> zmXl@<8XpXT2)YO#tgK02U`b-)s;L7xb~Vd$6f$EO3wKkm@qiu<+KRZ{=?%}gm%8V* z<8$*v3dT@oCw!nr$tj)$B=p`*uHFNY6nZ#P@xP-8ko5hYy5IIPH;E8O{BqtIilXTe zzItW;5+2^Mv%FgO*ZUT84sfA*)%mi-bCt_n<+9nA`u-!Ji{JER2>2svrK|gx9?;X1 z33)$BsY_vBp*I-*%H7>PGbzbvP{}W7VU@z1eCIp#!IcEFL#JtHZl_?UQ!l4yZmA(% z)9GdtU3bZ`uAR+GX-VBzGeVCy->YYYny#Ad9NraqE!t7B*PY_9*3{rIv>{9Cmqj}ehXR7zzd^($t|QVQQWBmKQ4k4<&k=0x!4#TLUsNtEuq!i-Bq`~`ppAQ(oV5E~0 zWxZCs;e%(s(;>cV#4Cy^39Z2EE%i*g3FIYd7Qu90n(Y%Qd19D%Td%TP{{B%9@Su|2 z9sAX4#ZoLP8;{#GA(7P3_ulzah}Cgjv6Ssx7&`KHt$L|;%@W{2$Q1dOiqCGAxvE4- zjuQh7sUkb!RLggS+o%^*w0J8T-v50&wEYvYEpNkIh_Ch8?g`j(-}q!|{mQF2|8I&` z7gd+)DtYi@FRCke%r&n4>#~*Lh4qs0t+b?|Itiz`M$G0$)~5}#v<_p?DS*C@2$0sN z&U~*JhBYB(=V19_3pp=&KOWE>9B}$`u+KW8bZzAz{Fkzy?v00PwrI7=hqqy# zW4FQX+^!_C9-Yn+PdaSr0B)_yCLYl%YNieRV?+N&Fgrp-8?2Z`$hWHfy4_;mSkVJ_ zXcxobPr%rp#@%d-EX7EZgE;qVi=_mToO=P?G2-K7KFOc5UW?OjkLVKaW#X?s7C+oB zqNGLzhpKkLON4qSr zg8}=)2;Xv^{;k5FbqCulSlnf#+@hJnBQU(DU2>J+uG-WqNYf(lE%0#y3xE0dnUIfF ztT)P*KF>7-dV)1_nWJ`S4k?u!kioaU;XWpSes+*0w5>{*cNCB8*>jLY;n+O=7$_AQ zoDK^z@6PJx#?biSk4k461lAOLeFQB7*9`}4S!<OOZy2dcUi?* zgrwM+SRSxY`~kFeA;;pb<%H%)9i32e#$9AU^1C%i&k^OQizRyXebD5i&IsW2r)KtDk zg*w+s1S@~cu$6C0$lwoSr^W5ohG+IAZw5HV^e-%*bW|(&ukd1(N0ep}G@Hyr8OtnMaWXzb|j3EI2I{-M)29 z9sDFQ^eEzo3-{eHm+IfMErHuP1^5^hTFaP)SG&W_7Je@hlgv)3EYhQUjoB=8x|&`9 zxBBiYDk(n=E$G zbEYUEHl8IhS+BmUUdl+UF68vdkC1JjN0cy_;~C)aMm zH}{$@FU$6!*^5XRl)dmsc_jK;DA@AHd|$$e+UXm1R@LVj(BS1z7Yo17yKSQq?pt|R zP%*a|pamT39Ekhtm=N~hyccdS5@10dA-bU@mR+*%Cu#tktWo+!@s#+Vpc?hB{v!&l zo(ldur%Vcf7OE?V+^}vNa)tTg_|swJ;q3|&%;QVC^*34I#U8i~^u(+JgXzPeUx89u zxcr}JpD7&D3vwnro9{~5*!>tdGCT~U4ZBpTMhqxz512!$dTQdDvtbR`nkar;_V&ow zs{oaoTk`h}TffDo1SZH^0;l3F? z@srKsOdBx4GRy^T)I2a_F)e#LUT}`gpm2Svcu&8*ay5kF;Za# z;R=B7(OH?F4qx^Vt+>1;8NZzyL87Y0Ky@3HoTEy9{B%I#WT z^>JhNKCI?DeY;O64-dHS{S3FiSFFL>es z+E8Kq+4WOaGnQbTC+e>rWB-lkm3VnZAI&(RU{yvuftK58)+7~x>*4ib~8eFKb%Q+!p~QA{g;5!K3O z-rg1QBxq|nV2sXu=!%5QNOh7nfv%T(HP-V(v|4aEMbspF@JRQ=qcCHB+}yna)tJao z^k(dylMmK18u(tc=k6E7%i21e$7IZ7mde*^fl`5(4;^=A!}tkynrPKmKlzL_XsL-@ ze+lf9tSp~In6qO;> zB+PZUg4UKC$5OGoUI+JV1^1ITb4KrT{Cat^5;K_}2Pe<0$1Hip@E@71sb;Vgo=1(Y zd0l+Z$}#@3K{qUo-Tb}GM%5!>Lo$})kQc%jrb1-k>+&aRGvU@4rArNVJB?JG=B!3! z!Vf)FwmW8OdFwjJ+p44vZgUk@(9=(_h@z2h@{;Z*XOwu(mH`)A+9DDqmy`4AJ5LTt zPmXMTVf~Gj(xf-b_FXke<<}9qVw+(e91t&a?Dex_KvNL*39a#K%gqnt?tT)XVK0MT{sF4$gyejZe7TCdY{8 z=NXG$sHo-3`$kTF5i3vCJ$-^qE%tK%r@6IjJ3d*KNQ(Wh_2AU_dFfHD{0sFa*SHAd z4PUIHX<#7y!Q1(5ItA=o}Aqr7QVYG+5 zAWR0b)QqF|qdYzws#y1DM&O12qGh6C;(xQDAtja;psGK8EEPQ%j%pFW4Jm@D%xYZl z_(X(_zhcVdoi+SvpG~eHa&hx2{}HYx5^u+;brAt7PA_tmm-oAcs4FlB@FPGU3K zeX@vVz>!{%C6Fm%^5G_KbH=I9r}g%a(BY zbpLFNVr*q`0}!cQ_fH0ViI1$PeH~mJzR=m%A(ZjsS{Hw+>*eAW2jfT?W-eNP*X{CF z!iZ76u_vUOhz8ps7Cy4eX$3sHqI8X1vQ%gXDTvC_lVhovmrBx~$6`+jk}kV%^lMbe zZ4h%(YfS7ed#7Od`8K=nrVRI2`Ej*tvF{0c@)lp0dQ&5Zo6hI#Vl z-q+K6GJVd6D*Qi26iwH?E&3U6xzD5r`lTYgF4{-yo+C#)T+b$OIcBMlQ?lspm>&|u@H*yXp44b`Ar4@MF#3)Xsli|%%2Z^LUx?>$CSn6B&U0;wOE34uA8!t+p z&O7{~5qFJ1{VcZv<|Mh4Zt)HL4j1p(zY&wQuQo?e=FVn3ywe5G zlu3U;Z<3d7^J8SJN$uO{4U7(Wf176WS(^`-zX+73&h|oDS}FasFy!iZOb!un2)E!LNgQmL1mE~-WAzlJJBq^< zXB3sn1(-v%k2yrSz&7Ij2%g+-7A+RI-xFy|2$RasS!K$cEpeVrK|7;?#gmyj6^2RY z6;lrCv0roUFFok4LsnQGVu{{F$C^CMfRioa_ZK|u)wRlBg8`xB6Ba6D-nPY#8I#av zQZWyD9ukq;ZAyU|-|b@#X)Xg?cl0k5kk{h9uA9nU2W+1jp)^ChmQ|cTG@cnK#fh?Z z4uuU*fU)WqVcA2+OTRN}oio2zI1My1EM{9UX8psjdaU>h%#$m?iY}6P6is#KM3uaSj>xbeuR5Fj@H@j`>plBG=y000F_Mn7v!{T&`v;uj zcBw+_&=4CnE2opo30kEkHP5R#x3V{3c!H8LJbEJoo`Lb|rHPbBMq|bpBQ-qEupjqD zYR7={#u#+S7WcYluMC#Feizrp!Wfn{%8o*RRG~{3n)0g@Kk4(keuKFGKgckBFX$Aq z9wp{SvFGF|qj2jgC8p2xf?*yKGfdl>-NyVy0sI5dyCQ+Wtji`}zYd*Ev)uGP1^Ox+GG&AcvEB9AOlo?$&dRz`O zOZ3K5E!m4ZY#gW6v05$LHfS#${4x4B-t9ez{o3?&7kPt5&P~4-*2rcxqr{7dCf&T2 za5%kdvU`=(du!qpbGUu1c>?;w{pBruh2!AqjX?mXZ}yFxmB5rxsKz%J6>;x%$?<|W zN$1CH<3gCzBzUn$AlnN>bZrKB8*The5UT922EtG>};IJFJ#3a>?-|d38fNr`}LD{Tb13sJ)xGuHhG{^hh#gQ6S)t^%vJ0fv4Lj&>6}rfRMx`( zW9%!V+S<0ZTS|czhZZRwtQ3b9Clq%m&?3b(P~3}!;=x^7B*me{-GdXPP^4IJcZx$G z@a5cd&pr3N<9)~Yer4=rkL{}CYFH}rd&Zb*tNlhcwYrDPJX7k zy1I1E`E*=pUhHXn_=ZoZWKTKi>XT3sudvzVbK zY;4V{lKMr)o;bdJrR|{y?p2NuH344_eNJaIncllAV}U3LJUJFAkXSUGJo@xlas``Jf?o`u*V%xJt<3nw34=Z1rFUSR5__lEo0=^CbhpM>5%tfeDU)( zQy(Gl7NR&CndHhei|g8bKbQoY@(f=p)>UHLvmvoxHycshc981`p^TX{_RGG0$az$K zd||&Zmh+M!aQ_j2;xlmq@m)`JNaC8A4xAkL>af)vPv8W9*F5 z6J+5XW3Pl&s>JqBOXl>lxB_sU7vfcDmHJU+qxhp>kNMv>f~gEf00QY(n75rhTte5IdUJX zx@47J<;8mVas7#w8jOh)?jF$5W2mw~ubM9(_->FcX_=IZTs};&?o_i(-Hf#Iy&uEm zUa@}NP2JXcN?zfYJSqyTwO%Q)qzC7d;RVLVo)3^ zT95G95wAtFb#M&Zi_tN;P@X|bK*)`== znehw1*GPZiN7}O=e1&ao^RLP8BM%iMih+XVd07|w4jdGzG5>IWDkK5d2jQ+|A*YJ> zt(GrXC9e82A1(&uigs@H1sRawAVIyAVd7!f5>@LA<1IIr%MZ0>%qsaO} zu;yVLXA4R&NZ}7QzP4yrh{K$y5AL%FTct zyHE-F`wt>`w}yUt%y-;l8Zl#|iy-e(Y^GHG9l)p;8hs2l#CuW2|6x5S*`sNpuM`Sm z27;g$?qAEvyWIIA{s@>0Vm$47|AbnGU6=487dQmYqe7Z8JLOHejZG3Ks@dQa8iB6L zcG<&5R#`q4RvuO-`DNc}VORNH+L#X{hU&$@N)}Or9S`DVy0fCJ)|*APth$5-)C&`i z@+7o8_P2K9cTcw6jXWn0x;tksz%H?j&gU}?l6jSx5f&x!@uNx-A>K6d67g0(x9Sm6XA% zT(TE^mWL5fK}2+ZK*5ni8v$Xd%lGE0>$~a-AD_QNHIF=VUaHnT1zUR(nV5|NwcJ!s ze|=rd+*AsUgmP3x%$Y1FbHHxL1v!;MMh=9i&j9!6U3Bea=PqBYJ=HUBt6?Jp*JCYr z1R{Uu*h`GWOe44R&4~@!F}>IU=Fb(me9U*2^b2=jykB+{5bEodBpt#PcI(qLvt$Zqj(qIQ}@6I7K_*Q0hwEXXL5m zbc^$?$kR4a62i5|G>5=oBbzn=t<4 z?M}a6^t}_wI3nWJD2d~6hnm?(c^0mxOPN}Dm)sueR@65?44W3hU22kEs`(@>ZhXHi zv`?k*vm1UEHQzz4et6yQc&k0#rf%`Ve~QR$Lm3^u{Nc-I>_2h#?;YJQ`F8&Md=C(b z{#?d6E9uHP1QM83zZ~}_RN5i~I|bXxKv2dx2v;KapEOrPm-A1XzlCA?({+%H$Ud9tIB0ek4@iZ-&CuLQAu55R{O;#gme|cO zg=Sq<2Bi-U%|g3r`}Ocgo$|%x?`3;>M~GL9a*tr2>48*2OILAhs)+9h__tSOQrd=BKsht9SRQ??|r3XmoyHV?H-1r=Fsx?pT zIHz%F(wAIkCDB1saFDtc-za?MrmKTcZS}Xq zb>Si_u}2*_*l{v;ezMPyfk^@@oZ5tEdFhp>`L?wYB;A1=W9>Q&t0u~Mw$UbuvW!dy zZO>c>_HnSi#7AxoY8DxrLSDVYEhRhRoP=E+eLl$4RjI#RtP_3I+1pk5MvX-up{rZ? z3TqtqXe0DVFjoiTJ2Zq+6LUoOU}M-$BB1z{N-L*`$Jo|+0ii|2{X`$*cJTJ zTwd$FkFU`OlKJ~`3XkRP>p{g@a^q!WNQhNs^C;sbPiaWTueM12LMu$N8wGvA^jNKF z5kB+fH*ztL?cxbIZg&Y4DUl@317y5wN1}GCyL_r?L#XE;2bFB(t*C8`&F-5~^Gr22vVktV@4AsiOh9QuH>T=C+FECyVg z?>&B_iA%{gmFGL$GSueM#+eW}dmjOQo$ZJH=qdupH&Tmg&JKgXdh6$IKHh^r1PEej zGi=Oxciq%5BEb=Nj-)FGB39TaZ~eo=2n4s;4D=~3IYKo2+?;?{n}0wNhGG9}5&BR= zZkF79Mq59~`YlQn8!Ag6LTqV^-3q)p3lTU^o>zC8jh%iH-@2g$7jqj-`FPl!qCfBp z3VGFKiLOGK(v`O(?RXS0KxP1!J?uI6gD|-1-j!Qv8bT&dEt0;;H-_B|{%X!Aam{&E z8Y|y3>SqrWJj5gd2r_(_B^VWIlYSiis8h@V!EQbxG?5Dbu+g;q_N1R)Wa|AeYnPwq z;=lKv+H?Um<1?9dK45AAD<7`Mh_OQyXNH)%==3ajmGt$3X&5@EBG8<=cyGH`d&1Z0 zMruzBh+FslcxHRg;Wk_KNUX)+u1hl>og9r(;5uwMQ9GeJp08Z34dPC?3n0|cU#fW% z815nBJG{n!4JY0 zV%MR&EHK^5{7VbqAOl)GSF-2{<&WM;&5__qh70UfYe)#!Iw#2h-8Tnaa@UG|4*W1r zeRjgzHgR9lhH4?L=B3rG>VCM1))D6XCS272YoPuW#iMu{KvzX6_oI>E(A=WL!i;)< zvHni2o7z2JV^k2Ri8aOX477$%2*wdpxPSiKJw=gw)S#@+SeU7IZA4b+wz^I))NzvR2W)$j23t}RB!Mv9MkKzlQV zNHJ6WG+$iEXt5n(J+0AocndYw?1t-#3;~q#jgyoS65ZMf;gJFz?fJa?rqGA-6e1N# zS%yN&qf`X=2&g7c*3}K^{TJIOw3(jCKl4Q@iJs~gboaRTy32J2MDBZ0^z$suUCtkZ zr{FX*vA?xhAGnVRy_m`_zus4BoEr3MfSo=Va=FRRpr%=%&gnWBg=Gji6|VB6#5!zq zi!7=u_b_JC+DxxsDu~OQtrqcFwNR#tnA|ANU<0Xw2rf2+1Xxa6cTXQLuj!zLbuuA5 zo#a&NT}T^h#)cw^OzMif+ze5t_$dB{_Q#bFezwie8mIgS#N+gK32HCE#m5I|hE*A- z;C5j~qOj6gYLdwPRa3mpZ{vUhYa)Q|4I5d7Cua8Qm2RBeFma{1IT5-lD^y1xGm~Jm zxW|jF$<&E9V-{A|qgkLpk8Q>a7GePUQpIb@R93*bI??%gjaNi!NgSR1xbj`546iFLB+#Ex-fCb19`VxQH0**&?6$DI=)m-{L*^^$h4y(t!6fOuO#QmI5H zcDtvOtPgU!&57}u`;7tR^zU^o4oOWyWgY$frHr?K2{qg>!x=!i_-;}XhtWux?_N90 zkS3CDj)wC@&LrT?u`i=%oZ{Qe+s@mZ{b-S`(k&qcE0Fo%OJbd^5*IsE!H*duR3*ED znF3~!E)tn+?$gW~81+jXxgELX>Ssp{d3-G1_+^+7hddjFDzQEO3}j&Tqa}8QBiFhR z&Hf*cvdL7fxdPO{ewEjUj^KGel2Mnq;iaRMRpKfz;v!twb$#Dh`ia!sG|W~x-55)z zLl^InA#EJqwddXM>RUM`QF4*Hb5=xCY+KXMoZyZdl_5!=+_`H8Ri#7iXSgCj(mZMz zDn7>GaD}tn8Sx<4J7>dztB#bgL#+50Wi^%GM@|mkH6NWzc=`Wez3BBFJ8VW8{E1w6 zgfXXg?A+R(2{6OrVDY{>3jAGQKG=zDMSbSg{mZFK?lsVN(1b3SZvEt4#o$=|M)l%! z&z|D0c(uSSo`JZBt?wjb36+$Qd`E(tvQ)S;~!tKvC=-#hq=qq-8nw1O%By4Mq^|&KY3|b}8OEaq zt+OWvtTbc4>4mutywquoqGpOGtzSGa!L&&Y>X6o!duf-HxqhDcU`LrzUY}B(thN6e zo{RI#p|C`kb=SOHDZgW0ld~U8n#O8r)(?(x6CONJU;M6iUAvNjo^W4SQimBLbi>(Vrywh+bYjm#nq`A@%X*1Y=n{jcOfN+&J z)3i9G&ecgs!{%e;X>PRUJp6j67(G7ARV}~_bWl`!#ke*CJ9^K@s z3tO^eE=dv!6~^J4a#-f^T;j)~8Z9Vf+9>Qag2hZa?lRR!YI{@=a0nA1AuK!JmQy8C z#)O@jDO#2HyuNQzB)Z5sy(pI!_n5BFtZv_%qm1#`jvyh{-s;BcJ0)gNB(Yy-ikt$a zR6UaQqJcdkvd2fmizvQE zg9b;qf=JRDG-uA+5dSVo&^Q1U!Fs8uPb|75B@j6pcHX^sK`$KFgHcutURb95GOPZV-a29KI=wQL#8syU4O4zbk%hTb&eSE;D=jwHvnj=%ZMz>E^AD1a$)G3Xf z+i|xuzi$eUX!`jq^lqfyyqBm>k=SSZm)EPGE6>SQS z5Aya~i$$Ma z6*G09Ba0WKpULq@mSnNWW7g+WhdUpDHI_&^Bwx(Q%!#vqN4i9ZE8iUx?5WrIVmsEz z#D!lTVYlP#0JK-IpRR*Nhovg!u*^qbvw2DXZP>_;Zuo><%GUTPKHjzP%md#{Uk*S2 zk%^4}b>{2naye+c-raV62nqF=eE;Su^S~)*ylm^hCmO@R&zQJvfGJaH!YhkymPG>L zHFd)vHq<+Ey)$yk{%6@qKPX{T?o`e<(b>|rtiyKabuQ4QzZqR?PlD#1gk)|0q|M|^s{TufnGhNWk za`ic1)|&a-VKfEF#pEF?VY`zEu?)SJ%zYQbD?B;E!Q*0M1ciF;(o5s(<0hA-uWHb3 zEi>fE-OWg8G$67l(Nx@y=K}HQfn`5o`jnARX@AU{`|A4&j6%Dq5+_^odH2F^+Xuko zJHNn}r`N;_kOvhi=4Pc%L+2%@UX{0j<${$aO8#4nb3zV(RJ93Whu(hT%nYRx538(- zmV~MDv3ahB9#UPOzDKRSr2X-r9Cqu?IN=nb;KsWtXwQ4F6;f%z!S}=u^v316kgH~U zI|^rf7VZbJWB!xI2Pbm3cWx^G5?+?euU|DkFBG#!J4A(yVUvX3F2C;f7JXZY z6Ch`Jfo2MUQBs6;h^@DA1|c%{d6t*;U2?188SViDFPEbMbd_zS$^v?HB^gZWPAlqG z7VokU1LqpcCEk`!plyqLx2on(xN3vmce=);JWUnv>Z7HrnaA!Yogo+_(@LIiC;Ll>^ z-AL97do-`FN$WYErnVA$Z5L0E?X3LT4Y0L?Qg#y~6dBs+3I9(3QuJ09kM-sw6yn<z_zv>Gt(K zMa+)S!@g>OXr`h4wPm~=f1pR*WHs|Oln##CJF8K>dNI8P!6OU1@}Hi%ATiV5K#euHPhlH>r{RS?EUq8hk0S|I>ExdM1>t{a?-B_K*Wp=42ce6~HD#1ei zvP*yR-i>B5bDx)4}nRdC-=M7Q{FK=(Lco!-gw%uG1HB&em)THHos45FvhNV`XA zM0;Qb0;P9!Z<)N|XQ4zy@q`A2xDkSPq{*BzD_yS!)@8q^^mXN4Q_|6rP}W6BP5tV> z-)`cXc^|%1-9B__-oRy{t7oFg)dgix1$R?~W(X7ojrOw9H|qGr--oZ5P6d_j?)*Y` zi&YmsmO!_Dvfrh@f`1nBdJr72&oJG(^ic{xwTl0%S9T5m_`<_uptnL6QC1q3%p z+s3u8!%qnFKz$*ggf8DB^b;?sK6ZKklS()KK0yUD!WRW^xBAqn-+IiL*ZvVLi|8OR zlu8HG%p%|Az8rJ*i*V}v$#)<`N%1?1+x&;$X~e>MTK)q?j&${`tFl7hE?9FGU!pQ> zonoa#3kDpiG!=6eh#Q$fb5xXlwvumVU3N~3t={mx*e2==Mq?>L&3?Nm&ec%7UMiZr z#SG3_Lh>dxiOls2QN!R6eqVUS6O@5n7G*Cx%(lL+Qx?V{Xp>r%RJ^y0>9`+SD!kKe zZ$_}UHVVGrNQqM=vLe71-k6d16fg_yQPpSEu#8;c{Gc`maz>aPV;dV;$8*?mhvbN^xNu~FDMz`9+=%RwT(5S1$iXFtv*k_98e7$RDrE|n}Ojh#(Zc(owKF-s-df|QtUqL|$@^b-^EhjcTy0AGLc z$<(tfxG;7QCIA`toB@ZqE~Dz*uR7gJ7zOImR0e~@vL~m`KBH!P8De&_m)k65+!R`E zUl8}!q+A?;*n&9@8`(3FiUi_Si6UuvZ71?ng7I*7d2WRuFBXFxbb0DOa`BC+#D$P4 zTf8UWYG$16)v6Rvx|Nj^p}DlFhDhdMk22VY)Y=k0Y?PCsBhHTdDyItY-J2GnzqVcs zx89t2K9yqciSS8buhv|WX3~^SEDsFj*QQCDVlt#JxD3M^0dO9`C3VYDSG~`swG_K5OoF!i zxyubNMAJBA*eIgWZcc_SYIQ#wd)9J~{x%+8aq!YUzAW@H%arCua<KzO?B29@!4ItsF>|uz@fKzVnlUzi2{fnb1CY`YNZDK7M*as& z_zlh+0_Wp_dj(D6+*0+Xj6h|o#UCqPb4Z3}DOGwcrg}4&X25@d{PJm@7P|HSDEVRg|mV{kDz^GZ3Q7akZy}#$T zt7lgnCd}KmezIXihL?QEa9A8&*g_FuUBxo#@+xy;)ur%N=w)uw2sYzLiE-8S7o-G85qDEw*jwb=_0_|P3kd0zOCA_HIKmILJJm?&!BP6)vMckbv?)tf-u(2&?4A)}Q(1oYkOJyE4Efz^>yI*AYr>8wNlR8W z@>i4-co@AKP7*E^P9l)CAiHdbi}v{JW=)}jlJ9;87mH>n%j~3s_UDUlkyHMtsRCGU z!0m+Kz)RZYkRq!aakS#PYJKmk2IMim`KsVB$*Q99s67EQ$kB)LJ6NX>_)@N4R<3F7 ztHeeKqkL?Qvn+XZIS2L;-1EXkic%{>lDqBFEM325^bjHXUx?D&y-)M8?#lBCu)i3f78t6JzIjpB*dgkQ31B0Wmu_E)N`AOa9((YCSj&!*^qm{|`|d9gYXCrs z!yxjT83RoA(<9cx5SJceRViHYDHYB{R!sa7j>?7P z05GCLtb<8p(t;$PBVWuOg%PUP+Xor}>zjSp4iLcBLgL)p<({C@#XBOQSs)HEK*>=` zCbb?Y&2lxmcW@Ar+l)wgoAdT(xN}1D*<$6Erv<4WzK)+GJ3CZfaX{|{7ugOJ$Bba$ znF|5Nmo3#C487_4%fqk6>b-t_L6B43tx8{rO)kq%nW&bO*epcWcGvDEmGC3shgW%v zMyL?C-!d}eS64@sE4`hMme2h_7Q0*%lZ?CfvEUskSt`>P>dW7Kn5 zglhO%mRv?n(;M4Y$DAI9bo3&48_yQopmAw&TXyD_`Y-atn0OGM8G?pVXD5{^-%*he z`o`q}Z;?{X^C`9xB{l~MTP$;Nt}E(_krk(wGLd`)ry!Pf{0ZF|2@b_JRaqnfWqs~V3>!%6P!jp2zj`^n!6G(Yj`Ptj_P4G>3 zmfTk2h0IoO`NH{zwH$6F48)s4(CvFRP$@po&kO$X6p;BNEIYv#NNEqGY=!L%|2z5X zbN~KG_@0M{So3&C=!UUW%#y3gI3eq~uzo5GGCxGOrl%!*BX{%6*M86UwzkIRN`T3- z>44S?6`d7)7hs^Q=&N<(abtuQk@7$H@(F;5$Gpv@@ujm^%Y|c#eiTN6>$f>8lU*+r z;JHz!_DxU;QV&2yQ6eE@<$w#T-!M3BWaRTLmi<@KgQ*MOA_5BvMdK4&Bw||@%g)3?M*a;1?8Y3v{fVAjVow0a5gXW zmGCr4J-G}})M&i$4=xLu$d|9DJ6L=~3m?2Rk>S(j7~}uMjE=Bza9rRooA+Vq?|f`G z59Y0)R82rqyuBjB=nCuDmi!3-ni1w;Gs30mrG6NMip_9<2U+#(KqWk1#NH&x)6jBc zf3%^xD;L+5s70Cz19F; z2-~vanP%f3v#~p;y^N9mk$J(+an{&2DYsuhy2g|ami;t}bupISW;Fi5Z@Fm=><1-C z(YTI0*q^Jd{-sVPV#<(wyF6&}2RI*hST zU*of|h21g~wgtij+6@hVYc4b(AE|rwsZz&p@b%BlOvO77x=4r#_x}M{^9+Rv(+HR8 zJ36Qz2H9OZE!~QNUSe0SZtL#V^8lwkUeswt7CqPu5(~ItP>N*J$0TD4$(0P0 zocMUWOv8x*LC1?Bg#491#i*Eq44t`i`aVOnO^+r%+zyqg5p9b=LZ2?LG>T8Y>g~YGqYGTC$@V&^Ca#9)&;rkDLp`LcAWdTVAIY4jwllVs( zl6e#^1cR38l%r8`JQJ}*XBr$^j7e*CK%}zhSs#0)3Cb-l&!dy^V3EjP7V{%7{N(_- zAxNGmr^eG2-pp3EO}EaNPjBSMG)>F1r*vM(pxbY!8#0%o^_uSZ(ozxZvKO6V6o8f*r3wwlBAp#?*L+^gu zpJ8ViA2yElW(r$q-B{y6PA#d^NbL$iw-jm{o!dgE^SF|L33Y|=QwD_eMh^+ctWdPb zkM!9bW7*N8^y)^1MwC~-W9|_9%pK_55VuPE{BHidU5YM^)h<0mp^b=I&e#+mMO%nf!|IzsIaRKl0VI(I` z<^I;!E14$(tCDnGy%c)S8$oO*dqDzFj$l)i_Me#_r$az{!P=dIKM5kkZ{{NLOuw&J z1%~P0*;|oW9=aHZVs}KAi1{Di{kH#jyT%`P$@2$5rsMP@zSUOmyyHnV4ueUe1~$3j zn}IdysNvfwt}5DTiRsBq`<*Z6zXcV1iNg~;;Jcpt=qN3%m1o z9&KEoAJ~Fv*KJOohl}x;HCv0e0un{8gvYq^RgU8g>8zpTtr=&puC7GRMqO^o2aIEk zBW`Ag>m5`&654+2v-wQqiO;AHC_!h7u6EmeNbj$XTnf@^#|Qls4^rCdXMx%)1)EE^Q#Rzg*SE!15~U1z`%U zQ1U+bYGT`Hf5=-b&8l%0T}-BIN8uVpNmz&X4h!l{AxtupXTz0WHrR#~8M=0&sky8v7y&cl+JS zve9(21frB?2Q(zCTpV-?xbAzkUL-;2(paX#HW-j7BR`QGc3Ys1ZXb^=SIk((rX_Ci z3(N;EFQ@)2QTZw0=J)VT@2JaZ^`nM%iF3ibk-IAiPZ(Of8DThUqQ-#jwL#^H`{b$> zx^>@RIUnT|C-WJpezz_0s?eZrL@X|?Xz_B2@buL+$|q#x=%Gz7pjqJ+oM&^Tf)ip* zo@2Z;q03bF(SFojhln5!gQMpCnt*CnB?S83Jr2pX3-o9dBN? z2AuA|BdDlSCI5`vb}0v^b(uNFk#i`FFq0d=OAHz^A|G1&xo}!wQkdPJsiiZ$M+x^_ z3lrcDR&g4m^)jjCx0oeP+m>!%Y3w`~u?UOyS)U@X$j{U$JC<$f4{@_+8Ik&UfjFqf zd2vTU%q0_iC6<#aMvJ?7*S9D}7J2r7ZxS+oPGdGc<`fP}X@9HFxix+kIlA%P8J}1h zIyxd|TfM7n<9J@*w1vG#@No3&wxx*#wY!2bJvDuIW*ddS{z{1iQ&SE%R&XhW|7h6w z=+x~P8=Yjp0YLFNl=dgP>!X!DbfYQ62K@Yjig+wA5>%Ma)Xj-3Qk^n)`p%x)@q9#= zl@#@fTO3HO5eT>BNboANAo%Ke>_?373J2u2&c-Md5DwuEkh((_Mv{jpJqI2cB;`uX zJGSu*+t=p%6Ok^zcu&?jYS1tC(t6~pYOe+zXw@nEWty?MQKN6E>{S!R>XjV)t07$a zRElPH3Yto^C-+$giE_*uQ?us-Gou22PQ|22)ODYQnSt{*$28B>yo_a;0P9q&1FqJ6 z22-*ZRjTPokG}ID1#hI_`UH2y*j=6Eco;f=y?|_BHP{$FN^C5xon$`%U)+Uy}BM zJ85bDBC*3PDX;ZZK66H}Sb7-wXmS>FWRGz_nXs*_OO_5^9S6lDDE4t(Rwh{I_{Gg^ zjI=ksC%CW4&T#)c=?<67y`hoTvV$PVxFj?Uj58{aBl^inyN$RB+HA5V>* zXpo3IO91Hkpu$}Sk{~VUR!djw{z)ChgY?b1MET*~6~4kZGwq;d5F zjj-w(O`)^hJ%1KfN*Z8dEMD$P%6@P%wr~Ff@tb7cxdre%YGfw10uHvYh?$MGjW&mI zvR`>k-0oc9s+>J!v<3D}EA^|CpMqkT*}v%*9)`NozLS6#stzcUJTiZo$r9Pmedq-K z81==!5h=_;C;!VIReo2C?sz%XCZ9ps=BT+C%`Dfj-*~&~e-pc3TCSKZt|LtLg0t*B z2ij|OOym?kbdH?Dp+U9PFMV(p3K!04zJbETW>Kfbe)jhfoz*@n&}03j>suA1A6pd| z^oUn#zOj9$smG**2KT&-Q5a8kWRd%n_6{qYs`6{8dd;Heew{lmKn*65GCRSOKCDT=-YCDUBK<@l+y;F?sW`dm*gTvn^Ci?yUT5_brwLri6Y< zAGDfU0tCCs_hrGpeyyK>+Qp#fcRDIRA5O~Jgzl0q5J&(>Jdf-=Iv(Uk6KDwUoLDta zHFu00XsbYXada_*+t7TtbZ8s(07V~nZIa0Ps6#MyH?;sw&hYy@zy=4QdlMf4Z{J`T zfHU0c7F}(>h_3UT8+!!&nyg!>XT;9#`Ip!LI*iDT<9nBUa-IKfw}oc^zT*dFPuG)n zlM_F--Yv5qG5|#k?9~^$lwnW!bZ7=l^DM6pjtm=fYou?19X+0XPMRs$roErnvNQ(h zxIo2=)opEdsFBFzT{Ir3zOJ5MO-dB%`pAZAm=xj)9nS-Y71y0t?)wjW8w`S%o#2O8 z*B>%}rN=P(?TTK7A$TIh^vKNL4xC4Sner13^`m9Ib2-!ULj|{NL_duoz7{R7G4IA` zdb7Oh@NPQQD`N!pPz^z8iiL*+zs5T9N_q(=b#;yt6~VN=@|llC>%JcIz?X0w zvc2uo?1naNHcLQc(s>|yb9-a-n#3{`zoW0ip+feF=DmdW_bgQ>s`(QA7u{~PYXfci z-;6&LyL>1Vxjy5nEgA4Ur2h(=IgB~QO*zOshkU+QoWhEvoXXRU;biR2)alb#}&7g}oJSMR!D&#ot26w)C3>FR8l;X3X zN*Tu<xzpP?d* z*O1A*9z`HFC|<-afc=9SiP;91yL(v~wiqcaA6c6cm?UylJ>!HXjU(+_%Xm5Rqmwte zFpAB4zjB@sxsHKH77rE-p??>heXnbU)GQS-zi>?{I=MK2rN+ zcms9*yBhva1sH7dfEw}^oZ#RtD-(7h##0G&Z;tuVT=`~kpYu5eEU^abQp3R+2gM<0 zG_`ZT2qJ7d#mcT%eLLCb`>B_OstvGE#$1pjLkCe z9B6CXvfBLm7u~VFHRu!UXI*)d+nKi~@}}15rCW3`FF#Fodr+FW=yxSTT8yc(#XT;g z#!dl=f^9Podm_$gq0th{~pWEVCC)%oOdk|Hd=dkVzoBPmmeot z71ze{2j@Kt+jhCoYV#3AQAWRttI3-Pfv9iS2SZm6`R$=Ivos5^W}nmF9S-Mk#`!Em zQK}j1R+1PpI~A1fUtaGC#svq4>`)`9*}?CTQFq4KFFIz`|K)D~hpE~-apoeXzB<&c z%83#l80C9@cI?H~5zOh;-#zJJ?@o@Ee1WJ?%N6iHIx9XfUc9Zi9@Ht^*s~ZdD3rQw zZLar>mGGs93;Q&+HrILbkLz!=z_`f%oNE6Xpo#>XC$IdOM-9L3T{FEI#o#RY$71|L z_2_S>`Cmt8k+1DOJVkf+yQv30;@c%}J>#U*9FE!7Z$>qHx)JWr|56aHtNr$mz5084 zr$w<46URkmi_~j2l@-)PN-ah3JJA2fD4?ec{np3WE32DSn`FSIg}Zwe4o00nbQTSf z$7$xOt9`5&%N#CZ?4PoM09)D&e~)6tTg(G0vtiBqMZSN3`LEc)e-^_(kA9l~J|!h8 z&SgyN@0|OWkqNU_HtGy32sZ~pTzPWVsD*3F5#+?Kb(81sE|e~kGyeM5cE3pUQa(VG zm4^O%E&pc*{&mVfuh?#eg9jJJ5vdRSG9LRSaUuh-#Syyv9z~nZ=0+vxuvZ{pzgJMN z;x1ZmZ*M`3(AkfL2)F4>+|}-6eyjfrkUhslS7C;xxt7~oCey@$5Ki;o3BxYF&4v4A z{!KCa(|7$ZcBHk##Bz4UN`G(PK*JAuJxVMIdOhn^LHr(CDnBqoqT%8(+w|dZ*x~LV zQ*vmAHaO$6+(hwN5Ekb`0C-flc6%)7Mw%m1{&|6(RuOiX~sP^J|Da2^HT408rroQuUKdcHfr z@4&h%SF&H#uCiI}o$mbb&GMvC^S_66mofmbQL?$vh;6}cHZ1;Ak=g9OfBrvKKo#*; zqHBs_8pb3idfk@lk4lxOPFH>V$3I_SpdTo@K>_f++$H2{$K2@EfBUchHsU{**FTTi zTLaqt$ayyxKA~>ot02M`>c-m2{~sf=A-AOMX?sAI;_Ch|!XQRS0#2o2GilodM#*`i zr^2w8cmev=(xa`f-&E}W?APXOG}Zk2;8fGDGcB{m!(B-#no9TSf@tE(=d;#SmC;9D zW45w_4huVF2F)|`4PLHKj=oTjYdvVR&~>)SkNjaj?ufS(58#O;{&%dkL=1N}X2K75L^gOs(f4Dw)Am2Vy zV`E-#`9X_XYkp|5*rBhk)Ole{Roy;jk>xw9g79Ki^O#|ai=X7xao-}IGj(>Uj2_Q&VX zKht;FWjc%o6s#iO%8Y)N!OV0`EptEg^{%^KIqmGuesbGhwtaA!+uRg(Zik=qv-e=^ z*Y13OIk%g!+^ytVW(fA|0*!u?d03rin1|!O$FR!7D{FHy!_B{*%mnbKz7n>eJf6KE zF(%!Tl!@U#NN|6h`9J-41C&UCZ}+8xsCLm3iXy+ASS=trTM)JjptO|!BU&yd>IWtY4}&h_rv82w@ufp;iC{8bzn=bdpukmu$j^PvGw|-aI;Ae z2psqG+fec}jfaa89?s9U{vpbVE;)ME0lJ$wZ7?*t_&2NP2?j2}X_45qX7YiQJ*d_M z#(muvky34k^ZnER^_c&L8x#-j1C9HlsRWyBr^kNxTLc)q03;L?HejVec8$4p} zy5?GQt~p`9JQ9X}u&+$OC8`k5tKDI23I?QcLy^Es{m7fCY8XfM1|*D7Ja7Z_=IQNn zB0Vasl-GqhLt?hV?iK!LXcgEYl?xhOT%?lTJX3?5sr23wcG&hfL)8_csPlP4BPMlg z(nMa_m7w4Y&C(eFcklp+0pMD~pxfoyKQM7#|2Gw+3bZ$(fJt-<#XKu!z#;oyUgrqc z?D;>J(|;;n899#9@eij>byv*s=HAa>B2*$*pcU=dSb1LW=`Jcm(zE`{moMUIL~t+t zx`;CyjsZ4oI5q$+hciI z;JKC6vlR}4D|AmS_iblCLQ^%)0}s-8iUgrymr%QtDgQ4nn>#iB4AguJ8!B>m>Ga{? z;U34W|E@#5BUFCi-*Y90NO(^9tP8g((BYl+At5_5p0DmT_y1N77cU8cZu&?D0NG&X zzDl|n7Mu+D40Q zeIxT`ziq0eKlX%G3lx){AeS_8C`T6W7z6tpAv9-(RR*84;AG>VB0zKk>!W#hWx9UU zeE3x@>S8-P^VV*fVFb1p_t8hYcUh~3pnUCO>f@F@1oT2njD{T{3)uar2OgY$cw#qO z@lD*rUaw(7_(M%42D6j6cLdLXtz9(nh+iM3FF*?X}Z&Q~8Tg(>hn8W3r~#vl{{-he;#K;yXa!W#po4t}?hxeiI5 z>AUVsmG6xxFhkj+SG#9wyu#{{SYwa9Kg(NZtpv z!hWMG>8`AxnA>k-*Nx#Uc`IE-sxz*YqakTKH?Q-h(B0E6CeI?zlL@VcE@t1xSep>v z%tB@4M{i3aZgYkc?7puHR-U%c$g>$~$F+vEU7fytD zz61{}2^oG;Pb#jIx;bCr2z5+&`iz)1nLg3-nFh76U6xz6Xo*p;OhPJ`nzy;|IM#QFetd?bhqV+xupa!do#h9(k!Q{| ztMf&CjvQM+erH|f9=|-%W5E`Z@(u&=V4cUcIs-speGMo$FbD-i+IW1Qr2#jd`zl$_ z{u+ohKZ3+S<(1NeZ8uI;-?|E6vt0gz1)ALp8^|-E7S2bV#qjTyS(TDWPLgnO?Bwac zmmyJp5&3xS&S0Pi0LD@Hip%0kS+n<9@&5t`_MGuzq+^7@eeTyMlMS`=9)+<^F^*u% z=~_FZTJOtK`Q?}M7kdr+y6@>{^;r&pt~(3O%_ugyaGv;QB#O#q80^eLe8e`!)R=@V zH$CySB4|soBEf_F+p&OPRJUUG&iNkq)(D@^K{%B^=Y7feB%YjAa8b1U;1{Qh4~8ti zfvkR>uorNTg>}ln$^|z?6d4jMcLU#4R(gp0Pd3~8*Un>ooP5+D? zIyl5Qp2|OD>Z$&x1!#HAW^Y6Zd-e${gUrufRi+yHU;p(rg*ztT@kg_~EU{BRUsIqu zT$B`|7&cvYXC@2>!8i z-kyE>P(aVKCuBTZvNzu7V$|!Tq3hcBqR;3*gMDS_P;i2*p>tTu%H@2Mr(P2}<$XY? zRHa4I6H$zx@;V=4UnP@{ywLa?bMp;mTeJAGC)hZTY>}V=!WCg$0D;mxMiK9nD6|u= z+qUhRtrTw%l*69=d5hV}e>kjs1P=%x`J63>j6St9pb>BBwJdcGmv^#9^=d6z(Mb z`zZeVV*Y>2pf(9!%=vjEk}&+g{~Kd_}Bkt^%`n72rLD=W%nlG@*zh7upwV zsILp!OPg^{PRWrxY&;qik2DXA$h~5xWckG)=}{$@Ue5S10H8kjCpbBzwP7DQYu_tT z#?~Mh%`83NgxM{vo`oo|uOwBO2pJV~=}JE+~kJ@5VA|Lq?AbH_Smji+kE69OQC2>&UT><5Ao zR(#$3+2cQ=as&Sg&=0s`p(8RSHC2$GamUufJ;K100EQ!MWI^rac#gycz$t{1mRycJ z5G@m>My3OHlLHO=If zoXvZd_9$xZvJ^_FN*sXW9|3wLX9|rH(IRVaO!rH15m#}XTf%gq z*T3rjuyOL>7Wk{dx<75`>0F0xmvY2yQ|Hw&{4w`#O$T-z$qC|mHpY48k|tuifh^%E zO2T(s{qHzin*!%dQh?wY)38w0YTUc;$|P+8S^rDPec(Uw>LuPoHBE9`a(zLT&NexcQM(B z0&C%)j%F8bYCN$ENOZ08I$J$nI=&gz*U8OS7V$zBJGExT(nkwyxE2eor0%cNs(3EL zc6=xux@q?gw|vi{Qn3fce&xoco=8J-ZI`B*o-qDX>|{P?S?lZ(&I-_8a%}=e9bn}; zLgggV)l;AANgbYmeY~$m3{c;Qv}QFm-+a#%{Ab?u=H;S4aIK{S7o*=fM#1m7SkY24 z+7x<6HqnfQEb5N;reaQ_+WrsbxKp~7L5k3j8nOQajqvY-{Ns##v5n$H{+JeE1dhi# zyDD{$ zvu*fX0Q|@;$CY^7nbEXCH12XC<-1hw*j4og>igaM*Mx#4VOA@yxxn^#hk#|Qn`ElhBQ$sJ%+g*%wN1AH-!^G4_DQ{j^M`)FD5Za=u( zDORMExDd^4{)?0L3n8+^x~*Un6PbpZej{W4`8Idv=co7RE#xa>FFe~$w+62hUDr*= ze(%YWE^VqXo5O7ity0v^V8@S}jlLrkvy>RB$zfR;lI5ItA9DFUBH`%g2nuzRJ|4eYWQdhIAoxqYO;nDj&oS8d5H%FNwj!`Z?fBFmQn-yu{LsAOt%6!l= z7q)fLW^4ZarS%L93G67v=bQETW21-K*@3hT?-y=rX)x2CfSU78SvV}(vyY)O6%Pcu zBNfoewgx7n%?D za6V5r3vlRjv#45|rWd%jZwa5Qx!HT!yQH|d6BER{*8e+%q;2lpsL*_H>E#I7AVnjk zs~Pk~I)hDcTa$jN)bo%-JNONqT&ZZEpwWD@Ax2jZa~L@zjTXtMvMip7s~|h;`f+G} zS#6NUu#zP3%=Y?H-S*^?JJ08MUvFTDWQ=@O-tCS0*N$xw*LyAw5M6(fbgEKcdjdR- zn=}^bTGI7F`Ja&~y0o!p9!FFpGi1O9ipJ4YJw zl+^1e?#|fTxXq=8ABKqiGeHS%@%Dc?^XW>wn-d~zP8{n2M&$2XuboU9 z*;$^6$7q~iRZ+`eNz&Es_?sqnR7?17j~zbQ3A{*FC6^`C6Gb2daIWf-w2AG`;vvm^ zd??PY9d?RKX3lfQ^6;X6AN0wzZ3iVV!MdjW!8}6Er10{A|6^8isc!zo^G7i-1`Q2? z!DQ_j`x^0BN32y!laILiU~)b_dZ3PV6_Um@zS9Wxn6-EKdsh}q0}J3WJuxMHNDK2` zvFwOxB6|?>A$CM&T~8153pPuJi8sO)PSx zEQw8T^O%o&nm%9a8rM@fV;ZmvmcJeF9j@&0d4yHh(hQth^b8oxz1K%|vG@u-!U4l2 zFcIUWjt|ZFR0#rKZtm%_dmG!b)z-8;M6PtRWuC&USZVtq>2)hD`kMy@wiSg7cgNI@ zt@FxkF?XW3Ygw!8H{?WL%PYNQe#Kb>nI9g2gh44dufC2w5&V(g!)zW_Sgu-!D-Awh zie#C5@{MQD#y7#?pm%ZL!|oD-s5?R?-AYtd0~-(*Yz}_h&s_0nd#oy z@)tE)Kd$A|58hd?0#aaeAyT%}YWHKR^!JGo6 zz{EH5RMpWX#z=3fbw|PC_bn))ujZ_QPz-DIemidXGd`2~!DgqVwd_%PPd*~C#}Dv^j>6Q-J`r8jyUp$ppd@%{fJmcjt_Q;8h`V(5#cQ}MFo*ut9bio zckJ$#c7uol>zSycsFTWk-r{WtvZXm(V`m-CzSt5?FQDM2`!r0Hc>4J=?N0V8{QF9! zL+g|E0-E*RxC9fPp$I2}EA~{)&2?ap~Fuv*PBmg9pB!dl43SDLtR}iUzMoE9RwLWJUg5XWw^Vt z;lImsdn^cN$Qf_S*q3<)^2H8cwti0RZBNCg`vBMUBVxv~J0YJ%l(FI9rbDOS_7+Qu z%Ly)>#)6?T>YECcI<}xT%u~$3NZ@R}xA<3j8B3{blDO&Qd_EBh64Q~>m~E@Ev5_{l zG3;I+MD$pNfHZ%QY|xU96cHw(R|Ukfh6mWWC1Ge2AF18Eu%mW=N4Rp=w$8m@?`uw7 zc{3J*D4WTHK)w?$c=sV37l=98}6pT_Kfav4r@ z0~h>?yFxf>1*=<^U>wbs?`_VwWF1dYtz@x|kC|pVOPoM!{&r$p)kk{e#5x*^)igkjPOv}=I*UbI@y>>gjyadYUW{I^ zVNeT;jDmmYdaSoEK~xH^#GpReVMOd}Y49E4aGtFjek}tXE~#qEZ;Z}k}@h{ zH4=PT86pUUdQasu%8^qHgClYr?z`^M2*KCO{NtKS)v?CEKBu%Pg$7wEJB&M&IC@0M z_6snx>RqpiG+SF-ofzFZx5da#EVL#nV@O%H0?DMTcHeO-?C1hu;BJ{tY?lB--Ckm} zu#zeS^S6ua-W^nVo~Hl zk>^#92zCAKdi9VV`mfuQRaw-L!t(xl6k&YxPYl0_D2FD4`k-fK`tJPAG32a^(z3qU z%kya&Eo3QZ|Cu1*ux+pfGn`S3I*Q)ixmlaWMU8}--o+7E7G|@kTP5Dkw}|(>%?Ztx z8agT;7ErxKkaW?d|24Fb(KISq;mZLBr={6O;Hal{Vy&S{xGiNKs4W&bd zwqif;!rOy`f*7jAkM~Brg-{0yt}ebO$H+a!9g~GFlLWEtj0VCMEvK(`+51uvQgbx* zr^jsKfp}EmA7mcn1h%c5`g*RutGy_1{b9fzFosIq()gRtdq-I0ig&I-u5w;+UNzNx zr4MnasY~Xba^2yceNmU}8`0F^;$$$-erWC~xzExq`o~;RTc;))#NqOaTB{tiWCzY@Byvc4QUJDeJVBk)Aw%SA}+xxC&VG0q&{-b{~qq`hr_xB)ufyGO9%%+N;SFQbGa*zDVyr+KCOv?Q)O46^~59e0D|B#z}BQ5m{TtC~hPp(9@ z*Yx2_Y^>2oO)ap`qc7`J%kg__6t80Uoen`0xpv0J5zL=2^Xm?<)I?3)5q}0z#9dO; zL6bq#L35aE)@Zu+5|tWyJeVFO9c8!^)y{g}3CEb&WUL;z44cPtawmv~gxtv5;VH#X#1S*-e9xmL4;khw=+|*q(W#^V!5fA4)-{{{rj6n9tX_Fa{s4su1C$ zBON)VlH=@!lLu9;U6t^dp)f$+52;c(tdFauBCOtKwu?G3Q^Z=Wq@L4HFg^O2B@TF@ z(7Z)$78B@ts3zbo`5pZ4Tgokg18MSl8f>t4kzU&OYX|bQhTn6}{f1bWM?88VoM-6N z%nmf=Y26?q4un)5*PZXCUR|m0Tzs3_KP>EuJ^kUau?%FhIAZCJ7kim3*~dJ37RJ71 z?Mt@K*%dqKeYV_xe(rX3@kEtxs@#?Fiud=jNR00fs8eoj+u4P(PPD2ACYP%p+U&fH z4m+h_xjIK5m-8&06kpBA`YzVg7DqVtB%2w$`rX5Y3N<-m7I&`Jqdyh*Gy*Sf zi+@7#JA5Dd?;&0%yLiJj1JfGHPH1#lMl;+>UT%;RV2W+`BpWnt-M6!oT|^Gr{uR6@5M0nR$2xNkU+ozwDAv zSovUCekmb6oKW%U>8;P70heJfY?vM2-gpsq+|5aGUw?CX#vhk}u1B7hMsX%$JZSE? zm#g%-W#~oj{MmwUGuGy3XfM_gz5YHxI-g?aaPdhNA4Rm_mMqg>(Wh)H_PJKgQfr8q zgeA8)BOUJ=o3j#apuxN%OJ|7K$;uy&!*hy+lD~?8B^(hNmi$Bzp{S@3cgo97`ZnzH z3YQJw#D49+^L$6YLG?h#=yiVLg&mTHzO?C7w++fc`h4Z*)eBN@k$Hv3X1WG5HoTf1 z5?I;E#R-?5uG8nY2lz|hfwU2?vxDG-6(9tY-Y-??Y=N5N*4*-|JG87h30J#?mAjYW zfO{Wrqh9blP-aUF+Q)o?SX531&K#`eyE+BGm+#P z_^Z{mOdm2j1>9G^PC^PV2k2$GXBcM-PzF@7p92cm9HXq>HhVbq`4K)+`yA87!A7ZK z6#h)@@0Q>pPWausr+lMC;&a$UhkIU6ukgKuAsQ~vqzP}SJ(GMxe8b5I-P=wiMqN}J zQiqf>s`4)65;T}IDSU^n;|{%equ3`!yDZzg64Ahuu^xe)@Vd;aB^$IA^KHz&`2p;J z^1842>dobQ4uLV!`krkI7AayR_4!HssrY19wy$>yvQU9>%dusubv;SVBAQv;hfKt6 z)6$1cstL%e-VdQVEE6DBFy=gblN4>_PYLS{|TY%kis>HtooQ{AN%D4R^ z_o-+l@%RZd&#zg$T1R;Wfo%y43Vg79k;K||8FUbD9HXoQbx3QV`#M(=Su zLljPk7{&c;0$D9LV>Bb07zUjy zMQGS{4<`l&YWT~%UFe};%ba6hv!4h0JqewPQ@Un&wDn$EfjBPUlw~%CtpaXCC^SQM zE3wge!S*>6#=!cG>t{qD(-$v=f|%s^P!&uR)1b{QXQtO^_p7qJ=*veDAa%r;&nn<) zf^1_}PQ_rp24P6a$iUPv?@Bjhh)k@xeb$Xp*mr$&Q5@BOkZ(!^o15 zs`6Gnhi#d^g*gXg2{$KmS8E!<8PP2u;Tzg9g>UBy3h-4|NZGU;^OJIH}nZ(UyE`K zB7D&rp@(a^^%$D38NG!@s6&Y-dHO0+XRi)>22O{=xE+OhwT-?CTV#?Z=f&GFG5^Kl){vD`7Dwz}`47g)bf zsVjmroC}j(dtcWZBmlsnM_S8vuXgU+nizr^Cb`@D8a}R$P6WqIF-Ptk?Cj9km zA8o@3xZI%;QN*AE9wRo$b@19LJ=*4&il^(B0qZJZSd%dS9A#9K2yWndhT&yS(-Pwsa-<-ZOkJg*?+o=ISEf7$vU z*`TR&7I}&gUjO8(i3pF8@Na0pCjTXr8&c0muNh2c<6YU>q`2~O%Axol5ouWpc}m@* zvJ_o`HwHA+#8e@p@hMf_yd*`OfVN;Oq4m3MsWQEOCuxAeZiU8|#3nJM`Ok}D-*dQiKGoG$OOmx)EzLiAN8gaeJ4t`|U2_*8yPQfB$mKh{5 zDmZwQ&SMJK$4t!3nN`o+M**$IlTt90YlRRry4xLBOIrJxT zS5q}9zqakgRBWt5R6KHw!}+l8o!ba?MkhHp$eM~GREpzzALP5(V`8zxW%y&S_r?Dtf`RBECQ>M2b$PcIZQ71s>fVFyD@p^kfon&b*In z#s%4psfp;+G1PV@)lU@O&cV8~GTlF2rHT*M<-^%m=i9#|sdO`p(Pk+%+cY0hq*uw* zMRezp3DS10ky>UD%I`65aRg>`QW7WnZssM=xMC!ptV#?s!`%>7Gv60Bqu6s9(W%=A zzpXy>4LT!5fr8jXp7ZSEE)9~*;v)E_YDxk~*q)ZW)c!Jv_Dgn8((Vfs>sd#k2x~>h zMGZ<-4?$|lsUBJIEIVv!tIPDX(~TZbw-jvo%Y|?Evt&b?4l6e7mix-rRhSSEI#|sU zge0pV$R@z?+mPrcW39cCm=Bt%gbs8=10ZH67Tjx@L>29m8DwDlF<>(GY()?=%P z7Wqcb?P6<)qMA^sDRzzXoYiii@$hokF@;z^2R=aDSAS4vCiB2etj5-3ZxblAi<&~R zaR7Jb`116BOa*NB#dtpVtfkmsy&ZRfQhnBT%CjqMY=84 zV;S*=a2dt=8ATGb9dUNxDe3PAtU7G3P)4PC2dx3;V9)v4KrjCj{T}N?$#&luzNm~kij{qJwpCyfEnm&h41LfGsBKWpeH{E(T@`s; zoN%T$Y~P1hsUFbx;PGN8G*vs&eke+F&i+ykEH00*^M1;LhD3{8p0af8qJ}$T1ynmB z7`^K*Q?AmSC}3gmkG;)EF8XrfO}Sh)b6RIf%9Zor^zHiEQ<61>^Vj>R*%-%efxR2R z%1#4~C3S$Gd5I^RgT&fOpFn1$mU(gF8~n4#2yA-L_|eD`4Em;wvKWQiK(rJgbs2b^ zhl0xx3e=E#-Y(YdjZJlK=|fmf-}i?~dwEm@P7{f*=!k523a^#9+Y(8cIm3Ga2g`A%&mc1O=R332@jdA~5)$XCx$Jb0LGT;H3bV<4g9jpR z2V8C}p;8p|nm(cXWv>)X$@<8oZ#bqbGc@g!ZeuQMo3D#c9v5xosGJJZ>*!4#b7{3# z8dQyiTR6Rx56C$*^1XIG>aa>fg6EbLjBne{eU$Vh*I6OHl&8tXMKw^uL(Qpge=wf0 z6u^Ec>~iUI8TCqcGH5MZp3gfhW~GDLW#)4VT8ir*`a7J&|BylDSdos6_^!$vCyTYv zN|$Ax0@>XtJb59r#GIC?M4E4kA_$6{`7Na!aq0!y<6a?S5VK>JD2ao!;+94du8Egd z;{atl*#+$-v03PKnxD=(T3LV+=H2unbVQ(*`#C>t!%-IoBpk=xvbhW8S+63m09dfM~BjXD{ z+mgiU3ge3C#-?F=28rvVwWFlP$ltmDD z^YC%XiAisas@iK4HbyY-$_l?K5`Cyr7Tlf{t#sQ^j;rB~tJQoW7bUy?b=g?xSH%56 zF%=;^`jt1C7NgXic*@3RH=DAsj-}Lcmz9s)+MT{WuwL5u!|kqb9L}_*y=Z~&JHHo5 zj>a2boJPK#+IX{TKaiDQ93O9PEA1$$(X+iQha z$h7^LT)uGf>2+CD6Hp;+kX_=~|L-fh;|&e^hDoj4GbdyV*@sU4*W>j(lQ{Mwyf@IPQu4^POZS#F&e z_SaW?l~hU(<{YhwX3OsT@CcgB(`k*}x6eITErDvS8s6mTL?H9CG1P{H2mHA?geTlx=b9kKv(m2gvA@%sskSN2)L4#U6GABAV|>6;_EfzfSuV!8n7$uj`bH6T?Duv=d&P|LR=fLftUWK9BZ$$8@HHdt z>H1}P2l+@wBwcRSLCRZxpCp3u9meV{nXsHgN@H!YfFBJtns#cV93 zJh6W?-IMZHXa#!o!V6tHF>^^URC7t8gUyR2e%&oH$TF=T0sp|a7qJn1@bR%EIMb0Y zy8-lAFv>GD&~PTWYIU9?8^EedK|7H3kAU$d3%~21$|x9PbGkb_36ZYh7mqIP;7~Os zr4N6*VeDJZ3{X!v+Uy_`PPsz%Dd1m5Lc8{orgnTP$(WC8)3247!%$BSW}kaX`cyu; z@ILoDH=7D_O0y0Kbs2fva85jx4{_XmNoeXi7Ch9-$ygQ2*z22>zGHeLECy0w(t&7O zjI&jgC#&zROZ$*ZwM|y~hRl{m9oJbg`=&@y?xq?=>`|bqw8WQ)RacFR7MKzwzL(E3 z+>Gs=H#7GB%9#r!FWL@nIR;oA(Lc3x2TMho>5y9<=4EcaxwO8|2Y~8G{T{kL{*?ZY zmjx9WPJF2{4-+GEDhKhskg#C!t`&p?G3c3OJ)t&G#jM88ZEd+Jc;D$s#HT!)2;EMx zu6wk^_JWs(72E;$12fg4+{~iP+oRaSfcW-!=xjIMcCv`MGDpv+>uIW!pjX@hdfjXK zCVt^DCN_+A$1xUnjkNUuuQxt7{`_)?*)Ba}AKKN@9_||H3O>Ri{q2`k6F$`{a3^i< z*#m2Md>~~Q@gQ!`zR}HLGS6oPu|Uyx3}wS2w2>>-lZG~eRkZSd+(tB1w?Y#w$Lz-XV8^N$am}t!>(VL#Car-tn*pKEKh!vPD=PhBm+SWIeF#LsX!$ z-tCd}ZRH6Y=a(B6YAV9V4DN(nYl$#SR9jg5YeTaDZjXxrwQpBWms-uSiRQd8xA%?< ze{m95se#UjS2Ciu^60zL-Ir#vfI6{Usgdr8AmO9=S?I-AgxFF($)1Q0Jz1y3C`Hlu zw;Scd7!q8pLx4*C?ZsQpTsj!v0`jOc^Rn+c&I7AXY_9Q)+~17J)#{!22>ZaAR{3tB zKs9rfK7Q@5x*`?muhO+5FMiis1IT@~7MB-PMNK9_sc=6 znzep=aXLzM?=kpoDleY(Ql%YVl@;?oK^6|J$*qtLtDv&z-iq1-oy9(Q$)Pp4NhYBD z-RabR;AH{>J2o@rP??Q&htioXI}}bA@X~~cIf(cN@agpH2Ozu}IDsOF zfHNCcl5sX_msEm4+cmPq9yoKk`>hbQkdzL^eZ;hP#;eP897V<_}G>0WL8-|5I5Sqt7Hmi!oI?&!FQ}MAFU^%l3!Uky^*IEh9FM1wKS5vw-$e+ISkjW_Lp^59? z_o-oH6_2b&rdj)jw+9xPdRDN>s0(C}&w<@%DAlmQJ!M2Qx1`jJfqpv>X*7z}VqZ@s zy?LL1doU6v>?@b%@pSFDe#7!eCZVwPm!;P1oDt>h5^Y;2zIu4K5Ho|=!WypxxGx|d z85Dlm+9hBKFwcm#3Ecgu>C{jgQH>O3hB1e*%13q~e8k{(fg8dyboI^+;`H8;jpl!P&t=$L5gkcI1&ldC#E* zK}}wBGi2~suKew^b#^TVJO>QF?vyvPJL1ZF+k-Ses;~gabT7eTy9dpFe1N=n5%rA8 z3`x%?U7g-4X~OD#85dMAzF#p#{NGgsCwlfcl94;A!W{G;uAS6=!D79xUS(fZbd4GL zMc`5j^U-V43j0tPO%)@L9Abr@<@sEnqR(!sB}87!XJ|;LrS0~g2}nuJdwp~YnY|u; zk3Q@3R6jyAif4qHJu?1nD)U?b_#fl7BR@Z$h%U{pa2>T+Y@23*r#*g&ZuxG$Ut^nmW3VlL?L8;bdqnFDPZ@CHIv_+l4Ag`K z=A$w`79hfe%9x>Y({eAQhG~+vb@S>r1!oq~W`V2!wla-jFs=ku?h03yq3_Ro&eyS>U4;XYhl6ocYAWhIAmM+uvFmW}^pS?J8js2n|f4hWd3U zNqWEPWWwaoamuIcXt8dVAsVK{s_6KLz<8-O9EJ3{{jncw$2s(jqbzPuxi>$hLWTvOTO8CLT9#Wx7f9w@m5v~OeLd2$eO~c%UFJbORru}O zJ)!I`MYe();W4Z~hgyNc@H>aP8%^GA-@n$nM875^QM}*Frcy){F#1Mj{3QdqLx+oc zw$ofaHfLJc+@_XL{E(Fy_ySG?lOzDK{yZanH`QYP3 z`m0;qUiMS3)T0rFT*~$*?F_jE^2hhzy&a(OK-enU4fzN}WiZ zYRr<9UkdKFu*folDOFr7?#P3SAs1LTwDVhCmJupiibOEMv2?d?92)6xtsIjzH$)!H z+ok=(g-ua&B){8^U!xY=4BU!(e7k%`Xx|+Kcc085i}WYt`0zh=Sl0;s*|XgQ!ZYF5 z9jB(AUwKkp5*v4PeI5#DU+sWh1U*$u3q9`l3(?Fb83h-A<(}SAvY0N~`EW5Vho3g- zE=A?6sD$ph+R;1QXvMcsMPk<$b~W7V4~NR$8YIW4A~bwCPDN%@iwAsiMOaDnB(7!6 zbD5kzmVUJ(ls6wXkn(8Z-+rHbS{S(hwjmxaOCVfw7=J;Xt|g+s6e~pt%?hvBqC;lv z(%w=^7-N{`mrT>H<<XINLb#bDg9if4N{k5Z83KCkOWP&G%f`IKL3|ase)~H6;^N}NlwL_Z zTZGJoJO#0+-?16RW3T=ADw8B@0pcX z%F>qMT5->*#7R4I5AMTY%kRilBkq|^(eU%`Pwpc^;mp~sik<;KSOO?4b>d9DL~aI{ z*_qlYuDkpTNDnolj=Gr2uVz#Ea+PFnM27*m&TevO@ymhGj=8JEw$K4 zG%Q+b?MY)ynCPR;q0Rgz23ywYCqr2uZ`Kik@s4Ya6>F{b*^{L{Cxxf@#)IM)i}{k- zben9sG|)uf_qPRYESUm|c6Pu|uv^3R22Va5(l}C_SYc4UD!|UTKsCX2C_7XP6cWjp z%Mj`q&SLP$2p&8_WC0JDnpk*9`2=vrgw!Bul(MYX5d zerL@crrK(%U7Tt3k#>OsewlF;cNr9y^IcF_^ey*>G`-qWau37lY9xR#1`3IF5@Xc{ zKAAG?)D)pw4jl`DmmT#b*ErejFVMt-Ou3}n(+WH%f2Bn#R_!kK?%n#_Uc%owHbFUq ztSX-p;tMorZ)Q`keVNlC77;K#Yg)E1A7>ZROnt7_ zERM`H!k+~r0_K)An8RIyA!dCXL0@e|_T--zgrl;6t}I7z$CDvGh&7DX;eM>OT?{lr zg!C;Q5Rd+l+#xFCK8`g)MDymNV_;-2GSaxx2V$=Z-qU+@Ml=!-g@_=G8Zojly_y>R zd}}nGdN$*xJKC8GtuX#-=lJG(d#5EEVaWU%G`#FlI}r1w;r3yBl_lHvNU34r^#O4` z-MUQp+{%q?t#V=Z9u&~Ab<(sw>wV9Gl!Z`0VF6wNlTfs;i%f>D#@u`q`dG1t`5!ya z%xo=t=6&U3=W$bMI~FdF#G{%z_Tr+`d$d-9`7fqZzDyNlcl}O=C*??_#b<)C67Sfl zB8o5OPx{tw{$!}`XiQG|lvJ8b7dff(u0Cx`&XW)aZLhDW*Mg0l8+B-X%V>*Nl@({7itQU@T$LV`+Wk0mTYG!D zu(y;o2q_f{DYd+A%S88NdWjGgiR-cF?$quudkamVEuxAKh{IDT8xgkm>A8@e z_}(d-D#t77zc*2lr+l5q?-vgf1ny?arTi-TOzn6of!g){ILkU-5`MrClrT{ft%ta7 z@Y(&o*C9E(oblqMPKfd^0@&=eyWG^e^pxIMqVqFI-|a=ux2XqSwqDkd4FHS@U=$o|IPE;x?a58vjtgPvk06o~1d5kFU?+HNOw-WL-fGdUuCDHC~%E9u1(o=W z-VXMJatEa3Nn__gMc9(k1IeA_Y$mn+Cy?;rYuT{rGW*sVXa_`rucO0PLn z$(Qy%%$OC4QOM>#GSk+9&5QoJb?57IPgupQNbCplvlEmn%pNg)C0#uxtB9xz0)lUz zM4(^ngYU+7T{#wn1}5^x4HX3{gTWW8W2!UTOaw$KiqLJ&-HFmIr7zdp(fwy(daL{v zCOV=BU8gYk5C1un?JuqKOEL^imDxO_4Yqfi8D((D6J^b#B>{5nX{y<)*aV*6cCE9l z_1!P#O<(j`e?CA)@!N^JEVpfqN<duCc81$YBt~* zoVmtnlt59KPoDiL9v=;xfD*>g=yS=@B(SrmkQnPoA1gRe)ok*Y-AR6NF1h}3p~Dk{ z&&3ll=Dg2;>Om~2uWcF)NW8%|73pKAK&+CzTct)bJjIBF6B_k&xJ+0?ZhmC2Z&IuM z>T|)T8OGg*yPYmpaC=(+Zp(s4YRO>z&*fB@Fk9T{?bzt3&R9@LOrX_z>ginF{QKJx z>9@7Rt&MVDg*08eyjNF&7h!i2v`1+Hz4Xb7Y8PK~FAiT1v4jVmD$hlOF*2!?ONzRc z&uYYoIwn}i;?j{+b?bxQX``o@tMh#R^-yYw1{eoEZVf7B+wJpe{dpuPKGZ62(HB`^ zWS}K=EesJP@Qm_uK^HiBLos-%5<{AKlsb?%hyM>@XB`z)zxI6v6ci+NXc$_hLtu!Z zhAsgWk&+NlnxRv=W9S}AKuCt!MSj^h7_TIm^^7B2P zL~gOvfoNi5KWWTy{`4#AV^$GyS1A5$K3&{ux4!mU+onZzsrJuhJFD`e=@f8a*Wv8m^;bX7CB0ZBe&SFgWswm}-Rg!Zn{DvGApt|xm#p%7f`VTg z{4djQ_INmq+}S!94I}dThKxeh8goY$Ay)S$x}OnIJ96-4pdNk2vjZK*TR=|gmciM(zt=LFhZthI z{?24oU_*zT%%5BaIf69)5Nm9yAHz$5_pp82T>|%#*v>;s4Syyqq+mrfSY6u@zt2Zx zY?sRQsn`BeZMnUd?2M#)gI7le4b<^32uEf)X`8O zP=6~oFtmEY;?c7JDaPbqHK&Cy@bF$(QhmlI4&KYQr<=HG(-FUC zCx(Qmwo&zq8}X4)A)_Y*(Uph|l0fbHg9Icuhz63g6Yd{c_wAxJ~W~{7O6l#2Ib?`K21Nlm-(0TmOT|mjw zo!mlA_-J8@RUFYI>vC0N93D*>7r*do_{?ajI6$)Fz$B@t(;`XQ+BgT% z_x}1;Y))?)=iMz|-@ATZB5xL}l(XSCoh&!ti&giLzIcd+vg!jXr1+c}9|y&dVMsgn znG;rcDS4LU&4O?*1&mdNJ|u}(4kRCvsonY_h^3z)@73C;FtJZ3Qn^W|)u89G$eB4Um-RJlr z=v9ZK*B0;5Y-5i=xe{d zIM<+;M_}Y0l}4)KrP+@cWgnh5R8sQehPS>s@Zny>D-TrlW1mskATXLu98*m~_lSc# z$qhrA#!nZpuqUW4VQhhAya`?I(!^!T@dvAC`4Cy7Zi#B8e#-2KTftu29oLBOw>6q` zi82A)Ub8X1mS=d0$(qHg`~hhb*sIJQmku#Sa{+tP@3+IZGw zr35cG1Of0vx6$(Bj?ef3WiVXGI1Hms#&G|$Ev1CCF@WfO6!aQkE456hLha`b;K+`< z-4H{-)T+&1fp(XPFN{~(422pu&@}s~Wn`FZU<~qAwNI>#sJ#u!=dOKl=?&zbMEUC` z_R#q{vDx8M9#1t5RmBIgy9ZzqVr`0&g)AC4W| z`A$E{guzq_^17_?P44%I0c(ctp7tP&Y%kUD$3@W?f#4I?Uk}zqe7fVDv-f1eNQ{1C zM^C-ZJl_x^#x?958qu7t0q8ymUQrJ{#u}J>NQd(o2}-@W$l znOMlrl4Ri<_01r;h5(QT8DwbPRhOvY#bmS$T6#}}VK(OLmP9!fYw|(i@8WZG4SM1Wj9Z$D z`_#N;1?AUDTeKXq=~5I3<-nu<&b?YQ)#fKhnM;eYJh52^&U z-+p+85sGmi`f~eupv(AhTj!f&qt4*Em&pYQnGH2O@Ge|xgvcC)w)m05+rRCARSNGR zR(7avj}bE`DGtOX^huZ@wCKIR?}BO4X&7-ogxyLCC1k+aL_U zG3)~v%{AW5js$<1*O0PX_Hyl9M^YzQ|A5PZn{Q7_>ILTAE5?LHR0WksKYLHV!aeN2 zNDrNp;{mM0u-F{U=YRTQ*Z(O__t2c4^5GplnR!tVVx;-=Ay z-jfH2*L>wngGE5v_EEJZO_x8eM&P%j#|#>va^xs>t2|BJI>~%jSrBJQ`E30aWT%uWA!b?909=) za1128N#tfHYMH^P+|5bPdPxAlAE#)i(!Ji3zTvb)u!`e>Bi*r>KiVP7X2%ikuQ|{A zEj^}+;bB{t3ZzL+9Z6(s(O^;md|H=Zgc=fee`@jT^&KAFl~t^tWT$=eXfRs zPgflnq(+G#zH?k%Jz^Jplx!JrL9e1KeR$dzgJ6#}JP#p6f2OY-r5vXmF3aN1^C4ko z>um1n0Y$MoML95)NOlJ%twrHpbgyKW8KPn!98DAj8GrJzW%%RH_1Icew=3y(ywfBKUD9Yl)lIr`qLF06U=g z*`agI{qx~eXHA~~g{5-fYrl+K6-I8JRpB90D=tOU0{(7~n zA`-Ww)Nag~q4u2XqnxlW*$z(dZsl+n%34%;b@1ROL9@E*qPw%DA3nhNp@@CT336$X%aV0W#>kV?07Q`3?=AS+KN79T&sWNS8x7{8xZfrza2OIK z7-)EvfUc!)fQhgn7=(chKnHhpjVWqE-wQ>r9^_7E=j;DznFc~&XPQE&NK;9Q5Mr}n z=z1TN7kGLe{x$0fV3WO{p&&@c$Nm@%q2S}eTYTKumB^MvX)M?~p<^$b3vxfVVQ53f zXlsDrSoe8W@(3(oK2esx<>UD;RnThoSWhOa+B4pdb=4usJDx7c>9fsG z;)xc|;Ajhul@fzI!0sJoUAcLF^qn^{-r(_F%df=u&l|!tAPkn;a#XqZ-u7VVdl*px zgUMNfKsYR`iJlNH%3ZH!gPl6W=AGJ-;98)FlcB9}&kNy%V3A2mMHeMa&rSX~yVVN* z6sKK{&IszQKX|KEAJErSBa6^?F`fDRDMrRgBu%Xqkq-U(4i~%0w>+Ac7wV>KzklzG zKC;{8-5lUC{*Fe@&(1o~?x(oB=2x6(FH@`Bcv6q;wO-Jk?#*_~E|Y}rWH~9ZT;L7^ ze&(I$wMo|G1Cdm;*tJWvvS_tn=@~HduV-#X@}p0O6bTtH<|a2cI){WVFX=9V{20^& zcM1fxm{!;t=J7fU%yAfs=_rloPgmLq%~r70yy)$e2#E##9WAv`S#CgJ<>>N581Lp& zd2CyM-ax5p+Ml_cDduN}gkW%FQ7z$z2w5}J_cMm;Q@~mv16?f8?KLfP3{z1IcG=hd z3T1?ckSE9Pr-ft559M*Or#1@Tjie!OGPQ@#qTZ)A6I$Nw|E>+nFP%9`Uoz zDT`$?%1hwLo6fv4p2_Ght~F*eAI(c^2->_ZkFS6} z&Dc5;%QEIcx8a{=*qG@2M3zQ$N}t7gHp`j?BU%MUl&-WvhSP%b1_YO3*BNUfy3dxT z&Sz5oY5>PKla8#AH{m$VluKt7+PRUq&E1pT zWviiO?Wal%1u9i0r{Ly+{m|zc?D`jf-vyhFYr5h(L*3zq1?QL)@93wfm{+t5fJHFi zs*?@!cBm8Uz8?T4#73+=E7Q$=N@x$s68}|JV-eQv%lI*q@}QlbXE=1noC=Tt;~{jW zd6%Ub?MwKEYWnwikN)j&u)0xZx=NEdgB@ODRmvRwTeFFLZa5bzH_Ui5dj_Z3K6?lLwI>yl|YBRyFe!v527 z_6O}b{?MgM!EWqesWS!axy8C8{V*f1sGHC`CoDhPq78oPcCykxj=bNDKS-Y|FA)c0Uc%|+$>P_s4Onwz?3 zwHyP{C+XN!Bh!*9C*r#c|2~wt+qmp#Vi^J%`4K9FNFs7sc3ze*?|U9bh{d)qO!&w` zp9x*pty52tLo{;$!~{+Nhw)^aVfv%dZWGy_+5|gKB!8tWQ3~T{hA|q?(os%EY_>R+ z6=^tIfuMxRgoFVxYCpn)Kyd7Zhc8SAPRn-VbhqlOPc1EmOrwMdA6TR~@ZiuPF$h@c z6ecAV6U`tbGw&K}pq7Hgn(ZmsVuUZ%Y5ho4SgjMkSlcLOd2ruhka*Y}_iis=|Hc`h zSUV%3Q`RPme%C&(gk(nY%Yh0s@v`{Chkt*tIAg3E5-9tYuoDG|lqgll-!NwDz$C_T zN$}ZbcuX)GkL4L{j%ci-n&|JG6sM^$`S&;F zYp$|Sx7XWKh!j#-(%d|{8s}olCiwevM5{PoZkUK)A!AaY+(&(yda||#T=vb;=H{yG z6N$Nzs}qs+EA@W;qiN?bq2dSqa)!rJ(G*t(fmQsK50AE8o%lOyZ{L%um8e%^kG-Av z-<(hyF|1@UIwY1Z!+IO8c(UMvx%2f0ZT6Q~tN7j~3sqGG`}%&mh&GXsf#adRT;?Wn zKXQJQJM(Ey-JjO%|Jiwu{-JzvUyFbejn7kAv1eP$`*JmZ_2sx{^~@mY=KAEOOGext zcZiAy1DB8Ols*w?LuLwyWn#SF9L}Tv~*|HJ@5)$ubsrF zZ6;yh(d~g66=0tLpJ*kLzJ2I|)x;&s^}dN7aMU@@X8{WvWu>Ktyp`~bR=uNoOdlnr zE44XU{z-9*a+x9Y1o@CYBwsNF4LF`R6q~i{p*8x7-`~gonNx)HaY+cNJhHInVP7b&p0 zJMc&_$*6+^h5SlsM{tNozG+(`(a_~GZ!L8idfV+%qPN35p~H}{d)eo1j{-StiXD+9 zbvqadUd34i)AWCWbIKS-AOSdB3B~P_CvSEiEtc+ULnL$SN912TWJDc8j_;%gbRi}B+DR5JK z{jzAIRGY#N?DpGM5bWa_qE3_#edQE6w5V@BTj+(jeA1yW2S$8O zP`wF#ZO@VQJ}IS%fuSf8N3r9+Avp+XGY^?T$jaFS+Ct{KfL8SGKZ)Wc4PYB3hF{rSyuOhwit1cB5Y!jHwjtmu|s^k}m-@N4sfMx9=FoUymhRLL*^ z3G6xAgSUUQ*!xda5U%BDZ8<9K*?6{(WTAjDcA92%8OK>x|F*NMY_((GbiDPCC5Imu z$%EL}MiOYGtG=~mmmE+IG=)}u#o>hWhs5)u14%esfI$|;=`Y;6ZZLgh#p_7IsL)Mg z?bp?D3I1NVtI?-g7S&Oag!egxCRIYI;K)~pS{6?DFR%^d7*o=Oo%rd|s*D_$aw(&n zoW%aBq31Z%o9G15Y-TmPC!r@{G%-othu^YFlKDQE#P^A%R+I2Tzo)Xmi*fx)?H-0z zdC}P>K7QBg5{7wmwwhv%Xd*0oe7`l3LS>aroeG^DPrmrEV@wtZCTjTy^Y{Kqbf*i}nQ*XxI!_js^%m=DzlS5=?ls?ej&xoN zaE_|7oX}y)uL_FJ%0@k?l13qTrM|`<7vw}97r0?x>{5raPeng^)L~fFzW+)BNfu{F z=BT(tn?pRrxB7Bwu@fN&I20&q90LqWH=OeT_QV9Ap3S{)X(nd$rj+*&-{E#tu4Ac7J! zUkMhKNmD>{%)owkq9HJ(2QBCkwo`U9&S{97?S>UkSKBHEBZ=a%xL{0<+}jj#KeRi8 z>!?4EVjDK9E;};>M_Cy2-3`rL&Q8h>xdK}Clc=^h8Lm$_g9d9 z>9o|~iL6Gx@)4{Jtgbb~N>hHjDAtdMrNkC6Aws@%FB8jxD`YS$dw004i4Og1OPof? z2g%9{{;Xy9!@i4MYjijkQqVYqcJ5@~Bq_sWn?;RM({xcmbn|IV#Feph^+3E@{Ea-@)0n>|6_4{o_dMlVIjOgBw+eQS4imKf97uNRJ^+2a#MDp*v0 zqe*C%6>%!S9?LzU`k+h&Ol>^qc z58rz~@)=6%m-V;r`9B-3Di_AVELlk=7Kp|D`@z#0M9pHLnQG+$)ijOtl)snsX#9z>GUXg;&Bsl+U1wJPp9AjR?| z3+y7@m}H^W{3v;u)|G)q4lu}WqisR@6SK_Q0zI0MfyUWSj>q}oh^j9g;z%3RVnaI` zFEg-Z9FzmIjO+Ai#1Sy2`xPYxwtK?1tOgEdJ$#?RA{9*=A;(kdgC3sS1ebK%$hC$h zHPWfgOpWoFnpWNX z$8*DnhS-wf+}r4caJ3t{#6h1fO{doN8%!|F1A@PV<&AmJN`*Z5vpB6Lcn+6z_};4c zMAj-l9DW1>c?E9K-tK_Cag0PLp&+Zkqwfhy<H-nIV7&EbJP}B za~u?ZL#A;#I_V};XEQ`6{vtXfTOa0F>Ca|ID*gC5aYTWHE76dz2_?hJj_%2R(>Ia| zgk6~4zgJeaypV@R@4W=Y7E=lV!_v|c0h5)EE|9^Rk*xQHc*8wOK%6NB?2DC|NoE`( zM!qN8549ycNN2={7CsQZ<3WYtkaq{$Nftlws-vg$INfQN5uBzX64k>Qeqe~d_{634 z5se&0R(eAvR5#QHnj@`ROWfvt#6FCTChrD3XQB6!GiShfOpno+Jq^zdhN9WF{KXb9 z8A9$R;yC7Rx-8)8q|7EH3KLQ|L{5JT{2BBHewZ_V7nXf7f zVr-*h7aA%HB7_3vG_uG=Nr+@WRZ$s9s-mgxu^7uE9a>tj(Iw5;m;%*h`+B^ zp|&V9ms(rbw67p-L-U=6vnOb&&K0Uhb`hRZ?_6GWH(qaFT6r9Q!x7m^V6)aXYJ9pT z!cPW=yF)V<7z{%Zo_{l)cr_g1Itm&wuIa<7f5*}N%WHnboPrzs`}*v+ z6lEu&V2ouN8B;Pr7K#X-V?A={Nc=|}6(3c&Lq=wy_I55k_7eH%x_aV1i&v5NOV9|c;*K9A zw6hp?OXrn?5@h%c)erLANpu^6GQd2GvR7qhW0=1gnBJSC-Z@Lci+tDL9(-_s$(^U1 z+!Zo*@z?c#2h0A>o9ktviN?qvbmk+H%fh1%mUdSYW>eAzA_Ogdq9k2u0Ilre3~1MS zfm|Dq*L8s|IC6@71EgIHF+Y6o*T18OK32|&rcAPFBe!tE=X?MS)C1TC?4q%;{1g+` zeC32w$GMyyl^UR~th463t7fM^5$GA~i47Jpi^^v(>3TvsBw{_nEXuA*z8eQnYWKuj z9PEe7LzMh6naGK3JqXjIaXr_9`Ghs_wJ&2v1S3_~R+MQCjmyRl{K*lR-ZV?|DN~a8 z732*}KZ}5RImJrdgzjLt7(9pTkG9x*nz(5GSnP7Ia2Q(Uo3|n3)#2@^|zRbQ3_XVEPj@t=^H=<5;Z#t>Umk0#o)hCexPcDzNOR71{TB8~qUi z9Oge0IW&-%(7#?JRkmiNcrmlb8?(1s?lvjB*Dz9gcfJ%^FGVtBsl!W}qQP-kcee8G z>*XWI%gygLU&tyh-|{h9=CEAYzdGC4_4Hw^AsZ%mZjbr~RCl-Ecb{p>x4hyBPADb9 z_hCGgfjpFWN+cSX+EjQ|#a@ohtgG!_EjLhGYu!05V|C6ZhYP2v9SnN4DX~wu^DC>i z-T9Jk`u(!(!*$L0V+2_wkfd?gp8T7z z&(DD~YYQTq%C*-~Mz?EuER;j{f~AhB7_*8bWG=KI7%tp6&%$Fd2%b?ozmwzxpu zjYbrg@v*`qv-{*Q=$}nVn(v*KS$U!J*j8nqM)lFmzkE=2y_d%;dh;h;X$A0o0MSZ2 zaI$58B*^cu9l{4B(y+kvTW=!`A6ypmu*M4__Kh_Z&U!^;;@Vyw1$A?@Bjj6rw?7!e zKIna6X_(;t1wQ6$yz{J9VvSE#$x*)+}3%i9j@F2=R-UI{W$087jS zt@bMFHZkFtfZVO%(ap{II!^B&;|}T2#UZECc;WT(>6{sM`@D^*mGj{Z;oJlcvL7; zX5Y<4+>-33(jWuw4kAZTX#?1P!Vo&YCI_N#YjI>1TGevk)s|ioOPSAn(BQY_e{J>u z;HrN#AR?P|D2w~zc53`cImcNpeJ6(YvxQIkXe?TLX$Aoal}RBA=|zyp1i2fkaOc;R zYLmNb$e)!)aPC~S1xN9ogAUwYmcLZ}i$t}L^~YskDIF>3ie$mCTwno&v*Qcf9gyA6 zQSFu3u9^QOjE27Pl79fivBwLWNVbv3r<~9e3$9xqO#gBvemeTez+Lc#dFC9q*02@s zHDHCv?yz*f1e_QJZDxFo9gUB&`#GOCFHZ;z>^0ER5~E(0r5FUuPkEB_j4}*qT&=K3 zU9Hf@!~F{0nc?5cdyqcnBx8F>MXX>%@cr>=sSHvMLwWV!JFTv44WTX4+JmEn=khJ> zmz3x79rrK-d9w128Cl9>xmTmQ&`pm{k@=9^*b`%8I;^KgptY`jvd{j^w{*Vlb#!5L zXG%<;#ZY4s0hbUTI|A{_A@Brm*HX3Pp6jZ-(_islk_ensP|*^DUnbb8mXW*# zy9Z2om2o)vJ}@fP{b?LX{?PCRetT-x`vqLghD&WKcH!@DQ^J;~v*7pyZDy}8TV z(;#b#;iBO%BC~8EcjFNi|Eo&6g1@4C{`ata@PvEvI)9a+R}}y|h6hH_9w7T@opX+T z{9(>5U~j{G$PM9|*L$s0Ym@XvecT`ul?Um!FI6Vjax8cJTrYLIfwWfW`zf~BcX@TQ zUwZJtj4DdzkLnVVd^$qM`@J`k`z2jSkfiU^1ek6VB#JXu`u3~ks#L!%Fhh`W`S~F= zl1bjAt1PR5$Fx87GrS`k6x|ltE2bdk0#l0}a18GjaOdc(b!4PdWu#n=W9xSJYk*|FHr}Bi?mIV}3`tT01s7mQ~D7Rlk^jvvvD!TXThJC(EmuW(}f6Z%r5d zU?qj*IXFvyGOKG* zU^v{DrW+nvWD;dZXgzB{1Z8WBeDGbeNxOQFPyOpP&V3|c)8aP4kSF1TN=gm^DW_ZZm5G4jAk643O<%mlWJ_m49A|&k)X*zw_gx&r z!RdTTwXaRYRFyZ2aoUzwG!s!8>w^RN(|d^wB|*IRMu$oP|+xQKZ)a zUBsA-!xk%Km^TL-0?{FkKu0B+er|lrdV8jz52Z4r5*a)Jxyhpr3>^#Y>rfB>*NY_h zmZ4>2nLSxW2yGuRyfDrs1L;u#@sF=lxbwZj)1cfPU87<_A3Laeqa?>4d^Rz%*f7zd z^r|BXI2_1nu9_KFJ!X~?%{m(_ga2mY(dZ{#;eYVCV;1*>LQB@gk8dcY*l$-LE6kG_ zxA-EdF2>p_sfVi~Jhogb@Rrv}dn))MThHSM%Eneq5`#y~U?mw;%6wR$dZ)y1O=L5= zk}kQ&n(6EKYh=H-P9Us3K9`Q*b{}?JWx0Qxw6*FKVL8z?p110MMXpN(o%UcbUOrFw z_h!+5)U@scu_kFkxlb&U+w=7|{wDMOak{hp&`Ps3-{xNG{xd93qm9M3Nmq6p48jqS zZ*$S1Oy7~wk%&U;l$f~@x5zB;JdW&L<6Qx^5mpNJmT5nrG}?ui9eM1fyRpm*WL&!qnK7&oqeXEUN8=FC}Oo#8`1tIs7J4 z*sg*|>5RlM;9*9cI!$@Y*F@f+D4%@^4Xk@7ZCqh|WWF94{l3cs?GB>e>%Jf9k{|Ox zCA_Xx(a%KnHc{?6;#9p z=PCDE*VfTrI?*lk(i!Yj55GesnUmNvjbZ#pEMGng9EJ`cN-b^O7NbBVSNJ3oGT(Xy zlK$t#HHfH}blH=|C#+PeN^T~7nKIB)?cV!Z4^wkjsH<1YRzJu4J)*!u|DKo z;rpTR@Nft7@xsr6y(2H+zvAW*mNTN{E=}P7otyhRv;CKs{-5fh?{U!js}+R#nrZ*| zJ|e3)z%eQj=Td5YkCV4ftG!8A2MadP=3ql(O?7v zI<#MWg^tMAse&(Zgk6uz1(cUU%FMY*l3Mp5h0Ja$p8#o zpRd?oAdj?(P<0LkAnJHd{W%cf3qkU9zqb4u&l^`>4l;|Lcy_m}X)zQS;SnbXk^_-) z>ZMv8bnE|%NBSSn*ODdO@(Zj_1Sj_*f@?8*GI;W8J8GxW#0?MFFRg~}_U{0L{O**a z@8pl`!+)892k3HUYp8Gg9cS0A=Bc9|1#CgyXf-jqe^u19XH6*Dju$;~zL0>~70)(M zET;;iLOw0*g(Eoh>#oZffS;O^t5*wE!8^z1q#zce`cNEwbZihixw$Hj=06M1T0N-mm{} z#(bnB6mMHdA>27bi}RdaKiz0nfIqy0wcm-r>kheti3>%vb1RjwVb1!9OI0g$QBZ$hG=m)%plVN08}yPHJ?42lFnpdK`BrtfeJ zH5pRIAvV$B#27CN(rvQ+ChUAF=XA1KW!3uQ1eH5Rxi~plq}Q~mW;L#Azvkw+FG?%s z+0X}qhQWuOPcc@}(jR(t*h%mWRaz@w$J)I8o-YM^N3hxE^Wlp@)Wr2%E!6aA=Lhej z*GFnjU4J(W(lBAs6tP6eKAIY?B6LbyeiDt@vRV0qFWS(BV$?JfbL;>{%E!e&EU0gaFDS z$xr0X#!sB}fj=A?)a}jHjN{u56*1d&=a{g}^(gd5n7{#9Hyp_DZWFdOq16C)Ing?T zqJf(FG9H}&Rm%0Z0q|?S-C|oXfc^YWWoqcnjq8P&=he(S!rulQb^A+9KQSTV_IgdM zTCNpP8I$OXtS}u&ck`_FXI{vJsGb>3k@z;MqwTA`<~?-7?R<6FHF`aoI}$xzTgx7FSY6DN3Y`9_*=;KH@}TT+ z>8R{a7yFSyudS*GTEpqAQ|z`(nrRW+^B-zGd1zrW*S!9}z;nTWCXM{|$rmUc*Apzk zha&Go$`Uym{T<#_OtYvne9aJrxQQG#?3`GeD6k|GWIjgD1U=x`#1*7~1}Ar|${ zr$LdUUS8<*?fCD`Yg(g@6k_B%dUmFZRnNXt$0T?8pu?mW^XL0B=i7l{{URq?|9yFJ z2S%Zl5_-zwsuQLzg9$eLN%httIH%?69M{5dwSYLO%>`0LLwxun`XIX)>7VNpyz}!H zcMX-f9$5y)SU_V2#I{knwD(iq*yqZ`D!A|s99 zvS{uMZ=R-N0CWoNltu)pfMU}ryiv3c;=Z5{Ql69LeuxQk%R4~=6AP5Wh(t)LfK65QidD|BwTh{q_>Nb#^f2o!0!7vKLM%lbbJl9r|5d-Pw| ziRgQrH=Z-h10MxZuq!}0K5+WYoBzlkIanXc4j&Y^z7@yDAo5yRijK0o?SbBnhs*M% z=DCK~IjlJ{6sI=1=L);;+Bh8!2k#Qns}gGQM_NW8c?#;;&G{XS6akJsW$E1`mvx>i zb-Pv6CHABOyCG)c+Vy!8mV57{?K5OW=>nX8zF{PXXK9mn(r1Mx$AH^qtJ26fsHc!U zQM&>5 zj>0mQTxFyretg7Xs_l4XJv#aL%gT>9gLUv&Dj^=@GA=%@HHEw!SO51OCv+6U7fhd% zAL%~dm4M7+kzyclRloyW#AWk>4$pJ4x4>oRmYQ)Yfef+{m8U;jE@tiIwN6kC&#ZNI zYNwY^=f&mn6c+PP<`YH#*se-7-bF{=+8?gMRO^msa;#2Vj_6ux*J$RBr4`irwI6|R z9Kl-wnTK3C%|}V2(t9G;@8b0iPq-~_0Og~o1X%g%VGPZsq8QMeZiPzFCZme`i9+W4 z6LQoFb)34&B0T}#VP7#7ZFcu~?SHqIOnmO?KY%~P_*w+#IWThm8FUvK8A7ls5gaK4 z3f*7MjcmPWx(kT(y$iexm}l^ibywP46gW(3qCGj>%V`c3i`-Mzayj3EEQ_1`uFw$k zxaeL!K=rs~t?yS2`olT4GNDa!-y+m5VVT9Fs-73a1Z&m&BN{V0N4hEMr#j~@9X=U{ zzsA?v_IX^N=w8iZwEJhMh+a)Cfe6=py|e9881+u91{!Wmwz z6a9}TAXa*wf#jg!iq9tj?^vj;+D?up=zRuuD0Q8;gi83k=QrzJ8tchV0?x{`n;1Jf zLiUwqAV6a(m}d?Xol*Pnify3*SVxO$s%^i8qw3r&bky3Y5aw`aQ>aZ+np*q;SYUfwunl`V>ZusN- z1UyxO^=rh$Ru)E92>M+<>&dO+U2JmCql_T=5{X+Z7kvcKz(Q=Mv-snq5tPVUf@V6n zeOt|5H+fSkyOBEx23IRj{gMrf+QddC`dej{6+T)!|i><;l;) zrro|xSh=nX?P1d8CzrY>XHu*>cYk{lBjRYQdE{O}Il9J`9^uRkOxaYVxhoZj@I$V;_&T7rX`SzX4)Cc=Qw|?oee5G%wg-at3 z+xa1G5*5MpngRTliV#oeJ=fz(+82S@{u(~lu?xu%fRe@1@nH{V^{MHk{=~wT0WAW_ zviSy^LCuhlo2zo=5BGz}4auOX9*OuIb2yz)_cO${^LfQ9GR!Anbqbzo`LotQf@79`@RS__I37Il zGIh;sF_Nw)e#N@+DD5y zs&DRy)tvYp=hMXiolZOAKZvSUO)RQT#x|r-8>|Zjc!y8a2Xlef+w%=2ZdX$R^`;YI zD711LFD><5p)ffk-KG4oB$<%s!PTz1Mg4<2hk|j3dV%fT^!d7GHwQOD!T2Xxuk|0P z6Q=d+fGFFN_G2E>1*qUaDqsaF%UKoPJ)QGjbJQYG!D+9odiy2AkEEB7D>!Q3f?;v+ zKFI89+7i`&GGOl}0G_c-Wa7rgx3|ghicw9bwRw4@a*>pCoSj0s*LbFP#5*(GpD1m& z)cH;f&i+fWU(*#0RLHZOOC-1+S!ITEcyTosx&-H-MXq8wZ+@@Fx_zy?3_Okb){o5u z-Q4rKb31Z~QC|I%CB2?r&Ywsf1>d?K*@&z%p?YV!hWAAVuHZ-N#9u2lusjZrpXqby zd|XWbO$C1?y79Y-rTHph`F5#YB-Sm;)X(|3j3nsk!zSOd^e!j6G1xKy>zSQ}c^mnHc#WKIh&w)k z>o8{*15E}w$)?Qd>vWw9kwG^zrzZ=tV1jMp#+&%{-pN5d?2GB72qG2ihX5^N1_|o! z?{Eu(i+!UgTMjc^a=b_H{}9XjrtXnAFq(utu#3yXWMC@$7}O%4GW=80`Xrwc_Q1Ox z^pGr#Okp}Lu?42_fFF-~WwZ#K$g{crJ?toq=thJ#ct+O(kDv`ECUe-iO||l7jP(;EI#&7SGym!4#wbrr`585H>YT^nSgC@pSv?& zYFeV$anKjPyC+3RNG!pVMy*jglqk+b@IfO(5Kr~=E39-0ec91nVC3_q>F=zjb(kz@ zGP)=&%*k=i^-@+_vdfq4Xvg0OwZ;P8;eXh*nx-p;>(ieM_`HU!1v00diy*lt>h>l? zN9dPR8_-#CLN(`N?2ENslrZYxQ$Zc;0k!QzH92D%m}8VHewyfL5SK(C`EN={MyT3+ z9PpP;q}i(;Bex$%bc&2=mSqnTkf38(G9jX&uNFhE07K3QnWwZaCah;NgrQL$O+wK8 ztMy416s#gpDxn8eB*>}T;0cx!O86km*}0ZHxVG7M^Q6MmCy{T)*osUH$$8o1%!7Yn zcJXM9^?Pf>Z~a$m?#XNRAi&X*&NaVguY+q%F`maoJeuqnB%+ckGPIf_a-HR#xfoTP1^Njr>tGqy(|hNul`ku2V_w+mCS7qGra zgPByu(1K5FQBG$sEElVD>s2Q~kxEk7XZM7@aNZQM1A7EjHOcm2a3PN$_0X$8v%})r z$X`!}RLT7$g*l4LwvK~@zIgqeu=*(UCzvN>_I6J4_ag|9 z7AY$3hVx11{Fu_Vb-`c~vsm>ZVk>%{=KTuH-PAs6!{8k4y`~>$qA*=(wBlhZw|`{P z!|YC+GfuhKT;8AE+ij>5)hU>1MF{CS%{gFG$FATZ89u%8KD&_h?8mTcp!=PPo>$9M zI5fjY-l)7lPZtka90Ri9G@^q)ejV(~$8{mykqk8g44A4+s4r1)VUFegn{?<1r%>7;6Z;oWUdasR z|A_d5Arw4Uo3; z1T+C}sU|L-vk58|KVfNN22*;{;x2S|nW`mPGn_M`l15CE3w5@cV+MS6nol2Ohdnh6t>mpN=-e zEw`SxrI=S2%Q%UQ1qsp5AAf%AsZXHZJr)Fku3WAF#|yt`3kJW0F1@SdHcoGJ|50zl zgy?)6JT5#JTN$ zfarlvZO;x4RO>iZX>cU_!*`R+a3zTkW9lN(uX3x-WTIf>k(L{X!YM#o&d)@08l#aL z%diw$xfdN#wpQ1I0|Z|h`eb?D{mxTy*w;AvHY|Odw6j3sm^ScC?ZMCXo3AAkjhGkG zjINitoKvavP9hSf55oUq0qDJBWLZfmSjB)>x}Mv0MRoYJRIneuJlZTMQ2}8Hd6isl zZ0jwhFU(0HNR|7fZ$F%&d%3|*Si)+?<-6eG<8i>zALT}EhQ^i-(GJCgT`5mT~u4UbnxA2L@wYuVk#ql~^%SY#cKef=!De2Z+-$$!KwEH4OU zw>m%XNR_qOkd0IeY;8q1ytbAHewI;erv88Iy?0bo>$*O=R1~mKLml+2m;d&~1Z@AK}ws?lc-nD_gY1`K<1y;xw)J3Th`M3@}min-_|Nr4_- zwN$}WvD3`&jDceA{`*w>dsf;)a*4iUI}mU1O5`t-4ypC2-h zZZTUgzdr8Bto{J>QijHJ;hAURjf%xwXtLbTp<-Y_g&}|Rt)NYe$Ml^549z;G>+mqk(*>j7J+k-S1J3O^0OHOObypnHLvYe?*WKMEgqlQvqA+?)d=5Z-qGK($H27s_8yrOS`IH)zgWA4IL+(LB@`N?0|e%M z@a+}Lp3D`oIZrNxdxf^@(SdvOd&s?_A%#(>z{mxe~kq44yE5tC2OPpF>=&fMK~{NQLd_~?i}Y!4m-EwfKKI-$@6y&Kub1Pl2iQL zPI%z8`0w0~@LFT&j#~{^ULKo>c1DKOlwIvo>UR&24wvLRV2li`=#HM#-GCmA(#qsh zIl>`vUy{ks@azw5C4*wJ5K*VcRAlH&3P5%hAH5$I~dundwXJMXLpksJim}_hxKv zu)1Lt3z3PgARhwQTuIJk*9}r9{lh0mnw4mLes&1=(aF3M1$zR#Mb-zHip-_vxuR{7 zK9r@NtCxZ(Scu?DUN$AR`%*TtCmK^oL%7@yXYcg7opE#wa?7jp2|DzyHiIILg4 zmnV_np8d2odM;=axy%?ZD$}rece}f8l&ERXN>_vzdl64j5?;D&S<#1Vh=LIH$!H3+ zF0oIrxcGj@8QPM5ckMneymBkp7U#c4?`e&{rv>UX3;>fDdMxOhsjqCsR6f|dDA@eO z9^${E1y9`tTT{bW^-;XN?i0U&_I%bI$Kj}_F&_G4a4Q(&;#C;M&9i^ANv~hB*LkNb z%9XlcR)3-B7^s3!Pq#!fJt_*>p#G|=-{N10MW|n9x_fxEq)r?Y;y5bt;ODVo=4b59 z!ukF;NYi@WbKJKCY}Gyn?GYc%3XKA+=heVLe3!20syy zt-{(6?*9tE_pm68R@C7r&sU!r46$}`S>AlpswKk)+g5@NrgE?~KY!oY2u5%`RnM=;4Sr>=CdoAW#s>1H5){=0F%)Iy0{d<%3gs!2)16$gG(f zgnS+-e~=F(K?e=l=tcfcvd*3%L)6ly(u*;yZjl7? z8I*Kswv&6o>jzbpyEt8vzBd4h2T^fR7O|i)&3Jj!_M35!!CnpoJ98V6N&c||-GG-Q zoyVJHneTNwme1;*0?F~awbH1KxmwX=@*qO8cokZ@imj`b0w|ak4~$%#(V9<*OG3Qh zbdw$qzZrV7dhm-A=^najshhA8f)JP4fPTXj`Xgs9G3WnuwCJNaoZq!9)^V~D=U$OB z4Z{0jQBCvJxP+i@a>Al@HyiE7WW^w2~ewEsU^%c7N0{J7h089X&w=F3Cyk{ zk>Ir5k+Y^cU$(*pe%b7qQ)13cwtj-PiWT%Pg7%4rQxO;Zd^fh~x5D6)$=~2BN~kqE zep(7_l;>`5tK%{l67oQ%XlD>V=3;H8 zUfEScomN+7uf$G7c<#94HvP&|3dg^vjmx+jpbX!UkItRNfc=2c9ZxzF$SXGVx+sGV zOZ(4Wsgai~_S%CU#ZVQ=l^CBJz_KWM?dXan)W`CYyeOy4)wj19Dnsk*g1!= z-7)1WE~^_Zt4T?P%GM2xaksZi?Du|j2lSNR?->wX8wJWJ_gLl5`6FiSF#CBFYtaR$#R0(t4 z^~1&x$X0c*NP~V4O6r^noWtg#&8d{3K14gcW23G3N~xZ}?t>;w7-#+78{I64nCTbF zXzh<8fp71FT^Bl+bQ9`7LDOq57UJ$&u`NHbQe3VoE;#AsbS@Q5t^?~^WjPhSec4s9!=jQ(ehGedzTIiNLp1(# zYTEuSy6a5qqeSUyOQaiG+Q8;|QnpiQ+(r6xo2XA--uPI;PL+63p6f@+AC(?7jq7W&Nq(7ym(hC~>QLHe}|iqT-D=gOvVg(;4R zP4g84!r7QbtlFp32sv1JgI}Eu+HR0U1kHVt+bfPzfbc)ehIXHDZ7}-WamLrq7HpgE zOY*o~>$3Y2zSTX_Dbo<&FKurq$Q}Q5LN<}dl-%lZQ|!*ooGG`Yvf+HGNqiw=dJn0e zvcBIHBA*fng-OI%q5RTi6|espWtb9m7?B2d7D;Br8ow*k?LS*<`_vI(UNZevckYCR zIkV4I9ek`eYqsW5RV9Ua;*mNu8SgBc3p?EB*k=!RrzPQDxNY0D)Mn;DnX>{JYVlkg zhU4YMDkoJ}vE9;ZgpV6nd(=S#ACtt17}fb8!1S}tU!c^A?&4$?hq3Sk?y!TqnNZiu zp`3R*i&-nQRg~TI(}2U{xOR1DRbGGO8wOISQa?6mluw%dQRcc*Ws+T{EMKy9ccxu0 zR6s__zb1XWhfU_8f2q$8M(20xJpi)MRD~*svps+7rHcOe4?PKVG;Ta$`|q&&*KPkJ z#4-thj#95_NPv~WlA_g|KvkRWZCUkW{FSk3kz}jTmv^$bIeVYWiwyliiEf=P-lZauO(Ri+!%l@HI zEcH3ckuUhH;03{W1E2)gZpcB4wPKLwJjY9>W4?rhs3JjEtRAZPJKVA2{gdRH#b9;5M z3?s%(6ue0%2cDTnSUhp79eh0PU8{u@>MgBEVo|Smj!&IT$+M}^;$D7PF)mz&+I8A^ zQR&w_A}zn{sJ0K^=a7v!_#v~aF@NV?i_**OOm*5<8+kY_ED4ohDFe9;pHbhPOm2}W zjNvg#b%yWnXb#L1HRnD;U&6tS>2Cd-*|!4QQkzc4%f(}0ACRWKzP{pqrSoH1eW#GX z>v16Xr4GR+uesRLkTINp)RRnppVMNv7nvT-&sSTFEE1klE7-N|kF$liwk0+yHtna+ zD=C)V;q4e;$h|buYQ_MKr}5rw9hZE1QqCP^m^kA=UVTMba;pV+K@S6g z@_ucjHF-_!o@aW}G5o#d#260EGA!A8VW=HF%gx5nC=QZ@Uk!-$VC($A&4b&o$0N9b zg*kN}`4+zxrFOqLW+SjXm{+lh8!89n1#-mw_Ip4OkBJF$@3`VgQX(4z;SBP{6{t)M zsEfUDPmjeu73uziziRXIW`qwm=Dr7>P+P#)NEy209-~M`A%?_?$VqhuCzyA)I~9WU zXx>d;xTOf1B3Of``NhL~m+5VLX-RWS4pMlp*8PhE_S#W@l=;BYq7Z3=a*qama`TK# z4U%H=ljk3tU$jhXGDZz8-WqjuI+rFs!COK1MYT6Y*pwURC78&b;XHFhCnkQdU$lbv zYEdp{AdA<#m3a5o@vV#Jv$c*Cze!&&&!6hsg=nB3$IrRfM0h1u{TO{BA}CdEsmhX= zm3JGjWp%=(;$)O!A=a!q;(3rvE!qfE`%O#WBBxKz0ZCbuROInBQu7365$FfnhO2be zs?|apWp*IIv`hn{DoXy{4ph;<$m5Osm)pPgzQ!zvp)M6T-!PyTfsgXoAHz(#6DaFE z2#FL((hjcuk+XIq$!&;I7;_5xut$3CgIv^8aDBA%wp|3sB5mABcV+Ey@s@7=2j1wH z76^Kf`7R*q!@9g*0p2let31+cFNHadkcL6OBt5EPTR@Wj%7J-AUy-s`X86e}hyHB+ z_RkPE*bKvl^O&tMo9bJa-q-qcg~0r*Pb0j=i`AxKv|sD>+)U9SDM}ZtWz?cu8Yu{DAN6SacU*r`k}rr9 zB|PTTox_!6kkn_N(xdGN5*L#Oo1jdCQ3-Vo@hr02k6(MrL`Z4}NS*6{cq0AwZ2VjL zJGQA-IMwU%Z(~jA&@B$PF&~b?tcTwzz)z^`u)?d}y#`i-?`kT(A26h(Hn#Zymde(5=i|uJ#nPE9Q zucZU-OiwS~<>^#Y4ayPh-2glIhYNWBQE_*DMo{KF)Z;RBBTNuF7Jr~2v%HPGCUdR_ ze{QR6HtAkSgp|v&Z-C(zp_weZCpYspqWn9Z&>b0>Bu=@S+W6~Qn7TSGgwvkiOIl!g zaYnDQ9--@B`ca#3w@tPU5MocinyB~+WlJ7+F0Q=AT9ONXI<|LE~7RZVoAwbhwt(9~slpA>jJ+9$(3u>VAb$pL4w5tRb(=6`;2{`2atGeu>eJG=E-`NS>KA=WYJ={`#_01R*d^YD zs%w96BQ?&*oY$O?S1`>yQF%|Ux?iy1d-uyc#3q!>WFA1WX_-}!$(~S9xYbj4vDA?v zTz~QUkCWb_&PTi@6a7 zwUt;{TD{j{C3h&G8iF8G)lR400p%HTQA$I)RUB(?L&3XMe>pJH$SwzR8Ys zpP$j-5KjuU5tiHfR9w(ZjlPcY1x1X#z0oNXrF?%#=*#IKwoA8$ti>BRUZw9TyylK+ zp8pZd(y`FqTFWoQf8F{?pyw)`hM+u&3Ts+=+sdGMWnTN%i^2L^`Yab|ns9o(qSf9G&O5=qreW6?ayz9RvH|D3B@Q<~mQ(PI3KJ z`y)bqhkH-^`OqG`lnh7-u1v3Qf@UwuQakq>Sk_!*1rFA*2d)x2){4BRKB#wGw{Lxv z8*?UP?u~D0RL#3zAK{o9K-MFxaS-ZnW_E$q>wE&BkM`0*bl#q<$Y|E~hz925S|V|Q4^-X@)99~KC_zNL1WYuvwd@#uN>X@Rn} zxfh$0$ai7SI_tW!Q^r?T;ShQwG_Os0KL~Y65=%;8$owkS7FiFCi5koCnFt3QS(VA>2@3XxG~BGSUe-DJ z@63%QI6?R)=_mdtzLow{aNsR~dIx+N%mo$(OGoO(Z#RxT$6pW@+NRw@iFL)oW3+3j zcc`}1TWU6>$u39Xv<+>wjK`bgO^NG{uRfun4x7x^jd*v_^F)A}!X(uc>W}ECD`5U0%V-U&(!a{lgGE(lp5+vD}BgN_BmD+kJG7 zP+R0F+=G^@s{Jr(5c{^NX2*dG=W;RS#Zl2wlIM9G*mo<^bCtT~2?@MYv0HP)tj~8b zY5xyFIU8AeE{hB`W&cyt0{_vo2M_T}G5b$Q%qw#pSAY(XNhoa~3qb&Jl^!Zg_8GV$ z4%V1+_e2Uv^B{ zj|ki=t;FnQ^||)utu5J!uKjLGpU1=T_KwU!Hx_llj;AOeEIN#Uq#qq?)^5PRDp}j> zM>!WHHAN|m*G*hUX%3i>_32|(VM~XVkJ>{JWwpHT0{ggd-NShmw-4b_flSz{2h62O}Dlq!AK&!8m3^dK0)gAqjyWljRN)*X_ z94c=jwxiu+eEu`ob2qN+o7S*U@8{eWaik5zbA6Dog$5-JCIKb9d{_%Q6*w1ME@I3w zr+X7!=P*ceem1gneElv#!pdr_ zGf8|am~tl@iVurvyVh8IIKi+ltVoa$y8@?f>`?r}XGuP-8 zV)qKJHuDtc3hkjWR`CE#?e3Ini7M5AI>;y_o;f-lgR5R}SG+*Tmgr|x=WgHuE$Ajt zl(^+ml6qjpyK0i|G2)+ZMV)I@+ua&)W)RR|25N}FOOCCF4$&$>LK+7eOjLnwynF;lV zFyH9o>?nbzM&-Sol2v8=Q4dzfH}(=~V|T8#k*9;FPK7W-Tn!OJt~+wS;*!qcRSd3N zzMCAPpqk~Se6!&X<*;MgfL)BT#*J4^U_}{GYug6hT)|T^`aLAOZ~UDAs|LwE;vb^c z;x0X_(HDBRjJPqYerFn{uVYRnCILr(%8ov}YJI!TD@-jvw&d~%K}Xv4X*H!;Co@Rm zZ*=Tms%Vohe}I_bv5Oqwn`&f~*o?L*YNqp|K2=H)$c5j;Zywtk+{AL!bUB`#!h-LM z)>Aa947i9r(}-CSFKIf9;8{e>2KZIaOpIq!-H`pK^fj;%RUB1rV+A&#dh8-wJME5I zrm!4K&2(wHrZc$Nxp^Icz|WaJQcM{FC_pwipPBUZds&?HreHRGc&PQPURoVJW|NyK z`9Kxq4XfO{saAY`#vPaEtLP3W2(fWeN?% zwNLI0gk)`ZpdBG)-O-45C)RgfkU1EcdOPQ*vNEz%Xf?(m?kb3V61}Qyi2*mJHtc9l zHc4*4_rFTN^l6gv@bMtnIXp^I`qE@fOHs-jdl_t6Phx&=*;l+AIP~ejMfFC}V&;C+ z)@yci?BFQa7hbWub#)~L+%oP0{)LZN052Qu3n9ui=C!+KlYxPo)6Ki;29OEmIp&Iv zGj%X4^1Ro`iQREW)tQuqOcR7`2?d?vp-#Vc?9|*@ zwhq1~58N*KsNHS*7|#zX5EtWkgok&LZ0&Yi8&Ah`)dc5NZ(zrG`Ni!s>U~J~pE-Vv zP;dgg5PDkZ$9-z<4Jq~*cy&_ULWSezHEKE0qh@a zYhfU8fZIAd->#8j!(dAK0yRNf;d%lxS9%_DXLp^FY529gv$>)9^HiMY(2~P#V;)pY zx=q@0r^dmd45%udx?Rn5%er@H09M%MhTz}Sksyj=B0q2oQ(@FO!D=60R&UGl;U<|| z*%^8Z5RD^8?=_t;BAIWV+FIf$-Sqg( zp=7>p^vss4J$f7KZIc|~i2H0@wME_(qU~gZTotK@nONcdS4|@1ybJ);Ga0WqMcn8@Nh12DUs**Xd3 zVouHz4UwF){=qAf(7zTA5K%abdIo?HjR9V+1_940!DW!v5yAv2Q$e`b8{o0EP3T@C zeYxyl6;(1w4SHNq<^Z-}Y@RKpHf$~_>0XV^^@~#G0`2F#yj?DZBCVRCq$% z3zbux8SCFJVQ=mh!DdF3#7IF|u&K(|Ve?Fo#2Xrd^Y~#pxt;X^%ghH>YOPvNFGIdP z1(urJVP*>>k=S-$(&E|2#OOB%qQ1jc=)fq`Pk{-UbguqQ5k7b5PNpCvq|}SJ0gUEB z>`^tlV2naCr4W+3{j$Nk$&;Lu=Da$eWG(&@CRqJJiGb|X*1z%HvTF9Zq+}dk8)chX^ zOkMBJ?W(tLK2}F#5@X`GE^EBvjjIVWojuE5M0=|(Aj2f^TJ4I8zQ$Y)vxf|`Kx%|( z{3y~PB(UMM|5smY*liQI=l-$U{jpp&_AL~?chT4DkFo5i*6!9yuSQh~8vyEYaS}iK zzCcSQ2HA@5N?;95Zsa#PkGk=t=m>YM=<$1ONE^ur5o}tWzMt*WCUi020s=72n zz10sN81U4nI;7G1*4TKO08g$UH|MKWk{|u;>pEQ0*nD!J4Wo|nESX|1az#Y1eGl3T zvkxB(R|~xj6udICgfsZus$bCTN%0|g<`ycS!6HKN`2T<)O+@_xKM6Yp8#ruV0iW58 z&>f&DQpLa=lN%YjTtA`L+8yV?O+pSFt$W`-L%_%%IpgFaVXHtGu0idOv*#rwS+J!lekC*vtb6D6T?_7hUmMO2urFFo9y8Ruq9aqN ze?$0eFBcaEIbhYFRL-T~)V(n2iWa3PMMo!Ps=x~uigJ-=t~s$!QHgxuP-0y{zH4>C zV6ZFm7ldn2U%j*adDfwQEer3@;-|*~zv`m&hwsT_da{VvDtmP+ap#RhHHRkYGzB%# zV3Wbnk8c_{K#7k{VdaSB$~dBuAr*TtQ&E~2P@`3XHbKIkQ;aGbG0s7GUG@+nY=mZ! zU$U56MUSYWcP#cXU-PT@U=rI`wI$_XQ%Oc&(jNb9ZMyu)TK;iO8>-LVq9;92S0hcr zNEjls*?zWs_}(&(3#Wbcwuj8J2rb#B4KqA4pdA5kRf@Su@f=xu*55vbGKJlG6Rj zgOO_TSJpRy*2HDxs$5fvAunN5eg(TU=v>6_azn{#RE#Q<4o#JvpeS?fQ`jr^ahHcc73x~>lK{f?dX?+^MhH)4 z!R3VEH6j|b>Y!QMlcc)p91{j{kQl9v_S0HkN3jJAExpwRSevC5^Aq=mV+j?^rmm)m zz5sY30U7rBy9=>rwAiS+H6WzwMRwV$qiJ!WhW@XQ3NoA_KCQPlVP>OpDK z7L4LfFQm?}!(KR+OzNED8yi})5vExAl^5F>MQ1~8$UY-5S~0zn7N{;{RtJo>|1n7$ zf6>3`gK%yK2>rV_G|g=AvrP_u40a50Qt&wZ7$So8C7dc9y`AENIF9K3YI6?=TAbxx zj_a9ZAEenfMz1S<>ik}^y`SYqA?sX$CQEyk!VwHv7s6x{5P(wp4Kbj)Q?K?s?tGrc)9wPty|={7reJGGV$oCbP6eBHS%?zB1)-b<$cAq%(sh9eJ%#J zZix?}4UYqVlZ#1ltT`by*`veyIrN;lp#*u&`UsD3*cea+WT8L2+dE{ScuVWX_h^bl z^}dWVI<7JbZbL1z%(LUq5ari$`p%D?G$mrkZM1qi@zDu%ax*>WO%Eil*$I+3&ox5uj;BY9vAM`xyUr>D@Anzk}+> zd8Lk=6wpY(ewEx%s~ZK<;|Ri}Wdp06C?(j>#b%Iwghvl?TDuCi3HCkh`KV~iG<`~7 z?V*X_DTtqCTl!pQOm#RQG#ExF!LY(e=h0P5VzCX}Wo`Muhxi~3ci8}IRsDY~GZwEz z9m&HLR&eVGJ6T|wa;zfEgMBN6y#ZAkaDR!tM_{+{^y&2K@eW^$`b!7T)v^1hMjOXx zj(6$2-3{_6A?{*zcvlz66rCulpM8s4Ly*c)zwg>+S#xxKg6s>p&wDxrM&H2oDPdh? z?zueaYtuqqM!unC(|6$G#quWQP4SohJzR`Ee*}6FC<5h+8@>D8g{Pi2R~SS@UsH#s zY=A*vaViK5@*~*$2=dTlH&APsS2UDi=T~?4tI!;x)m9TW&~2jNp!eeN^CGuUKnB7k z2dQYDwG{Sgf={d{RT@TJE);)MH<7WuI#H9kI^}Kg>dLKO=oLItMGPxJ-^A00!O`@C zTDlpuZbMtD2enZ8mPHD5?m7E!{e~dKE~G?;orGl0#oyU3W9&zD$R9XirfcT>NkV>z z-L&Pa9Ku(Y2ZAbS3_x11ZNC1yuA#r=H=l-+UcDmW<8(h5$*(#i_RMH7zT^@+7xHP) zw%?yjnoRZSux*XKMawv|i$_8JM+Lka$4Jc9a9U82lnqWyi4n z#%Gea_?o0!3C%*nL|;+a z!@28!8^FIHr2dvHo0uI|7vMD*Hl{y|k{18e6AZxZctLjODbrX82%}rDZE#=B9du9w zb#jb3UGDb%psP^P`&pkW2s1an-t7DI^3h+~jxh-a{esRq_G^=6)zlaL5_+j~y8y2> z%Ycm?W?WpkU{Zb@cTZ7WJogN$NKeIiCj+B%{!C)ZMkLn@(!^_!Ikvoi_r{MqDp98< zJMO5Ur)aZHSSilQ@C(oGOM%#~=5PW!KLrb)zHU*$|K?9xIL}$BPEUX6biMKN-1oUO z{3HBx*k4;+l3}+)0myF&UWe$8{88ErRsa01*E;SFO&_PaDC_HQD|oJA!u@m4?2B4# zXgiHy*6qu}=QL?`pXZs3X8z{}7fxJ`(d6H0;6FCL|GB9lv{}Ppc3p5RWFK00$!~6a zr4|zqgV_=fGvm-O)@x(eiQev$xSx8n6xty5lu_n^t@84uql`08q}2sp{8lG9`j&x~ ziTPYFMdBj*f_lDP3PZ*{PA;#N4!P?7o4}$P4)PCBP#Z)0@o}&Ff@^%&MCXN$#!H52 zht7p%fZL7#$YGx!gJm!Hn`BdeC>@<^MC96)0JEia<6#caOj z!@P_i(zfHJ9*zOfQ7_ygN(9#xUB@)ASM~OuIPt@GjvBt;ruq1Az~QV2^3I?^3QL$w z;EgX0iv)`*m+Kk0211X71GWj$2b(yq=9e$l`8Oc|~E&G*trvcU}nn+-mLf!lM$FmX<^bW$u*SH3tUs z+?7=#gCFQ@=ke-urscSqjHdMuE(y-U9&0_%ck3=2C zkiArws4rfbmA>DlWuLv2Kw7!-VvIYql`0S-y#c4)-)(%oU+z`zK-howIn{DZ!ws=T zYLGU4WvKdg9%Ga9)3UG1X`$U8jB& zDv_bHk&tg%oG4*hRMGADCY90WT;_mG+^vPu_CoHxI-!8Xtmv=Z=Xn96^ z1qc<-bRJD0imV{V>PpMp`noo+$LwG=0Hcm#`O0`LbA($(rH}Wso!E2v6UeuVkan?w zFM5ITZ2%WB=>a@?@~t1~_W?9o?p9=XZg*7>xYb`)>mF7iW=hnjUNqm3Ubw5TZz43_ zM}SnBs>J=0wJ$JO)tpy;i7iBUNLx3UN!v7mb>h{NMFOK) zDG?D7qP^%kA7^xxs|C6tzzbdRa>Ph(2w z7-Bvyj_VBZt|(6I+t&t@agQ&1(e{aoZhCrpmCh|3qi@a>Av>b@OrJF@$9jHPu~w|VmJItFXVqj;y=cHM@q6gpf8dtZjPna|K3dRhi`B6G#7fop6GEg1=L*UoG^gZ%suQqOuk)NI# z?U#CDXEBtkb=BM^6Q^5%HS=|yc$|(t@CH=SK6@@kWT$*|82Av=r9b-}3)!W3?vA6! ziyV8(RoIfa(+e`fJ0B*i_xdul!UzWR9RocM0;4;w3b-{NX8@FI;cxkLIGOSv+k;LF z^#iMeR@fdp>1r?M^{{Jr`T>@X$-qP^LTWseN%ZZ*O+he`wO+VE1CW#;c=Ha|!cGpi+IZ-Fm$(5Y z)eJ*3)PvA7ut*Le+D8$5c(RuV;SqV6V2Ud*7fi_gjC5G(m~?1(?|vfgo`jduLQn6I zZmp%rishGvhlf<~L9ap0w$xG|pNRar-r=rsZLf%Ya^3e)V^`!eE0j~9TUO=c6hYz; zC8FtfX1Uoy6w5rLTh>kI&e~*sHAj^)>8XLs6-go6fp>mW0oh>Uu<6?unnRy)qV}y5 ztCo=!n*4JcN!Ee6QBqCCCc+nQ3UJ<|F`#s}qate@l@TkHn`36`{+%tXUPM0T_%n##)@-5@YaX=3r5SE&tJd{-dJOPJPXaW9U95 z9?HQ{kqD|z$^?+1G3J~Q;e3=|XD9rV{ZY%%GjU`M)(n^A9no>a=q>@C11VxpOX5^>G)Wt&K~etk)l1>JjwWOa=IH zWq@4g;Nle!K#IimR2@^e_zP9WfM6Wdqu6$G#7Napp}hEhhUi%J;*CUE;gjf6`%aNz zjicvX_o&poWUZ53UfeJE0+_sZgn z<__!O!$_%+XSGh#rlA_xQTUm20 zYH(Oder3BW)EW3)03QkchH`_0)#S8dEN4~8!^28n#;;9K-IPXc6wTh{y+0QcNAfMSMx#ew z+~#wNhl@|;xt?FreqvFK-H#opi>mDRaUQL778m{%5})tl`Z?X^TIcVlA)EmtQ3cMf zJm&Jhlbr-pB)=&bw(DmOXEEPjze*ZukK|6d5%sGz3$)1XQeJ5CsH$m@!etQ%^PCK% zKSYEp?T?NWT3FD3DN&_}o-C^xUB5{QxEjyI2jd2*^OhN^zB>oQ+oc9JzheR+jjH4l zPhG?(ZQdS!moSfK3|+=PH*C;QiZyC8Th5XhX~ubRW|L9gVV_tVsC8VYq0QP)z{hyz z=KWG3tA6QQMVJ?YDY8E4h$uh@1K)FZsnDtI$VcVln;Cyn7`jTFnE}Jcr_j4)9~@Af zTAowW#D@pd{&;uAoJfCNwMXuz)%{{A%-iLC@46?8)Fp1x|BUxwnZxjDU6TF#e_oFq zHA+WCpXww~L|Ea!%qyUse^Y5Btb5!{+aIDi!_IbSCi$)Ye{3{fX^_oz5uUh0St&+LiB2Rq zJ~G!3P|Qhs(fNI=`66)^526Q1=v)!x+R31Euo15Y=xS|D)RwCfhl~n=KEBO*+@hsi zr??h`U6U*PprDPZ@F^hjkuj4{N@r7J{BTwZU&@PT74Pr6O{B`$SW_FS@=prp@hpt5 z600f_97>_|B&C%$FoSlwjK}@rGN>kuL`$+rhI{H;0gR$;_8J;s`~41sV_5`uHhPxU zB-*5@#yi8!rZOr=Yv>EQZfHadAD!ei+MoDYyh-Tfxb<;R7+l%d8bfzNBODC4iaGil zAOxCc*89*gvC>G}#rXsyoRgI=l89+|>J{L$VuVAdg&}nefTq5M7U4Wzjql%#c^~Y* z6;I7g&AE4@sY> z$X~^jRjZb}!qEG#Tlo3MY`tP?OoOnm;Kjb4Sgt>6izn&AlZ!!CKfl5Ff#90Y9u4nZ zckF2cYN|DDmNMEl0~_`CG+|e2EW~rQvDIytj$k9W#GcR}r33FS+D$A9bAwOBHPEU3 z^a)`%PShg4hodmRx)m7fW)0qIC-lnqS2#4Ui?S@m!cKHb&oN!lxYfbdqBa-nk?&RR z4Oc8B#lWcc&PfPj&|E~0N>O=f_~vmj&VV7uUfy#;Y*pnjvG1~4yfKpOF9M`U5wu6# zf4syqhH#Ddw8mb`j&bK150h0v+e&%?^*{NBUkLrtux>DyL20HfxhvKHVMJ5da0iH7 zfh4a!g0GBk=Q9LEtP0cC1zmmE|7x{UO+74Km5X%>{sYrI4yv0FFgz*OQ#jr7wve)S4ex5gT>AHoD^1!aLxgvB3EX;a+CaYpRx2!Ucm}fX=Xw9L`GxwD7 zcaUkhEy8wi1C*eQJKVn-44o-=(uqPnBI%fB$vlol4UeFliC*@Vj-I=(?5nCz#tGc4 zm8nZ^AqpG6g4c&lfb!E=x`r6;SLj5nHup(@{!Kgh)rD{*V}1wU-Kg{(R!GQgP1wg& z_4vCx@~IVu^dW3KhLQxWe}%2{8?BgA^E_qo?k!+HjmqHr@~R9K!&ROSaue^+Y#MuP z>j_B@iXzJ@M#t_vRJWApsYUvDHGVMov^!Q}WLnJ4ZFez)J0KpP?D3)I-D^!qG}75Z zvh>oiH<}r|ce8-{x3C_rEqCkDd1ruDaG*eWuyS}*htV$QL`~6d`7p$GLM3S3rB)X4 zP+B(5zQclatT{pTP=(XIl9x31_$Ll(XM;xp9Y`w~w4U;qsB!q8)XvRUoZ&a{Gj#%` zqW@!EbD7DF^9I1>_pnAuw*@Tag<`PY;@vCi-ZxTLMJ9xPaXt>?`FW-M{tobIp=w+g zg0|+4JS3=!M(Ev6v0djHS39Z=KgTW9*#;Phc{Xb)5VzOkn7u(DK~ShR6dO0G5zOSc$Vh10`%bAM|Si+l3-YW2opViOk>Fw<^ ziS4QMoEb-Zk^`*-^5282^j0M)RqGN4u0eV4&5BH7k+q%+p68zp?_)7F<(gHz7zL9B zXC^_rv~mqq1$dz7h`YE&zH68t(886`dQ-QWC}%KA6?uGbr*}8IC-LH)Fk0e*aB}S} zNWjyjf{U)5jl{1w>(snhJ^cowDFE4xyw?lQzwDT`TIectPZ-*=jf{l7TuaNf`!2<$ z(i0t3pLb(vm++F^Q))74o%OC1=`E#`<^uYb9OP2@iSjlaXLEqF`J&wBmvP4MX`R}- z^W9UO;86oKnxcRS#B!3~j%ipAJlwGmj-Dc&zzk4$O1loJ&!Y1WMGaFuZmaEorS5aw z3ExiuB*$lhxVV=P%c2_S#vngF6@GgJ@V!_SlV|9){kcc8{BvwazIk53jR*dI?y!Vk zm7BU2H?3Vu5WMBRnCLowxh1Joy|zsYwm%5J=;zBP9}atE(JM-PDKYb#&%I$lL#9X_ z?*JxNuRJU8ycF*D0TL?X(%xeB?8CEagQ}ijQOn&`1n z1>I9PH8-;XCDlI<@#Ro_s(fu(Xl!6Xy34TRYgfKJ>QBt{C+V<_b0Z1n^p8f3e|psa zesPsG`H%G$ey(m&*Q;?QqPBWN(u#)eN715RLRn|hb=SvP&gr63rWI!-g+n5q;Uk|t z6>f0S>mfRiT7$z-(-^YdYl@Piz{LYJi&ztw|(y3Ychk@FL0$)fV~N6o<&+6fX5gH=FaN? zxbADQ?^LCQdO8S}id@^p8q@|UZR8@2qDB{072f>*f2mucw%SyB-)jE$?U{6QMm^kw zZg*aHr@b!;eT6#vYTbvF51a=2K@l*bhi{$O&Q)4xKqunD&qE5B-tA0KzB*r695|yf zyd-mD(I(^NQa%&dpQD&#V(-d5^8)PY@vpkHWRB2ZnxN+?vaemoyY%ftyy|Z}8&iaF zUE@5mglxMx-Xcmwhg*()*QMFC7{p&5)kAk&U5y_gUpvK44!S87@QU=(#eDS($>Y>1 zJ9XE~^Ud!+zAxz@3Uvux^G?5~IX<1*OkEv55%8fq^nsP_ToV_oHklS!qL*SRGKv)Y zHn`J!_sS1zU;Iqq*}a+=wh>%y{KmzxY?q7&iu-P}KwNh3UYa7QjAfOoAXE%t51V`afIcd?TggY)?^Ifro z6f8AM6aA{_9{WEMC(%Yd0Zh2RZhXoM0sTqoxX$9HeJ6>q*PWqt;TH&BxHRiAF0^P0MWa6AC2rg=dzT%fL?yN7iy=YK9d;Z$EMU&$ z*_pUi4S?5UU3v3%CDY|c4_BPOryRq;ugk1Kb5C0P8pn2uK znOmc2c4&l;yM?Dy(aFH75f&w3Y-$5llQH!#SL?gb_u4d;Lam-EJ%dCWLF!{hd3;K$ z7zy?7Y!=@{n`}eP->QwSe0uVD&p4@eTm`{VIq%CG39+!Q zQ|tb}!|u1OJ^k zIQ4raw+gOgxW-=hxBjS^3T~|%&~v%1ImNX}1zUM}!Nc-1;&8^D*}G!`C=cl@7olHE z_tglP`O)>m>2FfEI53+l#9N~fZH;-JV9u>af9@atY}I6V&#;xiwlDwPVg6t5)M$P+ zuuM6<+b9;-C+es7OImSjRTZD2oH&|2B-W-mb>p)HAvD5@>lY*1hjO*>7sSn;y%W66 z;=JEhk;5m7(tFM>#?`6;U^*hxxbukO_6hz+*Oj_`-HiW-z4wf2a^Kdz6+vNvQnF~$ z1f+;`kQRy}f=UyW-aAU~B?P2MktWh>1O(~5mqdE+y@p;BI-v%ho3-{nXOHuovpvtp zcZ~OogJd`~BzOMzoY%a5*DPx`sWX;5e^|W}UZVVvkmg*$EGc=6ViCuh&24wKkEzUj3O#$FxGFKRR6$c%7@zR+DoW~*Nfz>(GwnO6Mwts^?F*ROwuXXll7bLl-GqNNR0xECe$&)ji7ys%w{*}wydPET z&%;`#*aV(q8-!LNamxq}Q`>)*21XHVGGN!`6r+vFrXE6I;+Re|&~&&ax7 zl6R6Drpj^B%xG%4{*`Ui_U02lYlia)FT{s`i5d9w+1}eD^m|IQGbyIO4s6!4Zj;kQU4az^#`j1ScA_F=^{e1b4 zohS<%t}f3N-Sz?0uAE@zFsF{?XzShC7Z2C>H$(P=WzVqRiH<;Jq{W9TaEixu)35NK zmybv7M?dCi43@MWUWL>Yd5BI~qDU*Yu3KafGih0|?hh_#iouI3HFITuK?x%z{o7xt zf**X|&Ez>ZRanOy)%(H1t*7dXQBnM0u!<|bi4LJLeM$G;{R&x{QNHBpW;5v*+K4%< z*E$ndx5I)EdzT7DU;4&`%=pGDFJn%cXrEp{a`AfHrGm)n+n_y@cxe02BS>x@Gd$aD zuFd(oE}22_=HefqTIjciW?lmMUq%A2Bt_h}X`{*{jQj}n*Iu!nDjI-oaHh=qsISm%)P9=y9zpgWF?l_CKdwIJK zK85KBi`qIs-YoO#90Tz~hS0G1k__32iCS+)r{3|;8nPd2xmBGvH#hd}$)j{~a8mMy z=>#m(Cgr&dx3_j+TAwihsynpdXRbfd?n?6sjYr`Bg6-qWff-m7O+U#6;Zj5#d z`y!x+?;+M*`$_p+G9AUywI?ib&%SS{@=(jtgz|WnzpK^H;6v^oPBtT^zhv_UEQZg* zoRcgnkG~~JRHV>+TuviZ3EcFKc4Pv9eCo*qCCO2F%8IgC{f-Nj!VoSqwq=^6YX!d5yGc;`V30F2|m+ zZnK}(Cv&vi!^o*-HK=~egFzq8jnTL&=6c9pD`nD+q6ksH6{;kB1N?x{;LQ^w}|6w)xFQw6c zJe*I(KMY1L1ZzjNvwnLKLQ>M$vSYv>X~~3asbVqm5WGFe8qJb0P?0brWN2kHE);+J zIBD0pu+w|m2(*WKLOP2okdE~7Dl)G|2|d&6j`0+3gxi#_)|EV9;9Ka;f~1@UorM*HxU?*e3Vc$jhcwd)8XfPrsKGJ6934Unp~c|KU_PvXUtilu(2d4S8iE zN|+Dl3TBl5z?66%oh|6~x%f_V45;S60Pj4Z^ns4C%x|b|cuJC1k|i&fqqLc7ub0>y z*+ReE@O$|qMDBvi_m90GNkuog;^u%x$tOAy#tpFZxuVgxYj@?`s3mP{ylLWO+^Ff} z@Ev5PfbQ%~$hxrtGe5NwYE*XV6+_@fSl~Lp?n{AW+Y{TO^du1mA;v0^0k$qz=X1kd~Wh)OgwWVEeXS_+OxWEbZ3fcQ2)X@Obq1 zaFCbYk%7&H8l?ZuqyUt>fJKhAO_2gL=hP@45x*CQxX66bL-ML^k6y9OzoLUU-8O!v zO7Nbc66I-!t@QY*kaK*t(5aO`=XBH#6yd1Aee4;cDCi1-EZYp_TyUj*lK2-fKpvk;GAv9avFrpFihF?lM%WT$Mb;cQ>#{;x4cCleG=fxqy zGa0%fi*u-BIJV`H1lboa#tn1yTp=`WKw2-I2KMdJZe=sZVP!wYB^7#B9wZu@^|sK% zs!w&yK*|L^vbBND9!J<6CS8@DDzzirF)3F@3=1JIMI~O}QwbH5*%bqXLI4X;pCZ%t zXGvMI%Rx~*$(Fl0tS9eBtsE4EA4f~6+78bolNC~dghtAQ{>7s&yh?ulXEtzUwlzr_ zr&uD3O2nY!x7#PJP)!CRlWThG$Iqp>5&R{;!rr|4li!+7VVl?#++IPa=V7%w)V9P6Dc5v=I6-4e~s+W|n+IU!oF0lx~GK49TYOGd&hl zTXe+M9+|ecv32zY)0skn7A1B5$CuYxoBZk(KF*@apV_JY33K27b6#U{0{|W0E9e)3 z4$2-I4n5awtF1j8%^IvdRMD;NB2yJm{ptt^=(W0>+LrIrxqA}|h`3ehd_=YHp3Qg~(_ z2H26)!q%2Gx7xb-B$jlj9$4v8aJNk$ogRG!OnXB1!x#!%Q_|b3xkWUDdfYl=5!h*j zwQ1*c>$CK8D!Dx^j=SNmSXyg?BunFbJ&tb?mh*bNSv2|(hlUklrP=@#NM#)`$YP^0 zoi0o)E8RT(DJM{7$IHK6sV7R;7emUTlIeeb?*GNf_4mVKaU5NLrc!?bji@eG2gP-I zV6B4U`=t3N!CN|uEO{KVbb$W5ekJc_{48$<#OKTQ}T*6;b>-sB^&#?huhF~8nT=>SwoV@z4* zbgs@IqXPUfqyqPLhN?+vCbwyY>2uw;mS~F_m4Ah-eM4$(t;rb_`xIl##`|L4srN?% z*=q&8jt7p;$kYANm8cb~St+nLWU=AET}N_U6;LI$+l1ow&K{KxP{zTn(vua9L5jbY zC98E_h{AL)(rLlo^?(FhA59~}hZ z*`A3%i@5x5wMYupgt6)jOC6dn`5oZDIwXgNYdmLohrpgW7I|l$hm{1}MUBhF*y@z( zUp>(ScqsK=JsnQn7tzg1uMlxFkN2m`6GUDlw270tPDIf04a@U1j4-ts$X7oPwXhnp zG4`)-Cg52k>&-4KRpsgaMG_*k7=XA8I-#W^MS#P$_JQ?)j)B(Gfw?8RG z5LVni02vy!+&gkkokD}PCjtu~IT0D*|D1SuUQawu3Sa%Uma`=xxw{0cH?WOrhBA$t z6m+g5vD>aIfLj30qQn<$H8g2b{7M@T>r%PM4;Djsgi3Cd5T-?}-)Ak2G5%Jq%pYQ8 z>^uIWv_pJT@YE8~dFc1V!H>YKeJ3{h1OMb(&OSDU-BVzvQl;3WA=9)!9Gm_j-ig)W zYrlUrfrT{lAb)xHLqS7dW7gtekQy`TZ0KmVs1S1Pk3U;{}JL+ z{(;x|St|b-)ptc)(YA=KsPS$`IZ7s*;P~gELH+G8&l#o}s)G^02YxzctXjIbX#F@=?=L(1>QrFwWBiS^xQ}bT32s8}*6CAhU|P0vMp=IQop{fwJTxnoAOt0R z+E)ottTH~GxpetiuNW`DuhzC1(5_R_%deoGOJY9WMyDEDngckK%XAXz-()N$b1)>+ z=PYJbaJ>y(=hfFO^v$|2k;T2M+Wcvza04?wjkICVy_jA!oq=4cT_vh@Es}dSxTdwN z#Dv^U8^Y`r0DkP|ir(?=PI>(iboc&c4)NwUwY6z7bxJlKHCE@-?;G+*l68P{fUrJ@ zAXa6G4vm;@93|ndW|+-8`+T0<(S&_HktTU8Fi3PW0OIwAbJ49EQ6c+|8m2Mg34M-O z_oakJ6u*FeyJ)+!Nap(l+fw}?LNfE}^kMPvPAEr=VY-aXw2c5VD z%ekVst)b5%`_P8#;#>QvKjqyfWr%(SVCot$iDs3DqTn$}?(oW2zOCHIGB({Ny4XCaSg4;5X(u%N|sglP?7yB0QPANQsWl z8=d)OX+r3BT)LFsY@=8ZQ|_}b0kw*?C%UOS6V-M3x0JbaZKOjo zoR84{rcXJR`K~;dl>g+>3|j`)dJ*}sq;uz)O89|-j`r&|+!BJ-ROL)vT;u&|lZIYE zmq$r{DAJ2+Nf)Z$Z6aJ5NPAvD3+iQEc#%;Y9a{fw36n4))Hwsv9_g24qH&4;Y6gKB zsL4irYt)mbKg!jIvZg4m?*P8bQMPpdr7rk*xt9qv%?Tte|AV=1bgNwJSR>!!nitn#0`k@(CbQ-F?E}GwF~~X>V-?zQ3Q6A-v}*Ce$k;7Y2bWwZvix$|#VT9g6o7wz_KatiWv!V{ z9RsmM(b?j6D!210xPufiwzJc*1NUt@+O$>|?l+og7i=YN6Mk5vZ*XSW0Yg@L8i2HTYzR^IP~ zuEQN*U-+eWpG6PuOvdC6H=y;LVW9Wb{i9pM5{1fEbEP$?$p(_ifDT{l{fWbh%(pu< z4GbLWG#+m&6=!vC{8TXdCds|d*1~_5^3WsVfZ$~)-llri-Y2&p3uS#ZC1t4igB}R5 zEm5m3I?W1LIeGR|4BB4H@GjN@wu&oW-)AJ*gwONw=K(%GQdD+E(N~Hia>$H7wHFRwx^i|dXQo=^v&0@iEKQ$=8}j)Q9MU&5Ef@1E*c^9yt|f`nbG9Q{ zzf3NAf-X>BM~0bw;~&r`KXifDdPrDR<|y+A7g#afT!}ZqR1thQy-qY}sf11~iGz0G zNcqIS0MYR-Rc&3P*Mi9sNifckV54Z)Vee5JcO`v^;TFCn)lX$PUn~I=@=(jeu3DYA z%32ZCxW&W-&N9_qfw!Lp{KY`3)dG5-1&1moJ8Nyz;3XX4HCnBY#&hp>u_{b9xQ|z% z;&Xjt37t>b7E?|+EU1d4Q|MIZnNzMj5Fgx}NzM*yrwEQ1oRe!EvMhx8!d*(CE?$+% zx#$F7IgwMS!ArVh^=%n0W8(_PQcthFZik#N_CbkuR5_6jo~}GWgv<_oze7aler{VH zhn!DJEJ^AG|4^=#Bk1K+UQVS)v2DJIgN$QsbADd$Btg?(@m#GS%3r?UzIsO5^If+A zR(rLgPVp?@+?)3iA_6a6Wh1V^dqaJyhr6?Su+`mdeX9?>NDRmnE`~V;C=zTNvc;#* zxbDKdFZqMedIkm*C(`eA>WR=Eg3lqsZfy$o8T2TjDHyY>GN&^eVuKv$ArssP`#~?* z{v@)U9R2GHdQ+vUzN<|Rs!>xJPP#N91&gTmDOU_u%v~dl~aYeSM?hn~@A7c9imi%Pn zoX3laR~N7=$B?N-TBFDq1>J#+ZEpTS1bwSy(!A1B;keQ;4CE~LdFd1?>9N;szSZ*w*9oEX2Sc{GHV658&-#?cy;R=f+u-x# zJIXvQ;Y?zQ!=Hl#6UTj^Rf+|M!@09t>77BB>4<{1h9c#S{%>Ac;iK7WV88l5$QK4h zsinXf&y=7#DB2yf+kR z_4`!R--3~w2#`E~V5bxKTcPmy#joXj0_|zPtQgKMo679&vCpEe;44E(J?+v5ZrO`b zcMyLNJIe1^C7I^2jzv_loseKcka`l6o#id5P#9djm ze96we^bbd&ans^u==*m&!ni_A2_#0zfcrk!9$%32iA=n)C|i#ZqHeRF$r{9X+gPEErUbb<^ zc&EQnPPxFRv85hPg{qu)|NE%9f1R%e#{SBz^&d5-L|CXea;iJ8Zst_B!~qIp+`B9F zt30DfIvUs5HCGAs@NMz8;izo+i%?t3b2{%oqcEDcH?9r;zCg~Lp1F@o zl&+zL3|x?76c-QT!a>w+*I*KWB|+OWVto4a?l{oSN5Ed*H)nrgp6z7nek#4da_l#i zR+B?AT1EByuP3W)_3J2Wzpq8!iM;){fp>-pfG4ZoEX)7KL4N(We__euHKbI4g(BKMfEV?0e`tHH`x1z3S}h4PMJHAqYp*qWW28Kf5w>9s?y)X-`l8ShlFJmY?hC9+siK3~e)ARFl70T7ayceGY zz+i1o{TE*ao;(?q5l#Q$w0EyDeC11!Q(Idb1rwfXn(WY$oPkT>9N{?IGtXWH=s~{{7S&|zh9p4Z>X+cLG@E2({Bbn|1t6T&maD= zFXlJr?|=El#cqo4PX+o-}$7Gt&Rs zesDFT)J$>ZeN|*FR2{?hv4;aSf;pu*lFT(Na#r@7ZS#=Jm$pWFM>Zf%oVd<0SnW!2 zLJz5LdB?Dom^WHiThNj?M}#R?)%L={xpRs4lcHA)!lMErm#5%=Ci=k%fuqEeGWy-` z0HFLEH`9|lJ3E0vKSX~qIWjVmUr0zzR!*+F)_Dsq&=?%NmynQ9+1uM&;W0TN>;k0> zez0<7zS9RgS(;uO_Hz^2)ile3dYCxZj?l z@CnfJ7N!?Vjifw0q-jrn;P1I%Mj=aj`hBOh=LPv<$_xm*)5ZSr&kOs?aX)Ee-pO@}&`^W(vQr*6I79-TIE+5AwbF4?D% z)RM7}4ZUoQ3OI&?&v*hUtI)`{QYq*#`j0?&$-7g5AZ*p1vupMo&MMM_aDGjk&%e!iCxusPlAAiB_55=)_4)Z(5JC?N;>*rY0|h^ zWxQsoxkf%}(SPrf^|NS59lQC;;!5)umd62Sx})2{8mo??jti*W(-NLQM@qJUitaI@ zt+*oV@mj;vMbm@H0_23Abo9HQN_UaoSJ^qcvbq)2yx!9YrLp@c1lRokKvZKVSWQ>y>zI;B_G! z4*An-=66mH8b-Ur^}&Tvb$vQX?BQ^poYw5ibKpC5FG2#a$^-XyL!i`gRfuQCN4W0H z8D3pto70{Ldi;W}^`*?-yXhe92UdCgb-!}~+I-1;9%yXto$R0?C)PPv^24-Sv&3Yj z_`^z;;q;kYAl+8gUGTPdIiY5jm54Gu8C%1Ct7UVffLx;fbigc)%ROcB0U6Tc@4Rxc z)Rx=GD=f0^!E()7>Xzk|&mH=B;cGZFLh91(1Q0oEzpnx-wrMZBy-#_BS$ZO`=}@W5 zB`J@w0>s4%kHmR6bE=tkHj`y9kW?!{EAD1D0Tfx3TQR)tC;lGVRW|4Y;Oa?TxCD|$ zGdaclMP(F`(Bs)%U{kiM;Q;LN@TVcbm_YLvMdyj8O++U_}a_N*|TrY6r$>w>gXMG-`c^PQUzlz_{eZu_s zHs8H864>_pNWQg`8c=f;8-fllQfeDtZRem-x8G>!4nl@fB&c!g)(GmvSMw%YB{(Z3 zgN2gCy+4Y0Z(KYu?i+f64$5=s8EwKlI0(w?M!zfyUhS@NNgh3wr>rgVOx2;2NTF$i96GkB&BTvb5*IyEo zj<`l?`+BfebBV%Dx=8DGb-6Y-x zcbjA?d$JJ1AI&#`6eXa(nmGsRfSThq5&C}*tIsgwn#4tJ7*AJPBghu#HxgfuMfERNX?`EX8!NX zsAH;|nh$B8y>NQ&|3*_P$l-$PJD0rw_ev{493=lT$s&86TEXTbt~2cp(|Qr{O}lKT7AIprsM zXfcP?_Jx*~PA^oHl|Z|eEZxfGisEj$iFZAr#!3lAQInhUQ&TLZZ@^FF;@}8==|PC9wrCAQ$YI z2w3yP%|vGzXk9qTaaPHIuVRvED36Zx64cTztDr-qGH-4?<12?tzF>r*EG;|#}Q&=;Aoz@Yu)Vlv7uPq>-Z(A z$4FYpF@TMsMKQTnhoQakxofrxP)f5zTdR{y zeNE>t>J^Q03}^cid3(EWkUYo`>!TI022x64geHX;8{zdB+k4xXG=f_05u!^HVMTA< zFGR^Yo)|dy;gaC5mbm)y^_TDQgn-3gN7?_}SImgqY@GWE^YWMgdRXEqwHuRzy!^f* z`&qt2ILL5odsG;w=$G$>^9R6DEK(1wa%q5nD*0uP2f>>Xww@ZndwS+}vJeO%(S!Kd zPJ(IWR+#W}KQ%lM8uWQu7b*#G-zJoJx)Bxu?CpCxILBvfF9%@8fvJFCJjdgMSBEzT zT!-JW0D^0)6~DyJ;rn$|-Ww7J zY*iG^ee^jsJQ*H~3!poenmX4mwtE}@Jlb*nJ-BIWBCfez%2O;~tM=nbT*}o<>2^L1 zTyjsh4vwZ&3Cn*%Ijy37$4gw~!mW~^An_!C-2w{8xb@!cyNk{JcZrMKvqRK_%;&By zmlo_m1SQIbl#K(&KkwcBIbEW?>UdN-+B{aF9XLzldQVwmccS>iph;VNnc|-kNfG_zN&bjRl<2(*9G9H=EMoQGo`=C` zmSJ~dZ*!$HF|nnsO1$4K8l4LI0)5M`*@l%N4RSY3c)WllaVQsvAm)?by3zHGn8=96 z=nJKuTUwD}dW~3V&RGczJxrD}k%XCbY88>;!VIy&Jcn9cG5MFg51dMVzFvOIl(wln zeI4ryy_!64VaD~uxSuXiH^Q*iFwmv0Jc%#Cv+&E)o`BgpS62#8R;#_RedrXz`%2XY z1BOkUW2S9g5f|1-#Nm*fxX?8h>hAkw5q{$icbl8-qk*+kK~?=@pQ#(&rU~- zmnw;?zrN9;jAX$Hogdzy5i?Cexpfa)d5E5;Z z;99BUv92yBo%5XY;HYyaph3!0FyE3XDURMfjlf9_qX5>YxApzRfDy9qX|c07T1{nK z+f8V0?u`As<_WsG7{LW2d9;L*t&jZ$O1nFHn1}DF^fwycsUykiG!(O>wpFUw>l0$^ zIgKqavj?Yc%52F46EQSCcY@=DGVkH@?%T^jh2&f`{cW+Waw*#AV`&ft={c>4hn?G1 z9dRH}1a#2q!=&bvNb!eb?YVmvw$KWbTvuHtFg@e{s!A(AgzQ~Bk2Ky^YEa`@ z0@{d+X6={Ht>!xno&%q$FkhhtFgIIkh#B$JV;S->-ZWYAtrU{5mtpZQR07#sFX?zZ z%aw!74=s6WP{*Cl;F_X_q8=_QYRc74iICdju=>j;DGPe(*bOqj#pm1a$d$ z&K8R#6amWfT{^XAPtxiB?3OJ2p$gM2m&x$EV6-RkG7vxCsE3+2BL+jwo@uuafDAhrBmp;!qb zF=53g8Mz*c2V6%$-0_!S(<131C!su{{Hm_Zt7V`gOSFUQO@N1g54RNWM1t}`(T#F3 ze&hFVbQ$=y71T$BV{ep&QsE5vuE^krqrUA0zU&ZqX60ko_~DsFqxlYQg{>=N7wLd& z{z#*-{$2fY6h8><^W92A!4OjXF5@9V+krrwqc562yA9c zy74{iYOpIzqO5mFQ9iJn%a6FEkCZ8OtWV@|Z2tW*p=n^z(5#iMZ>kF_R?^!7O)11U zSb3x!_=L;v-Z@?G3tq>wi_XrBGObEF%LTP_3CThJB+ui<-Ns{vd*{U7Q@Em(0lmGH zz7H4K#1W#Gs>i@(%3PpXVmMLk@D9YPNFVHycffxB<;k;bQ(cWfTu`cZ6)t(1YCa`X zEYL4iggoZ}hZ2dJ5-;3N`|P@nmT)oB$00*tzW4C7uugRa7{N&G->aplNS%C$APJd3 zVj?9?4H$cX`eI=uO;?rI7zD)r%Ja4Y&+Q1 zWP2zgU0)o`wX=eOKK1lF{{1;_*(Ju+9AW&!;!_A!}u9ioz3KVyU zmTYn?&-$7T>kCmC4rEn6KLg!aHC?-tg+`^@%C158N9MZFj3T9~Pyo=3OB4z84zFWc zr{>bN8<63pEYh)RN+CplWC}I2^`6@V7k*49u;Q+A4N0;yeGWpR=8E$OG{-Q;nY6;r zrKaB#EGA9imYS|YPe{8lM-do%o=PI&sZ>c>XWqE+^u{ZhXYZ#*FWlXFV}Vjdh%UjsugW;TuL32ymZ(pec(?7U z;>U;R#Umu>IZ0Y|G?}P|`B;w?fn)x2%MlTO3V&vQmT*6h#bCCx#7aE^F4VIlORUy7 zOHYY9O5o`AjRPQ>!}Vqu_5+b*I3*e-+$1VZ1{qXQ!DL~>0-bt2W=SUy1$2HBPrl#^M=#X{qf-4$w<}Vj1Yk z?mN~&M))zW4Vvi_><;x08igbim=9Ls1~x7}cPhLdb=~8I!6U{7F)mgym%TpbXRM_r z8Ibe`>i(iof|b8yp!j#!2k5)u-vgT%P&N^1j3Wm(?AYe)qBsBTQMa_uN-Q1-FH zs{hh2uE9a$42Ku*S#hK_t!oI(GDu;uyT{Ngq58eiB=y$kz5tIIml(gQ141T$lzYbY zA;E@T6JhPP+4wVY9=Q4vIIZP)Q(!q;G#*vCrs3`zgHm_<#ZQ)Zcc*%>9Xaxf17($_ z)sd&G=TZ8{M8a%ZviGGE^WVEQmwC(b=#gNIoAEs~DesE@4pLlg8oZuyNxgdYV+^MbCVE zF5WTMx+Z{SN?KgPF|FO|5!H!ZzNd%zV*)<7k)JD01Ens~LLTA24q`#w+^WHZ6LBtm z{^kEGz2(W6a2mcn5o4QE<`iiU35MkB1&|R%ALKD%gdn-K5f~K8+eS4^4(Zbq#9YVQ zx3W*IYkT{4BB0$f3VwF|-a zi;>DNVrPF*ncJhnlr7?OTL(Per}F(b04j4cJLJ*cS6eIcC5xS*R3v=3^j67T#C>($ zt?-}@Fb?x#X7t}$W6d>^S@VS9_hAg+n~fV}W%zD{^MCLX5~+W$E_K5fQn%M{FUL=h zxKmR>O6BqhXPw^?-vu;?|B`S&!n}awJ$IVVuUfuCu63SnIlSu09FmicKO4;!aeSkQqoK`*-$nTWKn@uJCdKzFVxT!!n|eG^*zi=GG>nsMj_^kJ(C#tf3Y&ZK>67f!GbTRd&$Y!?S6ze^g!(K2j!~s# z2{B;t!?X+MaEo0w9m^3ET>2q>mbiH;iEdvQeFFbMXI$YjnaJIxS-%@M(AyMy*Zc*j zlvBGtoxqX1oEbJ{ugQ#)^Ma6qI8RHBHo-o~-6S*TyM%q%aR`Obp!C+-PuBICi-Y>h zwxE(b!c#k^taf6Pr7I+>B2)4KX?L_^lkcckgvxQ%y@_er;5~EPErlJHItmq*VHoFVtAYiIgw~}I zXdEgFzTJAp(8gxQY*3>9^RX!#V<=Ydh4rM#b4m173M6^-yoEetc=H;!{jXe)8_w2W$AzxW1G)KOUv$Z8h2g4@56I)%IC-XGJ zKPrtrx>pfUuK172AA;+3YU}FQU#aoWU4H+g%4bJ|kH1`tI}mJ#3o&*2TF_J^&wBgj zrafucLNFn27GAdLj|cY#Wl6N{bV=Pn2RZr~+JUv=5%6Nxcwl^(X9T#!p zE?trK)@iuHlD$E=A@*It8f1KQsG0uQu~?Qari+QdCxG!wi}3(ob-l*KVGTp%SdmM5}is?MIpBvjCls%1PAwYzyFNJ zjKx$kydGe(yqgwMGw{=vv97*oXV^`@l|XrBi^f#P>ufBMVaJZsY*e`*kc@2f;G@1{ zF3;94Vb}6?ech$a5ajTr*cA8i-yg7RDlo`*O?8+3U63sJ^;0{|ual8{*8ve%l+>bjQe3nFqsupGrPeeyW4?UX)QoXKC z19rlpe6-d$Deus3kEo5fg>QBp{aKp)YOu+ch0o+s%cH2Pn$$Tj-JgU+x0~pjvZmP_ zY6iW)MXYAE`;M{EhB#jN+|qrp)&PA#L$nu&wXltLYeDj7 zlSFhpc>dlTS(?^A+b>Cn_JpK zvV3hgX%43YdjQy<#cLe_F&xP0dff~E<$YoL((dT5X63!_kEcKlBz4!}=*?>|?>Rh? zwufhoS$hqR)WcQ1;-QqZP=R2dnTS{|;v=HrEv^;wUh}YLzXY^Gw-P+2L>%&4o@E?4 zF#KgnwmT-cz<7U|fhHL}`%Nq4|I7N}0}GxG9`?&lHW<;J={-&vo>FD;lV5&VH@asQ znl>Dcee1h82;ccHPm43${=8{L?SPLPM7W7hb@!C2D>J)5oRsj5D7&XnWWWzO)=fOl-l zM;>QPA0zqNB?pV_bv9#P$~6qWT{8cUl_Wh~@Be9xSpY9|O#q`5K7g|c`e;cjA2264&Pl!dN%`T~L-)B9hb*Yi|ErU&KF z>l||LbuE080Uf$x>!i5Sl!fC%Uu)aE%#yv);^7|` z29y93?P57uqMHpJs8h3j?okUJE-NOeVPiIXUf~=L>r*_u4Evdi*`yv$9VGTiLh-dR zQPGfxlGwS)SuJ7}7Gr4at`b!(XRck2=GR*kj-gVtx)s3mn_Du6IU#Fn;F9gfHJ4gP z>;^j8)gPn5o_ocIB$T$tQ_^}8Ltzv9n~jF_)~DpH3BBrclC8x&Uu zD~y$AA-_RkkyLcI3&d3=r+?TRkAKPCq`jB)BpaM~jH38B!%B%rXXaC$+>tI5ok z1mq$w=@Y}~Bt()WywCLP5rjth=5ZC)MWn$b;iRe>8JJr)Z!h5QC1Kw;(Qc`)=Hx2J z3uxiLLUE-NLZ>PHtuYb#qL@?#9!#VMCbGfEmdtWutL=E^vS82l7|wXl@YXq61q+IE zQ>vCbF=r^Fq?A)9T`7K-WBauty&~2}@)jRHgnQYQ=-E*JwH^cBtFxhSQ02xSd!0Y*+KC4zJEH ztq0m)6WxMttX~g(Z++wL!pa5_cj z9Me9{QagLl0;hE)7cFaI$MUbpTg|1hs)4*~O_^;~bl%SM?frpRIY~M%5x~-tL+9gM zL;?8pPfgH{=FmfzTf~L7`Z`GD)rpU zT~?-r0lx4ybBbmhB6Ph8d6p6>zx93f0N9^Is{|93K+RRuh16w{{h^BZr_>3#{?lgEYGO z>~C=)mMc$IY_ckK*KB6HKu<;(WXr)z?7ObLHkx?jg?`I#K<)r<+yAW@U zi3^56azu9DQbXaxyUI@?&(+t7UZ0kxO8x)5UY(6lATa`@!hvE4{I{ zyP+cdCt=9&!e$y<31`BsK`oZPlJHD-BNbDveyal?(Kc3O(%9qvPANEfyg)18GtM43FR5% zQ>Hc!FxJ{@^huGw=vhgjopG6Mv6sCn!@J9MFw0+_9fNJsj&b+x-rv*H)32XC&~~A@ z>3$PPD8FfVQ@_#brqv>ik5i+ek7=WwH>8MU65oN4osfhOA9VjZ2|$^9Kmo)KG+UX8-5d0zdzJz9kFKOaZd@oR>LG)b1N5Wcb!pDkIPo;SuF`x8K zBgHi)aM=(hJ(?5z3&}KIBL9cI_Y7-l-MWSaK|qu$Md@8dnrw=65Jjb_C>`mb5So!@ zXaZ8CsfaWIg)J6p=ry4Rr3C~6gbosr1VRji5J>nI`;>j2bN1Q$dA{H8b@3ytaFMmv zy)x&#$DCt~@cZgvn0JR0RB_vVD7G}8>5+>aZ$y481Lxz@nnc}i^;LFecAJ(~dF(bh zU+?>sNQELJ$8^u#0Jmm%+l_=%?>a=TE>O?{A88}GCVR-T?dFK?riFu$@7>OUn%~xd z-udbU9SW^=VF@n;08ej!VUq=R@+V7#&#*0xsDG`yd?Zj?uvi6>!`CqO{zh3gC9in$ zq%KNCKQ{2-kW0-fsk6V?35zIm?~vL{dTSsge%uCJLma$Y9oiXFe=JtIYUm3>JUx|a za$nq-Rf%;9XWrdvqMeZue=>0Uc>QLY-|+=zI!=*9i)3t7MPf#nW|$1lPpkMmpg_6- zAWBJQqsy$rDF(^aK2vox?9^LwgcAs~czRCCbwO#PARwGusV#nWd3)RuZxff`+iyg5 zc@K*SCyq+R5=?W;v)Bv8Rk=<+sF;~;g#ca*sF=`~grtz+y($aHoGEd48jRe=7wmLt zANmK>m&IyPCSc(ZKIyN;ofZE;3XKktLN%GcB#-$fM2R*?cqi!F^Zd#;&GVh7h36~~ z>}ryMZM)xzUsF9J7~>hk8Idd=4DGc%Z%I3V$HMSlNirLPo5PBsI_3QUP!NwCMSv{a zA~>8p(>0<|lt4d_zWbAKx-cY<>+xS$VOB%gdI=3}WXdT?_-vKi!pqkS#f<&qe%?K| z9L}hgP7E1Jrjty^nA#Ie6n84Db#aqhWPi6See+OCdju$bMV3!Ngan$N$x&4rDRh@p z20D5-T%m83>?Ip!Qv)Bn%}C@SDX5^4Ufx*0B~DzaUvyrXx5(*1hc~M`C9n5UsqX6^ zO9#A#&&byoRS!DChVWK3Mep2}L*3S2%l9ievRBMZrwyIiadyXoJ^P%h%TsE7s3;7c zfsPO41ETe|z#*_AY7Ylsf&2@J!N>|y`hYeJ|pKO72mPgc#e+v3_{He6P z%gWk&@3?rh=IKB)Q$c*h{AveC^;rOmwqU5?^p#C#Q=7~6)e}caX)4Ayog+_=S5V#j zO7B%a4q0vg+OKNJ3mQRODOGZCE`gN>?;)ulk6fOUckL4_g6Sk)PjH#Aw7JWyf2vSp z+nfE2V?mc<*X4`&2|VRfVJh6+BNPYiX>tU%4}^7#y^*X9Xn8P2!s9gc-W~W7ymlpt zH8RYIY~HnKw<65cZB0YuAQ|F8qqE)YMEqWdTKN5#q@}j7$O(imcrj0Ptuc&sAakXh zaxVjq5T>&I=zK4H4DD*9*mVp)T0y(f(rJ$O(iuqE-Kr<46f7%Gg)s%=CWhgSAe7EO zV-|q)vFCj~sMFd3C0+Tiz|ude+;RQ4Rec=7WZgaPW@a~5|2>{fDYl#DsN;)#JM_am zAU;FhF3T;ksC#sTwH{@Cx9s3#sVuow`;MdQjJYZ26wC9D)a+c0zi*+xwpNYfuJlNs z4|Fd&t2=Ra@2RnNB-Tm>t>J1NwhTx7mz!sd#6W{czz5&jT6R2^HM8iI9IIAg|hv?2vvVHfl2*tJ#=Wh=_9F3<_pa{DHJIX1 zy1F&veEM_>Z1xXmijloRm(rF*H+4VzETqx`THq6mzCFNg-?uj!%HItdCaj z)QV`cA;F7W+gju0A&(@AcjF^#;fbcn4`_;;ri*ILeVJ8B@QaT>^$6E_BmONOS|{OgOVfSx4iRfh9e23c*Xh|h=MB;u#$!#1UtSS z5Oirqx-j#+q)c>JzV`S^K$s5U6RT$gMYBWj`|ZQ38pxP{m)ZX&W7^Vr@MHyhj)z%>f#WUjrtrGa(C*GDQP|IsjQlao?LH7-*R$7G6$DCm&52Ox&LMA% zvUg6+!Nv`5$UuVfQ)Janu$NP=_)BUI9S+K{!w&9~>&*|~u&cC+9vXh3piXP58drKx zBb&}0rc^>Te6`ddwP%2Oqfz&4=%`4cvXS+trH_Y9MbaG24v8*I)qbSO+MHmR>>$p) z_j9L4t)jac=H8_SbA>&TpR3GapSv!Yv8ZkVKV&IpwDwH<`t_J{o@=z)I>(wOQNyO{ zoQnSlx^l1$r;X=x-v3PV)@ha1yqIb6qPP067mx)2z`6%1{u|J=zxsmu_pmU3Kdz-(2ddzJhd|c+?`5+>d@960$ zlF%-Qyua#6y$=LpRd%G#mI>cV&3Na6vM!qlwK7=U-|fJkG7WXM(Ckg}_Z<10A_p1X z7P+p6?O+m@iTtseZ6sj}UBGeViYhT8Gy(I63K6f3;>+t_ zYy<2IKsqBCRU>;j$RBH%(BCTXw&_D(l)%v)|0I4@o4sZDrT*eg#u~iw9Yo#Ke2tyhoE$#7eioYENi7jM z69k&Lq%{sx-GjG9bE8o&>NE1pyY~lDbT$K?30Tk}8jeig*@P z8e=kFWwVU#n_6LQ@jX)+&wFfd3JP6|T~a^aA&3XM@{HM!1!LUhI5zqv`@e>zrVq{? zUWQ^M^Y^nQyUJG3U;fKT`0pGl2F!&g-0RFG3aODZ)fn~iwap)wsR;_%*K*`PP~oHk zY2L}ohfAB};+{Bdf#8$Psx=Z3uPbuW zmJ%to%*BDIM9l>j;e;1Y+sIELb1n7?U#K%wFvMR|e7v!B*(>(3!i%Jvlu__c0 zPs-B_W@F&%;;F)@7k?SPdeQjOw;PC4*|z`_sF^F5s&Kzf+cCJEd&udq4%p*VFd4n( z^!@tAt<_orpmK<)`pr(KRf_l8)cf9k8{{{f@b`0OmUJ~k{UyA6$f16yR03;*#4HlmgHc_}8De}0bjiEV)L!!P!Y58WHWc7OT$IujPVJEIxl)Pwj-XjH4}+JhCH<(%I<=V|Uc@~c2epj{1D6B3habOL(^R#+#KZA5W=GW#&qKXKp&&-bUbC%W^Q@7-0M8{ z`EKXhSV=q(wrIV5$3wg|m6V2K#-mY&T78)c_~?8`l`kSYfGjX0w(YUMxuMelN6@)# z*sP5VkVl#+w0lon9i5!mcWY&EOtkYOi5gdT@5fBNlrq-nfjk|^+j8C7dqmsc)nS*l zEM0|o~LT!O~T?9!5>Of)w6i9*X^Hklo-zF$E7xvG9hLJg(PFZQ`Ui*`q1bL;$xW-tz zvoWI?22=u_05@)?n<2WL9u1H?NEy_)YkD`yfKnjd<#@Ev_Ex}r+mnS9wb|9e6WbyX z98@0BFa=bN*O)m0i^z*%yUUrKKSn-4{PV9Jau7z2OKuE5CizqW3n?EJRFV-u;;MMGuB!RNWt!uDQM8RGA%59Kzsy92^*qQylcnsJ@@K=)#M8IrP;<`Awfo5MxVa zC@+O#8&=vpnd06Xzb3gt7&7od=bYIbbpd*%{LsQ2(BqDa7k+&IIpf)766aaYqt2-+ zgtnvXTS*B?&pIYJsl>X$5ga%I=c7vBifM%JwxAN(`Qfs=r^4))e@G5znExecEc3u|)s6sp`l*XW91R~!;ElF7vO0CeB^M=vxSwvyEe~>t z(yU=&p~zF0Pu=agbZG-;S=~N;n%&pu)FOwAzzMPfkp=qg5x_%5--r7+WIN*QvcnRh z56fElXE}WUewY9KZ#37x6+vx}4s%1QN&eqhhQEi6f31<)gd|1N<4e_3J$3=3(Kw)l zZbr3RDxR&5xlTX?^5wgAt<4LHQy zo2F@q4$N$H9PU`x>@}!8k~)KMgcT0V6n+itGOEi-8{X~;TDsrk7oI$r;aS!5(G^zH zV?SJs0+o%^7Bvo(EX(FJZakYw{q$`o<&-M4j6^5Df45gRJXn4}=1RCh%i$hTeWD6B z!Gx-Ufak5{ARdo|w&E=IJK0s8tguC>tLtfJM3jkTACbp=l60F9D;e6PBAsa~h-=ot z8LO!?RtI5fYG}fH#HMczzqz!GO(G!ZX)93sp~h1)vixpMcUQ!xZI2Vke$8BRdwDkL z)2HuJyNm{(q$J!x)%#nSmkFy;bZ+Kugkm z)BM(JIlH(>+Dha`Pt6yMSf$*AgF)0$Cbkm?B)lwfINsLcB;2=gC|KD8PlD!CrzrO_8b;s}l3yG&%An&ybPf zSQpzN#f?3ddx3xImIIFOIo~wzcaOaTisL~^(y9EA#Y7!8D8M9uS_vdG^zfJn9$aqg zim+l)Nq9I55ReBg!xy6+nuE53tot;+H`X0cDh|-$c!eI?Cp|drkVv&I^K09$(Nu<` z4cY+aJ~Id8xkqy>bf5WI6~OD_H^qgA;8D7kRbvfTSDqZK5vIs!t~*&h!-PcZeu>#s zQULCoNp}OrSeN|sli?|h>*v4@U07A#NrvMfV!xKP2*S#bkaOLcZEx2s#913EqKRaY ztXU!s-3+IWjXTu1lx1fgDuA-Om!2kNAN6LHHS+yrLDWvKD8lzA>M(0F>-bj+L)!pz zrcePN%`t8LXW?B6Mm^3C%w_kht@>r`0*31;Ym3gNu0M7U{nBV=)Fv-oGnG0|l1@Ik zx6WN;H`kHWvr0xW)jd;6@hn*KW1Dmr$=vKxE>zhYZ{cEjt;nRXHx8?jH+rf<>B(G2 zYwt{h!+p$DtU$FQDM<^^x?*{?0_I=)8GSFZKphIk&yyIY`j5US^5b?av#7(Si{Nq| z6rfAW(H~GJg0gAC>1p(v!Q~YW3 zQU0z#qSPjrDd<0L_a?DeW6cHVKy}q=%uxf;yRp2WPZUEw`Yowr9HYU0f^`LgZ-6Q*q0mDABE3x|*=COV%LtH#z4zTCoUER0!#gDZ^B}wL{`&0J^BgAr*8c0zk7G zwMwmFg~)=yijO@st|AUfCATIRdDS(e?dfV)4t&_<VQ@JwFmhq>X|Zur#eok9awQz=)l}$*Pw_P6pF?+jL@(f5 zTb=U<@;pfuo+N=e!cKpfB^7|pvYG0jj?jvA2+_+P`wphL)-+xpK`x4fFI~|EjaLw$ z-|{0tWwp-QT(e{3B`VoYU^+~KY-u3nqvV{fWWNTz4!wuB8@i~Pmj~*TQT-7ge%uas z4%>l736K1}8T4CpNdz*b^6=N@KPk638ZI*=crg?({$PC07~#Gl>P3)czVO!Z3NZhd z#j#isNyTn{GgjD+s(GsfeKNoGQT;7?tl~)R+o~w=7#MHExHTrFwfV*YHv-pSV8n4& z%ADftBK1R5p{4@*-q8}&1~>Hu_PyTtst2DP*Heu2h!kwaj-Doja53&SUfZ(-mLk=M z_(+}Q)N4p~pnWb%@oq9WEBeK;P*9vxHH#1Ys~j&#^p5CxQQ=K{OSZ_&`}7KnC&SM) z?+=MMA1z=xT?6F>Ou;Aj4y##BZv(ng9&pNsOP!&wQdSK4V^56mkf7~ZlwW#6)_RD9 zz}x#1r6>0i?oX__z&*)lG8+)K(t3}#$E=#EYfcq@+vN3D!gl+P%i?LxQ(=jX07y=e z`0|bB!TKV>ME-cQqGXOZ2|df8InIS~(tG7fg%c104qf`I&UsTz`>qPUv9jdHR{=TQ zr4nzs)ucHZ!VE$$2Jvc&7Ne1RWGh;}jen$-eeDHVbQGK^2=8XIYbo=rJDp~V=9&)) zu4IWQWx}FD7;dduYhnhPM(aZ>F%!NHlw5K*3_7c{Xw2e#HT)v7f!oH}b%1zOZ3!DA zf?4&8+_S51Cy($f!ANLt2Zohdt=?DN@^5x~J+U|2Z^oSi8(ixuI5Y@F3bm(doBZNw zLj20R-JeT+`7eg-%7@Uov_>f3pNXLcCqx}-6b)dJWk|59u$ei!Y)EgxxWZV@h)lfv z>i8Xq{E*xcZHHUqSyH7pk*%I%X`WJS!|MF3hIGxF`;`T^~a z^dG~!&PUVem+8s*nM1MGdQOfvXYWSyxt!1se*y_V6Eh!^5LFhvL%%Lh7v{=L^L8Y$ z=2uUK@Rt;^cd&sHL$q?s@AsJl!;KcHqPheIJK4A9A4RRrhuSPvIc=Hut4&F49YXls z_V}Nd9blD6bBQB2rEwI~!GKsmd{7o~2BP!8kDXt@99gPH$`G0jEl+6_bgUTJ zy)**`o85QO!g49&yu&L;n%|*q8J$wZtU-r4%P07Q%CTV zftG~69dSSx$?Qe;(cK*0=lDKKQ-!0FHI^0y=h%=q7Wy;WTCSRLAn8~od}>XVua<}c ziZ>TpdP(eKfC=A1N>FX)WW>+WuY;y=To2mX@DWZWZuhKn>h89^(E0#e>F<<+L%_*N z`VcEJl)?WI`t*;z{5%^c$289yzB@q*dMyvTe_i#Ec0NKx937+Rk#78c`&HeG@0ptfj2s!{M|gDBawp=V z^hF)R#gldS7uZ|b=^02>MhivIiE9md>g{}V;eyHdEa~VYV6)0j_LkD4(d0}7q41e6^-Qn0!{S4-@@PN-a1)=du`f3 zy%t}Fd+4{ZZbWr1^h5SbEu;iKjUj;+SqoU=OC7O8?;URZzW3@k1c1xz)$b0!ANsvi z`N#fdT8y3^Mvs!;0XFfS@;f6LUNRH7d$$Vw5!I6y5U1-08lv&?SNt81IOx6k7`<6c z|2>*g6&~_I1a`cE+mLyyy5P&xh8Syux!7YANy*KU)Tl-X{>BNz)%IiIynLjOeQd6; znBP{#_e-`7y-q=F+D)^tCGdHL85|L3ri+MJ%T0;i3Co}}T=;MVjuVOA$IOro#X*wi z=OqH-V7pHeABRaE@3<>e=cK&$;jz^1hCFPP5J>WssPIBe2O@Yby#CS*a7B4t3;6I9;0PlEY*8u%@;HxB(9ePZHV=r-q1Ht| z7#h4*;u(Ginp$e-BJ_qO?3DLy5e%m%b+xnGQkARBIudO=j>^4Mqc$-=UhSne_IT7x zx9bt>n%nZ)-cPPfyWri*SzOxYaA&B`+u_&`IVw>`9azld~+0SSU zNZBl;8j@M<(7EL3k63zjs6HT01Z#K@&&pfmn(;j9tWgj^@8f6wR-);!P=u3~FBEaz zmHO`A#a=Kvtn19^3?N&g$w)+whyjzy)u&Q8>D3`i%?p!;l~3|i0XZ5L4om)mC_@kS z=%b5Llh(|PWCs@>>g-^LQ6Z=JDVJwlH_3sg$H^*@5&J1fp3_ zn~>FPlcl&vNI{0_fluOL{uczUV&_&{BixLfE4I6j8E{1KLq12BHIBt;1A)F~BCXE3 zM8!G69y9)Uh-QhoCX(?rQ3&<@(DqCvhQ0YzU@3ct=zhX2FYU)joen{wm)U3Qn0+PP z+RO>iU}@m=evTv2*EGkEDp0~;hMr&@;c>NI0XZ4-K)54)CPlqpL+y)5R@9#<@QpHw z$vXc9)@B}6ipPW+sTlvXDP-49o0K(j3*se6%JH6KcYXi+!U6k-;)A)19$p#2|3p>( z@I`%w9%CD0(+ul*R*uZKmp5V2xT2%1irjI`7riAymmRoeQfFs&GdJi$e=S&AlVU|FGO*8n*(zU>Z8LS@i|*LYm!hf1-Z~Sm4%SE%w^ID}qTf^YyJWSs zq7`>tM??~A7jy9d67w?+LXHO zxF)zXVs}8Mu4%$ar`2^TgwSt~W^x}7$=etDy&k(VUw`0JIQw1gA1U__TOjkPoFDc4 zShyML+)<(ioOy9hFXC8-zaEi|%VNIB{7WEO8}L#jKWCuUD%_TGf0K zE}q*Pz3_$a^d+}`|5GQNqmF3o%|PlR4`}(@-w(@7WdAKdddd5O>v33GYKi=@^5djt zz6Qsdkm1ddjO$Q#HilzM;`DDAryYvuV?TlT!>?WwPfu|S7XkGMW>jF>1UY5M+iC%^ zeF17+!u(ZF%MgJXA}VIPU5L+EA-0QY`#q8SMg=SBAJFRD>4o%#M_`r4+TG~vvJO>O z&hg}*%6zkpdAG@C~bk#3|j?MOrzM@({kG;+CHV%qcd%l-RB;W(n!ns zGhGU#ZJ<`ac!)oTr|ijpX>v8C=rkm+*g5(1-{jw>*97wN#>IP6WBBC&Pwm{^(U5uGiAX=JD_=87%Hk2 z-=IAAVmLY%YAkALWJ#!g2Ag=K-EF^VR96ll_$0qmzKB}2Ex!~rYv90WFkyIQ72PqG zLxMHrVA1AFb-e814fVkZ&%@G_@!5su7s$>+TuM2RNNh97wV=W>xaC#rY^K9f;LqUS z%brs}d*hj5#6Q>k531#Pc`nh5oO;q-yFfby;$*#PeKU9UYUtXh4^3|khq9tB$DNtf zN;u6hJI42xPq3nG?~y#qO@ItA&3c=An8ReXkDq=ppH(M~=_8d2m?M$&>Pk12WgLsDfyXYFYHVj`v#wUme$W}PT_YR>4$7ixTc+ikZ_IXVYdqI zN55rZy&{_bJkgwf{Vw^jax`L|XG4}!N(wW7?)V2V#` zbQM3de*H3urJsGu-f8BThbkdADuJtHD z%j^$fEVDSPH#7K6<`7h@;{kg7b0}1LGEPW8yGfZRY;Digtk&v=UZ)3f!w>9~{0XO4 zT#M;=U0|+7Fs$!u_Hw;fCguH%9bu6vwBfBo$nn+L{qAD*FI7>$;e6(Sk6W?rpYN2# z6iXP+yRzvYaZVA{5Q}qCfyTdWF1a|u%n!uEbDT*Zi2thbmrUk3WjlCOQd{7Rza<<1gu=`>6pr#K&}B)-uo^q%w-77Tzg z5 z2FN|lx(6}w0n=D+&qZD;3P#xQNTT(}1bVI8cESLom8{Ab?X^&Rm8ub(uKUGFQXM{_ zR(#2vWp=`3KRS<4si-Eul$@Evj)yA61rSDL@h5eX=AU9S0NWXAq?4axb&SeiD z(yM&H?|&@q|86G}y&U~58keR+xzB!_+l%>E?$69o+)tS=uwGYASk@dKw#ffIpatAzWPIzue|1EgUVTPXQDvN|?G+q{zM?B+FR_ zTB30BExO$wh9})ew)c5OTpK0^)Ko!!lh-G{9)mTjiNqek{K|0E@E3V|`$IynCqma} zQ-VSCxagx0fThwrM-h8CZ^RPeU$5g|&0ER|?oqW5xHS$LpE-{zQTh6M%-b?pi4iin zd#F6h$@Ln9=*p3eC{kUz(Xccj>HRGy@Kt6g)YBi9*G_4}30qKOJnl@;Na z0#c@mC-pqWLP-t7Z9%m|p8LHHR^)#2=^%I7MDC~jpCxRA`8sY z+-%{)@txpXwaOHVU|ejK84~?0>Ot5D%HY!ZVRf^BKf#fny#F2S4px%ZtW#?T05V8R z=98y+v1iq}ACi~T8ZX>B>3noJv2iqCq+zW`6WkGLz}~?ax#ynVIKuaWPIz|4)R}E+ zKq;Uzf5vq6;eyM~X|-q`aVF4M^)dg&+}qe`zuo)n^LKohe<6o8CsBuY?r_(0Jaz6v z1?VT%Hv8<=>FDBgAilTt+q|QG-9otI3 zhGyC4w)BeLMJExdt9wW?w~CpkUk?;~1zAFo>3_zaK#1Naz!>f4YNrTosGA-AFiCW3I;S#DkxOMbiff;x=AGqFW|aaM z>7g`Y!NGKRb^d_Z+@x88zKL@yckvhUVsI0!H}ezXb8tPS#1HIZnynVpJ8Jk*$#w$f zx|izQ9?nY9vIRi9-79?H3D;-ZPQK*(k!xvfhIX#64oGL@_v_P2*}Xdd$c|4845&W8>XeL~-h&Y3>-xE^;hM+L}_F4zbUkg?=h{k=;R z7CG7>`VTsI=t2W)c(O>db5dLj7d06^GZszWqI*caFCi*7Z}!VuZHO$GX^#QM)9#J0 z5xeH}K^3$Rqa$du?AIY`!0^7xrE=JsKcq)k2U^R(gnnEh+wtza2 z`s3O5{I}Vr;0i}^e(!rqu-bOtrCh!}K)m$R1v|_bSm*9=CVbM5N*Ku>6KUSPcA?V} z0(5~?ieaPRY&CnW;A|#7>DUBR^|ljjGPN2K=vBrw1o8vlQav*jE_BHwZB;;-h_KR{ za0A&-R#c9+d_}nX*cqXDjknDTG-kqI`FzR$ne2yNj7wa!i_P==*R%nk)#ylQBP71h z%>7pWpS5TKI}foKJ-G3Z?U0?5O;qj|{UbJ_6JtlaWvxz>SeYK{V;o^DOO&hOQAe5G z%x2aVB9TVM)!qQkSEY)8D4Jn5&k zWa)5%ClZ@}qYT6CPbF0|)ZNM9lRJ)m zj-5e_lKr_-NML(%6;B9&j`;;F7O2X1^Jk@itEEq^*Pe?&vIJ*&{kLgPW+C=c+O84sNSs9c_FD6A(}mUjFnasOA$oiaX`<>?k=ZhiI+vlf- z?$3d@VHi})`F-Vz;D-@j4l{{_K1Z;F_qV3+;PpA%`XW+;h(c|}Xz-7ooh9472Q38z zc!^^js{TsBcnHvidca@ZKrn%h)I^QmFV()vWOwyKxfJ-_qtwbN{Cvg6!NcpkqcU7S-4 zbRg-r&MODylOKF1wJDkw0{ATltkWI0oA%{nG+;w3@Z#j~i$K|KTGOKB84?{C$^DmK{Hc;$6=ADnDq{+f3& zWxTJ7Ql`_ho?Bk<)A^2Dtk1vfHKv&Xu)sIga_!HiliM7`MfEO-YT4;KQ(1N9ZZzH% zs10@~xzjquSqVp3kF>&tR9Ba-u55XrO;$SdLVQ!*k^}6A0A*?j>5T?u6>tHj@=FEU zTaOalrFsLjINB$|WHCQ1cSg#{VIO)b0x#taj6Zf<9|g~BJd~wcPBrhgHqK~$bNmOU z(UvFmyr^{J0zR4dcNp>ypIx{tebuWk&xt_OfqgEFIB+-u8DT$0SnAn>dNN$NYRqlP2p+t-`AjAgO;QA5FC_nhGcVU_v7;kBmDONK39kTwz?nB$O)I5KQ zai;u9prtGTqfxL6fZHZ8s_0tcCVtL&5RoFlRQLu^lict;EzdzmguaFFe`*0j} z1k{cCXZ82TFS6np+y(+1T0#QUAtVsr_Xe4XTW$kl9RawppUZvm-AQ~qp49=R!&+T@ z>HqrNVw4U)fbhM0>yQ8MAC83mcCgXdseM{S?)9H%k3YCQZTnKdjP-tAv$gPFEZD!V zqGJ4^Om)1BO!5EqL;D>MyTfyhNz4EKKmRkf{m&!dtiw^xkcTd||9Ig3aGn42;QzbK zzqr-^Z#Dnoi2@$lBuvE*sMeqD?KiZZ6hu%S_&YQ{eDAoC!N|&cf|XAm2M?b8oE%cm zY|WIZ;JD%as*YdUWi?}a^;0SMqg7W6aAw^5tAVKH34sf#f0N$ zm2p(I`bo9lpT%YNH2H~att`}zwQP*N{^3dtOL%&NL7d|Wv!RnN z0~ZDtZy6|Y_?sv_XM*0}-uUNBZgagtvqBPpJKtkVys=!EByZ~g_|x}=HvhVw7$1Cs zF6&_Y9>iyRP!oS)HqS3WG=H~^b_nC_URXe;0lTk9feTN2BvM47z5_o;7WN{Z8EWwY zn?%Axr$?|;;QkfB8Q!&4?7FQ(&=2<{cAV}fvNu^5Xg{FlFn9lyD`P+@lMlbnw9{yV@HkhFvhOgUti1 z&h$?@fZJ9>nZZZY_lrG=R>$<*wQ^8#$Q$>>b~i1{yz4HVYSmD|;?fKDzpp75?DmUy zOnFa%ZIJu6X<;;A?xdFa)*uP&r9eyd@6ZbI4e<@fxr*53=ZSQIP5N~Kq@hG@?lCxU zEW=6;i$Y5^wf8sC0EOZP%1BbAZR^jUvVBA>V3@wOW>G*HeHc>bbGHtg`PNgR`|);I zBtT!VilmGb@IriB2#95yUTMM}pwsIdTAr^F=w47h~T4XYMsC^7tO`N z#QAFwTs!{(=5^Lr<$nN#(gadT(zQ$|E~{Z zIM?&7y-8$59i>|#0?K?^@>a&-vXoSWEz6eJ`8K*sJ!y7W3@Yf%`y{Ww_FVK(7#Jp* zGV=?ntbVpBfl(Dt_Ae$dZR(3}WdfE=E}6)EV2QAN%?Yn1ucX;2iF66iQ8$9EfaXBg z;;gCKObrF225%%oCYLkw!1~{OiQ@0%7WPLi%*!Ikog(ECdsB1r*AJ$An|(NM2=Y0z z-8z(xt9}e15q$c`3U(~IDnsf}k}FbVWV}I^-oDE{fT+=arF0WuR$M@yTPy+B64M&0 zFLM+6ew=g`coFLUs(hup3qQ3JTxFQh|3LU+3R= z?WWZ~(OjF+_NqlHLiFCt2Dh}>UwC^=vy=&nn&&6F4PG>4OsH82_?^^Pajy?`o==dJ ztYeYFv4H_|0{IHRODk-G@93(zlZd$aUo?HJ$Aiw^N)Yncd2M4+-He#zO%{CD?AgRs zvVV$oacrR(yYktjpA*4xY0)I?A~GerpxS+o*-TNaMQq;R2So%UwdSgN?>YInkn^r`Nc+aL zPu{W5Ft;>`UU~RC{qb|va|g7YB*h*uA!Z(yZaS0nTZ|>9>vg;#f@iBV{vhi97LUkL zEG7@_9=;OoaSo0-#})CiA@+2^-t@|A;&OWjU6f9E}7T3I%m>f z4{D}xuwIgU-AY?-$hbbp4ftYRF0_JzU*=sHT;8~XEe+1)yY97k4(vA0(F|_5x)}(= z9k1TN2Ng)o15^skl_95||FaORTRsy`;8SA(yG@*hEwJY4L%?8k8FBerUtL(_vzRzF_;Fa zR1bD%%@HcX*S(vMg9q!V+akZEDw%lT{-rCAho1Q_nTuI4I?647CZOeWm*a{tZgNcD zz~*J>%b2dSTutTIpQS}zj<(=JYkxKT`tZ7a2ix_7>o2iAjwGbxvN3PT2i+y~3q*;E zM?7ycq@g$;X?XPI3*5!j0BSe9dWLNQ`|)$DdIAB{}qaphCRiTe-dn^K(M6m&NGHu(n(Ke13&hAnrVFHOa6RHXgT zgexiMKgJbkpk)!ad-UHfG#A2JF_}Ys1p4yY}f4ADCO@V-t1cYf(YyoW}mG~`*&P39qhKQ<1n*< z;|}#r%P2d`@q6I)DpGNCO6J$H+3=Ma_jRb1gK8SSv%oa$rt}~b?3301#ziLIex;y_ z7LEvr%t}%G(&TaQSdcJ6jF#&(o643bm@L_}fdrh-ud$eDpIs=`L7?K5b){7i?B2+C zGmk-4*`{IN;IRI86O!LGTM1`m#f`?pEi)6oSfd+sf(D5$3e$W>LF9PqKF8%jiW_%v zkORS1N3g4Jbg4PUbSh(j`Z;O$R7)HWsnb|!sbS?3UL%)etEhX!4AC=13{%ib_PN(n zAGoK~9*XMe;+PAv6Co!u`gDKuAR*jkl*VwVt4<+3Lctdn4V1cg;a5&Cc(%tp{T6)BCJ@2ndiZ*=TH6~dy|NMK- z-r6<)>s{CHnYx($+ymsT2;%o%?7!U1(5mTtW>_PcW0)n0-hu73zR!oX&R)^pi2$;+ zeB`!nq2NyI+2EZdWn0rMLSHPTWa(|;n!=fi=LSqWAOVb<$Da8Km!uH2HV@P+s}3r6N@!GC!#3OjMH%IdELCx(5yY){oZJ8HG}28d-eB{2}Q)If=R7rap+|M7RLbj zAI9WD1%k%cd+XgXUAycNn=$URI2lKlkQos~Y(mk!F!vs2^+W5x$BBE~AVi~LB5j6w5hBAR3s?67P^ z7-B$hDP`}F117&d`2?FROX|@@vJl1YkySg6;K=Q4>!q^PpSK+I!Leo^R3|EpOP?u; zKkE^O;QGDe$=JsDiY9O+_vMydSsJM$924A3oJh%2xv%r4j83y8WU#*$a5?gCOx2|2 zGh)I@su~@Mn#kkZO5I*;Nctt0rzH`!3q_4+?^`=(Fa2PRzZA5zaLH+-E4s0plhr-n z95YJcxGW*BJZOeJ${rA}nVS3b64Cxl>Key1dA)MBQ~-K5wY!*-FM+Xqvr(VypHzVv zsLF541 zHB+kV{q@Y~hm{XP@3ipRB9%(<6y*dMab+)!BA)X>V`zCzhR2FTle zr1swuXhNHTo{Oo0v5WP?yW)cz>ReU~tS?FB#-rE0uB(?HxHn6_H_Ugd`JON2CizqF z<@ieo8}-_AZOUpjxr~lKmmGD_g@N3{@eS9+<7W{fU zQ0f<{K!<|wA0(j!(rtvo^-!2`*m*zLHci%9GS4CXOFFsd+U?D)tMt#)246OGNrOE2 zB4rjnwMt6lj*8uR{!#MmO>;Pg-cR*@g**w<-EK6#{Z-*Y&3FNREl0UPs#~fkF>JJb zJkT{;t+YQg7Agt-E?>TO9+N)Muxb175WnN<@W(8VDMcgc3s_g!bLI&-09P&a-{LecoU1kLMp_zzA3F`?}V( z)?91OnZOqYevX1Rv>>WN`qF!NgoqxZhloQWY@Jf5+BoievW$#^W(jB(k+Pu+H=UOa z=bz8a6MFQcKO%PbVTEFQ3tgA=T1_AZ1Y?jZ&FdB*S}$;ku8;j{MAjdjo^ zuC~9Ae%3C{Zkbyh2KXN987GN=p%4#IO^glst%kt>U)}Q5LE#~E4s zp7}vQ&fia@0~>-Wgh2LVr8TQnc8sB?X8MWD6V&#S;NPZ-tEkjR-xo|}CVQz3cRbsy zV_n>@hl8%%NzzB=zP-^1sPAwIRWtAD=;cHr_3h&lLtFXZgoNC^hI|gnFMc`efg@q@Ez)4`4$kVDtf*TVRCl$M%9KA z#ZeDrDtdFatX()DzXCP4>EVW-Zjw9@U3s;eKT9HOmAPWS82N zXpF&asDD;WO+dxOriLt>{PihF=Le5~D=@m?I&j0<_8LMqA%l?pB{Nj~cG}E?Tm4UC z(x2z?zbH4!f1b?{kF`PKVqSScEmseCPF~Orw*?dFel{-CmX(9wBSqh3J{ZYxyen$> zs%=N-ZLIF%`C5yy3qKa-c8uILbO}2-eg4bvM4VHmh47n<(PX@5FOxR~1U5x`k_VCn zE3d3ByDFFV{DGjYt-b}K^QL32q83L!vLuOC8v`C5A*(u1n>)UdHqIAc5{d8Y z4inj-@Yahi+_J&IwFHUITad@*Ds>%*n`!MO-=mw5>b@J?l#S{#_wvU14+c&2a#q1i znOa`pJ-mVtTra?9k1na+!BZo}@+)Y)WA)(h<4QKaFjM(+YnH5g+lXlLyzc->(1-h; zo)zG!4!A>PlCTz&uS!q)xAaSQ20A9nvs88n=~lSKY2$6g8woe;t~*rc4N;!Pyt~Pj zZlefu-gfi`PU<=si1YHicZPG}RuJ^}gz<y%mMDROtN%%Fh6Z09at%3es zD(m*DG-4pNy>XGU`n92{C~sqpEkNr{&wXFyP(M#hy})$U*w0+bQ8GE$C(zVCSOT97 z8qmHh%oi^e0@z+=1gj~fZl}{N6QykQ9VulFI&h>)qPEO*x&aAO?XyaQJxySIFjudv3~3)3RR}HRbPFH`1<<)#H9XAnVN+Z591ZZ zl(IN0Q{heEEX6d%UPZW~reYzOD3ew%Zl4w~-EDn{m-;0Q{AtXS```f#P z)(2aRX*hk@@F~iWL0XJ3C$Ti;lQhIZw5p8t(Ne`JEh8-$q1h7cB90`;!fjF-(pbz^ zqbry11f{(aFLiEdiJM5w4HZUMCQo;0J5RQdCTg>^HWCXA8ZFQ3Vk~5sYQD@zzxC(c zGIUxRq<5=M_SW|z@fYf=dtIo5C()F`a4hNOAv1+a|nyJ+=w|=Xs=gS^91=v)Y1%Hc!6AICfBj4M8A?(1|?K2r@qpXqdVQf6?4loF2eo4SwiqS z-cCzJ3f*E&+Tz=S_v3NlJk5LfijI8}>Pjz#-#e}3#)sG;$0(0F2D$K0y?SmP7c1Wg zc`oVCeI5n&zN8l!xXVllw=e8gB^_MtE~d0xMR^AuhZ|UJ?)zE~GGPXxM`+QL>)vNj zCN4lv#nEvtF>>zas|>2fc@(r;JDV)9SnVL<;4fK9|3 znbe?lhXOVezk_wTL%(iskFTa&5A}NzHOOCWV=W4$_%nttS|4`n9kaix@vDfg9mx-_ z2D8DH&Y`RqUr*>L8A2=qu{uI013y2)Jg|CL4S@rN!7VU4)jYs_-6Hh3c!Y@Fd^1ok ze7h0H|0G5`wGDi1ZF{|}6*yYaQqhJSbk?ozIUdhd5*w;c7wdZbJ_61TekLjU20_K&YhsiT9_j6k9r&w z!GlLrF)59h%iz6=g>SPJgc!9t%hR?^wG<5@n^Md{w>^0q?C^DxV$BJx#rQn%(}oTR z+-gh9!hW`}pQ<}WAFmx7ZbJ&igOW69WQ`pTXge){GPd*vA^4F{ly9Jk*~Ku| zD>IfwNjb>osC_Enz`AD!<|*_VYWgUPsx1F`w-=i85xLw}Y+VJ{@fu0~Dl<4`seXKs)R;Y|;@#=Dm z%^1l?eE4th_L2smhv@vOc%K=mjVDNM&~Mve>a$B`MJ$(Y(JvS+bnrdV-L&WimOfjr z>+{k`;H$d+Io9RD>jsjndv-rY?XnZR(h|jAq+!KV=;uEA@jesJEZy6-G3r@$qlUG2?R(re6WOBh9}jRR zESwU#6MES07k!7;CnP-hH3$OmYh1=Y!(LwAx*~qC<4}M{-e}vy3%s^V$l_h9NKM?V z@bVl2&i~ zeV`%%OoI51wAIx#sinb@ykhr5^Y}burEvRB2#|(P)>SXnUjp309Bn>l*JK8tsgjcT zm?;qG-;!~wbM?C!%TZpu&(wf+RzI%M^Macei=^b(L45JddtF_naUjm$>@E>X)wdOt z<|-2@uGZa69Cws>8wQ3>unf4hPI=Np_@mn6S@o9(n!?m1uTeMe?&v#IVvo1#q_0!f zHqGU3B{cqalM#*+=^@MNU|;L*4FJ06FZNs;hz%ilRo$U*M@quk$Cb(`*f^<4NikVd z92FzqbPj2!I$iTJZ>naBf0%W((m^gDyrGms`jp&2-{l1hNp`*P#Vj}^)L8FJ-c%{4 z!*qS73e#gFO@g8964=r(z4euux2E)^mh0mT!(}@d6AEu3mJ_ZkVSY%1>H4;m`0_a# z=?rVCoP~e8L+?RRO303xuQhk<>by4L#Bpsgpzo05q2cngE%x8tuYUq(K+1=2E7nZi zzmfht?YDHoWB!Wg>0n$~nmxZ>T9mlskPMhF_>vhoAnD|ws^O8A6CtW%-N|A*xy;&_ zr#EO7tlS=9NKLm3&$rSndqi zHM&~}E{>T1)K{u75b*t;q>%e+vMwA(ZA>+RshH8&)E346OfxM#-sG7&SWT8r(dWp( zB!*>IEHdACim&ocsm7N_qkf z32!JK3C}AjsTDVsJ&*?*nOVzui0rt&YiX2;a-NAEsOX?@$x2Z;q{ckHPBTC;dGK!K zPF(wQV0u$#bzHzeyIe&{r#N}^>}MF2tB(!Cg9iu^(FX2adL@z^q2IdK2rpK&8PTP} z{*fz*b9yXY|0bLrpbioVLpgeNu68?KOzc^;&!Fe3MUuGM2^Tt13X1CQacd9T?odf~ zp5?RiKkB8PrhAHdIOvn~1O$5p5roe^Umfqzq??L%u{xA1*=_+7g>O}qWbDeEx~Dzs zIQSaeXrqXyzuEk)D%B?SoiF*13Rl8!PFb(9`*iULS$Lj%Zt(t|@K)hn%%w_vY8^3OEM zS%fPUxy6C&oVo^KbR|+|vjmp3s{DOb@y(55`-_$e z4VDtaTFlm07+*Y2G)mAgJybani^2_0QlE_mR-5pPi*Va@knGS4^ z&Sn&a8`$O$9!C3WG_AU?Ttj2!mSkx6MulfM&w88(I%?+_IL?_SWfL8AU47$wPnOkh zw5#5T1Xfn@%F*&h%I%r!(Tv!O$h&2*Ro!Dy22A#gMz+xJ81DhxNhXK718u;Xh@CL+*&vrPc;yhHBaRI|qd2>ls;Dgsm8-Y~vABpc)5hn)$xFPz&gug$!?(3M z-xb6Ah^P+d_uwPTS{9a*MJX?EVy8ugstI5+Y`i5+9|_V1f=;RjqQY>11u%a>QK zd~aIM3S*ELSF(~ILUJQ=4ht8UYLK_cjTNekc#nJED)P%S=|&f1XWF}TJH^MB!`*K; zReIaXZS(*rNAD_Wtp>_gW9zd{bAVZ&K)(3JAs@OLhtz*yTcb7wqg96mbmkmwE=s$r z<2%D+eLoFMjmG0;93JJ)dS4A!B`}qk8-uk z$2Kox{YR9TUQmS#-gp_u2{a0wev%z|FaX$YXuTo_=x>l$}L7(M_0Q=9G_3S>wo4 zHw!{1A&qtkS)AbO7yU@}*rT`(+nIs2$1KO}!O244|Y{FsFkkghZWdm5G}x z&>X=6iVg>s=K^fV?N6>&)%7>d`Mq%CJ`0~&?n35vNJ1^MQkXw zz(_IX+Qe!OR z)I4+V{5;D{E8yMtB5S38)nLrIv{n4ly1V0*-wCJEY{kO}V(agb4OZ^XQP-nB7CH(B zfal-Zf;&u2@*9Bzng$c+rD4~nsZ$O6HH(#*-~sw0h4qh4`42^)rOF-THCQRQ`bPGb zoEy@8Jx%M3raU~VTwmM#t~lNNm5)mJGGj$SxbY|hv78uPDgN>&f|b(`+a>mJTX>+N zkRl)XLGpun!d-*TE3h1qb23?4otIdkvF=V_cBhfISza*xB6_~>X-Uq_V%{mscoX7` z@Pg%amoD$O6lS}l!G8PHkK_@gvirrN1IpNCH+f5;`_{4pSRjw~i9vCCnG=)s2uB}J za9O2If4(Ppv%#(daUbuME;Zm$enFzVL*BXlx$TD#qJ1kPx}X{JAsd(%S80-P?phGpm>201Rz;WUCnRzVAF*O_abGn^VjDf< zuYoFqG%drRgEFN4rF+P#`@%>%|3%) zYg?GkfDbE5%7XmTK9lo4w7lhos3%M<=J}2nG1Yw5EvVj%qkm+zw z0%h2G%adZ0VjYPk8fMG6kuqbPdv-C65-R5F-YsJ}_Qpmo?j+s0TQ|`SE>nN%e4>?= zEF!08#*8ZHS8Fym>d;*V}nqt zE2X2ki|N-Q>atqKJ|P-p*0ScPAt^w6cYQuAB0;+{Pmk<@vLls*_&|KCRLE;O@41e1 zL8`c^GlvaVzLugoMCnNh_4@+Dx`HF;AD~7d)gcK;0bk5&JR%u#LOs z28;TZXFd(QEgtx`5Vm*f*?602PLTjfpXA_4awC^d=pqHCai%EnzqA0l z>~~{5mk6n&C{wHP+D2>xI9mP)j`N zodKO}O!cQ#R+s$=0MJOCit~)p%sUyz@|PxObkTYP1Z?nARqeAE23}J3hQTvw+VZw$Fx_LMAt$Lv3H>JFzFS(?gAXu?4WWq769&VZ za-KJ}wpUJQ&nWpH`_+CWbSRz7ZPIH+Z$##XjJ#+PtFGuZ8v(GGu@b$hn) zZr$!BA9nsB{2-!T+kI3DE~-jTyx;1G-4}3D-2m&~supm<5D>k*OL0HJawqSx_<{{& z5neu1(Fj^CpEbJ#2yZ9j1q1?M&XlhqPaCx>O_#F42EWU_G-ymT2$(7JWa&V(+}wQx zRSAX5#Czt^6haV7hxJEt^{KdlGaewO-vxL`da!e7y4anZlo9d)Hktist~?zZAo$4i zNi>k&t>k45%mHOz#}tkHS~_DYda4=dj;TJ;h+7p8T73i6lM`Ij^IaXgm7gz4(1;tmk5xaTG}^EmTVUQW?%}^0>?L`_^edCe|4Nx&P2s z1Qf_E9`mjmt*qS~)$+5M(oQc;ygOJITB;a%=9Juiz^SQw!TQx=8U5JBQ}7^ z$Jy=`#JT!f-`$L}A2YVVV)izcTdIen~^!ZxGs&Uwuog)~_jNF}m*>9#29Bu|i zul*QwQGsk_Sx#h3#nceT0c%Gmlk-Ef?Hw;3c(1eIeQ{E!lc{o$cZ2;2quJg|vQ)@y z^<-ihJiBCL_6NOjy`(kV$xd#m3`yAL_B^-R+E>u8_^I65H?ywm6LFrxy}bjENhNdr z*ZjW^(wmGi(Dvtnx0YlWHU?-L851MIO>{=?+v2}}_3%rI39CoZN;eeQj@dDjy24nq zPIZXB`H}D&Af3fU)BH$2tF&$a#*?yy`rJB7f#1?_mO4jgzGY`k|MJF9JI+`bSeuwU&?>6!?)K5+kFE8U3=is z7i&-4gcl{9@h7ODbrFvxHGiB#`Hz=tYR_NY_@umpIU&c2QE$&Tq#CWe6`>7n`{Hie3trl1APo8u|ymCzl3&`BXX4YwE$sS95 z(ZF>RY3i1v`t^y_4R-bREKcvTSqt;IHF-dtt=lNorgzZAh35}U|L`QWxXX>}BU0-z zy9aV=K9YPU|9Ey1H`~aV)xhg7+Rd40#hj54ArhmLkn9tnFA4(^E=U{C)x2s^DzgM|?9$MtktuIYUIO7Z{PMKjo zW)LN9(8_-9ZPwd8C*OPusXDb*ei?btt1$Qj&hs@@-K*OZEtf7*VmBJx1|(2h*(EA>HLMq=?misvW2jL*5P&;U@co)+P?Y;uu3?0Cs{l3upKTKVk;-1pLe zpHuYki{nvA8@I+#s`WF&LCU`>lQGSBh4Jt#T`(8?N~uqor6i=(r?Uh z*22QZTd}+15%ULmEv>}HjAn_OEi{OdMVJ#TqI&c#ng-p7cAywHkrkP=%1dS&zX;$uB!%qPUriQe+ORy-9S0x)ma+Z};VF8rKfUILo)7}@icQRxhKw=QRE8_syfi@+_E+}`g@gY1fJVuZ^vdjYZXs2Yg1B*e z>R98Fa@;e4%U>^1dbI??dKk{+4&j$4tJJcnS2PndZxkTd$lya!IPvmMb zm}VydYB1r~OxBMkC$TbN|RbKLwoSMB2X_sOyO2cFs)FAIsi4=`w z>Lrp{0QT84@t@^-80fy5r*F<5`+iltn~c8`G)G)$^%ZwXa}mbAZP@Rv_)Y1+QJsRd zYn`V8!6J&^_68~r4@le%lMSj01(<{H0ToVT8%!l`gSt6WUbtX zoLKe98%C}b35H({DqfLT^2cU0!RvLWRp**?f7ka|+y*OUzj44xG)-p}W|coU<*O|& zcSJ+7D?@@yB7D|!sEpsfRS?{LKs@)?Fz{cj&M#*3J<502 z#{cC)evw`_i<)O@fOXCOlhut20(Gs9QD)$;PsCq+oIG%h%9Djhf&N@w@_k@=pcs4G|7gof0Xz1d$)AIud3np$$-|3?=re&yJQjc zM`}tDl-HM{;z>RJAH`Ej5)g&Y7@6n*7{lT;Kpnl_0N6epr{(NYt@jHAv{Xw~8Wui& zwR-YE+jGx#x&P$SuLS$$$Lhpr6SOu7chiq-o$VWV(}jgUdDMVY#zwB%)SE69hfSV! zVG6+AEj(UQ?H>5`k*$hRu8D48KV+`H+)oxn4D*4KuvEQ)c zojS=5C8VzkPbr&taV*}h{l_M6n~SDN;OwM)sI`#4{U~ydZ&kjYY5Wl8nKP9a=EK?K zjpm^i0W3tdv05+yVv&&cta z=oqYhzbM)0eie=e=*ZV;^P3phG@$uA+UOtjZ@2BQZ&+z-8v%574GhEOigZ;6WOz?%t#><`;L6oi2+Iytpv&*vZBE4&J!;Cnp=& zV-2Bz`}%&?p1!19zx4^Zb+-vma>nK&3zc!mI~;(Q<=+wrNT{mTeWzTG<9>g>E2lC+ zo|zZEzI<*W1~>PU!C74g$Ocq-0X)cGulZkZ@m}>viKE?sje}^s-ogjt!cF%lu?Li_ zFPqGKv9F@TqVb}JqIs|8_SwehA%^4!@WaAx1Pe*J)ox$qB7EB}MAORX+_g<>qm0qO zikg|q;cH}o#Qvl9csDlSSQzN=q4p;Me z!S=74y!1Pccl*v+>eqb>-u?-YReqx|w%p^3F2|UwG_C&Ve{E9$&CyulyB;gy;mnRb z?o2pOq3nqZj;Zb}EW!k+*_PB!|8Z{QVfBrT2eI}ei=^WHe|N` z>k*Xbslk&SQ%8Bhvo%xKBiY&7RAGD=q379d*++tDUF{yQf8*VC+vaj^CoKf zkF)FZ;6ufveUYhXb=PPfwxF&%RQno#Z#aAloCe&%_(PIO_G@%e3^WlOz9+I z`Mj4Lgc(bTT5DzG$PP+TFM_4x-zxyGSYa1RGrn+^$ejz3&uWn!`?NafA??QXFpq{k z5ER3>`x~q}!3LNrkRr+G5>o<+07(SGChf;92ko87u;%?5h5DBiIQh)x7CrzBdp4S^ zJ{04(`hjkguo(fQMHAGS+tS`^`a0n%uRy@Fw1-nn38{pr_>J*yjf7 zTT>&*T7yYu;XjX()0^-HD>=k8Ye;!b%tKVZEa_~-2l_4-ZtF)8b+L&pO@i0ZfIe~Q zCNZ-FgW1bzTnEhbCV9|xiLTX6nv4R#FAe|##B}&-$_AdYuo+CmK7Q+}2E-T&B*jDx zu!sE)AHjoqWg2l%R4D$DR8(1~erud^Ua&i7lUH9VqLJ>t8zOdl@~R#3Qx$se;z;<$ zTG#P6g1)ArEgfcZsJj!EAEF9J6SyItE@+*k<-dsbUT=|E=_2|FQq)GfM^_%Hk+G^N{L+2)p0tm40H%PGFXD))TbMk7b|lA1 zF26KLSHAoHJLUcNzbLlF>(zOyY z@A(-Uln)E3Z{-vA=QG^uA5B;Xg4F)YJH;aW9bs(`Exy? zgG`MFTtSx8^HiOSVUy5{<>zuhXl}tPv{aVoy>*eHJl{_wG#y|2_d>Pbx|nuqs|?m(AyRCtyTYQv0I&`05>}ZNJ0~i@ zwR&#&P`=Ng&PUWr^!kyQql6vQ<=&wEsqKP0F+yPOKKCtTA$`F!f--`_;O`t&*2Q^4 zwN~N$=s`XcIijMj6rUJ=i#!HB0?S_E$zM8beu;j@yelIq9J3Q6@;T7AA^ z<7y~7Lg}(ylhFPa=K}wjp-E7hPTXQ7-9e`fVdTnk(21RVQzFim8?bg;U zvdHYS*7x^7qxQpyYEuHFJqKWQRQQ{Zlft)`y$y!$&Fx-_Ijoyc@D^bm1 zwWOLy9f`tCPhNOu^dUkPzf%)*Pd&Znlro5Ew}Y}Ef?G&BfZ%y*8Pu1&*S4F+B@qnV zw|Uj!6Ps2KK7A+XKRY8oV_)6p%x0h9M{NdYs4R5&&>MF0haD;oLci|~Ul_tH)nX0R~>oI9K>@elt^e1lp|9f~k za6c2AJLW$u3BVZnnq}+yPc4AQ_ZNc92@JShU`S8 zMYoA=0e;B`FoCvpF#=-J&;Siy?nK=)(Vgb=68pb7pw_-cS9j<~#R3zHwI4}9a`;wI zm-6+@6)JC_DqVf>z3Eo|N-^1-;{5PdLHEb3T&bG~EqR2!DnS5LD{m@aFV&J1N-%X< z86sb+qM}z6es75hkCEa(F)dDiHO0fo3!cPCV_dii4@RQ|y%0mOF=Zx@HHkaH#bS7o z@W$1zs;Z6HJzI}Qi3W7;SYL))BgY(p#sVwJ7YX}*(ZED@V|{L;QSfb|1rzj=tFu)^ zVV{4CL84$0b7!<_B~p|6lP84kUP5=dtHbY@=e56=4Qem(_vXbiA{HU)qirfkJ0YZ+>Qg1oVbUkcqhH+%j-oUhzI*zrz-3a~qx(SDl31ST z!x8@u*{)2)#9_TYE%}ICN6ZBNgBPC$#U~M9QMIP4a)XaiM5Iz zajmEkJtB6YEQJbJn$0qbG5pqMMwTtmj4|gIn-<5r&NRn=yJz0&mW)(xZN_@XoP{wG z@n~X4CR6ie<+Q}PEVI}xZOD>c$`}jBVI+#Dq%A*vSWP~^71OjT|HfTaT>!W*BHsbx zhvkh6k&MlO1j-a7jB-KLn0uojV=cXnr_XRhWC5L_c&7jzJ+)Jwh}sr70(EKwjv$S= z#^6aM3EQKT@=r2sWWIH|L38_qxJ8*4nL-qgM*7(NX#>FDOt+B{M%?E~voCVJ^I;J- zmpJ@4QQTzJ&0H_*`$JddgYsL-!gi{^6Ewse#@vu$ACum?C+a{{D&RaFY%yXvAhGS% zlrC!r-&X%fr@P>Jp%b4|=ar9qkZXx`?;g~T6FutyJ2E78KX`P%p^*F*{4Vvb&1t8w z@4f^>6D6pGdQMU+y7kFsRZ$4oxP;;N7-)JcUeU(vK~}U0DCDNKtfOYm#c-npU%VycxAT=rRW0
zI93f%j2B)&!(>1TlO2XbDNnqKGk>#j(~l>s|`*CAn4{Jxu|-6D)7LV-US=F!A@t&j7}oj-Siosq&A!FbM}wr#`6 z39rh`9_#xLpxJQY&bnnQQ&&0xqf>7oqg~-|aMN30Dn?S^% z{kc2N@~01v%d7c$^Ye?Z5%|)UP564>3xZ8u#}BfE#9A?zUWzQ4ny4x4A@7|Tw9r#WLp;oRXiZXp75FL`w7V6E$2`PkY9khHH5;AatgO z2&-K+e7Wo9A-OOn{^pXx)&n3-u`gs$glgHOI}}m2-7Nzt^lyQ;f!jhM!&1RO9RD?qE4?pe6aXY>i%wbDOZvR>5=a#bEyyk)-;^$#Zt!T}Bw6fYPZig*77SYohD^-c#Mx2@?r~L0Ou$ z^kk;Azp>!m`-|?0#eb?A{VQhm&#C2qbynB;t)HuTIx)xJny1ZzLh;*nr6%9d=Zm17 zc-<6YwzqXl{uYZ+j^ex!pVEY{ZBi+kQwY2Zp1f$8u?QY3tBuSUS(;X&iO*2fV_jQR zWYf6(TZjt6WrZC<00;RT_)DGb%m~25?aI0U0{9{raZE7%jDd+Daxq>-lX*>0gH9=) zMbDD-=7*<_YKNWc-zBX{!yXWOWK+4 zHNM?AdjAd3F%2#Hx7P)E*Y#Nhpw@Q?j|l*A%;aZeF$= z_dRx>O^!o3K#jJw?#|!WDst}5n9nhSO4>@}@w9~)F#T+u7}fKwgW}j(Bp`ArQMwaQmTD$*`-PFgku#b0S%s2oO%0bA>&)dJ7uH$^UC#yH35}O>HYW!yr3L& zyj2BqPl}E2IOISCyIwYPs6%!8sF>^|i1Y4<3X8re^WB*(2Na3t5{+e^3SW-$7XK(hLq&TwMq)m~-E-CR5SNH`xji1T6T}S(nngGoXc^wAmr;^`D_hKF~^P+nu zQw$o5Bw7yz_B_?ysh+GIG^!VW5(2UxrsrWguD@0C)bmb^8!WnNBr{TM5b#w^i#B<9 zrr0pRyJtv4{LBX|>m|;azn2g)pS7&F%(MJpId0i1vG@p)L)?dEDdeREe9b%=Pkmzp$rsMwK?7IfBh{i@ z5YhfhIexgq?Y%nH<9|S{Gn36t_0bZxHC5!+9gs>^nufLg{QG{`#doSgMu%;b8G6_!c)!EsU6hh&GZ%t_61iiRtya>Y9 z1T%VbsD?oAbhanQ3=`Y+dXHPWj(Q4E1U-7Jtrfm{wP-uM=NVpkF=tc)Ht^KQwe8S3 zCqN?-;PW<8-1PA+ZoBb*K$!P~JMLiO{$`m|xf$ur^7FQwZQTG&*?0=S<28DiS3nc0 z7IRkS>a^YNvQdZfBQ_a(ft!e#w;2KCwgJf328yXsmSdXZ?nenme1H+8`IWUs?|yo- z3j;t27VxvT0#;_KRPzsO0)BVjnrBZ9F-TeF(am3e&L9J1>4a~t{BryPK|qn@-Xwd@ z)bH%KvwPk2+z+9}V&>WK^oa}e%^q9#D#+&xX&%6pxi_ZVkNXy>8l!6>ZXy+5^NIA9 z>|9qiR5%l($aOd8KD`*Ak-s}Y4MG#O9`7!hAU28#6sPF0b*DICfCNs4tRolZI9b+( zbc>1BIYp){pAcSjNPa)pMl8^7?jOrdK4r=0+RZtQa&PXR`$2R;I5{4PeeZ6+z#6c2YitWqO!y_3A|frOdg=SF_w$ zjvqCkR3dPjBx!{Ehc4@1P44{jYmy1ys(dBWcz*nFVi^oes561IB$`0P_VC>!H1Idn zrmX_7l(-3>b1}$XwWXxEO&<~lC`RV03knOW0Kkty;>{}A&X72DCz}d?UlYC(i9X8J z7=4~!3MC>lS`;(hoFzMp|CD@q^~!?bmzEtjSHZil;O1kcOeQ4dfX|Q;RHVcJvyQNl zo>*X<)&rK>O{j;M+&wY9zNVMr@_Z37LgBi z+nL(3P~h8%hKVVN`hV*R5pIn*`ZD@#ig1&ZzpqYd=#9h;)s&8f_Mzc{i@7JKB@P|1>6_{*t*;NA=_jX?rrnWBV1$>2g@Sm^4 z1|+}tg&sPGOxCZ?+$OWEhfa`%6>Gd5yT{lXG%1(3F!3P{HXsD$>UuaY6r^Mga{^x$ zqX|cd<<9s~cpS8S#BDf5?r)CPjdR7d`&x5)KIumB2`yR7TQ*z%C{^Yx9pjfD`XK2q zV7SdN;yHAp%ehNXp3(+IB?19s$1vvwpl+dGCqnB$tjR6t;$&ddGWq0ro+<<=%zoS#V z*E9ovE#>wv4UlAk$cE!4$#G4PP(W}jRaIW~fa*kvs6ch4TxxU0rxYmXMfe)CAn^h* z4~j%EuX)sqZR=TDPs|Zq5AansiKm-Iqqze-1 z)>g|&=@7LX0zo9yKnNtjnOX1I*Sp{S zK6|^Kuje|K`Jk8?&Ai7w=D5fBjsM8VQVc4h8@Yv^oPx7Th<)8Iy5qsanOr%SR%)q3G4J6)_J@J{b#72`~?7L8+>y;HcD*M4(E%;A7P5P zFg5RIEvNm!%&x2V!|iGG%F!r#4%GlaZhRAqoy@g0P4TT#XAiBS2$gtlr$sefu$;ariB1{$M**`clbcU9H# z8fZ4WQ}XN5lCX3O1qF1zF!d6hoSi%)tJ|tzj#F`B2P^ z|M2-=)e1bIe;=@+x8DC7bOLaztVDlacSwBXIPj0Hm3x2TcEG}i-uk5Q&q;{C|1Dr2 z34|M@{;T!+S6_-7-ZX~-n*(n5X70ax!@vA6V_pDp{r|t#|GGE-pIxnq)_MQ?r@2M~ z$npTPbq@?Wfn!Atg0^l45P;w9z?)yE8~Qzeohln@nuy)!AUrQ`?x-C2O#rAlMLQ4j zH4P25Zmv12ReD2Yi>UNPGsO)OM`0$rzvh454NNYxj=XanP#`uA<+b$yyy{{xAd_A4 zx5H+If3=Y=F=yh^{Qz>Si|mRKu#oF)mv(PJJxn`P4D@FR^A@YlQBsY@ocy{*9Wrk| zBv(l3Xy+8{rrBhbM?liyEH#U9{PJ4rof9`NQtaw%tUW_!Da+ zz#BQ}0G#445dg1SIrt2UT*>(R(P+M$1l5W=l9wd(*Y(BeOn0|NUQvHkemafJ8%!9zXn|bZVvX~_J$Ok@(EMn z(JYjM$y>hXKC*2dy%n7%@g==l=0b+~0H(7x5<)BysB~Sw_}4;?Z>cjkv`X#@YTSxI zt5*I79BqyPno3MWt7U;SUuS$XJK&9?>m8v=jGlkam;vuZfac8r;q~N4 zb~i2KdbI1|=gf&8Tz+WG)LPpj>50t(2F2pa2b2vdhV^en=TibZbApMVLP}O9k1Xww z>&3OZ-yfT&woL5$s2=-8`RqB6#mIkX0i3kIXtOW2WS{dU->0U#(J^j~Zqfpuk8~a8 zd#hS_d=&lsupX;p@i{aoHLW)V`1Sey!~PpJuv#AyyJ@{{HW-{2zPPoX@P)=T50=Tv zLRl=zCW}h8gRkX|bG(^=oCu7je*~M9qcAlEe4AhtFT<=HcF!>6Mp$NDnFFC%6O%$JGptg<}XkM_wT`_;E`7K(jIauD)_NPWtr zh+OY+sSezzkJi#;jmaBi{o;*}15SNTzPXC-)tmiUtQjAKvTfI?+0M?A8W4Wjtev)VyBHmr zc0N|@&R9B8_JHiEeMynoy?d_f%6iEnc9Zj55(Hb>6%fkRfjVRfg;HNz2<~QJM@`hpC$ALbugJHM7{~ zN;(yz5vxYh%f$S!v4~rnP{!3Btw?q%_l=-Cca90hPQ8~ic0^sk*Vp~kbZ$L(AA zK|(G``;>(yhJZ{n7;*n|l9VBNkCs|BlAbgk=T7Ofx6g4@e3%;DHS(a#Sw>PPpnOvje#012O4~tyYM(Tfc@VY|>*Od1OaGez*Mz=qiHD$zZN; z_c)oq2O2T;hg6}2NsQXll*o=A7516_xuR~PNEDcvi||l6`a!=zV0|i8H%pY>?Rk7V zISCW399IV%G@$L0_PRbbTd$k8v=*F_7z{oYad3A%}rJ3@z_{$dd^V`_DV;JCjeiqU73?i0U8O=jIt zBg)m?-ZH!3F!{s$H?9P5G=YX(Zvscbq5x0nWV)Nrhwr}4-FQU1pADi_FqY3Ij4ygI zU*7^=1d37zrfX}$#RaV|!yqtFh?tc67`xwTT(cbeXa_*Ig_n~GYw&S<(DMWyYYm|h z-hU!eNV%M~Rfi*l2 zC!OnpMWw7(Ro4~wK`bT58Bbp&o*o=PJswe8o#+X@bWz}7vCbIn)&~PDOeFNi9`Qw$ z!EfW6n>TXrDC3=TcdW%h$fU1di|qLM*GW3-y-KArP%5B8Gmr^2$&Nn|X9i0-paAn| zh}5q3Ofq?CZ87NnGHf!g%pkS$r9rBD70%wjBKb>OI8RP{hoMQ@yT&QKpzJIFn{pwi zgTBZR`SYUex|G9k6$xgpeo}NyC`$6Fj2y~g>ainR(UeJ{;@4L& z>NE3UV+D=up)flwaXBrRQ7Pr|(+5+@K1sxEox+;1@yM{ymJQ7O0GQ~dX~-0HFVP&M zO(kx8JK>qJ4L`aIw#PHwV7j!iH0d}kxa|XSFwpOXn)8RUe_{?Gt~sE+qci|>klW2S z&qaPEz9;RL7?aGE#N=PJlz4RqqV?J%|4lljTKqDSLsb-++#mB8R?X9Rp(Rh<5}e7y za#0)0In>DhsUD8Kyco$B9Lr-9Na42>w27vw+`qhf^s|!smw@a*YejRX^jD9g(dM#mrNY<$~Ba)Z(!{@ zmIn%@QlshppQt^@wfnYOR#d@# zM}|^yosN>U2~=hHWY*sf{DFG!)8?9gDX21c>UhVj z3WDZi%?F<_l@nB2y7d@yp8}<%mTc&cAFO%7x^EA>wP4ZJ;%;R0EW_|bX5LCCd(}r{|ej|oa z&fcLZyQ-Y5#IQQFgT7&OU`h@&=c;UR#fW?6CaTpduG;0&GpLqgRpB^YxQ7?7{X$aw z*2U(>IC(K0OuN0A+2d8!2@n*kom6Ia^vyD&yr8l9C_DL8;**5a9j4A_wc1~lAj&&o z7{?8nE9qtx1yPhE%DT(FlU|UH9^+Dak^qI ze)?@6*LccBcqK9=D-9M0Bv^vCC2AzGHwWG{&--`@SW0XN%1KmmyVaH%jzR;Q>K}K2 zf|jhBr!}YW1a!pm5vB1$jJxMp7-vp1YQZc-^&d`gKAE zl6B_QBMUfAK?=PySW#^sm;!An5Pc*=Xov9ZvU#Yydr?|$ck6MWIqcU=pu}?ie#>iK z+P-23oo4vKtQL*0j|z8G(#?W@=^qNI%q$sHkySz*E9q5b3c2@U z?VBCq1l8y%s-YIhE7~H+f8cEBVqSCg=G#+xN@#YRM4UXAgSAC$=!EqpgV=4OtlP2W zqz9ker;CiKZ8|WSx+jS8M3$EMiq%LMB^^i5WWHlvVvH3KeU=B`Sjw*>(k|QKshTr{ zrxOXy?P2t-%;=3wrCyN8dcMDxpNST8C98?nTVRkmA_k?%`HHG_n=l^w*G|3OFtehA zs2M%X*$&Q>YKSMbBEgYSM^ceCtE78j*F@EEA|V^7>n?)N&-~_jcDwGramD@V2Fq2#GVe@R;oJPykKIWM0$~DH$_|0x_<#_Ni;-)MmenSL zft)&?%_pLOa1MPzGUof zCdjLn6>tFiAUq&yzr^N7+|zBhfz{+!0>t(=T~L9E=jmLowtPY^hIenA#UOzY)ygxz z%m8#fO{!i#KtQ!(v-deN?>Kqcg};gi3)3lzIB|E>Nr5r}5Wz|X6$q7g^n@RR3z+xX zJpt7N2iG5GK5pdsagh$E_-ZEkfXQFPdf&!HF4LkjNSy)vpx!t^h zbfV91E2S!ND&EyoGgl9yP(M>?klOiOW1M^X5_kbp=fa%|K0v=3C8v zO@C89OPnd}K>QpEXrDmAB^Z(;eMRM^Sqp^csDl!8$?1U<)8)Y=GQT78gwQV`D((7x z$48ETt>+Wtz(Q;2C80SwsqdU7ymIgBN6A@nQ9?`RK8*tWthc1aIC+XZFq}S4Snc6N zTPL@lRbUs%4_}hAmjCAUoq#A{Ae<|SotCa0)!w=;#Myk41m)caI@#_YDtewfb_*M4 z7^iCd>+w5BU8N)C2^rAhyT#t~CJ)(Wy*T4l7S$S!2P#O6w&;jXC{EQJ6oLs-y4Zq_>YRZ! zL94vS>ES(UDa+#gk*=-+y-QV4<=&R-(YQ`m`s&kYW*wh7;hTq#RP5vMw zl2-fUw};B}6Ryfj&es{PE1&Q{^5c24A+JQG|myzFg(auck6Ey)e=^!p^nB{x%vNhZTd z9vzm^J9aVSIUP$ae=fIemcF}0A42USbsv>qnviC*Exz5p_-+YmTR#8nY1x>a=t*59u(#$ zIg^6H%Rc72A~^Y1MMA3i2U=*B){jBk1sX-HT+hM3$( ztiA0v*Og1i6}_-sd%;kIz<+^uS%cw7wM=a2fsZ zM3C%@yd`;xFCr8}gxFNzIX(E?{>JZ-b7FOqR?n%~>Mw(nML)l@w>BZ%p+WlL20n`~ ze&>dtHI|BBtQt!PWaGLd738t zjTfTRd}tNFUe`)CUc~uW5tMV6YmP>aN`r?W@q+d)nT^tlg}g9?KlciHz;!XqoB=(#TFB2KSHKIE8?(m@MRQ$ycLytAvJTulZrCA2(6rza;8j_E); zH_e7kp3JxR7fUkMNZ8@kBTbxff`Jt=7<3UPT(lAaj8#^MBy6_|xzri>#T_Ttb8y&{ z8gRR7%KU9*4R#Kx|L#TRvu!2{&+$sklPnSlp*&_Y|J;}^Hp@3mDW;k$rut;!fuWb{ zKHRSKa+__?nkZ{2|4D}v*r(mS@G#Z!F5HX3ogGcad3Abs&Ts;igP##)9!?WKt$#np ztb+dLVG2BL<)whUj{M8Bf29FwN-LW)F>bgUy6(GgdS;Vio>(f4{wZ1_Q>u6+ab4q; z_5^%L_!9a<5$?nv(_swM}=yP$;G@LcB5DR{vCy@s+7#jUT z%9K2_^>I1uih$MG*1bsatOlS&-{y}8omxsg&QXTJ6vrLz4q|?kkIu~IZQ3?WL}IY5 z%6JwqowlQ+{0g{N@HpvbYp%%VLF(8wl&5fE;!@5JUwmIqME+UJ{T2;}L*pCbfD7dBk3<|Nse@{p$gln z7PoELjPOMMqXSMXbAaT*^+%>8+0e65W2ox{!iOG(%$+#ZU}fCXVOn-I-H#zF?g@!A zr}S*o@9G{#fTZiJv#bvPL3RNR{x|` zU}o+Nd0^Yo8@K6e8P|~tJ(Z%0q2jwo0SRELT0O!I@lq?dZ2X^Wyo=n8XYy&S1lV{( zyb|9e-wE!~SG%lkYzk85IfR86ThP8cx*y_uz_W)hu#n&8e!DMUBo;ZPc2i=oD(}tf zM*)!dP~UKa#2^#$5p1M7(pPAS_W(_`_KV~0=wp{<4h^wH+=IW_B9ESL$v-;*j7Fa{ zzWX@*5oxMh4I^*3viwRel$4ym4!?g7g%Q~tTgMx@k>gC5+Q``pAI})a6p_SEO#P*( z`(V7m6eNqvkO5hQrzhE;FZR5N3vt&P9U4hKoFE!E;fb=`W@f9$T}Gd1MT}u_L(ovr z#jG|#brEyc1l+42H%@`05B{1^)xLIL->|%W8z#N;(qO$@I{k!_VK&+?MGsKt-jw@L zwn!jD?Uo(o=M6@dRi~8FluqDk1h21lIx5zA>E!en_#v9VGFl@en@55>Lo8CJ6@yY5 zQdm9mo1gdbXk4PNGWK_coGHzg+vC;Ft$)cccD`!bvAReA$10kjUx)`46Tw=nOV8tC zkQ^`S4*Tm)ZX<(f+7_@uoSDz@d18hH(%YRM!bbr4P^%Pbwui#@!mToRjrU^a@7-wj zd0;j541%lbfz@;uR9Dx`v(uWC5W=2OzcVuvfD&wQbQ*#x^6aL_*#yQj-=UiG(CazK z4slei#6aI`6o#kCM8FV%Eu%!n?S>QvK1)>o6a+d-mzIcMCw-QR-P<0`7hS!S?FUcE zB7C5+NsaEz+ipxzK2e7wp#y7VEn!4pl3Ei=Gmv?{%VNnASisjHmn+BxIRO)Ka9T}g>u9{i-U~eg2c!HU&c0A&lv6G81=;fWw%?+i( z6vERc?d4N%jk_I#n)QRwB0X-)qAr=XaIq^dYVRW2+SW}@dyi!8Q1vE-nl{dx!JU~7Y*NVl4Pbb%F$%#OR9ippkd7 z5Ypx1qaT;VZP4E}&TDRJI+z+*UD;-F{*dFl2c#ClY=WqH_X=N?PFo*hsNc}+)hri} z+i%gqsegB;HB4xdVDdpgu9^{0&xR}iarR2S2(K_Fd~0ne(T}j#dM~diq`xImar4s~ zWJw$IJJLwvH>m_H(PEPdbGNB*{)#HI{XjNcUHDRU1vZLWK1lx%U-%kT>6? zz<22q>)P|C0%+^zz69dmvE%=|)tbrw*=zdoN_>+1mSn`qOQBR+ns}zRnm$dW)VAs` z9{`EG-lYU3Xrfnz=Fy`-MrU{29-3Xbq*B?fRpk0lb_HrgRk8|Nh^3yADdf}j!XfbQYLs7uyzal8waiM5|U z*xyD;oiG>+ixUN1$2lu{f$k zxF9YdS`8x3BU+I7WRcgr!|e&LZ2C9hr(^eF*19!LWe2<%V!d4~JpCcX{ZEmjh#JQR zF&R+8L>F=njXmAXzdV(@*e2|Q8|XA%-=Cf@hM?Oa8kkx(+7|2q*4`4r$t_i z4V!tPd`ZoQUh^SyxqDow%Rmuku7d&HKP5ff*R!{*M>dz_?dR*<1U$8VTWgZ!)|xb* zmSTXcPNeb^N`vL*XxR8^^jh40lV#hl9Il#)`W`H8^ssPzL+TlI)$dITB5wIo3d>BBgHYnjyp+B3FV0oWm)Z?e6E1itjqL1VDcDa{}6Qyh0hDLx~x3NYY=w~6e0Td z56~IWw8`UYf;vkzd!th`OwC=|PBXl`PsgT=c4(w9TQ!hc_?6*vx`a#Gu_v&6p52PQ z-w0m}suIiW(iIMI%J2UWoqQItMm6$G=Jd~A@3@uL+c@fr=*(zjmPQ2}vXs>of6Stq zZ{sV=P>v~=#EEl8rI`Hau?-am2sHRth7THv8x7%+u9J^Bus3IBudY-RmOAz7f;nhd zW=;}1J}k$K;JwF;Ck{S$=Y_9yMlDb9H@!3niRkavt%gLo4mSOxFvWVJ^86rq>N+=( zuO1^QsE#rvTk5ib0(McB|687lLe@u(v+T52%CX|{4=!1pisf&Q3OROPd_T)A$PJfk z92;&ObVIQ<1!b8M;tpoi=T*P9Dfc?W7iP0_q3QmtsFU^n6Q}%FL`+GgXZoRP^W_#w z$YcAHJet>U2(4PH#+jIlYnOMPQ;pMJ?v|xdGT%564SD$IY|lROgIA}qDUXuZ1Z6mg z^eR?5j#X}SV1V2DfcS`{4|KZ9fZke{d#2^N!k(DJZ)YM0shCAU;ia_!D_G(2A13&b z7H~Muc|VB&#p;0UFK)HA$1BzAkCgWd8o>!DBEqL8JZIPypMEDc%T8gob1wsSHwvfX zAclSC!Rr{ik1=tlQD1k(xpyk24%hlDf3qjjQo5JymBNMyoa}F)WBPM9j3V1pwm5k- zMN@K?(&oDFfuiOLa`>j+Y2 zQ{K!JT#gBu*o1<^<{Zj8oQb4=!Y>R7y=lNcn3jmJs2dxfM(oqt5mkTxE zcJvbuY<2T~o{3g_W^HVRc=Z$RiXMi%_yV=af7I-g?%Ez|bB`}P{$c!sc;9i&oH=Y0 z4)Q6Sj_d3ob{XR=qxr(I(Oj&>>WF?-K;=4TD%V6%SAtIVHRlnU^Tqp=A#p~bKq(h< zi1vk;2Z!joj|*rQzuQRJ2ik=M3Do*THH`SA>kjt+33gZze;~U54olQD;G7ltHr&A7dT&xp-74|6a3MM+wOb1E)E+Rz$bzmjtZMzdj z_X~9USI<%Q*AE+rrreJH;x^lmVISJ<;Z6*~EOK!)gPmtAWV(@>j7OaH!k7e8GQzVl zTiyIrf`)4)|7wq0pK(aGy`t|yyE`gr2sdwxZVrC)F2$q&dQR=cDSAVA0aO}!lzHPe zg8kdX>+|Mp?juysRO^QAx8?-l))vktD|E0;)q$^f;xKVh@WFu85BoP-u#bU|jWG6gl zrH88a90^qMg+aYdTfk2UGX&CeFY&km(tc_JalXXf+gMrBJJNk zfo}2Cv!N}-Tj{=$+%t6EULH`X`V-CAdI2FRIb1W~Ds&?Oq|BR(rqKZ&K@s@2ZXxdz zFi(~%jUeO3H&eHGcca+|v@t9jitXu`{X|za^Zd{O8J~AlVTtNtUl>~?H2)&|U=A(j zn@!E+*&{5EmIb9qE^%w9Ahp#j6EcL|Jgpz zG)xoRfi=w;9TZ286?ZU8qh2nyusyPbK{u%4FV z+X3z8k|&C>d8iQ1^X~x8{@_mZ$zR3$y0j0WPjucj=}bG63(y$8R!69Rp08lSJ2SiP6{?YRPXYZ^9)WCh7o(2Ctx$s|(M9s3?_1uff zFZ}n{QxLfF>>rqadqF)80OdO8Jp{;=e|uGuxmO zd)5B?pW6UeYKhIx{FC?oAMbJ61Gv<|T{!#yyi)(@_sz<{l~PHoA^)}^6vs5j`(_o{ zKW~10bg8Ck%kSkJpz?NbF$}f@@VQ3y04MGZ1TfpZ?Av`D8c;Y&@0@P*&YR=v^M#MX zIg~uy`T!JssHS-S$6dX26H3imU)!7Oe&pGld3TVc3%m*{PFghJS%wZr@y_;YDiQ_3_!$ zhJMoPy<31hQK~_4(ZMhPYAtwZ4xJhO)wtn0iMu{NI-(c0(!p-`9wkJN206LKfwM!3djA?-y0%?!(Y}e5k5OLoVe{(AU=3waD{`@-ky^ZF87p$hL(L>L3m3ykxcxS$MrS< z=~M;Wk{h)X7&H&=!>_z3OYxylNV8@Un2iS5#Esjo9Rce(8%D{ARjlp+Vi0DiZrg#0 zyClq&89r#@6_sfQ8?Z2p=#`4d$n)-eA3fwSn%Nyz_T!T?lsa#shM}_`ES5b^xika- zH~;;%;rXDN#w9gp}l;l4j)hXJ)R3UlX{- z`DpkQT1vuY>jLT1iejcVK7rLXdhKizcR0kZ+a^Ke+nIf(b+jCu&NjPgCz8np(dpumztEbc zG8=yB-9>F*=rh#LbP>}qSbEeZpa2^}=OaoIW$-YX^TalIhXGEX>2_c|Opuu|nP?r= zBd`Fm*kH3{v&faa^1%wJ-2$Z@mAgZfd4g89+^qK%T&Hvk6XW}MEt9Kp@rs;f^Vq$4 zW{Fth%#?GS3GODO&J5e9w48DAOsT?o? zs^JrsckRB=RNzIOvK5)bq3nE4fS>>RKpl& z0mbVR%xADV%bJmEw;W%+R3n-(dP$0C>WZWP&l$F(Tf}XNiRrYh2S7}ic3N#n#Huor zD+WXcIy)LvLRgb^7AKy~TtuigHv<#>wic>^#tc3xmz*a94HJe1m*TWKsOMbIG*8qq z05_XnWvHi5R1O8Yt`t=_`qt6-{&>0Ik zwh&=1{skbUo3go$y0Y}G16`F7n~1cx2~NvDzZSJA_Nw%rD?qn!345R;0BKaPMVrw4 zn&4CE+DJ_^?Wnx4aGP_?{31ImtUlaH+SeDwcLm*K&n?B5F?pH>znW#Xx0**yJ+!~{rlD|D-V{ovn;hu;z@4InrJJ;ZBb3MI zmFDW2G85)kqcP#kx+Oo&-?;@XJoq*70B@;PqoIY#akvX$QhlEiea@s6v~1dkESYSX%y6x&NmSJ@o$5H*9t3=np0_9zS8G*&UN3wB znJCK^grNjb##2?krjwV2whNT3H48w9_T6&C1ObZ`v+kUTs?6Y%0#^isl5D#?(#emr zs3|y~qQ(S#en~(-wY6sX#32 zh}|<68e@zDB&fCv+DT|&oFl#C@x&2M&~jKCgZu#>?gDl9y=~@3Q*HRz@Bbro8FjN@ z5+Sy4gDBFciIWH&V9j?(d7Tgw*)Mo``s5D5gps)AKSb)QfL}Lm28K2YrRxqcDOYAFs$X+>h}5BERX}reV6=LZrfH7!GEG z51^he7I_Z}z2#ixy^W3Z^D&AE8$?Y>FNT=~%2gSKD4VGgaZLd& zdXe5c-00YF7N)orybb?u2FSeA<59Y+HBd8^6?jX_$bk8#%~`7XmG}&|Z$n#svh@o< zGp@Z_=ae%w;;3ywv^B5w6>AVdslaCe^wFACPUZ|pLH{~?#=_&lV&i%mf^={H`1Co5 z)pA}hBm2(P?Jm_qaNVDZx_oJywj(x2SU?LJJTNs7O(dC|Lmh6v0g-+7cuYLjW3n!9 z0EN<8XxiDnxi3U2>ShW*-pTHo^>eSC@Tgz!u(p7LNpy62wYcH&(i)aCBL64sfR-+& zMFe>QlKQmBY2G33W9{H6K7u(-HOy)ERR;Y8)ELT!j3-Nj_C38+V6osbln0_^ zb%vzgd0gAB_pI5o?0^eW+`in?OZ(Q3_uW=osckmesx=MkTFaFlG-W2px0t0|cG{}_ zDE8fC(u)fV%S6VI5AimRifx=UITF3RhUP~iuln8e(^W8M4EzW_gY=jV5C~Fp+32;4 zJR9AYHO1RBonR*IxF5@gq|KFKNR)0P;sv5DZfJajlxyYYam#HaWxcVscC~#>b9lt3 zX}drLUkQcz6}1i5i#2K5%SYZk-GQ~KF}rq~|B)R~8Fd%J#<+Lb{j_?L zpvxYpz%K?e&^3!2=?5KPl*PvnuCSm1p2x2IGP2!nkHu{_~ZO8}<|{TgJvBL?iSD zgPW|wx~Zoe)7LcfMb~%8e8}pC#PT3d9k;ylY-{jDI>y)p$zeHJUHP_-PBZ@LTa0n^ zEQ{Q%&(n{A4tlB#OMD6dDn~&pahAu@=(@I7t}Tyb&7MM+@InVc``R7MbfW^Uu}@N;l7{)|VY5v) z^)N_!MHuN`+Eh;6G!HvX-sGeZ6HhHO7EkVmdg&`mZqbQ%-QYfU3s~a-og@&_#URMY zTX&nHE=+%Zfhm17nUTcjPx-9Ka2ndAjTUFSEBe-+)G5#&C;O5%1y7olsCJS*4s}2b zJsQLzARUzSmA&Kh7A9K$1Y3^sEH6#p7$`(jXyVHRWS2j2Nc)0=F2gs!=k*wO{XIbX_3Q>Q3UMR zPDq5?a^E^Pj^=j<|LJ|eaPqbryLWr7=G%yI_ISt)&`~B^Ms8a%;<+@=l|w38Ji%Jr z=_)hB>eA9@hG0|-o)Zns2jSlnnRO_fl{;C`WC28V#CMY}^F34bKA@)4#9~iy_s(5B zTOL11&73({f8vR6!J&e9_*qxT!HVwO{EpNTzuyMmiRnNJGEwJf)z3!)>fy8|5BKUn zTHh4rTQ1m0yX&)mwx!!%fy6++4t;p@>ZkQyK9_h?4EEC3t1*s2S9d$=rr)u6N{Wm( zs#a$nv(0UTuB(sXEWqv8;?4QY<82uc`BHnaz7NaZ-tpg(I8W}~%d=-(?gQ%lVGH?N zQSmmHJYBBXdww;8dzdPiSK90K%L_dvh;*pCR@mDcceMSWmnWF8DBs*I9Kn5?LV2|2 zOOr#uJi+)pf!C})#rUfu6m0B~P<+jsNLZ&W76e+GNNVKK+wrQ51w#B0gkgcmhK7u9 z`6W+jxlqv;<@xTO*5$9Br0bX}pA8z%and;H<*)6#EG_PPY-F?lN0xfESJQR-BLy8- zQpg^c4AF9O%a`Rt(--N(S<;je-x3>$&>Af?y$LamAquV2Tc3jmb#>f}lU4f%rNASC z#VM1RZDUxg;ekDd8)l7D51DyvnaO(RN_>y8^0AF)4NnsXXPO#`w#eB4f8{;nE*S7i zLP^f)gRl4Og#LhKR^2q#_I0{c)^1ME$Yk<=P0d16Vbnd7fT;ejPJe3DSh2Hdq}`5< z`{)3otgNngHoN?LLvBe8J+ipNaAtSqL3cSV81w z+;QZ>%p!Uw{Q;CSJvFPxJX_GW34=p(#@7d*`!cg#V_eI9L#arSlhd7Y! zB=ydybrNK00#7M3k<=omy|6qPIpY2&ZVnK*l*Q_b-jMm=Dft?u{PsaHQD>Yg4SE(^ z8G*=LA2dn`fQM-O6syZ}X(Qiu^EK6oeY;nWG2y9@$&y)oaiY219oGR;q%Omfk@TSK z@jMFkYQqa3NND-CL*Gbpkp5u9;0a*ZcqviCHTlKHs1cMT(<8jKAr-ormFp3|2$I1K z`SAIqo;^d4O2QnR=GlZDk~UX3y~H zfa1mPvh{wn=FiCTy{7YFLxpcWjeM7^5bdhOKl^BtWox>+w!JA&sS(=ZFpnx{J^ryc zCCGh{+6XyDb229qPo+M`nM{A$x|KfhUa^CJl=U-WCI@}BJP_1x2B7fN>^^?HM_U+C zft}-#s7PB$X_}&3uvOenszN%MURQ&u2Cc|1)toV3$qzib;o*i|Ao)U^T8B;(73jU1 z-jv(XI)>v(kGDRQHFXa7T-sDiqXnHx-)83Y$ly0k(})fmPc~NW9S>PLP1J|84zG`N zHZ!&WQAs)c50#aKT}c?akvZ`-OKY5?MW6T&Edaw|J(>X#LQRN{i(G#F`+h_$2C!2s zcyHB&KIroPRrearQ`){lFVP6}6Lg@tCg+pHDF9Y!k$~mfz80-<`e;zbM#tvB%HCiK z|A@sXANm@bAtP~`aqqyopqa!CN#oQjKkd~W;3LIveGd3)#2#}>=lRK2gF-LU8I3Pn z-?kQ7zA}(p&TsJWA7?n?99|Fj_*U){X+NuWP~CX5-zgr7y;fUxIoF=&Y968Y+-~@Z zgeaPK*|L+k$z)nn5IVzpuhV4G6?D;pZyXQov((nI(ifaCm|VJ?abWOA7DgX-6P{|2 ziI~V(rof5|kGV@^CDlnxw435iCN9HIxJ6f+E_Z5z(}~azmsv+S=k{;Bs_oH95maE^ zz0~f-oNO8EBS+#~^*I=#vM;zB<~*JdsvM~i znKG2(fFhC9Hh%Ul5t}HcN4Mj86sEQZ9P;_~lukWj5K}^P8C3oL!__GsOHUvkAT(x>4;IhCLN(ts_zZzHi2maJ$jZWV6s(G4+C zfwI)JIT^zd+fEsWdJ62FJIy-CiX9a;;wqUlF>Gd+8U}fiLV&;?o z&f3R)5rLBc>c31@4W+2p+;lmurY8cXeyAh0V3KuAvUgT4E&MTw;(Mb#SJBQO7MTA$ zid_jiE9|i}X{WZ+fhuaq&?$f-j|O^e!?3?iEk}YK;v-OlzmTjJZ zgR60m0lQ06xZtky_z1JAF~y)`n?~|+o1TYdgtTRvEUq#}VbsbsT@Ot75?MBQU`*}# za_>vIyX4Gm@1@kTyMwql#*t~0ankr?h4S0x&NlodphBr%J?UiK}%fYat1 zyX{|V8t&F|ixM@;^)$L4ywrD6DWR!Uc!4^{G}WZNk+iXpWcJ+#caDfpOVWC>sn)}Y0yVZUegi3l7U$o5esIL+XJ0b1c z{)0Q<-s|`l&MPq@1-EeQlDq(KAOh%Lv5k@EXXaus9Kd(#cHGA$CAU^OnJjl}no1O4 z0Eq(6vc^|G=Wi)S6rqR^ri&f8kPptft5BH7?S^l&jlYM3VHyJ6uZ-V-m5SmAydfr7 zQU8Xz_6rSMEsp|prsXEL$B*)7t+xA=21arqXsQhHwLx*?;dZdam(WnO${4)0gVQ7H zk~BWWZ(3~PM;Fy0iiTu4UhMVFTU0)JnY3=MEV!Z8RCA9O0Ysg42(q&94>d;K_O1SJ zh~Px^83u;UL1D9QcpVET-AgT5h!9>n*I%A_AlIPoC~;cYBEctvm20*A(5<#%{H|UN zjupMq4&A$WS8EF2q-bLBIfrR*g>tRAa^OMOFs2cv7)@qO@~dHPGw~Pb^IzGEP0>@C zt|$A~nYs%Frp$yox)pr>K&Dw+b~>}ZDy5};dobEd8~wCC`0EH0%6uKAVP23v=bu#5 zWD#tuZN{coViC*+@0upeK%SUgG}X*^zGuWsC86L&w`SDF&x7sJc=A3{PFrNF7V#iO z?P4H@m66`L-JIm~C8U|rT}do(?%3nV_!KHuW2P8*=l8itGoT1$VupLGijCQ2(Oyv~ zV~<$;rZ%){8kHd9*T%m7@pkd2F=QKk%8Ra+b}2kTWV;??`G)6D4|_!C;_ENlODQ=N()4A2`V52lp-BM z04Wh!LXb{EqKi;MC;iI$GoAD%yh}{TdBIGhpLbAi%!cQSZT)cw#8laD+uvM375Z{ zR8CzC3`l)uR8 z(;}}#TC{>7^zLpiY<3?$<#;C7h5_xPGR!g24_{-~dBRLWXF0;lr@RBW;)Fc6bBUYgcP2FlU6Mv_{r{(b@@VOu*Rp^20S@Wg+H& zy14|Qgwq2orhYXp)t)T%2q)%v=^3p#jQHiIjr0;zJJmylkF1)5#|e=ymzYilvr_nt zXfF||($`%}zkbY#MgO^pliT))iq9ah05SSHsgV`jdlvgrc#kiXk1=in?z1>XiwMI7 zTDJ(Sh4@M5jjIkW__Ew8si>5+fae2UK+^G#^2W_{MqmV|E|{~5;6z7 zg7-O_V7YXv~=(uX9E$UuWz#^%K4||I98<`LuD1EBi4p&OvNKm32TC0m9Vx; zTfWjgesJAL*qlr9V*XeaEvzA?wI0#m<+Q=5_OxEO#*|sw82;!mu6@`nf_8An)YY(p zp7Qj1bm739d;ddeYbv^B)G{j2i$5iJBPK#1GdH{|HO-*LinaNTR%#4r3K~9@J9IyW zkzp98z^%f80BGUPV3VURiN5;2vag=m2?4yagswrsT8e^2HZUhx6=thBm!CE7y_&m= z+3%oo9W`9sl7|+WSR2!E%9*yT8fjz8^}d>etrdCLRq{B1?FxA*P+%W>K=aU@*6_oXu6Bej*zO!3wF6R^v{l@}T>CASJVva@>(IW*0D~Jzaw{V32d)6F$0Q zsKn1CJ?GNjk4Nz(0I4chXG58U=^^vOk?C`P`?01mboZrrQMB>ErFN*SmHS9paei@W zN5L|P2$B#nw*eJ?E~~OWRy1>kyQq~;QCPXrrk!!9p!i*J2_`+@vqjt~^F!hc82Bb% ze$wp`XV&d{J6v!mK|R%J$~u4U6PWi~IQ2D{7ZrXh92aB%q#`+$GE0J*pBcFxE$O9vBJ!rZniKj+Kq{r<>6qeYryaF4CU9hu`Kq zyn!uf#_vdR?J|op5?&)$uQ8hUls{%@*fG6MS!yLk@OI*4=;M#)=3#v>xMkdF*9L^T zOTIS&rtf*jTrS-D;`rv6B4Rdi%XuUfv%P3p1D%^UakMd3q$C?~+SR(v?xb5AUDXN7j%#(@U0y$MSOC3+9#e_PKG_ak*r7myF=8M^*SFEr zW#xzKN4H`D2dA{pmVB<;&Mt{o^Q2Wp4V~Y+2}}CS-AdSmt9zQA)^_b z<2yp;-a#@_XQe$^`s!P;@2J(7%GJUO+=;FG^2=!^l4ti;9neI|^Al=hL%!`E#VD_} ze7&7^16FERKe6JJuX=;lu94SSMx$|lO>K*jj{{sEZs(AK*^}zoOyg`v>&|ly6-^j` zA(BZNdxd=w1mhNXvUE_u%|W$l#&s;jX^AxdgBGfEf^xj^5#C2cebMH3(>PTw>=~MD zFE685Zl$L4vu%X$YTmr9$==ELlvye`<)&@u5cDS0J*~|&I^4ni$7bXzj^}7{!?tRq zquuN@n)e~JfysuLZJ+mOGw3q32vEc;?Va~AE1G0J((kfosNli~!CmH}CsNWTp>4B= zTSd-_XpDWhcT(LY!uD9uWYD-z&1=gHb?9akm{cZ{KE{@QLIt+_#u*rk(EUH3SRBKd zB0y+ei~hw^QFDsY@(+pq)>?WHN8+EYEn-#;ccm)vWbLr<6W2((mAR^cN&@hAm6Qnc zBfqqL5sZwM{4wed^L~V$)nXpBLb_${`kx8yA6p-lj6N*iTvEtrs&k>NM7G`;!Jc>t zrY1L!q_14qtH%_Fu0~C6^y<=^?hgJ{lPKoqkdn4ds&mNdGzS-&K#A3LGY0QB7uUHJg2f(kP!d2VHj#yU)rFo2wEjEzhKaY!Q!6 zS}O?-31H+jztlIK(MO$?ie%2pexAW#-mIqTVM4hB0=^x?CNKrBo_};4US1LSIj>&0 zT>#n}VeH@54fc0mXi%0u;WH3CrVZW8tjzzUtRGd!im0i*^p!joeY0?g`6YAmWBL4} z$+QgU7kXVkfqs^k;{U0VNxiLZpmnt=4Q|7PEp~)rw+2m+^r`w^jEC6$hf~0+K^FXe z%FRFwSTz#Is2>8jd6Z;SWmG=+jQ$~Jqm=F@*QIN52VyuM)^V~@IYEU__40X6#Zon; zdWYfrqRQKX^G$%UYYltc8uUbj27<2Nl-S5J1ws#fjU#0dG4(4#PxZ%VW=ZJ4Cj_`f zNk^pxsJbIZe;rZ=+$L4Y!W09O_NevB3s8Yi_DYx&a?N&5dY6VU+y@{|+)AJhE!<(V z7@4k{O?BR^n^n-EA6_kuO|FO*>&)>8JvF0WPyAqKrHSgTppuBn*)O&WtI=)A{K`)o zOEsj+i!P{5Ykt~$KVZ)gfVn-RG%piB@VoXe{ODiiKHG6-;+reFzbnX&QvH%(gG|** zA;HJbyH>22fjH6cH}Qp41Dbpn;C zWuMoEWhP2WQY51di10#r*vxIdRN;0?2e_r8T3=0uQs{0^?CT|{sh7@IGJj-kyxoP$ z-0gP}|6rdaJ5?4SBubE_Yp0t)*RdHWr3uFq+J?y@?!t7=LY6#={CbxK_!m4j#+Y;u zEI1@qU_t}`&ZR!pwYKbYftPwI?BK+?j$uJI8}#nYJ?L598~!VZuifaJlXVV#d$P6d zi)=k9cGXEcxX^XuX=p#8_I@AuOU-nd_o-6jx_;8`=lmwwmM{A!1jg^f`CS*KuQM+^ zx!byxUO}l^u3|VBo*WJ(F1fHSr;Dcyz_k4~+Rl?|>!H6NVU#!q49D>$1lk$Kas1(L zv-ab?@~bzf5~Yfj9GjD%zd*kVLn*m8Q*8tuSzoj|sNNOOA*tA^6khVvM~d?riG5n5 zGj8+tCO>YmTiIm`RpShk(aBvD6E!Q@oMzrJUIyY!{Y~cWTeqqn{qZ#B?0$jXtIA~& zCDEfxbJ2?X&7)l6@YMC@xl$X`NWk4YJxc|YFY*^%UO?v1o#2|1Vqo;V>(SqqBZ158 z!jqiZMsjXeG*NQs0EWzck+c1Kro1Ioq-P9zqunbsM&3Qnj%4RU>m*vVJ@@pP#yU&W ziASs&7G(72?m2Pb%KuIWRtowsB_M^3iZPkjb^N$zy*5DRw6w>!nW zvx&dTl>A&*?G6@xjdnd{yOckQ-bN3e+2nC2o-QSBvq@-KG@hU@+JV3&W0{L7e@*QBEO#?ov|xeeFK5`E3|25wA7zh?hd@(Vsxd4 z#}jJbo1|IFXrL`;dnHCJhSD0;6uDDyli;qlKp=iU=Qb(DRwp85Nttu-uqC)V>x=WwQ!b)%CTL1Zxor?ev?Q?r_$e#o$>**^xvmqv883t+oA<;&;o^G zU^BK>NSjRC%r*?$Knr*&mj4N>Q_I@Fs22esB0kWiVFXC zPiV>9GF?|Hd7to>j0|&2gBH0Zh{ZQT&gaclm{wudXKGB#=jx7X=zS)Dht@%r}l z%-70J$vhCAP^OPdrMUurA*G_jaP}yew`l3)@@+N=A<5fr~3`XY^JbC${cR zk{FoPfEZL(2F_&`Y^u=ZAMZi;uuJ}0Kl7)1JC~Dla$A63sp*L(|BTD+P9t|!XEVY( zhgnIZo}7MqZ?G>+IIiNWQ!pV!%cK-B`gekI=qJ5|lZyKTpIis-AZGQ$boJ_WHXNkA zMP0VP*nmStY$65WL{gqu9$}YEtH;hkVCjwPJwwqghTnxIIMC}7uG-o4Dil1N`3iW>!-l8#I zy0dC4y<8zLwNY)bK&_z@e|fk5xP~V!P%CKOkadsa-ZkG}G|w411!c+iv2^NK*9YUU z%kz~cl;y+H#u(!4a*7QScNwzR;|hpNYpntuqz(Nf!K8))erz9Fec_9l6L$w$e+PB* z83SB1Z69PWPmrRgUsex`HL2fmQKvmu&M)`yjl^~`6bTa^P1zA%QgC9tIE087O{K4^ zSq;u_E#!7vA|`7f1xu8g*35uZlW<4G>O;uf|2PzZ2zw03ejo^6d(OIO$oeHf<30X* z=B@ui&0Jnq0Lm9+#p4aSB0>VuL5rZn5v3Q(>r1QdH67AGW|+D|mA~2v-n3bhN36UL z@j}z{OfkA3)G{y+a#Kt~pN}cMr<$jFq!H9u@MZR4a@v;lkp_ZaWGKP6 z(MDW*B)oBXQMV~r-kkVFeY3juk*H$k~Z;Q{Gz!|!DT03y1D-tW3PT1t)TV|~n zj9KXy_CMMy-#+#!otpc|IFR_om*5GpxVOkU`*pNQkZa6ZYykjiX5jVV^U+N&7s_gJ zX6PjSMI%#P?=RT}06R&K04EU1M)|A=Zr9Ka24+8;#uG)AUzyvlA%Kkx7?Sd$jyNA| z62oP#*1+}mJ0qH%prc*(oOgAEZkRvx8P%i633oABqYKFeEjCWK)Nb9n#k9#`yno|X z!F+e&tu>A(xrEo3mulE)wojgld`5n#%Xu|u#SEL>octn3rMlYM zT{m^T%Sy|BNH7b<#r)GgDYgOhnttPS_qa#uGBSua6IMcSsi-*M%|}apwAD0v$sU+M zkw|KY(Kf1hWAA--x)B5$#_XyGDg8+j9Ey&C?3Bg_MWX7$ZK}FBOUc73Emg`+pig7V zH91O9Z(3GtoYII;-{Pr8)J>)(sHA65Jw_l#?mEsCA{T=z5P-p@g5l&g6>lG}J|GH} z&7?Kx&ff~);17d>-8-O_;X;-c8_F75V6dp#y=|06OJlF!( zj2=PcX>R}CF76IyFGgo~^g9lW1$pm$`_bqFv@yIhdTz&Laevp#w*#<9D0P|$*sq^) ze0g)qB=j+evVrlwnPc`9<|_1zNWy`Lee00stx_x@O}ma+(+75*$O34JSaYq_`;r4- zxV$=t%G}9I_2W^r?!w(V;4~BB!6i<_c}dL2qt9IKOq4E!Y@j0CrQn*+G2v5cCCna^ z2$M3+Y7DX46F%Q)h`ffFgqv;TQ8QW(vkbt~-i0Kw1wU|0jciN#m*)|Ptk)Y5_hTn0 z&!mlf=%eBCbrf*uyjNjq;XB$3KUN!uIG7~vN?uqCtu;y!qu4r+ z)^MX82UE8@5_YK%i(_KY#U(LS`=%~h(2EYycC)v%p|aMqLj;V3$>WEV(;#5{m`u7U zaNEh*mZhV|xUQwbbm-3U*es3MWHiUrXT^f-MO+H-rmQ0FD>ARD{x%5{krIhtZlD(4 z1cnBJp_gvxm;}*?Gw&*4rOTic8~d(nSiM6>At&D)kZ6x-06i1I(CgImn}GK0qnb03 z;R;Fs!>&a)d1$WU@=|rJ!Ov3evY+n_ptsu3aN5Q@w$QIkF&k7{?lyfAiQKx&k_?{E zY6LV;iTCiEj$?KA>ek7p!^&mth+Ywfq{4bO$vhU4ANx$@my)eJR#x-0DAbjNH#2H45nIwN0MexC3j0ZoS6XC!O42$=ll_wi|=SLBG24 zj~9wC5#LwjDDVs>qPE4L%CWq_pjhyp08f1#@kB&VQ_8ePx+6A>>0}bt9=N2HHt}qN5J{LSE>&kqJPFkcEKt#h`{8trAY1#xk@08|E7B0>&?!oHp zQsc6eF;8IsCObvUH&LvhZUGq4iXM|=baF5vcN^}qGgdPo^^Gun;ETWRRn#nWUX zWZQR!cyJh$s(rs-F9F8 zRhxx}Tyj5&f(Z07o4*sCo0axmb+~?!PB%^Vr9OASyM7P6QsgZeL#I3hh$PqD4 zhiOLN`Q&b}>IWeY^Pfcq6;JO&NubtJNC7#r9wiubvh3@-SACY8UJL=MfUQ!3%J~`_ z*}v=uwrO{zU75Y$1-zo72+lX6js!Y*$PuZTp%r&n&@#9C4ljAG41;#h-&04TB)5fl z+7sx zkUSv_GeUX4alZ%ST6O-gOgJxu4Fnz_MOG*bGZ%x zEmuD^&Cv<{jIH%7zd34HV72qq>|2H9$-0gAY5c41r2_`FDH#T1%OPRQE_pk|6F?=)*P*5mO*Tj%5Bp1!C!~rukT{jdb)g(QpN!AMbND2)xVlYo$^7 zM|FZj?g5S3b>R&1^xTi6*It)a%?f+tnSQA`_2k-}sTys9bOi2)apCceh`lL#r_;t? zFg7<`e!72QO9Hw?@^n6?wdoMPb=M_SO*A`=Fnv5?O}~{G3boJPhbc_62Iy0p_j|1| zT~k0$vI(KBAw?V;r=h`^bJi|hxF0OWl!13nA6414_KtRgr%U8qJ$=)-Z%-StLD@hv zBs}vCorXgzK0TO#1+%%6|971#A6GPkV)07yWg1`N?5AQ5TZLB!B}%1VuU@Er(5}x= z{j7Yf1oc+tt#XsfIhDV1N6QE!e<=M0B7qJ!tIR2X4)x+ZA)^ZGVf|`niT(%(W5rwyhkEOn;(RncaPJIod%QK%IC@& zG;f68`st~$qq~qdt;|cYmK`?f7fBv#k4&v_k0gcrxiB0pwOtsw5xhmb9rWR5|IUMl zH^&-JmNM`Ade>whxEzJmD?F>*^0%LFEmppze!}=PNk;M?Pu>c!DzMv1nu4K zC{{P^S1Cq{p|Uc?CUU%OkE(S@ti%qmdc8KV(aw&@b1uW9UBMD7n)^D-TZMh+SHF0* z+$|)(uw&4jBdY|DPR|agFQA5Wmd8v{uqL9L?qumm`^4`3rZwHVJw&yOdOyIYP-xYO zAe%%H1tIu>vwQCQ5BCv6>(ags83wTh?%wK{Bn@KG*im{`^QvGX^Uu~Zm*|!f%;Y*& zPWtHRCPiOwYhmZU_Vg{3tmnx3mxSBmj;Vs8I}=Sbhev#&S>Y$F_wG?Ttyqlk4F{=) zjsuFM6O3C+Sgycbs)$u!jpjTALr^vZa2xsr4<568#jOpE-&0Q?kPg27Z zkG)}+5b{>mgx(oD1K__)Kn}A=UWJDZJTmAGjWZov*3J>Lo&J@5-0cel2RgiU8b?U9?c$BkkGh;zO))h znxF=w#^cNA=aDO^#J8=~b4Fuu^j5!jWJKvsSRK-lV%GZmmS_aKB^p6%Tm{rx2ccT_ z2d$A%o2R!@Z4fq1M)~6VBFL@fZ_J9XxpRNp?x}uO`6JJ=QhwE}ln#l~Vq*YyO)7nl4k-GGKGYQMq_2UzMyt@y zUQCT?dQ?ImUk*X&x3V&`U36a0jDd+6FR6(V zZkPy9ztdbnpO^J&v|?oBj+%mCX}9yJ2_|M$raA4kyNJ+82YCXfS+h;luCkCPWv6eG z*r*?&cXwa{7>_QXZ&z#^Pg>hH=7#-kUAVXqBK5(@w~<}}|G$5Zfu9CJ-J(?7jOt#? z1H=QxS2=GFWFJyZ1kHs*>Gag#fTLW@oETNDXB4Bo%0HaG>o=VX`Z6Fv`IIo{yKTLb zYQg_gx)!>7w;PJkDiny;aQ~P^A8)v?&VZ(jk2q0JCE1K_HPWEPeUdtTMe!q62M%y{9LPX6zuy{fI7}7Q3-fT}1D}pe*-_o$Ke`zZZ?!}nYYBnu4n7}! z2brqu!#wIB*P?!soALm>bGAkLKDTxBS3h14K~&NVLr?+J2piZ24kc$}}tjOlD}$FMAcl zZ<@_NOf!7Q@sG^&pM((*uR(XwLn@bq=wz>QZkxe1JXhG)`}P1<;QKqw`uJcaw;|0- zZTzJW7xN^gd|{1|Nl{+`9w*cf(NgRKq12M|C-yBFF`apx0Y0TTAI_Xjtg(jdK4hU&htG2d zpmS={F@1MZiVFVx-L3o2eQxj~`!~m4tnB;keDZ&}V8?ofwd-1}`mco0|GQeft-%t| z83k%k*6aW6U-|z$ikSm!-PwVsjpqMDJ@xC(I6xCceLxT@Xt{K)C$?I_*iPHx9GApxaJ7nRx_%q@=OMs3#oqUPPj1X7b6 z3F$rOL(c^-HID>W6pZ?t(Ei`Ll>W2qF^i6rP?P^DcfsHQyFqn;{6EGK{D0;x>(@F0 z8>HB5iMBzWrv1I06u>dYZ#K|qu{cHjj?+WBMGBj(hy;+A8UUs<_6?~%yHlG%F-+W& z92{QTq$vQCVd`%*!b*3)MQt0>cBZLmSj5u({Z25|dTs{Y%u1KG?Aj}!T$}6R-&2^X zZ>+tUejQF48z>~~4y>L=AoccL^Z<5%p>)_%%`(?e34-V~#r6Y>NRx~^K5W?0>y^Wt z;LUiaF=+QGwN4s2h;F3I!`EtlA1J^;Ri=Nd-u!(K0GpOLf3S-Tm=Af;0U!Bv!lKxp zJrRIBIlZ{0qeX7N8mZn)`_}aWQy+i?c*7a9RL)J}3gG(2FP{4Pm!+CJjV<#=H+E0e zO+EIe0|)E$$g({;g4Q^ftjcEru-tfR3{FdZN)NpO+Ju4~e=lJ+=i{a`_p# zGU-5bJOFo*CgevCCdgVcn{TEZ=wR?2k|utqb#$vJSF# zM0nO2o4jStdF;Buo2yA|jzh^rY3=g7A(Xa~yRLF3pMF^?L|VcV-3bGDuUl+GvEPK= zw(cZgYDBv9-uDq;Uow+_T42r2Jh}j{(T_}Egr)YZ<3C1w;GHpoI+&45NHxPojTx}5 zV^6qlf6Kl^$7q@jMf?DLcLjWsy4YZYu4#Z+EL-ajDV8XXX8XyxgJT4C_W*O*G{!Fj z27@~gz_x1;+ge!6K7M*RxvNCmDq8c&bbj^4Sd;Rhmu7Sm>GtWvXUKtCGxsV{Ta2=cV?%qsNJc9Um z#ut!j31+XE^bqu?Lf}pMxMRV~!VOQZ;Oqk8o+vpH6_+e!2i-q?;a{%!H-I!>cCPI! zSOEdH#~o7{BQJqi7FiMEUrqML-;k! zzg&OhrugS;0jlPmdCPc1*}n+C)h(0Efqa4l8rc2=unF8D4A};)^#a^;i0mC~{&CmEm!}oU<3I|EWv*toHWh{iw&K){ zHhSz1Nn7#tT7=$H6j7Tmkl11aw^r2~D{?HgUz)kXg$M$YCsVlNnrY9BjP3`#%;50D z*QTQ)QY&)vn|p8T<&4Wt1tnF(g&H@DPfBM1Hj+}VyJ>lCzP>zcK=OGLCOKA1>}ENI2rgK;!x+{ zF&8!G5j@v_T=ty5fCrHdzb(%#^XltnNxtu${Sd%1$-Xh<6oFQODZNmwI22d2T*8evx7BxHMMeQQCnf z!hHlr1?(q*0xx`2sYOisD0pT3)Qm7vU?1RF_^Gup3YL$ZD8nIh;i?#p+pLfhs!5(F-~PaDz?JJNbCs)3AblcZ2#Yw#NCS|?udV*C!6Cl?itx`NW|9}tZ>x*4H( zh%S*ST*UPO|Kl=KhbrLCn<*%JwDDX^OHOrHy}2fb(gzEUIl~|e)?HuPH$RcY2FWdT z3Qm9>+!IlEwUJzU6i~`9CtAvdlzVZS7SpsR&$s-^HO{xzgsn+GjYR3>lR$6!zD2q~ zFe=M)3YYw(&md@&9(dpUJWJMP%m*&sIxi^>E$7Mo*r1uHWC<0VN4t(+O!m`9*Skm9 z<*x`Z$k|Qg!>;}EXOdb2N7vgY22JpI>j`FArI|~fxNB(sgor4-3aGruoO{+fs7pYd zG^)}qM4xM+gp$OvBwM03YB0{O8n1`R(py*l#7P-x8xjZnj66=-MT!e|>M)ip&;x#x zHKT?`K&hw2L7&oj_w_`I511}^v?9XoWRpZv(k#RG(i=ZN)IoF(Z z2^6@!_RGDJnnps2%>uSD4$Lw64w>4VzEV>|BM?4Yw^ryfQ(4mV`0j`1r5{}$*)Do< z+&FFy4s*UBQ`xy2Rf>5l+SVML;qt{=*@wWhF_Twmx1M2Cl5gD2JwEIm-jS{pZ;jW7 z7){DQcROkXEi^U^x`(tvpbjz#>V$jdD%;pZl$k``Ff&WyseJw}StznFm!YR|gobl4 zMxrw;d!8uH9*atayBOAI8py$T0}amkuHtkJGZx_)CbjB$Pvynu*KHlJS;qwju2c2F zt`*u?QX>%z6oEPk*)HMt>ABWXo~S{xO8)$CAi<QtqacU0@}N88*+nz zEg|lw^-~Eh$C{*hzG)_1P!1~JUYpXA!gE|&&9TNS=$xDKshiY;H5VZ521XbD%fa@) zd0B;F7aFWTO9FVw8KUrwIJ<{-KNZ7eoa{%qIVWt;b~KDW$y~GOKt;W89bbHKyw@nG zv?dG6aSWfk6sIK}yjM^Vq<@nLO}PX#9%@a3 zPsoF0&1Di~s%9|W<^s6m;UquY@sJ;5ibh^lS&QN#o9jro@+xaJ^LylBr??Av{Uh#2 z#MipBQEiVH*xaw}A2jnYB7vAEvDV{0*uIJM+ghEWkNc_a(|{kmcEA;|fHH8GL;A&~ zQATT-cH7u-Kyk69-J)2zHlVd=DOS`_T8T})e;lE+^e{DTFzsEgrBt4mLsoav_l7{~ zr72V;T3DX`{92qOP<2>Cy;p(anD?ZhTRxjFl_J*@0Zg_&Ok6yn{;X!w4P72Ntn8Kh7A5<2*oI!(vxlnxG_xP zPr5Q@uZi6)V5Z*=n3NE!8hYf7GU+XPXJR-yN`&=Bm7~&9)6%rCUE=w_>Q@Q7|0Yv= zz|Pbj2);7`97{cjAKhf7dW^n*8JJ6ZUQnI6SV+$9S?cvbGbi$fDdIfELc`5R99rJ<-0&>D|; zI5lx6R~3!i2dgY94Y>Cp&;n-h%PA9 z{+Hq<*t}+^Tf;|!KN8pBN$H2+!ovtp+8#U+6Y zPXz=MVXa;wU!6DKF$bXW8LVQBN@h+Xr2!Y)uT{G7;7mZB+qKnn4JJfy*IbeHWCr@ese0y5BH)#@M+p0*9!l^gxH82Y#L#1dvJVN zZ7PJCmgy=+ci0mQb}MSFC*55H8QA?@z6dpYke!S zo8Y!-lm8;F`rETVlWMC4bwFIT7r1fx%&VJE?5bQ8+-|DXsd62Ro!e;=bRtvVq`s@N z(tITV^gQcQVU32Mva~mst$K=;@v1DpH)D^eY#$S)rwf(#l`DSW_z1SwkX)?MQav2G zO74{{ANxY=+ZLD?FjgubzZo}))1b+)WK?9qH^PDAY%3&=!@l0PRJcQOj(2)-aMl4{ z@ql>J@jCbER4W;O!2DKY36L{t@$Us0ZqqZOjDxPvR?>O2QmvZ+e^@@b0|SN^yA?#2 z5tiw=EGrI9yz2;2g_48qL=WTnkc#%23CQ_)qzUy@6fQGF4*SqKT_7IuS1Mzrlb@f0 z$Q=F=wt6@14V^50b8tS}860#Odo4M^=zg z&e??_B!vaK{35E{?S4Q>{jSUP<)VRtKW4%x{kceWDQ54{O>)ZK2zFg(-=Ts?wsGSA zo4b|Vy4lgn>T6Ba^*cM`>$EHcPp*4(EeGy%`pB?W7=Y^iFMQ7b<7J{18!DB4feOHj znM7fkwH~d>7qGvH?|(wC){x~-WS62TEHipyUVBJ5J~7L;*;{k zxJ81&MR;%!d1dkumwPmQ(Ns$`@n@X_{56$36Lnoe`EM^vnHbLappJu^YYn>_ighDO;i-+k4rg;o$D@AOI_|)yW zvLR&3{!GiwYvGg#B7sk@=U@kSOLf81i-E`CdkdC0nZh{1d|ejV0jD6QV2SH^M5>OJ z;ppa;TQf<0iPo)is=Y)ugb@D*^9ic`q8b~{o}_07e&`D|l z!eKrsvBsaqpGo?@)YbbNv%VCnmy*#GC^m|uD~M*)S6f6Gwq&~bjfFP@w( z;bMcnV&x`7fogEN!ogx8$o%W4)=saLTFM23l+eP5mDC9NULy6nRNgHdXYwF*G!k}o zC@-Xqfs+$_s&v=SQovop@&~JN;e&Dl_44#$_}o+H2+-W(+oHFA*(oR^&^nJKXhvjt zD_${5vn{oUoTOODyo{U*w8?D$OFicBp`*pesLFqjNBji1T#lkYewqQS;}GU)&vtYH zOtu8^ofz8?Rz@lOp#@QNk#&-hGQe(Rfr)C~^}kYhq~f@SJ$?KAza<`sBVL?lt%$bkm4JO_5Ji!UN5ns{7A`KGCM1aDPTOv3@u=%)o<$ zy@PL~v(qA;jsn4Pp&c-fZJCXZM_zq26WYkh5AmdN(ES#uLW8`d*RE_v%8uv z!$vBw$l$4e@o+3NfufzImddAhKum<2$;2F$6S82cg0W7ll`TR*no3R&G!NIR)dim)sip#V-cXiME?K^UA9SDm zGbyn%!do03`s@)Va@ZOyu1(LnHd2AqEYu^3jXxgIU|8*}HT}MXoX#5^WqUI{+2Yeb zP;Bre4p2d5>)!SwEN9VR^~*!+N-`Q?*2#DGAu$IzJuS^;1x<{%$XTz;mU&6$!Z_jN z1#_rptlky&T1xr(`PL?iP2N+70rR(+63$W5-3_(Vay zK@)ba`*aOK4O&_A8loA-t33vz(@l7pxDE;axy%BcAft_%MV&!|x%^KroVr49cXme* zPOJBALwEH)Y$oL7+Mr90`+@7N*A#n-Lz^SQAP0`MO$@U?tV#Cl-tx6>O~ z@-N$yCk6e?IphWd3NUfHg(9Te15W0MI77ba`4{W=r1b3J_%OgkDf`3Ct=DIT-`BrH zKYzXR9a#MxYTn8VG7H?6wS;`{CAzYA8};0D^#>t+2_9eWul2r8ZWetbKBxD?rRkliO?;^r)05R-q)~zsp;1~3+-Rn+v11XW zQ#P1R1czdrCgPbkVqowpP#%{~ELiPXIvbD>;;C_mL(^Psa2|~0E$qXc2-BigYoYFQ zyRx-If~tr3)VP6@np2k!wiU4T0Ru-*CC_ae zIT>$pPU5;+vzBpJ87%?PAs40GuHe>z+0%6US><$`N{G*MKL%2NXYhl+fGS=7?VV5X zFP@}@&gQ-5? z28t%8!Fl?Dqlg#j4t%yRNb6}LLO0tCvPrvlT0bH?W%Z9HTb`E_!mpdg$uZtzf{FaF zWMOcK`0(cmC11z&#@+wEP5!?((cK{IMBW?xTRY4xjJn>x37nJMxkdCIyQs;X zb2#&MMmn0um1x0|5gl9lY%Ke-&FvH+jWY3fbH9(&@z!P`%8Z{{@kyio9#Jca0eMlH ztyzwI$%mbH@(t0VzIowv$z^uCfqZoK_{$Mx_e9p?y)CNcpy0+mN0E@KERzu7jUFv{ zO008u;p4yGVZHe;1$=L}ot}T&!Vxj&WhbxKaAtbGfI3!x+nsMb?W`Upb9%KsI^G*U z4Rt)>MM5R9s!wqL?SIeNyQ%UYiO` z#GJ49uzS1sq1S`4dFOY%QNVSB3)t;lH{omP4(yD%JM~EiEuz!5h2OY-;RALcR) zoAd!TWchY@(?FJ-V~0u^=ld?hwup2g*_U zDDH}#G@MS7qDjP5n1Vec(|_X5bVwZHA_&V)0DhqG@HvhcKBIC0vy~wx$2l+(KWbbU zoTmB2fXd2anhhtWjZcQ&HE!C-(AsSh!r%(;dA`1s>c3D?#|N?HQAw4Da3^!0mI#Qy z!Q~pcs_1ijaDFccJb$n#<&yvNNv~RM{6~#(2|QGWO|6 zlnvG1{-ClgoQ`VENivkp@Gk6(8;FyS)5$|WpTs!3Nz{BZ&2`?R~8@hcEYYq z+K1Dfr593_&W;j2tiqsZ=U@Rvhp;=uR1MWVtq-h622|-rv}L`iMD3(rMh)`Q%Diw9 z$h~0Q!=%p$L+R$$Mt)`!!+pPu-A>ybo%Y{#^Jqq8BXL?C5{MCt1hnytcR}X2`sIPY#$xhrRCD2yd#0RLs#T zJygfgi4iJ3${AV=aDoG(8Zj)}3yPQO0WD*~%sgQ5BGnaP5y=?=<0?Kf&2t7?o`eMI z=neI3`nK;W#9v+xof$m|%y#BB7-+JZ^|y187#WgtMbZb4A-N77Zoi0=8K&EHdH?Ww z9cWY{Ac>7O@X}5X`YCJf#($!Wk}V2gpJmKzNZhhnVGK|7{l0p}!Z5gRu=AvuDNjr9 zi%Go}p*I39sC}K$fsk} zKEnaRh(z)zfMWH9ZTlzx@VD)U3OYoGp65eGW7G0OK`ZO>sW(GxiHetQGzB|H6{p&i zLXQunFCW6bn4#(mZVfAE|{1WJa|A<^>L^E%c1 zcws?zL4jZ};^W~YiOvBAhkD@<<{r$^=jRnsI@uv?QFh^HU$RA_c0&CKe<3oLI-}Nv(vS-MAftl3e%1r5~qo&?Z_ZVQB za~Bez(uM12)AKjmKDzg6MDPF}crGzqTt|{O!v~_p6L7+X^5$9?B$3jxWn9XCX34sH zG#j&BqXFMRLlPnRI-5MfsTmhJBy2D*3m8YOG2{0sS^P9+wWys0z-Z5- zlB|izWleeD%4OR*pQihJ$=BvO6Dss-Lu{So(C&orV@;CVBn=ExDPeF+%g3RTrlYgA>2ZQb&KrFm&P)Yet6DvKL6K(RJX&5{`I zt_eu3T%voQ;*Z#6oq#-R-L@w6sFUDrQpLqYoK)y5tM0UE5}qtgx=un!g#NNgC2G&7 zaJG*5`R+lt6r}0S6s`oC9yF*P78^=Tk79Yo(zh^3#gfgB|Kgm>7T%{{S9)-ny-PaP zbAYx))_C|Ncz1Z0d6x!*$5FDNorni<+~x<$ICvThmQBOJrY})AN=M#pV_eQmQb_C! zOufoseIlq*I1ClRcZk2?(=bz_~jS+T)coyFmQ(|P^E0p7;sNPArKcKl~MZ^J5F=(+Ixf+hf^Q~rB7Ua8yuq+O<~ z_6KjDFtaBzR6us-$VN@Tv*lpVD|8@!4|f*pGB6%OQQeU`^q1DKLc<{8eX>Wu%K?0# z$!19)B{KXP#)D8w=>8S>xA(}RFB2L;AaWS~a6v_skT(BC$zUVDZSPjw#`xdquK#>u z{L8tOT8uW{=Rgv(=Jr^RTbGX?3{SdFGc~iH-?7cx6a0vHv+uiIb-ygg=YO&H-ce0$ zf4i>*H;77+P^1bdRX{|Vp(vbRs1tfCxzFB1$hR0v1|8dWR5dh>Ak! zB|ualAqkKK0-VKu@44qK=iQ#)pLg6d#u^UA=vZZDt~tNo&+|O4c?q?agFX*^mGQn^ zNd;fs4Y%9#-F?_nj< z5tL8AnRoun#MGS)l*_uoE$hFf#sB^4eQ&~gwg(gcIRp0o{aW!qfOc)vy`(y7l5g|B zdYL+>_E>`BObgrJO|SpyGX0tW?Ix^=XiNG1#^gUVfyWN?n}H@(JDdOhBP^zm-p;&#=C zX$J>h!yxZ&O7kaY!)|T_<D%825kV{T(M5<`^mv% zV}ay7(s{n;V|mLazhh}Qr3xRmQ6=kU(mp+(igd1JI~RwXN}kVL;(RH;)$aO>R2k^3 zzdAZigdpbxh`ygMHNyx`(a81*HqR34)krkk^$3{eybC!x1~;z$RBg4%EKbE zMg$kVPRISmqpU*kfbCZJVC}gZd(pMaiu(_F9Qp{vDV4|nUD!rJ4|EOU!*}TM_)s#e z!0F@3S}F5(vxr}%wbOT-_UJ3VR~eMEq}xY)8>`LR2VU#MMHAJ+S^aZ+nyb0JI-df! z(pSYljxQn?HS+I}BKaN2lt5|U#xO;P%Is|~Ys}4u?!-N0CW@bg&zAyNOObHE^)PWO zu2h9D>;*I1x22xBm(&#Eth}Q$7yD69pBpEd2RsTbDiMIdr992^`cJTo`J{^{cl)ec z&LJo74|q)V>MxC5ptAP@MB7z7m-b+08>@b7zi7vIx>s9{0ug{j)@GX1J?_hsYCFAlf)A$)$Xgt3;l4d8BJw(B+E|=z{z3 z(dMqb%s2J;x#dLdsk%DAqF89pT0tRbrE|^rMZrv(vE}uYU-6FYxjEop(2AZsZmv|^ zJq`B%x;GNqiF?03l741?H|}j*F!u{>prkuz3+#3k=9GU)+?EE5FK<#mI&MFp0u6GN zJ?6?1Y25}g>AG^LZ_o1<;F8%;JsIrohVQNx*`C!KIr{okO_;dB;eC?u`IN3bBMQQcz(%tu~7sVB`( z#;djH(2@YEI&K0e9Fa&Pmz~}5c@nVF$}ggQJklJ(G{#dg9}$Z5+&)jg^5M)U20>Hd zynqCQ&v~|;yvx|#nQj53{Y%?M91CqdqLZG^N$aX|Y*+|BEBkRf_?Ri0Y9Kq9!n>DC z!MB9&YVN!rCGvNn9Jiy2#zHdkK*=F7R}UV5d?XLA zrCdoBAOH|(s{?y4KX2xY0sR}kFS0D6}}sio{8U{{B#SR==(2?1t(I}G?0;2@B={mAc^*}X2uWa zB+QNjCb}6XN(8y zOOBWrc+u(F#n%^?u8Z8hX{dy|dm3fglolB<1WDV>qUu~~-RLUtpSxEXh zEN{oEwKRVDPv;dr=KbNPP4Rpjru80)ca{Rw8Zjcaf3w?tx0b%zC?-@XA7W5HtM0^} zO)*zw+b*&9!r*kF%~A&h%_>fwU)|~IMewFJ-=8X2tQhRlP{;YmBEPM5-d#TIyy_IW zMYBWl6u7#lp!~R0=IbUPM9o?KsOZJi`^WN|dxqGd z6^L&@{HOkuB1h%0Ep}Sjaw3S(>$(eWZ;a+LNv-v$+tViyCASDl!Sp=fU8r8~YG-tk za*-l6LY05o*U;NEzHRq%WL%>7cTNsYg1|TWcFNE7<2v^aaNmxz=U#}j^@z(W+bYMd zTjq=#pH|ImW)Gp8ccG0u-b$n#}ZZgHR|c``?Hgng>ExNTJJ5KV^P4 zUKX4xtw!y$xhHW~fHrY=)!^*Lr{u7yBRmX2daN0c0d_0V9#|}p_yl!o1yFKorQx3O z3XuwZqOyNo_X<{PG%OWGfJf*%2!HRSvnMUih6e%etbtYO#~i&5ziwinoCuZHiE2@Y z2&309;M)#1TRi^4+OS8(@ZCKDL?mmk`mEy~oYq|Fp@x-CShZM;ms6d>br<4B4mro> zq^Cl!yl}Aw=kw~XeYWu>#}>AJZ^T(GC0y(mU#!|1Xx-gvO}2NLKZ>4qj;+>;Q*Ye% z<$_z>G<$b15jXyU;WFidrMA#jt9-b|*c2Jg>13-i-00fh@#X)1cKr{(Z=>!-A5&2- z%{o^(ZtS!6yMcl+)w%?y4xa~AgS-Tb1wVj(f(}-H9#e*j1Sv6R7Z6drl%~ep zs8FJ@n^$~3_IpJSejE8~DF3bNcfJrKmrql3SZ5OyXVLyt)calW9VPtB1`vy}VuTC%SRoTX+ zcWM$dIc3AP26@aAiao!MPe91x`r+S>s&%xJ)1XL=LMhAlwe}X-=OcJVE^{X0P7i zI1?4R#gY1e4vw&WK!AOIdo9@yWj=IZ{>P{Jd_T<^H}S46@nt~>>y=2a_5|zweIPWi zh|uHKSUD4UD{Cu&*ejoJS1S>3Ivj_xF7N8rz`o#(`yk3Q%^D0ICN?R#0UlcTN|RR4 z^2Q+l(ZcjZ@t4K4Th#A7S7{fW)3<3yAy0UBoE6>TiZ34l*8fWtk!oe$g}v7GvcAg| zPDr|ZdRL~(vJRX!-y=H$a#9B<^>p|1p2mQ?mHF`8ZLcr#SDaZ2K>_y)p60tRHsRv; zgZ`PmKM~qgT@n_y@-{Qb+nA_ zIce?r3{52=cD@$NB^Qh`zy zRxZ|Jlyjcf94WjqM@#&N=lN$p$5v0KGzB*wl!311e&P8&>zmLalZ%x|cEh7|G?>ji)-YutPqT?(XR+$`5+lQcDvxqBqN2W^! ztW6;(5>`Qwmgv_Z6qd3(P;O!*IKNE3H2cL)E~w`WVZ zddhKG>R)rJOe~YUoeTUv^rSth+$vL?5)cg;OnLKu_xbeTA8c(4?ys9lzbd6ncg*Wi z@f{jylOU?IE%ztx1h21nx`@I)pKk;pnKPlzgS%-l$llrV%xa*NDQp$r4D$4Ezd8FA zhDGs3snFtZdsgnN?KZwUL|oS%XiMv{ffo41#bWzAqruISPHhHyp-|Et zv@7FW5;b#Wu?=4rrG(VrNeY=+7z#9qJ3Xx)h!a_a;gNGmR%LVnU#(U)DP5S-%5Yv@ zS1+q)IpMCNM7M&rdh>oZwA~w%aUyAhUNx!YTD@e zbolPZub4)>Q)+AbCQX-Zr!BuR9$ReJZqD8grE0GMJ1?sdtS*9mtRu~^MIfNkcPybB zsF7gh&@WBBFy=d(jc0Cq%z|BW5C&)%#D_WG3X7o+#5?Z%u|O##O=@!@#E0DiJXOg< zhTa#*L32V|gt6c8m_UhmNKYX`_UgrUQ}twmQG^A5zmG(m$)Mm1(5DG20JnSX#eZW%QmBwF-@W}V9$8_@qnF| z{Id#yryXNg&Hb||JO*$fv7OcY>&7XN{kBq9+=)PHM4aMPfrwR3;Vviev5H&HLm^&) z)2~tQUj0bj*XDfmnCYSKCX(=bpqNo79vJ<*O~&TUHnWtMiQK=$G_^u3Qf_v}HA9~| z&czlOD2_Eybn}-PIU;YFKdr**!T=>PI{QgSP>{MvYN>Zy`)%-13=a&Ow0c7LHoE}g zbv`XJm-~gy;G|XR#J%YN#!fPBby|4+NEwfYVns98QEbOxzRB zFC9ORb^-(T+z=dPNF3~qZR@)LNLm(JJMT_<3-HbzXwj_=s%pV?<56{(Kh0#1p9`jq z=RTEapWnoDa{ zri5luq1Ckq90P9-BhJu??$nFd!=Z;sUP6XiuFWi4YTH5H1jh3%_Q^1p)Ew>b;R_a< ze3?|%7ZWLorGuWtEJJ_HX(K7HEP7g>sOCvVxX-wG(cMeyGZj|44zW=>CVEA0!Pf6+%fe^tJs3!dqtH>WAy_FmsH{ndq}u?r(D)TC*L#F+*?a z$nGemiVq5w2zF5|-@z16)JxLW_iOAZ_bJ|0(h(qJSkDg9H_^((xsq1a%2ABNhDk}} zv0Yps)Vuq*Kdt1AG&65Kh246v6>o|Dv7 zsDedY3;Bew23vhK7q0_Zd1gZP9%c%OEj(w5dubSe1x%xF2DvegY-9d3sJLZFhvFpt z-69cK+tWNwvSKnHq=f`LM=*cAw#iA1e(g`J^533M(VK)0s$Y(>#pLkgEoLzS`WB>I~szL%Q-}B;&Lfd8kyOYSP7xQ^iuG41)od3 zoZQq2rl@V0jn!xhDkEBD^AV-;(wRi#&u^F%{U@dC(a93vHS?*vo&eq^~_dv zNXcDd^iL7oO9TQc3hZ<(1I!@ zoSpE|Te!W$FdFVS=L1zJip>{Mv$aMFjhBe2FI4F^$zt?8uOtb^n+USFByDStemXQ z-#Th#V`a7hv)K#KsW5R2YAEk1^ebWp0s0%uqgG}&-&xI{?x^VG$QG0m>;NeVwt%d0 zRV9jF_iA(r1+BW`iaSb}3)ast-LFNS3WEjJ1nU7CYTwbnKpm=Au0=QQ6}AhUP=@0> zTdli@<$Z6j%S&Rh!%fSYEg1^0#PUOz6YjS$x#RO7>O-)8?(lx(RG(%vt&!4#Xg>Rq z&qMwdh>0s^zC$Vccpc}AInYR8FSYJ0p;9w`sy)q`vPZU z4w!^%GhM~S*QitA5^R{ifME@FcBR$;DJ<_W{K{?z=m6+v^tDL1KbCklSJhg(oHj%*W{p3{MddPF$i2Epy9hRO zAJyb&sNH~D%T&#)!;uVqs(_qM%hpt*%l6FWPaZ0^zB@d3VEr^i!{A)!?YT6@d7j_x zv>&&TKQb3=0K*M=0OSJ6S%sKd-n?|p(xQE^{)uZ`8TU<&lW}*-1THPtD1K1P$~6Ac z_xf33$cKH4c`~Z#7^6%Xk^TDsD|>}|0t|&WN;;3ORw1r&J8xFqd2OzaN0*@GyIjMR z2wjeNi}*mrlFT8df#9pe*YAo5nKBw_8AUTV`B2uo)%QO>D&T@f&Z2fRt^IF$B^1Uw z$4y}a-d6m`qcnQYy^5}ytFb@JEL6Dm`4y^{2b}_;VP+=)YRo=IuRKJ7;YZ7PQs^Hv zgaJmfAX~yFE*5FI`Y;1x`At6*bR*RI%y*{I!0Dg>dAM+U&%{;#McFNu7a$}%b=wPe z?1(xo0gkcCISc~r`hVq8oX}8{3L4kykH)|ac(+E||Il^XroUVXxc#?;%zYc>jX6bE zFP^4WwUj91oiM{5{u6b}Cy%#V@{cuq3obT4iWvh(@Fg9KqhMduF1pwb*J=J)cap&> zX0~L@mCeR{gta(3g}!a`38Qds)Ca9(B~D4xT*g)d%ni}}vSwYH>Zh(hTjF^muB*n& z>NTWPU~ZeqTGVYccy@;pp{@fTM}{+V4J}HVyPuCN`>Q(Wpks0D=fWI50TPBwyay@! z=}zQ{P3L5MMgC^}B&=Sh%d8Y;kHvd~N7NeEDpRDeG8=I^-*i~E zuN}vH=G?Y_Yn%tH+69hppNTM*dnHoH_wk2V`;#ZxGPxbkjb1)U@34*wF}vM%y6-yb zE9F&|Oxj$U>>-sP#m~wFlcO`DDaIVCarZaDpxHf({~M_6aeO#~>m1Fo&E>_D>bXMW z*}Kwavli-*hz)nZl|X@(DJB*wZlio$>7J5>vXD!>cH_zfbVN$4(p*}7_mCk1Er-0` zB%~3O(Ip3?xF2y3(phzuA8dgf$>EM~kuTPZR}5dXCagDka*rcp%NwcteXqE>xVayc z;T(Vpv@E(siYThmjn5D+5>7gVjh*hggMXCN+@D2IsyTjH2M@zs-9yDQI$@i9n?F*a z6&Zckg8H=mwezu(S6mslarzg^nDS3JSV2&8J@j9HKFMy$m~jq;WY-J$hGK5*ESVBs zQTxH|kDRJ>5KmXKHA9~Z52HvWM-os{7(g=DCpbVrX^hX0Slq>G6^V!#&b{E;B(!Od zNQ}b!$;zBx{AP;lSwHpVA@j>kvkH<^w~|nHdTOm*cf#Pq;F#8cM2ZL^2K6zR8ygfK zcX5H!n(bsJAEPh3VO_ppG(33z4>JF?w#q8`%jzdGx_=t(weY^x)21(`;;%fB zZp61bXHPvibn>oa01k%rKSNy3%2+m;APoKUHqmc(4+7|j4b<14;{Y7dL4x_9Tu{OJ zXt|NYcR?ATB0>M5oN@0l{FBrX7yuB1(Mm^Yg~V*y~2&oMPt~KbkmqS0nf$38?G6ckPEU>E@g9=!q;XbUZ3T;c`o_ zv5~le%Y5jJKJ+L%RwH4l9+!nYS=5Io&4)tmyh={Pp)K>RP1b~(qJjEUIVqW|>S>zu zsgL(ZVsM)q_UyG_N3Zp6@7656fSG3p40ww-o0J@)tZ(s zw8Uj088Q=LdH+U&*(Iei7WqU6r{SR&MvutK4-|EDi>_#rIF4AgU(Fl+Rs7~i$gy82QAN0X;)X{*#Qk#$0jGj9zgFN!|(N5BkNPc=jf%qw>w9kFQSUNm%$|c>|D{)`bdJ&rDu+dlgUr>I|_{C z#)s@k$u4&jayE<9ri)GT9D`ICLA1;R@1PCz^Yj^_pJaPWMA=>9B?Z~!^lHMp-K#--T6`?N!W?k04~W%#WL|s! z^FU(mtIwsr0fD1AL!NOHVR8y{(#WrVl{^+Df`cni*v9fGwd`Cv15}ihd!Awlrp|At z2QFFdH(<>+ws=`ycTcWPK3KLj`rKsk&#+2mUCmrD!(?Zt{4)le*w%*V15Y)aJPI;l zkq{hOQO+Y_+hfwl*UCD#_E7m2y0E)AE$Oan_cu8bb-z`(bzbPkXgglN#-=<#Lq=p9 z6I4ECQdd&iUHYknv1X42GkCU!Hf0|im!;@Xg{`TvQp@t(Xw@#nSbz#e70qaI{LXVO zFSDQIplxvc8SAGaSPvH+U-ziB*E)Zv2;msg4No$kjbHz;{U>`$3XCn2@*-=4DQ zFtN2;e)3PH?|x;2o^_x*s_`I|b*uTCb@KDqG@g1YCSe_dRJYU?(AEjJFS?f>sDpFu zTehRF4MEiY4*C+g7b$UVd92`_irn1Wv;BG7_if7Ov{ZE5d`4t4_;`;A7#%Y}TdCV1 zL8%h9g8uiCT=N$@jFE(A*sv=eCNw|MnE5njMdz&DG%H2DIRAtpRjU^ECQ-oi$#MU$ z04?L^^r|g;(zOV-vCA9 zwt-J!uOK|}(CoOKj+07AcaN~4`r`KMKUI_UswS~PF@dB>l08-%D(D%%GE`I^=wY#4 zYDF=qS+sw)L$k}#PfcUl1^=kZmD_-NR5nmfey)vAF8@DqehIK^50_M2Dt9GMp}RiJIzBouaEbc5RQ!r(N$D z8fbfNR-8y*wd)cS=FH}8a8~1)yZkPVp_De)8>YALF05?JR!jI*g?-7Iy4-9m>2~$} zsLxzxNRI~0Ezyifys*PVLQG;ZxIS-~?m z+nWe_2dX@*40?@Jy18}p=7>qeUzQ$suUowffY9xB`op3>B*ukrb`K^vBcsoAZcRYG zn4OdA(&v2T3Z2)LoPw8UP4G!I4J66v4h5KOP!W6DT|+53KXYmXItf z%asp;|I-UVJy~M(*ppok^5mX{{X{2c^w3`4k6H9AS3-XP_Uqewv}ww2-!uVxid$wz zy7B^dA|;MJw`eDztX@fcZ1gT)6$-{Kr<)+YR0IP$_bL4!g(AID0uPWn@WH5t7s{+`@p)d{+6`D$Rj@PfqH?u?o?rLws1hizBaS>X|E3RPHee=;{rg}xV zt+uVNqFb}P;W_~??`^w9|U{et|dBK%PV{{-);UD^+zrPTR{jI{rP z)N7Nel*J1rZayvaR5dS$#bO4EQ6JcbkK8K|XrWXhk{Ho;H9MF@DJEF{)D}Rs)a#(- zgmSc47w=;sVW-GL2177be0<2tsXUsBZnrAjJMlC=0I!EUH{!Ny>-=EaA-Bp{wa1#G zMbR5*Mi9=p@1@1{3WQ-eYeKG)!Sl6Ew4BIfdYV7uf)5?O+5FEmCR2df`hAs1f$5)# zkiBy95U?KWIu)&JPPGLR;_grty44+9+?cCkzZo|(qGi~$ZZ%@ z4T^icPPQ@}A$X`#_(zKB+ZR1r8oCYCVe_GWi@*9ty zqN1LeI%IW+^xm58iqva)7D*)c9(rjY=YzTb3>!96&CE^pqm7N?Hjvp^bl>Y#$3;44xrH_0a#S#pkRw%?&67Q|j2*t|qUF2N-?7=-W( zv?qJr!=D4Aor=+{uG>q|;o}}y>d{8O&RlP3gnG-=!GKeqdf{{?suVGmIvhzEuH4=3 zkOW$C|F)+4`MWh8#0jxm&ZLqoc}-{94lA4~eD}_&Q1PPTOemvq+I|9PITu{dhGrOL z8NHY^2OSiAg|uFkIcNDw=3=IB3+TvRMNLgIoK02FbH|qpMhS`vzFSN@#*N`2p@;0a zTe$7^c&7}=>`m%l>UZ~3A~+>Q4D}1o_WB&MiayH_+$Z=%@G}Sp8ohza8*{SgI?qU? zoYiW~2ZXwo>{FjDUunu^Pmqc`o8lfsDa845G;8TO1@ zg(^WNs^q#PJY9%RUb^J%*$RvF*n>=0$$A$vA~>P6)lOY2LPk2`>j}B5xY)?s9W+stYpp$WEDCqG~o`RI>&=pU9|dV$@FL3=aim zSNCeMRwy^8z{6>pLM^=l{`A(@DBsntoxQeL4oj6J;l*DIH{EfaSiyQ5IJEA$i~sir zAU0DPsP}Vm#3ys@kJbk(EBsItS4>hor+6XXMnGDDTv#WV3iMVv1^x9uW-Q+65I+Pz zq?v!2BhG@X*rHUX_~6RTS2l}{)PdIvsS=80`)=Hkd)x;qyqh>%vzEO}K__x|L4i>^y9%Cr;0$_5I5WGfbqj z)!0~WYvPF*L++9joq;iM!7;(Dgf2-tTyHpCVTU?8ktNbVVV!qbD(8L5`Te-b231V{ z&|fvHFISc^#!_Cg4l~(d9xr5nD>W^rhcSul%Pg7X0$1|XX!uNp&&|QnR&@+=ACpdH zzIGafuw+_zkOJW@d=B+j&pJy|U-CwA=FxXLlhv>f@0I>o%JHRr)op*)Vo=K8wW&RY zbU{&(3?GfBI`hqEMoJq6cC%|2hwz9is`nT?Eu>tik9zvnWqy9=#Ysi&Rp97U6x_1`bbvgcdS9po&I43=LAwB z&;T^|s_k+l_6F+y5|E|nh0BXCzm#V`BN9;3E!3_@89S6`u3@pAK0u-PcPV-^OaP7{ zF|jpyp=MmbjD7Lr*>t0~&}Xn{G(6=CV@*~_I?D1PE%Qm~+{!WSiL=}h0Dhd!eUlri zm)|D z=#s7Rx2jB?2=h%srpcG<{9kmqJ>yK^%iIveUwx^bObMPKOpfd!UDhdX}6=>PJS+V($^msQ)f(1GTcQge9?*{ePBHKHz$mWRV!oG@<04}GT+G`vp)hlP&0kHU3a?l;qMo8vyIzECv- zueN+t)C%q1t*WUaFuq5SqE6wW4AxG=5nM-`w!(Jk`N@;l5=i(}#Omf&bk$F1vA)N*isqMjU__a##!H;h3a-Mr5pS zXj8erYqCFTCdo<%Wo^gK8$C;#EVwm%$ zW4YDS=?cq=a!R7V1kiuXChl-MV?Tqs1bYG$kcxk1kA!f$@xQ4fH5ipYk(G{u51{Y3 z#J6CtN8LU%U*3^N9i_~q8<3ShDSigt>O-0gab(Ap_AiX(06uDD3wH@OVN>xo?MtXS zv{T`QV<=8x(Uq*%Emfn4^u$`8eJDi%?Msxsok9P?j7=wj-U!-Fb~+gDtWIh&MqfU4 z#n_acnAYIw`ob^_3D30-7}TkYJtn4K{`l9UxNHt|umFZIu&fU(%V7oOjLn1VX+23QWonxu(G>W~VJD#JwDK z>F(UDfuyZB8#MC2ECgVTuYiTX72i;lc17l53~VIfr=0^LP0a@DNQ5qAt)oBmx=-k- zIplpYWHtY)X9X_B7(z*WoM;TZ0K3s7A1gQiEU?4J|6c!lt72?}EMNs%6IYx;d>paU zs|>p^|7>EtNuAa0$T(;4lw<$$$+DiUm$*%lNW&=Ix~g7wz(`d{Z~VZuo2MwN*{#~6 z16L`|`Cff6EY-lgjL$G+N^e*K%!ET%n)`j1>;!z_PXxa3JD%5Cw3id2=6WxmEn?}D zLhtEOU6Plkz7H*7=|^{Q-Kx4Jt@QI7V)3QL7I~sUV%AceZp-P@*i)d6K7IAJ34SWi z*1V9=RA{`n&+WxKK)Bz$J|BXfxsG9!Sov4Wv3hPN>hp+C1e5bxU&7_l(>k=fQ%(_w zr4mo4(HW1`^=gMgM1=v!av_pXPicERxs2Pjm}#v;MF%eC=zG8+OpB03iRBjmQ6D?K z3VKpkck?ZCvw_@dk}gc7zZnPt%mPy62z6`>6*Kl|HLRNe8>?h%K6Tk6!fN`G9RQok z{~}KQUq7i^-LpMF65Q(NIwEf-n>raDR&Y~%CorraQ`GjB@b>YGH(w_HIP}+*Kkf!f+1$fuRLtq!B$BIv~9c(FHu&D@v=5yX%exgci#Z zz>p^29NGPu?Ukv8WOfaEh%;yT)U;?CCQ70x{4Lte^ zkNJpVHdBdreKy^W>&VFQ@zs+xv+lzm0zV|H4^Sjdwu*cC-(O6(r32V)LVedQvnrp^ z;0k+3Vuk4$v}P-1uEQs>6zdcT*}x_d+8l!94e~U2eeNI@zp2(gTq#Z|V}1^TQGvqD z(UF9B`Cavw$%`n%p`}gw`nvY;RCYPrbwIiWN%LZVj#-&a@edVEE*)+RTlUNK&Vn&k z^d5$-xMwV1Ha@zZzst&BGMui>&ABJAV}KrLIFX9m`2tZzU$SGyYex*- zHtEj2Y;~>k$m?x4p$_Ww_F_iu>WB)4!J-hrB}v{Ka1oDKS>*Sv&tNzVN0Qd434~Zq zMAwLhXc#nA^G&ipVTF=Z7h>3FIKo^xJ6&0nk9;PNbjxSFb;tBGck`RUyF(x9=&j-R z8$23D4{=-p`B|U;a_@L+ul~3%!)0e;i-d5K)TkFo&FIaTc>;d-?&Q`mcw+s`nSce_ zua`wXp9Y`qhLyn{u)cl#b>?RCmX(*K1F`>U&~7~bDmBy*ovQ!!*ss>P!^+;ZmjZk0 z`M4tf;V?u58%(q{_LT-J`jn>DJarnc|wa(~@jD%F{?3ovk>~~A%@_9Oa z8eeX)N5927EF6oQ&JI%S#wDe-HpAqun#TbNbgHKf1)7@$f`Rzc&12k#9~>2JC&#oL z8!o|1zw2t*4icw#x2Kn$8^~#z|3wpVIxxG1u6vRA?<2Xz}8Qs)~`mnz?DPZ6PtZVkASeB9A;74kUL}L`tm~km&CGrDsB*j96X6@^JPHV-KT3Sx|rM9pD&AUY%bj}`dD6vON#zl z&4|FpH(S1NnzY*Bi=aO&oFNRQ7 zH|T1+G__kzMmEMJ=od3A6~`wp<&by&YIuqDSWyl2EJrva%2!6m#6bj;y@dMJDZ0>|y(KfGM3Q0vmG*N@*9NR%ryTdnD>S+Fpx ze(3#Vka7vGGt%dk+8Z|hN0+1OL-?myO@shei?!7}5%9MC%Cq3{#j5^5*NhUcg}on< z`gg-}+^aU^d(N@?jU##=NGhQGLJpnfH%;hl+>bxpRJ7;f5tu3dCU#3AD^aJ?vTcO* zolC_r6bNOn6NSqsmpv*z`D^pn$MO%g+;cgF9BGD~DG?)dHnn)YKXzS${P6IO!AVh5yZvkBM*lHTYQH&g(`tUx)%%2I^ew zCsF73sdq~UzAM#EZ+stcbnw2>(3-;vD|chjU!uoI=H~9wd>)HpH4asX#Mm(+Q$1%PFi>9%y z6FQ#X$)!8})qr%wNpo{2V5!lD-BqT&MH!J4=;)Q>k&!&rK@}%zb_aIk)QBI4IGH3% zuI%+{lqzQZi`RP#2?}NAj<*?G5--pb47`z+%CH0DQRwT+5n^@Muf`kM!0@{Lz~pQ6 zMXsp-{_qk%aHsD9@zbp{$M%Mypg_ey(qXSW`7@qfsT}@1&OEP^dhh4?10@x!X7fG2 z)mu3(y2@|zzUSyU_@936H+lSXD@razdv@V+{@uS$*4^*7`5DPD9jFD_9(Uj7zi;{n z+@~3EEm$AD5@qaHkgR!0z}FZYOfE6CkuVHv0*9FP9!2YJF(?$SfROyN zv$=o&f4tB4eHqEraQ}6~qZpD;R<*19%pb}BHRLKk$N<<++%oI2O)6yCnf~Kc`ERr6 z`o)tSIqRRf(b617Cpm^uZ3D|@+Pdm@Qs3(@i`Ruh|sB- zdQE@Wn@9KNUdD2YKQW3J5yvYXkjV2n(>#*aa`)_z-O+!X4E{bTO&LDtLD2{c_21l` z|8o11g(6;?d~A5{{Z9kpe>+x6jsZ7%_$lnKf1a!U>pcs>a9z%zqclT4i2d$i{-@Ke z(BhC8xFa^GczW;V{tL$pzbtTnWQS+o`}~X7**3<9I_vJOQ~cr|-je^@zcNYKdv)4G z7B~E>&&1Oh7(>;wIXC`=EByO*NU9&;U(|yh%lq~8zyEgs--r4CgNHd=tteL*)z%4_ z87{S*oS3k4SMwVC^j=Ii#khglC@s=xiFcd2@IBnH-=zil$3pX0*rVyvubl2AuiA#ufPuW%H+EJR%_n=@ z7&{WSo1VAc1j6RhcMpFh^z91#9}W1K*L#zl@%Nnf1Cghk0@#bV?8$)vv)mlHt0zvJ zsBdIVikUf$WAZa{ez>{2ExA%_yhd*%7Z(?QP(mpeeg zja+V_+10X2&(yp#Y29uP{Hc$f#U_er@{$TAq@>EXs`-wrja8;3lZbz|eHptS6jZbE zdKmxd`;BoA*{ac=n%b?;&03n8x5j-RKKwK#MXtTX=$Ry8FqZ`@6TBrFw-P=>_FoaN#gw9o(lU=nLHzUbtt!REcxWfje5<9$G`8=s>)WZ#0U zm(4i&j8NlkTZ>?jeRg-Y*XwG{{JG#5!(g&q@b>2VqdQrlTN~erbP|5#TCf0_ri*0y z)xt=FQ(kGv3AxpmdwApbN(5a(@&s8mr}g>@0KVSES>NfKYz+3**3znZTZsy=$i490 zzTx3}>~yuKb+xTQ24C7l`9+0`t8HP32piX}jg>%1iC^oXBOh!%D>8dLA*-O%_jw4pHoR@8V`>p*V(oW6U1&-L2SB|ipXLnUy$ z(dE;4eJR&k+@CSR63+jFvagP7@_qOJh)5{P1_CM#3P_{U1EefO1f;v986hxQx&&0Z z28c*^#{fyCyT>F3QX3;i%)#&Bd(Qct*U9ha{3l~`dmirH_jO(G0zPqTvxIs#q((xz z=SkD$=6a9$7fm7m?auz6AB3e)Sdwj<*$qyh18Ia^{Fl2TG4X1$g&m+T3_(fjSpltt z6Tl3G(=0af0NO^ZBWza>+ci~N4{H;sSs*h+Ae^$W7cj#yDY6M~P?A-W=A&B!6@f1E zU;#mwra-f}iB|w%Z05td@gz)Z&afR5{j87+#6u;YlJx#5f0|JkXX5<5kEZgkhd%~Lr_I85na+Jpo_t& zPTJpKA6oKTL8`@JFvgIYT%S{%BeD!9-I2j3{SDBUQ`06SSd z#7u-rg1qJTW-9jUfs~i7<3b@icUDzMvlrw~^jS8RyQ8%d#+5|-mb*J^yMz8XLSUen z7C=BD8Y$HV^rMsb`R@51B9#&qgc>ZH!E5Xj0{o4deE#5in`M0P4iQ#xdSJ%{Spd+n zxEAS?8Z*urTjLPFa(66z3Xbvna>uXXNOBZoV4VGRyho^IR{;^% z?JgHUJe)>~qx?1ZbvKDkh7z7h@TSSk<_EKWe+DNpo#ka(E<-29npTBHIaO4XuMZz85 z1PeR5M!Q2Q+R_VZPJwVovDewM7^De@w|7_~BE1%!DW1)J78~Rv6Cn%yiGV4pi!q{A z|1zCTpZswYcnQnbpfjRS&~6|5{%EVb&G;lb9z;SN_5N(#cv?;XFi}u6>|~5ZV<-rx zS+vxBQ)jM#@k3%>@wcXU6O)}YR9nVQD?(y+XTCAdXR8|lv{aPV=Vi-U#jqBz)2qSB z1{0H%->tF%)r8*zwHR~o#zbC8)L*@ywf^R{763)wy1MPg#C|L7Eu@rws3Z{3>UTKP zK^qp6f32S{^$MjSotABiR(BVSh1a#m`bZLA12e-{FU3d!5Q(^-2cGcqP^Y9#r!5dVQ zqbGT!UMqQi!U=6|+ue1w4U+`Pxce_ha`RFXwbFmLIk=k?PJ3S! z=dx}d?%UJC2DsNsyW^jT#;~o|T6?%ZrFLlJ9JsHdLm48O)GXDo{+4dVut5x|DxuRQ znGAjD8RvLzfP99AO@5NA99}To$Q^e>Zd1cOV6(Hp;poW9i1GLk^QK(7k7v+mTs5gd z$LUq_LEd0gFEiZB(7xZ+kx%U61f3CCh;dk)NOZ728&iEb?9qpks%l(F_H;<$_o}ci zSh$v}Y*T~aFRk^Ne`fSnCB2^elk3F(+Ju8=b^dmeQ-Wvq^79`LjNH1}+B{?9ZVoGW z-Wp8u=G}T*cvM?e*{fmN2TDdrzsT&3B69A1Qdjsp$8p$QplAtmsJn$=p>$wI&M`|k zXJ1u_xTVj=#zwOwA1)DS(|%V*J~$FayQev>sioxndnAV!1CAR}YQnY>%A&*~s9vs# ztB3sj1EdGHc`pjjqC&3ji5EgoxL7=4bL@ZFpofy7w67jDC+0#r*lj6Z@CPavR5Z>U z!N4FHfS88_Y@#$Bxk&V1y!Je)JO@NuC5ws?;0)`_xXUgy%>uKEC3gq7Lq5j74>P0UGSTQnQ_o)XWNSkH2jV&s|+xjnW_ zl;*^~MDnU_%8AJ-j&zUea}UfQ}*LZxjZ zD=FMn4lHTr_2(EDkt44=*vUe^FmwCrzmV~9fK9tKejt22R)$YJ>WIO3g+wYYH?UsBcFpI}&sb;eRo zsn#>aH;P18ck^!)uf!N|1EsugY_}<_BFu-3Qa8`u?6ql~lB-EiN9!v4`*=ERz-1Z` zK4+0T$Y`}4{zXVPpxIC!Y8{OYp|Po4{+AKGbacuIO&45$d(f z1`Zr>uiv?Rk+x`duDxPLLBz|nII5S6i4qjx`2&(B5%y6tI-IYAPst^!#o)G#~tj;i^GmzPN4%tT)uwu@nU!()D0@OJU6??*+Rw!g&Q0w(&s1E}!Dm6ds`fY?KIhx-#Si%vAkp`R z(F@#j(=vs~#EV4QzKxkmi^&@ux83!eM(~@eyPewZ3Hguz`U>&u^72za8BpgulBer8 z6zWLRR8~^><60A86wADz-7OeLt(#<55TOF|n$;$i&WqL=?ME{{?A{u9efYRD{^hlY z${@E=3JnT7*dY5v;~U791UywBD6?M^nfTeGiNYIy)8$$YHSQWz+&U_y`+*+T-=$I7 zNa;b#9%u%1>g?!NHp2_$U_83>=|-D*3*J{NX-3NB^u-}u@s9%OS0{$Dq`j#2u*U^a z?_R(sEnfZXggb{lWmQO{%@^{_q8AxHQhw9rZ_aqK`(@wgXS6OQIaLDD0nC6)P%PLt z0^dZOqZ4__PQOvS6QnO!P*Pbo7Vt6>^5~}yXSlJ$(*c+6;GX_lt)Un_Fp?6}_UWE; zfB9ZF6c+M6TL5Pnlx5Q%B7Nx+RG^DJgaTrnQUc8Z3P9tu_unXCtD{N}x;O)fTPDqb zH{+Yp5Q)7v-}WM(!Is(odM#^qIfrC4&No4l2le*pyg0i&Ejvi!N4jp1-=IG(!Zw19 zo`x30`G_780ty4ESCmJ-iO`G)=&A>oIR`$=S-Jz+XWv5SUUM)q?B4N+@b~`qjPeZG%k^WyurqS#O2_-s6V1t-RE72zM z4Fx!joG~cSIF6h2&gNt190+d0vGs9@g0FKC?3t~~EwfPCPeC=RmVtEgq=ZTAk58Pn zPP&Ia@i+th1&XXyV81eXBTr_K@O?bSzU#UF3VQ(jg>!3Bm+AZ{bchhR5e+UxjJj@YDe4jav~(|!KnE10>gr>qMu8}%k_pP$H{%Eg{G z!345^eeblTUCe7st;6V*)&mVSfepEsM&H?7Wcuq@nGkUpsmAo0davIXh#y9tJc!;L zOp>-XYIRg6pvBbbHfSFBF;75Fl&9=oA*^XuBSa`J`Cdi@EW(09zg2)Fk-DvQD)T-G zo%t>E#Qf;&u!e8mrT3>PO`k;RhDxj*k6UfzbCj7eR8j{HCU-x)ffe(qi<@|qglwv& z(gv^J9DYRG%{l1&+pG{3O)=Q&0g&1azIILEe8Td$jw*zBGhISC~|KoiS1v(@tY`r47|pECFd%Z z4R9m2qvHeV>Yi-?J_Kz=o?xo=?0tDLY%^>y zwm!3m(iPu{v({NV{=iPF^C_Zf<%obfZ;g89Tm!~MsC?*3lFy+`OW-u7Wv;|Bv>hvZ z0@bfR`Qe0+gVNo(a2W^gG5vD0Jfa#hLakz2Z0|ey(S_Yd(k<>vqB%K-7FV5pYA6#O~YsX zJSj^!txmX&yU?*V0*AD7XC;6bAUbi-nJQb`jAb?V9wc@6x{ z_u}H(8nv&;kv>vVf+CJ}!BY#kZx40m2)_fRNwT1Vue8-)3vrp}P`5TsW%7PuuG=Fy zh+aSP*tDDFGj@l2`5MqIftXe4&4yFWdjprB+q6vBmcSTa{8w1!zdkaf3&1V2x;u3C z;(uM@xCGaUcmey&-ohrqZZ5Jkq)DlMW9hwx0?RTuJeGETPBJFy3G1VB6Fk+P*2wYoN5WsJCIh`qi!KL#GG=x@Cebd}{fZ1B5|56A)Zv{Dg1LrYn>@`)oG zJYj^WN%xMNPr57Zv8@~P@VDGtS!C&Ip07ExRfS6CWT^?yBq>v$YYV*d#c9^*FH9?& z)c42jis^RbIkj@J5)GaUe55*%!2@#NNLSdT7dla==!dd&KRFh?$h~I~y>~!PG-89h z2-(gjS+@xU`!}yg>{IOPn|uyGJwamJ@#e0FbCZ2Ps@`so)U(>~#3sPJD<|H}PCf={ zq7_s|sGci2-z!|v*x}8vb-yEMp1^EH$-@GTrFQ}7sAkuf1{X@JwoSnK4X?5_>N!4} z_3MHQA&7GRI59E9@x405nFo>&b*?-Dnq5zEZm*Nn;*LB0I*FE^!88fg)vH8boJXhT z@KDt@{jf~O;m=gwutxnWGTq~<_mhA0!jItBtSI@vq**9~l%Ba2M0k_BBXz_`GPqx4 zQUn!Q=dkvziG(w-8oy^j$>=Bd*ck_NHEGm;p3*MTAdsnphJmT+o=&)m*WWt|<1UWE z@pH0!TKyS{*K5~8c$CDJY572R zLAK8n`Ewz?q*Mrx7oO<7M13`m$ewqtuJqC-`vm#B-8Yu6AAWz1O#A(%9=>hc!As!c zz5ON$KPuM6u&4c$@lw2p2G@-)t}gj5Zmovbr$3%h@a+U_64A*fi;e+~swK(=hs`J= zrtB{=6A~U=W#u&Y0i)wIYlM}t&rU6GA7Du`FY|4_RUL?yW z3auaPq%K>99V;=g@}}x87F}6%#WSeK#4xa!r|zB8h-tY*HLP^23{0_-KHLNIGagQ< z5SsY9^d87qy^*uhE#Q1{oR1Y+7cLEE5#Vf)s!u5|+Aa|By z-U80m$jEDyogz(;BuAy2miiKi@dmxi;MAu!ugdTJLbhJN8l4LHdDCeoP-EH*<*J_R zeN~1t1?4RX0o)5q2ae#C3cscb=>W%~Z%dWj|A4yx$!yki1sEqLP~c$J1@N0v><@8I3Xl3aU~G!d>$=?e_gAvjk^ zWdS)mRba4h1T*P+Bv}M~gnL?hP(DNN{*`G85bQC#gP9ubeg!jkj@JiuweMcI^w+uY zmhf}a_Y4Lz6Cqp@h1AU<9>zw@(fuB_3`O||F{0_+$*-hm=O^aJo|Kh_KV};VviLAj zdF0#NI(8Nw>Mf4h<5C#KLsUdBMZr%00+6N~23^b(0kev4lnrZrhA~dQ+ppS2Jq6@SMQt~~?&7Dlq7)2Dlg9YjC3&86LcR7rDuN~; z=o$*rhh5J&JJX{oev_BW=e<@@1z(yq(6%E?wJ{t#6N0+W<`^qG2j5Po48eRAQut!N zqJ+a+*p>|{3whc0qkJB6{=O}s5*Cx3H;$=*CRwrm4dlLJ6u|kbNA>yfWj-Wbq+KKs zrx{of`-dg)3h@Y4j+b@{6b_H*858ZIbM?BLiIpMH(E0fdzu7S|Zz5)-DW ze#P^5S-4k^T27$lHHmY!vtkNapE3u7pF?f?<)-iJ~*y=a`oJ};tAD$?0KKIGge zriBRT>yB)8QxuXOu$TC;v)oq0~;WDlD%H(Q1QzLnB65>bXe4 zam)@Cm59lwncks1u|C~20!Z`2yLI2QKr4V3_$GiyYIW;DsF-1Y9+$@9N3;jsh7|DS zYohKp2K_oR>5mJ%9&@l&ZWstG*q>_#oGeXaz`b*LpquiP>9mP^8z-BcwZ)2|#u0*B zW*prHhUx&m7C|Y?l8+rMJ;V8XCtU^rpYay$>SN<)E*1!>lr%^<&#et4mwo6Vl!Dw_% zOATTW4eF)Qk7E!4SJI^WTqfWWJ%E18eFgD%_xEEQ|9VfcBL9s*0NlDOt0s|DxQyuO z@}>1psSy)VBL&t%0>_y`2dymyBL!6U?0BUa?!4*(jBwM)*(v3Ju$~|H#)>nuyi0qn zB)Kdo`V0t6ZRCBxE^PC?(kd`Qt&9DQsT?);wOMQ$EQC_k`Fr&6HKCRf1d!&Um5@y( zw$JkgzS)WCWesS-B-SpxXT*fi#G=2G@kg-UVeQ?YjAt%Id^PxS!-cYf-{k9{*Guy@ z?+bci`J(PuIW1CYNN&NFKB)qQqpAHD3iPIC7pqvESe)Fkw9}F?A?rb;>pbC4lpPhZnEf&)vAOy9g&$oI&??v)qKV7dMJOi^m(20LjI}_ixGTLe9EXv z4C&`-TSeGz-U5cx$Z^#~cQ}P9Y-;1W>SMRU2=mrf)55|Hj45~!kUQsfaS!P||A^@u z+}OOgK5rhb5rF$+fh<IDJYLa{QW-?rVnlxgK6@LIoW4Bx#AHSE zxB312X*ZC1tgM7y!|tkE8UM=ROl*~su_P*~x2kIr?uv=*?P*$v&x|6@2Ff@_W=}o2 z_8uLa%x-BNE4Bxp8wC!dx(DL^GcNvLI|_=x!&jy!)e9(g(>wp>1+ay9QYl0_MR|#X zyn~6BNP3IQ-cbN_d-tOmr>^d^?a{Gk(2g@8uA_j2$vh+ zerx~?!pYbv_&%kQeyT35IoP5gyk%z4|3dt&rFl9O^2A@2Z3!l?*p5oKECn^*^yf{z7vVzOh=vMtiG>hc)^|<=a^#h+Y9PR zTHg3ycwVIw)Np>&__~t^tG{0@|Df1}OskuHy~T&=64&#@^T9+ZLxiNJvPhP{4}#Fj z&uw*`d(q*>4)az|RE1Su2UAei&kgBC%QCeDtI-#&1P`Z;8w~10IfxZ^JsySI9U1oM zQ(&sB40!_&><*5pslTiHoCgHapYhL*un`2Fd19~W9$A)49wl7K|5UjqH)D(TAu<`2 zoJShX^Luk}wlC~K{x_j7cQ?uzpi;+?ZIyh1_BJyqyO z`0-at(}=!r?sFPJJLC%1u+5Dwo zIuUdrFbFiX{$Gfbsn3+pIj)Y8SDXTJHC~;PL?M<>0|45!QtU)TJDct+$hT=zGmfg#mT#;dACa#c#66@v2*b zBfRO|hs7DC}S||LbJ$1E-Aom8EC2 zpLcr{W=tuWt$ym9ggoRriVRJB|+4%PcnAv+e%oiChDzLg!pc z9CaTVaF@SdU5axnqy6@1n4OgVSkL2y%&z8eXwV~v9`fk5#<1gHif+-9-3Lm$E?+5> zcR8Fm#74!GIQu>&!Jah{2}jcnn~k@6jRQlaz_)&?oF!>@d;l7!D+X4D^)WnT(Nj<9 zi?&{rEp>k5x>e0|n~@@niPm{3*YVWxBi{Iwb**nQfHFgPG@!CqIjh5%SXs3$@Fc7I zC97@;Q3MDW8!KIv9i7^I{%E4hV$bTDwkia@ko`VgwM~P2XMo;MkL8&9YF|wxXIa%D zEq}$Pp9j^8fVSZnIyE7+?c01`5}1*E;hrAc;&g#}xEK3p$vQPL z6mH)rJY|nJH<9;BjQ#Dbj|Nuen-Er%`Jqf{GQpzr}5SAcHZ}X(?5EkMjT5Dxc990re1VDbL%=Y@#>(q>W!hveL3K^zBSt%jY zx6z<${GuP6ZkI08AF@%|c=csH4vYwDK%cBThY4?&}0h>SPqN_?MXwq&0z-=>Tk1e14GvHW!dkj@{#!I3)!-G)HbM6hReBKSRjA?-?>izZEk3 z^pVns{fX8}-XCU~UfYvbtG9G3o8QIrZZ7%#MvUt;$uVFRlBzmpr;={?6vFO%7xABK zC0~se&KFhu8(@1y&c!;)N#pBrx=Rwmr9=U#2lX7QW{+}B$1zE2aQ~vD@D61OVONrd zc!KUVyy{Yzc}!&yUq`9;a*@{K$NVbdrPS-%Bi4{S<-9Z}MZgBy5e}%y6(sJG2}ZxY@IM*fFpK&IIQEp{LccaSP&E z&H&O5^QIapp@6x()qJ(OR|wlS=dahE_<`Qa+G5C+ZGPJBq0kU@QP&gdwWlx6M|GS( z)lKR(&Lu{KIG$b}78yd>pLZ&udbEu9Zt`b|Mo^o@!X#cqH=l)TZpn?=GpwJ68t@_1 zCQLGGMb%S0K>Bi1PMa0Frsenv)`?4QVgmGC>iJXFhvezMB$pv?_pHNP@3#7vA|cS3 zhB>o>2&1|LN2s1h;_%~(%d z`-Le*?yEUrzeSTXy5vViJUI!GPx?Tk=H0L6C}f^~)}F+TcPQn^Ev#ai{LZm=dtM_+ zn9toq6&aB*b7qBeaw~+jrrx)@Zy*Kt5=5qyp`rJCQxd+qm-}{nOOJ0ZmVsb0+N88( zs3UuaHZu!m&K;KO>nCPU{uY-t=DDe6>T2`D_&SlDxJmX34CQWyS!(uL>Q|&IDGk^8 zj*s#cB^pwW7E4ef$T6iw%nYgkKMr~==Scwr&LES;35nQxIsGxH%#n|EJDqewY6*=2E;n*nd@w)AWzz2Kmlx!0{NmvE0( zbrF*}mDLPgy>D9k?x4gL9T`#Dsnp9n%>H60FHkUB>)vZuJPNq+NuM-4y*9*4hYN~P z%Z2@R2Xfo1oNZG9m7~<&jm23F4mOO<2h8qtZAi7v`>_NP>67`8Mb{*sEYlDHsHg*N z!uWigf#jk2aKMaBe5;p=xg6m@hUk-%-DbylI3aj?rSfAoVpY|*lcnUq3?l5fkWi~D zd2P00q}HImZ$F#D*@>Qg2+aX$)J}3vOA$1#>4PrX`@Y_N_E5H|G3t%S2RcG@IAfdF zC-yZkmr9z-bbj+s4sURT5v*f_VO^KE?|~kvn37b8keIR8*ES6Ywm$Y2b+PY*x<7#k zk?^kc%w{cRp=H_@&`t3Av0o`{*lawIdTfEEf^>wOh3;(FwAXfeO%;c?mMrpRwrlUWqAU{JfJgKsyL0ki;J}~02DAYn>y#St!nQZ^{YWZuCeY;{ z>95;e`h7(ApkG**Z<>>M;jbo=rBttKIwT5x3S%>YkVW!#vAez=re}DHFL`v}8XrKr z_9Iz<`awl$q)tuoNEcwGdB+lqz{`m}!X^B18Ku@vA4bec7GNZ&wS=4^h1u72YX?O{ch}Qrt=dg;Rw74yiDgT#o8y`=Kdmv0~vK)QpyS z_Nz+Qm^h8@9}oYzK``^I)@t0&9Da+t(@TG3+|7` zVc2(BV*vmOJKwO^@itF>KW%o8UZIha_<9u2dN4=J$)9G%?&iTYOew>W$1tR|)O&Kt zY<(a!9Z5i+GFU_Du`=D9YTy+}EwsgZZ$qe_hBxdwL+0XZGoDB4>H$D1?LXJFfo4)J z@W+%L`eMajf`MEm4LabTNUC+o29u!AghXgDnQUEm2XPtJwpFi-pl@ou(?m4uZt-lz7YuTxakd|;@gZ_m6im(I}8#?ahq=o9&tKT((a8XP94n|gg|4rQbHv<`5u0KLrZGDxNqxu?m2qrQ@{;S$^FRN*g#bn! zYHTWcrRQQ%q5r>B+-?OIkF(Fw=6G{cuut7o9s)-Dw@_PH4Mms5Q10a7ziay1uTp-9b->`UcZ&MW z28WQF9v`SZt%uZZRt~)rf9Tm9quUGu~_W$3e|M4Fc6c^&oO9}n|<1_xJy1ashM3L`#!lVE53qd5; zXaVHPOt$gP-zfS2-(lCbBjDLU2qu}txZ7pq@#$KZ(_k_wI%yu7$f-~f5srUba{sd+l*RQz7P)#oWAk4= zUx^$EsYJaKvMB5@%Edd5_3LDMSd^m7ycFHK14mz@%(QzwR@nDr30WebgZdV0wWXEv z@Xg$m_7&Y=-j8m7=k63=h)}c?Q$PGGDr)Lag80JN>lpv#Xzq_>2{LhBNpWkg_^R2f zN|(&5ZcQ8m>_fJ2x2ZC8k~WRHZp#{fjF$O7znSd_8bG&TOv&Z2h>5+uvZ)!37#VNHVog<=Bz4SI64K0ajq&oDv8m;{}kTm*)Y&SDnX&@M~kX5%XuV5aaM% zNJt?wFC-Wcf|WDR`sw_tPIwgg&*qU^pMXo;s?;*q{Fh5qlxKsxvP8Xoc~9ZjC$4sb zp@4v0zoThQousa%NEw6XEP(jK9N=wzG?tTzbkGAj!gWB>996 zH4O9Cjhlh5{#h;IV0rN^?3{Y|Nm^W7r#>|`waL?I{F^EgfUTRCmX@3>Ex)pqEACGZ zeC^o)WTEb2Ns4X`w;=h&bS*lDZYvG&|3UEUhw0HY+$NuX3+*zZKRrkKFw>AG(-H zFhitD=!8%_tVS2k>vme1uj8ONYEq&bV%B0;fZ;wurb?gTQ z@9pgVSaT{6zIAhL&B^nE{|;Gv$=om*DfBYYCx~Z#exLU6sFVik@y!=oW&3zWQdT2r zIE~TS3a1J0ZO!54=C)@BHU#~)baDhWL;s-k)RDM00=tZ7sWpka6x@tYdcr#QsD>9M zzSljv$n~pwr2d&u)?1zGK~Y0STY7Z~`Bb6sx;Yq~0WpFU z8ycrsC)S2R9#j=FvNG~=Y#iSIk&>Q@;er}&)j8hYgwM6*&)56-)LYC?oi4JyYcvWG zHh2<6M4#a{busmK)ra6lxya+G?^tFPy_h5kfe%Sl6z*`|MW}ijJ>7g*#Tb*SxWiZh#V%G4YvJX?P>+3u9Y4G^^BYCG-Tf7D zv`#SAwPRl8M&XG&K9-Uw+MQ;y{@K~tIoVb#2M0r^0K+1rN=&{#;=J`Y(PMKkx3=vJ z$F_rUw8mFj$@ul+AN2pvkT|4|bZY(C9_JhfSwXo`^Ava`WdY%ee&Knthi0X8^Vpsl zl>w5ekr-t2g*b5HCrNMzSr2R8;|j};D}388uWqB-+FWM=VQ!|dh>NhO%%rNC>h$wS zv0b$Z%U@DUqnxH4-IkxvR*akdiQaT&DKpNjV8lOUM8BpkGy5I?IoUtd`*o6CevK>S zz4e@E^8 z#Yseri$Rsdhs6(2_|^{I*^;;A4zKgnQDJ;VT{^(nl?~B3T==%!Z|#=KFGM)J^`vj# z$D!oqGXCXRIk$9vpY-}>qP>;t(Gal-?qJcsv;2KEl;e&*kSOR*J6_*Zt(>rbcqBf} z*km_s18&)#QSWLun93Kners3N>~;7;+*sDFZ@hXA0%>W4eY?#IhdR9(7A1H%Y*5(1 z@;2y3`L7&Las2RFYlj^v;UYK%u<{7Ad7y&qnys&V0cYo6;XqWk$~Mt+3<$34}&xw?gxxJw+Iox z$Ep`SL!g7fUC=` z!__YJR#cvK?$Mt`gwNJd1T?jKOeo>V&d~n6)dp`S&Ql<6;CSZTQJ*kaDTOTl%kFh* z@R03gKjcwKcBWtN9QS{yi~eIZSi(x8&Q+3CSMVhxPWH6mLO;CO^*!_&!=`0zBx(Hi zpkml1)5~oa5Ytf)BXoi|lqt=6{1~sF`)Fei(~R{1Vj-SeSud{wT3#f;ubG!QoX~WC zoROX5cq>Y=r5YZ>U_ZE{E;Ov9OrEHg?50fZEr zljD`RW&i}QA*`iC?MxVYL*ghK{-JSB4me2$7CJ=yB})+s3XpO!Z*a?FjBp^5;`EPm zTZ*Cg>kg)6x^%+q{xFd@-u*VhSOFaQyv?*;Je4gtL0}cA1PBB&ysO5h7gHuw{ zHik0V4CzIhkdAx`b^TUX3eS(XW-%|o-Aos!CIC($D=*I{3k=K}cR|XT7H1LogEA*5 zc;N#B`zz=z!H#kP<%);KE&4mRst-25Ibma^aX?GAnAr%cmOeSGb*h75B;EWEyLA@* zvI5S$h)G;*I2K&lyR)5oR2g>SUlzte>%Dio@I65;LH0gdI6p)GioYQ@8!q-J^Owx4 zo8x>pT^Ge4xM1w&R3`oaF~gHBk?QsI?0wa}v6EOVgE7o(G$8KTJ^%+!c)TVLu-pY) zlBNzw1*r8?a{VuqC-y|&bXY$1)m9bX=o9hO0F&b~DPUHMB!rEMPjP@8FN8zVZfh^; zvSa`^n%Zr^b|uKa@4rLW|E1u|yJ%Q8N-zd510ror=T-wLVdj?tM(sh!?4nC{!EMVY z#Xs=L>11`P-_rjQvmJq`EQl*k>-DI@-e1nSaw7$7{0Pd<^|P`j5;|fCx%|7ut0Xzgs_k+vI36rE>PDL+)v80gGlnR65HYqH+dck6tnHC0Dp83bO zpgovN_@VT)5kf>gM7$H(Z7<%8Ek|Nx&(`cbPkIcGba5$e&cjnqZ=Fv3?#B05qZC@~BNH$6-nzKf!ElX8N5S>7%B~_XDAV8&u}#jh03Xm=QL=^afuwm^Er^b=_Ura@RF9g8wc`I$BSF+XVO!H>(}WLy4H3Vpf$YHO zV-;(7!ONM_qjo2vuAKO7%e?|m%1AMnw~Z9T@rQRP&}jujh&fUvxJ7x}nYSbB^t8{e zFT3Cay)B7<5JFiVT~Zq|P+onYo}^7UTdu_{9L^&x^*RPfbf`HD(^EDs7*0fVbU)w| z&^qIXE6uxWUBqLprnmT_;hj}cdb&GDY}s8&P})jEaM#IzSm>ZmjN>QL7U z2<;L!Q|32{LrG!rlTs&MNnp+aK?cEbz{cWyx$ClHH*+zq_YTE)Lyov zQzZ?VR$5$Mbo@0B4Vblc)@2I#Y1Ut<||c&cogvAH1;2ucYL7~ zc1}qCNeP+d{*gc1w^)g*bh3nfHRkkxO>dB%{nV%{J{hup&B19W8?up@$2?Qq8jBuu z!l6}c&{En0h^;=2r?iUx{hHpkt5)&kJ_cUPwRJjAOKQ>|#WG$-gFGujWBTlHGh@p? z>klOldq~v0o=@9DtF<5=2U8|zV^z1GT2X8#Q%l#>%iP#xD*%ENgfb8I)Tgr9-ud7Rt@f%$`aR^7IOj6K0kEk4zA& zjv7DqJHN8u*ap=o`WP2vlkGR(b{5yBp9Kc)BU-*D$GMGNjY+LRw9I^y z^8>KY!`ZhQDkJ$BnykHJHmp`Z4W!(m=x5{eTyx!Fh-}rau-^?ZHWT*SvP@d}`fdyLo9_sG$;ijYx@%U~M@87wL8f)M25^54We+m9{jiOy zZ-C9Gy7>xL05#Kv1Uxv*>C|iNgYk>k_V#Yr4xgRD7OyN`UJPiBpCjZR5%0(0k;_C* z6L~2wm33yc-vj}(t(wnL<^!e4r0qHum-opkO!kEC>*rEolb#vh(+q1Udv2|1{pG%o zN5dk)@0U0sZv)L++Ejb6Tv6NR*sV#;PxRS-26jCpb02wo7L!YyDZ-keTcmy{p0wVZ z8EGvybZhmiKW@E!Aj20cRr@X$J(*6gi58U|e1fmVimXp`dww`WF*j%NRWMo&+W(eW5zme>->}&_W0n1GP$)u?Ge4p~!Z*lg;Jt}f8YRl!s+1#v(IAY(p_+hYE<+^Wk z*1TVZZu7?FYlpe=?wc~HXXKGk6o8?28hesUvx$98k z;J#w2LUZ|q!MSFgD-QIM)a=`!iT?@ulxmE-wjT(|HOLY@eR>U3dqLx_mAhZ;cnbjj z@eG@{C{B59%qg|irXGVQI%Rm6=O@++JJH_YUxoRqsqnboIM#Bn$b2FxF$RfYb^or= z{)BlFWo$B@PjfM_Gev+ZzD_1?(OU6w%vQalL*kfim*#XOvhta}WF+4JD3yNZK6Goo z7v$Yj-^yy(px*P{j3eXSlA{lJ*K@edxsdDpe;uTUO>Vmb&+9gfB~}a*vi); zmxwj^;rZGpfOpuZ>uCDZ-r$-{pJ2|k6V@2TvG}tO`9NxpnP+=pV1r;U9|F=#Y# zi}05svEApkkmKvdq^6-niN&SOHG`!|9h1_R2Ukw;WbRr|af#ELbIUbu4r5c><7EET zd4583#rfXF)MWnA+|)ZkB0k=2)<((NjG}QxbDE5jsIyWY`2HNguT*T_q}^Qh-$l%| z&g&aGoFC-NO-k{gJkvam2DzZ_d(7J?rapC@L;D>ejEWHA&3Fyg->V+W>9~xFAKPMV z6|I~iYb`w3k|!3=CjZ;a^S|~@5Ps4#f71WQ*Ly}awMK2bq7;>;8wG>_x>0aT7b&5N z4V9`0p(a!TLl1-!LJ<^|rXnB^nn)Ky4JA~O-a7=6Ae~S`4&pd+v{3sM&dq{26~R;R-n(Tm)}6^tVa!btcFXUM zk*i&ak0x0g^d7wIFSUqLB(w(ySQ z+!xrl4TD`H#?R5wx*nDA7Ay0^Bx^&={uct>23D{1uJE{Pd}+e%Gt(KZKnE(mx?^1K z?S?3FF#N*~4gkfJX<`4lFD!;<-b%o^O(c()zOr^Nw1c|WO$;%+xU8jKh4ok7>9TPT z9ys21c109;D4At}t99mrK<@i0L^0Jpi9o!#X;R3uV0eX|Ftte0HzuP?{X2)lqzAOQ z^!R4+6S@BCeYHU`YD8~+A3RgEx#i*^Nkxf4}9kuney-JqIUK)tZGBjMnSCYqMq#4Bh1LGR$-vu;h z2D$^L@Inle$CMm=DgAnuXz&RaBu0-g#T=hDMG&mFT_gmm<+>6w75g`|nki-{iZ|0J zIyCq`*Y5g^YC*kb{O<`w8^N6{PrlMPsWU*rU+?REzkMt~okCEa`-9#u2}wr#rcn~B z3GpQSYf2hfX-la5{_*S~+y+`$PEOoU%?jKS;M4tbBj!lR?zdnD%VE_{@sm}HQOY}C z3{v@T~&RM;^$td1m=I- z%}abR#O`u+dr)lKDO0F0S1o`@pY3P|5`||wjMP$yr&T9^3${~?X~j-ReE~(gu=0w& z!(sDR>L{VE?rTu1?o^{y+c z9%qICbLExqV$Tm&41&$OJ@8ixc3DGP9nPe1+?|?tygEww9um#zB7asAa;B;)p|7-g zh8M+SFv*(5IfT)2-g4>s(!jZDrS5?=chVLj9g zaXa7jLIkYHA=#aNGxI(3yd16kE_av-@cCtgtkCp~#CW?` zG}FzO&mXsN9ojX?U)W;X=(>oTe+zTr<3_|iYdqdV7fNj}FO(Q!_w?N)Tc;ImM5R+? zn>yw1B^^x6y@W|tJo$TlU@C3T_AnNH@&|~na&Ga;)uIs92Asao=tD$#i~r$Y?5`M+ z)nR`nz5=Vbr#npXC@USEkEKnh?{lAQa*y&V?y;>*Ehg{cEX~tmalahP9|C+7&b2;T z{~MxqbXkRFP=&7wcqSgA1nArAJiBNYbyNy0b8~3a^&ywc9Ri~@xzBvAN z?H{pi*&HbPn|kvnN}0ZUX0)2H`!_37W8iD%Y@#t0Dv_XgP6M|Bbl5=G>j zfO_|-Dv6lc3*d?jT^HGHPO(%n8-^#eT-!Q=lg*(EBt)NT_YB^2TWlU|ygx$pWd-|{ zwd{#q->%P7x3W9IlhLc=1yY2=hPX<$DJ1A|Y>i^en9}s0*7~tsQpJcN@#M|>+Y|Y2 z6C??}A6W)e=|em@2-Iz)VNogK*5%0E%*D%JfDYd25BOoe#T~(>0zOA6wNhk1#k~y98<+TO2BbOihh6SMC03zrj@r ztxD*1kW=&nd%lQv5JkLZ{x42fZqd*~V-{{Z*Ky}m4mW{!;s;lgW%=DUN3P~DS%ejf z;af{9?ZVKkFFCz&98Ln38pf0LH`pLY- zF@w#Cwj9^|l1u+WPTp^CbzjKCVxW7z;M8yvCI9!LcMw*B_UNE2?t&_>U(S1NCYr6E zgs4eR!WSo5?=Mxx1-w#SsFCJY{2(d40pAb>OIa2JI4}tF*=nVxSpfCrw}TaC$cx5Mdt;bILGO9$zJ;=GM5pA zQG6zvR%;jXiI%Taq3EM$rOpRfwOccCpRSC|Xy{__ryJm-Z7!QfLyE7xskxZp2+Q^_ zuCAxs567gbamQ0@Ez{q#6kED>9tO}`nd7M#N2a%=!R^s@JOcL|Q3^3z2S0x_H?NX5 z`wra~RIVK;$&6a9AE4L2dt6iH&dIAf8h&W-r{?*C?ev0~!i&0j4nBPIS(`rOi$%#% zqdtR}jg1%$uv%f%wrp9xYBd5%p?koo_v)zB`b85~2N6=U}Ej-;zusP|}X-%9HB}lZ1Ge z#Er*?RUs7!iAVGDc`JOA@UvAepmA^Z?6jV&GpSuUbp&Cvl>)xys1IKD&=uM zs-9V#n*2+5pR^*_O9Rv1FVTy7E30mg!U9M!O(p^G!t~on_R>(R>Vhmax+*Koze3ix z&@VJ8Eq`l^Txn#rrzAOfO=dkSN-XLX5nDX>!%{~>=pU5l2n7|Fa7x<>Ksz2}Z7|xs z;yfCQBxTCf7IRM)8m$U;xnv?KGyibWCh;P_jJUT-c=~hJEe>%;jl)V7higU=Vtvie z1$%9$^!0|7Kyj?*h<5t(h{ExWBg7Nc1FmuK(0>;jnp*x?7kmc3u~u+C5V)i; z;j&pyXUSxG(a1R=7_Ruv?ZVj05ZtL;t>l2{u<>sR4Rdnq&l8I)5l8TOc0o% zqfe-4m5&8#XPU)WIo=7VDI7@FYgu9rB}Rjf+Jk?b>0Og^IubeA5;3l@mK|>@Y6dCp zah*l;nui^4E54&sSJ5Ck=jHA-E)LgHL1iB6hb}U}|<#m4u>h7A^dz4+FtaHwk z|3In9HG~k(7>=TN(Kf@%yMol57?p z8Jw<&!r+ax#t@22oHkFB6CdRV;-2t&&_0ck?K`;gxqjE>0Vyv0-e=_aV@C`TDDED$ zsxiYc#WAxCzoX2N{Bc{-CRg!gWSQ({k58|C){G@Gjf3v@na{%FUpe}R<7r_d(S?gI z1#4qte`E)AUF({ZTU2#5s;ctq^k| zs!%j`!K>U2>w)9|;sNU}Sm;unQk{FfXaI{K-E#^tqWZHc(cO@ESugN6V!-#^j8&ID z@mzzFiwnFD@koH^N_a@vX%MQ9jhEpTL?F)ukPm#d5aT+lNZ;>ADR;r!!n+U?E(aO5RKJQem?+dHBN{vn6MXZ?>}36vz}EQ09n!j*1=7yXot?kC zibZk?i>nG%$&XcZB5Jsdz_;Cr4Gm1xFfy~S)_a>kO=@(6JE!aSAhpSZ^{HPJp)skR zzUhnmv|{Tj_tT+BoX%nOV=`7JfcRdKc()#f+<|sOj%N8PrN25umXPFkk>&d%4ck92 z$d7N1V*UsZC&$C+Y}Z)_&l3I-LXn0Jo{@mB4;`x)2p!QDD#yWCi-y~R?V9uzgG!A) zn^4XvBy9!zC**>j{6$3ObF@<<5Qacf%nb~wO2EhLRVt6ne_Ei%UNcMvP%ZyYw>jno zn-r{d0M@cYAwRSn&osNr-+dJ_9FlF_G~CqCtQXeG8o~K0*+o|NmPm@0L5dg0+wsl? zFG5B>lVRL+BHFU}Oo13?>S|S}O<3!?41hCAFiP?22KqH;MORA`BL@GP;u`tF-z;(J z0gEEKZu@EKuIxv2+Akiume6GQ`%rL-ir|bxE|hd{BAjRI%?^KlY^aCsNGT^~Zmm|W zGJ4;(B}2=O<5!{yVVlQ5;BOY6j}hR^++1<~l3uP|7Awk6vfpvIut9&AyCd z4)w-$`>#6>J;hGiMLOsVLsO1isD-`A5o+O%iyY8yh>0CA`Acyj>Fg_>zfGw{>(dtD zA~-!ksQbz;2$`+9Ta~i^42~72&Gl0(2wfh*uiUj#)x;|b_Z66OTvLY))GbQ#DFG`} zVXys2lT@_E2PxVvJH4PP_+3O8+*5KUAFzco(s+y0X%d0%DIVSkwn+KnRnOvcDMD|y z+)8s$!^N4Nu{CTU`z!&vHl&Doc^^&38CW`C4P0;!YFvE(ut|z3w^DR5WtW2ws|HAG zsJVZA{ibNscbm*BHb7cp3BLsRJPus3hAe|-CD_tNy zBFma_J0Zh`c!l)NfoHuyDWxv99*fNCNI^;&9PY6l`cr5oJH0dbC~j*;^=ZB&Ln7d75hMT%?Pw`DEyp{Sb)>AzE7htW8EOt=6)}DZ9?oKH#G1RB4mpN?zV&(5qn33dm z=6Ea6owbl{dT?#ZKd=NIcEv}MC{@!K5V1j6G984Ay0nmxnPSv5Fg1gSuqjuW75z)J6k^vsjN(_)(xiI)x}k*?!|$Qw{4I@I7Nj5Y@TRDYe?ivK zS?7v#G1046EfQdTbnBmtSC&-IT@e>h`Gjd!niBRTF+e01jzxm&q7( zS`+krt#jY9$!f&5`%29YU!1&n-} zGz6-7>OH7Gah6(s_7NZPjKkyu5WB%&g42lkD;U0@%H?_F^FN_-ltk!p&?x!q@Zt@n z8F(cw)l2b1++h@=lF^=BhbTi}?ED&wsXgjFLQ%fMQv42?au(pZ@VZ65MWaOn_8Tys zP;46y?37)Oe`xC6o*PjqZALp!?7oyfxShtI1=X;RLT=P7QSlt3E3#aJ{eky^>Cpx* zmeAWVL+mne{1_=8*y#a)=^0keyqfzPxzf~N<3&J~zYdLVUc9(?nO}t*ab~SOuGi(N zV+C@g-%V9_Xt26iErDS7kjvzFF;6Za&!FBhu69|g{i*lR%W9G+ZWQ%1RPRQuG32&m ze$eYT4`dvAu@w?lbt(~TX1hLC;GHr)NMI>Z;rXGh6gOgCp4f~B(wT9(KmR`#0M1cm zED+`tvE8Ck;Fdm~vK+lvlXTc9;bEC~JbTmnamw%4pZ56xeXLa48kuW9#m+bd8my8c zQ(ypI3Iwe!uc(Iq!`1R#Z%>B^A+iO2{!b{8HG^S?1^G&tRz38d_nzliyqc(dSE?H3 z=B4cQ_Wa$Mki|726*(Xcs0ow-#sYnSNkfr98^ z!hmVgAo%o!kqGt_0N_AFhcu82z!?#Sj4cK{E7SvvVc6&VLfis2tR<#X$ZAUb)y3!pO)JHr`!pTXL$16C z_1$(We1qt2>sPnk?~$;0UZ|4SSn`)vBRVziAt%u%@`|^&$fi}>FP}($lVJKg z&JBeYMh_;cFz8Xt?X{MBAER4U#|u#p`6*)lvRmpq!aBMU#80ie1sYGK=^hCcv50Rp z3?({dES_szGBD5s=+=YE0TZ8_wbQX>90;7jvnWZ<*lD)E7#FUzi@B|neNAScEB02^Kus-XV0C~=!-daFxe6Nk z4R`<8reJ3j-U5xRG)hhMtw^&ud_w>sB}T&n=>`^4(Va3j^m{n;yGrfKSWL$ZBEJC-dntSVHU7 zP#~%Bp|Gw*Q+PuIE$fLe@gY0p&twdmRkeP)bL@M5vyY`SEgGH3oX^yW1=`k5`bnL; zlIo71S_&rLS~wE4TQ)IUtF2?5E2$WHgKDE%hc*T)8mlTV)su5hNyXmxYKh_ZQ3 zmMwYZ=c+&+bsju#0rztduaEn;Oo=*MEm3J^fz`ay-5}UMR+yCmU{W9>;l+qUd>lSS zB+YH^SD;~^9Yth0%p~kym47SP(!m#ZLsUOzflz@q*=RL zy%A(#_YjX*oxX<|t*)!;Ra(=xJu<(^+0H-LVcV%j=nGn0YEy!>pTDzt>!ay~JpZUu zF;DHs0J|a#tUJ{|D#U)SrgW&dcRjBqQK~8t{t#bY5Pj>afF44`FBSQ;+1@TYZ>DkXK zf7|+Zxxu26wL#$Z`v#gHOGH0gf3zv^Co+&0RR1rB1^ZcrLWbRGvo6JV(=_CCee^XP zUPYt_Ne}j(a7K`AG0NoHu&6yTm6_i+@ueP@(AlKC?QEUBooCocgHZ?lLf77G*~A{Q zm55lg?Ov5c%D)epQ4an;FYCe!69+dVifj-0S2{y`le#nALs7#?u!*0g5KAwOr+^9+ z9gvEy-!~^;m2wozRf>>_$On17`dtGLzZsFo;D6_;v6Z|z=OuMZLOTZf`YLHW>W!{h zpGWP0(G`_$2$!Wl(9+6RnGEZ6C~rDHo}hy&CVBj5XR_d$8OkOtTv+0G$+J|;!pnYM zxuh~kD>5C@ozPO(727aW%1Z)f-SC-xjCsTBV=kd5){iNQ`e?-)=Dh!%^P`iKnx zY~NH;TVBtDZFB6E1XUeIT+V^bc4+Is;ahO48+d2W-M;9JJ{%v z7zmN{xFiTD)Z|+ACio8C%Q%%D&~qj2bgE>?$IM>lW9wPMvwYmJg!(l5E|21{0Tx(c zO#eKM?zG}V%^iR5aVj7~_lTsF_M(P?pW2GH!pgiG#7PUB(N8D;5uLYi^k< z1du$qe03+Z$D9fd6RZW9bdRNn6x%#^&ar9i-d+^QU8yoH@J~LP7&nE?)va&ss+{bz@q#hbgQVQm@N$~Ra958`huiHX<20EvLu z%U@d#J8V{F=J%4y%K&+(5mbET&0ej(i?6n5TkRr^i)S%@XlWlWs@Eylm%?Eg50-3? z60aXaZ{m`MD?URTsiSBrUU#{>`KU$YxMVy&DpNEuittIzgSFf46PP|fgvkG;(LRub zeR14c?8y%cidD%0(t(ka?~_6<*5(_5-|GQdQNJoX%1qTq^|ui+`4(8ykJrM^i14xV z!QTvub!Xekb~WF7T`LLji_K^7uvJE+es5S3F}-b{b9*-P_FJCbWvTZxB%FTNdYPQ4 z)3qwnHU2t`<-@BJcl|z1-u(HJNa1a2YfXc8FcK#dJGq>E6+=8)R2r}DEH~7Yswwnu z4X0DTlldoOW>>I(+k7b3FbbH3u&Lg0YpjjZGjQ036W%7vaW3{7 z@MXTW0WePBOrzIHJ404IPCPWNmO0w5YFuC7pcU?qh<_PXq@JbG%cha7h^5_~-}(?^ zG=j-3>%EXqaF!;5M7xg0c(|7vJ(n7|d7nTdHxj3=r;{;cNwb#iVrRj2+0J$`MRN{^ z#P26I`I?{Q&0|u%w^IilCldc%=D7zmUhHi^L@A3+6%rd!3^i!3&;=pBUwL8w)Gcb$W`rHmQ4(=Mzw8bAL;p^wiueZ>tW9$fie%4J*W5?)#X-*G)s z7j}mJ3$?tGo4%2gn~N^|d8FHPJ7wj&>zA!G4s@}z_!48V@MZDD)Z$g0wI45b4AOf$ zg|1MuD6v&n0O z<{;Z%U9<%jLEY7T^y-Hq@SPl zf|?^!npgvD+B0KX=VJ~ZURt}(<)v?{?>pXI+Gv>CS1M!>Rm}_g4d90E`8OL`=HQBXm?znnxgnE?$kv%NFNZfTHZDIFm-KwpPCFGL|>Ogrb~tY?v5!dwf8d? zHe%b?h>H$u2xUP?c2%4BfAJA!_xX(0+jFgT5>fGsO=Y=39nu1_`ExnHP@>3ERMuk( z11eQ5Mzbv|u^QjgS1qz*DJ~fi&~5J3%^<(srxhpjEk92ewi>brf2Il2C&T-lp0io3 z6vdhvI@tY>B5)=Y<7mU)=DgFQ!x(r~)XP0Zi+)Pi2O9CEm?CUpeULxZwkkJO z6~08+z??A>G#-8JHQgP3X5$cyZ^wrTDI^N=diU`^3zcch%l_;czV>EgrSD4mG=-7K zzy*S%1xdHlMkcPJOMjVW2ifJ^7oJOPQC_lUONvp)PAOA+^HQh1hTG52bQ) z?GJXUu^O!Z(x=wC_05%gDIHzyD`*Y<2wE>#~j<=5erumakb!aYHg}EX*Fy%`YuZ%&#r{ns4&%m0|DR>TWjE zhw3}TJo9JPL^&~CL^aQQ<#}C#^_&m5s#753qujIZ)eMA+Cw^|0D{$koQ33W3TEzr= z(Lax77JCWsnfD$48O+hJe;~Ct9C<&LQ|Z=V%Fe`Z^>!ICep8Q`sQqsJ0J}%-m=Yer zKCkyMbB}jcZY5ypcu~ugPrc4X;84yWpkzjv?)GTdnMJ4`BS$*kW77BD|J*wWR{c_< zGj*0@$anc~G{ncC3bHYzJ!Lx+xRNb0V32-2Q3xl3jTbvy$?RRp6XuY-{n3J2u7#(D zYqSi?7~g#dv4ILY6p)t_c{P-&TZ;n__n)VE=$GUrC6)agl!UBW_;iS8IuFl>XYNGn zQ!DPLzmU<`tA9~N`>;cu@uHig7d^Z{5t0VHbC6EO5L@pPD z^M6Paz1Oea)CTmqq>Q25LxCV_6DoS#>6KT|5{i!X^`;0rV_hq>E!K(k)VmR*uS|B- zDK9Cr$vZWPSngOc%3-qmxT!+2nDdkmRLn!SIrPzYW=(C*YME>7HYNPE>!LQ|b3QVAxR`+-^+Jy>763 zi}FrxQUUfTqI8Pu?&42xd*Zz(K%Zg!fxb-Oimh~pz7I4#J}SD+%_k}LW1-ZFgY@iR z%HrvX8EN$W)c78@pc5LA;yQ!!R|EGo9#dEAd*_oZs$cd!XdD&0U1?FtKDAsRWBVw$ zHW_ruacLp<(QxL0ZX@fZQ%m6)NWk)rzO+ z^Oc|ZJsLiqw&iTxTsslM0eMws;6PGh(ZuiTKzBKtY%aup2@l)$XsZdkNcHdTl^@4m zF@**VPwjtvVaHcq!?WN@2LFz&mCW-)OLg!$W$NwzgVZ-1^M*pxS2F06`N0E5ClaZ^ zo_Y|NCY1}B49R*Uf=rR_8gm3JOD#9|>3sc!?p^*Kgf!D21L;=dGEx_uKu81HNQ_>q zM-iGdnmb9NJ%XqeL|0^!FC%1!-tzh^$BGD6H1|H`H)Sr(`yM-JHsmhXoY|RzRX;#zLF0-ixCQkFbx~L1;pOm5AV0zP~Y#{D)Gq!uaZ{IXb9f81m7@-5H ziSY5ZZ5zZlDUs+<0 zPG2zbnTX&n;)&35znY7s!edT-12Y-y<4zt*_J0 z%`OmL{ZdUH0#8+&J1@AL?{+(y($*)y9t~Sl7&AG=lG>xmCy0JU$k1S_-`)8T>G{Pa zkXd(tW%#$+#F4g8JPA1|8XQe-3kvvZ@!2M()U;rG(Rgj>t;;v=;x?~rwFqX;Ta?L4b5c>(Q2>{UdSIPt_SxrKZce#>Vyb`koHS;RX zh=idgP%H4`8Nb92VdL@F4?PzAP8gm}oa(R6xx3;<5NmfSuoHG=@#eUDMPoy*-m?9* zh#*{c5YmEO5U`Fjv9P$?z>By6gB;J zkc|%C!mif%t|0QMe;*R57xD?@T1l{%dw?BCFD6~gm_@wH@28gi(jf9t=kg(pkpLJA zbaB^e9B-gx*Wts;??FqIi9tD>ygJI`e?RFQgzTOK+|jC(pQ>miW8edgB~4c9LFbCn z4;wn?_8wT=`4$^AkIDehbyE)Y4}%6`IV7PM^}yTp)M66xbbny(-i;@XCD!(XpWXUa zMZG^!#5E6}KtL;X{9*AZ$Cl^oBmtobE$t`4HopG`i!NeJp~~eToLhjDvLkI@0w1$q zou9qon3BI5K!TqraS1Bj8pv*o%(S$_0^fMz4@yV)^HDQ4b04;b_(Az?EIS2uL{`I( zj#UbjV6W)-#mjB7J3>B!!#%@w47Wirp?``wF`$#b9Eus zbB#5hx5gOs<2hkrg`mVfaA!{J)_dvgb6j#c1w_>#c#bkO9^y~OiyPw|jPZu!% z^;Mo#s{3N^VW(DSO_x@eT<2il?=B#ioe7l*XawW|re8BD2TJiPi+6T=_HT9m1KkSU zNekq95d+!=2Ctgop}0gCw~eC&+0M7GU7=3}I- zzK9MhsI1EGwzhtIJh>(y2H4{@-({NqP*^K@^K4I(j#=6AeMryz-mRGkSNoMl`)*Jt zVyPqDEt1o_s^gx<6%C18ByestTIP5=QIeg%i-VTY%C#S0oF=sP9A5K1wp54*v*J5H zpzZH!j3M!Of6EQIcsQvWjEs<^>zqmet!HZ%Rp1kbM}79J!=6bg|As_9E@rrhos`z~ zH4jsBv--Od@Gj!R%_eME=5Fhzu~)=b+DQHMkIaugy=S*Vfnv$^+k|msE;a3FdnmTp zd+@|FQ8ueQuy0ku`%|RoJE(st zGwFSy;&IvoVMJ#^!z6_NNSm!})IdD+;aV;_=+nQVh*De#VkjJ8j&&CzIP-m0jF?uK zT7RXQsP5OEwlP4!SWL!{+t%YraL@gxl-89Txd}rhllkqQip;%=mg{=zEz2ty+jF_&7IM^huEMi&accX# zSpHVUqQ<;I74V9Mr!~)&w*IFCRZu6O>FfK@+iSa5B7Xa=CR11P@Dn#>6k878DSknT zr_dME=4n5W@mW){z=))Po+mx;E3M>=a~ruQ9E(FEfAPXpo&>>kb@_B~_x!h+Z%WvK zzpr<0o9~D!jvT#H8((M*DfWwN<@N9UB5SQppU@H_;qhz4$@%Vn>FqCDB$%d1z)Uk( z=Ev@tor`5#gT?cCpd>0bq|aqP$psme_Ii?G0j?=_=QC`nQ*aKfL`z5Eg81J_$7Xyw7~Z}Uxtl=T-IL7_P{2fdpziozrHQ?CS-3oe$K=! zqITs$!%^4gYn>9Dd(RqMnf7!@rG6PMKt!rI=<4ghu7OaVYLcPI5_*+aYvP)S>UA%v zzdE!H8hJ4-5mwqq_Yyg`4=PEG^r^?L>z%*d z+WngMN*^f2PAB60gpD{5a6;OLf*Z|`|9CZz6Kjgq_Ydswn#ZxhYwKcI>g)YmLdMsLPX`&1VK8U~c98pr zo|=2|3g-$$b{6dKf(rtq2+Q(*gGLtsLY~&K89dY{4po6`Z94wrllR_klL|tlF`s47 zDwE>QY@Ln^5;va;%}bEurRNLK^Dk1yeU0&-_xDnbVgTudLHmc@Uv%+jn`iyJYc_an3H<8srx(XoI)lKWoPF6%l3uA!N%5d|rm7k@7`8=PKfc>N zXfy@3IrD-taYo9NV#t_16tU>!!@fwhYpv9Kpcsur&O51fc})Jfs{mnxmGY1>0Ecbfb~p zvTb?YhXS!9?i{jhW2SDFJW zuV7Ork*~m$?Y^FjPUo&APsk0fCnt~VXebW0R9mdClG%1h3_?cc_#xzE4??>t>{F@- zUVjX7=;7J4wdg&Wa17co7R-x8g882AUbPVNHpy88&12Yx)TF`AxEp4Z6pL??HGJG(5?Hd!*{EwON{_3we8j4-3)(0jlDk zLbIszpcv`aXBw^+3tNK;FpzoE31%HhxD^XpDVsPdK;Jp}PYPQTuh>T+SX@CRn7nydd z6P;8N(#+*E{;bcoR?5V;vw-D*S~noUwz0{FWnXxfn{kdU2lWdu^qDe2btrFVLmj|V zd?T5wVRvY)KOm7q5ro1gUk{Ccv|sUfG@M!O@qBLk$y9X5xB7nEm`@L|JF*hKse@bi z%c9gbFESNZd|(IG-+6Non*fU85w0Q`ffp=pn%&_C8$|?_1rv`pbm8~&gIQ*WAGTM_p|%BHE%&J;Jlkf3A?yrl9)8FCJCsuN&RiB^cc6|2=v2 zMZ@uY)rsDQ(+>U2&JL1EgZ+jVpN{=FQD}Rhh9p(bcMHnFYu{k{e9Y#!;>Ay|YiYaR zs{FZmzmBNAwzR95uTv}j$EaDtSp16LR8iji&8)h^yPf$*dtD8<{G2|iy{9q21iR82 z^GY6|@@mR+PvKOannl3m>u&%B8QZWHOS_KyNrT^sUlV}^$WBj!*6}@`M1YTVVc;l) zKCZA(Fa5<5Hj^)NGgqhrG>NgKvv>B^{I6fjKQ|}-4P|qrKwLWd4t;igx=YHMtETy+ zg-Z)(i^_C>HrflfAndrVQMqLI-SS#x7%(Uvy_iwH)mZ*iIu#6t-Y}I{FTMwvf9toNO+-A zlE^jm^blmmE(Tp!%AyY6*Su=*T)VQ>iST=g6igjjri4VLeEOg*kM2 zJ)GjPWuJ{P(V&Z<2lvGqw_l`Yp^Le94%N(aG=|HmgHLG`C#2Hs4xi&o>RrP1?k3;!6(NkOt0MCdM-6B2rC16)0xo$}-Iw3Fju&Ffscmu1Ma4<+YkS0u z8#y+{`RFaH5GZWUqb;Ca+|E4{-4dGvouawW#+7(#fI zMNGMm0@rw}t{EDjKkW5E#-#n;QoNvwJ6{*9yq`0*i9m@2ttS?~_B?tz@jBf45<18R z!I$soVH(}gH8I#ewnHjK#59RmbSB!Rs_DPNZYwi0=@}SMA|Ds#Lo@3fR?TSx zuV~RnE~Tel=3Z2T%KhNIFg7GoAX1dQcF%Gwz^zhk%)I|);?!Bf>NqZp@%}rq#%{Eg6`RSP$x~t=s0;v&tjD7Q&7AmbIH!57|=LAs#mX>4HJS$cG zi`S`RvsUKQ-N{483`@=&rAkN%ha_d8adWpFUyfgQtbCJ0cENtENDVMarqbL}8qaxS z)(vaw{edzjYDPec7^vldtux$x*^kAK0&vsA^^9WQDWc>H)VWpQ&c z9sa8MnMQl{^;!n_xnOT1b1i@xzr9)%tiI5mP5-9-FOKs64;A|#m)DPYMqz&Jsq_Ly zjfpEzAE9;qwa40X(UM<<5R0jtlTbF{Wj;R?#I>wD=C0tjx@{90!?JX?r_}U?&m;kF zLc{d(p|UnAnNJs)G)DKif6_9HOMSa(pZMlt?3t+#`!09VJ7_@{t0{~ok>F$e;KuZ! zD(#I|Yzc0Zw*y-fiErImsx?|#s_l1#d*@2KKx4l+$5uMU#wWQJF2Y>JrAkkOaHkLA z_a8O&EnqS>>az^i9S{~*c|Oj4b&=S5&5z6H)S1FmTH!D<^mmb*g)W?oWzzRn_0_Ft z?cL%CTv^1;slqWroW(Iq8;>XSOzo}C&{Ce+H$dE1bIetV@8)R>v@x*x)#)l?q2RMY zy@Y>mvX!5q;Y1%&)}XQdDa+Emu29OSeg161*y?56DE#LYjV=UP7o4PFB%Ai>FnR$rL zbN!DWX421}`s?f5_>^$x3!ih=pA7ECs4U<=oHGao3X6Ewtlzr>3(85s0Y|59+!gkXtDE1owivTiE+a|4wutSoXQG& zxsSm=#jPsmsB)86_t#uvLj|=LlJ&6`9-USZIkitNswiDO*HhzdHZT<*#P{`l_-YvF zmVPdar{9#wdgGxQ9OoQbsajRpRD#i&j>&;6K*G{?9yW4bcqP&S*7K4nQ61*erb(~4 zg(}kEdhiH}KHL`*!b`cf#Iju^K3Vy)eg3YrCUUQACJ7tRO%=Yrgu2{=8KJRIOtcxpVcKlj2! z8M8N%={AZYFE5XjeQ)K>{B>=*rX12y`gLwPU}$Y&a^vSAJKgS*+EcB85B6HUf8Ueh zH%=uHKb+IS{PzF-^GzRmxYUlZXW(LQnkcCubPoI4?^^dkH`ywMh8HMiv zOB_M$X@LHNH61C#^|eFAGzNZLmCKorsJ`7_g)d-k^OrGnJTCPjA}NbF3VIvN+cVrH zkmE~si|tKv5gPRMj*ec#T5T__iaBNfKcu~9SQCrdHYx~$q9D3x5{jar(m_fn0YpKi zDqTu|P?X+#s46PGg9HL1AT{(}MF_oj2%wZefY4iLXISOkYk%+d+v}X;uW>G}nas@7 z%YFBg%vvq4codhD7Jyz6agj~@7&6C_&a1Od-_WSZrx(^98(TKJ9Tt#i;X)31sg!+~ zTiLoaka_6&7Nb_=Ic;&*rnUxOxiB`40dT8ct{-Y%ahvwwz8dm>5&zg*;SGrN#krS% zvtbR~kEI(DS+6&W(&bZhCir;iHRqOd!6A*J#oTLF6rI%3MR}wjSFEZR)F<)JWM9kO z-1P%13F|Mgn2z|bll&}|#mwy@X|`pgKgH1oo1?kH3X2t#Nf$=-&7c(Rg^4P^uI~Lg z9NVye^@(Bke#zqpzr#cQ89`NCBq70$rax0vl+3_Y>*p-cgv&n)W@Q-hN|rK5vl9s{ zt3~f0NF&@UadzY+MG8Hp;#Tk+WX*2NCm*Y!qqH271;sLbM|Mfg6-u zjeAVL9IX1&dGkF3#!=HAZT`E^+6WNIND({MO~_y;8W1ksLgMpoxdUEyq4FA~JH{mP>Kc^*(BK>3s9A-~MA{&u^68%l0?cXhPx=8Idmf32gxFs2V@#E8Z7 z52LC674`6$ku0z*IhH0SC9U%4R}EU0d+ywESrz!n|ivkNGXCg)(?sfUE@1q^> zdoH1}vhobL7t`6PSZ+O0X0tX`iy0hLKb^udCJY`llCd~xm8EhnpZ4|?0fWcL zu#@D!5qnFNwZn1&(92qHENF+kw-SI0J-e$fbHd-R{;Olghk)2J`tv{hOCp9jRQRUNR_e5Jx>QcJyD@<5aF3banJuybRJ2^+s7!jn9TB7NIak>yzB@@iaTcHd>SO=U z`I%vM;Q4f`;V}I_U%eGQk@%on@KBaoCPDt$So1;~2Xci6S~uz5ITWJkZ$t7!g0Pqd zKNs2ns*+f5IDZ@}$!s0dfM6UWj`*+Y2)yS4T0ppUC`zOMvJZF2$qfsugOl`03itJ@ zs}SO@`BZo9^;tj2ORtUOs}Low$9HSbLYUQ~LVrMN48#oB1Em)TEvMKvx3d8yN|;%Hx*0|@Y0@Bn4n3Q+B;*Q3DM>-SO|v`cPv3c_`WMEuQ2$S)Qh%(9%0 z#?dW)hCDqNCe1jX?19UIjJC|?O!=H1Lr%rlY6jrPJ0rD{yfy0&_kb_&G%zqQLP;b- zY_-_LytCg4DH>xt1z4(mk|Z}Ml?`-?aIZg_ z2|dT$$VvP>n)=Pwu&%AMOB%J)3p5t$ou^zPXMS`tq-qTU_P-aPnQ9% ztkj3*k*rNUV{dv8d?qZ(iDbP13ox!OS`U45*(g+?N*d)5Z39c zNdS8qlVpviA0aIz&n=t?ITz(I7(J5+Q0DFs@wjABx8Cx7rzhSOo>rr8W#F-t4%vUK zPe#SO0<0U$Ge>haOkOdy95WjiP5+r^2h>0eW33-e&{SWG>6u1@^1H9cY+h>Vhpm z>b42wPmI-8AKVTPWG1$)sZuVJ*EDK!3G2F6AnS85uk`y6#Q(T~Ume)90oKUgGGg%$ zLIFF4t6yh8VIkLU&uWARp+znlN32k?v7+%0h!sgPB>>cLxN4}p=EY~!ifk1vY>9I; zzs=UL(h>=uVHUHbH~^kym*|1iej^p(9Fy~<*5`5%=dW7fAxW?G9K%sOeqev|R!zB- zpawdk$COnQ>F-V;6z{0{*zU>~gr8#)A0xk2G^t*F7_;O!*#eMRe18$P=Wjzu zNv5?+^CnyxZ&WRy=4p>&Yn4|GZvd4cA&yl&9aFy^qWe2KF}(<23K`02KS2KQ1LYv1 z#yct!8;|w?ENNF@Cw|S<+vtllZFN7nCbgZT{KQkN?dh)Vp#-rJNkeNfFb@jZf5aE+ z%it#jh)AJ&LC+b6^~H8-ue4x&(`Zj%>FBVgJwpDSCB$=$UzON_WK@)-iTq7racGjG z7sP2%mE~$04|3^QehRZ6gqgQl&I8=TUUhNbNJY}Sol?9zN`(%1M*)kR1z)cn-=Cg7 z#!Nd?Ngdfi-!T)rNfdQ-gIC0*9P6l9jTQc(6=XcKg1SMEkbhdiP~V2@;$3bMF@`iU zT#*>lm7`m&UzoACkxgtVDq}pjB>gt%DFdcNlJ503D+{~yZ!GexjXhN%^f!C9VK4Pl zBiNAAH?g-57N3+Qd9746mGyTiL$=W*EG`iVzn;|pyQ`j+1C(`k_*Ngs?=E&GmQ9t2 zn^={kSpC}jB0wV{t&dVZ7}7U#&@}uUayGsHsyS4syG=gdm)`F_v5~}$4&?{L-pa`m zV{1|H!mbWV?Ea)2j*0?2VC#3U;#!sH$Mp3hu`$C!qFhxke3TzgGhm3k=|K~ogM-a!){Gtj}wKvA3B zffu6OB}Y(O-EK9D$LXy)&2AX@+m7DR;Ol*r(y#h%Pd2vM*3IUD59IHO!M{c7e-D`( zy4cq-_c2}h;GO@8__s+k85NDu41Qo1`1bQX0HS2qPY9qNfS!)t^Lya$xFWo@GXjN* zTgZR9grI)JwmyGyxVFZkF(0Af(oyKCT3#3W8KOI}o{7KVNv zp|Z<2rUaNciS1NW$}RDgCr&$+F5ekqcakan-9Vd{#9neYJx@3Q{kP=m_o!LKA8_0@ zBtxx#MdOT5q*XgOkY4?3@w5ETU8mg5_ky5DpFM%_CSQ{9w4fcJ>F)jAGY2?+1!5Z2 zV6^6#sl0EPH1j;b^@=~KLTimlVnalx7N^{?Np+M5J;{<2Qsdb!r8=!O9wafophm$t z2*3)C+1}oE9$U(#l##M3ObeYs8eMzEwrG~>%P2vJhdl9841(wyle zw5RFdJyv;eqeOd^u%ZI#n0oED@mjK^=gIv7^ShjD|DR<4sx}#s3~vkxhI|kV*a?<% zY>V>%uLXX}V>u6h7VVyQBPk&7jUXv;yBsUw(p2aea$8j1E@YGPV6}+tm7F`;&isZ$ z9RZhfvY|zj>v-lstImiI;24*wo3iU+fToLbC?ginA`JXWWih6SN^){-xLBQ488ta0 zT;q^`d4#Ll?|{HKEJ)04eWp|zYHDt3x-z0D05%Tw(D+~ykNj%FUscNWmx6TlJ;~W9qh8UvrAGC+B7372(rU2fm zKnm5}6iDUjvPS)8wolt>U6h;Xfxiz)-)MTERrN-=Q5>^NE_;ETAtf011z?^()Y9-~ zsHyyC($9G22^ilA`2UrtEnK}iyBgU42G5OqEM*0yAesm1A6V zS*~sYEX7cY=#8&gfNj|m?;7n}0H_u4s4;5!-cTkt34FWqgEOmi77FY&)_;=y^cNO@ zV3at=q4W(uzWj^0=Tk7$3StYuYu-Cpt3N%#CzvNM0pECGGzklshc)MG>S>hDXZ-|I zl1=i0o3J}iS*edn7Tla-S<{C|c5}ak&i$2#{VFE=<9rA!5SfLf&@=q$2l-f4!@P?o zf*WZ4?z%)3&Ia$rtVpmm$XyL-xK`aMs@3LR-AHONtx5n74zIsCqhKk7Vy zAX#r5EOHd)Q96)bJ?K37`6I>~2W+W&0+@tSE&x*4XpJk=nVFeyDeq;nDXOj8g@lt9J z`9HG8zdP0cbetP^*3?npoBz|e8b)1Ix9c*t$epWVa8ySrWDc=T~B<)yA` zQ;@sn9v)Q_#pXRBxnf2C%*^YP*i81!j-+c|`d2{FutVaqHico5^eB;9+G&koTOR*_ z^;kOiXA;1uVyJsaLl_F7WBl;1(6)e|L{S5T?Ln?7Cy5PFNCGLsS(T+5Rs?$bufO`c z!u%h{e4%GEo}qcy{#%Xne=1Oa9h6Df`s;RQ z_S$gyI#5U)wr$e>2Y30m$LD)@_L%QNuEqQ-4*mO;e|=mAISa6lg@Z}|?SlW{2mJqe z%+bOfrT723qlkaQP>z=En^EVE$ujGSD$_^*Nn8H6tWRD(D@-#)tNqnC<&rkkI}iOT z)?L-d+X%kXKCT{NjPe}d0pXXm^8ba6$j^(tj9{j)k zDmT~Rw}Ek|y0ARv#6Nzla8g4BKDKzt@H6lx8kBDGxz1a-0&}e*uJi4YJ4z+!79}N{ zHTE*wYM$w59a^c)bAvlI|HZmAH#J`IgPkS+oA0yBxdtdg-gr?V=N}1e#`OYF#NqT@_XisQ+w1Jf@wcN^M+c_dVrZ+Y z$=eU=4fZQMcfM!Q8VZgOlf<1cV;tSI_ExAUT~#FQQmAsrFfP4?1EDF1Q*2C6l`8}+ z4*el;D&^Eu2MKIQe(N%$li|kmFQ@Pym9HW8tck%r;bhGgP{#LK%E26~m*+Sa?+=0-j=yhE~$96$a-2t5(R!{1NE@j~ld zF4v!T_POh+`R#{hyV~9bua6Y8_#}go$W}4WU8_By8t4S|Id(oxUg>ipZ0_RID58=^ z3KboTF+&uB`mXLSTtn3ouIbfwugemxQM2)sETX;oV~`=H8!BFy`QbZZpsA77ljVFc zttf9P=1aH1r{!*aZ34AIV*1K&HdhQzt8|_P4_b<`)tQ-#PyaNiN;mU1?}p+HPS>6G zaL9Umy5cC&D}Wpze}?#VerX`a25%#d(W_a0R_?g@>~x>AQ4Q%0Pd%LL*cyqcG8QSI zU58=Q`nJ*0r_oI=l=42r4E*$eWOC=*^lo(bQ%@*6-(uV z?Yuo$(AW!pb?+-?n)a5|d} z_L^=9D;)F3E6TKvle~L)%!Hth!h}JUQv^rL5cQLH!lZCt7z~J02K2#Gw*}Q!Is~JzHGlxp4{6PqU zg}c7L2cM3F@9MA^dMx&)dQ)avakpw+ZF1NQ9$WBP@O}ZHQXPAa<24P?{6lW02!WQ~ zfzet6xMip``xqpVMO0IO_ik;p>nkd&7$%1)u8g$Z*K2RDh3>%Kbqyxg`9COm-1SuW za}r(AfhJhX+RA(C+x9ry4J@`~MYjWFBUrJ|xD%xUS(-T6otRMeTwU9X@j4Zj9T{Z! zMIodBa@NXXsaLw!bztqM&lgHMqn4?>Q!N;LVd0FlQruzgi8=M``j4jL&rR!bEs39x z*wsH$1afGO+E-26qnRY>DAsAIxd?hSPKDJ@8~vqFE{BgVzhHA!2q)A9cR#Onc6M&d zaQo}lj6l6INoqA_CPe!R*B`kO#!H*wpMm{$5iUXnmG+z8;+rn)RzL@h8$`#7`>M1> zxq!+49St97Lo?WVSv-t_9gi&4uAU@T;l8>o=r$YX{CmB2l8bovy!}WaTYu(x*k&&dOtHii**~+Qh4^{mkahS{x<0lsFd`=riB zW;*@^s9kOc%>dn{W4I67wn1KSQB|{QXJaP0G5vgOVLpo+j;w4Z^7%w&qPoy*tai87xoVu;=^Q^Ac=yQ;5YTaRz~40 zGo0CLz0tXRtE(3-i2Ldu=Z**Vr$@=&VSx4!{B)x8T+UFD9o6B{R?p#K1jJ*)x_rkc z6X*HMV7Hvs?-9tVad)l+4sbsb9)AQ{VfDoood*C>_0zUSE5AxmC!$J<&==-`%_lX- z)~kF7I0V2+@h3-EsXa9pEPXeo@K{W80vGSle=diEPz(Z$z zAq<^A#zsG{sa(3At`GIvXimNk!6W@`x!f+P5*n$zdyiMlrIztPoq*=pLB~C)hSdnjirpR=S$XZzSUP@RpVpC zF1ss;#oe{t_k-3L>qG1olVnk*xI6yE-Vm_6!w8^gK?n{R)7n?i?$_MCj z?#!HH&8xd}tp>-yH{ouibCE6U4m$ znp3!VJ&@TRPv2tf>!uiT=0pf0cQEU-L1$#2R9u3#f>L{~i2AqX4+VH>L^D>;Sf-r( zgO02t8W4HC9)0}(IRVj>2cnLg+0v}aW8>JUgju|b4?%4YEp<`|RWGCOjOa}5EaYOr z+8q1GM@^$7xvx2DI&63B!JpDXxngGaAUPtJZ-d3u9st9Dbtf#Nlxr{eZ;8SUPp^>cZ{(8nTAS(V{?OL?c&D%!!h zOtAZ})-JMU2H$sD__Odpm-Rh=_lTZ_ za|?1Op;di;%ualJgXN?~2YwA^yB%FN_47E?+S{^zs+~$e3rmpo*5CJFl>`x7)mj{w z-IF9T!RHeBmSq1LE#bsk9TV{@?sGdwU`iOfuxX%WjVShhimv2RmA?AH5wSR6t>8)N z6-0~p=}pQt`u*Gaz(We&nqF}Nu21u~gE|uI{!l7Z+tJOjeYTl@jfOY)=;N^e`^U@& zT)q_$N(zc)+=F{H7R2+5c0Zd@N%9 z@i}*UrH}FJ)EA7FhcOoxDmbcr{y>}TVrM;SzG>pLlggJfvfFtuhVKefaYe!Hs(y#o z6GxV?F7DptlmQ1)Den-&ldCJL&;*7+rWPZa#Rc;gJ@EvFj!E+ntpz(g#{aI*W)#xN z<-5-Pi$zyg4Ev_^Z1t|`k6P5(NR%yKX5?fj=kA|qek{P8kY?lbIgme+2dJwjH&{L3 zOBqKy`^1s6XV)<14al_%?wvOCdiVAGc;|DzaAFcYtwh||8FG}Dex)<9lj!K!Deb(Y zaQ|@YxEP5=oE*FtG^M9*7Yi{j&#))x!j@G-GX8VC#g}}RD*Q!^}d;FTxYUggxQfuLA@v^_s;^$y>ifLNyQP#q^kaR)!(kQay!N;~@oU&!j z=aKy3@|Yt&lxNA>HginUm7XkB8!sj}EqSgQPn{+H4NAA{1pOU=L60Sp z`I^0hN_@;--IDhyUIYU`MB=-XD+kHuC%sLyPG50}&(|Dvx4E!D^{*<1_}~YK?*!+a z_2)BmuU6R?v|OJJi37Eu&^uLW%BiKi0bATZ;5 zM5_$v-3I@U&E7%OpnQ&(=lrDUofY!iL@=}(T;U3!DDQjLss09tNm_$N87a_;*N+=y zLSs0SYTxr8B$Snc#6ijX_D+{H8+yzU62mQ=%%DuGZy%zP^|Dqpce65+ajriY#A@0I ztF}H9%Q<;mzx`;*PiHPv4Ta5}~JjC>}CXHA2X4UKJk+muYm`)d6(u7yRG?O6GBYVjTMY5g#Yo2O&u z$khqesh<)@$%i&SbEkgnO&Z6E!Obm25zT_9eNcPZc(;Ww-9 z;k2BBDGOp+xM!>E{jIi0V95?bfp*=Z*@%rHZ7VL3dVwkWQ__J?Qt|TfLHB@Y_5gW| zjMU9SS{-z9c3R%=%faQl7xULH$S#~;Fkg_FOHk-@e_-^1*L^`?TL=o8&*qtZ z0532WKfy5+nIfBx-XtxQk831m%YtVM-ES}tgk*wAbeu$}+yJB_5B3(%i)>qTVs}-b zTbnVR8{clDzMrM~Y1W3(%)jfl{gPSe&r zSt&Czv+-z~1$H48l!NvwsxrP<^!L(FWS{!rYJAPmg@IAXR;RN2-j56^<6! z)j2y+{VIuZ`awgfRv3j&ktL-J$CW-adYrC`)UnekW)N9|s)ue5EDdvDTnXT9zvc6e zt%NRH?|~lVj*HFDX=P})WT?4-=&|S(@I9~&XMv0~NoMg8 z0A|tI9h3IbjtqN~q1_ST@cEL-OiKRSb0YxEAOkpg2z}7;dO2o5?W%9p9glX~)U+4e z!FCo7UXFGHNE-n6d<9GZr|8238MceUANCZ325U4%j!?w{HXB?TJaa{v> z@aWE!Hgk%nD~8ZgDNWo9-Xy#4EFvC@=#@+gT_&rvz_O-gjbjwJFb6YtGmM!SJeVh$ z15{(+@$G(L4D5iDzTD;|W5xxoNjLD0{wk45Fv4u>-M(dQY=WGw5w!x1NRgQR%leI_*o2?9!aO6IMi2$t(MT3jTasi+xWHo-wXHH=sNV8J{C@o-Ev_BgmC|DMz1ou|fz!f-knP@) z5;mqqWp>Y{44yq2#JV<*(c^b%giZ%j2g$Q64p=UF>(ZW;f{^rNudb$C!!cKxL-$q+ zWie4eJ`6crcvx;ef;%0wgXSLu?X{5a8Ka!1d2NI=D{aMNpS2v{I1G_bd+P2;>O3N9 zq9_!arKOMGXi=CAtZ9k55fIF>aGuz@=7#RO`C$DPm3e@pfXyA@7=OJW&c=!F0+>U) zVNxdfQv60`WMRi-+QRHqPJpD7nIq}jw-yA%!?Wm9$wWiU;&cEw4RVWy`!; zp3VoM1gx^aZ@}l+wfmXqW;6C^SbqxY%=IE`7l?4*WSI>3BSsn_dh;9#$0o&&4=NXq z9>T=HCKVYq!r@RxFt)goJapx&FNN^W6){QBW}%D#cB-s4##)BI{nBW^@)V3MOP&e$ z&A>gFlZL`k0<;5`x=OFq92Z8TT54JnZoJ@?h_yb&d%6}C#)NrIgQuxh%d|fu z>tvG;*uAy;bv$D*bpb0eBtPzAJy9XJ!>>GWwQPqSSEMjnnwgL_`{*$7k$5#-K~PAf z*XTjF#r5E%0qE%5A=*WvOLpm-9D)5;Z6GT+vCnGhBtaNcL~A$xGXCnhM!(iJbRqfx zJM^$PKrL(_R&ey8h&l2In>nS~>po*&Q)>Wb2Hw08PS=f0}Bi}lNT!^sr7 zZLup?1Ck`hrKM2kxE)5{WfEG4X=x?BL@fHgY;!X%mv&P@mlnfI&L1GK-a3{W@0oBB z>zM(L`@PW9@Y6GDFHil}uVuRVELw*<-^fnEDiH9#CIsYlO4_zR6~c}s!B=GNr-N(9 zG9*g{Dd)KNBzY;F49_2yJm+>$_dGm_P*GWHLpg4~vUJvYRM@1h1Yl-44L_R<)2qUA`ArK= zj`(>P&eJO3k6WCHz6zD|na3w5p5$N-MCrwLr@C^#A1V%il*Y2*2WuZ zBX|66FIb4aqH^!#ZWVxTh=fYRc4C>^Ckp#Hle%d1p|6dR0eLH*qX-4AU$q!5#Kph# z(l{QGAaWEYW~`Kryl)vB)>g~C-w}5o_+z>Jv?YQvuYlCeHl&VQa6AlX-c7KB3H|e*q`n zcS4)z&L_N3$!UFlv*1pE7nhev0nOzm7rk}rt9RXf?j?2FOe#VVMpy zSCs+%p|R6H-AL=A{q;F=x!JhVTd)2J%TP6~{;0+(YZze(@Tx3I+jcW~wf7Hfz^wyV z`028zGS-4aBKKSmB5?_9nA5YULJQH0D<>%IoQFr}Y4=^Rh3i+p6n&9dw^W*6m=wC7 z)$~I=t-dGC&DvD`B$JAM@t(+MSGTA+?2Q?ejY!)0ByN4+xsvJKi2Zw-fnZGdw@q5BdCSf zk%Q>B1=AL3EU8E$DuFX=qGIh6p^GXBC2x0QEyT5j+wl_YVr@tOF75iUo@=-B?AZ49 zkyq2c=0!Twnj7X8R@bx}yk|=fKKU|-jlR&-U%0=(=#+t!=#kh`lp?!3X0X&|K%j?> z8R1PJhPgs(jgQvlyI$KxWQQ6tW?g4-s4=JG*%n+V9+CJiaY;>J!6hry%A|x7-76{Sp*m(VzQR z1(|>x85pUF;Sqw~VNAnm&3&DAcaKIVLSjlvdn$UHas<+{2qeQXJ>b>!H6xScdCq!4A6^@hn_2aVTI!1*%yNl*H^>VS! z`5_t9X??BuaerFHrbKe&v%_~+zr~mGT}aI17hc8h5)rzN&cnNF76E91h$Vc&2xb-1 zvbzCnF1HvyDn`X#9WVc&@|)J}r;?>v>Y7jM!;1uvo!aJEXXt1S<6eOMorLDJxCe8t&>Lur`kxv}6=czt!MIO^NRfkr z&w)n;-mBv^yf?Gjr=VYTQZ1U}s}H&m~(FLKOKU-Y?2Nyf1=i+N8SVvM)Q+ z(T(ug%d@xAIMRLst%Ctz0Zih;qjyhQ6y@Ywo-Z-nYcngFJ)8V&G27%5J~^yE^%U6@ zU=`#O2o(^HG^#zX&)~rD3EXMeoYG;}t}St8+r)_{?S8=BB>J!5ZNuH~*HxS#+ori0 z^=;SRS=%ofHIW^Dsxb_X6)6i+Vviub{+d5}s->WZVe$9%6VCT@X>~(y#RQF5yPx^D zU2?V{CZQu3KXS2tv!SgY4^p)qCv#8I3f+ueE{66#<3#OH2IJ`YUrGU6bG1ybVQw$M zq^?nSL=)%@Ylbu0rZ}YQ1_P**yqtz8k~LNnJ3`p$epsDF=YEaDP*~kmg&{USsR_O} zHPVq*Zn$EmXMvh3p-^Ua`W9QGQQsn6hdkiXbe88?9i(q{&9~5s|M{hEDb;%Pi!Mr< zSKuHt;Dhjz=exq9%{IKYzAu7IH~}uwFI+fcW`(OC?=jBJwvLkV9{;#TZW5E}^4YorU7-gsM~uYG ze{y+hyXU{Fzb6g19k7|~XvrO~6l>AZ6JS)iAEn$5(6@*mybh}vhGV@o%B~1H*n88( z<#yQX14&J9dcv5yc-(9U+6H z)?mIo+|93O5F{;f0e1abaHT012L(f>U+B*#XkVK&K{i3Kej3F>Kdkxb8-7}7qGLZ_ z{J25gq4mhZfaMA7tN{5*3*b){g3_{-N8 z)x8)77;4(?)@{Xv5u%X}gP!RO9q)A+nlKdYW?ZNgrbKOl)LQ_@_#{O;Y*A!PhKvi& zp6LY+f3HBdvsxFvXMFuq@wh!E>W<3l(AzQ-#*rY87sC2XytLoZN-sTSyq4_L_u>ao zct^s3eI168xa{X^k!>0+PyVNV=m)(MBp=MQoRdoPrV)D!a^hJs)oL-1!RJ)$Ggf>$<2tE^KcpZ!7p&ZyLo zHZ{;zwy);nd@au`9i$8{eW4uljR@=M3ZMU~hK-x>Ji&2FmhUhOiSWkROp=ApcH_>Y z?LnSpoC>GbpsZ8=?%}n?>ENJNb$F}d+JU;YCMI=H5OY8q_%4SECiVD_5w z`TnKD5W#4T`TM%zaT1v_kd|9QWlbLgNCTPOIaR`u(junx6ho!nKRTpL;*@pfei){P z3DHI?-_4}hVPi($I#|}tl=0U7DLWic&Q+tV&hnIFTC2b*I6_HpLUgdAv7$*-F~OgZ zUl?njf3!aA+7VSZJA+s@K?&cany=J!=2WpJV%w47L4;s}X9wd#PUbQb4WYB=XBnn@ zzSB|3t~R=!pJkq9vo->?%1F(tC(kZhf-{hXp&nn(E4Qmcz$EsA7s1UpGu?11;{*ep z$?r`a=8gQdg^^)fPFfj-pDceWF4M?5&wV2Fh9a(0Nr6B?Y*e$)&N#d?D>TCN!`oPv z7GED_boD;HG5;ecI5H@?{l&=Ky1uOL29C@5D<){oJLa7YUflXS{26o66BtOr);@`o z{f=d!W=7dd_o}^CE4ECx%|4zag}+S%{@bMB=n=`!_~JHP`13HLwL&!W^1C4cFM*Yz z1rXc)faHeoR({``QO{p6o2HolmfQ1F@UDP>n3-L)7O7n4EIO1bj#nSw@2~NY#6zFvcoD9>wrZ4AYZQBg$C-puU#Tw}`M!ee! zxOhYC71^8_pSf3Txj@MWeqPgX=(RRLYn2J2t916mBFJCJs;E<>z(%g%s_XA}`)Z;^Ysou0{c_k{K zUv$H)6m~F{Vbn3qo%Gl~XIW_M%Lzy&z)^VfRmZNsSAocebjt?Z;5s-zjw>`ti!7Ng zrloEZ6leYO2f$afrVK`Sl}HM-_tde=NpOgMYg5yYj6pzV@x>L%GUFMNQ+Gxx6wz~M zi+X`^Q18mq&pHNpuycCSUER7NJQ&}~LT118gmULY-!>$CWU`Djlc}hU_vTrpPB`+M zIl>w$hRqFxb-&bVuY8##DHReTF|oryNLUioV(6|k1Hp2$Q2Jbh`_E1-wt@|~7O5<_ zZt?bb!)y49Yft=&eK(Trt#*E}_k|F7l`SoYRENkbt~h=xyr(PSewZuOXM4|_TZSbE zRkDQIe%{`oD5cT3EEVDx>?1lHIU=ft3{)aL&OA|t{LDUIh0C<34t8D2v{euF`jQHm zov^f`!Z?VwcfUhu^L2&bg2D0*+d|bHV=fV%z?E^|2Wq}I5vBE4w_F>) z!O#lhQn8(HqoUg!uDBpF3tN47AcRjrN_at4Gh6O7kD70^W}Totq7fCG#co3fIhbnX zqCECKREOqc+Tr3E=UjftossmkfZfMroIF1qOY$zMiM2v?)sdo=92m})=lj`I&Rj_Q z-uywbgh`Q#Cpp1qGcG8buT0}5#UBN2TdiN6x7$Odt2cntdWWmlgJ`qpmTX(P7 z`8Lji{Sq~`k=xK~Jv-<}cK%*K#^#%ija-RXF%``g*gG%JNe2H7*q3y{^N^|FOt%PG z_Pte4Bm73Rfn-;2X=PpZy>3 z_>1UmYPx{R)h@r!sC#@dW(Pe^-+aBt{SF`9b&m;16?J+pjA>QSDvF5W>ekTCXVR)! z@}3wbm8~n&T=|}wc{rZI9bpTatR62gZ*==c;z;@28D%MAl3YeXxae#khW^ z#r1IaJjmG%$9yzqr{d2Q%Si0Oy)K-o5#8iwq7903Je!xW%OiIaV`%IuMQ{C7$sUUF z`nmn~$B?V?<HsnH7y4UO335+C(5}a13i>P}u9^0Em(p z>@so(O!Fn8eB-0W={C`j4+9I%T&dr&g`d5Tcd03uYMasWiYSA^=-h{>!Um_6O1RBo z!EwVkZ5O@a=<{;%ks*lf2W9;%xS$xc{afYjW=uu=$?S%F%=2TEM>HM#{yi=4Q0uw4 zyFX^ra0Xrafer<@ee$}Ffm3mLmaBDI%h9Zt zUKEwSg;xmCv9-ya0L%7-3)saEOE1SCZB__rFc!0cokKOSkYuBFTSBZUe}n$ zIljd1zw>O2*-=YfV02&nKsqyup?*7{v11lN`{rehkdkYE@hbj=l~(NM9D67UMZ? zW2M^3#`JY8oa`nT%z9-rx2E$frgOjA8?WgZFY*eBgJaXm%T`{_FO9-cx%LgDDtl5) zlL}6nWH`)1u_&(Fn`>GW^~+4viidc*?Rs!?(p`^%?k;NIH9Z_CgB7 zh{89{wvSo7PqvVIZDtoF-)X#iDkm-1RG8Z1%$SyxD?hO_W zANuYKeJQ{i^*W1hLF2LobO~2{{Oln^bmlwPr8X@I^C9^pR25@f7PB=>2Q=btpE(~q z!0Qq`UB<8)$Rlh&CvH=GdEu-a=jyvjq3vw6`2KLJ#-yb)g@`z9}Ii{G`sX& z;Ykdc&0I+cP|c=O$!*zDT}szg5O2w3_XTOC)PEq6P~-~c(EZMD5qhV5F)Hs)M zL{bxUP@%sa1U+FuqC8$I)<2V#vo2OqETu8au2l!q^cYMvqacY#Guo!}JdGq*%LZJ= zc*~?>q~J>JqY@mYMD>@Ewmr%Mucg@iOPHcK+ha0om-i#aj`M0>r-+YPZK7HKaBv@` zJ=LAfL6V8dhDNttozoq24~{3Sg{sP8)VlZ9z`aQh-bcMZ@X@o*8pXt!h|}YaG7Cio z-p=VdbGO|Hwx&Higby@TUo4K}iXC>G*eWNyZ6@d~;SN?RbC80}2KU{R!w%bC>%mCi zsr+wghC!ZFo_NOdiP0mjgI|tBC3Qq^9AN17n`5d+8cgYzmf2Y6-=IT?e3Ufe@LtNfL1W5c|z585ebAx1|osbbqZ72@gk z)a&%vz<(Sf{eMS|{yL&jvvPG9_wdK`b?mMDgZo0Sgg!p(eE6}(&Dd&WSWrGG^`g{s znX@wM?_6ymmxc2i>_WfE@RBwCi1&zYhwEidmy1+weYCZ*?u?fd#(6BdMR5I5Tj{|4 z(fa2(R%rSeS=H~cu(uhKAz$Rk@pwqV^MLxt###BkFf&TRnM5Rt?qYc>*Wfj8LZh@M zAJKdPSyI)qA3gLKriW=ptUq!p`77*qCIJ-89UsADj+f%--Rl zNuZX<7iTFEioWsDKKbkIIRBN(lZ+Q;gJ)+l$6AZQV)_F8@PT#YiE;9moOh?bKYP5Ob z&9&lb@}Hk&o>@F^{@C~H#Hrn-jll&- zmBLTo8(t*ONFR~XwV#K28mR9sf7!S;(Bjc8;j~{+gd2LcEvX5cOV!$bQ$G4=qEFtY zR~EE=PYv22lN`Wzv+>d1L(5U#7pbN{)<4sq!Pp(|@Ls8Nc6c{4P~=oMsjhc)!#)HgQ12c>uLOmH2vJl$V9^p3c+ zr7*t0R+X3DuJ2nV{Z?Gp{%^u%P7$2X)l5(f;@ql zUE0gOb&oKS9vjyve-c59cQxBPap4oW&NGD7C$LWoni1NW3vbk+Qt!=+Qrt}FJz8&Q zOP+UJkjSqQDfN1A+F53_{*DJxb07j58=^KAl}BV(i#^Jv#4Gz+nSD8D8b!KVrr`U4 z0QJhd<`zYQ?7A6!=_U4vsSZF>Tla-u3Ljklu2=ECBEQfLa~i8tM&&Uh>df+3_iz5( zfLy4RAG$2Yx$xPY;+^L%)pJUqLcL*bFviR{G1Okt!!3oc|F!n&#(;J^DO>H&36 z>!otVtz}NjNXb0C6nmm=Z2xuTT9JpmS3qK7rK@`y7kbW736BA7;>ApV5}KFyk#wi33L(HQHC zzS&sb&jPIE*o33Ui1u8(>xsQmw_mhreS2&z>mlYC!iV_d52b6jZ(M!Q_%X)PGaW@9 z$@HijC^IWzoHFBokt;{bo#GS=^>K$2vh+|wi$)Sn_Ui31S1V3x`j*BoBqH&IRVo_o zS~y7(^v{|8)a?3rvkM>O`LaL9JwDdOk1!RdV&xXu&<|4?5Zj*t>G-O8mSd-AdHULT{b_*pxsa1$8Y1S?(R-KoO(A^<{03X8{)n!iIJ(dkWaG3`r5orNts)m z5M2?lYE$fd+v8KW-elvcW|y1us7XB6XX8)WN!nve71}M!$tlI255{wVfW+jNTX-;Y z(%WHcuJTI+BeGZ;7i2C3D3R!`so{u0qXtpkhS#b!^*Bzze*TFR_}C-K$HgzkI~Nh6 z;zUlz^|`wQ8r}cT<$0w?e8N)*IpGmGMD_Miezt%Ge5*~qqx zY(=gj=R^c}#puC>I)79ylEts z%n^{iFV=GK*-@@9_7!;!2>GKTPDO6+==^@)4SQ{> zUss>|m*#Q_^&BEf$}r|;|LI7DTot8}5C;%-@+sp12a^ip|9CI&D3|!ceY^-is<^x} z$d#yJvT{vgs~(nf=Un28s02T`EDQG=1IRhV+96Y5BW2Q*d5a+!1xvo%Z9`IeOfOK# zO14XBM^0<|-(`5zbgGUR3Y12gaM@y9q&;*&dEqXxVKOQOQ-U9boTsczBvF%7LEZvl z^o!o$si2H7mAY7z;L%^X;^iYIjx7lYM*pSI%S)=tNd-@gM{HDk4CWKFcc@thl*9DY z)cYHOT@ga<$STw5aEtzSl-l>D!GfM8556g;ILfx1v3~#k#`7J=s;y>QM8x!R8g{^v zRxSe`=+DXcRF-5^Ilhc8DJ2gIdqP&%tnYzI?PxA6TkK)OSIN8|* zEzQ7tD@mi#;n!awh@u;3nf$D-^TmFzfRzkTgTfE_g4Cl8Qp5>CW@c97g~Gw*llnZY zV|De0xA+#mD~2klMe(rw+4TKHLZwH^BiL1NQ~l#uxZBry?-RPGrmpT-C7p-U%X-bQ z1}?oZCq&CFt_C@M@zA#=8jN}AZg!{@9@>D23W-N-A}381onlb>{stj@Ck7z4zAP7L+7x;4OBSe-a<P;Yox zL!&{)F}(IjK%vf{>_KwERr3H17}xybN&A}uq4k@y6?pzzpbKK08rs&fJ24Hup#h;L zp_{otgFt<*Ns=Y;{v9a`N1Jo2+S=|N$zKgj-z_zs2pp324>?-jJWxg4L0Pk;_l|xJ2ZH*XWQh3+ zjSJsZpT7GdjWcJ&7$j#MaEcONLTYHdm`R2j={Y&xWmrCoGAK6q-6+Q*yd5 z7=DP+RkF<6SIFwSwRiqcSDSiBWa>3`D))IQHuAiXf}tIQc6Y77NB&v5sYTK+C5oQs ze`I2+!RoTSZa>_g^hlNDLNDZ5UR5q}5y(j_2wwZ)%E+wK9kmKhC*;wZK`0 z_&7&1I7LzjK43?vP)D*O@hJwmV3^Z<$5S9UUQj>D(ldjNRCORK8A-%KVppOy1t)2O zN~79BG}u>&mhrcSUbNttyv?fVwc+|Ry#Bl`f=)Dcrb94yoSqQ>AT@37&X^qbPvlqR zJGYv64EN{nhb(>hJF?)+Y;7fbVQTU67+UgOgm-5`v6V@LJYh0JEg&+>?JS|*pgh`J z33lln(SPNJjdi2BQu$Hxc~Q_j56=Omh`AKEpbpEGR|Lkvz_F2?^I|qc%pxpolsciQ zxRQsee^IesmBG~r=W^h;XbFBO7cBY_4bTQn51rybimrPxz+Md{z5DP z>F-5B_0O+f`(0i-F6z2c(2l;?$Hb-D654I3wDe@&UN`!@%8{`N&hrX;wmvWW4qD6X3{xn$` zv4FYPG4|rNzXt3uOyfxpuV9_+y!D?e5BsbhsTgET_guPw7Xih+WMxiVKdjNlBIuLK zpE&6(^DUHRyB>LBr?j<*jjY`u&K^?qg^m}Wbd8p*V#_DP^R~LsKHOvZ?N{_n(y-G>^?AG-3&W?<`q-ff-wTEqI54Py4 z)6=vY4ewPrgvP$sdahg%p*u7#y5&UN&NPMbx3Av0^H$2xZd<$#Rz{UG9dM!U6Ex5y zyPB85-Ok7!P#2ijpPRBEJTG?m@N!W8;)|DGbV{G}SoeCTblc{1RrEj}wvos74SYW@ zIXGojC7&gHIWT+hhuCsY@ppQurlw4;TsE6>OTsT_~Bf z)s>(^9%=oEbXN(RxS{T^@*~L#q%NdW-??<<9qqP`^?J@=!;7$D+t|#@zY0d(AvH`^ zC#2WvH`lH2eIvDcy6l@t2bz=p`^RST`dYpf#Z}}&M-5+-ile_GVCu6_ybwM=p7BJr zV`jw~+eVDU@Tm+X9K)^<_hE;yZ5^p5G*j#8p5^yM%?$zTN#SLFqMh3G0rcgn{Fg0V zl3JOZZDAMT)z+A68uV8#CTS_1-V@$Vs0jX#-gnG=B}DKG@jDv9xFPwV4wqA+fgmrU zx2IXq#x-m1#)~W|ro#M;0~^!{3(R|bPiA}C^L5%VVqvT|XCqE7K3AW$o18Wbo03pW3a=P}xAqvM7T& zs~U4N2&E)HI-pa%?2;KYzqPrye}zvmZpWa7{9X~OnS=;acqTL;U59c6c46?TdDAT1 z^wiOyuMYxS^^JU^-~zGQzVaE;YQpN>;K_!2P7krOtnNprT6*2nInGndRzK$OAwVt{ zBiZsSLphTh?r2y(u*-1L3s@^vY51U~rKnUfUN#!^Aat3U;xOya+@M@x@x6ebYL~qd zXwTY3c_aJKwf1Ge;OvJlDGV)(B_C$o!tBsGXQ9la~v0BYk>$Ijw8>9&01GZ!; zKvJMIf!H!+rAKVMI2+od0`^RWw`~Z2bHQT$d?W%{f&vXU-X%Y2ywKPho-Q}V=&ohu zQYy`rlXAuhX_$M`;PF!rOvyb^$_YU8@%W?9+?^@x!khHxkTKECJ@l^Vn=#w_*Rj<7 zfsK%e-GHQ9pP#?k=vn2BYWN=gDga3lIrJvMlIt!Tbt-YBPyVOPp_vBWN;*H4eRQte>t&>u!~JW1{=O32uy3)?ObM^TU%KDzj0p8ruI!N`el6}qFD~P3OUFclk5AZ8>XUee@EFbD;zqE>n)f|MH zp|%RnZCFTa-uD$jUFs_++z>oA)qy+_A+OS+T9R;#=m)zZr?KFOJ>23M7`}+yCf|NI zGzH3YkqerCnDXHc*BoWK8c9P&E~gOB5%Q(_SjD+M0yYwhniHmc0G zr{IL4wp~gLy!sV7$Rzld0{-kJ(WCUou!ZI34Mp8k9_Q5`IwUQoW=dD)zgIbAd6ikJ zz`l6ml6qkJm$;C0>4PHY6I3R|^|sHC6VultNp>4XN~1T_jp=7xdIIrMME&4(B{;1i z3sa!RBYFpYGlTEI@Re>pbzDzcuG5B~1vnL`1Lq}YnLE8FmOvcsP|jnyXlo>>t3;0K zR^;OJOjz^K;*$f5MWFtTyr<`NX>PCP{w&Tu;M=+WCa5K@;qJjIwKSjYMt^>=zYdPH;@8on->I10jbhkjyGJm z5PbN0EDN$7`C_rSA^hq+zZ8&I!x?i8Oy*!x)Ey+;08|aC)BJ+W3E?bqzjj@o|GGifWDs_+UdI=6aZa6N6MJR;&J)>Bg zSpQ-3L-q6Sm4ru*rzM~^Gm@rNo$TLB;Q|S>$T4(@pO@L7f@3yTN(%HW`Dq!#gP&z` z>@{~XZ${S%36VouwmnkP56_xA{ZY~${)~?$R^F%U#eUy<9QJ#W?URh8%0hErxSEw% za=Nff(OhC`R47oOOle6c1<2Zq>07Lebd<6`zG4pa=q;xvlqYeIX^Lzlj$O>4ey+Uq zO20gD;XO2_IN14o01Wg<$Qhcs3&Cx$#3Q#)?Dr^7u;vozg5 zDSi=S)uvl!G{?Y`FThR3c~tm2t5XTtVlCVx1Nfr!`^>>Q?-tKYOgE)PF37_(C~K9L z2r;yr!^$)~f9LM_%lLm*+Mla#chu3ZQUHLNR(t2^7?$+h5pWcvI}VLcQNQ)AYUM<# zyWk?_G*k3#CG+}^L3al*VYn$E&ML9GiRf6 z4*O-9XX7Q`^h9J`hc~ZstGO>Zg^V-zz;128fG!rd4;x@cJ&Zo3ED)Akd zRfL6rz9XhZYV#{m$Dw>e5BuPT)dF$DV5bN!Qs$bP=_judOY~)4I)Gqyl@<`Mdw?U8 zyLe2$CnWfoW7a_G$YmR)x{|W4OgG;Q@c9 zQ>C5=hZJ!K(yn8~Yymbn%R4jT#G;zlME3d1M7QJ@;{u#ugjhY(hDbB9qh$Q53;3cs zO>)gJ-&5DP4lZyZM0ZRSw|U!t{O$vB6LVyOUq*;ho5I*gf&XFk9&Imwrh#&Lng_K) zwd~TcGh%SPGrw|bv&g5{GCdhMW2453MNFXw7w-m7N&Jf5{!OREF!9XibWo>^c4@;@ z>_~JQ>T2;SGL7wx0^sDUsl5E4?Vq6Ipks%aw?sLK51Hs_bo2w0!(X*pq^2vk4r$nF z_)YRA&}%;#65S^oTbn-J7Mw)m`!0r`Ff*dZnnvjc_+K@#zOBtEBOQ0i*l2R>w(jAF z)~FEoI~Sq~79aT-gqAKpTK7y=Sw0>q_a>;^ zTBN+&EtHvd5Idpso3U8LDGW+wp(_$TDE&~S6H@Nnphvcsutf45 zqu$7n#qYThDzgWCk#oY6ae9J`$*@QdF*_1XCP7Jpi3@7Du91RHH=$d~I*U$`$`W5? z$$ZY)G*Jd*88Gnp^^SmmyiJn;%H^@+ar#`MQ;$q|7WHB1>o!S?2Y-&`CO41*Cky-$ z7`;dp=5-g(v+6T-vLv^nHowq%xbZ^S>`{L^^hmjvqa@`qy*-Wka@G0yYh9`s^hJxL zqrf1vfG{We!AwX<5-TK;V&~~KLM3R-+y6;$%_ooBucY@+=RiE1iAHjuKznW%_XTU*47zTn2Z)utauc2^H&Vsf1d z^MITl&k@J8hiBRo>3iuAg!f3;aornH|yzLzO#y^VK_ZDWMGW?ZOl_A6Lfw zwttfT<>cZd6YPK{YU$zZdPp?}gbk)i*4jEr24tca)PvpE=1VvSm45h6&pz6ntxLY- z5MaOZCYYHJJh@E)bf*WBjdeawAIJ_CREH7aNqRmop6qgCyUfr7ZpmhOM`X59nLR}B$28sW<8U-dV%ty$F|b~ z@>^Nnh^o6~3uPXFHt5)Vm8XN9#45I?t9Kj{=7a!M-7uztf|pJzerqyah1m&ni(9E{ zSlu2UdmcoIT$u(%>_pYVXs4{}&8tcuUDWgIEY-(^Cs4C*7*}PRJ9hGy*<((dZ*;96 zMx>|%N7M4lq~Qw=?x z-v^}|tXUEoBytr#G-=@m!D9VPbZAVZv*#SpUz^u<54#8pU8NT|7{+(-1mc{ua9Cnx zeSuRRLByW%ZOw|=_MYn9<>|(ilDC+pBpmLRTI$h>C5;5|D<-QI&1;#L-_qoH)>L#G zpIqNUm{4xp18aMl+Mm}R3XenoyHw*>0oU?)?<>$t{*X@3kB1HjhJ1{;8Y7KICRkfo zzpzf__KttlDGxM$oW4vGrr`?NTH7TQ7R0Nj6+&-@&L)10s>-`;^EnQNWy>r!3cTsZ zNVhI5G?$Pvl;q(x-JbHus`*5=`xbgp__FFjN08%oc_&^lviS9W65HCS)bW;*hW)WO z$wp2!CROnt6GTBU_9|a;CO{Pl66kl5j$FlSMESiY{39%&{ zvl53EK_`nIA`eaCuo=v#^s%>Q#y<X(Az9ki-=FyQ zn8ez8}tRkvnf% zv4PHX?l3dKyTwj#oS1Y_8TO07;ac!MG#Pu`nyk8)2|n2 zC%iHDU*H&U|K#H7oJH7j)Q&KI!vd|>hGEjh%z&P^vdQ(s;q8{)5PMAIGVGr8saRd6 zfrM$-Le6QkE-qiEDJyCakG8pWsi+eg=4Tv{J5k*&72oo<)G96Wiu#IU($u8y-63ld zJ)sJKv}3JA8S82uX5gMUnv(COPf1!*k%a2*H;FcpObxb%!x2r!WS!O=1JXCxTEC9PvxU)4IQl6 zb*}^+-C1muT-)l*GWWmKA!)7SYB)7;OqOj7&aZWNBj)N`zI0pkJ-bOlx0h_QaUXTb zehIfq`lLqStS>w9-Y3d@9@G&9E89BAG!NhO6!&f@ zYxja`uP*rd+RrVwcE`c#NLud7o@)@;X>m~loB1sjC@che^(uNlDxI#OkD$C71?~Fj zeM`0umt`h_AgB9QjqLE&A!Fl)pVtsn2V5)0i9?KaQ#&xYN@o!3BG%;^-&dTQd#Bgz zjC@Jpw)2?Cx975V#b{tCrKo36&NhiAN_}J7{Pkwp%p>{2i?r3s4f4e>GE1U3z#LXp z321p%%-1~f$lEx=ChK!SL*UpI#C|(5`c^uW=1R#|->KVJMJLSL`@%o-wK(PesNGEB zBA@&@#dJYhI$DonEK!Ui?sIvbpB72Fe{8TYrV|NjSsM}5*xbmLYJ6Sk?}kHT+fgcR!;))WF5{NynRAXX{IJyRHpz>mwi)yCTt88c_yIvGS4O5 zBF+WvHF|rw*@`g+X5>sarMK8ubGvk#ZD=XywLDya`&TWN2r=dgoQmF4@6gY4GA{k8 zmx4nU#pyG3)(pGWEUk5xR%wUvHkI-)sR6uPjT2>W9#@=jzdQ-kC)(56)7dgtMP6>%$1ZeA+k&gHV|OiM>ZfsA z<%!RNik@+WneQ{6#Y87_MWJ=J?7y%LEye*tenSH#nu5Modt2&sd~v1SR=a1e)03i^ zP~B2DsmC6tz{;5dISSS7VSe@uJ4xVYXm1MQg=IPaBJtVT3I-+N8C_J*tFC2UwZrMSlkOMMIPR-K7vdhAca?n?{e7QVSNb5_b1d9e8f4t=sgFvHs$5AE=)m0xzL zmhZDTv>P7eT+)e1>y#!PEQb_(HxE`mBi9pTW{Cyqr)U!U=2Y&6ACtUj- zL~70uV^%PadG$bdK-!H^5}xn!S4lzIw0#J@2TJXfJhyEhgWw>6QA&#|I5(LKr1o*E zks}Yp3UZ2@s%-+QM{Rsor|>clIvUD0UapzzSH%hJpT0UwDSlk1z8#1tRo>2>J!s+f zN`Xi1=abki#C3l{V;O7hAeU{Jo?bB=nQSNa7~_DCall!%sFlJbPzv^%tQJhFhW-S?H>+Ta;ZhjoUNBuMi3(Y@1J!qk$t z&GMRqIQs&cFXE`3_ya%CaQMom>(|>9XitCMoY?Nlfr)YFt4uHR_`v9bDF13Ft4$eL zq|y#;XNX|gkA^>tz+d)qtV_%kg374)Fm^@;#^2K94f#*ZJ`sFFHnWrJmVteCf8TZP zAdcJJY`id5x>oD9bDj_V!uz1yWn1nWVh((=d3Dj&2s`b zj@^U}DEY4b_#D{NH&Roj&=61#9miC+3Lqe3;*O(5aBAUEUwp@8>{8`^qhUwjeX3QZ z!NW?y3^2mPIKee5_XgtqWk!b;L9n*2G2%ADY7to}Sc^5S2&)a_VDr!GU=$i)yo%so zbEtFHAr^B_X8c@?4XmA4pPviYx=*p2JwR$^jFY$T>q5aEc@Rc`@6RBk5`Wu!4SD!j zSWaI7LeD#f2}8s~Jy*Yl9x1hDn0N~s**zkv6In3cgsRvOU0Af?MM5)wi%PW zB*~({yxRv*=ZY$WbdKb`e}x^}dbw2luEW+3L|$Okx>`lHC^ZmdoF+}@SNcm@gzUO% zr3zscp6M!V0Obh~IuTbAsGQB#ibltBUkyip$(h7W_XXO>kBL{#eMiCgOj>|nu?$U>8xh~b|Gl*0(V=n;PlJ~UheC|m*BrgNg49|@AQP?;1 zT4g9eebi^7qSa;xt7zO|@?VUQf1R+eEOn`Wb<+O-d^b6;zlWihj(_%{LFV6Uwf>vx z%YVTf6k@J`5ehM0lUz>vQy8PepX0KI}Js z{dYHC1Ax;}P(>XB&9?t`DZjhKe|kUSoK#8l%k-ST{;v7z@FKl2WTrh`I}lMx{cPPB z{}n(+6hE5oj#xTyxlhupve#O)WKq!w)wQU#^*29^KN1*9=SrmrjbF)S*Fi{$NvZ22 zi1)0?5TqqUnrs z8G!#D7+GJSsfxJ+ryn>=ez-Rp73h}#%^N*#C=Z-(%Hex>@|Wafgx>JazJgjngBnt~ zK;*9lFq|?0%WaxCSR#UQol{lrXMc%HEf|sWa}PW=LRcH*4_YlqVRxxw+9V`_SvbJ2 zEGgLz(ntTpH2vpa<%I&{oEK`=zQ>oSVb^*Z!pZ=PQ2v?XvxzTdgR zskgvZFS5jQy8Ejafc2XF+@UwF_xBz-AVzNmEF46vflnvIx6O>`qg8v zn~|&cu&~o56ZJ=LyUvV+2?^#1eq^JI{_DAh-x?(MJxd0DZ}O{!H!1255*OuC@%1nc z(=+pc;NyycjoQvS4G;bLij-vy1iU0w043r5jS2MDhA9pOAl5rlcH~^2=O2rxgbwa8 zqVYFsrNcrZyPiO%eYUw-_!iRrC+P`cC5f_85RCoDl5CVprCRt>NOjs>DiI+nO z*0qO+3rk2`YT8?l0)9CU3S5S&#Co7d0|E8?fxb}TR)F*EFBnd+d83bl8(y65soq&Z z8~EgqY1{_wMl&RPD0%$KITveIGQCrn)A#(}ejr+VOY(f&$-fw;76u0)WW#r%*HK-z z2XXB_=R5bZI`!)9)&6aHIM^|#MpjX=N5HC5=1mBy%YIKbb!4eUz4{9uY8lp>HQjFH zs|luD0Xso;OwBpqIt~uAkex9!Ls2Cl6X1zRt|8kZo4_ zTY#TE%VHq{*wLmNv8;OzoUf5t<`?=tTH zu@IW}GVpg#-2Vy*4w?weDIdqUoavQRlre3s;>6Bk_5cj_UM8ZH^I2h z&Pz}L7{kAZ=G-Y4>&bZ5(Y!~Z*MflUa^5UW;qa^6?m0EJzM%z?5c04gf|2a)B`QF_ zQ`?8I#aG^M%vZV8JV@}b&_{K-Lk3+1M{_K>1yNVN%??ylSW?7EDR#bplROGc17He^ z&ioah4JRJ|ychx3c8eP^to=Iv{WaHi!M1Hmpn_q824?b-oS6U}Z2-0e79>G%4Ip}| z3W?3QM&*yPXM#y!kMEu)VzWqexzEU`>(sX_vCQuaWO3a51#>K{=?qI)iu*=c%xRXt z=HCDQ{qy1dZcPsl6rTIb#v3L+<#r1K00`{A40g`KJ7G2kk65SSJK`HTumxno6$N6! zNB99!_F*o8)5d}5WI#4=cUEj}Pu+f?ER2hinG&RgQSo&LJQU{!ao+@R8#B5}w8~S; zygW5nZVIMmle2i8O82NWqV}6p)YYXl*(Tl+rNu>|M~sT;Aha-l>aI|d_K9K z?o(hmG-5(HOV(0=73TYz#-Sa902z5UPPKIl^EM(>R~nX^KN_+Ad^zvo&CI-qM{Rd^ zwrkTO)u;ltIq{Z!aZWN8KwJznq`KdcJOgCm}!UL%=I!T+o@|{}k%{SGMrWXdT`EP}=q=r0g#yAF!2A zFGcKpV~zmx4-PiYi9*Z|f2}g&V4M;gFpLQ0(#?(1w?o&T83zzb;K88HnJT5`Ju|!p z1b!1tS4tlc5qZbt`|0Gm3%6wHqqcJ10!}L+GjpV-wlD1ZaTZLPX5{Mh($Z4kOF>^( zHoxta9u~Kkdsutr8P~!54ZYvBef|$?mGc6K9AVcky8U&pup6mnONE||J;`tgwGwF2 z-bUXA!qf*#-&iF*GI{YCC&}-D$!uo20)z%$4Bno-yf4{hwtPBo_O0c4LEO(*q5>o# zCbiPbo^Q-f9*l4I1LS@yVmTSuy&y$-s1o2a2Obi#qu1Gy97-$4wR`}=V<l1nubQf2>ofkUvNm_vcWLiB{M3`H?Cm4pBbbkZWsxdqfM^ zCC5;oBEJg5MFU1nx%>dC6owzUVwuiIT10a)1Q(z`E`)XH<~j;Tx<$pCz<#`Q5u+J4 zhjr)>>>C|$JIk5so(v)1sPF(-#|g;REq&bPaxC+AM2LT$*}q;l?ipN}r|!^Sxu^yG zAXz$0I2^9&Yu&}~QrRY8x<`%Sgk_AJwlv_QC}JEbDe6N4PLw118UmX4Ul$2xAaFd*C13 zdn=lO9>6r8fx27tmAIIDH@SVwM?2fggHcd`9Clhb&%mJ)ruEhZ*HEaOv)-wbNIXI?LV%*CiDdGxI*9SUj5x1 h|K0nFT+ennuX7ZrsQ6j6{Q-R4zNvd7@0!K){{uk7kv;$b literal 176974 zcmeF3XEdB$7xx80^ezaJh#+c2bViNnM2}9i=w)<N}2juzb4uC6f>ht(*9%R$XH@Qs+L>Pl;gfaO9sG-qTz7=V0j->-tJOp zbOj`IPocam=|36X2xf`55j>VIz)W@RbK}E&hxYCrHg-CDVE(NK%9Geo!zUQ!p;_42 zxxye0>8G8)U%p|tyz6d#I{gJ-xzt!6-`+@{hdtwIIRn8yQnw9rmwER)e69;8G_*DhEN|LIH}?RHk(8vR{WU>J*aC;ywio_fd(`^Ujp%mkR<_ zuJlzYsWYiFGbuB>cXJDtTHrqA->Nn(sya3z(%NrjntapjR}L2P62&4fob81B4S^ozr1P7N!I0$f42-Yd|Na=qMShzpu2s78bN}q%wunlcScU-q!M{Jg;^!04 zL4#%;?bL-XvWiBZAA%YF*AVzI#=jEoU9#}k6;FQNsBSh)cRwG9{E#~l{xd@(_PZ`eWG3}W=U5v1Z9W?)-x*S*2buR zoT;*$`8{EyNE=``YDT=~;x*%uc1{2ZJv2Xn)j7x?-R*KYwyubl`z%Nz}DR ztE^_ZQf@8z4b@MKjprL(`}6odN*1%KRmpv1hxlCYUmw{2*deX1{^yEy(f=ybPaiWGqHl}`hb<>^vDj8L)?}zkL_0|R!O#hgA zMGs^NXar4|BckikLe?o%gBP3w)&7&gL3!_7D;>he;s{=IPDg; zeu&^#ma4<=q>suAsW9=^l#fmdswa=Kt$biXe+<_fV;tKjlWCMW0qzC2ULQ=@-$;Wv zK@dlQJHIk~wen{9tP65wa$3%LKlS3%i08?tXy06gU>_Xgl>Rsxa%@RasK+5esmnH= zE;C@AJrxzQZmpMtUnwA%LPm;OOzK9B-ZgZGQC_o=sI51Tm(UNnLj^Sal$zc4_E#gGspxR=pWb3ei#$jU^T&z{&#G+MwIt95o zQ}2l=U2)on%@o(ICz@aI^me9{ZXZi%ema{_ckX=Dc5}KlHaA^vbYu6pS%)5%_Fd@V z6hHGq+mtHg+f^K6Bi&hjj*Qkn?oS^mF!9orKY#u_ZJj~;RcA>9LD)Ip-CU?Yoy%9E zy(op;4D6M4;93J)bKj3h{&4D`ZiD*HxKSG+`QU@{KGTcmZ6eLP@zRtTfubQ_o_-%3)RXIm#=oa$nJ0=5;(wp~CNXHwTcptDRE4ZwNeBRrNYgU>@e^(>#_Z zb)0o-R>Jw$sSduSod5Hd|BUA&k*g^YT%ozWTE912WAjnd`}e2oy_$o<&sY8RMj+dpyb7k$Nkn*I?iDC z{kpseQK7BH?!#v81&^_&lVSe(entTru=-acM6bjTLK@})S)X_DY%K_TzPTi;>Vyci ze)b})M;nm4YdyZt)xh`{@wPCd8zqw z87nXb8bE0oe;)9Fb?N48N^kyipbz`?9<<6q@`;_FI_Bc#7c5rWEXfF`>)nconkd81 zP+;He3=3UbmYTH>9kkr-TQ|c^ddA-u6>ve#Xld|@7C%cL8&5SjA2<@%%v$r?#GFS! z5FJzwgRgG|`lAL|8wPPGID7T9jKbc+G)+_1pfxrJB%C#KPDxr8#Z`#PRx(|ew!P|E z>wO?XpI&5!lC$f3r+vWq-hs0>&TcqKfi z1y5(2;sAPvp~Z7xU%7$Gk!cr{OfrbwkNfDUjCojxHS#mkUs7w}agdN@up&;Ip zuO*V|gC-c)!U{5@>59(YgMvkJ8E8BX+nPNH(3Y0bW9`4wE1b)g4QEk#`^WU8I6?d3 zQfyjoCJn4iVM5otZ=6>nCEY^!w_C*jJSN4{t??Ot5Qeoe`%Y@t6#1C^1h%((&KJ1D z4R~snj@$e=`&5YE&$lbIxX(`lr=Ki$;I-`?aC`5U$|duRGSTmvL_A?5xCc zh^b*A{GfIP7ea%HliPG8*5`A_;8oIiEM1yk4sO8s?$U+}qIO*+IQh-QbI@=gG*w}$ zFjXcZ<~%Av2DhN%cXG6qH0U6nJDb#E^NiTj-}7q^bm_@i%Lt@CmgZaeHZa-#zW(h# z+*;4`OqGUM*-{C+V=Ih%-WCWJ3UM`fMweeaeH>3c5T1t=kUAYG97-xZ86EL7p@U#yU34nQ#IpY(y-JK zcxrPBiTNMFDQzPjyd@4W5JWb zH(S@d%}3poO@wwGINYcG(`0%cqmmDDo*JV*e11kv*BB$&@9{EYaP@ds>hR~xrT7Ik&nRi5C&U`K`hD~bO>|OL>AtaRE3q6enC7lPUk_U4V>s%U0n-(F} z9l#t$&a`W7&Ua^Y4g8mb&sOJMPQtqS*{vB+dNG-b7D?=4S(o+CaGaC5`#PseD&9gY zuY10NeghGCGFl{h7DHMMX22MvA0wIaPEntTj99DpgAB`Bs^_^T1T@wZ*+cC<$n-?c z$LSWn(GaKW)>qC&DLRwF-i$!Oy%pc48Pfs6t95Czm`|YogZBv?AAc{W3{1LbLjIXH zR)q`Ov5c;>QP^&XhHu4S_&OEki6(G>gKJnV6kB|D#_N=?2z_B~zUbFRXcMw*$vg&` zY1MGx3SF6qnJr}%#n~B5TP?pl4=V`h%}evz#^qT=YFMpdVB!O76Px&lJQULSM&BSi z+6t1P?58I-#0VDi1DEO>wmVAs39!!?hOh`IH&6ExOII^|h?h70Vn`p&PE~6&*_hB0 zdW;+LH2%$m^@OTASk~&3h{v+j{8I+GmvYI`YV+wTDFWv9Ip#A)7xrrf7ughd{#yfY zA1D39#$s+$#CR^tO`Z4=J?v+7-^-+0=LOFxg}19RRTyKO<%ku5e(K}aVIrEM8b!qn z#}zRo*&ezygFp_U%|d%S*LC=zj` z?0~q^`;W4UFpjwxj{Vt0|0hA*#`4eqnZedz5UU2Mae-P&9XM(w6tmek-?bSk7haui zt>l%KPm+_$WW*z*(;;7QJjSI{!N|ZDF2HB?3)gb+HEb`+U>1^LYpP6YNr(L$UjFV# z*-hK$)x7(@qN}fc>B((E6@l_CvlRpVH${>GDC-lPz~e4X161Q>$n8lQrifvf z6U?qBXX?hc3yAmYbB+h|a|&!c#c)b!hDHO zEKg}!CQ_d~uBAA#0n*zLUkd0-jG@i_$nlV{t(4tnW%~=`DFUYeq9mjy*>UAh7^_!d zGSuh}OMk&>z(>FF{`;e~o+6ck1B@O7{`OmG$>o9ApVA5ChjwEH%6XW|UA(4*VKqnZ zf4&7({DOCeI>+8j0>KwgoZ64ju8G{A^9$9!qE11c}`WVl?&ks*TwpH*72 zTPZI-7U#B^6z~keIk&PPlP&Ng!BWP88$9ZO>dZ~<$Ev#x9%$Fpv3DHgUz@q4kiDW; z($IivsE1bitC`_Ya-Di5K-MFuX1Uzv!AHFg<~I;rzg_WQebmy@@ueqCn?nFDVp@hi55v}esag2A{V%_}Vo_lYOh%JPp^bS&#UaQnV^WSczrS@>>!U2eZ69?^^Wf^JR~e$Oom2 z-HKn2x>|oha>HKFee>ZZrT3U1!s5FA`9kk9ncrsAksYlYMsw9Bqz5;WdrACA7Kc`r z7|XAZxb2r`$yR@tXGTXGXfn{~ie~H~{PW9LgIsTYK@Me7dSSL#mnZc`G;I+) zkp5EE)&6&fHlyCRxtwyIi4JXf>HY`z_w5A3u69Q>afykw@ifxP}Id7}I@XdG89 zAez5uULzv(G@ZTUIMOKh`%hAu_BcsKbd9yMz1gYaF)J!aD#U}`?+p8BU{X$upqJCS zpJOa+g#SQb1h?=V>SPg|R6710lTqhJvcs@lliQheT*D#t-k&wNHwJS8YVhYE6U-Rx z8f?p&852rppgQD7yZOraOHJ3+xRk-DUafYXPqKym1f>=VM($*TteNQaB{rgiK zlRpGMBSIL3aYjFuRS+leK@MY2z?@1qcs1xRoe88=?|_u*wxd+6aBH` zZ+1^Y{}mu*)e(WmoU<(w$uGQV8PajFytb^P4z3F1T?f$Ws88A*m(`9OhZ3H9!wT0hf< zb;C`y<5yWd2?A40PNDuXpkBImLS7?N3(GcNZ1vH79cW0vdPx-A6~baCOz@Q*>@*KM zj!5*gj@k{pwb1)#o_mg;A4hSjI&L_w2utvqs8cpoFTKqKhh~aoRrWijJYZ4d)W}Oe z6-n^Vc@*L`lBz89xVme)CxYgnSS^v$dO6mrsi-Az%M81^+{8edfkCnCwOP-IxpH3J z70&jxL+21e*vB^t?IG^i%2#w0FSNA`71FyyK0T#R{T~S#QTf|-4Z7?LDT&FlX$*9@ ziHS9ywD}f%$qqTqDDr08kc^<%J3&)p7YoIs@$y!fY2@H`f@wg8j(uzu+rQah^vi-C zzOJ$+*~uVa{qhsTnTx9I>}T;@uM+J|=B2h6#_Y0uW0?l+f(A;+!xi{@k=#nZ-6qpW zNJ7b!xKO0E%nAS|JQLeDHOX^e;gtNtd1EdC`?EC`2P6$JEK?#x9xU^e0^akUyVNqZ zlz){R++KvRrLix&F1zZ)Br17YNA7h8)IqZ!?_h8pH&#m&y zUM-liWU-|pb@^2%ThS1{h7##lZe|0NA|?P`>aM}x4-rMWSMv2N0&l}Ej%LfFrl zd;?l;gJsl_7t9eWJz)wx5!G4Xq{(JkLRRB2j@05XOt>9hlnnW+7O}g3$#;E(M`O(V zy}dI-!0BbpL0-jgzTd$*^!X5(dM|4dhTnMNNQnPte*^D;XLJOt7l{w7>#3F+s0JD$ zTO@rTaLE$rBc(qk;bvfy#Y~9(FhDP3B%~D$V#vN+6lMhnmQiq<@A9hW+^Udncx-Q` zIL~dxm}&f=hQN-zg3WXJK5t3FNn_()9JR|ep9#frsAy_C>il!ltz!^_g5}#0 zfODeP6>?oqHK}I8ONeeGRX6-){f)`sp&oCjj46z_yb%5~!{cM)3sy$3?NM}08l&UY zF{4<*a_-5WANmyetwOq)MJ9lJC#$UIpmM1kGefC@vk)pageWy;<>6z_B|EE@;kcVJ z>1Q{#-jW{*c6L8=956aWE37HIZfQqt>&wMt{fN%Uw)*lfNK*ym<5uAFX}=Ik#;$wM zpN+ah?Z$HD20pv0wLX<;XDZT;j}21QWW}sxKL*F>^rqKe#3)S2x%6GM;2WFu zsf{0G|9MOQhu{Z`3rkl%_faqVUzb`H=pAiVMPEv+{11-$pRMQfoZuT_@%u%RqW?b? z|4*iZf|ODFVG_EJW`#-JLoCuyO(&o`xWeR4oo*w5P8^!(3a@7>Oii0oce<|5unN(s zIn5MHlzO+D0gA3!8c4P?QMmCw;)a6LY>}rY!qbcJ-xkhi8J~fd?xUt=!ZpB4=XxV) zbpmmzw0pv-<^XuD^MsF%cpCvfVNy+xQFJ7U5x_`)nu9nQcmd$~VaI~WG#Y-VB8&OP zx^4I35h=RAi$TE7ij_sC%M!m{{jTstO#$HY8lk5Fl-8T_0!QLC-4qrowUZ6bHWxq! zRR>VSX}wb0@&B@o)`35n?qP$LWx4vyl_x@r*yjA1e*jJhR6J*eyk=~3 zU2d)sexz1Ll~@JtDXEC#hpc32e@6>ju`VrlKyP$Rbw^Bb!xkYz2ldPqqGA4p-hXI{ z;`&mwisJ;^0F&P%F~W{l3Z2pC$E@?bKDUn3ak2wMkw*8EA?}J-qxbj{^cVz?J)PNV zD<+q-t-Q4Qja2vH;;PAy`+$-oqj!xs8EN8(z@r#U<{c|{;qPxi!4voenMMi(RELx# z9y`OUR6w?zWb^iFeSlv37u~!Kr17#H5jY9G28c1$m>l=CJ_=A^B-H_;3)dbRUB|t? z68rHpW6N5|x1zo7dRuip&nmGDTi_#IFRu3+N-tHIy^;2+`uD8Ai*?^TY;Z=H>BJTM-rd^5~LJGOQT2MGPl z0U&c7g)oi=V3RM;L+(7e!Q11Ngkf753@4%quZ;rq(r{Y#0immA`$u6tfJDpN)rmCo zaF(*rbH&{^GXj72vz_3rJ#Hy&zIs)R5R(!X_$GJ(8%yxV?P0*vuo)^@0Rf!fp;qBJ za#f6Xab0&0$}tJCmFxpak+}-f5sxOuO;5IcyA{J*Pmn>G)<(w7T!`4bvyq9c?Etq$ zMSM~s-|Iyh_1ZwPQzx7q?)-wMMjcBX1PIY3i=xpus--j0g9sYXeCbYJ)di-U+U?); zP2stxij{a+`YFeNsOi=}*Wj!mo>ZXYpFHm`5zqL`Y1#nYCM=-6P$5l7x1cBeE}Vb= z(5{0JXXg!3Zie09M|q4%OB2BW@E{q9@njIm^lP0a5HqSYn z0+)^FvU5GiJczxjDP3HP*#{_StIMkM)^Oz0x{cJNiyQd-*O((<|HdBd)hqlxGf5Z6 zARZ#L+MO;pM^)-lx)ObR5c>G_bE%9#=$IZnE_tVX#VL*%Q3B(S-quC#^svLc1 zUbCxDMcSQYp55DqPP2O-;af&p?u8YFYjN_6!w$S_#>}md0xIUu)=g(>H9$h0tB|GW zzLgV)umd<3f|QP8Os7IFOvXK(iaU{$88%yQ`D2jiJ*G^sws*&zMBC`Xld-rdjHN;yoeilVbOa2(~%g zk>p)30YqOi?1tUo>jlI~6HYvneVf%e%bqisBxIVaj`2}|%k>n4=JuOP;|W2uITsiL zh=a{K;%%K|30ZrVLbs+3z}azr6g+Kd{bL1su>d)UqvhGk{{ExG9nSc7u&F3^pF?Aj z(XdT&b`7WSyAM1j)&ziYJ^y~~3Jf@gw(`g_dMS8p^)oHw0Wr;p{j0vJ4(bhQZbWl-aTC3RRxTuMqV32Hx6KR@UfRxNkV^=LLy1Nz7mlL35g9 zwpl$p+Ee=X`7dsobpgKKednlt%B+5iEv1=TJbNb$5!%-KZRD=$bG;U)H{W7*39RCn zg*>%=@3Z-jVdl3OH0evms5=sdeP7R5l^5ZQgFyb!MAPJcaXz;13O@wYc^yonJrws>RbK=cIDC^SE& ziUCCG0fVh>XhNq05KGUz2oAXCT{=Ky=YWiRf7tQrjw6789v;UA$lhn>Jg2|JZUisy z1-xr8uO^!Q;bL2O)dSaRKMU7)9aH6h&kcmnh!~)b@@NV#iyo*Y3?{xEC_e=4B$7y% z1(ao9vV=EM4a4=0HaI(N86yT3BJa0%`hhA#d(J=1oouhFdfa$-yo zr?8h=lU~#g&)$Cpi{|TfS~xUc>%YaBE#0CRFT5-TGADc*N|EKsUgTO?(iq+M^6*&?<{7 zLcOC;iqUdqx$Nf9%7VR@&}D94d9-w(a`0Iu;wibF50Nk&Ah}z2@;X;>?%`Ib>yI|G z>-Q9}{9Sfrc-x)wew)?Jruy1sIAtw`yQsxQi&STfW!uKMhaqz}2eqHx-5@f=zPD%+ z!vgTH{5V`Q)cqBY9JW(=kh{9f6AdJ315CeoB((?P`=hlnKSfLf-f!mY@SW0@dLWwg^pu#6q-q-VAz>O&B`6+Ju4ypb z`xGD@BPvuxXi6Hm`YHNe#y<_q+|@J;{42Ght(6Qg31U$-DE(+uzShgQ4eHD5?Z`^Y z?2D(bKeAjfSEkwg6|MSFWnEH;`OrNT>s0cFWz}v>+I(rG`~JpBX*C1mwo7zMl_MI-shD21zNNjB52L$E!BI_Qu4&+cAs3wTI|bxB*= zbe^bp?Jsd9)`pD3^wzH+N|Fk!;^JC1j2!F_{)Lc@%zVL?wEtyg4x0^dmE z&$SQu@^b*q20J*D5cl%2@DW&HHPNEP4lvAFL3!%QgpvXzTkJMdis*vF+59;@(W#z%P1E%lBh4XmTTDjPo=* za!a(5R1r|?8{lN1Dpq>a4QKqZMf7mGoXtKlhh4f1ux{jdny?HelI9v@cK$pY#h~ee z0@5cLB?)x*zP8c|Dc?n+;a-HF0|IVW{Xz0a{$juq6^=(d9YH3Z!UvXa*vLZ8@BoU3 z+cLnWwPgOMtT6u6r)y(-Tz?T2eAGqOfW_AMi|MDL$sQ0|y^0^_c4N~B2?#d%`;hp4 z@6Rotj$<#kT->pituTl*nV+Afu#6QiDL3OpdAi)HeizRgY&e8w4HV8Lhi9@UC=bbe znTw{&*v0BDfb#E}t)jdd$LqD=+sdyU($9j}<3B-hqxL#b7=AyVJ{tJ+Eb@iGJ#u6a z@KKZdSX=&vzP^o7rb6|D9u$^qgm*D+W7Ci|Upwx}v9494vTgK$+w;b335E4r1h>tZ zP0gM%eSL>+!?fV}0A&ey%osM-(zIeSl%$-F-{15)2VZYV+>Bq(W$9tyhafU?&#O~L zXN`_K_3WUm<3)O5hAHFvky z?lO(dp4WWsSvuUp=8z~aOoi#ZJ#g6dxU!A=~n>N9tDbK>Q`%jXVfvy@MJ zGp2JV0dej!z2GI?Z!!CBL@4pYpy{|^JBD|7mG#QUJ0CQ9U9DLcNLjm}V|`}KUq=?$ zctDLXS|; z_0-Nn${xFL;Y$JTeZbpbgCmz*mKY*m@#1%5DDQ4rN8@j+5y1<`z$f-!)iY;8z5~9R zX%Lw_{~qUE`&U!)YY!aaoY{wD+KVv(_vNRlzLyY?8qN2UTdd1*&1APzd^Qh+{;>%8 zuaVC_kjkz(gnef2-FxLT3 z(p$~^%vmW+fO|X`H+=1OY8GhxQ2CwJH?yf94k16o8Pm&*TI2RmP?;5nByklnyK%6C z=uyN_QyJcpyKvde3kb&NT&AD;speara(5xHKtJ#ucsG>od{>eMA6c_wwJxf}ICVtG ztMq00bCcCGlU&J^+J%&;)6I={(aF6WO56^}Y4)nS`U%h~oVZsrt2aVxcs=N7^))&Z zLkXa%A%zKW(S|kEY3=nC|5UAebwe*z(=hb}Ap}a9J9vWJ9{q8GP?1ytbhSylz;$eg zwEbx0Sy8v!eSO1!&^XP{GGCb|wev~_y$oAqI`)Xb=N?m0*>w_FkLN3;isf?gaXshz zGxujf4HimyllY1>H2h<-#M}$sZVo20hiE9o#7+pzXHlf&h>l5q*5z*J*XW2}@6z54k_^7?7FtJi%-ihk}Z5WcQ zzHTj5ml_dcT3xx3sW`MAQ9_c|nv+CzQ0=;nS~E9B4`)n~qu{h(2)`SW+yx1S+v>z> zW?V)Vbw5tF>n5MS3T3l26p?9w0@$x+m~ThLD1S{CIESDgQdRklAO78(Uq{4Hx0PH+ zFXDW?yc&br}e+c z`8Jx*6__?bI_Ka29>~;I6#%x5nAwv5lh*uSH0Z-83j6|J)Tngs|M(2yTR-hhJ_=PP8yld)RHXm^pp?Xi-HvmVP?sL^E zYU7iBSnbgYY_ZLE&TXeSP9xF;F_dFMxWT0>zJy4$=;3WO1^Z!EHBe5~!G zIiSYU#KFA5#eAmy@xv@TlT0N6+}4ND>%I8XaN*`PreU@eq6Y-|NvP7>k2;9u(YT`8 z@xAF}zhhc!RhliOoWyo}C6{?t^N~i>t5N5Z{gBDgx+v%AeG93T+|yh-=1dd#b&_j~ zD3)XO->85N9N#Llx;`^G8jL52w3x*m`0xXdluXb@y0mGa$^-u3+g~_RI9OPUag^O+ zig>s^Mn~s?JLdJlOw1F&SkGVQt)gUXks=fCY{miBd#o zJ_wG4NDajOrKTdGO7|1WY5von_ur75^B#y%9|mKdclXUB&rq^Zse5EGQiJ0pOC~-T z=a>vg3&FDlURhGFeMg6h2~8{X$e`iH_#?K}SYg}J*D+W`RWvg1u9d!z_-X_H^4`9K z1cWSQ-hYut{;!@WjbSENDUCEyH85HO%3hdvf;Ll0*3>s|;-r^j)Kt@~ne^1_ULkt3 zvPGLe&cCa<%-HeY_|9o#Yf2Mb*0_{>oc-AqxkFpLv9P;g9A3a2t(DdGOiFjHov9$r z&%(N+cyB6pM~L9jT~~RD)mGSg$Z}c{u0-989ii8wkhtW(urj^kPK?=Do`SQU$B{oJ z$g2_1)>3BmeQ~r;)`w!6-o9UqAb`~Vgd3L_55(Tz^bhe|fwq|Wk#HtwN9A03I{KYB-XPkPJhc>J28hQ#XyEhe#$1cj2F6{?f8qRnM00O`@$Qo zMl&U_v>|S4zgpBGf=8NhqW=pK7g?bi{#h|ZIO{qOyCvZ>$3di^OctS4z-pOZCnN#VcwQ90|JEuQz!lYZ9hUXQkXY$#j}$@(&etCe*4uzA7TS^x8(#g=y7yA zK5F@S7y`a&IB0g5(sl?_CloqtMG=~d$vHTpr@6rFrgDaj7B@q;%~=`r-%A=_%%1fr zziha>y=WPZ=7|8^y2xK>xr`RWX#{P0RGU+cJaJAYm-T1aSpi|O^e^%005}pHF6P|z z64%O>I$(fy_8k{KuB-q~x_}9I&LKjq#qL&?U)$U^Vx9$);RDOkfVYvK6@H9aR(s8y zY;|{HSkj1hWKm$Ks-I2y`KW+}DmpFi~74QRUR-@r1pvoNY%>a7weV0lD=2;)2 zs<>+=JZn}|w-kQ~72QsoqpJsco?jD6ci)M10rbCQ>F?QaHQOB}0itdL(8uI4z8S%# zRj*m^rA~jKeW@vsc@$AdJ6V$(Q4Ta=s59t|UJvQbs%@N^&F^U3Dj6& zpv&D$qg;TIsK*Hat(9Y{P{Xh@KqcXcCSw@=1~k@8FpW-bRm;a9Sl5#JXnzM)DSmJT z(9;uF**Vdm*iwNz#EF+JG*^ytL1btujc%C{gHpLl$A=nL_0p+~51@=;vr6nR<%W+Kc(zKLovYw)Z? z%75!fFJ(6V+`u_Q!I}gb&L(iE_{Q-#%?G%kwpm}qd7c~@ZhZPJ zcvZ^xDx?UB-)Xlt{;Z)YWrDm(v^s8rT;at1_6z*4#y4Ox;wbc zlM5h%z_}d(PM-FcT$3*1t9>xv)I=^Y_GrOn1;++$7$w}C`o4n>a=F|Cbh1WYd}44Y+UP3yB9b-fv_H4nek#dZ0Fv^fGcn)3>g8Gxt)~`$_oG? zkESu5g7~ybieOV$Cp!(6J++iPw&~7;<4C#9-*D$So-KHsg&8f5RavNm-roVZXzsp4 zf@{IUp|tGtB1P?k2wYnROs<(vL?fgFT+vK`eqS?dvJ-9G)_x{%7~vV6r?JPLlH2o6 z&;!wYxr_Ce9JdvO%?AVqPiH_bX^0)DbaYrBw$r#$B;6D~cCx!~t;|cd55Cv{G^h>> z8)JK5?XJ0}0nArSDd19;_Hx|7A9ufW3*rjuCCeSI1dLr&j=w}oHcSLPFV(f3{PEl` z=A18#LxvfZn=v|=>}dsP+*ec;8?FZY_BefP`D_&dihQdm^n{qXv^DY*@LFlxB0=F|CTRpQ|2w)?I% z=dun?MRmpWXO7H|?!KI`+4=@zI{{EPFKt_VzWw%|uSom%pbPBtuSTo$<#hVGz4o}Y zJ-}P@=Q+%cb6I9vzjZWZRJdYKI-)Zxbx(H}wkpAUdh0 zm{z3xUa=eqP4?Xx7xIsmiRzKR04xJgp;Fm$5?yYH)R>k{Q%veS;-cklPHT~af&E;| zx-RD$=y!tMx9#OMDwS^bvc}i6R`lpaXRc=Sxo)IwRL?nSf$}Nd*oQU%!ObIel+mPB zo_`u}%yL^@aq5-iE3ZC8kblL;E6j@4>Zi6YTh1Ibd?do#n&@{NC~*9XR*0q1HBYXx zUS0#xVB~XLQAV`=YxcUM+_PK>1-b5)^!N9kXdwnB=PRwoCUxeo?e#`>UB$FSX3vg# zXc_>8_;Q0(%}Rhqmf@l@h0><6e1-Ro{pI_++J3oe;GbmsGs&g>a!)@pDQ{~>HB3vVI-lbYUWMhOtjg-?zarc|1=Di+0fOg-v zRI+WgI;5;UVVD&d$O3nyX8AThS)Fx&s_;@S{P(*6Frs71@`e`4#glxhP_0M|t%%vS z8OAWc`CemJ`(qJ8XE%>hw*;=sDa4K~k&6Z1~o7nukvNc7jI*A4PS< zI7z=(B?}A?pg4lqC*pw*sf=ni=5dT#Z*qt8E9LDW3(cNQ zU!pk6#L+yTmTEAv-NwYC?+-{s#XdZ+&gc;^+Cul^I4MnNhKTcvd1lPF`rVSkWnH(R zIfd?bz87Tf0mlfj-300jfmPm~6KE$?=J@L^%|HnIT67L!e#EK7=FSW0G}lDiq}~w~ zsJW?2z;l6v@|SsdU5@flQsR!&$l_2V9T}Z;kQ|>`k_+q1F;*azQ@j?W>5NGLrFz;B zOR}Nk#69foxRSkNg0^(b0Uj{D3_OX1 zY)IDtuRE^t%Pp;sLP~-83rf1i4*ec?f+Yuo4wM&ZcLyia&D?7&CE@Jg*y?1j2O~VO zKCV}NU9+E|oZF#cc)gTYUu9PUv8)dF8}{~3)(U#a4w$~b)W!YS-1aiDdGodPi+iWx zH!L5RBz4`TZkY9ymUC0(!Cy!d#{O;EInC2EF9w}Ynt{((Oxfm#oFcL9@FaJ}_rBMY z7MutH4O>X75BeoK2_7@r9>c=pY<7(vL0_$C9PBGgZ%7|qoJ&psTtt#Q<3wQtvy0lM z{&DoZi$Rdi;B<3zUI9(Uv9&fWka&sxvGAtGo0v$497BV^_K3={0`a~tzeb;zU`*-v zi^a>6?{E6Orur_A(q0d$HE0b)4imxY!QO7Ua*NCy=a;|neddbs1~um;Y0#4sTVs|t z?MC%a+grW#rAK?<)Pimgfw@H00Mh}$M)$<6}v~tX}FtVc!+j9H2qba+W>I1k%QPw<}DFf ze`wVM29TLdv6m9W)wRtN+f7Dl-t$7|&O_w2eHWzkaE&cgulgL#S#3nowiSo8QZm~3 zx5Xabn3t@gw*4NkcQ5i34{ZlWq6M*k@R-=Pd)?jMcqWu>;Ox=by<3K&AJgQRRN^X? z=8JWa>ZZmYwtH8bB5IFP>GzgfozZghri_13a_J;+k*cv%*B;IY3;k;T*0L(X5ba)~ zEr=+E%Jwk|ZY|KC| z^|=6EzB~H{9d!1fk?I;dhefm~wObHx>rH1>+&C|k75e##qEd&!KFFy2VWAw<5p@ks z8k6`_a*ycVqA#9qQG_ss6jQ&-m1*Ui+N8OR*UjD(rg4zta%wZF=NeF}!4IVqLLwI1 zA*Xa(&K#tR61?qLFSXpcl^YC4v?q6lP8+l5C9d@kbFAn<426x%qCW3tjCO^aaIW*t zwZko1%u!qj`kM@Bf}MPYR5UiWi2UH^)Ti#$WmQ|#`}(o5f*HywdW!Zj8W&ru7a zx7v>$*j1koUcENz2^*TOsnC=b!oLBS48G6j=VfNADsm50;%W+u>Vx^0{D^+fp9FkC zgZE~II|6cbIM^%x%wB%rUPv6PBSf!V%IxN}%x{4{Ov67X`Jj!xfq zw=UeCbKm{qq{F}P$5TRk(s;vn+)H~SXlpctN=e-M?aZ1-A&+Wlx_u+w(_tj`GJ{z7 zE6PQp1P%%&+Lhx6{fOtG@>y~-s|MjQ3oUmyHV{zoSNc?!cCEHcq5?=~nh2dQ^|_LC zB-;G`Ge_KxO9Q*W%4QO-Pi4>SbUPTc}9NU zUM?FgjgAUGGrV2t?kAFE@fbFZs^$FU8W85`jkH6IrkPE%RLpf$*0?zIYSP5+NH`%h zhv19E_+HK2{s51wA6<_xp1`Jc!&!CZZw$yLERiFOy>|on@`$NxAJIs+fyI2lG~?xl zIaMy4RUvUx97<2^5C6V3jUL|t8kp@zaIr}doq4;-Ck|n5{09s9L=d(aC+FRT)641O6y_;B}QL%Fb$ ziuYDU^1 zoMUfmvE0L%l6y}k&4jka03tYk&|Uc~S8+|iW{Cie8oOH&BVKVTpR%$6BUVE|xtBhb zyGY4=gff~3yR}`Q{M?6&RhcR?CE~c((eg;Tey9`Z$8X}yNJ-_*($=(q2XbKKP-tzV zVMoe>QO;jt$J*IUIz{&hOhV1E(Nw@`=J-u_;HznKS+Iy*mjU;$k2U@iYC#sFDRIa@J1$y?S?q5_B?f z%(qP`%><1&lzgI8`MgoM4V1cqVJ_VFf7p8quPWEIYkUhR9gFTx=}tjfIweH9rBfP_ zZV{wA79br`(kU%cN_R?k`n#8V@3YVEJMVkO_Xm7qFvzu7_j;Z?u4`WNn$r~Jc8eR= zT|`o8B{Nj#D_H4_WDrr;0Jv}hv7pd155iwBzgZrAsZ|<4qpyXZ(k_Tz5|ktBc#JPC zdSw1YFvvq=$}!Z9&MhPbcP_Ai4*NMqZde{gJ2!dGV&qC!!`HqTXE|d317nO|k4;Hi zVy2r}a{VgF3v}nXncui`_jB{lo?!d0UgjS18vhh;oOyiav$oD;r(T<`hS_zUmOo$F z8Xp|bl$C4FGh2h^H}#9AwKm6iQJ8Ii>Vlp;-?(>(=tJFP;#7kl+LbxG_R96Z*r}N? zAK!0*jHhe}#2YIO6LTg3btlTgfJ)>&srQsK`{|ZIPU%Xtsk(8td|=B|s?C+&77qt6 zdIxSAf2GJXC98?;Eo!s{>ml#+pWCrY$lXSS#^Ru#xV^`OS+ke(;g_HhZWI(8sLT}tR5F;+~&Znx~|f>Zyg{+hA7fxu$)$xS*xP9gWutl&zZ+anZ0l7>}+z+MiOqd_4z z{6wWEv?_b7KMlP1*AZJ?Ikt^bR70T3DKDt@(Cl4wU!%1d$s&Fb=f;maG(}<2&>%#} zBb89CJ2q_(YL69fMP3gt{)BdL+4?eqj9L|6QWB6k+wm#LbP)ELb8y|9Tpm{-^Xfa1j+|K3)x$p&h#(v zw^(Mg1H3B_b6)BpehA>%+zR4q2~R?GN54SAS7?^1^kEj$I}+O_m zaBoe5#gX^uMloh&b{<@X1v3n|A>5w~Zg>ax$&lEsl_4qLPxs@MhIU%Y>agzxtGpyL zd80(c2>o7bI72>Q%sAm@z?0(kaQgiO`doGcRB2r|w)!1cxWvfJ!vTgqwT>0|<%e(s z!a;m91}s-N$iITGp8gn}z<=M`;4VNM^(kqM=&Z@F<(g>M?qx=|81!oFK6`E5Q|ZUb zk7(!MO}S#EqnuxOC}+XPv0uqKzt>quD2C=W)B1+=J{d2j0YNmH4ZEjBXB0uEqJE(P zHo0%Sx>1M();hZ#{vNyHlBAc%gU;CFn0KNx)ZUMrVp^HcIt@sJrW;0T0*)KMMKE6x zu%AK%_}ESdCD}xhhS~So>%QWvX2}W5YiD-o;EzwexQQ>*FD_kBBgpe9wmdcdf(WHN zM%xVLiC|+{x+m3GsA~_|RJ`;VE|^F9VKGp+rfn#E=Knm^(lQs7K|siE3D~us@cI z;&*(870ZFpA1?Skv*oLZ8qX@rQG*3^1dLxenr^T96b+d^VPU!Oz!|FL8{oBx&PJYT%5U{2F=@8qy0CbXACo_AbW=@Yd^+C|-bcSwGIrF-wxHc1gFTO=@qi+PgNBp-?{ zZv2JBMraNnlbIVWh6Pi#7sBSVM@pu7j4YlL$-y|>8(&}8`}JG0_3o3Qt*0wNyls|%$Y3+@r8@W0nhK~D>!LoRe@`$Yo<9jt3r8+!%!F^P1OQ|jj8HCgOa zZX+`L3|U${dXuvitBR8@lWJR*d(gI}S;&9l(648Ta8doLlWjQ zK|qDvP^U625q`my zjJMct&bw5CRx20Y5DPD8->0NzmID7!I;K)p%CQ5< z;$F=c-7A^ayJYK!nEqT>c$tM19g&0sw=gB8EN(_^Z~d7;bVa6#u-%`!k=MLZ0nSF| z;l4KYo-wM^ItN3Y?N98N*jFwy!$306z!e{ufirOhrs{Cd^?UJ@oD#AjB)eO{#WuI_tOmKy9H zs|iX=4{;C*P$dLdyar?W^(>5?7F#Yn&7n%thcSWvIV)5Gdxbhqij7sFt-4D&wb?Y* zCevju#+!M1%t;o*lAB$ZOXoi}Gtb6R2uqEEt;ifT=u`X?7@!5&`Rmbw@VoDd(bI=~ z-`lR>0S)s%N}cB2sx-$rHKZU8HJs7|7vR>;BclabXAdMy}_P#s+5K zQ=FQuN*njqYOYve{CU}2r`&lNTOlr#tV+?#NV?-#S1!tW!({x@$M5U-bB@10l7~jo z6xW94d{0pJ_B$dq${_BZD(=Ka=~yAk@om38{}QaRb=SXSsKd@G#yy#{RG0hHq<@&3 zOx#-L>dli;6kIDLk>fHFV^Ocs$5)}!)Saii#~EIP8(E)@8aN{(9EsUBl{$TjN?B{Y zzrPSB%`{y+=8|iPE{V?%b$jpO$7LRQANfRkXhw|KqAF&az_^MWq|s_BRdA9LP_%AV z)|VweiC&GS+3ux%N-DMM8H|PXdNJ?Y$``lP(lnbOkKzu%-y!dyjXo)D-;?G(dNt8- zOUH2ot!(!i4*EeRVyL!%Xxrl~fV&%b)LgBLa0-!6Q6Tg`PIeEb*elh=R8NYpWp11h zUcH&WSZpM*Z|>0BAG_!{9>+bMD%T?|OaI~2x+eqm;qw2Xb!dO+suC%dXlD3`o~ z(LW(N4Q>s&q=u+ zyOR~0ZC2f0%8Ep98j&O8Og~0XLOQbcuPedgyf;A^ec+?x{G(3k@PN*c7%Gh^)ve?YSmVlN@pev8 zqCfWp*23{n;AA^%xdsL2jK|vFpqZ1QMe|N?CvYi+*3Sc7iQ}KTCy~`Ep#n2i=w0Y1 zcx@}RQ4Ss%CwyV%85lwyI&|-TDMC7^A3@U#n>Bt(Ag@B6I7F!8V%@1HCM8hR?6j*a zVCa#=R&*J*bpAjEZnBXk-#KOGY?||9s=4_`(kFkr$ zrz#WUY%)REXPe_R^v+7)5-*~$zD(Z~g>6?s{lJd(+2`J=+cH~{g?QICF0nnYiMECl zCE_GK?o~bwZ`1ql*`)rJQ1K3l&l30J_&+8nU1Eb#&jxlLuBQe>lqrQCqbexjGM_h7 z9SB&E(csuz{QUl{gPTW>jO+CWyWz!rkQ>(Q=BI5dD+=bdtPKXBt+71GPbuCN3H7Km zD{LB6F25SQPg3OXqkiW;L%WmJLt~3tWs=*fn;EY) zCp~MdQMtlD&0JECkLW^S|6{X<&@GNj-qVZ;zAlVOpQZIj-F*n@z4-~Vt(YXcZW{Mv zo$RUwO8j3rI(iZBS*I%?Y)X-^~u76~bsfdl0_N73;2gLmX^i9!gqXe>n z;ugZeZaVARBQXNG9rk@iowRRLLbdL}HjJk6A};Y00Lv*ot)$(M)iXKEa;$nl4ZPYqJrCna3yWXi4m5#}a8tFOh7&ELI*2pA07Wowm$=QW~~G z>gAg%tQ5e)sKKlLG+FnkV!jKs0>pH+lC5)+E~@ozXNB5@7O*$a=nt;uIky~rKA zJg!UV&3Kd{&YNMgu#$}=-Vp#dfG@uuO(*uN@dUR(H6Wh3w#eJt1s4>feXRcA?~bc? zUe z4!-Z_$GN5Yr5Ml~D8SaDrnSaxadFv`@G#+{D^iMc4`t*o61-d~SLt}DFnOh`W*DYK z==+nL!*>?1BHd`^;Eg%AYR~E=F67RQ&B$yz%_1Id^NWiIJSQSE5Rzhy+@APaOlMnK zx&*J_p;n#(H8EU_k;>@4h^~(W1#C=W7h7PgSkk_K*84w?(dRI zX1mNU2-EXi2A+mJKru&H7g@*eK_UH;C2+2`zfl-$j-n%UlAyqnU->cn-b8%hO{Vjg zP!?7CzTg1E>O$&*8^Y_l(outxWE-ad?Vx*p0FK?I^1RRv>PPPN>?$?h*n_u) zqL-J=R>QGleh0GOX`S`qiR9`n`ly;{?%q1Amo;luk-(glH8bv1MOU@;yM)hG80H?c z1khmXRY-%8;Ij(btRu*AuSB%%8+FWk{G}mK_~Pah|QKxFblCH&z~5X8}(#=TR?d3<+4Oc0Bt}otA}&AW&6p<#<4*) z*qXAp`erwOSm@4Yy$JM?iVo)oGFhT^It4lhji3aIENq?B#bTh62_MIGaC=51|KNvz zx?d33`vxM0gu};3zim|7&>3-`2gh^~U-}a$8d#Kt@(GM;;o-5M>a5g_d3{ zS#Ki8A4kB!i{^V4(f_w%{^T2b7B zVNHJL6MQU2ZB|Sjc(n5b$ROr7NzyP>G2 zS7_`D&FEwwp~A<%k<^eUx~*~#@rW}_!1KrvSQRxC(@2b*+BhoB$LRZnxi(l`mK0qk z8I%$otIsgBQ$%wat3z5mdlmWS>skM;<|N?N*TrmW#a`;A$v0U>o=&!ShprvWUt1R;|RYNqv_5s`^+k>;x?rie15 zwa$zE@h=odRc*w|TJ@gvw5%IL&z+C+`|kynZl3uEie!@&qOa*HwpwB<7`k!Nyv!Wd zK{|RawZqhxV7MYJ7=XU{#im3-M5RnL2a87Ek@87CYEQVX$+y0|`OQ^%GOewp`EQt% zZCh>KP5=QlQEV_kV$1P2PEMR&7Cc`TPrst-4LIp$xE|{9pIK0>HN^tMNFjd`rtL?X#uk- zaa~-3+Q8%W(c3Q*+88Ipl8Sec(9w~a;R2uF3u_RcX4~WFP$yU^nBbdm#nMH~W$oTH zkdq`;1ek8Vw=IzjPh^_gl0=E5#Q##r?FXuDS0lKQiL*u|{)*unh+mn;-5-&OG;B*r z(e3zqWY|nqg}G~OyJpvREN-1DoCJn)`I$Av2$Nr4_B>@Kel%Uv!@e(cWPTl2rhwCu zCD0LAiG+jP)ziXcC_vG&xQKPjJmE%&WsoeJAM7*S%dU6%vJ117%}V5L(T)CGd%pOc zCOOMXLuS`5x0R3}^PxE%L5@r+g*mfGifNZ^tMyzdAgQN`Pw?5^Jm?s*5vm`1ZPS-h z?m{k>cQyDFiE}??zhdZD!)`N6w(nEb`|lH+3$L;~H&Xm6jV^xhvE6>$P)X0iC!fuq zE=yleWpFIacr0~evQBM#^1e6A9V2`yhxS6d9(!1pubP zoVC#>tG#~4-OR;w@eIcaV;6tZDyvk{nT#%l$xXj5iCdCQt>v?q-aT&n9llU!`zxGtrk2oRe-RDt_4r2Q zRm;szOQPl8;9=h%JMVfjyjfc|+U&SFKkeGCu2@btE?!V*_#AusZK$`hDB8jJ{n^d+ zEEsgMZ|c=bKLk$xG47E&-t>{VQ)KI>G2T431>3YRd-_J_tj>NO4Mj^qhIB)3 zTEA6;dpfHR`N$l}S(d5LTUU$WtOi#5m66c56!IlBnP# zr1{Y1+>12fS^OQL(%d8DRpVvp`X8O6iT)dy7x9SZXZVT3 zU+F}?8+4tohv(c=-4fN9f3_laPnqKn0!9{&>jEeyIw429Ud>T%Sn;Z&Q|F7m{d$q; z6NdO-=_=ncEe04O8SW41tc7Fj8_5aSU9`h)j%bQC@soBMl;_@8_Uga9OpmQg5(&)p zjNiV0$9uo|*vTp4+PbfYUIDR23S=q>ZI^Dd^U{gcM^v^V zt*s8dq9!XcXu0=F`@?R^4Bm?6eedMwL$8F2s!un0dgU%`UZNs=Vk|Aoe0riOpE>+C z9Fns(*qYW`ZQa6lFL7GfMPj@Sf~UhVO^zb$8?#!SPklVn_xuhT|0Hh5z#1(eI!c0l z3B=4%Gc0IPo_eG32^~)9m6r!v#h>gw{z(1dlP;VA>X$lP$Q)&XnAE58WhWsYmYb_R zTc5Ol43DXmLXeOCBTxofeiYR^Q)26vQvYU2u&-T-8qO`?5Bz6*rf-Ye7`bT+WS?GX zdU@s&!dlM!UJmScz2c3ebtxx=vE@!rOz$H@RUzkD(@inY;PX@K|6q7}Z`;@0DC>+axnHws&9^aTQ(0OjQngW4+&AXx^ips9d67pS zXTN(nUs7<>5Sq=hDoPLajtZfS`ORa6$a*o}4=Q6mC9HiPJ}~LSgv0_E176&?jaW&z zvi$#L4gMyB0|;wU8t}n)sPz=?%+=JT^qf=$b(+a92jUL1c2tL_a)N%SccdH_ReEYZ zuzw4G#Ln_deKjQ;DL*6O#atv_7SB5b;hqRnrO>oLq#rWyl8FUtf1a&E36@3t%?f!3 zdTx-#`&0h6c9?lORd!y(mRmdtbneToyjT;-pFj%fr)S1c>{I%Y6d%3vJLeR(?PjD2 zYdHUyJMO~M61MfAl7W~HV$WrC%rO%XpUX6Kdt_I?^VW*c72$RY-^{HErg~>{^;og! zSQ=~ubvVG%p{y=Y4*zO2kpq`Vq+^@!f9v~ck^~QEil3G* z{@>ncCTu$5q}#))<^R)1kua?9=l=}?G7BtK`#%rB3V_35^rV{!gV4OJFld zt4)}uSFFf!*ntX+hvQ0S?1arc3B zbptsJ(QpFi-@rU<_fdsMkbeQs`+EQ7YarU*M8s|3feW)`z+q5!H8(fk2V_Wh(EZ^8 zSX^r8M(S9w=luDhN(gtVzmmY!EFc0n`YsP}j(EjB6s~B9(a_LBX z>ozs44-sIz!ut!}2X19AJ~>0BXj-2EATk19={iG>jtyOIpN?XsQ@7Gc{Qc~@%-?wb(H7kTUaEkDD)g&lbLu1B(ayFYl;Y;a@zXBPRPWAY|897yA$$+(JTnurKHUK{TFthsg`Q9#&-mcMEd4G$|?JMYkTVSd)z>-OHspmq9|^0GkT4wFnz@Ube;R(0wOXHCh{J60?aP# zVccWT$B-fhQ+sQIweqVVFO_XXvbSzsUmP41K4bYbp}t>`WN=I-uecE_w8Qu-g8N@fl3>p`VXKP@ zHO}-{32&`tqmlyI`>Y&o^tf=TQ@DYu)g z8^L|9Sdc#|zWC6lk|sug+LzrI>-s6<9oDrbOL0f4{psL4AZD@Pd$}AcDw<|n9exl9 z6PSzf-8a2{eD|q8p27P9ILV}A25uMV477kZc`q`pcF0Mxd@?F_aaRWOupskXi@U!D z{NH8PDVZ43?eQ_t-|%`P#qyfzqd=wT6lkbfza!xO8&60Bye8TQ(H`lH>kJXs!{#RK1^0L1jO zXkE$J)*a9uQIo8f@vl>^NED8)Oe#;q%o!+R2rUc&dx7!SuV7u-r+p8%RKLcd6w-cu zfI2!xwUSIBu^Bfq3S`?h{O1N^V>Tr*pR7&2bWy5bPt@~@jM1t;l6bsQh2z3DJy$kGb(!G00>?o z$FTzAy1A0f|LU4AYC+IhMAZdd4&x|13ehb{wWhdMjPJ3oS({b6v3~)z>Z3tj(5sJO z4+y8NS#Rh_Onj+uWf>Q@Mkgu^7Y$J>U@B&dmME;ZfiXnr1J**oRHRQek|@ z`1xsD{|MNanaL<{)#*>`E309;Mt4jJQY@oKV6xxb+ zfON~pS7@|`6~OuD!0Tc^L(5=nITya0x};eNCS-VWocc+o5;t%4=-1DA#0GllBi=(Q zL}WPm7-}S>XX;O--kV9uMNn2#%F&{}jc9m7D;`cwEe?U;NsA*$y)|h^c$B+FDiud5 zj@x>7COiM@=g;Ql{e}6K&w6Jf4yoJQ`-0pDrzskfylC6vnQVY@8Bm9n*aY+PGEW0N z8D2)5ols11M~JGYEY_1CRZ&beM+AWtB1x}Q3-3<-ciL-F4B8ECbHuh_UD(cMaksdOl&GY&f{` zLEE75P+SLRZpR`7KBf+|1SWA&uSnf;?h5U78F?DFC@o4Og)C=Y%Ma1yh9D8|o^b#9 zhG6fjmmRL_g;dQh@cYpU{*r7!IVVQ`E2bOi4)egWjRWj)wsaRr?;p!YQ5Axz?@grx zs**;!7O7VAr+KZ>ZZ@S=GECLL9}Nw6J&xD+X@078R#=Ij?Bx?Uef}|bB$#aFi)%;_ zIup+a4c{oM^tIpJS)F%~kW)s9BbJ_VpN3UB;cdReXiZ9Ns0bf+=Sq%jOowDQ{ zV`+c1iRZ})kYr(|PF$CwwP*u=Q76aquHTzrO+N*8r0fmEoE6rU38{e{(@xkAr*R=i zT=Z|11%1Q5bXv?7EZH~&E(f3Ot$6#s*ao}X^KR)r1*xp!YU{rS4!&?KV-Ze`3ra^F z<+4?Op`HXCEMk-|Jh?VG`$#oZo0daJ9)0DSq%376T@bnw?tDi}kIl-G*mH0h#dUTT zS6C)`_tf)>r&HsxEOC{RnQmIWD%WxomF;oJclL2)Lavwo@3mTd{8V7m?$6mLhxi~} zMze_rlhnL}{bz?$ADOmu{xx~rR%_p>{Gw&D@gs_!o#s$gJibAenuH3{n7dS;s!}#C zUa|(OzT>d-+Wkm>fmh@Cdi>2<5owe2 z)N-G}Wx0>nzh5I`aE;=uoE?xp6GTvSqq4*WO}}2~p_GAYJnHxuX)~boj2r!}UDG!7 z18pVX`G2v>uU|-_7w1;$;_BvymTYK_2$@_x`FpM=yFxAxL&aVS#a29{X6H`{WJRTcQ)sQ=D>+F6 zfs)^wzJ4NR?H^&&AHfYCxw7=L_>xllGhae9_R?v{;`(RqPuem1<1aIqA*|n#e|_f@`?uY~3Ub+2S(;P! z`q?Y)=r>94v)B4nnoC7|IeFFd6A{rTTH=67neG*t@s;GaIB_lNS5euUOd7}K{T2AW z_Fi&n&rhJ`Ktj?SrNr%-zT%)&;_%|X%|B}@$|ndN(zL{gImLc%G@|`wJ-`X;Vh6Ow zv6{oA@UU~hcvVJ9W&5Ug3DXX8Qv_JUy>okK6mj*FxdfEOta*3FT*%RkNUWJV-;zip zWQZ=kIhq({48_r9i!Hl#KxhNUL#YU_W{O9(R|3V*ru(d&_HjLA%_J+;)Alu}*dHG< zVhVr7^xRwV=$1sDIX8akk>mS@qG|{=i}u=yr{+t)mfxD*@1_2G;8Cv;LfmL)Bv4s3 zu}5Y0xm4s)IOW+m!=FQtuY^~emZ0PSl0`>Vu_76Za&t?dA#KwY_3OL7=BpavT7)Q= z5K^g7&;{|9#m@t!vUTo-o!MS438Ul#J986XKl;CSmGJib;5XK3*gyMty0==MF>g~I zQph;c6frS~5qCU#O~m;)A~I}haK=Nq&~7cgW$7)dd^uOMf|<{qu{>!a_WgA5W0QD> zn9xs7w7aLKK;Mz62T7wL&NreKuOqsa2-EjH#fWB45qHB^u)_$yD~T#P1uXkSFOjfU zps6pmJYn=Zw6)SM(iw3B_8zJgfsRPPrlCd8OS*ZjkC`LQ}*LIQ)LHgW_i zL=sZLB;an*ykc~|^bnKynZiG79yzpKGp}<=cGOjDd4^n50ijry+%sP)C1nB2_~J}n z@ri0oAR_1eZwkhhoB1`RDx?t(*TgxoF`uxZs#_(ev^~aqNWPD)xcX zlq&f{!%-TpsHRkXxqq%|KiY$_LukC0)tll#>@x=-RXPcb#za!~toN1wnXI&MrlC(G ztV~1o>_7Z-Wc{-MVFwuO9C%2@QA)wW@sKQZZt~wp{I8AsuU{HRwbynvH~_aB1Htsq zNxuJfo0*UvG|Kn&yq}Ij^=h>HX9#F?aH|70A=T{h>iU8iF6u%reoX(jmk59RRCzO< zvC=Z{KZg0&Xn&u&*x)iL$;&d|r2e<}rH1tkVwc;V8U4?~`t#KO_y4UZkCY$A$3#~4 z)8?OH|Lc7J`;qNZ;*`?0;c(u3|MeA_sNPY}B~sy*aq+L}gESWtnOXJ-z~eZMg=Yp7frB?&4Q&z*18C znim++Bp!f3?lmt6$6}|-z9&iJ>*lAv`dIuC^j$gBp84_IoP&&7^A{ikl}Hol*CzM^ zAmJlSO4%p2p1*k+eUhbS2wP0{#s}yu3zd_U86{D`EJArf4)lCnk$c#x;2vl=YXQea zZ8w0FB;6Uf|1d{=l!r%rwhCrcF?d?$4$vTF++Dn$s&^Oh<9PjZdEDaH^>o`u5K4VI zMlJ!VPr2!jx;+=&KQCPSSIR>m->MDpnPaODN>9!vCV}vswi2-G$zwe@r|JDv=TtN6 zj$<-!bZt@U4Kg$NdXsbtTZ-sidkQkPLl(3wb z)1;elWrQ%;Q11|=VcKB)Zf-lpg<3HAi$W9l!O0d7;%XyO_&DmigM>mm>;AN`1dO{@ z2)Iu^8pr_mc2c|TdUD4l>*nPrn5sO?XX_KdGoA~B{Oc)^q2Kj*mKNAc%?d=`wrr%@ z#MCWdN=ynb9L2l>XJ{F|j#0pl-r}{ETQ8BNCeW4hgmk%sX|n^eO;Xd(1&J>{UT?{Y zz4)C}%ul!7pAAGh7L8!~EWu9#6$ITWs%sLvS~XqmiY)->FxyQfNoTd4f2w#sgLDHF z7z{ZsNB!?{BISj>e^$wU0`D1vd?e?fIc8Y&@{I4Ac1oXX~4Ho=*Gr(N#BRqyc zc4NcYg{$pwJkDUtWJ(!Ya3Ah9VLF$-(lo=M z6Mhm{=4!GwA9j-6u;l)<6uZ$q+co^vB&7ox>kKmh0EV0!2t1yH%-$VRY7ZPlsk0%z z3n!TuxK)e;7blHCg#9^~O{PrpMonPCKMso5xrlb~$@>%$jQ^tO`dd<~<`h*L^$bk3 zwc0_d0th&i3xNTb0BEl5g++?_fP=|g+T!g7^&!*fXJO~HFWAQT(L5~+j`lQ1ZCAq% zA65L0;~%2(6LyP2{J;WsI$LM=Z^R}-j09c`qm!^~p_2MA9J*RXAGLF^YjDv-{gde; zWz+{c1DeY|0$siaFCdv7k?js38Ep_@*>01=vF_~EWZl`GmA`H&@qkLV>#8(JuJvt% z(49Z|JD}{vco1hf$lm5-EHyQ0u&l|f z&X8y0^-c-q8%nP@m(z>`pUw8_YXS_-O-w}6&H3lr`p+{QJCTEJN~;GRpXL4?VH*)r zgAJHNgz-lK6G1?{vXHgWCXGaNSZ|ly&y_|q1jg~lX%>&^m(i;n^eUPeS=k}_-B76S z3Qks$lyP`6PUAZNk<7%z_hhDQXjup(Un(Qz{(HZN`R;5(0M$LDGR!o$?cF$Q{am%C zv46M~=Wx(rVGK|msYr;ms_^pWpiH1@Tf7>1y;Fjsre*1yh3p-~#;>LV%#@ihX$GYi zgHw)kl<%6sav3n`Q=tG7SZ&8QtPY+EXJ2SyH7zqeEsiAgeu35KhW4!mz?4Z_JSJ&P zJG4(`2LFyFjw0KESczUb(O86)?X-t-4%V1da-@~J9lxJbs%_0MUsbqs4s$>79Mi;lmh8nKjN#IlA_!3Q`VIPAH1= z5aHmB+{xFnJ@qRahu$MVjjB`}#=TNDN(E*T12Y}9{$QuZCk>Z9V?6NEYqg0z8Mr5WoBA}U(NYFvvP!XvDwa}u|ZoFIYw^ZGVc_O+H_sj7aPsUapUb{qLl zSg>W_;Im4yTQcx^1?RXNJqZ8Dh43|3@1ww%?U5=7Bxq_0lKTu|S8|Y(us~?m@o`fl z7UZd!Np2S*B)W0?kn58Mg5JEM%7@tvNs4P#U#EU)i$&?EPedyi1Xqh|&ZuwaS!_Pq zNHz9MD6R>yK*NO$s3#MwPhLcNao^EB>=#nK`&$d3_9@$+iz3+ri7?g_jed$_6}3aE zyBf)?aYE`{THSXZ$PXHs0(Hu!aGm2DO#e$sx}6k{E>RE!BvobpVe#_oLag6u9`-}? zpAe^V^cB25+_3>Zfj8^9ZGr{@Zi*di2_=PAR_Dd{pY>+XoMBC;)+)bX1q_V*U_#p) z1lGAn>|16)4G1vQj~i@kA4*cv zjD;+6_LI}WJ$%qu>roS~{ETUZ4>(UGxY=cX%?VOMz&+}Ze&AsJMEkqZtP@U0;LtmL ze)w)-gH`2TZ*ouVIkT`z8ZK#C1KUuf&nZUsJW|1&V-Jux95_brf#LIWk#EgsBCS%) zW0s)pwLbEl7N6Ii67B;V97028b$dTA3B zkF<+f!(R6~-pul4a6hK{21EEi9KAsB<+nK&ARRqY_$zQHyoZIpx3OyHjwmveQn(tl zig-^~u6Z+Oo3>~Tlfs~WQLp*muib-?kKL@6q+?Hu;Vr3}CDKNP-&qOQ^M@UFD6(pp zDy<~Q8;`o8;g=&{d%_M=lS!a-elB6<*`ZYII&OjzYE6e^jbuDPH7C-24xh5oBjX&% zB3p1gz^Q_1!#y}J)iYkZLdc}B6}hJrPW1-+LEP;Jtrx|LyjL6iZMw}{4qoGF@~3aM z%XdCb=dle=^eI;CC2p4E4%GjZNgRc|+4V!RB8C*Ors`@!WHD({Q!)wTdV|Wa?BxU0 z9H9H$gX`bSI2Q;972)Z+UwW>8F!w65+I{A4B!b&PYa$XvxmplwpG`Z zGV@(YiksZc=CeVPAJ^*VLcK1@h%ke_>D|NmvWz6 zJvFr#`Oc8JkKR^^>30~o&YT&<9-Oj41PSp76Dd%172?E>{6 zXn&~|b#=D>ujA_ymOmT{)Bnhm%BFn!WhKP|V*K3i^om|WzC5;hadY)x&=s z!l@+9h8@z(ZPsWV7tb=F(+27{7$Usm4jMG?)HF19@~4|PARaSiX-ve&!9Rg*O7J0D z!_r&e5NQ}zo`(6B!UjbdiRVc1m&bM794v-tHn~Q7Frp-`ILh-0A;q3J_Oo4>aa1LN zn#&g-EU&kPp{N{E2KU5Q4vG(tIS@4D6(uMAF{L;PdN^a#1+qY1Fj%+XG5>c6M~?v~ zg=0zp2?^}~lt%MKBT0X>D~vLTcbrk$l{+S(DYz>LDW4^h(_n&#a{yE;G}6VPB-pFu zpf;sG-x=k_Mc_4lf8EEeBst2uGnRX4Vd zJB*px6B9bAxMtFh7%xvsFY(Vgc5&U&VT;l%LsgrI#NX|%P=90h55K0q9@J1*LUbC} zj7@YIr+xh9dq`!)lUJ#sOce;NUiq0MWnOIQuxv7F;Cji&X^^eD36&`@;osF!O7V=DyLR9ViXEP`2?qOd!sj7&89}Dm~K(dHY0BsgH^f`a59xJ;kD!rFh0UqF?t) zy~Xc)FOnNXtxRCr38)4#yA4Q|tA(7w@F7^;x}8#@(wYPvG9&D97D4nf=`{ZA_JR3i#!cO{ zFsS`1%}~_XfaCWc`=Ex$BvHU6Y3=yV@`H=Qh@gfM4Vla#4<=D(=n8T=savnZdYVOs zk3B3?AzGXS3dfcSzE~~xD3x}3`y1z1!hhw)9>4c0A@Uz-u)V4X$TkE^y`6zhwiFG! z1zwe9HZ7SCP*oud#;gxqD%~EPG9xEI{NJXfJz5h%L*74^`pDP^e*k@h__>jN%swG{ zGa6@D$@oRWs<`4!13qjZ4{n`K})`YD##>^6S@nv=!# z+JgrJa*Sj$xHT<>*&q357&fG1i2Lwjj1lB=v$*17M}*w4Hn@mc-wpg3KGU1(UiJJ1VP*| zJZyfZBK8xjzLkEr8r{tB_mSGZ4kT12?B395Qo(8?d`xz(m`L2%bmtgffs0yykhg!Z zpD0XDe3-W&qBaR1PYy!p>MvE)q!G{)6Y>{-+|>Sc;w{I*P}SI9?1LMJ2M*^P;FdH| zzA>V)6oHCEKjTttKL66XjR6%sgib$MS$*kAfp=Jmv!d7-bxVIXEMl#=xTC?fyL)pL zbDL^l6Y*HZn}NoeYHYx5(1zMeT$x0}RfuRzv74+~l#ib@V`q5%uQj=f`M@-n(F_Cf zR-GNkJR~I46l#!ZW%iO2T02?5Y8s40ZyqM%rcr^!xH@Ijw_#u>E8HtPHg{exs1>bn zqvn4y-a0^rul#A_`sj#c6w66~O(a*Njnpvx>AYJ&wZ_%U%)byx5g2mkK^po~nwnz- ziE^L1U$KniVK?sAzc0XlV2Fgo2vG9WSJjEI1pPZwvO&Mzs~Hm%E1VlEvzz}BjEg6M zBMRqhBpaK{`j7wiyH@&V(TYkI7QvE!D}`O;{}U1Y`-w)*k^mm;-wWHg{}1T!-w;4F zK^VL%n6GR6-+;fh5CGc7&q9Xp{tfZ{`<4D26!?kI8FpAXKdb5dk3ap}IQ!9IP-8t! z>P`HALrT|Rf#BLZi{St9O8xaqaKQlTYA?%k!SLTM#TQ7j`xl!bltL?ndjRe{2jVE+ zfH|b{Jd&mQ*(Vt6rJ=+-iO{zOt8UuubOpWo1*-TZ27tE&!!SR>z)a-U(cwqHx*_=0 z^>$$eG{quy_5g?9(I~_t4j-Ry*Sgq1n3a zPo>%!Q({0OHBgBBH)tuik(`H@1l+FbQh`6>i)r4QnG%QL<9R9kReHeUw!D47<*VJ`xF4+{*yq%kCF zkWgr@clmq0SKG_LJ3hY=Q0Y}OOBc(F1`PPZi90|}N)QTtkGL$bns*TR;|jo9wOk7< zwO=Iyf7pIcx&ttr#&uZznQ4oFK@CPS(I(8#Yvbc>#$QY683L5pE$?E7vc7*MyqyrK z8Qg(Mg4V;}okFp@ZL8gPjUcUuzAy==lhG4R5CaL4p(a2BD@yRRbaG~_@}eUeYR+eD zlIVe#W9Jw`V2*9Q?lgcqb>;zF467L%+5t)3;Yrh5`*ZIoIMc=@7|akOa{Ofn7&Uqq z8w17gatr`kz8VqSWNGt1j?dC$$Z%DMm?GH%LXMLgKxOf(hMjI#2-;TFfb6Ys)jsXnaI;i7MN(TCpRZYMQdZ0=i3np#311_~8)hfVK zd}l2t$-RH7*#Rh|Y}e~dV-e6uE_bR2*_dZ-YV>@^zEFTXxVnwwBWex|{BoPIE_^QT z42#Hi-brE0xQ0?7 z!vh%LQiAd1^y|h>z#{lwucK%9D+CDP=mQQ7{H~9#ZB@^J;9)O+t5fZIhM&mQT6{qZ zOgItODHC=E+)@Br!wc=tKdTiW--PV420@X_Xh6K3uo|FBNn`-xi!TAm5P-3w7wwN< ze>V<&1?=5PbVQTc1Ia=WH7*(w$}Vi~uQn?%gtoGG+$y~F-9Qd@T3TmVei#?)livh1 zdIvpP0qBYT@-;g^op)e>>!|w-N;(W<2CO1YJAl!cr?B=7V~4Sm-!>5PcA~y%Qb**p#=)XmN2#? zUWXMBSuTwcz24eMYP&Uu4B)?X?0CxznRqRcLPC<92_V{sn-xCm%pdlCb83+CH~c^} zB!~xzP_3U(R{)@*+jHQe#DMGM0uWj<$wQ|FN-sd6d0F!@U~&Gs5s=-q4M9CZVG*nx z6#})e!~9Vx`mvWvH3s?9K$UlOy$skW<_Kqtg)yLQsO(E zwwD3$!Arae$_l2rM|}+u@Mv`hcRv@61{|ZLU0XPJR>3kb;-hdk7wmUr^jQJey|a19 zA0wZDKbcnVJ@jnFWBAyEb1Z5#zAMU9nDwuqZw>+u<~h#v6&uQT!H6(!Y6(6dJH8Ds zB3n?%G>?0Nl)Obr-Al4?7$TqW9Tt|u96Kk$3e`u50x(pQ$Ksq%Xy^=j)85=tiH0k# z+S8zft@@IR)}~t7TUZ>gp#K}V81LDT=JQI2g$B8@fS+{1N07jg+?bnUf zt>`}@uo@LnAHVc=DEY-V3&We4X?a(?Z(X}K#Pai}+TGIt4>z2_&EYob4wJ}4#vAs_ z0wG)h23+)g38vHCM4{;OVIZMxP9f#W*(oiRI5``2hfikVAEqUh=HR=lS3p=5c5(9t z9)q!M`4H@t#Xk|L5hV$+o)DrT838j9(<8IAooBaW>@u2?=RjF?KZ0_~LhN=K6IGPD zAej%@nz}*@RLZf9@n-Kpy1~9n3h#ShHh*30kb5|n9ae1`^gL){qKD7LApDW{rtqSm zjG30ysqYdz1>@%4Z~Pt?lAY{JMSn~yZ0UW5o-!$WQ$BB?VS;Rlz@x!?!+FNra&lhQ zI>dRFg{QAgfER8(qC`Chc1BL(Wtr~Dn)~bB*eP^@FgRZTL>VOFI3mBNJ`RhyhbMUH zC|1cV_(LPY6Cx;vB}U}zCU8Xc9E zZt0drx+I2@5D*4%sI$iRTj%$lbFOpw2bW$Wd(YZy@4eRZx$pb=b6IcVWo6d*od8yG zSE40pMVI-eSjVy^okwo0gBHIl*iw!ZN&S#z)7imhq-6A4F!bTNFP4aE&N_K{iO$B%; zz2(hlZLLJL7@!SaMBh8$kCMGK*HScpSTo>T_>ptoj-1HS$cZ9UJO^#YaIB3PE%K|_ zA?XoS@H^Yf)Dn4Czdh#OswNg~*~Ot)-?TdfYKo%ZtOmIo9B&uiI7H{t6%L60C-evwK{8N20BUa|+KiCWZ)i&>{Y8P16q6heJ~@GG*nv<@pO z^544nl{^yE40phZ>*`Q44S9u?_V z?ltffZ}>dKf5qa1YQsVs90mP_PooyR^BTrh&1w!sW`7?hj)7U~7=!EgFS%7P)FPG5;xRJQ|72ou#oYo}ygcjS8 zxpKo(7-z1={a%d5ooicYkq&QDsDch#Az4~`1CQB>?+2G=T`o5JSrOBHJ%b%_uwJ_x z_v0wwmt?E4In9DXo6B&D?{)~o`#adFaH9n`URHOCp@inka=SE*X|$CFX3>I5!VExe zytoq|sU;xb7uLw}1-$&O6LRuyfx4)N^M#i52~0DoU-6Ds=d`!nQ+=@TCG7BeEx|T}yJLjxp=cp89J@MGuvaZY0$#-l zk!mug@b}^#BC5K7)vc?Uc`(&lae~S}AW4!a_^U7!MyuuD3((;XI);o2CM=f zeN0^j>r+=>Op~WO-y&TJD}Dug(}&N*h@`5wM~E(7n2BCv#iOXe4}P#w#hXuJIj6Wm zrz8Q9sXYH$CMFcGDQuf}u?DPv-L&AlR(ggk85*2nZ>C?|bhf0Mm!SP(aB|`@-{;c0>IRAR1uYv8_T7K`%Yyo`b zT~kH*;aAZ2`+Z~{Pz413us{ps3TM^~wAe1Jn!Ev?1$}3_&K{S;fSxpyS(5AK$xt3< z^-kSjHIZ1f(WQl6{c4kP8m`FmT}8fl_sWngOTW6pt!D?MMDnkp%wxH84?fDJF*)|C z;G^%4-5{TgWpaUK&wt`VyoL&x2X_ zjlBU?5wE8@d-B1?VTdte69xT^AoXl9=5d2;zZAS+cINu5Wb$D$qmN&PgRiao=K51z zwlbPUGS2ir8s?kdSP2mXeLknK1ca+4igjMUa@nO=QWc~SBGY9end?a>=Yr@1t~-Hm z>-zSgKjA{^Sx`A?$le@+kQtV;$YN57UXZegS3@xW=9#y9W!Ep+|}$&tKpq>l(j{}|)U zPDfh}=RR5Y-DoSSy$@kKxVqg$WpXbl4l3@u=U4 zpZmm|>ncnCGflw4Z;dy7(FT6BH2PXNQiMF{YLK95f@N%VhaMXhp{xW9#*Pqvp3v0# zN}C%WG8rGKBsyJf&1O8B&5#gXTPKH5yMOb}l(0de!-G)SkGVyW+XuV_hL)RE?rmW^ zL9jV&-(7kcT|1-mYt4l*a_~c_KUPrqCYRSt zsSZ`7>k;=Q|GD`V{UlTMQ{A4z2nhC5i1Xy^k53BSm^fzm?MNPhCWsFpIt0oK;bm6X zjy1*2NA16HT5$~)ypE(U-zyh>^9NHLZHX=UJb(F`O9uamNGg^ltab7kL+iS_elv0T zGb8#F#>%P&jaw=tLu!p9TEa10X4fpEPtoj@xv>geUq<^xwtaLqV_ixZFup~HqF#zc zVxNn|mwXnoUrL=Q)dg5 zPWY*AhbFXhLC`(&lvH1~EB~$aasR^*sYt|UVT4`pO>`}IiCFxE~Ek%kRgO0HCjGL9=~Jjtn- zYsADCKj7h4c@1S<4qWIH9OcC)Hhl)?dlOVt54}ZX2HGt-J8D?%aXad>(DoDxO=pc9rPQW6IT}}={})qM>XuYQ53#ZSV^Y4 zgPGx7A7xm5EVXh1y}c$8`BJ*XKmr<7y?P=znh?J3nt>3p(^1aPi~Q)^N-nH@mVd@y zGUM?-nsPVUCT)36<3f!>U(^d(_=Y7NiQj`L807*f>@77Eq4Cvy6f-OM6j98Jw{2Wf zzg=kN5*=O{?WglSiu`A`QDr`@<=a|kHt4esqeazy!EORAppvR}ellWa7IBf8sVg0`s=H#bQdz8|cJpEnX7+PR0s) zpVB)UVx7SmKF_4jnGs53sEM!m{VIOB18j1?&u8HnD_>w59IX`Lb2~$>X$kTmC-`wY zcIN=5NIo9XO$6nwYv}W1wsXlKUzdzMwoV@j~VanM&DB?MJ5lHm6&rLiCv^`n7)QH>TI5 zs#A<8QKPN+?11FOA>!7g?@TT|k$MiF&2P1@vJT ztIj_jDEGl9yuaLXZS+0~@3UXkvSY}E>vZ;ws03y2F=kCQrT=yuca%4q{<8>%zoWo& zw@fXWqi%c9Yk#&yh|ip(V39;q&D z4@N|_^2X>fjO81tb4f;z;`3{8hKm&@M=`ddqUS|AqMDO-ziZsGAAlHj1mt91dMrTq zbEw>@jiyzUIW?xx@Q7yPg^qG|8^ttSAcoa9fxp1IgX_m_nKqlkZOpV}WE2$!uB(N$+^i!fF1}ullO^(& zbArli>@K~7CACqKJampDULu_&MuivG{#Hov&Xp~Q2$n`li$Om!#z9IICYX67UZ`x0 z%&~`!Fy2LS^F{}dnZC1!y8RY^<&1z>FTX$dqR}||Am>5CX69S_SN
)*F)wFIzB z{M}@3>lXguNyTIjm$I0U^d+(X`1vr?7l4%js(i39<;`@kr<* z-8hdhX%_FkSv~gQg)4DNhQi#TcAiRO({@#UCs1{kNzs*dy4bGEVftTcc+yKV8+OIS z8Hm!A@lyz;;m?hAj;3g3wq|zcw1G~x$5Z$kxmv0RxrQ3V7$q%!E4Xq`PRbZfFnU*c z(caW8b=0{eLuA9BXuGMkhINW7>vIb${n?PFB^Pc-%0U>275sR13$NplM-j{dqOa>=Lu~~+W4X*|#bRyd87x3?_KaYtsk0C@4MxE_1dcjtsYV3b7 z5d^DQi_HB+WyD}SqdDvdE%Cy>>)}n>jK;MW9yAlz)2{mwKV;^+?40A6t-469{$90w z!`yD*)vRhqYnLh3G~4BrOF?_n)>SENizop920g}Ms{%tya+VUuSv7aIuctZ(e;s}j zkyGPWb!*xsCNuTAX!TUr(!|)by|~6>ItCF~AKDZSr?(Z>ob501n|6vyr^V1#RNBYA z!1PkPI`TGhW~$RY|V^} zgo(DFN*af?NA2liDo4ytoTF`N{Y~pudUF)uuA$)~F41o)TdIHDeE4|7{B%FxFzl?R z&^4zb`qWe+HWuYz&9{i9TNcH?w~_E-bfW2D_<;MOTl!9zQiVeC&d;5ZtvRxjZ~m$cpUX-+@YS5) zeCp?VB1A7Ko-z42W~Y0k;QOQ+;P(1(V`coMn%7rdLDiMc)tu`;0E~k?FbbXuBYpAH z?f(m&8IyG>mN&n9WB#vO_^D z{~|8`@hSXE(|D+HwV`XeI{z7eE1hyyoS3j1i_k# zD*vxM63`|X?!6o#u~{?x?^*osQwx4zJk?CuUm7kg1vvpF;hYTLgUJ{!8%F zh1nlg(<%FE0ro?mV+izEHZP-3fhoW!&|)Ek zii_@&=S?GfYxZh@(t0)e1csGAZ7m!e03|(@13y4|q*aB{LzzvkQP)4*SUU%{gqcbk z6!!}tNAB?+BR${BF7rAEDGItr=RifI4dl8mvfXOOKUxuYg8gw;%~9|hhqdCn%lyt9 zL23Ow59s4-c8(kF6kW%HzZ%X07r`iSj~m?sm<@eOUCl5u*LY9b4W zQajL5IuG!##C5<@Z{0ZFk2E?#iT$ib=KNJt;c9;C4`U zpDp!piUfg%q3b}wS#SVM_s#Z6pZ>WyOacOSou=)p?%j9R@M*qGbq91R7vIK*Y8jsXFJt&8s=}yGudx$t>u|HVQ7Fc0FK4wBJr?^13ClL$R7( z6T%I<(eLw0&F+IBfR`0R^0e2R-?D=J{zD@pasG;+0lpqi3OQXV zoUm3CwAM4i07mf<-=+M94JVkn{P~W4d#RUvZ3_xR*BieT2>X*d;`5Q)dog4_to5wp zkx)^g?e5C4n59@3<{mVEYoM7T`|8?eenF|yF3CN>1S#@p&C2&c`fd%j*HTgg^c~d1 z`MmT2Z1Ea->=Z1)|_bK?3ohnDkm*M#5sYb1wApE z$O!w5y8SpkoR`v<`ok{$RgqU=i~8RIXU@Yp-XQebKI31M5JR(`a|R+maP0LIcw9My zbYp)$#UZui%e?w>0_V4nYk$^mXfj|2!y;aVdNP0%!}R@DAkRxY~6Iq}6Ox zlYF_>>So!1VOtSSHO_rf{y?ey4dEk=x_Uq;xf8e!o5W=3Rl^!$D=Khl$VPn-ez^O$lR{!sI;9W8SQ))uNw zFP(`$*ep(gp5v0pfDwN=owv|@0SG&24|bT_iacEvE#Ja{x^?h1U+&&G1NT54F2TbF zOtZUf)0F`Ee96a=u*>6B^;`Cz3U>TEWna`ynh@OY+`$>O5M9m`%Y*|}01KJdcx=-O z`kLd6X_A0}X+iFnITAcFnZVcW*;-v<>?V->o&1*~!J5$+rZDrW<_G_B7-KQujfuCs zfTh64BcA+R&~@paO2D+ta+McD&~W`(f=AF#N9pPeUl>)RbR<=qHwgMTqq(% z@Cv7TC8c8g!9(0!+0Z29dc?b3Kr%~byz(v`lBpklRr4!VK5Q|7ctoL`smPp$QuRznuPF9UK2K9} zXye3T&mj0Btiz1}I~V!@WT%7j+l=)Cv$`-O6&~Lf3cLK*T5o8JQ|_>u zR6U*WNqGY|{8>;th@LQvc}ORFyjfkTvr{a@6|!CZz!D=VG9Dwgo&yt0I$+i9j{Qk|boW7-Zbm zIVqDE#=*hM)<;yUaVLa3q0&hCxMzlzqM9!yiIe?9-C@ny;gC>NCAPxsRq55Poha2a?keEul(7~KHdOdIb% z6>>FN3{gYZ&Gwcniv0Fp+_>felk1ZVR2OFc#(SuOT?uiGMAU)b*CTdB@V(HM2V>;P z@F`ZA=I~M$LhR=mGlFZ#D82u*bivlK*hy;X+&ocvRv9x38XtCJdR)o^=L`C>kSpKY z{9C;@B*SYOoldrF^>~6^yS=w1qO7V)7s%?`{o6&IK$1%0@rZTy={b3%?KWtipkDYE zHLuT&ZfF;rt1q%@QjX`xCp5*=iX**|)D)ohu&<_z=J`4|OgYsTta!MW2xy!F`I}CG z8SV~(l8A+o3CShEbvDL)hpVmECD{EHGu0z1!5~`nB+nZ8;47L-C6V^m=v&-AD=05P z(YrF5)4fgV!t`XJALXP@h1f%P%*l$=BacDogoc5z(&pZ@AL9iA(nf_S@?m99IlmM6 zdF9Vu;H0|)LNEAC7WNsNeZg|tYZpVfiySTe<%*c9VC!B)t$OWu37FxO&Bh1JPT}?g zz-TDbhIFh`46H$KKNC)(K6+)&qU0Z*PN29*$9L5qXzLr1Z<|&12y9>?bkW^jX{qj}L8-3VWUuL@Rzv-u!H`D~ul2p&7q6Uiw;ii#F)fjtREw#K+VyjeEq8JSq-&dAk*YZ= znk;Z5F0gp;hT-=f8L#0)q~f{oIDQY&L9Z{N@=Sa|?RuVE-R*;UG~Au_W=zuR$=d?s7e20uj|1D>QmndAAt$u`OT7bx70jBON;KDI4FQ1*!~; zxf(2ZLSW+ZZp^1kh!kxKysv4 ziIG9o{SUgsZSRPpr6Js?=$WY6_qRF)B8*@T?-bGJd3PzB3v#XQ!j5f zDMU@8>ST?f4_lnOOm4RXLp~Bu&uez*&&WOhQX-81#^=SsW#JM>m+Wu}DD(_0af?PY z8fjPu`Zp#Lcu}4$7J3jc}HeS5{`3NI5L~ zdUwNo{y{3Wc|=AIgE{h!h|N+YMM!XL?XKm)fMeXgvN%FMrn@%xxK^EH6ub~VHHti` z2h$ySCq~P7<)qnCwwKvqVy?C;Kx#^@1@iNu(-TMdd4>H4$A0Fqjz*t_Ks2;xUEb-& zCmI4h-{Pnylu@swsPL{f)D>2UB>-I7=J7thBT?~ z-K1Tm^1Elsne{$Q`^IXVkFlbNnajl#$;dE@g9m&qa|Q0An#zFNzQ?WMjzDs!jz#6z|9(VQr) z2pQysbv!F>WtP%u|9Y+B>O+0253NuS+D6|L>}reZ8sFO9NJto$M6Ejgq={l=q$D28 zcni(|y?Be42UbpSEpz3J)~nUjc%IjoK)a0z_9ekm0&dt=?kvcJReS_dA$SiFyrCl} zeU}oU;(=;D3(-NYWH`gf~OKvz3XwQO{8Im z49(s@Pf)>ua$n>cYA8t!X#E(#h+2@T=fuw=`yAB?#~*Hk=Oy_p6U3Xv?Oc~pEAlb( zW!VxMBOj#ZLUEBaAook?xUVYQDrxmmebRql-ii16fY>oDOauP$jSTt+aTw&H3I%Yf z-Rq2gEUgdAm=xBh^lx)axiK@syMp|zC*pdKLR~|6!uk7H^kXU^YA(4hlP5-q1p|j} z^vfor4ZIW=IIF(EwJ;aS`OE@)+MX}F+%XzsM5JYp)`OBu6E9cx_snOuu~IJv0MMrWOH0kD)sHY1dkiF~v3}WzddI)9b;}4+bNppRxtIbF1}QIv$u5 zH-6~167LH3S+~a@$~hdK?}OoeK3V&X@h&%~3|c-=A=2{hUxGVO$axGI`M@pWxh}7N)dZI4 zxyWU|A-BvwoL@`xlU#VTIV@qd=^EijMG{m{$<&Adbj!UYTkmu%B$(GL${gd%>kzi^ z(gZ_mJLE?9DJ`IwsW90OBCCGjPER>KE=3fNQjWrS8A{!#hEF zZG>m=8C3KoSmJm4v_+=Agi#I+H3(0;zFa5iw3rQEcrp~ms@0N?o*`;3`>twGUM7L1 z?yR<_U9R0NO0pRzB6SyE$Xji?O%56hPYD@#+a>W zyA+_Y2(d#vREdbC==?bS^@=v1ibqH8XJX^dgJcCg9nr$)g4U1IGzzcU2VRO+!D4Cp zHz5~_Hd2$WXxX_a#(&T_XTqROd7_c67#) zQrkfEtyrypRYX3GW2{Elol8fnF(zue!TeM<5{UF=BQ3`?Wt$ksSzc?7f6bWVYQh(w zI%0gvg|vs=T>jp)7DP|qjC{;@D?nlRvbKv~a0#Sv$HV5)QPlzQI|$u3OEPKWeXXv_ zwnGdScT|riVlI{T8F$TGOI7OH(XYwa zdDz)##P)pBi}Sk-3v8^duSSvKM6^FyNS(_?UM@Z^m4mLpewuwc+CTY5x7fl$Qp%OC zT!U4QN+{EhYTnL%Z&(CJ+ICx&>I$fI1<)!;%9y9GIxEZFU--SwbmMm>x%WqnU5)_* zsjUc3Si>Z#M4iUctGy9kCeD;Q22`oyTMCUmvhQxZzqx9V*6OV4k>B%sLCVU<=7cbT zRKXYdI??(A*O|3~=s5AYR-Dn!^SH306Kc$u$ImZSOdh|DyhT#bGreG&^yss!oraKd zV}UKdM&7J{@)#Yo*nXLHzZ56nTlQzS8M<#EK@FEuHaN^=oS>KpHO<>bd~dYdJ_pX1 zt@Gk8N7<+g-(cDBVdc(zJueaHogulpD|HG54>wuf2*0Dbrms(okf2KNZBu3c@?at) zgGT?F74ogQ3_|6UMIP$5X!==|%7e=J<1phBhNnvApc5$L65gb3jKBSGj1g2>gXeZm zg7_t*~kc|)8nwXlMXN|3}S!TI& zDCm858G51&sZR2tN`_uh`n5a^Yx$4+xjXAZQWCEXcy&wC)8+#(6|Ib&yZf#%~=QcBGD1ko{3dB$45!*5f~@>`*<<`VOiyWFwwzJY9>a?7Z3nA92; zoh6lTW859yR}3n!MgAxSZHxHEhr2`=7QPXa+K+vnGrQE9^_cpH_c{hjbULoR-w5VD z0U?T0KV<`@?5BDY=OVJE%l?}8w(bh1`Kos~2SFVr7bi00>T{&Y#S0g&JuvNJfISZ& z_Zhgmuo$DvgHPl-L1O_j@Jyd0UN`3wHU$i zzl@qJc(Q>3&Y3iOW@36Hj}pN8Wy&d{Sas(1*cXo5Wb@=vkTD_qg0Bh?9aP=zQ?>>g z;c#{GX|yPPBAgn~h9xIz8H5M^q3m+m_5?smVlJOa(AATU2W-QZOSA{ zbC01~EJ1I0O{u{O(w(GbDBmwvf4$UBIfE66xO;@4S7eoJAWI;V#P#~eLssIg8#UlJ z6zW`~Ck(0DPDqu-1>5=I0^=Y~tv=>{*oDBs^C+wBPX-cmqMsP>R~#(PKLtlxte>ir|DZP%~J#gvl>S7u*|-S54%=0SxGuu znO_uqnN(MbrJ`Z>v0oKwO#VeN0<7Qqs`t`Xe!Su|)2W1Ne_-v~noO}W<$tyZ@5$D} zzK~sj-~krABng_eo6j(8KIQ3uC_Jlj_T1>xvnGy$4VL|R!wxO9+YFD8^OIQLmdq40 z*D||l$CI9kLQ<4UM;6Hte{rtDq@3OXl$7GfQ{fo7y8dV>Nz`c)fe!a7LGlhwoHEAU zer_w)3LBn0457L)Iu~gKg+b^oZi{pxuLjo#ZnrmLn}e;g($-ok_j<)vlh1TwSxOI^nJ6cV zR>eyxon3brTNZ8^%jLFTJ~p@!tkJkU-R<7`{`1L^w#5ZHTGL@`+92apFEL4MhWkHp zaIzuN+-p(UmQV&q zkb)nNMg_ck-}WH;t=Fidftg;R%C$|+@#-|Q`XenVuMq~k!Lb30Tcg@7E|@62-+JEA|nAvVb= zMcGKC2cu*zb!A_-8++hI`J>x>DcF`_E{6bCZ zznGVY9Ur^E_2fo%-~esjg9qX{Em`>y0_)GSe3S85+ZFD|8f5>9UvwGv5DzbiUp!OJ zk&K*zy$kP+Af#8$sU)0^aZOSbs+1Su|6CWZC|rqAy)vEb@DeQAqQ(o!hnA{?i*oa= z<0bJxpx-yC&>F&jkNm#@Gnytax6}1h$M^5K{=Yv0{yWcCW)OVA;tz2p{1=D+Z*Jcq z49Wq~j)=0ezZvrX{tS8GLwcNU;kRs(`2Tq<PJar#^rgiGFo#?9fK0W52r^;7#A7RGVyO(h z-Xkr5K~T>f7b*t`j(7l_^bJ8k4YV;4${hjT9F12kpJ_-hlFNCo`{ zLQEKbH?UynE-+zW#jO5mgyjQ9HCrA)yFl?$l zdHp}6=M|(JWCZBa-uYWa{GTJTis8X|U|O>{_t-kl|I2+7ecVbarSA*CPXP{OS6s0Y zN&+6-d;`zf3cF*lo__`c@rpWf*r@*TSi1vNMgE7Oqcac!nt-3h-^(K_2!fIY$Ii|( z$zR=c3rUFO26nn*rBO)hh{kFeV+U1#{rYnHOHljH2NcJlj(*$lbj`D#<4P7Au&gqF zYQzPU2?EJMLfcj2+*wl zEHJ*B*GpW%rTbz)2=22~#|G~T5KQfs2qMrK)VUO4UbjOjkp|V4Cg!=$NEu|Be(mOrj zQ{a~TfODAU!gxfBQ~@uw3AwJBLq)rPh{J!&5);qkRS4?0j%MZ)G*olF-VcC;yKlX_ zoeW0yxYSX=n}sD)`)dE0*KlJb)C&}f<@2Eo#GUh2cMXr3e6~Q=7`wVfmfhbXKd`zA zkq#Afm;ME^ty%*DvJ^E0;`9OoSi6@$Fq~TDE_X_uIsb{tgNly5Uk|T60y+bB-xJ_< z5YRmJ0wJ|M2M4Y%B`sbShzVy1-{d5s)hg)p@PwGXUi%4V)AB=2oGc@szhC5oL>w*< z@!pV|nV#pg2l>g_ueXhMdV=b2aX9^whJag}Mm@lf8R1jec7KUlaaW8WEFjz2<#(X1?I0OdkCt-X}PgH$vV_UXyere+lI<0Bs z$kmrbA3f)MJ$2>|Jir7@G%Mno_5*O|ZhNiCtTiodL|RZjwhkmU9$p{f8}sISqm|8Z z4IzFETAob59j9~cRnNN6i23YzZ*>wp(lp3d6II8)Clp<6@8Y;d=X!w42TXg^&#}6T z3ok64k-WRN0YViMge#K6XQIs5#=Ah=q<-RES`5|*X;CnZQ~!(|KbPoR{|y2Y1s-n# zrz$O3=b{v)*A}?ew*!k?ex*#XK?A3r8-WRtkIR^R*$4s$uQnWfJ>LV56FL!prNEbe zS^IEkAv`7^fx{o9JrZD<3_?nL!4v(K_SlJO)au&LgqTk{lK;j6pTcl5ChNi_!^I9& zbq$Q!N-aY*IwbQV6^6D!I}H@%)x9q|ppc+;|Lh2TSscaifD_(HcGU)DHGxy9;q*F}f{PXG_+|QV zw#Wv@lJ*;;0B38C?!DOpeFttY_P*;!Ud@ua(+XjduySX%BWr>mxPgy@SEwPZ1)dac zXNI`aase9n_1Fz)*{D!V-z-2n!DnU1s~6QEb!T0%&#HVz-Y7D)P(27`?EF9#XIaT4 z^fiBU+fNDaYU|yiIu}OLah&+dXKz_NP%(k#7$+M-5D)1nBdB&+idh2meSUnpw3xh& zVZUduE3bT&!k|Nz22U0Ku-!){QZnUCfl+2;Dyvc6jnbrh_jUlU21h!Ty|(9cC|M*6 zABv>J4YaVyrEA2cXq0mIKP<8cnk^m55pKCb+x;B7yPpr5EfnBnT^jKFs|fV`5k)yp zv1J1^o_AK0_hd4+8tswqeqyb>_HIyiQMLot3%`1+CHE_yrfZRpKYgn)tO|1ZOHnMk_e(bC)D__TC%B4?t<_Q z=a`=4Ax2W(n46-iM7Bq^y_!3W>~(==h1On(QWt7r`(kVHl}YzNUrr7PDI&@NoIeR6aL5^ZhlF^oEReRP zczMAKgnb8jS-+D-DlfaAY|yI&ncFD{@XHCHASRSBm60E1nf6F^mn~?9vcDWzxFpk5 zhXC5A9<#l>b5-3OmzU!E?02CNs-3_LneR)-+wJ2?mR#9r;=Z4vS>kRVKMN@A+g zF-Xv?7f_Md*q4IYczcDMyvU+`k(YR$v0dW7Q_PTMX$UUQ8;<^UHa?0mpXAd)|LHw> zaW*>Jq4a-l53s0zN!D}<&kSijB6jp@wNlF`B{bisZe%U5pfQ}WPG|Qn`bVMg4wuC< zCar+bR-SfqRaJDkG=OAAb%7{C9{I;Sn4zA&FIom&uRK39TvRcDwIV5CCCagSz-!A% z&!eu16ZSn4ApK^7{RpW7i570}l&iSmSJaJRxynP#l*#>sVYMR9iayMp7HqpBnGmnD zLPX?xufA_@m7sCs9f9PSjcw6a-XAn+&{mwq{4u1&f)k| zjOug}$(=B2-uRv8ck+-ehKoA;wzxvJk?aI;jH&Rt>#JC9nNN|#tHFj8fRAI9~jk%&AzJ9}C`%XU6)G#X~A}Qg;;CbxeWm~8vY!)75a|m)} z{PT;)$71$6C@s&gFXmXMooV5cVfA+P|I>Jcg685$w5;&AaOI_+&-eb?foZVghEZf^ z_rk?Xf)7}f9WKLCFz&pJW8+fGai&-Al|K%@17k#3Ml!NJQ9vCg$x|rOIIj#n8$Ap^$G|%+SI-(RHA9Y$Lia!WHf+RCjJ8B@wsXYHu-(L|@DCM?6?Ub=O$CxK9dN(D_ENhnJ8DMfyS-&~-8E9nduUoesKbM1DDN_mELDOBgi z^zS(&1ef+m_-kJ$TZ(Dt_v}`h=l7Or*_(7`sO0l&y8|A)%ah()b(EqvwcE~RX=jzo z86+UVu-S6iI=B8**z5m{__ z@Ri4>hzo<5Twz)0e9AxEi{xnIlnHh>GpjUjJZk5T(1xlJRVaTP0$ZDBQ;b+&u+!q) z(xqyl3=D?mY-S=he>H><1-@;YjilC&eHE{V;_{o&q|fs^q#(e*g7*+2h6+oO*QW4( zj-`i&uYo>TW%WFMJxCWn3XX+lq(8jMnO7#_$18Y^;s|$9I-4fY;;WQB9pg0wEpoa? zjE=3Ytbo9|?p^hU81M?0K*$sdwY)Op5Z0EMmnPv=e+?J1IF}Wr3PV*c1c~ zgQ(~s-q;gGWpR<;+8}U6CFQ0}6{$t<7(<}CFxCoM(asgHn0ylADnAErgofQG(kM?c zD*-!I&?iKZKpB5ye=hg1-Lo@cCu_U^!K&=0c!AE6B=2o)Bf6`;t}zttDe~+r};ij&NdY8shFj|AJYH#{Wu3jU|PS9_)plY8AblzRqL!**rP5ih1OX+bllSl z>tYf<&akmgrZdMEfQ2Gc)u*!RTOM^c5K_@=Nhi%+*mv$A-Aj^($p)9iDOAQSvUnm~ z$fK97WAT+1qad6X{&0M9DV72@FQ~4>*Vm)j-ieoky(r||XLEAvii3kc?%98}Ryi*t z#iu=Hx!7nATD;|a)1@7otsGkA2C+-Gsc3l*LEld<(ua>N`LhkHaniW!LSSog9~?NT6HBbcZl7H313DrssysUD+Mn8Yjzw0tN!E`}YCJ0%Ayjd32#FsQAuUJ{G~h4J z8J`OUGk89Sp`STOeJM77d~(}!!durG`FOT+m~FXYvXTML>fQ%-t-hGAqA(w|v)8-k z#ot0*Ih@GNEg7czH!P$~Gt=i>F8sSU{qLR5f|pL#^&>6)6;02w!wlL&GOt+K<={P^ z@JW4!v|{nz`txwq|QvT`A|FK9$wTBMC&9R72d|TEcBm6?f+R{5(6c>vKaaQ_GJ3^ z*HTmBJ$dd$9P-suJOKm-XGgocTijPFJlQ(@pLI;XBu7hl$Om{BiY)r0bOZYij|g3cl{;RQ4{um zAL&1T0~vXw5<5+=*JvWA|TNuV;-dKHXa==g5e_kZe0|DFfWCbSxC zN%P+t`dCjKy2^O$fLceaItdI_i^23%&#z{g7B^UAK94=K477rkEdls!TtYn3mp#5@V;E<(!L_p*dOAJ;MBXS|U#>jxNP78h2Diz@RWL`_^NS>l?PUDo z%r-zBiBr4vWp#?5*jUUlu#)iWe-Twb#()Ql<{uvq==4Fd&wR$v`g?-JO{ft!1X{SV z)D^YDKMt&a1`_OvxbSdNdv~y(*L=iT0(S?+CgoTC`p)gwM{)A{sag;*L^$;l^ij;a zy9YFEv zdR&16XtZVsJ;?A`&s_&y3G*eAU;jYkY9rh!bGDGAsfX-0-u&rO@TWBK1tjWgDco`& zv1)O%FMEjHz?HH3--9}f!|U&{n=fI5xS8R@l(6x?=9Td^{1i1VO7BWs23TSbu{$`h zCay|b0Yd0BK@+S=>gm`GJmD=|Fci+tL)E+pmi(VVvq>G?&Ttf5!C~EcwuiW;Cvs#F zc%C!P3gKD#3R&&79Lw=*vI&SeT$Wt%G?1Ta{q_TeN2;h8(C5w_(>3BT zfB%26_0>^Pu6?_7BSSOf(A`pkbb}Jo9YZ$=(nvQ9-5nw&Eg&M@AxMLCNDE4XoM-m8 z_xtYeob!ikmdnMGdE&l*_m%&B%l~>|&93RpDz^b_!5EGqF%meygVQkRJw@R)67Uw1 zYEmCLo<#teLNAUzuo~RGgI5kv0coVOa0d(luMhxurrA3K!GCiHQ6}ir-e#VFZ>y#D zqdLkY9CE4w97H?KK<#9jUB+BiNZYrUqG?eqLMJ1@Bj8u<4|f^B-A(%CzVNJ5+21et z$DfwSY2ZHvnp6CMXw^}F;n0^IJuvMOR>C`N~+meLV0RTPE$+RbsV2uCjw2z#`645@2xM#^VS~nJe1R> zRa0Q@6^rTyMH3;EUBX;5de$yq9$Mz`F3A{(j;G{{1DF3qi)#dackRXX6&HC|aid<9 zwULZP@*9o`r2>zU?s7zHpL`-Ur*Zr1Da%3}Jy|8Ss#vwnoQ+~7eZ`^O@ti|!Qi~N| zoAK8qZIJjOM>Bf=oO8+t}`a9$NZ!&mr1KIAFz5rTZb>?pt~PDyyC!08$Qj z1tY(5jEoC~4e+@;Ymj&y&b^3uLX~YqGw+s2{X98?wz$o?=1T?b@f##7=7)Yk=aq{2 z(g*H*AS&`!B5En&p&oc)`Sg7%+3&~k7m)E?37qi6-Ulisyro~q5Jvj@!w(NhYqUXMF-8yw6GYkj#%jS8M%uJG@37J9vn{KhFicEL zR3gCeS^%IS(bV)|Aha7VAfNM)7wDhWcIEC5pID+FAnTI^^x)IcnNDQ&GhLg2S#5R) zfD3dY@MySNi=+iXcu)E%M(8?aU`5uwKou``O%PDa*^cw4N%u=&z_Ck5pu+}=ORJ8w zyFXA(AvK=puchSh16bSQECCL5+dH{_@8U1azmLukzQ6 zDJCnhL+iFseS|IgFpL%I{EvVI=&Rc5(K8j34O0ucE)fi7)4By)`QHOec4wRYJ?xI4 zO?KVdv7c?>Jo5Usd&v&iS2l_!79Y)lj^eNqn}c19qk4?(HOoXtAu2MVLs_A;2CGpliKm~I z9Cwp)jPu2O8ahOrzusTd%gDE%rjsLDgD}-cij zx!OpouY&g?#!Ty_$~pDeQP1~MwC=M-Zuk*a9&89g;3@gU_7QTUgJ&g$FSF`Bf*U%Bh%DhfMBOeZ*vaEEpzHh zpz)NZk!s+5Tt-hcb}_1kX;Dnb%tk9@IBurCfn`ZO#T;(w;*+`r3gZr*QDDoENs|=y z@P|s$ryInQaK4_sy*L!mAty7h*awye5~?eAO3JM`Fn_5=iM#>nR!1};^MN=6dslpu z|IK+Nxk}aAl4P7v)H4jip-(zB26mXPvUV8Yp6eX4*TABbJM;dP!pFVI0#kKzi*h}B zr5+$qsEnJ)s$&yuFEs&1Ap}JyK4CQe>1s)hs)i)lodVV*Q%ma|L-jt-+bH*@&7tu!TJ)cf@>Nvh!{L+te#__jJoc-rzXMq20#hSDrqJV%8@ZpM^ru` zKvbD=$tQJ9dfNLPPr4_L%!wr`PSWyu@V(ndD8*jpseEBQm5>Vk6o$#69fnu25 zI^x;jjW}>87a13UO&?dg+Rf33kPHPfk5!7rm9KZ*A-&XA7T;waNmU+=3gNG>)UG;U zUKG{f%dSjLo2}GVVOnfbkNY^5YdB=8_|!;nQ8J zM?#ooE5C=a>vMPG4Nb~JgQz!oR{b8jIC}nR^spL?>kpAq-1r>h^i*6$FuDa zG&Yr??|992{do(a$4qb}p}$_~2Le%TW1tIBr~_dP7x5HAF9+hl zCC<4hwtJ#@=O|wko_hjxnuEwv=u<3;!CPP?a!_S9oVM+Wp`uumF_|LWly<9T@vZZIi1<;gDxAX_fJZ@i4ztw=72CS+k8vU~L8l#pa>)+q zng)~+uJ(CaCg+iPPkD5=HQNw7q@3}}u7sdsqx7I-69yTXR$zQ5pht+v{l)sK{D;7Z zz-+UF?q_Qa&WOoyMmI$*OX~^FfG&14;9fQXd!_P15j0G3%OI4f7jUC;oFR^1n}%%f z7%&^MpG@52t*;K(OZCX#O}NNhFI0>M%2b;?zD99pM|xRob(UpWXEsocDfJKj-~h^yA17nJm&Sk|ULB=~GB!X=|FPri%Z zpsyt)f{o+0WKPEF2P#u?(8)AN2YjOExt*iz@|c%CN>P)!8fjP#{04Y-q7{C!p={nQ zS(xU8P-ADR`P-ovFtD)^q3iRVrNj9TrsrnEmukO#4n-TG-{y|bFVn`$Y&S{^Qm6Z| zm9J#jtB{;FPakpX=A4(f_HeW5PG9OV$YJefj3(A{eJn`cj;QmVwQjpj+dp5NDQj^4 z^~BE8jMeAzo@tuL6+2+>uK)bd~q)*ccwihrrYt%r}pNg+8#w=T@Ak^$%rk~>v8$wZEyx6Kn$6AE}i_;VUqM-+JgyH%^a38 z13|40o9E<$Q6jG9Mh?nyuB9&&X_zK?#WWyK>t*wW47%2c5c9d&bZe{UiP*`{yGDbbV-aVL(7mij zpZw)kG;f`vFGji@Q8fenn2&=DbQl!Uu|GUXR!`Qbf>+|+rr0ieU$%nXzR9k}pyXIB z1Yxi`+mG$K@^4!W;KqO@=+zx1(3D9K(rBKK@D4%pk+&0OyI&*~CUKO=@@+AVbw$u9 zS~DLjBm|Q*zPSFpb*n&$!73R@C##S{>Il)d*13E^>`y(&L$gM zTf!pZXbdRzxH+@KW>LOi1oP-y)Wys>pAb+JV)A#z-sAJ{DJw41{T!7a#O0sgznuhT*o?Mf*S7QQN;enRHhT zJx4SrfwfGkCcl>wYet`Ngp|dS!g5w~S?4>R8OyIt8|c)ovB@ys)20NPat3^wOS%~q z8aYPvncse!?A{yQyfa^RxbszXU}OqC<6HaSqzNmjxDaC-yNZrWdZ-H}C$rE8hEQ%h zHq&tf+IP<{L*qKy3Mq+?ws!xS>9^5Pydi@%ypAv$I5jsg60*=u1s?!fGhbBlqFFf# zF{!)Ms~HS<6I@`Tkl{_mmG$&tE^>T3SA4r}uG%lmmd?u*Ok^fT9xE$@djWt53~C{K zQg#RL*t9D#xCgRSvNE(q+;*z#f zxxw6Fj8J6xHI3n-j$ldEnbW!+g*ORj^3UVGfoTZ7#9wAs>=A0lscA_%MqXRsV5XZR zlMa-!Op;ykd*P@BMvjKC4-P;@pqP73UkjytJHeuSH%~fIatC{nrju^RJq3(1g4CiB zEKdMkkKuQ#EOmt@1bozBl1a#SNYxEZ#M9=5Kpk=Vy}8Q`Bey?1S~jZZzx?|>vGF6M zFz8*^_2V7LHmRD)ixIT)G03hZLj1R$)o)`EvuS;OXyb||#gNjg(&04Tb4lpRkCI${ zepcvyz}x*>X2iAEh9sn`aAmBuUTPsMo(yBO-%a5~?8jnH$!b`Z89GZj6RQ*J}7tu+i1?Va_It+8P_)08j zt%#Xv&*vBDNC@Wlok4e#H{w1BdySI=G$4#)JZ-?IUEYhN-VzPvxV1;C568l#B}abC@Gx7F=bzk+tr<7kz14KpL8-HvqwE z1kc)nFZ-)0*TAK2FMTf#${J^JtobcGkdUgTJL~TVYa&aLPED7UCor_^CruXggwzaM zb-eB$61OKa6!L#esdI@mWD~g9(3r?sb1P+W9+j-tY_FI-I5E?dhApPvD21Ny4RU-9 zL!~hds+fCNcqC^tw?3BO-4uFnT;JLepzX0gSDY+H?RZM=FMfGr#J*YZ&*4!X#fjkC+5$eFUdN2jUW1R!yL>PR!@8t5Q{s|>oQZ3!ArDL&l5aA#x;jh@q)p? z_!2@zbtpHgfAfPm;O$#8f;3*C^rYJO7haf3obPtIZ)RuCXvZJsVkTN7yLx1n~)8`@=-4Bm_# zwoJf02O&jZS}=7iPC!L0JrqgA6nI>NOE-kc{HBAVv<@09>%4jjydxwdT#b3S!W_e4 z{SIiBrcr`o_7E+|>ti2UqeS@QJE;d7XZhF_Z?GqJcueGt(L19(qsT0g zk27j4q!Xl%zjhgsvm!PEEUo7e=ZUw)QjWzw{XFV^q?)t{i`>YxBuJcW zXLzS-9Q-+~>yc#Ac<*_H?*Ba8u}dkix!l?zm$~|fyBbfO%CDY5caG9t6~a;%i+QJ`6~ zAomV*O&STeX)L@&)I~z$#(_c>2(sFE84fazX->s>`Jq|wsvUo)P;d=M4m-m(_`E(Z za6i5bil{0xll%-}G9sK+?P>O;5YAh5Ftr(ykP%X6IU;pfb5S8~|9DI;ar z=(UBS45l#e^{vAvY1}rout1u7x|F?|q^lr3KoqnYfVG!$o0ekXURUdTV{vW+O2o~f zY>g{+ZEV+{uQ&Ip2|Z<5_%WEJrV7G>OFId!!C{fc*3f80&8#Rr#Wh-i3+n*Vn?dUFwG6qpwKbDEu^!8}rUO zZhaH+oS&luZHITDvZE)xVr|3yGVhnwfz+IoFOQw7gsmuLNdm7}N{8yE`%AJMzNv&o zy@D-BA7*R)a~(3!_l-573HGyE+r%>`EmpZlN6|@4)dhFljgdVP8bmEC46JLK&D(eZ z@x+sH5m~V=pIs69Th+`d50wX_LC~OitlX|!zJ|cGsn+Yuv93TP`igb+jD~oJk^I<& zmx^RU3T7^nT}n-kVx#gTQmqT)YtFmnpkh%F?G%m5T+Z##_|`I>#u@nyE?<87DyCNO2KMqpRSQYz*;$oEXKZsVDYokI zEVU@!C>%z8RQ0Q0YC7L=f@QJ*dMx^`8=-T4|L6T&dG4Hd19zt7Q0ob&5fTD!xkvP# zvfg<;@6SPRP%cSOHYL(dI5D7GZhY5f5z#r9ur4Qp&>2XDgr?6c1n@=)YQqMu&?}t& z^fvMA81|)(B&(?lN(#0Q-iS^mJNylCG~#SxV-_Ld=ziuMtyz|!w_^$QV{sh(OnSlr z!lUxe8Z^#vltS;bd5s*wsXP#K3FzVPej`NK_!i0V5zj)FeqPI%jqh#vp(NcFnc~4) ze0KV|w+Vr+-~0dtnf5*@ldeW@QW`TkDVIjh{$hmRw%p_+ONljy8osHc-+J-QNNf0X z`AaY>L8MsYEi<}uy_=Nu+Yc@-^pP}L^15rp448@^sI=a<#$jqXcB7a|<})1&+`iix z{xrY#MV2R){ru}9Zt{G@$K_@2L%%x;3$0)xk_ z$A`w` zKO-&vD%z3eN-xebPOjLkq@~rf&jbN^#UvWJ2ZwR6w>cxt>m>};qf5bK`v>CIc7JRR z23HPC?}_i@o}_+@&cMp8X>NGaQr^Yjv-hv;Xl?}6TX64RCy@z^==PF!wptxzPy{~~+AR41B1!Hq8_LMAGjlzxu0{^tjL?)9UFAL+M5*T!3no+uUcZF0NEfSo(Th;;0Rll+V}iu&>sX?# z5uK9KbBeKg%HN!>l7F2uLhE^mn*H$8>uv&Kt_sl|{2_#>hN(pnD_ero%^iIvABSdb z$ZKuTO(m0uS~M(}YR&Q$BR58couR1TOELh>9Bmi_QQY5UwqTR!uzZLCgdwWeH2ft-gL0HXP=?Vg{(xh%w_L)r=98US34b2 zk5+vQ(Oq9nI8(JO3%qMn`eYn*8dI4FTd%`u!bxk>ffGd9p-E2BNg<~_L%(lqOP;&sJWZ(qF>kwdnZ7B9Li4jTJW znz6kh|MI}fh*Z^y)EWDNtTw}uzahpFN)y%RU5^b}jY7v)!6d1gaLw^ot1_KB7j2BQ zNGL1?g_zGzRk4^r~iVs_)JMq-EFOBxHc{N64S-7O`Oahm3@Rrt-bRPR{9mD zMHcbB5Ll|!@%k=K&cSNKG)JJVD;?5F-nSa>G6B>0mN8oT=eSqSYWQG0dzkQi0{;E} zdnq6eAT!E%I?kh_q^xYq_+|cAiN{+E*|Zz*Nfky6AF`{Ea%-5DV9CT->M?8CWi;><}l z%+L9sCF_4!9sO8@E!%;%-PPfDIe!rVe&oQbacpPv8t)l(sx5<|q}$WiW`6^D3ytPf~kGo zch}z-a$T=)H)B{Vi-~)alyDL2g@N;BD-Qr{L-c(}G|GITu_zPPgNFZl)+YpjkHK)ZK zyeMw^{i}CGgU_5-gna;c^3zbi7~CXOzKl2f({lWo3W8R)XF&D14&TIWlJvF= z8PYM(3zf8&53FHMRtk-P*fHbv(SpXbOfmlB6+l*xBslv}{Pn$~Qfve9|3j?(b15Vf zWZ#aR0Ze{8Jg-f&&E4t-$k7xqdkjys>_q_+PHo580EO}x zZ1w@5si(xp3M5P+D;0-;8sMuh`BPg6AOfgb0mK@7mIFAfMy+*l<(@K@=3=Q*$~IvC z_}aiRU!|)CA>IF0k|@pwO&DZS%`9Zt1V{lUfORh!U_`Du+ZsINg$o9j2C1L>4gsP8 zMxYHZD_Jg*(mm?;{x{}D+)IGDfB}E>NVrX^Pu|0^u(`>ci$~y&sJ&FaoYghO^pt^G zw2Br}mvTg-si$5X%&J7CMP&16EFa;ea~ig!YL+S2XWI2PUloQvQT7N*kf=d%u+!N@ z1(p!A4^og*6OYos!hfEH|1htxDv-X#)@$LZNJ6mN#x!!12pbO*PG zA4nD9Mc!(ie{LPv^Y3qf6vUCA8@)T*DQY@p6tY;**|{3S9pK`o0+D#e(bb&+caaO)cf^9L%ODwBO)y?Z$uu()xx7NpCUtY^_Q-x=Ts=7UDf z_2*pB#Eq#tbLuz9l4`(=fy~0nB|y zK&--UtPMA6A(@XH0xB5Ad2y2=;CTi%BBSV*t9(BR_kU*ZzaDddAW@)yt2~Nh^$k~O zr4XwA$?N#F04BP+VbM{KiSIrm?$1ZyJ1G9Sze)QkZ5&}x{US%uU|@CbgXssX8lVPP zS{L8t7*Hi>^v_S1sd!RhwPXu8C|Bv!Xyu{4m-DZyw;Ze#-SgSYvzcWKl3tfwEpqP# zBpP~*qks{RSzE9ENeCt}g`oWmi>GN=UvNjy3nE4^)~9BNdA4A5OdSAL;>1%zw<8tY zOq1#~XAmZPY73hizHr4h|0fH8FN{=Kn8*iY0*eJ|fAR(*EJa5ITV%?mSawD;)Ree? zb}2HMvGE%xtskf9noK=(cU_`4ng%^+FPER5EjiCrBoWwxQaKHp6u&CpzW~{WqT}f# zp8;rpoGH%)$>vUxnTH2_ql3U_r+%4%d0=4u&-3}0KtM7<5`k30h?F+d6+H~n+xhs% zZ?Voiw4Jp9Me7bAeBQORA)0Y1j6U#7I=8 zKVNHH!=~L|WZ!E&nO{=?lDCFp_m_c?a4o^15FnCX+kGsPw{kt?hx9RV!Wa2RfFS%* zlJHDf>1h;uBVbN3{_+P<4BcX$MQyS>nstlAX)C6#g#>mdU!|uB-0c!dNIXPtdL1p@ z&>vrml)1=B+GWsI)e)potRp0QF9?(8{@T)mW#NP^2G!bsfYSdg)t_(?6T2I5$179d^ ztbnE38y>?t-Qn}gR(p7~Rn(%qM|l-=LvXD2dNHO%Smq{T^as!cSyWrost^&sg`0yP zx6!EwHjHGq7v5dq9i%KdsY99k*r3Dg#LQ&_a%8TOJ6|8*iC_2x0%RSh&(;E){_mStD~0e;7NaQ`8J+2qw85TkjX_!W zZ?`pm;2F-A?AZ8YE5C^lyJ&R?350xAx6Kk|KF-{*h(5HuI{AJP?!f3aDh_N0VGQKk z;O^f_AhVNflyA%fq`Y63TAUnSlpdytH%RnlSw%3Q87-)1gb#Bs4Mb6VH0ur_LZQ1h z110aSVx3(8o>SY15rCkzj}z2#El11Yvx+(C&B1@&nw34CCpzD0p9AdjswTUmz^+?c ze-OnP+3!HnZ2qnM#jA>g*O9L~HX`)jz06Y`)RgH?&u2i{{}Ds#2JB+bDT8NT@%ddN z-ff5=iH0IH219*guIO3@@}Z)C{wM?*x0~qHK5~0IUS9{Awn*tzFSGs9bxdx~X$coP zZa57>4}Esq;5*r*TvwVc(3p5Zzl@3jBBjrb zXIVm7ECi|yALVrSmaL>bnob|%pT2MT;6L+8J7()`b*0<5BG`Mbo9{=gkGor>{q(0- zHAGfk{XGi>kI9!0ok>f?v7?iCl*D7nf96N>P9Hcxeg#HB6=7c$TJY1V^lY)%DSW9}sV8Nq`X~2`f9Q$O@#+)Ja&M^z@r~M+>0)bE@cFF& z;yXpe{iR|}cb~`2e@>FZNN4zO*QLTy@dwx)Kl@%xU+ed=VpE*8F34LTZhGT06nI-# z4E3PBkH}%D4N8AVX@^8Aa{UAhl`O-r0eK(ODH?0PIGH{>ZH%G;kK;n!HlKg@4pB08 zF;;J0s<@P7II&SXZ@iF};f#yD2p??IR|L=;G3G5$_239(T9t%V(_xph4i~?GSrL)y zCsUa9`N%M$@hUpD07fm%X#S-xNM;nNLO`E`rI;AQ(1Ma*#mNtPw+SAjm&1?|uLjrR zu~0qwmlPx)oF=%POMs!cB7OA7a$hu|)%W(9l;Y@vjaH?G^=K4=qU~k)q8=leZ|2o& zMWCVOzla|dihvv^;4t@NPAP?HpBb67^pq?#DsJSr=OE?1*Ya2W>C;eC24%|UqQ3b? z&#o1+9vABQ0=miHYjkGLd=Dxx4rJI9f37qzZo>Kn6}s5ksW4>Ttit@!XGi5*vQpE} zt8JttZpvP4%iZCtHy>ji@rOO;(A{6{ExGse3TQf2wYty_Q&^w=;S9FNJ;x=e(Fxx? z-{|Xie;sA*_D6v$EOF+`6c?NbaS8aSa`gS|^r+%ozcDdpx zD$Cl^zG_f{3a+RbvV))W-sRz+swK!n#3+=?>9p*|YI*4+X=4?lt@+>KWG|q#MbP@a zdt!|Zp5bLzHi`wvpo&!LMbTt;K3Q{|1+LC(ix-meEGwQN?22}4fp z5h87YeC-x-SC@eG6p`sdWu^V|b(E|Yvu5Ypz!PP#heHkm-s6ox_5FWwao~i(k4yek zvDHv4gt>!I`MoOvLaeB@)k3*YI4YK;MPry9CgCml*U6K9#@-K|kNM=DLz};Q65-js3~upw^7zkm#F-#1c+> zZ>p}J43AONqV}oLTlp-&=2Ft4uBmat6j2k{5vBu>23}_JNX5QV@Y3(CWGpo)Fmj*$+x5Eej&ysUatF*B?&glTrfPFSR=dy^OB3V-OvGY#mk~LQ?y3NYrVPOfZt=_!at%xe0bfLLP$jY z8;=h@l7amBW_fwhZRn12iGiXx{QXN%(72|ofoFq&J^|9E^gVFg^36rD5wLs2WX^?! z$kGc4)#mm!!=uP&UIeSY1WtN00ZI>fLx4y%7Mh_pqFtr)V(8^8y(=qYdW3#~ctELk zRcbVnqhYmL6usipD98}6k%*eJ39Gj2gDH{#i6Ha{$>%hZk+m}W_p-f6?Nng9^R!D9 z)VlEJv)grdcQY7dJ;(G()HP41Yg;Bmk*bX)Vqc z>##0!_A~cU3Ywp=1)P00`y+P1CnBKeh|rd9;&*i?u?@3lEVH`^AUx5eRfMR2aX*4$ zQaW@av3}>nCLYSE6JPuLHTajx8NP=20Sai&t9;Q@pC>Y1(>WzS>1u={MBjnkrg@@x zwd_D9A3ARcz+QIC&w|cgvXpexeqDf3v|LIRo3iXDw7;@#N zyk-x)4{RBqg|!XlRO#}c3D&|YM-1+W=RNmY-m@v7vhh1^?H;>7zWag}d=>k{eO|I# z(0#x?>!W{3E>)ja6&Pk)F}?a^g-^{%TmO7-ykw`xW#+6eYHhod|6mxGdG6(ElhyK? z=E=00)sQas_J#)mI_b+**6VYv^%$T1q}kpozWdKF%L1UUT`7am*JL;^44>qd3P)bkePY^P0FJa7m5~f| zWY2w@juz@=Sk_wIEMJIAb|d47sq+Uup%hlXa+s^uS0&)oF4J#CI>#o!{WyWWLHK55VKA_ANloI!l ze`k?z{XK4W=ZcB3Eh2x8&58ga)zpSkN6v(dU+uQQgnO>`{aljjdSqAg>+NyYLuG9q zy|Z7*xaa+9qpzIo1JuDHk5QFNdjJMb3#h+{VIdTyS5^F)Y0bpp$g@k6vGecUYc&c7 zMoB|Cni~Gs()pj$F6JOG$b zh$u4_e=-{EZDGLO`qdtUDReyc~{yEmAUl1#O5>s=r-Fp&}aOiM*q zev;O7JcIreoN>_X{xDbgShfSuHABBlCsOP`i)VNN1Ggmw(?%#Cr%M>a;Yw3Td%fWC zuQKbSxOoZF)Bldi|C+w|@rMXe#2%S#yvF@@Ft)`}qTE`b!vw7Ag>PlI5;l5#fQ}ya zEH6NqKME9{vjZlUNI=^yZ}8EX6Nsw#K+W@*Jet9sflDE%*yg@d>V3vQI=m7wFi;~3 z_4vtVeE(&^R%C-L5VIkK;6sfn$wjQ`SH>zKYz9jX-I-k1u~qc{3(@%>{4!4gpqKLo zzdE?0Oot&t$=# zham{JE?+pmy;IX|HaK2t-hn$iu!uF$_t#?0&t?9oTU?*Ce1A$R!K zKq5@oaK5(}fZkSQFO9$$pOLql0VtD2k~9EfNU&;_MuA9Q1LgqMO2!r&073PX9v19l zvstzsY|td6HlN59sobtL`P|O5R_`uy?D1=Nyy5k}ChZMqFoz=kj|Zifc;g)zuo`oA zjA3|a04F!#EVO+~E65q0&F4TNO3ptp3SM1L1Shm+k#`DetP)VPz!tyIQFyC;j#Jre|?YTzie-CY%y+`YZ+c?vCRh}DG0o^bH7;&dVDTN zM>&!#8yWTtTApA9m`oasfSjRrR*g0QgB*cFu(~EbWiV7xM*;g87+~_@cvULjns0pe z0LZjbfGcDYL(I+uV6pnyFFJsj73GH4CoB2@vT8cF-^4~G>Qx;_BVp1QNCP}lcEoDU zdE4^2%ZIg}Ubes9Pqx3SHQ}}W3IXt7T}^>+mD=pbgxk~u7c!5)fW4_@_3hIp5Y_Na z^#FjvJXegp%JTq}d>E)qp#R%K!0b%X`L|LYvPTC?CFkC|ob>zGxgNe(#A6U~=&1uh zG^|ptK`|27BJL-Oju#LavN8U?v!u^{p{|H+7GkBI^7--4owRKz;Y(rKHsD<@Lk-to zfG7jLhj%FC%b4MHR-+_00H(h4aV?Ngu|(9>re)`?PNFe-u#!|p_ldoDFPsU5Pk>vH`s}vUy(6!As zNa+zLd2|nO60C|%2~?uxEHLgi*>sG|P2gL{Y zwXM2CP?4k-%3o0I&sOR3)f2BKSX2WSd-~HG7f8=3}00$wkEJ#6^77nryjc7OlYD<#g;J3CvGWwGl zU)R`hvrj;wN}8~8RS*=J2H2bCZUE?I2&nid+9q^vvCefM@&K-GsS)6SYtN3`fDbLX zKpq>MKVanUrV15{pQlO4IR@t2woM!p01Ta5_~7+o3az!!F*I!F{*_RG3e^0~fihm(wa;nS9XPqCkn5UH=-gs>@4WB~J6itaIR6Ug5k0BvNtZ{$*yfs6Q# zfP^T-RvFqS5GvWLI&8h8kuC+v!hfs;;COeyI9O#!*N1cBr9kNxc8NJ1l4)6wlZ@Gb z77AGgkX!K1m zZeCO_#{U?*WogP6S%R=d)9F01RXA3CTffq4E+4#{CMtI|Wq#sh|>$J?!(8!wWc7(tu5n@aqETbYd0? zzWV``F@1OlLAhUh{KH&<0*BP}x??BHKN{9rHq}Sio*tc2)&s4$_9R94Cc> zcR*&31w81=B&A3uk`3Q-`z1iA2sj&fM*@CA9q%jwrzNd4HyDkskz05&V-t4pAhtzG z(BmKJxatac5>q?YERfpvoe9qLB{PGBP+%XA^PYKX^5HaK@!1R~GrmtiE=IM8$kYKv z!1!T&AX~B#{u!v5f@u0%DWN}i=@mynI^26F8DYRNsV!aLs#~T8G6wSZlG}TxCFZ0K z!a|n^4`>E)e4>^)DnA1me*7y`4-+CXj1eJmeIVd@|K<^fZ9fz4u~27@c_)J-rsY&| z%JYfqI-wyx(6X<-OFMleB=GF_r*gE+ai}DJ@{)Q&Zyf zpk*2`d1^v6!k>WZu+mze9Bbszik~Ajhfl~u$^N9_2F~_v{Q(XTO*H_MWJ2gzXBv!z znwr}5>>mUHazzV>W==$HRxSU%hE z5?~F$n-T3jXgSsoF3K z7;Bkwnle*YyVr5$@@N|SJLeFV@9J9mKi@==5>S9rur=}cFs+E?!HR7FWhy`G^g9FL zs+D4sjl4zHs(0r|^dy0-55!fS*^KQoEYd}Ap@z-ijT{LuMU|Q{3@D&5G8Ct4j z<^dl>!if*>Ni=>Gx%b5twgWB$Pmx9!ognX^1XRr@q$XOgVRol&@6xB zBX}bDEFN=(Tcg;#Ua_g^HnbfFi6|ucSNI0W%~u(`_Y`brLv;W>y}vin5sb;kx8mPB z;6pdc+D9VG&c+}0iYN?#W%C24@f0;?V$C9B9T#vM4AGUqoz!}27T9qQv~=T;Bi9|q zK&p|(y6_K;KYbtnzAi97fHUE|%4I)&|G(B+O2naPfLkiJXI7Q1$4|rU%j%bq`Ao=2 zqp3|${dW6pA-p6h&6baJP|wPV>g)r=o5x7<)0@-UwO$nE)WV?xi7!lek~OkN{1&e^ zo-MR_*nH0*Gb>MSIA)oNQ;)XBn9ZYenHhKcD(IZcadou#Dut|2n!hLZ`!pK`(-ekg zVo{|uF}k_+S(Ezfq5&Cvo7z|d;dr4cejQ#SO_(iAlT6lv%y5~x=8yUlPf7-IWf2_4 zOxB-z@^*TF@wKz^kax-PTKa7-c0hN1gF}wAXD>4GWj80^V!XlNLH-n% z-E4Dr_O5kWpz3{O1m>V?YIv zK#+>DViHdoem)!lvIrz*`hf?s=JYmZrlpKKfdhUFh6Y*!_P9Ht1AZJnoMI;XJ{CZ` zcnNm)lRt9xw;D;spK*gJfLJ|t$216r`JQPq9VCEjXx#A_sr%?hgrekLImm5Am%XdT z5B{Ac`TKdu+Y6L7W?S(Xt^pj=z?(8JdL5ir+ef_?w<*BzoTdlQW_ZPD(G^cT9T|pA zCaYKNMvnqDc#yJ+7-fjC{~#TXRe2q<($iC$4DE(KXV5ZWAcxXzaYpga#I7T}SMbx= zGAQFX+lo3H6sIf!Luq3~YMF<$aEixpgkJI0P6zQNS%BQl=?!gwQ-}a7q7rH=wUUmM zm~cZ%K0qb$0g9Dzl##YAO84XTf8Ka!6ixB7y%WpDLthuWJ`q#RB-8h>qEmmlAiRf>Ol@a^$!|w*KS#XvPy(O_L>o%z@?vBs!`^;4#Jpc-x?P zpq%Xx(sSYL@7J4r_Eqvu&uTg&+lvm~0+a{M9LR3`E9eV}`Y-=0bbUhyV)tW^CA7pj z(Jeo!7nj$zo7w?}C}4(=)$4e0d>;I3I3j8ULpp>M&1c}V(804oK_Um(>p0Ve#6b5` z=4(;3Da0g&Y5}8HyiD8hqekl%6MaTS>?xl3b>RxYy~c#%$5iwU6@s5s7wG6OMf~0n zrVW7nB>SZI0Q2vFI6$n1^hQw3XL=qiDQdIhuxV4GX8UCwsvXNy=|3@$czu{9e(aQz zWjw&^&wLv8J1n(97z$^_b*A;jnp{6ecP^i;0HK%$I4iVf-EXV2TD~8w`*wK}HY7rE z$rv2I1U{q+m#D!~J0N7V2CSprut1Brt+BEN0^L+@Q|Gi_yqzTVBX6gUD8ZJu=d znC-4&@JyHSE7C7CB-!TDYYy@MiqHB@LojaLtFIR1`UolHZ5$xuZ%2{w`p*&J(hEh^ zIN=<7x+hg9+BFy<%vGAW|3TQkg-T9Cc+0BEsKYAao9eEVmef-XH8GVcj^+30=MfoT zERmK);hwp7SA-T%RPapp-+S2HrQ{2GmE+8#pVXQSd2>2?GrTRU+0)B(Y;k*P1zz4j zqKFm4p6`ECSLP@zn4MNc)j=TaqLGNozV(mIqYWL9ZtdiIq}9EH2|>v%yb{5)Ji#QZ z_ym!QI%wEr(+{hk{g@|-8jgY9$N6C7&?nYYKxjfqus-}S`aMz#mA_1A#Wl2@6Ve~a zVNp(MC)ORci1#J}-aCw5HFhn{#JkX}|FVX|n|=WOfM?T9(pgHLcPJbK$~1FOBnBlw zer2txWRo~bAC?B@LWFB1bahtPv%m>EtKCNe?ss)n41+Ht7}1PiPSEohfnGbOv*@NZ zYqotGAcNs)6>)d+8X#M7)nHus!S5zqEaf>9YQFtKfxUO+(^-_y9>hawEhkr z!tf6irvj4Pu{^7o_8Y2LKUyVs0Ca+bOUxZltTcRwNG_FtYi^n3D(bg3$#y=L;AeAZ zd2v*!hQU6NQqQ66)&GoyDc+}SuuG7-Gk4OJAnNzbp4C6k7#EU{hx2@g&AuE$^fPNN zlyn1e%Ddp>z68iFI)7P3DcKSJ*cDJaY}MF~3vr5MaQNQdj5hoL@Wf%Qpt0Tl>^8aE zmk?u&r;0)=IIa?5V-_?~G@A~ReHtH7Y39%#0rN?w`wVih{Bw>iWhqzFsz^`&!OSA7 ztlt~veup(5lS7ZBQ}I*ibX=h!3k_fT|M2Kb97Zcho6@5Dl+BadGhJAt7-@A zX9_Eu{l@xdP%26nDjt#7y5tJs&~qZs4o5)|gGBECM~(6~TGv%81pq+Cp$;u`_T53W z9S~i0PxHPO42n{GUg{`9DJE7A{;&n<~32XLsnjo(Lhbu6fGew;^0?Lzbs0ptJovFr55d1`P`rjJoz|D9LD^yo(%ni!02NA_ zN`HTV@mnQbIn@cwJ0xzz7bO~;{p5nmUbI2KpTEVD~;b8mX+qdjf z4T#%Ewmn)lrFMgm>YlSsy+}XE*@${lQE~_rivmm+%`IQ9rey?kXK9B_jm7gBQD-Q} zc{_r1J0Po&i>P=K!I0B;R_Swj?j+byI8f%**!2LMepTvMNqWv$jzoh^2(zfr2USW2_Eg^$ThirF8!SklT4-4+gTXDB^$X zH~tHC!hiQqc!;@AHVdv6t2)wlhDDk9P?9UJNHZt0Mc?(Rm~ zO?M+LAR#Rc(t^?@2+}Dn-CcL`t8>o%od0>ckM{}D1+G2UT62x@tw9&Q#08%t`RlGS z-su?N7w?8L`3H)WKfXgCjTAOFi=INnGRfPg5`9!2(ctH{HP#ogZQ+Hj*C{N95sJZ& z9GHldVACnzAX5t!WEf~Kf8@@DQ#X71Kkb_9R>-nNX4|`G`h`^FvfFVXYKDvJ0m8<_r8C;cN~`s>4J#rie7 zn4`b$#fRxWW^1PoRLi4@@uIJhoMXc_cHN#BZ!n`tOJ>$#icDchM&0GNl$4TA1$cxF zR?)`?O`G+Z%yMd)G8)Bq5DBP?5^QX@H`iFF#jc(9iOLiUmIwj1_l-m;C(Ks8<;cr< z^l1h4p(uh=-M5Eb0aw-l9&5Umq0Y>sAShwlqGTeJt*@lFY9Ou-dU+LJ`iPQvB1i0* z7HA$u{_6ngz8xg@!8*HH>*QBkzxbMx$Cptq@Wj@j}Gf03%#f^P@oEZ4HH?z11ObfU&-o6 zTRK4H>-;j1Nw3LLF?6wlk z-7RnBV4to{ZnKps$taH_q#*X=%^r$6tt z=59J^;5CQ+eRIi1#e|$7b0r9mQ0F?Ju5~zxNa@(0!dy``)DHFQ4G<6~Hoq2QdYP9a zKTw2%$0A}9BpQ{&5%`0gRb(nUfiK(+u1cJ+pd@k}2bOvbphkJC2BmC=6QXg;@6*y& z!Za)-CqG_@1;fGpa=^2wC1L)r#KXe{)CvV12Y#osBKZg=vC!(Pg|}Mt>NgwvV;PL! zF8bnRqCNxhZTFsv$le;lor+eJT5i(iWOa!U9ikqA}`3e2MWv!fv!@^&)xU9=( zvIu|)@f^h?AKqjRmKrwU&s|_*R0mwT$AMx zoA^_l@<&+#X~xiOdQ(hAd)S@6yfKwUUEEK@P%MCHYcOI|uI_p5o-3Nk-ziV)^#R#8GnbZ}r+TdS5gilQh zg7EC2SM+L z&5YFaDIgM%C=5SakLLmK5TPc_#TKup(d*RO z6A(Kq?<^aPeLyziD3Dq+4zQZ%gI$SUt1;x|K2VOU2ke7!0Pz4N)Y5-@C?R+Tf@x^g zwWjMP*TB>E$ID{}5%x?Q(+)^4+q0=bK3Ox2M#WEzDN1g6-f3$j}(2i5*n~TFHf>HEQw9yz1KzqH$=6;9Lj*7Vp zMB+boA5aFMmM)}8e~#3t0Y+wY)cOKIV1#ayzLV0#8D8q2Mix>?9>-ni&hqJ++pHg( zcP;VtYN|=h2oE|)t@$iFSkpvYF^nON27~+q=;QT`ls@!^aRE z8mFTlv481&1=uu~JU}h7rd#|%^OvkN3AhDOcN%&UOaNm?!Lbe8N#)-NNZR}c!6gcD zXT^e7ZM!;F?XTX_h?Q4MXm%_J=+t;v5I76~n)y~YA&qN{Ij*hx=(N7Q{9T|r08qFI z*9KWO)=1}V5dNs7(WM$3_#F1C#|%5rv}3B!Y#^C{966ZiaGE8IQ<@#Q8m(&r#5BEn zF;=FdF}d=NU>L+13eJmb^$LLiC=TJvRjdcE=4;cQQXrK&3%ou6o)4OWv5#K>QAw$f z#s7JsGSshAX+Pgg5GG}!?e2=c5zoE`L>Ag19$~mHN#N=w>UOvNQ*1RO`s>Q%vRaDv zD;SOfVzobBiiCD>Xss_n%E=3`;+%$?fZ^mA>I9<#UQlN^20WzKHDxy74026?46_v$ z^vU+`81Uo=fD*Q8!LGJ(DntLYS~qK-09p=JwW!1|AWL(cGeFcT*9LPk62xH2g*^>m zy=VoM0ViB|qlpsplSA-l#ety3)H1zB=lpJ2D?mxw2dJ*RFSIv*|FQeCVlu0)e5GVG zfmLZTn_2&Fwoc-23E>qgfH6j?eGBNj?Yako?Y@GifYsj{$!fFW%b9lcHJitQp;o

0a}})SR9kUJ4!j5S{N} zlg}X92*LN%b-aKs5lKXcqF4U%aLl{}Mf+#?jfXf+<+oa*EZZP8@)bx0bAeVq*1$%V zB8Ao_X90?FLX5NuBv4 zn&>6^K=m2KjX*uy4vm^JX&`_3lJ!iy@2S>vPH7HypX=|om{E&1>_`X0zzT1tr!C97 z=ShUreVd+wZ|4|mKVFhlHmbPigl@jhC7Z0$J<;6$^;`FHfwY_=zbQIpYX-hJ_4yGC zNkKym+utwzU+_D0u+EwM)ZD)wz>F2@Y`T?iDgFZwaff(^Ux(bQA0Hk>2@t6~YL8Ao z01mD6<${4g`Wk=x7!f>wfWQ?;KBYjCWWa*!1v5#LNtjJVu7!1 zI`#Ill81!f4*j2@$(eFIu~B`5Zis$D3O$K0FBFW4UL?|w%hAQU@$utjUlSlg&mgz~ zR)G@DX)!z}dpJ}g8i7txzvXqYwua;|qbE5b{c%DMgGRjWfDce@Njr}?(kYM-EswHX z$UNiq&<0?Ji~Y+t4HjFW&ke!^$$85JvZkS#L9;PF)XE|)8nvZ=ViIv$unK&e6!S^f zkNWBNxaC!Hj<{Cp=a*x^Gh5O0e$lc&jdfym;j8I)&?=rRba2f{Fc#nsfU00c%y;&c zXO{=e3U{hEs-%2fDGQYn%a|?A$c~w5rX5(CY5p9OEV7hvBQyz)%yrvZ^r&B0ZC;TMLS#db@pJPWOHU5aq39aA zi$u>P(@ik~HQQa&9yenQ7jcl;X)fl=;WEa)kK8dfd?XWuiF7&(B96;v$@*;TbTnWc zin8T5;nc@)eNro(mxeX*;-(o!?i&Q>GzE*E;6)jLSmCCK#V7DAuisR?dt6uC>+vhc zfQWP>aAk$t?-;hRN9@@u4*Na&KKdIX`i&on0lz#z@-QO%X3;kuY5_yh1u!@7G}|$7 zhm8_%;|V20cPqMiTsU6P3kpacezzFGtOuU4u&zd`$S=_+Z_|>VQCBx9qXs13nPj|% z*S~@>8hBzj<~Heyxf}Y|Ro8=O@o%TV5JycQxDmo^S&Z^#ZL5%HGHUQ}_N`sC(!+0Y z0xA#aT!WL>zUXuN?k{t?Xk>9yS&Lp{y+vOd&X;q;BPq{Od{*#3X=iYLqW+i_MJ~rRW zX^auC5rx}AWGC^#hsAIR5cTUyH{^$fT@{`3IG`xw3dDoM7jQK8edcR5EgW2pQ+;ze zYRzggd)%KKwHv3t>5G`B_Cjpd-FA59Zf$^c5rsjF7zy!wn_J{rjAW%Y9BnUoIe!=) z+`O(yoJ|~dC#&3>S3~LCiU=g;zW5Ya_dyM`MM=i^--7YS(peim8+I0zC`GYKhx}k= z>+YYSe&W*0O#VZ2IPeMco#OAdM*A6|lV~?ChlSEl5Wm4!Fz?_#?wiNiB2Rt)+78b# z!77FvZeDYt&Lo&+wdDCj^jUrp91QY!CQ7GWval|V{7Xz7)R*&frkdS6`*BVU)!Fv> zq;Z8+$dj^j+?RG_L#SIU2W&9W(63pcGUtK{Kg;QSAK55fzz2H?$}Ac9MaEHHR6REqNiZ_sqR zPVlC$gSK21Srbo5Gm5r0XjWy?uNAr#(aZjU~4t7V=% zeTSPnyX*gL`O}nJ^=0`b|LLuqy!3EAHTcyjn^gq)#xn|Mq}Ol|_D z^)~dtb5}&<;kP0nQoZbK`YmT#A7&TkR@abSp(v;&*OF$oJuZz=sZ;vkp}B}T0vZzTEX{s8LF*7;t1WbpQf&vwV{y8dG; zvlY0P%$meEBp>lM7X0LdPnFRQ71L75$*Iq49AA>{@dio46C&uA& z<1N`0@YO>%ahchv4%*kqCfWzohlF^~exLoWPKx%CjAd<_Px(tzLgb$jB2rF15*FyE zB~R8W=A7Mchz{#1LPYN}zigEGt7?#r&863;NXX+6*RmfU3(3tr!rk1N0&Em^&Bq=! ztiT=Pv&8MEPq9q#o*3%hR*FwjT>onN#%alfLh~IWZQLlwB7)i@^b;Bj}~G5BwJD@;4HuJylgC3$z$`uV4->&@d&- zS@{isx-M0Xx*aPeq4LlXWQE+JVE0LQL6wxj5|n6dkH(X z2@$z{77z@LuX|j)5Q-`8!#wpj`BfjW-LCp5gPy%twfq|p#8~sknbTOyoIF~0V{%fH z*z}XbWL9mWK?f486;+Cvy#|FH54K|&D#bsWv1-J}J}bNDeEOMI^A3$!R9Dd!@&uV< z0hW>Zs4_(9X6;WKM`ugKGCWKIOM{D#rPuaQw7 zw77M*Je0bUP`TMjPCO3{qn^T?5IH9i`Ap9yHB>}}Fv~XC|`UVw& zqK{+W9NU(^Vd63)BM70z&^UX1?RKm{B1Bs+iew4n(D)cUGTq!JrYlm7NsYwLhN7WZ z18!D!fRsS%Tjd!$PvH=v0*XMfPVfWQou7^Fz4rI5^#l(CyiVD%Fe8|>XE`QxjH+_z zG7X;&gkjL2wo_5H{J&!nsov5|w*BrzSvytj@r?<5o4xON$Ps$N%{GA(___o$1cm~~!plbLl zz8StGWz~kWyuNdElH2`y}1X77cfPb3gca_sw%!x?Xy%*){>K3{AuBHr8F0 zH(t~@h&T$+J$8fnp{yv6Ns4>mSG=I$3h#Q=_txNNJ@ppD^1bMs1j_hxXuz95%Yjmi zDbknQ0Q6BFr*>-Vwa4M?wVUY+8?Y1*_~^Rg)@VP^N0|W04?FKV?25?2ndfdAba>Wq zIQwlMX-iF7DpC=bkyNjMv^Fz-B@eY6Wv1)t)KxSLY5>E>PAU&SZfIpw?kzI2MUN|{ z{c!6BFMXdGCk88*`2-^4-LMVywEe&On19;ln6LJt5RC-OW<=IR%fm5CAXO#W^5zS# zbNRhG^rZucz64@mQ=Eq^Xl;h|3|7E&jWj=}%y;nZRFnW2o`d;~byJXq;H- zffykV!Wh8pW&>1dNJ3+|Bk#l=6Nu>aU2dPzWn zK=Pvv%?J*ST*^lb#OdH$)VP+gf7b^6Q49R_IaYaKo%P&?cEr&o-WQKn*XTJ6{7vls z#qa+67l;KJAoi>>Vp+o1_cE>g*3$oeFZ`(r_}2pm%-muRB2Z%Li7*bKji&pStB&=4xz-c{p{Qouqszo|1*ok4sA+p{>Tgb zrw|laXU_0Cr)|_UF5hSd-A0*`WZD<6Y!ctycpm3D_43y@IxbN3CQ5%S$)xyMJ(P7u zDp;BK@jgAK>pM%y(_6;pFd6-gzU3*h`H|8i0hcOvio+3RHg(b;eo-XSE7J&eB@AfJ zK0jk*=t3&Ish?}`Y-Z5^qGi3KA%CRe36tJtoRCPDY#@tG8<(xR&V4A%y)>y-ZM%z8 z4D6v+S~a!Uba6*?3ZDim7=@5Jl6WPNjvym+d>PR35yRsZ9!3^m27{i3X7xs*_< z-U>8gVs!Z~728NABA0!h#s)FY&G};WH$N4aV^dq@n*`ZJAMA(M*h7U`Yfq{VI&5^P z?uYXWk?|J2B?TJ~J7;LGPDWPqorhj0m`?vBW|x0iP}Gb2*S+S300Y>G1OT$h=DZBf zGwzCL&WoPc+9dV8Ap>1%!o!FhgOOGSKtY0XodNiRvao4YD4IM|nD&7o2!}~GwImwm zDWWtGPX_cAb|%;^5pR4?FN_400I_k%;ZD-XG|fG6Df`Lch{xT^$f@#xJwcl>4J|oKG@JN9PVtL0u+Cg3T(k?!7bVc z@{wxvI^wpoEzb#!G3~W~JP_{?9gN;23hjF|pZIH`4~&6KmZcQ0`*NR}cISQ`y$_sr z{!MIRwzSeCvo^Zfrff>Te9z@J99okvnuO^jQwzMG@wal%`(VS9$G&FSa1Ee1+)x0Y z5-(%~vA=VXyVp+4;wi<9@93g{Y=LM|j&4l~{nVYHMB~v)P7Nwf?`mlsCt-ab@U&|L z>;r$-CvE$QWmo5pZT7@p*DHN1i30MU2FukW%uk)Xp?~c+GVll4KEQ(5AA&;orVB{0 z@;wfhy|D$rteL7mI0U|JhQ^aox!1(AAms$oc7NBb3-XCTU`dMVtos_kAk}*TR*O}G z(EA6b45qO(q|SEGlX;bX0NE|C&w!cZV4dg^3hsv^5Y5yNNZ-Vw0NW;D%E$+pGwU@! zptTwS7qteuq_JJ#5=;T1pmJ$!X2!{+d~b@tARG2-*KKvmk_Six*Z@*6rFG9Aprwbb zmKiSbDK!nfq_=lgD-Yl%U0#?^G(UmrCKQE` z3QS}Rdl#4i=qdiKQ!`V-pban@kMa*8Upausn~0G|w+K<3bxGeI_}?u8FN_{j!!H=1 zlr1}>T$lw3LD&XYb$}e|q{nf&jXlyIU#{00?q!q*fjQo1Zs@5!vWwkKaCRpf%uRJ+pobR)7#ER%<2r(`?X$o9FRXR$*+qafey3@6; z%|wfN0Rn{R@#|OTcW9AjU!`iy#zV@q_b+!#VC}1``g$MSb(~MC&Fvk9t}@j%5OFUY z%y+*jZouxvUAoS`%0yG06qHUyqh^fPLedyW-o2L|H#E=JiM-8MDQ^E(I*xj2I6C^3 zlY2AOsZgL%D?fXv3dG*S^Z=UJMjY^Y;Xt6<~xZmy~o5d!aR+W7uHe-ucNeXkx zh8`LZ(qi4ST~zqqA>fbCtFbP#PD=J}CS?d!l>-9pS+@ZlVXMWIig|{@x)KH=i&Prf zH_@hbpRyFR=;~oN+-Cwldodcgy>V^4UFG{`sJI$jfzwO-onChS_FTpT`txk`t84Z7 zr=MnHQET6L>1ep!iy6_-uGRFvl&0;G?vXP6#Os$SmnhZ$(fx6QD{=Bk*GkjZ=f|;X z>8{XRw)}|aV{CVS*@xnq(NlSxjM@AphlQ_q*f&8k$R1x<=FTo~6+^&8ly|^H=}2ue z(>PDaX-xs5wUkCJw#JGBtl{6mR~LaTj$gwQb?*9u_g=h1>!oC0Nq-XkKw*{--4i%u z^j2=$DHe%XAlAO9I0`aAt>6*0!MWJx}qqX@k zkyH_}Bv3j=%uf+I%z%eb1Twj4YjIblFsz#*MwiDI7$+KgUy(zkTv&?9e36EBYac)7 zr9!;vElYg)c81N$WpWaRH$XJX)z>&z6+t?^aM|18yt&|jRg$h#iz)!2c2)kj$2kH#@Q)K!lfuA-ZdiHMQDnvo^ zeG9bz#?^U#yCzRsBv;w|yCk-~XJD%$vUG=*ie1}%jXRE``f~#!;n5uU(Jw(kkg^~M zBy4Iv7q4h)3Z)Kj%HKHG_qh(TYGU%cJ*KCdad3yii&{e~du!t`%{$AyS;OA@t==0$ zgSX%3ziiyDyANR}7mUmG$_X>S8(2uC;dNe5mTFC-pKKA7DM?{`fl`87rqu!frss?0~HO;`RLu9k26RQgUqcJ0ycl8YXBOa$1$PtloLT34wH&7^+mM z-6?FrH6~j-k;ht`5P5QgjlqtycUi&;2_X}EiU|4|c8_-ftExJh57&3Bj}E$O26qe9 zdISsHU$IKDTlVG=Mpr_4nci0Cws=Vhrqw!@iHMg_Hc8d$dI^#7D0-R)k={EvI4mQK z+Gi=IF#9UnxktPk3tI9(HhEHJpOjDL3fd zPIxqSe`$5iNuei}dF8SB#6sQObLz;$m|GdYEJaAc#dqM_WWXBXX(f~ZCd7}-5Z;s( z_Tq8OW~@V4aCS~>ONjCoKm#O!E-@vfS0vwM+AIaZOsCdz zXbc&-q?wJ^+&rXu|^p!0Ttwj)m(e&8K4pS_Gy79 zwoWylBCe#H#^o=@!5R><;Jb8F=oImIoQuhL_311WEXQ93(EdfUq_`b~zN1j=JE z$Vc>EH(3s~=4_rwi(4v&Q`t;WG>-E;0<&!%w&KIUz*QS5X{++8^j zLpY-9P+h3`ovd4z0GiPdl8r!~dScImQ08qe31 ziq)Dfsk1c72Ex^*KBwo{Z||roIiTr6h>5hqPD!eN^z0=N>7!6;m1jKb8z9~mPxMO2 zjM(4xKZXKFx^zcPCL2l;Rzs_>M-H}Ae+`TJ>KuWfd-S015^VABF@WA{Rc3GhO3)aoNwj{dOGd% ztT}rAyi5o~Hb1kZe#Y>lh<6F;O)Q-0g`;jVJi0_a=U=W2 zbQyTwqyDCCQAGk_q`X2kV6l8g@fI1&l7Y*RjzgkH2*v+4QlyT2gd@@Rd-ra0nK69g zAnK}nPVlNU`6X`&$LEiqun$)edJ@HL@Okuy$_~Qq++>MfB2*RlOCZwmy^GG*tQ*;N1(s!R`EOImwkuooSB?os|C;WNydqfIx{qk=yh39)^qQ*jmGjX@0=w&*qC%19LGwG=NAbO z`=P~d8A$Fi7Eu}%3a7VFo}uU`M%m1FJd;?%Cd3E`p6T-VGYwrBeMO4_XVFOKl=9<) z=q#|vVDq*i6R{+ad>$yQ<^uV& zWd!_}d+@IUdB}74WXXGJrAeicZ^_UOg9DJs_i_7{t)7rGPXM{gN*{?dUKj!;(c8!8 zlJLZx_>Etc{9a}7IK@hHlri5l7M^AS-`HrqgMq*J%d()H4C{Hm-A?@m!7%hNd}&Pm zDWL7xmm!v>qx=EEBJ+Y&!HbEXKK+Kn(G6!}+k4r4`6~<)i49>~uf=J!3ss2*ldbQ9 zbq_rq&m=Mjd8G+HD=2IiH2i4KLVg7#Mx~^qNTd-!bF*qg`A%>)trs=O}!7|Sp8sDs$6b#LfsPQPjxSa z0MAqeYcZI*^xS!M<$Ce{ZUeXNM_aQdbVWDnus}G-J(o&?7MDKbXI9)dheN7e#iErk z<`lj)30$?jVizI2t5CV0HF9(4A08Dn%SZWo+F)fht#fSD`?A=^%fD&sGiKzFI_ahC z$y2EH-Sdf+cQX@izN8^#RNeaVmhA_I#f{K1_p>`^22r>+q$|P>vxK$3w)LCsag}?PFTn@LWiorUP(UXN$el~maM6Zo3=qarPt zY!GX9`RW~VI1UQh(Bu_DD2hHBMW|3G4es||2`T$hwD~7U`J|>3_$YFHK~J||bk#Tg zm;%M}`f-ciZ|Qoj&k-^*=U=bg<}eKkzIa;>dDWF2K(4Juog3RR0)v!eyl;52KGCbh z&-(r947fGDwv>87`>p?6K!GwPO8G1)#`RWy?C`PclzNIg3ex10khn-4J$-J5)cdIY zxQdw`k8^mlz=B;#p7$Ud&6y%1sMx;XB7s3q(tNKrHE-II@w^%)VO1h?_xXDpi7y3&`F~bK6@e z2-JNC571YzxYejILFGn4Uvf@VFj2ZNFAD?c5lOsvs_yvn@ZoYEABN=fw8OxlR0qIr zqnq4bpPw6<7ezvtS>T^>_S5@IHdFlA;0c0p1M`!w6~_cuV*)(Lbrr@%YZW~RpkRC& zky^+l*7=d0!*=K|#FEp8aZ-6+*c3l_{GVd5pgcECuava~TGG1&$Fa)d5~NcMBq`YF zqoBTSLh7RcqoQ2+U{j)oXH1Q&m~D!N@s(StKaHH@#syh zrvlD%NnYo&%9n3C3H8L1?(Z|ISBd)yf{mrbNHy~ zmKceuLa)M7pRtO79Zp{h<09vmotB$5xwr9w$diKGHzN7&WeWl z{;<_*b?y=B?d2_eS@^ zDS+nzx#6-&(p_2|uwF$G(Nmuie00`bl%=aK5Y&|UMebe9D6QEBgIvoMKXJ+n2C7Fn zlA47~s@WS+t|fvb0b(S!WUKxi0hfsI-leZ|;vmrV(}Bdb(X&G%{pzS0Cd(i%hL=?8Z!Fsu3jLmFS-kF z5{_B)`MmFR^@lnRPaF^J<)T0>n^a}sYGE7A#nt|qlv1wBlYzm*NKY^^hF zmSw9#TYmC1emxT@8Oh$Jmp%wB#|MPzd{#77#4DPR(&K|e*J;Tut1a4MU>QH68D^<` z#i6hY^AQbQH|PfQ?h?FLu7jyL1d^KYxVH)w%6tK-KqPB}DTrA#gTutLrQCe;+Sd^Rs%aw5I zr=%kp>{+}9qh)QrW_(uEwHyLRKL(;8$OBS*-@ToF#(Ys9K7)p=Du`#x&0dG~YG&A= zRdj3a_EY7L*=eaHx25>!Hx6Zni_%}PxS~lA_hV1z8|SPB%`(ER86Y^X{Lpbs6J3q+ zHFLKkXQVBu@o4|)9gjx=#T_;#f_-cq!J2N4eURzGoQ&|92=|+# z9l$6iN)x%v=t8-I}oKQuo@nyGM|?jo^0GT9Dfo+oDKX8oMH4^l5X6!1L##`;s@>qz%l%d}q> z)KYOTZCyQFqtKFqvQ19q_f83Q?FGU(srofD&(a($JQ?oaoRfeASQokg*U)-JcWY6{ zmciRsQE$kaqxfaCOz_!#d)H(hA@%j~1l&jSyA*w{cs+W*OG$_I$!FH#;Ysl$i{r%j zA%1^jC>377Oi6R%;pH7`{cJ@_h(d{H5IrD_VlU1*4iOz)7Fnd5W+YGMnoL_$6dP%y z@`(TEp=5f|S^zU;`>1mC@UevFs|!-~fD!OdQTscJ-H<#!w;eoML|)7mw=wM?+*}Is zV!@mWqn{bNL8B_kxQ$gbXjN&J8A_b-5Yi)Qwcy&vX@%A_tqD@GO??BBnie38`|Stz zqe9lZ_i|CyMuAa*B4T8cG&T${(Z`9|fj6&zK85G-&Dz^YFqzXSI`|3{JWfQ*2ube= zR*5>DWo084CG>^Dq{Dz#G=vXCsyoMYVF$nxP$!tg988~XyGxM$Al-d}sGyqg%{zA7 zh!wuU1NxdMfPxrmp$l0*3w=txi$V4~vQLX@SD3hQN~%`gEBCX&+EKbFq*3^kqoB=U zbZ0P|dDUHAG=vM|Rb0IH#ws*-`zPNr^hImx`OTp|6o={uk3;MA7gpcPa+v@R*J)Pp z#_r{TtEx$HM!>};XGP{k6bkvPW56karpC>2Y9+Hd!Lgyco^qkdcG1 zJjb<(l?QPrLrKgCcuO_m{ga42b4s?ioIlL(Nf+}Ay`_OSeoyd7z+Zfyq1m{zRfx&9>%r-ciR~4FqeQ zD6_2en1~|!hu6u>->;Ji$wzEbGT6@I7!d^pTg1I6y#*)lG08wY`w1N#=ISRzlu^GV zI!+k@)A;p?Tw=_8tQfdU(NWL#)uLx==%%FfL|a=Co4dGcWC8*7($$K`npK&lpkb4F zY0u_t6fDD09={S7gw8|J7z*zV><~ZWOb%y$X6PjPJCY@0ItP1!0Il{gX9Sq1U8+;? z%@pG3r+6FLsN|P|JkAUwJQ&@Vz&eGnlpXhhp&%Phw5KFF&b0QM?7N^RKT$g|VIesY zhEJ?QOBikE0_SWPwEZIJ!pJ6N_E?XaSQiAn&#>F&Hu;gb(7UDYd&%K0BW0tXK88mK zQ8G1p(k@Sy^q$M^eS!T}zRw;djzV^pp`0|`D&)EmXHW1bDhGl@HvumgVpzZcn~7Ph zOBx~H9<4>;=e?2>N^(U1*%@W7r?enq1n!d)=FxzlBD@QTgMG2ocX-v^fYCkpMck6} zJa-*qRRB4l!g3_HIXLSo(EF0By)2{AQ)ecwzn>BjKH==5GRX8D&g%7{@1@LjvEt`L z-d;<${;ci%(qm>}g1gWxX{MbQin;k@(*sOY0)jNX9xEkY_D24Jt4;1y;;{Jd?I|^t zTegvsBkNJ(C#u%Ui();@c2hyIutN%4O&JcSTGZ#kD&|KQf{(oXD^T)|^4(>{Z0+{} z{|=+r?FBwg&AHIePZK!d>S;ZUemdXfQ|ryNv>X_#DM-WJHI^Kr#m%X06VnEe-J%dg z!Uo|Y!jpuekL9H_k-m8Pe8JiMOBZTi0#%~gz;9~JpuWaW+J(Q3?x&CY#{_WRiY`g% zmr*m9QVtLo){NdVDy1Kc&xnB(lilQ0=@;`IFruevXBE6mcjL<5p zUX{XnyPW*-Det+qJKCY3WofmTUKA^u`>!XN}}1IH$pqz=l<+Ay}3z*(B@Sg zczx0}_}L)UwFRkm5?fr|vE}*20o^;cL=ZG*xwNlyKt+Tfh!<&oBZ%U*rnQ%@=8RA% z(N5B`pFoS%!W^(G-nkO(x9@_12jFI>t5)tk-1fI7 zS_dB@)u>v8dqj&zSzpk$pJmH+IcKF;#&BB28ptqlJcpn^^;aCW-pk@m(&j)a5Jn(D zyDDfBK45?4-gPU_&^rY^GE@{taN;+5_JFC&y>N{4{`J)+6=1JleZig{15fYv{oGkA z^2qtrDMie0nWUWBTuvtNZaA)6(H zd^A@lCEJG(k;P-;3qMn`5;eCjD*x~iuZNI&=0zxRC!tC4<#;XXSa6gp5Ksuj7hJO+ zVVU7|j_ZiVh>0YldE_UPcx-jseN~wPG-Q;JJqCvh}sjz8meiu4Yx*ctyvR z#h1lVzv49tXEd64z{vQ3Z{AXKv{-<;2ZMgzK4eJ z3=RAbyQFhvYJc?Zf3@&`fAPPkfhn!yb+o>cskMwmy^$iT9pHjJLwfib+^I6<|MoL~ zf&j?=J3%6rOTTu;T(>{{eNFy4zyJIa#|w`ap3KXmL;nXE_eT-A0AMKzfbW$zGHg^j$(UeVbk@09RYp}LP%~%q0L1iu-)`OPE2GS0*+e$Wn1=t zoY9}up$Tl1_&|4{g&y~RKchcRvzO@MD58wA=l}PqBm3J%`Ne|~?FXgX|7#)A0>M#f zfi<6v|Kpnf=ZeKeW3YgE!e}2{=l|D2xS)fhTpa7v|LbD@^_SJG5BK28x0-UoKZ%e3 zx*n#e4@aR6O+@^!HRY20DqU+U;4$I)zZPOp6dX19ZCac4f30cv!)+7&Y`Dq(e=WrS ze{=ouu>5~~bM?is>LEF{`+eR+>iah9b0dhsy#xf= ztqqWzv@Wr7ars0buTR5SN@Rq$Ip|LaL2 zQi2ER?c~mau|iixZ^6iick~+C?#R{DTHYXQ#I!$=*5}3VUh-J^vs$S79teUwbbVT3 zlcm;*``oJSEH?t`wFS~q&a9y7|GfSFs3b?d?5+#Z5OgGHIecy|IMyC`Kwh8%{JDeV z;}0t*9zyYYeJ#ZRRXyFDKkYbyRxCe7&n?Y2-wJX-Iqmh15x0nM&z$Xa_f?w5#b}l3 z7OH_r^@e35z?a4~MnG~w<9_RP`}nRhX?v2T&eH9b;D{5WMb=-N*E3WYnk?m9_pLG7 zB?+^2mlxA@H1+k#AL@)RUHc|5$|}N#UcJ%XMovNUopkoTPLuyK(l0(+Uk|N1{(Fb_ zzkdNU9Uy)U^bh1FfmiT_$3O!S5obBb1#%I(S_eUbIOBt^@WI;LpUg=AV1%!=o1Gxt zp85O<%qatP!-KUHh!GF13xI-2>koD`kI)Da-6P{?m5G$Wu%hrS1^5)@3?O7qDQ3LF zz`(xs(p*MH=CH?ND1-JPmK9xFbWur%({@S;UW^!oOt*l*^j0E=T7YQx$O51l+6H8i z%-|Q*3J-A#+}|9tfkAQ)v)4eJhi*e_BvVuGdD0aEKci-4Hagr4n7R1i*-m88QfZj5 zWFPfHMe)O`Sp>fW7lqvefZqA4#VW1$AV%SM*E#wjlzY~pO|L|~Z0lMQ56m@arF($P z?E+38{OTZdE<(6mGYdRgLb^{+Ase9`Y8@9_p6Gdi7mY{2qVk9W-f!{7-MZP;aOw{S z*H%H-K;FW1AS^UfYqc`adeDx<=Dw%vtME}WT3|ht*e#Xx*%ztu<~_a#f#_g5_pre> zxZZ2m$Z6eEtU`y~m7ptX6*$NAP)>F#2h*$2hem-W!#9H)8}7mE3zQiY9wpSSD(sEwY6=z+^$(3Tb{2Rj>-&(Z`7H z@O)!?0>>WuDYUO8;i{ixR04nl+AYZ*<#}NKKq2rJQ#3kPM zwD|Yt%bnpzV4}*!2JhVxfJ8}csrI;p9{C_=UH(6O{bgKN-@5$|s~{oW-7P5~-QA6J zw=_tXzz2}-k`kmFq`Ol=8U$&iyZb-+Zuj2joZtPrpNJB&Vy?O7ysk0c<8j9Yet;>E z0Z=q91gOZ`uLuFdpy7#Jr~<+S4We8BN(jJZ5xPMez~&w!C9iC9*_9Yz{S+tocsFrq z;TmbH&)jyEBXlIBdwdTzuW>*XcPQkCd z66K0qum31SafVW`4z5D-Q)JaZ?=ko*f=3(aA~OG(k`=JHGqZVLIViddOT853Aggl7 zC#~M9;HltBj4^{S*)6tckp|G70*f@l3JGq{=S-;MJ~tZxOn2%ha&v!~xP<0D;moCo zBpH3!Vu=uN1o&*1^AOo_j%;#NUr5SF&Y4ea@;=KQjgZ3C8l$kc4I*ER46*81+B}J0 zz4hfXOqWD{nmR_JOM=^+5E-tK+hSC~oF6BO%-zFVQG*a{h}R;il(xcdveHz@JcjE; z42=)H4^ILC%kT)gps^@oYgAFc7eKDoBQ zZ=55Auw$AolHiP0yamOms1&1jkfrN{Jm(=y0y^>q zP_yo2RD&qFev-cIsW~Y|KM*M``L;vnOh6R9U*Yy*LIDnLd z#p9s69Q$P$94^mu7M*|oi0z1*C3B2#VrQxZ$D=&^ z%=iV(77EL8zbw*QzT)?Y?V%RK8XxUt0DHLyNmb&Zn%=Uk#H6=38|KeF}9yf){pd%YDPQ~(%E2RVQpm0 zGh8vOQJ=f#-pFs&Zg-q1c{$`(1F+hVQv?CjI+mv^oCf-yfSiMro9y5yoQ$#2T+T;p zKl$IH#HYF81a4iXZR>$96eFikkST?92oO#G&5PZQ}wBRDI zUb-7Vcig1%)v0InxUzpG&TyHE@h2A#a}SA!G;uMbzDMCZ`sgO)=$lH{TP0T^pUF3u zC1Ocxl#V;@lTLHf!9Y$Khrg1q>jq*h_|bZN-#uqk&VgV{-(ZB`cfC46?+UAYE`r_> zrlcsM$da!2V=Z8!^fq8dR*DYU!7~z7tMz~)imU4~WH;3OJHE4^Ay+Anb-77zGzQfo zt*GBBnkQFDXkQGTbA*>1hRCT2%!}AVuKxF9m@r&Bp$^lh`+rwr%N=4LLxl&Bsnw4( z-1~~DPHiY6McBXUQ&#|E#hi zK|j|Td=rt1{-fK_{@?qx4q~#HJZW}9d~?yMgSK0{-)C7|HW2wZK~YAoHA$`jZHFmZ zC$Ykp!z{e`qe^d!UXFHmLX^k}{T;-d?2Nb+zu&x(mH8y*DW~!0)y3NsCC<`|aoyLQ zShC$CSo^S-V;1SUE`2qA78x!{ZnGVkO6YK}(?#`Ze_+z2_k-o`ww1Er+Ys%^Ku6cO zdU5#*x>+~=KAM?qm_IUv@0T0|0@nrTyH;{B9t{jF!U2tPKYB7?EUA++=u~N4^v@ zc$d$O(18SZX6LTBOY(Ik(!ch}{!rM@Kc!eUc8}lM1g5K)WD}^D ztnnK1k$4D*w$VPHs5X=JlY^%^x1Ajnd5Jsfh$K9D_#m`}L>>XTmB|GxNk%pD^W6r= zHF5~nj-#4YVePo}4vlad-5}1{c0les&mDzzDCpf+H(bIbu|K6Fe6#Pm6V7B0XDxTo zRRX`bTZnK=Iou4nwY|J2ILDCwk?lJMHVy|qF<>g-QG_FwA16WF_(}+G6kE4;WTlUK;B_v5>&QR~j{uXsp7%jPJgXU*hLO>--OG(cz4D+&^nfnt3avdaL97XRs+yuhE{46dR>LsyHg=#_6D8&op z1Qhh7^lufBe5|N}v+ z8;h!OKy@;Hv<56&OulIjhHB5b>h27sAq&wX=`+4&3!#APC zcyau~l$5ZI$wvvq@agyyDiJf|GpjlU*V~r0tpY!wo*{*YU_DT|FDI|k3xC;3d9OcX zm}T>~6#i5sh!Z6pemzEt`$3)VM+jOArOOa$egIoE?Ua%xoqzD!$FErI>4YcNbp9Y- zbx1M=U1*0TxLtH{8-42x^3M@10yZH z*BJ*$k_ovjG0v#WTy?tiGySEutoc4DNxeqMBkrPi(9#kk{t3CGXGqnEG%0iMLVDa_UUA)m{y343;isqv(hK3?=@;-usW)1g6M+UfK zJVGrInhBG^2(p=>VzZbJ3)ah0^LwhSNp*P4i2aFeC!)c+9=fvR=@Szc^w$5H&XTI_*E3JEa;Y4hV$1gUbsxNEwyxuJCWFzXH~ z2^XdjmL{tPY_>Q77mdv4Ef1kE&u}~uOvz4fBJE(s8+j@6SM{1^v>|I^Dh31f?i;je zrDIGpiW!}EcAckm-8O(HVX9wKN7pT*J^7VVqcSRDAvgxw#Js)Mw$h-*y^2INLw+${ zfRZ;L|V|NILT#UDx}6#MYSKYhFZ=YRgs z_f^)znOaeOW@2lr$FbV{@4E_=>FeRr9lGGX)Pt_ydcDlbu=(%t-J<%po?BlSX;iDL z=Zg)$>o&`H`=4#VzYFaA!XL%irYj7XOPNMH=P8t0j z+#XMdp#7)Qknz78&%X!Q;UC-q@T_;Q%1iS{hKlg4xSju;U8fEw(5-1?z7=7l7^`dg zXKI@jgiArUeO$IH`|Fzd?j>2Q=7m(l;0-Ts!D<0K715YJO|!W;-OJfSgD8mu-OmH`0^RFYAIc#{F@|9(_c zp9;UJmO48B=VpSQj}hEcE$9ix3`%~Zl^o+bjxEyV(7X8mqfg{~!9f+*+CS>~ zS90<{1_ttTr%bcAFCzW5fIlvRh7QipN|2a0n@h+pzR+)sLtbCVwRkPets#zssKAOo0 z%-Bc=b${QY{x-o+eE^$|(&jlFe^A+>vD-{$&>}A-#Hs_h)F8JxxBtFwL5G5iPul+Do!&-hqbDf`8i zxp)G=FFr?UV<)O)wnF^~{et-&`4B)oel|%x-T+>%)9zkUVnm;Dc-l60Jm^$r?RALs;sPE{YwPM^ zS@nJvQF)m+{B5)A8i&?5$7VGK?E8l=qbC(op`xeMrNgg!Bn&rh%Bp|04&g9X&d=H} z9anq@q3~gZ>5^ehI2ZlJKdOJ1nI@_5m6^1AcE8iw?&Gu!dbOT}9yA^`8uvEMZ1=*? zjX>j)hR!*uJef6QBl(2~`ewl3z_CHi`*^}Q&SXrc6dml+SxWI&f8s>0one`O)PVuM z{7mHI&o&U;enVfZS(UR#Z}4Vs_j{J$QoS-T-a`i0wo&f86v-6du_H_?vU*a_Ic<(G zQWl~R&y)DRt9dXF@_Dl3zeK<`4Jz;1hkDOF&;|-R`E((VO;!`sM&gQWe`mqcNTI&_ z;*pQL51>?(CmMomrN5c!RSt^8+Ka%|wemaxjl=iOed(z(o2hmVQVx=U8%hPF)FrE3 zo4#7XRoW!5Z;6#VzJK=zl)%q4?}7V6`rODsABQaZiC?J;xOpc4DM~r@@IVW#jO|@C z9-#;a3KAduJa4Zia?MnVPcNMoOhof0(oFJzl`gMKUU2A>R>`=I`uGM3kc(Pd>#A%B z;vE+d-gk7Qn2}sN1?wJ5thSKF2)Oz2sZZO?Vt*@tJn{D!<{Vn7KTp>2RQ!b$jN_NS z8y%U}K24%iZW}1*{-Sef?!w(y z#I)3#Ky?(GUl-AWhiL#@rtx{Sb3#MvvW<=6Hje0V0DgPc^7e|w!&+%4y3|z>IiB-( z?UM(m!#}(OLy-^d7*8?+w`!D$eQWn_jhbYVGiXo7^vW>-M2gRtU?WYBzKTz12f}@y z>*Y7DrjSv*Zzft-9iWPynum@42^LFiYr3o`UKIexk9|qn&5n%bYprR(xc6z9`1jP_ z;VVI^LpK(ndHltnouVDYH9UQt8?b`;{LvZend`uGNdZ!)@-bhl(H8CaKHj@s16}f6 zgbJXK$MeuAX9Mu!4bOHrpbBRcUitF+4$&11JTV+VUC>>{XI5h-Nai?ELmw@D4k%~z zfXyyR?N+M@6_j#V04v;hKR?Y*MgvFLBjae@e!S6^YTH+sk(l!~V#edQ1 z@kco??{_Sfce2G}t-jr@VO;x@Y3kt8hr_9gFpgQBYT9c+r17@qeY4(TPVz5aEXf_a zN#lz1ESe0@$YhEk>^So&_C;RdZlad)O7xum>lh}lctASI*63Ka1_S44aMQ;VaS zk;wQi#sS2z0shmSQ6RwQS_#O&s0U4=`Kfg`0cw@u`*?4Y#7KBZ+Vtr7!c%%=yKmR4 zuAzp(c$#650GJ2zyUWqxWYaI1&u;Nc(kBdp4+`|6k;0Wd_GXpZ9(wtYA)XeU?VSp51zXnAz;6< z+Z7E*JIdj0sc*07M$BE2-=bs!znFeQP1N0Cz2Lfi0oSh@myqbA|cZ>r`-j{fK=0H0(yDK!hyyVspu9u%vw$cIIuI zmq6-k52uZoxa^<9oW2>%{C=r4(HRD^zh)tn;N|)D z`sA{Ad}Znl>so4^N}&)_V;3FFFj3NIe5v+>3+FqHI+<8+riGKltj(c}0%9g^LIL45 z!rE%_6JWMdPvrc~RdMAtgAx?Ko0u@>JTpLHppe`@$}1Aye4cCToGe!8Yv*33}TibKZbj^x{g}VnaJVC$6Nm}PmEiNul+~bjW z*eK10l8g8?N+V9|#`2e!V>e^jPV#A(+586?t3fA&r$p}&E(s*Atlc>44pH>hLG*Ov zZ7rB7bh#`u4HJENc^7G#^&tB!(8$lA+3}x|7=h$TR?xQS3?ZF@YI{CZ0|I=c07dq0 zD(8acWErFt9t#-pJ(6whs#RoOXbGEvIlj>wM1fVmSKs*-&N4c0*UTsnN8-~2NX5_w zk&~297GuTTj_b}FX)1IuRKStC3)~6%1XG9kTm;20ciXVAi#DI2@U&qO5`d>KYF@(9@S-f-FI+?{RII~b-Y=c4OBsiUwq<p35{y19F1gTLt2SjH+o8+K{ES`8mBP$co=BhG9SinAYsV^^T6iAOblci5w( zxBC4gr(z6IxJFd3Z84l zH*pha)MbRREK_Kvn*B5V17ED@*SEu?5(GGjsD`KQ5}6cQA+`?J(_D`cn-W9gE@>NA z8Skn6ezvyqJNVlZ#4%y}!&n8}Y*|S<{C1Ris|eS6kRXq1^w~KnpBm<1d7vq;e6M+i zBY`-Xkj|%{DYiUI@pQ&HjKZe!8{zYJ6m z=7V;#8D~pJZo)9pWmLB`QcfffzB&`bI(CWA46{nd#)0SU=U^4V4JGcw-m!7N6DI@j zp3*5_?{=bkF(!fh^ml0I7y;khHYGB1n|>Fek#UB|8PFYa^Cg;Q)L3TFv@!A51?W1- zc|<>U6w&8Q7x0oLE;>UqOs^m5X%e|Q5Gw`?9G}bKvDpKP+x6T6-R$S4Q{_&*UpWDz zML2|hQe%peJSVegmY13OdL#BK)3F5;e08+@`p!K}jXw@K$xcZ!lRTyF)QO{huQiO4ZI~Wbx7H;F^ zmRBtnJa@~i!XCh$EwiS(-2^>eS`+RqgZ_P|iSUvneSh=@52-%O-8!JS^c|J}=sfmm z)((w|z};%g*jLGd<9J#WD_FGt<1dQXKm77y#o92rmS^PBx!%Yx1sJ3ZtYfh|bZGQj z1b-SOCgBo>AdTm?D^^C>@OFg^z~A7Qqdg|1GOm`k@Zg$*`Lv zr}nO7SI{4`ghO!$uLT620UZ8y2qv4(p6DE;ORfy82{Lm#8gHI_m2O{YO(fvZEx-m> zA1#05T$RM(DMy(P5WW`&Khmx~j~WV1dw}2aw%oF zBWxq?Fd_)nBFRy!k-LhYT+ z>EGOeNyYdK)}4L-BA4q|zX>QM4tMcAd(k0;(_UNTjpK9~`pr4zy7T#QOE|mxY4e|@ z%A7*3xD;j8^L9= zoNmL2o}V*!okVd?c@)JAz95%W+k;Q28bjviQqI}Kg##XDx0MjeQAzblx9ur;*R+?w^%bZy^j02b5@e)AFnm?`KG9ga<2+um4kloCV1I%uO7e@ z%&z$=dCkyu(n96WDn|``)vZLlEm>MzS~5VjlznG^@~T%qgKz~?+wfzR$ei1@wTbD% zS{e7-{EYE03Ey222!+;vFSN;gcUoFp6a(;=tA#$Zbt<>$%#{NTDfI|(ghhht0ek01 z>aG!-h!fXLimlAu7;OtGw~8lpMe5K#O`Fma zzA4z#4J#PzoZ2GFC@z>XV77|ZB^epYPi=TCd(5B0LYYMmEnSWMV{_fJFd1G5`weL? zjO_%mul-M@O<;thBp*Y-9mBh-*`cXALVLhzV;6YaxTNF^mGsGfD*JoT})NOX%+-h+?UA6@lI7&W@Z-sem7}Yae*GZH|yquTUg%Er~9y z{?^rw;Y@YV<-3`%$WIK9V?x$D+zX3Mi7enmJs@kjFx)xx1XG2|-SIRK74)`#(MTRZ zaN4sNI3pAQ?|aJ%bPCL8R(#(?y^C!GG?Q$pejA_DNhuvW^f;2)!&2g_T>F?Emn+B5 z(M(86ReV`XDD?YYy(%YS<4A0p5G8%3JL&e`ZNV{QX)>{v6f+Qe4|c?Ome}V^xkAd6 zI`L=#sDEYuSlV>NVsd*^G5##67wEeU;aR5GWRAK^vVQ^py}gv^XGO^S9^tia-I*P}T%~ z(!~(RuSoOwjOtIGi9%n$kEw^b$Ay^41N+FlXqDD0qu%Eftg;qt9t`Ze9!Ys)gORt@WYP5r;%aG&{OEEV|no3$eS#G0F zM)yK~cxk1JO%S(kAVd_Vo1xxjekE>q^pfnWVv~b-M2v!nAN+GB*MV|}UgYwua<4^P z`hn6mg>84Uk}%%OgB$m;jA426zR3ukHpO*+g% zJ+H(qsuX^s}4vcK)>&~-6?_dpR`v_LPT*W}Pvt3d0&6QRms$|k1 zTompksRJhtx zB&b2{Zkso|o#!}9y}bA4pCWRHACkX~Y)fhuCJ>8`8P%y9ax%l-Ilus0eY=-y&}4@bRU_%J*sJVMr!>G23u0#-uG^ z#8?h$3Q7)D{vs2Dhd>#Hw(#&67cL;(L$J`9tz&O4W4vg{vd{kvy2O;uH|W*bEv>w} zApE1IXT9EKzL6(iW~?$^6bVOyi0@I=70d%y`{Vpj4mH8Nioq{){o(urZ*wpLH{-#; z=RARg)2XaYc#rzF*00-k>Y{MsCCR;515gFndv4ixjr<%WyyKsGZlUzx=+~9m!>5u* zo7pdVi|D1;pDp(`#ECGg<9O#T^q~q4}Y17%StImpo-iOmWJ<>8fj%~78fh* zDxAU7Y0hsT>TEexS>e)bdarev40_ZrY`jGXu65eOz`^#qKTe!GlQ?MjqV6DualHxm z1F2+eiy71h+^+Y!R@ypw;^v|_@*71i%ctWJU9~yxQUw)X3*#)d&)xGIf)D|;)3XieRlkUZ5L|g67(^ia3j4jH!6}Rn8qsFlp zsl()v-(=0wN)0*~s^{BWoJq`i=Lg#~4^+7Zu51UY)IU1$5=o58*4-#L^{;%DzJHs4 ztL^tsUxG^sP3j=o2D{~^l2Wf3Zm<&(4>AXs0K0DSoXJ2ALcr}+%Z>=iLCUHfqR}GRo#cz* zdGgBtC@Bag7r5*Fz|VM+%B%6~b{RT(7r{8&J)2J;mS6f8y!e8u&L+07`(lu_zN?Km znOqzWy}awKcN2;|;*3t-dS~X()Xgp^^+vI+P2J1Z_k()5strp-L`6~p4VUTTpDHEv zjlySePF!O{qJmIKl_sTrW>vF65`0!%$;K1_tqo<25?Xn_Y~x+?dVP2>y%SS*@%k&( z@ce`clkIC$qi20(OL{-X)BDI_QImwx8Ref}iXS4K!~mn=kpBPh?0dft_ECgsKI71z z`mKS>JvQ-=rt_ zlbWv3V4CATD6rW4Jp5pv-rXfnyMJ8N5m=2I9}@HaiWsOQu3{{35pJw~B@^nuEMfmh zUghHY0^NgRV*KXwSp9Mj%NQb0pIH7fLD1m1<+*8|59Ihn2Lgu+$L=ko<}19GNr{*& z6#+8yhDnU$Qg6cs)%7qb+@t<2TQTK(F_$D3{t}5&>Tt2uhY8LJCMHESBU!VD;1vye z-ie_|wpbLLx_K-Cg$?CS2nz+_5y-T=q(m_zeb18dW;NUWx691+ozeDwdzg>|Z_8%Haj(lsVGBy`&BN-B(xY7#||zfI`6zs~j}(kIx(mMz@uR;h6I zF9leG-KWda<0lTi-har|xD9vmI62 zyRH{2H&=_(bU(KUuGa&|l9KC7mron`|A<&gUkDDDDR~0Rzg}B8PqVtH6pP;>r20v9 z-Aoi+4^@3^fwLp4-(0EZ9id~|!MMZXv{_uZvgS>bOpw2}+z!16hj1`ec6H z8e7q2AxR;c&J@DQO@SG?Epq_eZ54cO_0s6Mqo_NLL=y^Jndu#@pGX9!6pGE!a;JbK z)3x5z3)mZO)=?Ul)%Sg1moUfoF&Y+e8D6|*TYx7`?WJe z!1%Bi`u3z>83#f!c-JW5O?CIj9VYYGo31Y~-2m07G>SM}tUH-TEO}7BOI_=V%yK<$ z4X<#es?$Wvg&+=HC62GZ5bHQ3&`S15r;)f1*#MJFs7$-|4#Hx}((w6j_ZgU6bNVax zvfB3I-#HSxH?YR{L6P)>aPt$DqoYuBzV&lL5q9|OZs}!zan{+&ShXxZOD+t`R+_zd z*JjuKop4sD!gbUknwy++$=&$i44C9!iRSUqHiGrFF35x+u5uuhWQLy-AgP8p+ZN9QwOAjq>k*Ux!aRgdROQy)JsVT*W9&(zOR^Ld+kq~$CuVNAXQEep}6X>$(LKd3LUa{;J%VH48XLsmgw-rC9 z(E!{cU;E+eUs=2?7?5Qw*|-9MNnan0q*k}Y@+k|FZhr<<8plK%@#guyo|EZ%AdFfM zGp9wa%A(HygKo^r4$;?h8{M`Fr5<{TgdQLBBmXzd@J~GYz!29G<{zE32IGNx2@`Jf zd5^(fdcTE`-G)Y}^0%7@CAf{N zsvIr4)(PyvTgez>DAcDoiyIX4SqT)DD`FP!d3jGY9ylmTZSnP*+|%cmKA(ZiFy)Qu zczl&;tD+z2-H}cwUNz=pOmA*=ygyO=*Q+nx!APfQEQRN1z3T#f_YixT5x(;H+cbC! z>_4o3wa7p*LAai~-n`m;p-947s!sTfDf(xhX+(95Di=CmI>rW-=&Ax7DWz5ohuZt{ zZ;&&xR-adw1eAaqq*D*cg@dAb1bkU3e%ip)-U1 zL9B3T?F&J?iznYA{Bmk1X?;Eoht#SMVGgEIZ(?t7W%(f&wpAt`+>$8}59N^@3X6Kobt`*W(;U`Yx3aFpj&5|#$!S!}=P2L(sG~C-gY^ldR)bU0~us{%#)}^W|_V4O6OaC~2rf!=~SyuiN#K zZ&tH5=~!E9t8|yT_&5A_5S(vejNIz0_e#U>@9Q4^YGO zxPJhI{Un|vocZ$fU_ZRB6#1|E;Jf&fwX%0!`|kz#kH-!?VYvR9l=vmxY}aNoHBjzH z`{O{Fv;0wV`J+z8=|_?`%|ic6cJib`8gPY+nU_Oil{eT^HETSWIMiMg_=2u*f7-W$WWadq#HB^||NK4x zZbN)Nz)ID8#x~bCt|Kt#S(g5W_0MQV7*!}Vxt0U0j!COTTI751uWX5};diB)$`=|UH+~P-lGu#e>$l3q zYIK9ZoweSqF9i}}pOR;9TcX#@AF}0VH;G9refrz=kJZ~WpAV*4A6DPl+-@4oU5p~r z#n4!EWzzaj7n@fVnyJcQCcNW~7NeNR$u#Bt=SB=rK3S+Kzm(zqpHU%%20DBsxu6;U z0zNN^%|fB^`cK*hFmp}ny%C%#)1e{a`Eb+^bQttNgAt|WYLARa%%|ylyU$sqQIT0= zzvNqA49)^=yB|b%k^=Ttq&EtV-B?c0g2!gYZE3}6a_K8hqooP|4ifr*$z6^WYm%C< zkRv_+%wAh>^c=M3albpL^k)52HSN1p?V_{(tDa^c`76%tq04K`IL`7?n#dC3q(AAF z28)gPzLQ6Cm=>D5k|7ejKQ8{c=9LCO6=&PtWgq#!N6+01s*vyn0Jz@{+s%|`BrqH3 z1G)8|!D2v-9YB*LBma zgn&EY1Pm<_J3>4hafuicq15@=rxY6gFm3{^TrAKy{Qjb`IHwPOlZKGLM9>py+dE7f>fxfSx6A9Qx|Fq0%lis%^Kg0-Uz zD&GyLPnZ5Os}cUVIWN?UsI9 z2MW53Kh3-RDL&qM z$j}H%XLKqoiUGu8f`pVmVHTEjjW^`Oee3rBk+#Jn1oG((!4p<*BTCgMknars3CqwTf3VPpwvur?)dC)69g9UbMWlJr319U2LRO6B6pReizWbe`}m@ytDN(5VPlgfURgH+R2qhZ$~fCXoF z_%=PVsbF$EM4LH=j_16{k-XJl6YFiYgW#uieEH7Em2j^(Q9v__HjVqSr0afm{*b2* zD;y`m(Ea=3wDptPSZG>;l!{8+u4$Xvc*FX{h*xAIMg>o1Qs+)dM+o zna_4Xj}irGN*uP37HEo2RziwOG{JMz*meNe^g}>^Xa<_C`ui84K?&~*olAV%c7LJc zvFs0<3nc6M_tPyN&K@A5czmkD!Kmw9Rd2ZJsfBQ9c2oFex&)Dn>zb33#Mkk-ZhjkN zUan{c{S5rRRC>ib3-Jb@TVf*rX2lZf=nOZApu723R+W`}k_|J-h}I|N35RnYp?YJb zw~MvOT+KNMS@Da{16*g;Z<4%&hiX;@9JEv@zDXK9jg<*vf zw%|Vd`5R+s)o^WHlI_zp@b8lU%Few+%NN;#HzFF7i^pmQ2|xmTRNATTw{S)v8brf% zxx@cBjEaxLw-euL+{Izi2}uPxl5!?en=~}TAX1B*NS}z}(brvKavW8b@`32uL6*Bo zCbrOw)p`@dU^Kz&R!)9`aOB+0jXe>lYHsdB!C2{TBHry54MX%^SO?|F7L^DD08)U^Odt#19pyT50Wp}5Ru?$}*6Veem=6JQQ0tl| z8*-qfM~R0aA5WuraqGUQ4^GrKZId-PM}GI5_o9;^mnc7+biwcO9^Sh3Jhyuzn>kt{1W2v5ELkozG-Ad}_OkFqL=aQ3dO~+63ZgYQ=;&(OZ&ET{2RDUZ z=tl5Sz>HSzl&*$G~b411XdH}fqmjL7u=w6*zHg4aCo{S+$O0$|>oPBRN~U?cld8V zRaj!HWq~v>p45(>&)R@<^noo>MNgWMn*A-h(3ElEugkn_CyGP-d|OIYKY;OWQ0iV~ zif}GM+3-H!^l)-&jXNnzYZ>%&AmbcM8!iIN9@WG|c+!u>a*S?L8<{jS!qs+bu4L=q z0QpY)yC27|ZP*_=b;9OcR1dnB*Tb#-QeS0?O1j&&MMU6t3At=N#wqcyM>NHrg=2Tk z{XlaDZ#`1I(NeOfMf1kL?A`wNLEw(vB3v_!Ga)0X_(YiXyX4WfMH6t2l*A0E3zs8e zDpKZ%C%TS_<9M8-j_B6DpMq2d=7@(oY0FW7q|Ju!uITt3ha029D*QfBXLK)%>F9sc^$^mAiU5<}(3_RHzgEU!ZugXD^8k`Y){cqQ_^`S&r z_Tz&mp@)S&kT>j~22GafXq3pLEOY9&zn?1J*qxgf5aU_n=*ZAA7ZGl_(1ei={uP1x z6J)pzKGoUY7zuK#u{e;U3pR}ESmHc1UzQj8bk zYO;FAYxUxFi^agr!+lryf|@WabN5ji)x4OrJiS(<5^wf9l)0U_Lw!GQDvzU5{5CBS zMLWk=#l9F1aqXOs6!OT-!$)D4;8E zIoURRHgi=ja&Pi-fqpON4+&V(w>vG-sCujDBjNe%wS0CGo^X0fivr>#<&!Os+I2d~ zy@=HMU*`ptXFF<&?-S;HF5LU#CQ0lWbh5&AUmr{@oS*V17nrZ+xU{7WCcqmgRxb$> zKNQ^Yc{E0NHY8H8sb;v0P#FxtDg<3zbi6#7`OK)&7Zgyg-L8IcWf^wI?pRdOc8=D& zks5A|V3M-c^pAEi;!0JaSVtt&*`y1|9*O!O1(-uw`0wsEwt7z>1W;WO+Ko*lPSPp6 z8glB+0nRv_)AxD?RT$~-SG8s?zHHNPv}L?*yHI?fp5dYDeJMXQG_f=iM=?zDTAV1m zZ#HNBcJ{X@1eMcKQ_#c67Kb%Wo0IK)rbZg=W}p^}IYBY=P-#ABBH_YzE)6`RrYO+v z)H!i|n@>cBtRoZF65U3pOtCes;!)vDA5<$o3N2jH&nqRhcE5k!YRbhJ7=lXFy_*k0oKq9?7!AFr-9p2d;s|NxkGVYuQs6{ipL>bF(@1-RZl9N+fyN2kK+pp8_J(5dfas@>2ReC>KK?S$y-rC zkHAoZ%zUsy9|&@dOZ)Qq zljE{e`r3?aZ;O6^!p+GCwJXeORZry6VXJ{jUwA&vm(JB|KE%+^p{_IAeyQyZ#tHh! zfaV)0AE1hs1w2-i=l4L!_;R%h$j zTKEQe-I7cVLh=j%XM};SLS+z2boQvIAoDdi8)tlSH3X3>kyVu>)CmBZnJYOWRjmT* z$L~ufs7WzlfUBY@7F+1Dx4#K;Kg~YN))06Le>OV(eR2RHS9mbV;PX+owc12KLmy}9 z;?DU;u1Lx`d~+r)DFg7Iac0)_GRY|854fI36Gcv^@apTKr_nKE&hMlr%Au%m?N;cW z>@P1<|4hjNe5zKa>pCJCQ^6%}QOJb1!0m4akDKN>C)wNAC$+;CDyNTo8SpLY6SpBt ze>{lUEtS~O26fqS(O147ipNtw-j~kv zT<~W&kG8rT?l0c7WsP!sXT`U*{7nwDzcq^M}^HH<-(JGoHrw?^4%UNi=VFV{_Q znot1?jBu8Glbrkad982HQM``2w5Rz|Sp_qihl0{cRj7FEp&RPkwBENa8W>)tyFq@> z=SYygXRPh??+A@d{Wiqp`DcE)W%yUj#ksBZ5l6e@sB|4UT(gT0tO(AFUlhrc94Fsf z1btHVF7MTj$ef&z2AD%!kez;f@Vb?3Ue66<>O5WQd{`q+S%wG> zq46~OuCI78p&AWi4_j*>f3`~#C4M;>ANmo_)b_=bmxq9KRP=kg2N-3|I4bU6jP};t zei4jXuD@R6elUTxy}wDZTL}F~(+=JofGOY6rnl2la%)KfKbtm^sOD zuF9wl-oKCBECooM`N-d+CxeIV%yw@wUwQ<)@0QgYb7*n7bDx_Y@k$VvFTIEjT7q*j zHg0hzP2S=hK7}S$y@{u$?!a`#9ym29PP$#XTD-4|by^-n`7>W5zFTIy?5JO_G9KQR zrvKRtA#f8a6U`2uuib;JcP~RSfqqZ_g8yXE+wCq+uzq%*ndS!{fh?k|5m#`(k#@9n z5q}~$-A_UiXfZU}60tw1_aDF>JvKIp*+8KBKx05LSQA}AK^D&l_R~A}FOfvN9C4~p z{s^4|1pgmD{dx1@9k3?Lvfbc52Nh)H)1-7QMT z&>}s6bbaUciT8QF?|uJ(-&()5X1PX%iTgfhpS`br?dy7Z^~ozW3ngu1pfWMZpLzX> zDq@3eO7<5YLg}i)GvGtFgh+wgVH268GeIG_&V$I6*WGZjXZXfYfQ5H9-E^>5Xr|{! zaY^66gnnpwc%1@g!lh4iFxxT%j2Gva@rKJNqs7*A1Zk$9@efPlxu|n!S}@S(sHIw0+r0*>MT)G6QjyM3 z(htM9?>oe?syH**3Odk{?1FIXLb8J#2eK{fvnm~x&YbTy3;=76Bmc4mEUe5jUxYO_ zB_ODCZ(VE@w#nkqW%DRrkc3lG!ez13 zvL&tv8HamZWgWKCL{IN<(2@F@e{ZjDH7rQARJK&H+FA}TTy=Rjz#ER8UlWf1i55pM z&dcBIoX;4-Jur#g!gcjM!dr1{HR)x3>4tqMvf5j?>uYo}gV;~(=SZ!~dmMuva_E+H zH`KbDW2S~@cy;Xp24*5A#J`q9)LdPAUngJUIoQp~tCuC?J9S#dTea)znIm!gM&A4F z=}u*@RJpRcS?y!9B$_8_5>m21ag)ORIAqQ-;^(i{dLIdIS+MH{Ycw3j)aug&WLwyZcmM6F0+WfBh~cs#%8wD_wCG3wGcSgs&L9k;7U(RFk+{SKH+sz zEg`;1Ga!ywakNVgyfMr@3EP^aeizhOrSE1uy1;u0Y$N=qHE&v^sM}0n+K5*H^z+BO zA@QiCO@{R2x#A|k9$9{NfAM8j{)V_3yHD1TMjZrJz%?rOS z1;k`0$8B(>P@E3I35aLq)`OiJ&PsXb)BT^~cl{hGeAo4w!u}==hAMrcKCBS^IS)iA zmO}B$aUgUz*IbAlcTMywWaY+Jn8jr4<=)#T{4T+h#=(H zbzna`Rod63 zeq5{*z1>1hl7Tik6NAMOhFNVV+_+&*C${A7k#R!N@nyzi+YyiZKey3 z$x@obDOM(C;g;O1@Ek3CX{tt*xzUdMll^Jbl#KjSr$v~Bk4tAuOh?Z#)H_&webVz0p33d@{-WlF z(n?EcA+c2WLBX`{SFvnHa{8CROYi?p6PjO32S+~+?NYr>XTsPQj!BC^mASeF#{6&gB2c&n9Z`|kl zZKCwG(}Pa04+pZ8E=ZcyD9o-2)wTj^S}N*NET9i^$_i7+~gnBAvD zh-%Z936Jl13b-!aD7#qaXC69Fur6+tyhY~3b*~Jm@(}jadihHv3^5#f#!Tdk}<=h}0OUqh*+W4sin zX-hF&4sdjccu@&0*)b`PwdPik_nDlZ!-b7|^gkB&>?TE1uK7y8BF1HQa8G+A3d_DZ zFTmUd4&w%oJW@L3u}Mp6+sLH4N!G=!Ryy~rm`QeJcOx*@nJ|zLWOZ6F#MsqnlP2!di;W7+$p$TbmP9;kKf-9ZyYCYOT^z?qA&~_QkpXSpp;Ki-GT# z*&eCcDlVZ-Y1Mp#Eh!i2uibaB)gAPB%Sq%sb(Nj+jAo+2sdj(vLKiiXFEFBz;hTrj z(}~*1cqRs%(V%mg8YPZb=|s8t-Wgd0cAW@UN*=7{&iKX0%7j_o8dS;Wd&r1Jcqn{H z?lSeraXUrFi(5Xdu>3+S9L!Fd?xC0vLM8hKiS?>o57>TErZ&P#j=$f5`$Y4bDk=Ot zX?rH_?lC)a81dt#PD-3$66GKh_n4sjaKPX6B`>tK(Y((Le<6-0f7jc*^TitiF&fIQU&s#5S%|ciXnQX61=%vF2b~ zD=jakP@!~G@jtxGm{_};5S+n#@Sq**e3x(gFD-z5*!^GRM9~R04OMn>s(Fpve|wKA1Iv4<&z|9 zk|bdapgeqNy}W97wuslUO?r`Z+w)S$ag%H>-|GIC7Jt;So&ko26}R4>kl8kts+m~a zY~!pxixxq&32$DBKIX(+iqJFDxIpZ2VJ0q>Z(fdFi$me3Y3#YJgXzx+v9#BMOGZj0 z;4B>1#;+m`lrKvvtgNM%)UORBK--yy1ve0nd>E^cYd2CcC@g*y({n1cB*0kOMjO6E zOtO{a!ClfVc4i_#YeW8hAMx?6tP`-Klxa41Q$=Io)(S7t z0*p1d`# zik<6pP0Ok#t&xi2el;SoCAPfzXu`^2ir~)c$n=^Omdjb1XDCxSCs9ccWX3mrcYVF;(p>ScV>&+ z6Lmk~jX5o4!U0i;37#TR*m$rus!B1A?VlR!diU{SddHc^W~-Q_*B+kVmjQ6)ia=9h}Og*>;I;>dM+m_=*mnCOJG}Z8-Z2uBjgDW0n?8rh9=m<%WZ_irb9QS z(?QKIL>e@|$ z4E$+g_lr>hUgzfwtqh&-e;~e@>K_ST62MLx^$+`x-b*TZF)CaWgQhSE)sZ_eo|j(v z`qp1B=={s8vsZ?NgdIXRn1y{WRB@BHg1=9{UnB&`+;@Av_a0S9SLpMj3f8VC}uJY&*44;!-2U=v0C1EiWPDBemG+f(7D3Rv{j5 z!leBab|#^d3(Si;Dd(rEcAIv{bLYHe_!f^|?j8#Y{g`J~9GPE_&<{4XUPh*e9ENF} zK4A!D#4vJ+9yYv`zZ$zaHO!|V%@Q?2F?qd`N|%zi=%C`2&K2hr8RGE5 zB9Xyfx*J076cd=7WKI&qI2W;`HZS6P;@!qYU1~6)=+@)%-GIyjRjs5ZHXwYl`1z9{ zJjQYsbBl);TgQ1}O+?g8)S2!R!q`}=bRLFY*h!W9PLbJq=0k_WZMi3^&9I% zoT^3fL{TMY)B@jIl}|c`PHHSMMh#@=p;HQI@GwR*W3L?awU{{Qe&xlxavo7~joZ)` z=zo=gW^&e1B>*w_kQOJ(X2IeBNOtBZe!&zwb3Z7793yXHDt^Kk>*2s+zju{gvtokD zgrp4X6f(5us=ve=C~X?fCHDy>BBX@~`zcIS+H(+lm#XhxhLi(1UR2QpVP-I6efH)` z=~i#QHj^BicJKSX{T)f1<;Wwk|b|G80N?{+coHGFhjo6ThvF4U_IIGoUwik#ui_m6`sT$+0QN3evJru^sA_pWu_WWPQyw;iX_ z{Ds_s-eZy<*EFj4sDj7$ zp}Um{la~&|$yE7$TmHB?(EagGIoClAyCkD5r9U?p;oAABDadZQFcMZSL@QJBrgGYx zdTPlMM1-jKy5N=y?!&USeLL{zCDIC^Px_xny`RhrkF2;Kv8qQNYnN$Wm0#=}6K}Az zd6QQ-QR^l0>c?G^Cgj@=XxT^r3%&CmndZ4GZ~OoA0)M=i1nQhn*!S;{R<(cq3VxEbS{3gqKOKzwpc&;vlbEct*@~Z2}z@OM8xgU zyie<(-!+p1F|BA$<9i}hBmTDs-|H97ndZ0cUESW0BY*YF*#YTmuT+cSJcG4VmtH6H zfe>@hh>I-MJNmD`UUUu|$vV?aX61}IdgZ-<^8zHVt}WoE){4ePpJ z^!eYladgpDW88mzdeAHxPVX#Hy92{}hO+I34XS7V8z=e{W*i6Teet zy`9k!c{9B(SmWkn*3sJV3GE7#kk_4B-HP+v$uX0w{kuSB+Wg|>D+R03)wYYhX=Q+a z04e&FQEt!`luAKg{h@-yOzLOc_ts~mlsv{SY5>s(xTb#8LF9iqv&MV$0`K5@`tvP? z@|Ci2`{Kp`f}pVPC1PH?wqElwa^ZN5kUS|Mek6aGyHmxdTu_ej>;gW_6y?ge0S!Nj&(-DZ zK#o571i)Wzrw5CAfNZi`mh_l*7^vrY*^U*6E^BfDA zybC_M;Q0vpMIiR|;mcW$<}Hplj3v3+g-qE*>T{%j?Wy(;xY*FT^{SPLpNri|uCdh%dhjET55{*Yjs60)PG3J z)eQMU_08?syYC-Vih(fj5Rm33Z>na07eO%qdkd6IRI;*9u0AEBdZGFIG82$T2ge6+ zt7_6-&l@|gpy1H{O0)@B2CnPlfS6KRbGV$n{`QM<65r4}Ig-Vf7!ZHkKR^5VwL!Pk z+I%35$K)lX9#CdG%C}fLSmkO{Js+#2KMDM=tAm0-A@J3rJJAI3;MGm_MxaqP*9mOu zZGl&JiA7KHsK^G|6yo@t99YI?i<=MvzwRhnV1b>-MdA@9ep+z?+6}J~x$Eo;j6CnWvI6HUsDqo$R(tqNay!;D_1ViZ~Am(H`#HRRyA}+&wHwm=! zI^MTWU%Y{l%j%pS1>FzBT_6xtITv=7-C_->Ec6zb~#*7@(9T9A# z9`$#axF3*?ATM#~mF0NFd(DQhO%&^ZsqVE10gU4s+$VX=GQBBOGdNXg(6~m+KV_{+ zCeC&tzI1mkqfWQj-Juje0pSotkv+d$Q|*X$O`hnxX0Yc+l(zSlC}S_i;F$#v&T5&WFcyy(h z8n0>{*~$!LYd#&t@qpl?C0pmR z*c+jd?2VM^~;Z%n(RPkz6q#^_A1GDo%+5aNZR=FcpFDS4`wBg zcMrdBx4$2lAQ9Mg{@f0vws-8CLY+4~CgOFTm!Eqt6vH_|0H%UIR$F=mvY8Y3Ucm01 zlkF8pGMwAmss(3>YK6w_D?xHb;0`1yGjt5H#))XS!(O^w)4L^k!^~H!^24yhTYledZd6_VgDlBEX~X4vHHQe|mc4ut?idOqNN> zjn^^0(=SI~xyufi30pRsXc^kW`307ATa2pMf!VMF;F#F~(Z|m|KNz`n@@RzY7VD~Q zo5ZQ@faDWPJ-ES5*G~(}qwGr$)`pqQ%XtB&aV}^i=W!gjoXE_>)EP=THbHB{ zvuXH-5uM%aC=&cY_2fjHbTM*n1DEL`L<5l)Md_F6#M(rakvj9XHfK>@z$1RBdcm? zHk)*fpY>^DO2#uPWoe1^k^)9sc&EY$UE`%bU4#;(|2ShE4&cHgh>B|CG5+hI_YL z4cp?vM`xsO^f2u~* zq;5EoIB50e<0uz&49XV^6R97h-m^Vp?v5%KsOxi(dAFktCuAAcDx@((Lcj@)|~8)bIrdH_<|tBI(DWE{uFS?EBs2~!GJEH zP?Fdd`i7OCW7PadrQoIX0-Z<6eFhtD?iD+-^#Xb3WYt;YBYYG8MIKxdX+Cb0deAF! zM8b3nO6xkK-*c;kuk}`Q?{5X})%m73vJ)|1ucuBwDMV2jrGG1On9F)hTdP`{d37=; zt&9I;(PXhpdwpNPH zd*$m#zF<_1i|V$d@h|Fo!^i=OS;`sGUu{wf*5Lx}zxr_TKZ~;7Tl=Fu0abDJeku$` zwa;ZMSo+8)0rU-*q)gjGFki_BJ9ypvErs;Wjp6(ZatWkP2Y;W2yE^m z%hD}4zZLo1K}bRXkTlU}={(2CVtcgJ-4vc0r9y2{cUIGQc|mo{R0{UDJpo?P;l^rmLtbiKHyPXiunYFdKKENBVBMHO3|epWzABl2*mK^0mN^ zOeV=Ft-uIx*cX`PsZ5!##okUMIVHARiOcwu7(0jES3~MWX-hPQta*Ve&1k51Xo^$R8&hk8;#3|1~q@H2X;NgM<^swJ|r_D8dy7BWep) zB)T_ICeEh4%ZdpBjfdlEGu=+K4MV$wBlR=E9E)^Zs&#kx0Z?SEHtl#s!{5vkPGf_( zu-)BP(@)mST*mHGb7z@V!*q%W_KcpOL1PPRndb-10R0#mZ12L)(0fy{JJynrd% zZ|8UUk_mE8ceczFr*_>?dV>hWK2P9Gb9@$(-_lvz1Tt~lJ-L;^16FF zT$20fyOb0WA?+yv?$?D(Pk3Q$Fe`My>cHnXuXX2$jRmdVX%Ei$*X-JzYd&WQA({vy z(iOLdBGgCYkF$3aZK~J52dxd6az)U@vumx2dpNHXSO4m2KVPlod=Z_=sXnQuM^s|k zHB>V6PI%md{#nTn4QlPr5)Zl(*efXvpH=sdfA+pcOrJ1>m2Ku`{V^-dY{+|ubq}X+ zfYKx-I7|6Ty)YBS0?dk%-21>Bi3W4T&P+6YM@TNFP$%v!n#cs*62>3>0)4Z}l=VpD z%zDir+&O9gMFM8R3-UVYYM|K-4Yz+Z{(B>+<2vIyT#hxoYWA4KS_{{WeEMy;qiXZ5 zvv=*(s)#1o%&((wmkAjo`4wN?oT1yFrHy*gHmmBL7QGf%tL-qp-a zx`(@Nfee^$x7c*|L#+({f@K((@uLN@V&Ic)j((E<>!$rbh#C#}fhyAf3YGmvE6#)e$3OR3`+vRwOXd0ZzrOSDf0UPCaDVB1 z^$)1_Z~T&`9@lMs;%VLKVK#87s}BX>0%$js0SM3{{qzC*?_H{j>wHYDEnNEE;_q+#$CDyb zzoa{sV}Rl10Ki2B{$(KkIg(qBu@(ZASDARRK$R+$88~puqBTcy3;XNU78aWC6eddw}VVIi#a831$WHH`oRBfiEGy zas4?^{Trfxlo~;6gfHy{qMwhpQuxY0Z;RiF+BA6OA|Th)N0bG0-8IYY4fKHMyh;jx zk8C8(5{ejAl4A^kZRpf;+wa#`?$}jKihRx(WH9x=|RFZ|JKbz z>#5qBt6cgchkF|iKvJp%5D|OrD;KU&-?0e43Whc(TX{?xaj8Yza>&cR<@SFHlm(>q z-giKN+zntuWA0PlmflCZu9!$*=V|cR&hB?jvdi!$)oOH3`01;E?c->BT$pUb9KOg} z$#+`-eGR==qJWG&*CIvM9gjAdz^q<^o)MTXg8&(Tx!n<&3%`?E;GaNN<}@c;y;{_@ z3XqKwkfM)*3%A(J&Bk2KuJ^VZy&D$)b@ zuG>wD&G@>lAGwbeyPSz^$Tz980~ga`$ej;A1%N?05KI`!{5ArjvpilKNdf3!Q6#NM zmCXQfdnshQNChNJG_2esBB1bYEg2;=d&(4UuY2+;qzzXcj4@tOu! z`~Hk0X`BVKSq8jxuKDP#P$DY6=1DJL6UdL?0E+Z0%g$mGrcjpS(EFn=gZ|bchW4KI z<1FPrwi}sW>aD9^{!!jTn{B$Dxd03)1S9YNv>1WClR9@_qJjEyAr2$16V83SrfgUY zPX8s6+tOta-UX>}Cs>AIMG~hu#>bgda>mfTK9dClcKO6ZY}_g4q91CF4OA<7?S&4Xq!gG{iHdnw zup3<&@Um{T;dk$m^ja>7bhB>p-+gW;ZH0b=RXq)|rGX8}2%3NtG}kZ}(p+ zAIlbxDB}(LIMS)ov0YW!npq1xLTy0xt$dAN2C;(*XQw-sX-V@?5OGpg2Iv?&-IR)QnY;}T=W_DA3 zXvnPk43Y%biZ~+fQ)u-ZP?xJbN{&|VeZ76Z0vJt3Lqev5i|IAMv6j>K^w@2qW=|Ik zqe3)devhqr$8uc?5Lx^Zaj*1Jk%zfK8~5)=y`R##nFhUp%U6%T=*8ki9BW_~R& zoEhc|RE3yuG3p=yxW0_Y*ZZJM#g?T`5rYdcCdkcT8<4;0>iYyk=Yu@>BXC6)W811j zZ#|zGea$YXI)1RhHID6*9Vdl7RpVd&$x107AtCd9PLHJQ&^&I-Q)S<@N=5HLDy!|d za%)t}E>QPW-(xKl0Kn~UWclq{NZdT%lQX#v~?<&1YUpFj6 zZnn5C9`1itXC=pGe#}>jJXD$vN)y;l?h4=f0%2(_eat(+waDyNyH z@%TzfZ%Rx}PVJaKE#v{AH<G^P>O0{D3V3$4w0QJ`QHH2{*gOfh*&rZJI4J z3b}cngi&0|6aTh@~W5Tcw+ zN`oVTd4!~Nf#S29aik2_1K7@40Vr8odQO^YzJPLDJZ}O3jWwIjSmkf&~iYw9~=#Ce18ZXn@>O@bS*4&7rMHCKkCmb zGpyeD1-dW7h20tJ&_p{~eX)w97yac#Ks`6QG5Z-6q&@<_Q&7O*=T2_oiPZIo!F|ic zIaIzm+3`h-UAMnytkU-Tvf0r|You>nqw0y9^{l6IJT6Pz{i7THYLSC!b_IBuftjv0`${3ts~Ah|#k6bvGAj~f`67mNwSCxpA* zY&!~u*Mk11NFU>rtaW^M`@u+w$?nk%uu#e6?FT%aS#poIZ%-Vh8JO;VTYRuR9O71SX{O)-0;hu<>^Pp*O=N_V zH*B=!xhj8oOe=Tn5%U`Zm!5k=pox$uM~(@TS0XM4_O{f2VjRKVFiYU1@=OLeZu?Xy z7GX_${SjU>+tY^IfiJpIR6b9LcH_?a7L~VmN}#05PaEXaEt#4BXl#ID!|??=|(3N|kLzKkwBIKOBV$_2laRql~Bsio{jD@0igviPu<00l>dR7!S>O>;PP1wo$Lm<^OSeOlE5Q;>ptvF6gMCmPs$pc&<~l$;!-yYY`qV# zv<-LFM;i4`(y7mQ=OjQlf+xTc+)Q>@ht{&(hPG2Ea);P+{UbvnbqN?KkuQXN_iluzN=E ziX`DbSiv6^-t-#=2{8_YGn@tuBU@?ico!hESQe@=yfF@3qP9bsP~U^R4vT*2+fUSv z4&mK#r_&Ymz~uf~Wqx1gcJWRSamrrIz0c25HEun`;45YGpFO>5XlsAwzW&ZlC&JFFE7EQD=GcucV++2 z0wDzdG1a$5jsJ=pR0P!IH<;m~zNUlLfg3QpuYT?V^&4qH#>+J|f+X8v<>hx&cPuI#m7N1YEmR+1N|vbu%(s-6#Iv7m0nw-G2P)`ctfmbHU70d1 zamUf(HG7s*u*B{S22#!=Ly-vm|55BS#KD?-i;@)5vZ?=PJ>O0YL9jvDS-k(s%>F*p zz{l1CdvBNB$Cc|Z-1PstqS3T~|I|gSrC|E68uB0Y)Jrpm3H3x}hjE7QiuZ!Ip)c6gff;Tk7!x1_q;t(Nx{C|I8c_SRl zRHr~tdvn^q1Lfg61JDk4>Uwm$0vL*~LIpL@r|h<`*%^BEng2z(AG>FsL(>~H@W3?BV?h3^1Fh~N(#;?F;)8E#-fOqJ`M z@OqyH#E*u`?DWcgL8;qI4rTo=CK)%B;-8sAkmUlzFNvV!Z3H8$++8UG`GfgJQ=n|} zOPoo8;I7TL7~uJ10SqNeff3s%$T~OBgG?9te>b3@i=mE_tj!Q+NJ*shugkv6<_83T z#zCMn0VX{~pw1ZKZKE*-ALgSd@B|C|wA&*vH8S1cpY~!1Z~`EHgi4z5AYe>1hz6d3 z6G*4d=LB+fcWWXNfbbcEPi%JGK)NaFa15Yrzn7p+48SgmW;OZ~|Ml&S#mV;p5|azE zJIzdK(0yKLk7+`ekAXbdR$1`{uTwo}qRu&I6J5XCe);O{SGMz0K1aDw0b_3j{0DNJ z=34D(O#tlTs`^05?FIjzFU}A~2G*zycX;59f34A@xO%5HTFV-*J$`gMo##(R0Gezt z#ld+5fJ@)b1Qwz}P#8RmiBW=D(r$wqS~sZq(`_kDQs<4guXUOBBplo@dEdRY6!D-R z+>Ei-DEbX|s+o_V%^YTHBlG7~ofqA7~kzJqPII-wW zT|zW??yf!sh(R$hjv58Uh5q^-qm7*6US~s_l1S2D%XbXqc$3O+V9tfz zYL0L&h{KfT@gr%*;CIysWlzeCZ$d8eT`U&I zt-sb&GPuSZ-X3=a|K3~BlM!)TFq`Sju3PdcC1x9Z{8B*4qcTaOEVeCs`n?sD+{=hQ zp!PS{DBR%{sHM9;adr{S!+4%P;{wq}F)6d`YDV0{{6B+E0SQ3*74Dx+2Fs1am3q=7PV;R>Y5|-)Y&XmYd%J`DEvv2ED{IzW%TU&GU310 z{y(5uzQh-X-M>vZ^m<^L7X|Nd)q19Uk2|1SYuw*Pu9qIY09 zMs=7Es%h4yt>K0AKKl))AoR2|oEjC>&PJ^nuMH|NYI)zUH<_bKvEyk-{v-QBr z_4tqp3RwRYchJ`N`qia<3Nkj)UCUBh>=Af@Ibs`d9IexoJ514)t14auVl;dAee*vw z%l$#^N(cAlJ)27DKO<4}dwhoIo#nwU-7?#!0ENpJJ^cC3?YoM==#2#ZClI}i`Y1=I zcy&2Dzl_?xg3+LN0YZSlHkmL4DeAMQ+{M?ICMt{Pc!0}kil|qa9>lcd%k~=X1MuGZ zp*Lvk@BQ2Zu8t+Z^3gf!q|l9_!ifmJz&8#tD51fCIX6H?yzU+ecyVERZ0}!6uYG)$ z)A`2&{U!_2u`%T5vVSak5Jn1YhPiA2I1Hw+hakKfnv?}2)5GAsMz$n72hI{=cc6B>CeTm?koBE6|HyA?Q8M6iS{$g?l+UGQKq|}?gLV&4p>^8?(6futZ=@4aPC7II{JI*3LyBd0-j@$kqUYR0u3ut4~06TMI62kH{miMD}yja#LH>f zJK}Q=_8L%ui$E&Txf8TU(MW#;n4vnu9Fj{i6b5F6h-HB5I)Ec7ksx>8Q{Qz^qxHa~ z#1+iajX{-Iqp#2~xHN6O!YdqFVy3NxZLx4|=ww)@A+@T3&|-}Do&m*muVzv?ZHR*W z{9vnAu!v=#0>GL4Eg^d>LAy#c~0K=f{ZfK%w} z$5*0s@89`rlA1BFeaL$R_$hA+FT`BhB;>t3!!icD$Re&QNKe@aKZ21+OJ4`A6!NcF zdnseasWE>NF{}Bl#ud&nY@k%I)J>_0%-qwHBM~Q5RqY7?R4Sh9&i91!OVT`L1I?tN zH}@~Fp&&ZV?wCz}<(Kdfk|$KAlFd@>Gyapn^T~b{c8{jcyzAN`XVBNHc1TBFS^BNz zB%Vu>qEz%uW+461_nqI9d?01jEa>y~&HX$gZZjW1l+5$>HOM?7bZGJuBkg!X#l$v~ zjO+$tdL4k-Zw>iGiey097T~MQ5=&vFr$hNs0<8pWxXSCKxShbAvHDA@$lG_pHZWaK zo4ZOw2c4Bl9%E@4U)twJz{KiHDLC8k-|jQ#uIWUdjf_pWFN398ONTd7hiEnHCYxgP8ppWrs`Is7UH^ znSyIoELbLp#yOZg1CCpRT_9G)2Ao?uV$7$xL?V5+wSIrI_O^N^iaZfJvK3eX1Z{)? zw?A|0s=ETM^8n+ybf3ee){+I11+rG+xH{Fbi>9+zw)VlIFcTx(=6AcpB!N%J!0!iJ0*{ zGSP#i5Uc#YHXo^kuEO><3OYDPksdBx>>^hcY{14h!k;%XT~GIX%FF6S35-n1vm5E2 zA{+X--=>(%p&@9ek*(>-G?!enlQ0Wa zbng9<^oEWYcq3c~pOGlc$}n2y!A|Q*z0ho0yQo{sRjqv{idm2@T2@6*wX5d-Q4E?< zLK$nICec4LRP$R&A;8I z^DbGGGQhVXqE_Zb^Fs4JtbMFzC&U<6#iEHMm zB;8Fv+OCTd=S7Lr_YKjv-TRF@fKrSYeDS0votpm?#h`k(#6kTe)eSZV`!uQ|)f^^* zV)is)U%fgXw9-PpF)2gL5nJWO{EWmTvI8I1M!V0G{M)syqEuy?dFO(T!za?#2BFjLqMu)1SaJC|_P{BNq;CqYnHO zfDL3Lf1B3*Vo9`3Kj2C&{Nv9A9pKbQzaa_x+>6A|HM{hH@NMLC{a^m24pSI3PjK{a zZdom9DG&EBE$}N$8=op z5Bwu|2STzrfp0aM{u#S(Vu5&av_?Tub7K;d7@>V#A|ZS z^n(gwil_Mw?wwUYz;kP)A}40J86%j1Hk((OLw795P+CpZdK;Vt2gB~0YzPfM2+!9} z6~uE$6;L?nC~sAcL+Ur2->7sHH4^7kLf2G1TOw1~>MMlYQ@bc+kDl|OszDN2;mS(; zg6NLhY)r8HRsOB9nG>0GLEPJ|?cwi(U2aj)z{_Bt7m}a+3J?sd5Banl;Z+|Mbo35n z8TQ}sEiUb}R`NF8YW#U`Sbxq9BtPQ@mi&p8aYuNCQ6zJq+HB?fT`~HFvqdu_UA7!r z-QUsut1e@(A(aS+5`JFriAwMpC+9d0B1Ck{HXd`CmR|ZH_{OloW9nz-;}@?y<0;Gt4Xa>DkfL*+lny z(U;FH$o=P!pbQ#H>a|pcZ!jjMveWGDKCcJ_Q%RV)!}oU|oM!1(yXAK2$zp`U-QerT zr55yzsq-O`dmKG~X#vcg%9{<`P}36}P8AP43Ou!9o2ca;{@wSIW-x`oKFU06jeo&C z#JV`ah;A}PdFChA=$nP(m3F{2Nd8^Q{C+a}8oH!*5Mthx|L}b%K);x)(>ui?yui>GRj2XF zP;S|imdk((mGPR}$S0#UdCVW^OR61#DsNVk8%T4%zO8ZF{-E~C>O|%Dcn+N>L3Ey< z7J6%Z&&+fIYvwAj;1)CrFNB8AH1qU@*dKjRF8Usxmy*LXGCf70%&wVZ1sEe`KtNj? z)Hd(3-&>2kcgp(Wcjb0J{~U*UrgXRND<`hhpx?v}zBq_~6jyfm=T!6WuPLvE1CS_=A+7LTXT#Iu#;)@hE@gd4s`=tyA#DV%&WRx{y_gjk=2))cMKP}`iY5>$?^Q?~dH)D3RT_24lSkc^~ zO-U{Sz1{&Z?{~F)89XzCh6pE|uR`ZA5Fs0>%h%vr&Kb!2N99|8UhSM(GRVf|ddbyi zUOiF`v1QX8t@W;k>f5|A&@poWg7B)(qPwD!Xl9CWYPCZ9&ic<+#?T2X6_f6+v#e3Z@n( zZMbu>_3+e+KKO!_K&~6e`l$@#N1)-VcnHGKUZl@kxoz=PE7NIJsT`_PfjGNvo_^)- zdn#IXXb)0JN~Bv+z0`>KCtM9wbkNoXpx;!z00OU_Nfe)SG=z0SlJK3IBTrKL0M@V~ zO{4{X?g~aDA(lsZ2;2>b(CL9V!DJ{drvGg?%FqzVud7O!`a;u!wMXkN8ZZ&CTvP5j zU?xff+6%1*Z)@>`DpBAXdI=)EX^EM&!?iMa!H-t=KKcaGX9(%0P(1*dK&5B6-9C0E zFAd!7;WUy(6i>TPWLrx3_B8U)7rd{5cP*%q&A~FtWrvtztG{zW{uW^h$we#Q-c+F& z{#w9qS|pM_0&J=DouBuEA^CEPe}>u(W3V&1SZ`cL)q zNy`{EQy|kwaUzCZpO!rP6@XpLg(f2br;HLoMZMPv6~wwPVP1EP- zKaql|j@qs-1`N!+Jnv;_9BKUwSKC=UicVDcdBqR_T8PPTgh*ny2cG_OfzTM^LSpak zvM91mDz9)Y|j0O%$lX|AO8<~e;y9y{{MmFC1qrrA$yiVRAkS-#gIy*MHI3|*$HFI zHW=B`hO!i$v`EOBA;w;^w2^)6vW=8OeEXJ%giSr^n{`G4R1-JP2t)? zt7iEEE*U-)cq9>hlimhitQYt`S2e=B(?3`GgMC4$J)(AfbM7cKFqeXs?qX}P=-7f|8Q=IUQNm(=hf<__v=I30`65IOu6%(=iMO0QQH#vb^pwCYQzs>V?! zg(LyYWV2|jR(Z1rsb_kX8Aoq$&2i_m-HVP57I%l${~#@CxH&~G&Ar(px;}gxFW-9e zTh%Q-t4&_*)uAp0dRmn&rUc-Q=zqiGJS{M9G1f@kO~(m+2wlh0*KdCBdY69OP-Lp3W53S~-{)3~Eh1i{->DhMIDT%_G;y z58jC>AAP9Zc4V)P^Ru(hZaKxE57+LX&CcuC;{vAoCc{N?);r$8uMQbOml5Ma{|L#N z`ego5Ac92R(S0@i;XwzYj9r6S+n(?;4|^7?J)j=7re=%6ppK+pTusJJB1qSX;?_>g9dzmUuP-NS^@AlkP%N6c z5u7fi3ituaUGGlos4|%$Yl2w%n1M#2X(S$GB&wV?H`>C-!?5+A;Rm5MY@@ZiUX>~= z#NK5P3%QD=UAy($<*AvFr&735P?xG;nDTFJKwx$T94Xvub<|YtJ0q#KW*6%BIPFF3 zDNY+H4(a*e@>i0QBAi?fV7QcE+OfC#wu)l!wcDB^6jtlx4F;(~;0eDWpiTU-4|vwY zEpOKloWlbHLLjC6^)suAu_p=>xwaBK2@DzmszVPs ze?#-l4>t{&CJv;c6=?UXokN?GBlSN)_&QaxB%Y2pa)0=EsNX9k?K$*M5|#$xv)$g< z!x`4&oCOuZweDe}_MK|#qIJ%J9~dJ`1#JNtCG0rn-nYUlXT(+o+puV9!`YN5#Zy9A z@to2@oqW^6XbOE~$v5EGIIoOOy>9i9GapQhxH+!l3Sqa^`e}u^?la9?U2se_J|b8M zo$}X_*2eATB|lc2ZvZk=Q#R{Gt60NY-TjffZ~Bsp*b;Z@hztc{%uA2jM3}7)nLfEZ zMOhDc9WTb$7fcxSA4)}skhrE+UiT!)REGwL`wMN$RkacCppWrZH$Otv{TUnM-toC5 z^4(jypKjqktC}IfPS5xfW>x!GtD9SiHx#>0NL380oyI&|TZvc9`WY-{$%|xCxDSA2 z%xA#jOj_b!b->;>TC{TD%)u>u!m>HGnJq!s4Yiw#=Csg4Fm((nw9{W=E}~3On0ETs ztdbw0mJ)*r{sNNUE27Ac@Wa;@f>v$K)@4?0YkoY|a}+FWxmX%R7OUuB$6hD2U`!?= zVd8~#d3 z8uk@MU^FSe6`rV6xhxcS*!bvbf4Eze{nZ9G4biXucv21tYB@9-DyzqE8!+fRlN`YoSgV_SGF_Yv%)9;g4eMdKTM+pxlHXNqP-Fk9OoqBq>|DGb3=OH+ccrmww%rfn-*4LPlGgbIA zw|?&Fhis~SmJfPn3B&W0e#xJjfsmEL+O8&jNOwRGD~OpEENfHwTf&UeDU`I+KE0=w zN@J{p=eq?HHG>)DZi^_F=(`$)`FZZsjd7%C-fFVdHI%Ag8g$-rk{?S6DBC)dWo%>3PF8T#j92wjhoMIb`NHap$_ zYQZUkTBeLezJ&d>e4Y|Wp~n|>vtHmEN(V;JR(QT&vA&t(#^H!usQl6WcBj zMg2mMP|ySx8Y#hUt2!=QRRuW zz`CA@Xo4X5U`GhyC-E8gzN_`+qSu2n=WdF}{}g`%Qz~a{akjaMZ{KG6Asn_}s`@6i zW=4&k(_xS{x8#o#0iX=kTS@`r)pad$6OPr^$ifFZ2#NwyYuDCldO{Xj)6GN+0DVU6ZnBjzWKDZyb5=} z%Q;?Y$CD)a&Xm57-J00JQVE5giVW+Mo-_FDeQ)|wWPCIga=i*y*<)HOW1}}hPaSPh z-E=!(qM4E}_1m0#Fxyt#IJZ>4#UdA`ajm4eB0XJsD|4eA7)GtHh&F1{k~i)j?t%XT z1A;N=Q%VjTWoVv1P_MrUp8f-${{Qqh5JiY2A@WZ}%75f6B!;^{~fjc`}cMhNW{`iQG4V6f;b9Lu!WipYddoN^Y#C{5VIsuHsl1j z$p3*;{rTemy$Z-M)&`v6|GORkxg-C7bUQp95C&umnun!XNn^8 ztG4mK3t-}i;QJWI6Za(Yftc(BlKNKKEEKp-h%f-KoS#P40DLH18$Ts(R`B@uG!^hZ zAq^AnteHa^GE3=%!t#%b_7~@+5LXsLW$j|*%6l7?we^1#NPj(0@7X$k?moMl2$Q;8 zLA&%CtYT#s?BCtVM@&hGfiLT5LNTAJPp1xx`Wg|u^2TK#5 z8eV3=)HE|g+``!b@&EP2=o<*07{ej0FHUQ+tLQ}q()@>i&#Phfau}_s$TV62fG4i+ zuCuDKU%@+i1;6vugkx=8~FS&(FP>iWW=H0}r5zQ4x~lrQ$Q? z2>rC(V8-%0&(!L4_H>Z(tP`_0Ignxknb=8XZm|3u)~rD*MM}%mGaR5#h(HK z=-)x2DK+ z8%r1zimy&R)dM~PRSzcY%>_hh!>b2~Z0{D3?eC(ULCcZ;14KZ}Z-4%1y(?@8Y341Z zxx^^I_W_)}F#?wq>wDwx>2`l#r&ok-{#K(5tI;VzZJtO8io}uy(O5=`ON7{-Wtc&r zJogj=niMWKbMXShhnNEsWl8x@!zo2eK*)DT30xnqWoSD;p;R*n&Sz_~Cq zkB(PLs3lpZ#?K9N`382s?8NpZwN`gt$`URpJ8}$yjxd-ubdOaZ1fq2?4OQ_6Z+euF zAWALmC^wC;tqvZsD5m;8e1Q%`5ZB*FCgc}_?#|G?CZO$Nj_0K(zx~xl{r$L{5)YM_ zj;?RQY6Q(UiAjpwbU&_f-E({GLGochRi>4Yalc=moa8Nu?_gbql~(GmuU061$r6;J zI)&fB8p`HMtvvpcV)g}Wit-;AZ-mHS-&`ds@&?&9%kqemSo0xu@T$U^-kz&wBWfNdv#0 z+aPnNqcOXH6BgjI0a3-d-$Td%LU?ng{=p>lFkX{@8GBm?IS;pH^fN5+KmWzUuSw>6@63fz_ zIhY$auunW@P{jQDv*4y|oI|WIlPDMK{6i9-*Sz1x?j;56Gwy#zi6%z*wJLqZNMKqU zqc~=GWf))SG+$C|Ng?^*3z75`y=1XLzLgyUxR39p-l|!bIh{pfU;Hn<)%5Uyao+T}q+x*y=ecKOP1m}yZ zJwcQO(OULyfa9@hHpC1062|=$pP{)$zrJw>V^$qCf?}T|%0VM%PiWhv5*X(B$wHAd z*%(t#U>*)f5)CaH#mc@&=bp^;Ja7Cz-qcNItg1N;XK^dj{N_#PNFPWRx9Y+by}+6v1QjMwT+QeA-N{z)y4ob3rTi+W%H*pV2|| z0sWEP=`MY)07G{uK8gh*t?;MAtXkFM{r9EKhf`nE(q{_pqHV@tY9&SqfJ zR{Y$*DrEcCO+Vv9Z-n!X@JiTe8Op88!uZtjQVjW)%y-PpI^4@{IOc~v>pr9^%RN!N zm{$?olE!Gn5up`pyPb}MtLjtib*tuma2wNLg11Zkrc8;1Xh>=O)!zuy1sR-PEq)dI zkF}`c>TnI=wjckl8T?zJKD9g4c{(~W+XVRG-tTu9Z=I8_2#M6!$@C+z&xc@HvCpxj zfuhI;R^=i@>AZULe60B;u09u{3|=RnMl_81(Y-q>ks*<*Lx;GokFTuxey5AOE<9sR zh;2!cYj7Dl&~W@gAE+rO6hBo64`W5K+A@Z>#@PmmV0sikxkg=69$!f$^#?>?m9QL; zOe%fy2Q|vBB@JU#+XN>`7kx(a1I{+IoSDRSTbXUOvT7TR4pAYI)CTiguj2zfhVzD$ zY`z0>miq8`(D%#N)YK${{+;FvZDws`YkE*5{QBg(>y2X($=BeVQQ6Lt_!6eV z(Lc!6u;Kqvb^ASoExmZ1vc9D3{If(ai4>n(>b`nv6U<{H5e*MnD2zV0Puwi4+tCu` zv@2oA4`C$~UlHab$278rs*4K`{o6we+6(MWTU=o?qonGW>T(7aCC1i+o`DqaegCVj zYuutO@HQ-*N|0LPSGfAQal}eki*Ec%Nkw#vGs!#SPSg7A2n7Y5DY%$>-W8Tp7Jt4w z07Bduj)l%rkE}J$m`iyXD@b+A?)Ag}An7%6SIn8;dow{{8bleZr+e-1J{Kz%(TDCW z>Uht?^8dbrqeFAB@Go*GPk#>_{rjc;`TepL;=vhc`qK9AZvk?DBasg8{|$d)ZDk<; ziemsO6!fv(z!>&Af?bcB@0@#Z)2)3koxc^d+jjlyNeA~y1iE_hE6-mBM^r)fs=d1Z zqPt51&*66!QeXyy7RdoH(-LTQx{wLWnCVN8Ug@3LM}?yF0?bG{{qo8DugmCLY6w~R z^&0K7fC+jDvw~VAF_N}WNcTo($o3sMU|^VFxSD{P+C>A?pYo&G#AYJpCjW2V#P3V* zrzp61emuDL#pfQIV9vq~@!UK(jV>aQrcf)DK>u*GRSp>=0cpR^U4>Un3LTAAN0V)* zzFu@Ig3Y?rf6o-CfX`k-LfM@rpFD6afIdQ~s|5gjI|x}U3};qp^n1f3pwXk?*5`~! zkX2_&YY_b|=y^Tu5!XmOcD%^3o$M&?)<^Pb+Lu(iED1rb9jySXbb)Qb1&L%uKqf@4 z4#-Ra#LIq)SDV*Af6qgGxzkp`XUeq0p`{;;sEGU#UZNFY9M9*TNRpIMpG8_4n2=Hc zbmbux4wuamS{?;wM}(3p_u1-}n{@v5-oj0ZcKj)7QIs}9OA67efp~*L5P9pihBkhN z5^N=9?-hgqeWVk`lkS8%18QzrT(Nacq%vk2n9BmBXPc+n>#kMErvO9HDLC;iMjmh8 z!&mLHQ$24W-YP?Voo3;{S;!yt8l?B@hQe&*`>=lRocRUS)(^U{_9Bk>lWO6O}{@?&Y|^E?xcG9g+6p) z*fMYzZxRrH*zX#4(2T!nMHvD6ExT-sGV6p+uNTQwJamt1kXdhQ;9I2F1^AG9B(;}N z-?Bgw?L8I2Y3Z*04Mb8i@kS}Kb?3cj&s+Na76dI}yP6rwju?Q*c2(o^ z@BO;L_=v&Mo;F-p@qI^&en?%!-YEd?6v+bM&SXOZVc+=DijYdez^VAQ>{K(-CHe(_ zNLN=q_{7iSkZGUWkcwTafNRJoJ9Sc9ZqBpDXR4F(qI?DE zA1gY*g64N`8d_Jwf!(`(bL78oBs6f7dF#uh$tPOXSIh%|9PS;%2UEiqg!Ycrl^^QK zzIPNL8={{kCyzDA87b8Efds(xKQhaAm$kFyDk4U$MJon~4H`hLYNcrwaU?E