From 5342c015ba47daa410128dfa5c79e2865e74d244 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Sat, 9 Sep 2023 01:24:10 +0000 Subject: [PATCH 001/521] Preparing development version 3.5.1-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- common/utils/pom.xml | 2 +- connector/avro/pom.xml | 2 +- connector/connect/client/jvm/pom.xml | 2 +- connector/connect/common/pom.xml | 2 +- connector/connect/server/pom.xml | 2 +- connector/docker-integration-tests/pom.xml | 2 +- connector/kafka-0-10-assembly/pom.xml | 2 +- connector/kafka-0-10-sql/pom.xml | 2 +- connector/kafka-0-10-token-provider/pom.xml | 2 +- connector/kafka-0-10/pom.xml | 2 +- connector/kinesis-asl-assembly/pom.xml | 2 +- connector/kinesis-asl/pom.xml | 2 +- connector/protobuf/pom.xml | 2 +- connector/spark-ganglia-lgpl/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 6 +++--- examples/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/api/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 45 files changed, 47 insertions(+), 47 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 1c093a4a98046..66faa8031c45d 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 3.5.0 +Version: 3.5.1 Title: R Front End for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: diff --git a/assembly/pom.xml b/assembly/pom.xml index a0aca22eab91d..45b68dd81cb9a 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index ce180f49ff128..1b1a8d0066f89 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 8da48076a43aa..54c10a05eed22 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 48e64d21a58b0..92bf5bc07854b 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 2bbacbe71a439..3003927e713c1 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index fca31591b1ef1..43982032a621d 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index a93e227655ea7..a54382c0f4d03 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 49f7e2d8c861e..bea8f1ba87c57 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/common/utils/pom.xml b/common/utils/pom.xml index c200c06a42e69..588fa3950dc5f 100644 --- a/common/utils/pom.xml +++ b/common/utils/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/connector/avro/pom.xml b/connector/avro/pom.xml index 63b411137ed7a..c24ebad632940 100644 --- a/connector/avro/pom.xml +++ b/connector/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index 8c9d11f64eec8..67227ef38eb8f 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../../../pom.xml diff --git a/connector/connect/common/pom.xml b/connector/connect/common/pom.xml index c78c5445e5073..40de7414051bd 100644 --- a/connector/connect/common/pom.xml +++ b/connector/connect/common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../../pom.xml diff --git a/connector/connect/server/pom.xml b/connector/connect/server/pom.xml index 10deea435d2bd..403255c543727 100644 --- a/connector/connect/server/pom.xml +++ b/connector/connect/server/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../../pom.xml diff --git a/connector/docker-integration-tests/pom.xml b/connector/docker-integration-tests/pom.xml index 87df8a9ff5bea..1bd6b8e7e4883 100644 --- a/connector/docker-integration-tests/pom.xml +++ b/connector/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-assembly/pom.xml b/connector/kafka-0-10-assembly/pom.xml index e7d86b6fd7560..69908e8bb4fb9 100644 --- a/connector/kafka-0-10-assembly/pom.xml +++ b/connector/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-sql/pom.xml b/connector/kafka-0-10-sql/pom.xml index 8f41efc15cacb..72b6d7038b7e5 100644 --- a/connector/kafka-0-10-sql/pom.xml +++ b/connector/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10-token-provider/pom.xml b/connector/kafka-0-10-token-provider/pom.xml index b22b937cd821e..e76f5d59d5048 100644 --- a/connector/kafka-0-10-token-provider/pom.xml +++ b/connector/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/connector/kafka-0-10/pom.xml b/connector/kafka-0-10/pom.xml index 825868ebd9581..d6a023392f87d 100644 --- a/connector/kafka-0-10/pom.xml +++ b/connector/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/connector/kinesis-asl-assembly/pom.xml b/connector/kinesis-asl-assembly/pom.xml index 312b9c460777a..90a568d1c3d83 100644 --- a/connector/kinesis-asl-assembly/pom.xml +++ b/connector/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/connector/kinesis-asl/pom.xml b/connector/kinesis-asl/pom.xml index 134f9c22d7436..666f757996d7d 100644 --- a/connector/kinesis-asl/pom.xml +++ b/connector/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/connector/protobuf/pom.xml b/connector/protobuf/pom.xml index 7b8b45704a5ef..8021b78141ac4 100644 --- a/connector/protobuf/pom.xml +++ b/connector/protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/connector/spark-ganglia-lgpl/pom.xml b/connector/spark-ganglia-lgpl/pom.xml index a5870edfc7c81..df3deb35a772b 100644 --- a/connector/spark-ganglia-lgpl/pom.xml +++ b/connector/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index c40f9905245cb..eb4a563f1f31f 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 19cadd69e61ba..afe015b2972da 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -19,8 +19,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 3.5.0 -SPARK_VERSION_SHORT: 3.5.0 +SPARK_VERSION: 3.5.1-SNAPSHOT +SPARK_VERSION_SHORT: 3.5.1 SCALA_BINARY_VERSION: "2.12" SCALA_VERSION: "2.12.18" MESOS_VERSION: 1.0.0 @@ -40,7 +40,7 @@ DOCSEARCH_SCRIPT: | inputSelector: '#docsearch-input', enhancedSearchInput: true, algoliaOptions: { - 'facetFilters': ["version:3.5.0"] + 'facetFilters': ["version:3.5.1"] }, debug: false // Set debug to true if you want to inspect the dropdown }); diff --git a/examples/pom.xml b/examples/pom.xml index c95269dbc4bb0..e85bc9ea02f8b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 30fe05957d569..19d58f8c8b148 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index f67475ac11bc0..2b19307862060 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index d3c52a713911a..18ad615675b6e 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 5ec981a7816be..0660631bb4a9d 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index fe7c3da9c4eb2..e5948ba6eeff2 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 93d696d494e84..8fc4b89a78cc2 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT pom Spark Project Parent POM https://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index daccb365340b7..64ef604affff2 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__: str = "3.5.0" +__version__: str = "3.5.1.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index 6214dc2e18555..875d0455ee778 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 11420fe7fa76c..39771c07ca549 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index c7e543137385b..b420eda3eac5e 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 1dda41e085178..4c2fc6ef28aa5 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index fc07d304a038f..4d5cec6aba5c0 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/sql/api/pom.xml b/sql/api/pom.xml index dd2ebad69a058..0763c5266a2df 100644 --- a/sql/api/pom.xml +++ b/sql/api/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index f9cd2dc677adf..d7fc835fddbcb 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 7313ee5c41340..c46c80ecff70b 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 7222d49ecb020..3659a0f846a5e 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index b04c7565f8a3b..f833b6c34fa94 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 55758d75ce54d..75a98c050b5e0 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index a63b2e1062dd8..1805ba06c5b49 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.5.0 + 3.5.1-SNAPSHOT ../pom.xml From 3c92ecf21e0a00ba96d7934a4e27ec5b8d339762 Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Mon, 11 Sep 2023 08:06:56 +0800 Subject: [PATCH 002/521] [SPARK-45109][SQL][CONNECT] Fix aes_decrypt and ln functions in Connect ### What changes were proposed in this pull request? Fix `aes_descrypt` and `ln` implementations in Spark Connect. The previous `aes_descrypt` reference to `aes_encrypt` is clearly a bug. The `ln` reference to `log` is more like a cosmetic issue, but because `ln` and `log` function implementations are different in Spark SQL we should use the same implementation in Spark Connect too. ### Why are the changes needed? Bugfix. ### Does this PR introduce _any_ user-facing change? No, these Spark Connect functions haven't been released. ### How was this patch tested? Esiting UTs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #42863 from peter-toth/SPARK-45109-fix-eas_decrypt-and-ln. Authored-by: Peter Toth Signed-off-by: Ruifeng Zheng (cherry picked from commit 5e97c79ad82914fc77601ad33cc304c61de93e87) Signed-off-by: Ruifeng Zheng --- .../scala/org/apache/spark/sql/functions.scala | 10 +++++----- .../function_aes_decrypt.explain | 2 +- .../function_aes_decrypt_with_mode.explain | 2 +- ...nction_aes_decrypt_with_mode_padding.explain | 2 +- ...on_aes_decrypt_with_mode_padding_aad.explain | 2 +- .../explain-results/function_ln.explain | 2 +- .../queries/function_aes_decrypt.json | 2 +- .../queries/function_aes_decrypt.proto.bin | Bin 187 -> 187 bytes .../queries/function_aes_decrypt_with_mode.json | 2 +- .../function_aes_decrypt_with_mode.proto.bin | Bin 194 -> 194 bytes .../function_aes_decrypt_with_mode_padding.json | 2 +- ...tion_aes_decrypt_with_mode_padding.proto.bin | Bin 201 -> 201 bytes ...ction_aes_decrypt_with_mode_padding_aad.json | 2 +- ..._aes_decrypt_with_mode_padding_aad.proto.bin | Bin 208 -> 208 bytes .../query-tests/queries/function_ln.json | 2 +- .../query-tests/queries/function_ln.proto.bin | Bin 172 -> 171 bytes 16 files changed, 15 insertions(+), 15 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala index fe992ae6740bf..16e787f825a6f 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala @@ -2624,7 +2624,7 @@ object functions { * @group math_funcs * @since 3.5.0 */ - def ln(e: Column): Column = log(e) + def ln(e: Column): Column = Column.fn("ln", e) /** * Computes the natural logarithm of the given value. @@ -3477,7 +3477,7 @@ object functions { mode: Column, padding: Column, aad: Column): Column = - Column.fn("aes_encrypt", input, key, mode, padding, aad) + Column.fn("aes_decrypt", input, key, mode, padding, aad) /** * Returns a decrypted value of `input`. @@ -3489,7 +3489,7 @@ object functions { * @since 3.5.0 */ def aes_decrypt(input: Column, key: Column, mode: Column, padding: Column): Column = - Column.fn("aes_encrypt", input, key, mode, padding) + Column.fn("aes_decrypt", input, key, mode, padding) /** * Returns a decrypted value of `input`. @@ -3501,7 +3501,7 @@ object functions { * @since 3.5.0 */ def aes_decrypt(input: Column, key: Column, mode: Column): Column = - Column.fn("aes_encrypt", input, key, mode) + Column.fn("aes_decrypt", input, key, mode) /** * Returns a decrypted value of `input`. @@ -3513,7 +3513,7 @@ object functions { * @since 3.5.0 */ def aes_decrypt(input: Column, key: Column): Column = - Column.fn("aes_encrypt", input, key) + Column.fn("aes_decrypt", input, key) /** * This is a special version of `aes_decrypt` that performs the same operation, but returns a diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/function_aes_decrypt.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/function_aes_decrypt.explain index 44084a8e60fb0..31e03b79eb987 100644 --- a/connector/connect/common/src/test/resources/query-tests/explain-results/function_aes_decrypt.explain +++ b/connector/connect/common/src/test/resources/query-tests/explain-results/function_aes_decrypt.explain @@ -1,2 +1,2 @@ -Project [staticinvoke(class org.apache.spark.sql.catalyst.expressions.ExpressionImplUtils, BinaryType, aesEncrypt, cast(g#0 as binary), cast(g#0 as binary), GCM, DEFAULT, cast( as binary), cast( as binary), BinaryType, BinaryType, StringType, StringType, BinaryType, BinaryType, true, true, true) AS aes_encrypt(g, g, GCM, DEFAULT, , )#0] +Project [staticinvoke(class org.apache.spark.sql.catalyst.expressions.ExpressionImplUtils, BinaryType, aesDecrypt, cast(g#0 as binary), cast(g#0 as binary), GCM, DEFAULT, cast( as binary), BinaryType, BinaryType, StringType, StringType, BinaryType, true, true, true) AS aes_decrypt(g, g, GCM, DEFAULT, )#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/function_aes_decrypt_with_mode.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/function_aes_decrypt_with_mode.explain index 29ccf0c1c833f..fc572e8fe7c67 100644 --- a/connector/connect/common/src/test/resources/query-tests/explain-results/function_aes_decrypt_with_mode.explain +++ b/connector/connect/common/src/test/resources/query-tests/explain-results/function_aes_decrypt_with_mode.explain @@ -1,2 +1,2 @@ -Project [staticinvoke(class org.apache.spark.sql.catalyst.expressions.ExpressionImplUtils, BinaryType, aesEncrypt, cast(g#0 as binary), cast(g#0 as binary), g#0, DEFAULT, cast( as binary), cast( as binary), BinaryType, BinaryType, StringType, StringType, BinaryType, BinaryType, true, true, true) AS aes_encrypt(g, g, g, DEFAULT, , )#0] +Project [staticinvoke(class org.apache.spark.sql.catalyst.expressions.ExpressionImplUtils, BinaryType, aesDecrypt, cast(g#0 as binary), cast(g#0 as binary), g#0, DEFAULT, cast( as binary), BinaryType, BinaryType, StringType, StringType, BinaryType, true, true, true) AS aes_decrypt(g, g, g, DEFAULT, )#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/function_aes_decrypt_with_mode_padding.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/function_aes_decrypt_with_mode_padding.explain index 5591363426ab5..c6c693013dd0a 100644 --- a/connector/connect/common/src/test/resources/query-tests/explain-results/function_aes_decrypt_with_mode_padding.explain +++ b/connector/connect/common/src/test/resources/query-tests/explain-results/function_aes_decrypt_with_mode_padding.explain @@ -1,2 +1,2 @@ -Project [staticinvoke(class org.apache.spark.sql.catalyst.expressions.ExpressionImplUtils, BinaryType, aesEncrypt, cast(g#0 as binary), cast(g#0 as binary), g#0, g#0, cast( as binary), cast( as binary), BinaryType, BinaryType, StringType, StringType, BinaryType, BinaryType, true, true, true) AS aes_encrypt(g, g, g, g, , )#0] +Project [staticinvoke(class org.apache.spark.sql.catalyst.expressions.ExpressionImplUtils, BinaryType, aesDecrypt, cast(g#0 as binary), cast(g#0 as binary), g#0, g#0, cast( as binary), BinaryType, BinaryType, StringType, StringType, BinaryType, true, true, true) AS aes_decrypt(g, g, g, g, )#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/function_aes_decrypt_with_mode_padding_aad.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/function_aes_decrypt_with_mode_padding_aad.explain index 0e8d4df71b38e..97bb528b84b3f 100644 --- a/connector/connect/common/src/test/resources/query-tests/explain-results/function_aes_decrypt_with_mode_padding_aad.explain +++ b/connector/connect/common/src/test/resources/query-tests/explain-results/function_aes_decrypt_with_mode_padding_aad.explain @@ -1,2 +1,2 @@ -Project [staticinvoke(class org.apache.spark.sql.catalyst.expressions.ExpressionImplUtils, BinaryType, aesEncrypt, cast(g#0 as binary), cast(g#0 as binary), g#0, g#0, cast(g#0 as binary), cast( as binary), BinaryType, BinaryType, StringType, StringType, BinaryType, BinaryType, true, true, true) AS aes_encrypt(g, g, g, g, g, )#0] +Project [staticinvoke(class org.apache.spark.sql.catalyst.expressions.ExpressionImplUtils, BinaryType, aesDecrypt, cast(g#0 as binary), cast(g#0 as binary), g#0, g#0, cast(g#0 as binary), BinaryType, BinaryType, StringType, StringType, BinaryType, true, true, true) AS aes_decrypt(g, g, g, g, g)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/function_ln.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/function_ln.explain index d3c3743b1ef40..66b782ac8170d 100644 --- a/connector/connect/common/src/test/resources/query-tests/explain-results/function_ln.explain +++ b/connector/connect/common/src/test/resources/query-tests/explain-results/function_ln.explain @@ -1,2 +1,2 @@ -Project [LOG(E(), b#0) AS LOG(E(), b)#0] +Project [ln(b#0) AS ln(b)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt.json b/connector/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt.json index 06469d4840547..4204a44b44ce0 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt.json @@ -13,7 +13,7 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "aes_encrypt", + "functionName": "aes_decrypt", "arguments": [{ "unresolvedAttribute": { "unparsedIdentifier": "g" diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt.proto.bin index c7a70b51707f321c5e0ca295920fd6a287693c1d..f635e1fc689b1127bebea3b7ad0930f73869006b 100644 GIT binary patch delta 28 ecmdnZxSMgpGSQUOc!6=kGNzQ&i7PY!A;|>= delta 12 Tcmcb>c!6=kGN#nLi7PY!A@T(e diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_ln.json b/connector/connect/common/src/test/resources/query-tests/queries/function_ln.json index 1b2d0ed0b1447..ababbc52d088d 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_ln.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_ln.json @@ -13,7 +13,7 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "log", + "functionName": "ln", "arguments": [{ "unresolvedAttribute": { "unparsedIdentifier": "b" diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_ln.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_ln.proto.bin index 548fb480dd27e65e18dd064ce7d7bfeb5215e293..ecb87a1fc410205cf387950c7e60b407d310d50e 100644 GIT binary patch delta 29 kcmZ3(xSElTi%Eb{YRN>l>3qCW++0jKc|xp0%v_8~09V}wC;$Ke delta 30 lcmZ3@xQ3C9i%Eb{YUxC_>HK_BJY38<`RPKeLd;x@NdRB=1(g5* From ecf507fd976752eb466ccba4a7ed005c1542a22d Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Mon, 11 Sep 2023 19:04:41 +0200 Subject: [PATCH 003/521] [SPARK-45109][SQL][CONNECT][FOLLOWUP] Fix log function in Connect ### What changes were proposed in this pull request? This is a follow-up PR to https://github.com/apache/spark/pull/42863, the 1 argument `log` function should also point to `ln`. ### Why are the changes needed? Bugfix. ### Does this PR introduce _any_ user-facing change? No, these Spark Connect functions haven't been released. ### How was this patch tested? Exsiting UTs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #42869 from peter-toth/SPARK-45109-fix-log. Authored-by: Peter Toth Signed-off-by: Peter Toth (cherry picked from commit 6c3d9f5d89dfc974a5f799b73325aebf10f3cf16) Signed-off-by: Peter Toth --- .../scala/org/apache/spark/sql/functions.scala | 2 +- .../explain-results/function_log.explain | 2 +- .../query-tests/queries/function_log.json | 2 +- .../query-tests/queries/function_log.proto.bin | Bin 172 -> 171 bytes 4 files changed, 3 insertions(+), 3 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala index 16e787f825a6f..8f55954a63f33 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala @@ -2632,7 +2632,7 @@ object functions { * @group math_funcs * @since 3.4.0 */ - def log(e: Column): Column = Column.fn("log", e) + def log(e: Column): Column = ln(e) /** * Computes the natural logarithm of the given column. diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/function_log.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/function_log.explain index d3c3743b1ef40..66b782ac8170d 100644 --- a/connector/connect/common/src/test/resources/query-tests/explain-results/function_log.explain +++ b/connector/connect/common/src/test/resources/query-tests/explain-results/function_log.explain @@ -1,2 +1,2 @@ -Project [LOG(E(), b#0) AS LOG(E(), b)#0] +Project [ln(b#0) AS ln(b)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_log.json b/connector/connect/common/src/test/resources/query-tests/queries/function_log.json index 1b2d0ed0b1447..ababbc52d088d 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/function_log.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/function_log.json @@ -13,7 +13,7 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "log", + "functionName": "ln", "arguments": [{ "unresolvedAttribute": { "unparsedIdentifier": "b" diff --git a/connector/connect/common/src/test/resources/query-tests/queries/function_log.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/function_log.proto.bin index 548fb480dd27e65e18dd064ce7d7bfeb5215e293..ecb87a1fc410205cf387950c7e60b407d310d50e 100644 GIT binary patch delta 29 kcmZ3(xSElTi%Eb{YRN>l>3qCW++0jKc|xp0%v_8~09V}wC;$Ke delta 30 lcmZ3@xQ3C9i%Eb{YUxC_>HK_BJY38<`RPKeLd;x@NdRB=1(g5* From 09b14f0968cebe0f2c5c9a369935f27d4ea228f6 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Tue, 12 Sep 2023 14:59:44 +0900 Subject: [PATCH 004/521] [SPARK-45124][CONNET] Do not use local user ID for Local Relations ### What changes were proposed in this pull request? This PR removes the use of `userId` and `sessionId` in `CachedLocalRelation` messages and subsequently make `SparkConnectPlanner` use the `userId`/`sessionId` of the active session rather than the user-provided information. ### Why are the changes needed? Allowing a fetch of a local relation using user-provided information is a potential security risk since this allows users to fetch arbitrary local relations. ### Does this PR introduce _any_ user-facing change? Virtually no. It will ignore the session id or user id that users set (but instead use internal ones that users cannot manipulate). ### How was this patch tested? Manually. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #42880 from HyukjinKwon/no-local-user. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit 47d801e5e9ded3fb50d274a720ee7874e0b37cc3) Signed-off-by: Hyukjin Kwon --- .../org/apache/spark/sql/SparkSession.scala | 2 - .../protobuf/spark/connect/relations.proto | 10 +- .../connect/planner/SparkConnectPlanner.scala | 2 +- python/pyspark/sql/connect/plan.py | 3 - .../sql/connect/proto/relations_pb2.py | 160 +++++++++--------- .../sql/connect/proto/relations_pb2.pyi | 15 +- 6 files changed, 87 insertions(+), 105 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 7882ea6401354..7bd8fa59aea8f 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -134,8 +134,6 @@ class SparkSession private[sql] ( } else { val hash = client.cacheLocalRelation(arrowData, encoder.schema.json) builder.getCachedLocalRelationBuilder - .setUserId(client.userId) - .setSessionId(client.sessionId) .setHash(hash) } } else { diff --git a/connector/connect/common/src/main/protobuf/spark/connect/relations.proto b/connector/connect/common/src/main/protobuf/spark/connect/relations.proto index 8001b3cbcfaa4..f7f1315ede0f8 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -400,11 +400,11 @@ message LocalRelation { // A local relation that has been cached already. message CachedLocalRelation { - // (Required) An identifier of the user which created the local relation - string userId = 1; - - // (Required) An identifier of the Spark SQL session in which the user created the local relation. - string sessionId = 2; + // `userId` and `sessionId` fields are deleted since the server must always use the active + // session/user rather than arbitrary values provided by the client. It is never valid to access + // a local relation from a different session/user. + reserved 1, 2; + reserved "userId", "sessionId"; // (Required) A sha-256 hash of the serialized local relation in proto, see LocalRelation. string hash = 3; diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 2abbacc5a9b7f..641dfc5dcd3c8 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -970,7 +970,7 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { private def transformCachedLocalRelation(rel: proto.CachedLocalRelation): LogicalPlan = { val blockManager = session.sparkContext.env.blockManager - val blockId = CacheId(rel.getUserId, rel.getSessionId, rel.getHash) + val blockId = CacheId(sessionHolder.userId, sessionHolder.sessionId, rel.getHash) val bytes = blockManager.getLocalBytes(blockId) bytes .map { blockData => diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 84fd013d0144a..196b1f119ba41 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -398,9 +398,6 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: plan = self._create_proto_relation() clr = plan.cached_local_relation - if session._user_id: - clr.userId = session._user_id - clr.sessionId = session._session_id clr.hash = self._hash return plan diff --git a/python/pyspark/sql/connect/proto/relations_pb2.py b/python/pyspark/sql/connect/proto/relations_pb2.py index 3a0a7ff71fd3b..3f7e57949373b 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.py +++ b/python/pyspark/sql/connect/proto/relations_pb2.py @@ -35,7 +35,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto"\xe1\x18\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\t\n\x07Unknown"[\n\x0eRelationCommon\x12\x1f\n\x0bsource_info\x18\x01 \x01(\tR\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id"\xe7\x01\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x30\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryR\x04\x61rgs\x12<\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x07posArgs\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xc6\x04\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x81\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"_\n\x13\x43\x61\x63hedLocalRelation\x12\x16\n\x06userId\x18\x01 \x01(\tR\x06userId\x12\x1c\n\tsessionId\x18\x02 \x01(\tR\tsessionId\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hash"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xef\x01\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x65\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryR\x10renameColumnsMap\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xb5\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x42\r\n\x0b_is_barrier"\xfb\x04\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_conf"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schemaB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto"\xe1\x18\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\t\n\x07Unknown"[\n\x0eRelationCommon\x12\x1f\n\x0bsource_info\x18\x01 \x01(\tR\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id"\xe7\x01\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x30\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryR\x04\x61rgs\x12<\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x07posArgs\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xc6\x04\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x81\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xef\x01\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x65\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryR\x10renameColumnsMap\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xb5\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x42\r\n\x0b_is_barrier"\xfb\x04\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_conf"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schemaB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, globals()) @@ -111,85 +111,85 @@ _LOCALRELATION._serialized_start = 7090 _LOCALRELATION._serialized_end = 7179 _CACHEDLOCALRELATION._serialized_start = 7181 - _CACHEDLOCALRELATION._serialized_end = 7276 - _CACHEDREMOTERELATION._serialized_start = 7278 - _CACHEDREMOTERELATION._serialized_end = 7333 - _SAMPLE._serialized_start = 7336 - _SAMPLE._serialized_end = 7609 - _RANGE._serialized_start = 7612 - _RANGE._serialized_end = 7757 - _SUBQUERYALIAS._serialized_start = 7759 - _SUBQUERYALIAS._serialized_end = 7873 - _REPARTITION._serialized_start = 7876 - _REPARTITION._serialized_end = 8018 - _SHOWSTRING._serialized_start = 8021 - _SHOWSTRING._serialized_end = 8163 - _HTMLSTRING._serialized_start = 8165 - _HTMLSTRING._serialized_end = 8279 - _STATSUMMARY._serialized_start = 8281 - _STATSUMMARY._serialized_end = 8373 - _STATDESCRIBE._serialized_start = 8375 - _STATDESCRIBE._serialized_end = 8456 - _STATCROSSTAB._serialized_start = 8458 - _STATCROSSTAB._serialized_end = 8559 - _STATCOV._serialized_start = 8561 - _STATCOV._serialized_end = 8657 - _STATCORR._serialized_start = 8660 - _STATCORR._serialized_end = 8797 - _STATAPPROXQUANTILE._serialized_start = 8800 - _STATAPPROXQUANTILE._serialized_end = 8964 - _STATFREQITEMS._serialized_start = 8966 - _STATFREQITEMS._serialized_end = 9091 - _STATSAMPLEBY._serialized_start = 9094 - _STATSAMPLEBY._serialized_end = 9403 - _STATSAMPLEBY_FRACTION._serialized_start = 9295 - _STATSAMPLEBY_FRACTION._serialized_end = 9394 - _NAFILL._serialized_start = 9406 - _NAFILL._serialized_end = 9540 - _NADROP._serialized_start = 9543 - _NADROP._serialized_end = 9677 - _NAREPLACE._serialized_start = 9680 - _NAREPLACE._serialized_end = 9976 - _NAREPLACE_REPLACEMENT._serialized_start = 9835 - _NAREPLACE_REPLACEMENT._serialized_end = 9976 - _TODF._serialized_start = 9978 - _TODF._serialized_end = 10066 - _WITHCOLUMNSRENAMED._serialized_start = 10069 - _WITHCOLUMNSRENAMED._serialized_end = 10308 - _WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY._serialized_start = 10241 - _WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY._serialized_end = 10308 - _WITHCOLUMNS._serialized_start = 10310 - _WITHCOLUMNS._serialized_end = 10429 - _WITHWATERMARK._serialized_start = 10432 - _WITHWATERMARK._serialized_end = 10566 - _HINT._serialized_start = 10569 - _HINT._serialized_end = 10701 - _UNPIVOT._serialized_start = 10704 - _UNPIVOT._serialized_end = 11031 - _UNPIVOT_VALUES._serialized_start = 10961 - _UNPIVOT_VALUES._serialized_end = 11020 - _TOSCHEMA._serialized_start = 11033 - _TOSCHEMA._serialized_end = 11139 - _REPARTITIONBYEXPRESSION._serialized_start = 11142 - _REPARTITIONBYEXPRESSION._serialized_end = 11345 - _MAPPARTITIONS._serialized_start = 11348 - _MAPPARTITIONS._serialized_end = 11529 - _GROUPMAP._serialized_start = 11532 - _GROUPMAP._serialized_end = 12167 - _COGROUPMAP._serialized_start = 12170 - _COGROUPMAP._serialized_end = 12696 - _APPLYINPANDASWITHSTATE._serialized_start = 12699 - _APPLYINPANDASWITHSTATE._serialized_end = 13056 - _COMMONINLINEUSERDEFINEDTABLEFUNCTION._serialized_start = 13059 - _COMMONINLINEUSERDEFINEDTABLEFUNCTION._serialized_end = 13303 - _PYTHONUDTF._serialized_start = 13306 - _PYTHONUDTF._serialized_end = 13483 - _COLLECTMETRICS._serialized_start = 13486 - _COLLECTMETRICS._serialized_end = 13622 - _PARSE._serialized_start = 13625 - _PARSE._serialized_end = 14013 + _CACHEDLOCALRELATION._serialized_end = 7253 + _CACHEDREMOTERELATION._serialized_start = 7255 + _CACHEDREMOTERELATION._serialized_end = 7310 + _SAMPLE._serialized_start = 7313 + _SAMPLE._serialized_end = 7586 + _RANGE._serialized_start = 7589 + _RANGE._serialized_end = 7734 + _SUBQUERYALIAS._serialized_start = 7736 + _SUBQUERYALIAS._serialized_end = 7850 + _REPARTITION._serialized_start = 7853 + _REPARTITION._serialized_end = 7995 + _SHOWSTRING._serialized_start = 7998 + _SHOWSTRING._serialized_end = 8140 + _HTMLSTRING._serialized_start = 8142 + _HTMLSTRING._serialized_end = 8256 + _STATSUMMARY._serialized_start = 8258 + _STATSUMMARY._serialized_end = 8350 + _STATDESCRIBE._serialized_start = 8352 + _STATDESCRIBE._serialized_end = 8433 + _STATCROSSTAB._serialized_start = 8435 + _STATCROSSTAB._serialized_end = 8536 + _STATCOV._serialized_start = 8538 + _STATCOV._serialized_end = 8634 + _STATCORR._serialized_start = 8637 + _STATCORR._serialized_end = 8774 + _STATAPPROXQUANTILE._serialized_start = 8777 + _STATAPPROXQUANTILE._serialized_end = 8941 + _STATFREQITEMS._serialized_start = 8943 + _STATFREQITEMS._serialized_end = 9068 + _STATSAMPLEBY._serialized_start = 9071 + _STATSAMPLEBY._serialized_end = 9380 + _STATSAMPLEBY_FRACTION._serialized_start = 9272 + _STATSAMPLEBY_FRACTION._serialized_end = 9371 + _NAFILL._serialized_start = 9383 + _NAFILL._serialized_end = 9517 + _NADROP._serialized_start = 9520 + _NADROP._serialized_end = 9654 + _NAREPLACE._serialized_start = 9657 + _NAREPLACE._serialized_end = 9953 + _NAREPLACE_REPLACEMENT._serialized_start = 9812 + _NAREPLACE_REPLACEMENT._serialized_end = 9953 + _TODF._serialized_start = 9955 + _TODF._serialized_end = 10043 + _WITHCOLUMNSRENAMED._serialized_start = 10046 + _WITHCOLUMNSRENAMED._serialized_end = 10285 + _WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY._serialized_start = 10218 + _WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY._serialized_end = 10285 + _WITHCOLUMNS._serialized_start = 10287 + _WITHCOLUMNS._serialized_end = 10406 + _WITHWATERMARK._serialized_start = 10409 + _WITHWATERMARK._serialized_end = 10543 + _HINT._serialized_start = 10546 + _HINT._serialized_end = 10678 + _UNPIVOT._serialized_start = 10681 + _UNPIVOT._serialized_end = 11008 + _UNPIVOT_VALUES._serialized_start = 10938 + _UNPIVOT_VALUES._serialized_end = 10997 + _TOSCHEMA._serialized_start = 11010 + _TOSCHEMA._serialized_end = 11116 + _REPARTITIONBYEXPRESSION._serialized_start = 11119 + _REPARTITIONBYEXPRESSION._serialized_end = 11322 + _MAPPARTITIONS._serialized_start = 11325 + _MAPPARTITIONS._serialized_end = 11506 + _GROUPMAP._serialized_start = 11509 + _GROUPMAP._serialized_end = 12144 + _COGROUPMAP._serialized_start = 12147 + _COGROUPMAP._serialized_end = 12673 + _APPLYINPANDASWITHSTATE._serialized_start = 12676 + _APPLYINPANDASWITHSTATE._serialized_end = 13033 + _COMMONINLINEUSERDEFINEDTABLEFUNCTION._serialized_start = 13036 + _COMMONINLINEUSERDEFINEDTABLEFUNCTION._serialized_end = 13280 + _PYTHONUDTF._serialized_start = 13283 + _PYTHONUDTF._serialized_end = 13460 + _COLLECTMETRICS._serialized_start = 13463 + _COLLECTMETRICS._serialized_end = 13599 + _PARSE._serialized_start = 13602 + _PARSE._serialized_end = 13990 _PARSE_OPTIONSENTRY._serialized_start = 3987 _PARSE_OPTIONSENTRY._serialized_end = 4045 - _PARSE_PARSEFORMAT._serialized_start = 13914 - _PARSE_PARSEFORMAT._serialized_end = 14002 + _PARSE_PARSEFORMAT._serialized_start = 13891 + _PARSE_PARSEFORMAT._serialized_end = 13979 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/relations_pb2.pyi b/python/pyspark/sql/connect/proto/relations_pb2.pyi index 9cadd4acc5224..007b92ef5f42d 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -1647,28 +1647,15 @@ class CachedLocalRelation(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - USERID_FIELD_NUMBER: builtins.int - SESSIONID_FIELD_NUMBER: builtins.int HASH_FIELD_NUMBER: builtins.int - userId: builtins.str - """(Required) An identifier of the user which created the local relation""" - sessionId: builtins.str - """(Required) An identifier of the Spark SQL session in which the user created the local relation.""" hash: builtins.str """(Required) A sha-256 hash of the serialized local relation in proto, see LocalRelation.""" def __init__( self, *, - userId: builtins.str = ..., - sessionId: builtins.str = ..., hash: builtins.str = ..., ) -> None: ... - def ClearField( - self, - field_name: typing_extensions.Literal[ - "hash", b"hash", "sessionId", b"sessionId", "userId", b"userId" - ], - ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["hash", b"hash"]) -> None: ... global___CachedLocalRelation = CachedLocalRelation From 6a2aa1d48c304095dcdf2816a46ec1f5a8af41a2 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 12 Sep 2023 00:29:38 -0700 Subject: [PATCH 005/521] [SPARK-45110][BUILD] Upgrade rocksdbjni to 8.5.3 ### What changes were proposed in this pull request? This pr aims to upgrade rocksdbjni from 8.3.2 to 8.5.3. ### Why are the changes needed? 1.The full release notes: - https://github.com/facebook/rocksdb/releases/tag/v8.5.3 - https://github.com/facebook/rocksdb/releases/tag/v8.4.4 - https://github.com/facebook/rocksdb/releases/tag/v8.3.3 2.Bug Fixes: image - Fix a bug where if there is an error reading from offset 0 of a file from L1+ and that the file is not the first file in the sorted run, data can be lost in compaction and read/scan can return incorrect results. - Fix a bug where iterator may return incorrect result for DeleteRange() users if there was an error reading from a file. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Pass GA. - Manually test: ``` ./build/mvn clean install -pl core -am -Dtest.exclude.tags=org.apache.spark.tags.ExtendedLevelDBTest -fn ... [INFO] ------------------------------------------------------------------------ [INFO] Reactor Summary for Spark Project Parent POM 4.0.0-SNAPSHOT: [INFO] [INFO] Spark Project Parent POM ........................... SUCCESS [ 7.121 s] [INFO] Spark Project Tags ................................. SUCCESS [ 10.181 s] [INFO] Spark Project Local DB ............................. SUCCESS [ 21.153 s] [INFO] Spark Project Common Utils ......................... SUCCESS [ 14.960 s] [INFO] Spark Project Networking ........................... SUCCESS [01:01 min] [INFO] Spark Project Shuffle Streaming Service ............ SUCCESS [ 16.992 s] [INFO] Spark Project Unsafe ............................... SUCCESS [ 14.967 s] [INFO] Spark Project Launcher ............................. SUCCESS [ 11.737 s] [INFO] Spark Project Core ................................. SUCCESS [38:06 min] [INFO] ------------------------------------------------------------------------ [INFO] BUILD SUCCESS [INFO] ------------------------------------------------------------------------ [INFO] Total time: 40:45 min [INFO] Finished at: 2023-09-10T17:25:26+08:00 [INFO] ------------------------------------------------------------------------ ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #42862 from panbingkun/SPARK-45110. Authored-by: panbingkun Signed-off-by: Dongjoon Hyun (cherry picked from commit fa2bc21ba1e6cbde31f33faa681f5a1c47219c69) Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- pom.xml | 2 +- ...BasicOperationsBenchmark-jdk11-results.txt | 120 +++++++++--------- ...BasicOperationsBenchmark-jdk17-results.txt | 120 +++++++++--------- ...eStoreBasicOperationsBenchmark-results.txt | 120 +++++++++--------- 5 files changed, 182 insertions(+), 182 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 1d02f8dba567e..3d3f710e74cc4 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -227,7 +227,7 @@ parquet-jackson/1.13.1//parquet-jackson-1.13.1.jar pickle/1.3//pickle-1.3.jar py4j/0.10.9.7//py4j-0.10.9.7.jar remotetea-oncrpc/1.1.2//remotetea-oncrpc-1.1.2.jar -rocksdbjni/8.3.2//rocksdbjni-8.3.2.jar +rocksdbjni/8.5.3//rocksdbjni-8.5.3.jar scala-collection-compat_2.12/2.7.0//scala-collection-compat_2.12-2.7.0.jar scala-compiler/2.12.18//scala-compiler-2.12.18.jar scala-library/2.12.18//scala-library-2.12.18.jar diff --git a/pom.xml b/pom.xml index 8fc4b89a78cc2..70e1ee7156855 100644 --- a/pom.xml +++ b/pom.xml @@ -679,7 +679,7 @@ org.rocksdb rocksdbjni - 8.3.2 + 8.5.3 ${leveldbjni.group} diff --git a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk11-results.txt b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk11-results.txt index d5c175a320d3f..70e9849572c51 100644 --- a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk11-results.txt @@ -2,110 +2,110 @@ put rows ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz +OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz putting 10000 rows (10000 rows to overwrite - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -In-memory 9 11 2 1.1 872.7 1.0X -RocksDB (trackTotalNumberOfRows: true) 61 63 1 0.2 6148.5 0.1X -RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2108.9 0.4X +In-memory 8 9 1 1.3 770.7 1.0X +RocksDB (trackTotalNumberOfRows: true) 62 63 1 0.2 6174.3 0.1X +RocksDB (trackTotalNumberOfRows: false) 22 23 1 0.5 2220.7 0.3X -OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz +OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz putting 10000 rows (5000 rows to overwrite - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -In-memory 9 10 1 1.1 872.0 1.0X -RocksDB (trackTotalNumberOfRows: true) 51 53 1 0.2 5134.7 0.2X -RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2149.6 0.4X +In-memory 8 9 1 1.3 781.2 1.0X +RocksDB (trackTotalNumberOfRows: true) 52 53 1 0.2 5196.0 0.2X +RocksDB (trackTotalNumberOfRows: false) 22 24 1 0.4 2230.3 0.4X -OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz +OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz putting 10000 rows (1000 rows to overwrite - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 10 1 1.2 833.6 1.0X -RocksDB (trackTotalNumberOfRows: true) 41 43 1 0.2 4128.6 0.2X -RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2114.3 0.4X +In-memory 7 8 1 1.3 747.6 1.0X +RocksDB (trackTotalNumberOfRows: true) 42 44 1 0.2 4224.4 0.2X +RocksDB (trackTotalNumberOfRows: false) 22 23 1 0.4 2222.6 0.3X -OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz +OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz putting 10000 rows (0 rows to overwrite - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 9 1 1.2 812.9 1.0X -RocksDB (trackTotalNumberOfRows: true) 39 40 1 0.3 3855.8 0.2X -RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2111.9 0.4X +In-memory 7 8 1 1.3 740.9 1.0X +RocksDB (trackTotalNumberOfRows: true) 40 42 1 0.2 4019.6 0.2X +RocksDB (trackTotalNumberOfRows: false) 22 23 1 0.5 2201.9 0.3X ================================================================================================ delete rows ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz +OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz trying to delete 10000 rows from 10000 rows(10000 rows are non-existing - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 1 1 0 15.7 63.6 1.0X -RocksDB (trackTotalNumberOfRows: true) 39 41 0 0.3 3935.3 0.0X -RocksDB (trackTotalNumberOfRows: false) 22 22 0 0.5 2158.8 0.0X +In-memory 1 1 0 16.7 59.7 1.0X +RocksDB (trackTotalNumberOfRows: true) 40 41 1 0.3 3993.2 0.0X +RocksDB (trackTotalNumberOfRows: false) 22 23 0 0.5 2187.3 0.0X -OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz +OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz trying to delete 10000 rows from 10000 rows(5000 rows are non-existing - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 6 7 0 1.7 597.0 1.0X -RocksDB (trackTotalNumberOfRows: true) 51 53 0 0.2 5120.0 0.1X -RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2068.2 0.3X +In-memory 6 6 0 1.8 562.1 1.0X +RocksDB (trackTotalNumberOfRows: true) 50 52 1 0.2 5035.8 0.1X +RocksDB (trackTotalNumberOfRows: false) 22 22 1 0.5 2151.1 0.3X -OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz +OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz trying to delete 10000 rows from 10000 rows(1000 rows are non-existing - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 7 8 0 1.5 676.2 1.0X -RocksDB (trackTotalNumberOfRows: true) 60 62 1 0.2 6040.0 0.1X -RocksDB (trackTotalNumberOfRows: false) 21 21 0 0.5 2067.2 0.3X +In-memory 6 7 1 1.6 634.2 1.0X +RocksDB (trackTotalNumberOfRows: true) 59 60 1 0.2 5878.6 0.1X +RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2144.5 0.3X -OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz +OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz trying to delete 10000 rows from 10000 rows(0 rows are non-existing - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 7 8 0 1.5 684.0 1.0X -RocksDB (trackTotalNumberOfRows: true) 62 64 1 0.2 6208.6 0.1X -RocksDB (trackTotalNumberOfRows: false) 20 21 0 0.5 2030.6 0.3X +In-memory 6 7 0 1.6 631.1 1.0X +RocksDB (trackTotalNumberOfRows: true) 61 63 1 0.2 6103.6 0.1X +RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2112.1 0.3X ================================================================================================ evict rows ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz +OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz evicting 10000 rows (maxTimestampToEvictInMillis: 9999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 6 7 0 1.6 643.9 1.0X -RocksDB (trackTotalNumberOfRows: true) 65 66 1 0.2 6454.6 0.1X -RocksDB (trackTotalNumberOfRows: false) 24 25 0 0.4 2379.3 0.3X +In-memory 6 7 0 1.6 635.3 1.0X +RocksDB (trackTotalNumberOfRows: true) 64 65 1 0.2 6371.6 0.1X +RocksDB (trackTotalNumberOfRows: false) 23 24 1 0.4 2346.4 0.3X -OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz +OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz evicting 5000 rows (maxTimestampToEvictInMillis: 4999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------ -In-memory 6 6 0 1.8 568.7 1.0X -RocksDB (trackTotalNumberOfRows: true) 34 36 1 0.3 3383.3 0.2X -RocksDB (trackTotalNumberOfRows: false) 14 14 0 0.7 1390.8 0.4X +In-memory 6 6 0 1.8 570.1 1.0X +RocksDB (trackTotalNumberOfRows: true) 34 36 1 0.3 3410.5 0.2X +RocksDB (trackTotalNumberOfRows: false) 14 15 0 0.7 1412.9 0.4X -OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz +OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz evicting 1000 rows (maxTimestampToEvictInMillis: 999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 5 5 0 2.1 474.6 1.0X -RocksDB (trackTotalNumberOfRows: true) 11 11 0 0.9 1082.8 0.4X -RocksDB (trackTotalNumberOfRows: false) 7 7 0 1.4 693.5 0.7X +In-memory 5 5 0 2.1 483.9 1.0X +RocksDB (trackTotalNumberOfRows: true) 11 11 0 0.9 1092.6 0.4X +RocksDB (trackTotalNumberOfRows: false) 7 7 0 1.4 698.6 0.7X -OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz +OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz evicting 0 rows (maxTimestampToEvictInMillis: -1) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 1 1 0 14.8 67.4 1.0X -RocksDB (trackTotalNumberOfRows: true) 5 5 0 2.0 501.7 0.1X -RocksDB (trackTotalNumberOfRows: false) 5 5 0 2.0 502.3 0.1X +In-memory 1 1 0 15.7 63.9 1.0X +RocksDB (trackTotalNumberOfRows: true) 5 5 0 2.0 499.7 0.1X +RocksDB (trackTotalNumberOfRows: false) 5 5 0 2.0 498.3 0.1X diff --git a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk17-results.txt b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk17-results.txt index 5ec60f5beb90e..5424a55cd2012 100644 --- a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk17-results.txt +++ b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk17-results.txt @@ -2,110 +2,110 @@ put rows ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz putting 10000 rows (10000 rows to overwrite - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -In-memory 9 9 1 1.2 852.9 1.0X -RocksDB (trackTotalNumberOfRows: true) 60 62 2 0.2 6009.7 0.1X -RocksDB (trackTotalNumberOfRows: false) 21 23 1 0.5 2139.2 0.4X +In-memory 11 15 2 0.9 1067.0 1.0X +RocksDB (trackTotalNumberOfRows: true) 78 85 3 0.1 7772.1 0.1X +RocksDB (trackTotalNumberOfRows: false) 24 29 2 0.4 2384.6 0.4X -OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz putting 10000 rows (5000 rows to overwrite - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -In-memory 9 10 1 1.2 861.2 1.0X -RocksDB (trackTotalNumberOfRows: true) 50 52 1 0.2 5032.7 0.2X -RocksDB (trackTotalNumberOfRows: false) 21 23 0 0.5 2110.3 0.4X +In-memory 10 14 2 1.0 984.8 1.0X +RocksDB (trackTotalNumberOfRows: true) 62 70 3 0.2 6190.3 0.2X +RocksDB (trackTotalNumberOfRows: false) 25 29 2 0.4 2467.9 0.4X -OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz putting 10000 rows (1000 rows to overwrite - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 9 1 1.2 822.2 1.0X -RocksDB (trackTotalNumberOfRows: true) 40 42 1 0.2 4043.3 0.2X -RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2070.5 0.4X +In-memory 9 13 2 1.1 949.0 1.0X +RocksDB (trackTotalNumberOfRows: true) 48 56 3 0.2 4829.3 0.2X +RocksDB (trackTotalNumberOfRows: false) 24 28 2 0.4 2373.0 0.4X -OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz putting 10000 rows (0 rows to overwrite - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 9 1 1.2 802.6 1.0X -RocksDB (trackTotalNumberOfRows: true) 38 39 1 0.3 3773.1 0.2X -RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2050.8 0.4X +In-memory 10 13 2 1.0 992.6 1.0X +RocksDB (trackTotalNumberOfRows: true) 43 51 2 0.2 4331.3 0.2X +RocksDB (trackTotalNumberOfRows: false) 24 28 2 0.4 2372.8 0.4X ================================================================================================ delete rows ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz trying to delete 10000 rows from 10000 rows(10000 rows are non-existing - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 1 1 0 17.3 57.9 1.0X -RocksDB (trackTotalNumberOfRows: true) 39 40 0 0.3 3903.8 0.0X -RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2145.7 0.0X +In-memory 1 1 0 17.2 58.1 1.0X +RocksDB (trackTotalNumberOfRows: true) 43 50 3 0.2 4300.0 0.0X +RocksDB (trackTotalNumberOfRows: false) 23 27 2 0.4 2328.2 0.0X -OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz trying to delete 10000 rows from 10000 rows(5000 rows are non-existing - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 6 7 0 1.6 639.5 1.0X -RocksDB (trackTotalNumberOfRows: true) 50 51 1 0.2 4996.4 0.1X -RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2136.3 0.3X +In-memory 7 10 2 1.4 722.7 1.0X +RocksDB (trackTotalNumberOfRows: true) 62 68 3 0.2 6157.8 0.1X +RocksDB (trackTotalNumberOfRows: false) 23 27 2 0.4 2303.3 0.3X -OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz trying to delete 10000 rows from 10000 rows(1000 rows are non-existing - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 7 8 0 1.5 688.6 1.0X -RocksDB (trackTotalNumberOfRows: true) 58 59 1 0.2 5769.8 0.1X -RocksDB (trackTotalNumberOfRows: false) 21 22 1 0.5 2111.7 0.3X +In-memory 8 11 2 1.2 829.5 1.0X +RocksDB (trackTotalNumberOfRows: true) 71 80 3 0.1 7110.1 0.1X +RocksDB (trackTotalNumberOfRows: false) 23 26 2 0.4 2259.8 0.4X -OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz trying to delete 10000 rows from 10000 rows(0 rows are non-existing - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 7 8 0 1.4 706.4 1.0X -RocksDB (trackTotalNumberOfRows: true) 60 61 0 0.2 6012.9 0.1X -RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2135.2 0.3X +In-memory 8 11 2 1.2 806.5 1.0X +RocksDB (trackTotalNumberOfRows: true) 74 83 4 0.1 7354.0 0.1X +RocksDB (trackTotalNumberOfRows: false) 22 27 2 0.5 2197.1 0.4X ================================================================================================ evict rows ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz evicting 10000 rows (maxTimestampToEvictInMillis: 9999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 7 8 0 1.4 724.6 1.0X -RocksDB (trackTotalNumberOfRows: true) 61 62 1 0.2 6107.3 0.1X -RocksDB (trackTotalNumberOfRows: false) 23 24 0 0.4 2337.9 0.3X +In-memory 8 11 1 1.3 796.3 1.0X +RocksDB (trackTotalNumberOfRows: true) 77 88 3 0.1 7695.2 0.1X +RocksDB (trackTotalNumberOfRows: false) 23 27 2 0.4 2296.5 0.3X -OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz evicting 5000 rows (maxTimestampToEvictInMillis: 4999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------ -In-memory 7 7 0 1.5 657.7 1.0X -RocksDB (trackTotalNumberOfRows: true) 33 33 0 0.3 3266.8 0.2X -RocksDB (trackTotalNumberOfRows: false) 14 14 1 0.7 1366.6 0.5X +In-memory 7 10 1 1.4 740.5 1.0X +RocksDB (trackTotalNumberOfRows: true) 40 47 2 0.2 4010.7 0.2X +RocksDB (trackTotalNumberOfRows: false) 13 16 1 0.8 1301.1 0.6X -OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz evicting 1000 rows (maxTimestampToEvictInMillis: 999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 6 6 0 1.8 560.9 1.0X -RocksDB (trackTotalNumberOfRows: true) 10 11 0 1.0 1006.6 0.6X -RocksDB (trackTotalNumberOfRows: false) 6 6 0 1.6 630.5 0.9X +In-memory 6 8 1 1.6 624.9 1.0X +RocksDB (trackTotalNumberOfRows: true) 12 15 1 0.8 1228.5 0.5X +RocksDB (trackTotalNumberOfRows: false) 7 9 1 1.5 669.8 0.9X -OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz evicting 0 rows (maxTimestampToEvictInMillis: -1) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 1 1 0 15.3 65.5 1.0X -RocksDB (trackTotalNumberOfRows: true) 4 5 0 2.3 431.5 0.2X -RocksDB (trackTotalNumberOfRows: false) 4 5 0 2.3 431.9 0.2X +In-memory 1 1 0 15.8 63.1 1.0X +RocksDB (trackTotalNumberOfRows: true) 5 6 1 2.0 508.5 0.1X +RocksDB (trackTotalNumberOfRows: false) 4 6 1 2.2 448.4 0.1X diff --git a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-results.txt b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-results.txt index 7a9ef27a3884f..ae9f61457ca64 100644 --- a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-results.txt +++ b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-results.txt @@ -2,110 +2,110 @@ put rows ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz putting 10000 rows (10000 rows to overwrite - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -In-memory 7 9 1 1.4 720.9 1.0X -RocksDB (trackTotalNumberOfRows: true) 79 92 6 0.1 7934.4 0.1X -RocksDB (trackTotalNumberOfRows: false) 23 27 3 0.4 2263.3 0.3X +In-memory 8 8 1 1.3 759.2 1.0X +RocksDB (trackTotalNumberOfRows: true) 59 60 1 0.2 5873.6 0.1X +RocksDB (trackTotalNumberOfRows: false) 20 21 1 0.5 2008.5 0.4X -OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz putting 10000 rows (5000 rows to overwrite - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 12 2 1.2 831.9 1.0X -RocksDB (trackTotalNumberOfRows: true) 65 77 5 0.2 6476.8 0.1X -RocksDB (trackTotalNumberOfRows: false) 22 28 2 0.4 2235.4 0.4X +In-memory 8 8 0 1.3 782.8 1.0X +RocksDB (trackTotalNumberOfRows: true) 50 52 1 0.2 5004.2 0.2X +RocksDB (trackTotalNumberOfRows: false) 21 22 1 0.5 2079.1 0.4X -OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz putting 10000 rows (1000 rows to overwrite - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -In-memory 7 10 2 1.4 728.4 1.0X -RocksDB (trackTotalNumberOfRows: true) 47 57 4 0.2 4715.0 0.2X -RocksDB (trackTotalNumberOfRows: false) 22 27 2 0.5 2207.1 0.3X +In-memory 8 9 1 1.3 774.8 1.0X +RocksDB (trackTotalNumberOfRows: true) 41 42 1 0.2 4062.5 0.2X +RocksDB (trackTotalNumberOfRows: false) 21 21 0 0.5 2055.8 0.4X -OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz putting 10000 rows (0 rows to overwrite - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 10 1 1.3 750.4 1.0X -RocksDB (trackTotalNumberOfRows: true) 41 51 4 0.2 4116.2 0.2X -RocksDB (trackTotalNumberOfRows: false) 20 25 2 0.5 1962.6 0.4X +In-memory 7 8 0 1.4 719.3 1.0X +RocksDB (trackTotalNumberOfRows: true) 38 39 1 0.3 3830.2 0.2X +RocksDB (trackTotalNumberOfRows: false) 20 21 0 0.5 2021.5 0.4X ================================================================================================ delete rows ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz trying to delete 10000 rows from 10000 rows(10000 rows are non-existing - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 0 1 0 24.5 40.9 1.0X -RocksDB (trackTotalNumberOfRows: true) 42 50 4 0.2 4170.1 0.0X -RocksDB (trackTotalNumberOfRows: false) 19 24 2 0.5 1878.5 0.0X +In-memory 0 0 0 30.3 33.0 1.0X +RocksDB (trackTotalNumberOfRows: true) 39 40 0 0.3 3855.9 0.0X +RocksDB (trackTotalNumberOfRows: false) 20 21 1 0.5 2006.9 0.0X -OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz trying to delete 10000 rows from 10000 rows(5000 rows are non-existing - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 6 8 1 1.7 578.7 1.0X -RocksDB (trackTotalNumberOfRows: true) 57 68 5 0.2 5697.0 0.1X -RocksDB (trackTotalNumberOfRows: false) 19 24 2 0.5 1861.9 0.3X +In-memory 6 6 1 1.8 569.0 1.0X +RocksDB (trackTotalNumberOfRows: true) 48 49 1 0.2 4819.1 0.1X +RocksDB (trackTotalNumberOfRows: false) 20 21 0 0.5 1984.7 0.3X -OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz trying to delete 10000 rows from 10000 rows(1000 rows are non-existing - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 6 9 2 1.6 635.0 1.0X -RocksDB (trackTotalNumberOfRows: true) 66 80 5 0.2 6605.7 0.1X -RocksDB (trackTotalNumberOfRows: false) 19 24 2 0.5 1861.2 0.3X +In-memory 6 7 0 1.6 626.5 1.0X +RocksDB (trackTotalNumberOfRows: true) 57 59 1 0.2 5701.7 0.1X +RocksDB (trackTotalNumberOfRows: false) 20 21 1 0.5 2006.4 0.3X -OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz trying to delete 10000 rows from 10000 rows(0 rows are non-existing - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 7 9 1 1.5 651.4 1.0X -RocksDB (trackTotalNumberOfRows: true) 71 83 5 0.1 7108.8 0.1X -RocksDB (trackTotalNumberOfRows: false) 18 24 2 0.6 1812.3 0.4X +In-memory 6 7 1 1.5 648.5 1.0X +RocksDB (trackTotalNumberOfRows: true) 59 61 1 0.2 5927.9 0.1X +RocksDB (trackTotalNumberOfRows: false) 20 21 0 0.5 1984.2 0.3X ================================================================================================ evict rows ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz evicting 10000 rows (maxTimestampToEvictInMillis: 9999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 6 9 1 1.6 619.6 1.0X -RocksDB (trackTotalNumberOfRows: true) 74 86 5 0.1 7380.0 0.1X -RocksDB (trackTotalNumberOfRows: false) 22 27 2 0.4 2245.3 0.3X +In-memory 6 7 0 1.6 632.1 1.0X +RocksDB (trackTotalNumberOfRows: true) 59 61 1 0.2 5948.2 0.1X +RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2123.5 0.3X -OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz evicting 5000 rows (maxTimestampToEvictInMillis: 4999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------ -In-memory 6 8 1 1.7 579.2 1.0X -RocksDB (trackTotalNumberOfRows: true) 40 47 3 0.3 3985.4 0.1X -RocksDB (trackTotalNumberOfRows: false) 12 15 1 0.8 1214.1 0.5X +In-memory 6 6 0 1.8 555.2 1.0X +RocksDB (trackTotalNumberOfRows: true) 31 32 1 0.3 3136.7 0.2X +RocksDB (trackTotalNumberOfRows: false) 13 13 0 0.8 1252.6 0.4X -OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz evicting 1000 rows (maxTimestampToEvictInMillis: 999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 5 6 1 2.1 470.2 1.0X -RocksDB (trackTotalNumberOfRows: true) 11 13 1 0.9 1080.6 0.4X -RocksDB (trackTotalNumberOfRows: false) 6 7 1 1.8 560.1 0.8X +In-memory 5 5 0 2.2 463.6 1.0X +RocksDB (trackTotalNumberOfRows: true) 9 10 0 1.1 937.6 0.5X +RocksDB (trackTotalNumberOfRows: false) 6 6 0 1.8 562.7 0.8X -OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz evicting 0 rows (maxTimestampToEvictInMillis: -1) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 1 1 0 18.4 54.3 1.0X -RocksDB (trackTotalNumberOfRows: true) 4 4 1 2.8 352.7 0.2X -RocksDB (trackTotalNumberOfRows: false) 3 4 1 3.0 328.0 0.2X +In-memory 1 1 0 17.2 58.0 1.0X +RocksDB (trackTotalNumberOfRows: true) 4 4 0 2.8 359.0 0.2X +RocksDB (trackTotalNumberOfRows: false) 4 4 0 2.8 354.8 0.2X From ffa4127c774ea13b4d6bbcc82bc5a9bee23d7156 Mon Sep 17 00:00:00 2001 From: Giambattista Bloisi Date: Tue, 12 Sep 2023 16:16:04 +0200 Subject: [PATCH 006/521] [SPARK-45081][SQL] Encoders.bean does no longer work with read-only properties ### What changes were proposed in this pull request? This PR re-enables Encoders.bean to be called against beans having read-only properties, that is properties that have only getters and no setter method. Beans with read only properties are even used in internal tests. Setter methods of a Java bean encoder are stored within an Option wrapper because they are missing in case of read-only properties. When a java bean has to be initialized, setter methods for the bean properties have to be called: this PR filters out read-only properties from that process. ### Why are the changes needed? The changes are required to avoid an exception to the thrown by getting the value of a None option object. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? An additional regression test has been added ### Was this patch authored or co-authored using generative AI tooling? No Closes #42829 from gbloisi-openaire/SPARK-45081. Authored-by: Giambattista Bloisi Signed-off-by: Herman van Hovell (cherry picked from commit d8298bffd91de01299f9456b37e4454e8b4a6ae8) Signed-off-by: Herman van Hovell --- .../client/arrow/ArrowDeserializer.scala | 20 ++++++++++--------- .../catalyst/DeserializerBuildHelper.scala | 4 +++- .../apache/spark/sql/JavaDatasetSuite.java | 17 ++++++++++++++++ 3 files changed, 31 insertions(+), 10 deletions(-) diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala index cd54966ccf54d..9429578598712 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala @@ -332,15 +332,17 @@ object ArrowDeserializers { val constructor = methodLookup.findConstructor(tag.runtimeClass, MethodType.methodType(classOf[Unit])) val lookup = createFieldLookup(vectors) - val setters = fields.map { field => - val vector = lookup(field.name) - val deserializer = deserializerFor(field.enc, vector, timeZoneId) - val setter = methodLookup.findVirtual( - tag.runtimeClass, - field.writeMethod.get, - MethodType.methodType(classOf[Unit], field.enc.clsTag.runtimeClass)) - (bean: Any, i: Int) => setter.invoke(bean, deserializer.get(i)) - } + val setters = fields + .filter(_.writeMethod.isDefined) + .map { field => + val vector = lookup(field.name) + val deserializer = deserializerFor(field.enc, vector, timeZoneId) + val setter = methodLookup.findVirtual( + tag.runtimeClass, + field.writeMethod.get, + MethodType.methodType(classOf[Unit], field.enc.clsTag.runtimeClass)) + (bean: Any, i: Int) => setter.invoke(bean, deserializer.get(i)) + } new StructFieldSerializer[Any](struct) { def value(i: Int): Any = { val instance = constructor.invoke() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala index 16a7d7ff06526..0b88d5a4130e3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala @@ -390,7 +390,9 @@ object DeserializerBuildHelper { CreateExternalRow(convertedFields, enc.schema)) case JavaBeanEncoder(tag, fields) => - val setters = fields.map { f => + val setters = fields + .filter(_.writeMethod.isDefined) + .map { f => val newTypePath = walkedTypePath.recordField( f.enc.clsTag.runtimeClass.getName, f.name) diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java index 4f7cf8da78722..f416d411322ee 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java @@ -1783,6 +1783,23 @@ public void testEmptyBean() { Assert.assertEquals(1, df.collectAsList().size()); } + public static class ReadOnlyPropertyBean implements Serializable { + public boolean isEmpty() { + return true; + } + } + + @Test + public void testReadOnlyPropertyBean() { + ReadOnlyPropertyBean bean = new ReadOnlyPropertyBean(); + List data = Arrays.asList(bean); + Dataset df = spark.createDataset(data, + Encoders.bean(ReadOnlyPropertyBean.class)); + Assert.assertEquals(1, df.schema().length()); + Assert.assertEquals(1, df.collectAsList().size()); + + } + public class CircularReference1Bean implements Serializable { private CircularReference2Bean child; From 6a2284feaac4f632d645a93361d29e693eeb9d32 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 12 Sep 2023 08:49:40 -0700 Subject: [PATCH 007/521] Revert "[SPARK-45110][BUILD] Upgrade rocksdbjni to 8.5.3" This reverts commit 6a2aa1d48c304095dcdf2816a46ec1f5a8af41a2. --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- pom.xml | 2 +- ...BasicOperationsBenchmark-jdk11-results.txt | 120 +++++++++--------- ...BasicOperationsBenchmark-jdk17-results.txt | 120 +++++++++--------- ...eStoreBasicOperationsBenchmark-results.txt | 120 +++++++++--------- 5 files changed, 182 insertions(+), 182 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 3d3f710e74cc4..1d02f8dba567e 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -227,7 +227,7 @@ parquet-jackson/1.13.1//parquet-jackson-1.13.1.jar pickle/1.3//pickle-1.3.jar py4j/0.10.9.7//py4j-0.10.9.7.jar remotetea-oncrpc/1.1.2//remotetea-oncrpc-1.1.2.jar -rocksdbjni/8.5.3//rocksdbjni-8.5.3.jar +rocksdbjni/8.3.2//rocksdbjni-8.3.2.jar scala-collection-compat_2.12/2.7.0//scala-collection-compat_2.12-2.7.0.jar scala-compiler/2.12.18//scala-compiler-2.12.18.jar scala-library/2.12.18//scala-library-2.12.18.jar diff --git a/pom.xml b/pom.xml index 70e1ee7156855..8fc4b89a78cc2 100644 --- a/pom.xml +++ b/pom.xml @@ -679,7 +679,7 @@ org.rocksdb rocksdbjni - 8.5.3 + 8.3.2 ${leveldbjni.group} diff --git a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk11-results.txt b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk11-results.txt index 70e9849572c51..d5c175a320d3f 100644 --- a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk11-results.txt @@ -2,110 +2,110 @@ put rows ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz putting 10000 rows (10000 rows to overwrite - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 9 1 1.3 770.7 1.0X -RocksDB (trackTotalNumberOfRows: true) 62 63 1 0.2 6174.3 0.1X -RocksDB (trackTotalNumberOfRows: false) 22 23 1 0.5 2220.7 0.3X +In-memory 9 11 2 1.1 872.7 1.0X +RocksDB (trackTotalNumberOfRows: true) 61 63 1 0.2 6148.5 0.1X +RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2108.9 0.4X -OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz putting 10000 rows (5000 rows to overwrite - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 9 1 1.3 781.2 1.0X -RocksDB (trackTotalNumberOfRows: true) 52 53 1 0.2 5196.0 0.2X -RocksDB (trackTotalNumberOfRows: false) 22 24 1 0.4 2230.3 0.4X +In-memory 9 10 1 1.1 872.0 1.0X +RocksDB (trackTotalNumberOfRows: true) 51 53 1 0.2 5134.7 0.2X +RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2149.6 0.4X -OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz putting 10000 rows (1000 rows to overwrite - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -In-memory 7 8 1 1.3 747.6 1.0X -RocksDB (trackTotalNumberOfRows: true) 42 44 1 0.2 4224.4 0.2X -RocksDB (trackTotalNumberOfRows: false) 22 23 1 0.4 2222.6 0.3X +In-memory 8 10 1 1.2 833.6 1.0X +RocksDB (trackTotalNumberOfRows: true) 41 43 1 0.2 4128.6 0.2X +RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2114.3 0.4X -OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz putting 10000 rows (0 rows to overwrite - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -In-memory 7 8 1 1.3 740.9 1.0X -RocksDB (trackTotalNumberOfRows: true) 40 42 1 0.2 4019.6 0.2X -RocksDB (trackTotalNumberOfRows: false) 22 23 1 0.5 2201.9 0.3X +In-memory 8 9 1 1.2 812.9 1.0X +RocksDB (trackTotalNumberOfRows: true) 39 40 1 0.3 3855.8 0.2X +RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2111.9 0.4X ================================================================================================ delete rows ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz trying to delete 10000 rows from 10000 rows(10000 rows are non-existing - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 1 1 0 16.7 59.7 1.0X -RocksDB (trackTotalNumberOfRows: true) 40 41 1 0.3 3993.2 0.0X -RocksDB (trackTotalNumberOfRows: false) 22 23 0 0.5 2187.3 0.0X +In-memory 1 1 0 15.7 63.6 1.0X +RocksDB (trackTotalNumberOfRows: true) 39 41 0 0.3 3935.3 0.0X +RocksDB (trackTotalNumberOfRows: false) 22 22 0 0.5 2158.8 0.0X -OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz trying to delete 10000 rows from 10000 rows(5000 rows are non-existing - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 6 6 0 1.8 562.1 1.0X -RocksDB (trackTotalNumberOfRows: true) 50 52 1 0.2 5035.8 0.1X -RocksDB (trackTotalNumberOfRows: false) 22 22 1 0.5 2151.1 0.3X +In-memory 6 7 0 1.7 597.0 1.0X +RocksDB (trackTotalNumberOfRows: true) 51 53 0 0.2 5120.0 0.1X +RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2068.2 0.3X -OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz trying to delete 10000 rows from 10000 rows(1000 rows are non-existing - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 6 7 1 1.6 634.2 1.0X -RocksDB (trackTotalNumberOfRows: true) 59 60 1 0.2 5878.6 0.1X -RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2144.5 0.3X +In-memory 7 8 0 1.5 676.2 1.0X +RocksDB (trackTotalNumberOfRows: true) 60 62 1 0.2 6040.0 0.1X +RocksDB (trackTotalNumberOfRows: false) 21 21 0 0.5 2067.2 0.3X -OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz trying to delete 10000 rows from 10000 rows(0 rows are non-existing - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 6 7 0 1.6 631.1 1.0X -RocksDB (trackTotalNumberOfRows: true) 61 63 1 0.2 6103.6 0.1X -RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2112.1 0.3X +In-memory 7 8 0 1.5 684.0 1.0X +RocksDB (trackTotalNumberOfRows: true) 62 64 1 0.2 6208.6 0.1X +RocksDB (trackTotalNumberOfRows: false) 20 21 0 0.5 2030.6 0.3X ================================================================================================ evict rows ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz evicting 10000 rows (maxTimestampToEvictInMillis: 9999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 6 7 0 1.6 635.3 1.0X -RocksDB (trackTotalNumberOfRows: true) 64 65 1 0.2 6371.6 0.1X -RocksDB (trackTotalNumberOfRows: false) 23 24 1 0.4 2346.4 0.3X +In-memory 6 7 0 1.6 643.9 1.0X +RocksDB (trackTotalNumberOfRows: true) 65 66 1 0.2 6454.6 0.1X +RocksDB (trackTotalNumberOfRows: false) 24 25 0 0.4 2379.3 0.3X -OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz evicting 5000 rows (maxTimestampToEvictInMillis: 4999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------ -In-memory 6 6 0 1.8 570.1 1.0X -RocksDB (trackTotalNumberOfRows: true) 34 36 1 0.3 3410.5 0.2X -RocksDB (trackTotalNumberOfRows: false) 14 15 0 0.7 1412.9 0.4X +In-memory 6 6 0 1.8 568.7 1.0X +RocksDB (trackTotalNumberOfRows: true) 34 36 1 0.3 3383.3 0.2X +RocksDB (trackTotalNumberOfRows: false) 14 14 0 0.7 1390.8 0.4X -OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz evicting 1000 rows (maxTimestampToEvictInMillis: 999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 5 5 0 2.1 483.9 1.0X -RocksDB (trackTotalNumberOfRows: true) 11 11 0 0.9 1092.6 0.4X -RocksDB (trackTotalNumberOfRows: false) 7 7 0 1.4 698.6 0.7X +In-memory 5 5 0 2.1 474.6 1.0X +RocksDB (trackTotalNumberOfRows: true) 11 11 0 0.9 1082.8 0.4X +RocksDB (trackTotalNumberOfRows: false) 7 7 0 1.4 693.5 0.7X -OpenJDK 64-Bit Server VM 11.0.20.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 11.0.19+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8272CL CPU @ 2.60GHz evicting 0 rows (maxTimestampToEvictInMillis: -1) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 1 1 0 15.7 63.9 1.0X -RocksDB (trackTotalNumberOfRows: true) 5 5 0 2.0 499.7 0.1X -RocksDB (trackTotalNumberOfRows: false) 5 5 0 2.0 498.3 0.1X +In-memory 1 1 0 14.8 67.4 1.0X +RocksDB (trackTotalNumberOfRows: true) 5 5 0 2.0 501.7 0.1X +RocksDB (trackTotalNumberOfRows: false) 5 5 0 2.0 502.3 0.1X diff --git a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk17-results.txt b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk17-results.txt index 5424a55cd2012..5ec60f5beb90e 100644 --- a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk17-results.txt +++ b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-jdk17-results.txt @@ -2,110 +2,110 @@ put rows ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz putting 10000 rows (10000 rows to overwrite - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -In-memory 11 15 2 0.9 1067.0 1.0X -RocksDB (trackTotalNumberOfRows: true) 78 85 3 0.1 7772.1 0.1X -RocksDB (trackTotalNumberOfRows: false) 24 29 2 0.4 2384.6 0.4X +In-memory 9 9 1 1.2 852.9 1.0X +RocksDB (trackTotalNumberOfRows: true) 60 62 2 0.2 6009.7 0.1X +RocksDB (trackTotalNumberOfRows: false) 21 23 1 0.5 2139.2 0.4X -OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz putting 10000 rows (5000 rows to overwrite - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -In-memory 10 14 2 1.0 984.8 1.0X -RocksDB (trackTotalNumberOfRows: true) 62 70 3 0.2 6190.3 0.2X -RocksDB (trackTotalNumberOfRows: false) 25 29 2 0.4 2467.9 0.4X +In-memory 9 10 1 1.2 861.2 1.0X +RocksDB (trackTotalNumberOfRows: true) 50 52 1 0.2 5032.7 0.2X +RocksDB (trackTotalNumberOfRows: false) 21 23 0 0.5 2110.3 0.4X -OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz putting 10000 rows (1000 rows to overwrite - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -In-memory 9 13 2 1.1 949.0 1.0X -RocksDB (trackTotalNumberOfRows: true) 48 56 3 0.2 4829.3 0.2X -RocksDB (trackTotalNumberOfRows: false) 24 28 2 0.4 2373.0 0.4X +In-memory 8 9 1 1.2 822.2 1.0X +RocksDB (trackTotalNumberOfRows: true) 40 42 1 0.2 4043.3 0.2X +RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2070.5 0.4X -OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz putting 10000 rows (0 rows to overwrite - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -In-memory 10 13 2 1.0 992.6 1.0X -RocksDB (trackTotalNumberOfRows: true) 43 51 2 0.2 4331.3 0.2X -RocksDB (trackTotalNumberOfRows: false) 24 28 2 0.4 2372.8 0.4X +In-memory 8 9 1 1.2 802.6 1.0X +RocksDB (trackTotalNumberOfRows: true) 38 39 1 0.3 3773.1 0.2X +RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2050.8 0.4X ================================================================================================ delete rows ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz trying to delete 10000 rows from 10000 rows(10000 rows are non-existing - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 1 1 0 17.2 58.1 1.0X -RocksDB (trackTotalNumberOfRows: true) 43 50 3 0.2 4300.0 0.0X -RocksDB (trackTotalNumberOfRows: false) 23 27 2 0.4 2328.2 0.0X +In-memory 1 1 0 17.3 57.9 1.0X +RocksDB (trackTotalNumberOfRows: true) 39 40 0 0.3 3903.8 0.0X +RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2145.7 0.0X -OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz trying to delete 10000 rows from 10000 rows(5000 rows are non-existing - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 7 10 2 1.4 722.7 1.0X -RocksDB (trackTotalNumberOfRows: true) 62 68 3 0.2 6157.8 0.1X -RocksDB (trackTotalNumberOfRows: false) 23 27 2 0.4 2303.3 0.3X +In-memory 6 7 0 1.6 639.5 1.0X +RocksDB (trackTotalNumberOfRows: true) 50 51 1 0.2 4996.4 0.1X +RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2136.3 0.3X -OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz trying to delete 10000 rows from 10000 rows(1000 rows are non-existing - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 11 2 1.2 829.5 1.0X -RocksDB (trackTotalNumberOfRows: true) 71 80 3 0.1 7110.1 0.1X -RocksDB (trackTotalNumberOfRows: false) 23 26 2 0.4 2259.8 0.4X +In-memory 7 8 0 1.5 688.6 1.0X +RocksDB (trackTotalNumberOfRows: true) 58 59 1 0.2 5769.8 0.1X +RocksDB (trackTotalNumberOfRows: false) 21 22 1 0.5 2111.7 0.3X -OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz trying to delete 10000 rows from 10000 rows(0 rows are non-existing - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 11 2 1.2 806.5 1.0X -RocksDB (trackTotalNumberOfRows: true) 74 83 4 0.1 7354.0 0.1X -RocksDB (trackTotalNumberOfRows: false) 22 27 2 0.5 2197.1 0.4X +In-memory 7 8 0 1.4 706.4 1.0X +RocksDB (trackTotalNumberOfRows: true) 60 61 0 0.2 6012.9 0.1X +RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2135.2 0.3X ================================================================================================ evict rows ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz evicting 10000 rows (maxTimestampToEvictInMillis: 9999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 11 1 1.3 796.3 1.0X -RocksDB (trackTotalNumberOfRows: true) 77 88 3 0.1 7695.2 0.1X -RocksDB (trackTotalNumberOfRows: false) 23 27 2 0.4 2296.5 0.3X +In-memory 7 8 0 1.4 724.6 1.0X +RocksDB (trackTotalNumberOfRows: true) 61 62 1 0.2 6107.3 0.1X +RocksDB (trackTotalNumberOfRows: false) 23 24 0 0.4 2337.9 0.3X -OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz evicting 5000 rows (maxTimestampToEvictInMillis: 4999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------ -In-memory 7 10 1 1.4 740.5 1.0X -RocksDB (trackTotalNumberOfRows: true) 40 47 2 0.2 4010.7 0.2X -RocksDB (trackTotalNumberOfRows: false) 13 16 1 0.8 1301.1 0.6X +In-memory 7 7 0 1.5 657.7 1.0X +RocksDB (trackTotalNumberOfRows: true) 33 33 0 0.3 3266.8 0.2X +RocksDB (trackTotalNumberOfRows: false) 14 14 1 0.7 1366.6 0.5X -OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz evicting 1000 rows (maxTimestampToEvictInMillis: 999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 6 8 1 1.6 624.9 1.0X -RocksDB (trackTotalNumberOfRows: true) 12 15 1 0.8 1228.5 0.5X -RocksDB (trackTotalNumberOfRows: false) 7 9 1 1.5 669.8 0.9X +In-memory 6 6 0 1.8 560.9 1.0X +RocksDB (trackTotalNumberOfRows: true) 10 11 0 1.0 1006.6 0.6X +RocksDB (trackTotalNumberOfRows: false) 6 6 0 1.6 630.5 0.9X -OpenJDK 64-Bit Server VM 17.0.8.1+1 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz +OpenJDK 64-Bit Server VM 17.0.7+7 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz evicting 0 rows (maxTimestampToEvictInMillis: -1) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 1 1 0 15.8 63.1 1.0X -RocksDB (trackTotalNumberOfRows: true) 5 6 1 2.0 508.5 0.1X -RocksDB (trackTotalNumberOfRows: false) 4 6 1 2.2 448.4 0.1X +In-memory 1 1 0 15.3 65.5 1.0X +RocksDB (trackTotalNumberOfRows: true) 4 5 0 2.3 431.5 0.2X +RocksDB (trackTotalNumberOfRows: false) 4 5 0 2.3 431.9 0.2X diff --git a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-results.txt b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-results.txt index ae9f61457ca64..7a9ef27a3884f 100644 --- a/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-results.txt +++ b/sql/core/benchmarks/StateStoreBasicOperationsBenchmark-results.txt @@ -2,110 +2,110 @@ put rows ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz putting 10000 rows (10000 rows to overwrite - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 8 1 1.3 759.2 1.0X -RocksDB (trackTotalNumberOfRows: true) 59 60 1 0.2 5873.6 0.1X -RocksDB (trackTotalNumberOfRows: false) 20 21 1 0.5 2008.5 0.4X +In-memory 7 9 1 1.4 720.9 1.0X +RocksDB (trackTotalNumberOfRows: true) 79 92 6 0.1 7934.4 0.1X +RocksDB (trackTotalNumberOfRows: false) 23 27 3 0.4 2263.3 0.3X -OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz putting 10000 rows (5000 rows to overwrite - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 8 0 1.3 782.8 1.0X -RocksDB (trackTotalNumberOfRows: true) 50 52 1 0.2 5004.2 0.2X -RocksDB (trackTotalNumberOfRows: false) 21 22 1 0.5 2079.1 0.4X +In-memory 8 12 2 1.2 831.9 1.0X +RocksDB (trackTotalNumberOfRows: true) 65 77 5 0.2 6476.8 0.1X +RocksDB (trackTotalNumberOfRows: false) 22 28 2 0.4 2235.4 0.4X -OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz putting 10000 rows (1000 rows to overwrite - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------- -In-memory 8 9 1 1.3 774.8 1.0X -RocksDB (trackTotalNumberOfRows: true) 41 42 1 0.2 4062.5 0.2X -RocksDB (trackTotalNumberOfRows: false) 21 21 0 0.5 2055.8 0.4X +In-memory 7 10 2 1.4 728.4 1.0X +RocksDB (trackTotalNumberOfRows: true) 47 57 4 0.2 4715.0 0.2X +RocksDB (trackTotalNumberOfRows: false) 22 27 2 0.5 2207.1 0.3X -OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz putting 10000 rows (0 rows to overwrite - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative --------------------------------------------------------------------------------------------------------------------------------- -In-memory 7 8 0 1.4 719.3 1.0X -RocksDB (trackTotalNumberOfRows: true) 38 39 1 0.3 3830.2 0.2X -RocksDB (trackTotalNumberOfRows: false) 20 21 0 0.5 2021.5 0.4X +In-memory 8 10 1 1.3 750.4 1.0X +RocksDB (trackTotalNumberOfRows: true) 41 51 4 0.2 4116.2 0.2X +RocksDB (trackTotalNumberOfRows: false) 20 25 2 0.5 1962.6 0.4X ================================================================================================ delete rows ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz trying to delete 10000 rows from 10000 rows(10000 rows are non-existing - rate 100): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 0 0 0 30.3 33.0 1.0X -RocksDB (trackTotalNumberOfRows: true) 39 40 0 0.3 3855.9 0.0X -RocksDB (trackTotalNumberOfRows: false) 20 21 1 0.5 2006.9 0.0X +In-memory 0 1 0 24.5 40.9 1.0X +RocksDB (trackTotalNumberOfRows: true) 42 50 4 0.2 4170.1 0.0X +RocksDB (trackTotalNumberOfRows: false) 19 24 2 0.5 1878.5 0.0X -OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz trying to delete 10000 rows from 10000 rows(5000 rows are non-existing - rate 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 6 6 1 1.8 569.0 1.0X -RocksDB (trackTotalNumberOfRows: true) 48 49 1 0.2 4819.1 0.1X -RocksDB (trackTotalNumberOfRows: false) 20 21 0 0.5 1984.7 0.3X +In-memory 6 8 1 1.7 578.7 1.0X +RocksDB (trackTotalNumberOfRows: true) 57 68 5 0.2 5697.0 0.1X +RocksDB (trackTotalNumberOfRows: false) 19 24 2 0.5 1861.9 0.3X -OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz trying to delete 10000 rows from 10000 rows(1000 rows are non-existing - rate 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 6 7 0 1.6 626.5 1.0X -RocksDB (trackTotalNumberOfRows: true) 57 59 1 0.2 5701.7 0.1X -RocksDB (trackTotalNumberOfRows: false) 20 21 1 0.5 2006.4 0.3X +In-memory 6 9 2 1.6 635.0 1.0X +RocksDB (trackTotalNumberOfRows: true) 66 80 5 0.2 6605.7 0.1X +RocksDB (trackTotalNumberOfRows: false) 19 24 2 0.5 1861.2 0.3X -OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz trying to delete 10000 rows from 10000 rows(0 rows are non-existing - rate 0): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 6 7 1 1.5 648.5 1.0X -RocksDB (trackTotalNumberOfRows: true) 59 61 1 0.2 5927.9 0.1X -RocksDB (trackTotalNumberOfRows: false) 20 21 0 0.5 1984.2 0.3X +In-memory 7 9 1 1.5 651.4 1.0X +RocksDB (trackTotalNumberOfRows: true) 71 83 5 0.1 7108.8 0.1X +RocksDB (trackTotalNumberOfRows: false) 18 24 2 0.6 1812.3 0.4X ================================================================================================ evict rows ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz evicting 10000 rows (maxTimestampToEvictInMillis: 9999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 6 7 0 1.6 632.1 1.0X -RocksDB (trackTotalNumberOfRows: true) 59 61 1 0.2 5948.2 0.1X -RocksDB (trackTotalNumberOfRows: false) 21 22 0 0.5 2123.5 0.3X +In-memory 6 9 1 1.6 619.6 1.0X +RocksDB (trackTotalNumberOfRows: true) 74 86 5 0.1 7380.0 0.1X +RocksDB (trackTotalNumberOfRows: false) 22 27 2 0.4 2245.3 0.3X -OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz evicting 5000 rows (maxTimestampToEvictInMillis: 4999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------------ -In-memory 6 6 0 1.8 555.2 1.0X -RocksDB (trackTotalNumberOfRows: true) 31 32 1 0.3 3136.7 0.2X -RocksDB (trackTotalNumberOfRows: false) 13 13 0 0.8 1252.6 0.4X +In-memory 6 8 1 1.7 579.2 1.0X +RocksDB (trackTotalNumberOfRows: true) 40 47 3 0.3 3985.4 0.1X +RocksDB (trackTotalNumberOfRows: false) 12 15 1 0.8 1214.1 0.5X -OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz evicting 1000 rows (maxTimestampToEvictInMillis: 999) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ----------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 5 5 0 2.2 463.6 1.0X -RocksDB (trackTotalNumberOfRows: true) 9 10 0 1.1 937.6 0.5X -RocksDB (trackTotalNumberOfRows: false) 6 6 0 1.8 562.7 0.8X +In-memory 5 6 1 2.1 470.2 1.0X +RocksDB (trackTotalNumberOfRows: true) 11 13 1 0.9 1080.6 0.4X +RocksDB (trackTotalNumberOfRows: false) 6 7 1 1.8 560.1 0.8X -OpenJDK 64-Bit Server VM 1.8.0_382-b05 on Linux 5.15.0-1045-azure -Intel(R) Xeon(R) Platinum 8370C CPU @ 2.80GHz +OpenJDK 64-Bit Server VM 1.8.0_372-b07 on Linux 5.15.0-1040-azure +Intel(R) Xeon(R) CPU E5-2673 v4 @ 2.30GHz evicting 0 rows (maxTimestampToEvictInMillis: -1) from 10000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------------------------------- -In-memory 1 1 0 17.2 58.0 1.0X -RocksDB (trackTotalNumberOfRows: true) 4 4 0 2.8 359.0 0.2X -RocksDB (trackTotalNumberOfRows: false) 4 4 0 2.8 354.8 0.2X +In-memory 1 1 0 18.4 54.3 1.0X +RocksDB (trackTotalNumberOfRows: true) 4 4 1 2.8 352.7 0.2X +RocksDB (trackTotalNumberOfRows: false) 3 4 1 3.0 328.0 0.2X From 4e44d929005ac457fc853b256c02fd93f35fcceb Mon Sep 17 00:00:00 2001 From: Supun Nakandala Date: Tue, 12 Sep 2023 23:52:22 +0800 Subject: [PATCH 008/521] [SPARK-45117][SQL] Implement missing otherCopyArgs for the MultiCommutativeOp expression ### What changes were proposed in this pull request? - This PR implements the missing otherCopyArgs in the MultiCommutativeOp expression ### Why are the changes needed? - Without this method implementation, calling toJSON will throw an exception from the TreeNode::jsonFields method. - This is because the jsonFields method has an assertion that the number of fields defined in the constructor is equal to the number of field values (productIterator.toSeq ++ otherCopyArgs). - The originalRoot field of the MultiCommutativeOp is not part of the productIterator. Hence, it has to be explicitly set in the otherCopyArgs field. ### Does this PR introduce _any_ user-facing change? - No ### How was this patch tested? - Added unit test ### Was this patch authored or co-authored using generative AI tooling? - No Closes #42873 from db-scnakandala/multi-commutative-op. Authored-by: Supun Nakandala Signed-off-by: Wenchen Fan (cherry picked from commit d999f622dc68b4fb2734e2ac7cbe203b062c257f) Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/expressions/Expression.scala | 2 ++ .../catalyst/expressions/CanonicalizeSuite.scala | 13 +++++++++++++ 2 files changed, 15 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index c2330cdb59dbc..bd7369e57b057 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -1410,4 +1410,6 @@ case class MultiCommutativeOp( override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression = this.copy(operands = newChildren)(originalRoot) + + override protected final def otherCopyArgs: Seq[AnyRef] = originalRoot :: Nil } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala index 0e22b0d2876d7..89175ea1970cc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala @@ -338,4 +338,17 @@ class CanonicalizeSuite extends SparkFunSuite { SQLConf.get.setConfString(MULTI_COMMUTATIVE_OP_OPT_THRESHOLD.key, default.toString) } + + test("toJSON works properly with MultiCommutativeOp") { + val default = SQLConf.get.getConf(MULTI_COMMUTATIVE_OP_OPT_THRESHOLD) + SQLConf.get.setConfString(MULTI_COMMUTATIVE_OP_OPT_THRESHOLD.key, "1") + + val d = Decimal(1.2) + val literal1 = Literal.create(d, DecimalType(2, 1)) + val literal2 = Literal.create(d, DecimalType(2, 1)) + val literal3 = Literal.create(d, DecimalType(3, 2)) + val op = Add(literal1, Add(literal2, literal3)) + assert(op.canonicalized.toJSON.nonEmpty) + SQLConf.get.setConfString(MULTI_COMMUTATIVE_OP_OPT_THRESHOLD.key, default.toString) + } } From af8c0b999be746b661efe2439ac015a0c7d12c00 Mon Sep 17 00:00:00 2001 From: Juliusz Sompolski Date: Tue, 12 Sep 2023 16:48:26 +0200 Subject: [PATCH 009/521] [SPARK-44872][CONNECT] Server testing infra and ReattachableExecuteSuite ### What changes were proposed in this pull request? Add `SparkConnectServerTest` with infra to test real server with real client in the same process, but communicating over RPC. Add `ReattachableExecuteSuite` with some tests for reattachable execute. Two bugs were found by the tests: * Fix bug in `SparkConnectExecutionManager.createExecuteHolder` when attempting to resubmit an operation that was deemed abandoned. This bug is benign in reattachable execute, because reattachable execute would first send a ReattachExecute, which would be handled correctly in SparkConnectReattachExecuteHandler. For non-reattachable execute (disabled or old client), this is also a very unlikely scenario, because the retrying mechanism should be able to resubmit before the query is declared abandoned, and hence get an INVALID_HANDLE.OPERATION_ALREADY_EXISTS. This bug can manifest only if a non-reattachable execution is retried with so much delay that the operation was declared abandoned. * In `ExecuteGrpcResponseSender` there was an assertion that assumed that if `sendResponse` did not send, it was because deadline was reached. But it can also be because of interrupt. This would have resulted in interrupt returning an assertion error instead of CURSOR_DISCONNECTED in testing. Outside of testing assertions are not enabled, so this was not a problem outside of testing. ### Why are the changes needed? Testing of reattachable execute. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Tests added. Closes #42560 from juliuszsompolski/sc-reattachable-tests. Authored-by: Juliusz Sompolski Signed-off-by: Herman van Hovell (cherry picked from commit 4b96add471d292ed5c63ccc625489ff78cfb9b25) Signed-off-by: Herman van Hovell --- .../connect/client/CloseableIterator.scala | 22 +- .../CustomSparkConnectBlockingStub.scala | 2 +- ...cutePlanResponseReattachableIterator.scala | 18 +- .../client/GrpcExceptionConverter.scala | 5 +- .../sql/connect/client/GrpcRetryHandler.scala | 4 +- .../execution/ExecuteGrpcResponseSender.scala | 17 +- .../execution/ExecuteResponseObserver.scala | 8 +- .../sql/connect/service/ExecuteHolder.scala | 10 + .../SparkConnectExecutionManager.scala | 40 +- .../sql/connect/SparkConnectServerTest.scala | 261 +++++++++++++ .../execution/ReattachableExecuteSuite.scala | 352 ++++++++++++++++++ .../org/apache/spark/SparkFunSuite.scala | 24 ++ 12 files changed, 735 insertions(+), 28 deletions(-) create mode 100644 connector/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala create mode 100644 connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala index 891e50ed6e7bd..d3fc9963edc7a 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala @@ -27,6 +27,20 @@ private[sql] trait CloseableIterator[E] extends Iterator[E] with AutoCloseable { } } +private[sql] abstract class WrappedCloseableIterator[E] extends CloseableIterator[E] { + + def innerIterator: Iterator[E] + + override def next(): E = innerIterator.next() + + override def hasNext(): Boolean = innerIterator.hasNext + + override def close(): Unit = innerIterator match { + case it: CloseableIterator[E] => it.close() + case _ => // nothing + } +} + private[sql] object CloseableIterator { /** @@ -35,12 +49,8 @@ private[sql] object CloseableIterator { def apply[T](iterator: Iterator[T]): CloseableIterator[T] = iterator match { case closeable: CloseableIterator[T] => closeable case _ => - new CloseableIterator[T] { - override def next(): T = iterator.next() - - override def hasNext(): Boolean = iterator.hasNext - - override def close() = { /* empty */ } + new WrappedCloseableIterator[T] { + override def innerIterator = iterator } } } diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectBlockingStub.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectBlockingStub.scala index 73ff01e223f29..80edcfa8be16a 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectBlockingStub.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectBlockingStub.scala @@ -22,7 +22,7 @@ import io.grpc.ManagedChannel import org.apache.spark.connect.proto._ -private[client] class CustomSparkConnectBlockingStub( +private[connect] class CustomSparkConnectBlockingStub( channel: ManagedChannel, retryPolicy: GrpcRetryHandler.RetryPolicy) { diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala index 9bf7de33da8a7..57a629264be10 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.connect.client import java.util.UUID +import scala.collection.JavaConverters._ import scala.util.control.NonFatal import io.grpc.{ManagedChannel, StatusRuntimeException} @@ -50,7 +51,7 @@ class ExecutePlanResponseReattachableIterator( request: proto.ExecutePlanRequest, channel: ManagedChannel, retryPolicy: GrpcRetryHandler.RetryPolicy) - extends CloseableIterator[proto.ExecutePlanResponse] + extends WrappedCloseableIterator[proto.ExecutePlanResponse] with Logging { val operationId = if (request.hasOperationId) { @@ -86,14 +87,25 @@ class ExecutePlanResponseReattachableIterator( // True after ResultComplete message was seen in the stream. // Server will always send this message at the end of the stream, if the underlying iterator // finishes without producing one, another iterator needs to be reattached. - private var resultComplete: Boolean = false + // Visible for testing. + private[connect] var resultComplete: Boolean = false // Initial iterator comes from ExecutePlan request. // Note: This is not retried, because no error would ever be thrown here, and GRPC will only // throw error on first iter.hasNext() or iter.next() - private var iter: Option[java.util.Iterator[proto.ExecutePlanResponse]] = + // Visible for testing. + private[connect] var iter: Option[java.util.Iterator[proto.ExecutePlanResponse]] = Some(rawBlockingStub.executePlan(initialRequest)) + override def innerIterator: Iterator[proto.ExecutePlanResponse] = iter match { + case Some(it) => it.asScala + case None => + // The iterator is only unset for short moments while retry exception is thrown. + // It should only happen in the middle of internal processing. Since this iterator is not + // thread safe, no-one should be accessing it at this moment. + throw new IllegalStateException("innerIterator unset") + } + override def next(): proto.ExecutePlanResponse = synchronized { // hasNext will trigger reattach in case the stream completed without resultComplete if (!hasNext()) { diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala index c430485bd4184..fe9f6dc2b4a9a 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala @@ -43,7 +43,10 @@ private[client] object GrpcExceptionConverter extends JsonUtils { } def convertIterator[T](iter: CloseableIterator[T]): CloseableIterator[T] = { - new CloseableIterator[T] { + new WrappedCloseableIterator[T] { + + override def innerIterator: Iterator[T] = iter + override def hasNext: Boolean = { convert { iter.hasNext diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala index 8791530607c3a..3c0b750fd46e7 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala @@ -48,11 +48,13 @@ private[sql] class GrpcRetryHandler( * The type of the response. */ class RetryIterator[T, U](request: T, call: T => CloseableIterator[U]) - extends CloseableIterator[U] { + extends WrappedCloseableIterator[U] { private var opened = false // we only retry if it fails on first call when using the iterator private var iter = call(request) + override def innerIterator: Iterator[U] = iter + private def retryIter[V](f: Iterator[U] => V) = { if (!opened) { opened = true diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala index 6b8fcde1156ed..c3c33a85d6517 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala @@ -47,6 +47,9 @@ private[connect] class ExecuteGrpcResponseSender[T <: Message]( private var interrupted = false + // Time at which this sender should finish if the response stream is not finished by then. + private var deadlineTimeMillis = Long.MaxValue + // Signal to wake up when grpcCallObserver.isReady() private val grpcCallObserverReadySignal = new Object @@ -65,6 +68,12 @@ private[connect] class ExecuteGrpcResponseSender[T <: Message]( executionObserver.notifyAll() } + // For testing + private[connect] def setDeadline(deadlineMs: Long) = executionObserver.synchronized { + deadlineTimeMillis = deadlineMs + executionObserver.notifyAll() + } + def run(lastConsumedStreamIndex: Long): Unit = { if (executeHolder.reattachable) { // In reattachable execution we use setOnReadyHandler and grpcCallObserver.isReady to control @@ -150,7 +159,7 @@ private[connect] class ExecuteGrpcResponseSender[T <: Message]( var finished = false // Time at which this sender should finish if the response stream is not finished by then. - val deadlineTimeMillis = if (!executeHolder.reattachable) { + deadlineTimeMillis = if (!executeHolder.reattachable) { Long.MaxValue } else { val confSize = @@ -232,8 +241,8 @@ private[connect] class ExecuteGrpcResponseSender[T <: Message]( assert(finished == false) } else { // If it wasn't sent, time deadline must have been reached before stream became available, - // will exit in the enxt loop iterattion. - assert(deadlineLimitReached) + // or it was intterupted. Will exit in the next loop iterattion. + assert(deadlineLimitReached || interrupted) } } else if (streamFinished) { // Stream is finished and all responses have been sent @@ -301,7 +310,7 @@ private[connect] class ExecuteGrpcResponseSender[T <: Message]( val sleepStart = System.nanoTime() var sleepEnd = 0L // Conditions for exiting the inner loop - // 1. was detached + // 1. was interrupted // 2. grpcCallObserver is ready to send more data // 3. time deadline is reached while (!interrupted && diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala index d9db07fd228ed..df0fb3ac3a592 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala @@ -73,11 +73,16 @@ private[connect] class ExecuteResponseObserver[T <: Message](val executeHolder: /** The index of the last response produced by execution. */ private var lastProducedIndex: Long = 0 // first response will have index 1 + // For testing + private[connect] var releasedUntilIndex: Long = 0 + /** * Highest response index that was consumed. Keeps track of it to decide which responses needs * to be cached, and to assert that all responses are consumed. + * + * Visible for testing. */ - private var highestConsumedIndex: Long = 0 + private[connect] var highestConsumedIndex: Long = 0 /** * Consumer that waits for available responses. There can be only one at a time, @see @@ -284,6 +289,7 @@ private[connect] class ExecuteResponseObserver[T <: Message](val executeHolder: responses.remove(i) i -= 1 } + releasedUntilIndex = index } /** diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala index bce0713339228..974c13b08e318 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala @@ -183,6 +183,16 @@ private[connect] class ExecuteHolder( } } + // For testing. + private[connect] def setGrpcResponseSendersDeadline(deadlineMs: Long) = synchronized { + grpcResponseSenders.foreach(_.setDeadline(deadlineMs)) + } + + // For testing + private[connect] def interruptGrpcResponseSenders() = synchronized { + grpcResponseSenders.foreach(_.interrupt()) + } + /** * For a short period in ExecutePlan after creation and until runGrpcResponseSender is called, * there is no attached response sender, but yet we start with lastAttachedRpcTime = None, so we diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutionManager.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutionManager.scala index ce1f6c93f6cfe..21f59bdd68ea5 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutionManager.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutionManager.scala @@ -71,15 +71,14 @@ private[connect] class SparkConnectExecutionManager() extends Logging { // The latter is to prevent double execution when a client retries execution, thinking it // never reached the server, but in fact it did, and already got removed as abandoned. if (executions.get(executeHolder.key).isDefined) { - if (getAbandonedTombstone(executeHolder.key).isDefined) { - throw new SparkSQLException( - errorClass = "INVALID_HANDLE.OPERATION_ABANDONED", - messageParameters = Map("handle" -> executeHolder.operationId)) - } else { - throw new SparkSQLException( - errorClass = "INVALID_HANDLE.OPERATION_ALREADY_EXISTS", - messageParameters = Map("handle" -> executeHolder.operationId)) - } + throw new SparkSQLException( + errorClass = "INVALID_HANDLE.OPERATION_ALREADY_EXISTS", + messageParameters = Map("handle" -> executeHolder.operationId)) + } + if (getAbandonedTombstone(executeHolder.key).isDefined) { + throw new SparkSQLException( + errorClass = "INVALID_HANDLE.OPERATION_ABANDONED", + messageParameters = Map("handle" -> executeHolder.operationId)) } sessionHolder.addExecuteHolder(executeHolder) executions.put(executeHolder.key, executeHolder) @@ -141,12 +140,17 @@ private[connect] class SparkConnectExecutionManager() extends Logging { abandonedTombstones.asMap.asScala.values.toBuffer.toSeq } - private[service] def shutdown(): Unit = executionsLock.synchronized { + private[connect] def shutdown(): Unit = executionsLock.synchronized { scheduledExecutor.foreach { executor => executor.shutdown() executor.awaitTermination(1, TimeUnit.MINUTES) } scheduledExecutor = None + executions.clear() + abandonedTombstones.invalidateAll() + if (!lastExecutionTime.isDefined) { + lastExecutionTime = Some(System.currentTimeMillis()) + } } /** @@ -188,7 +192,7 @@ private[connect] class SparkConnectExecutionManager() extends Logging { executions.values.foreach { executeHolder => executeHolder.lastAttachedRpcTime match { case Some(detached) => - if (detached + timeout < nowMs) { + if (detached + timeout <= nowMs) { toRemove += executeHolder } case _ => // execution is active @@ -206,4 +210,18 @@ private[connect] class SparkConnectExecutionManager() extends Logging { } logInfo("Finished periodic run of SparkConnectExecutionManager maintenance.") } + + // For testing. + private[connect] def setAllRPCsDeadline(deadlineMs: Long) = executionsLock.synchronized { + executions.values.foreach(_.setGrpcResponseSendersDeadline(deadlineMs)) + } + + // For testing. + private[connect] def interruptAllRPCs() = executionsLock.synchronized { + executions.values.foreach(_.interruptGrpcResponseSenders()) + } + + private[connect] def listExecuteHolders = executionsLock.synchronized { + executions.values.toBuffer.toSeq + } } diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala new file mode 100644 index 0000000000000..488858d33ea12 --- /dev/null +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala @@ -0,0 +1,261 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connect + +import java.util.UUID + +import org.scalatest.concurrent.{Eventually, TimeLimits} +import org.scalatest.time.Span +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.connect.proto +import org.apache.spark.sql.connect.client.{CloseableIterator, CustomSparkConnectBlockingStub, ExecutePlanResponseReattachableIterator, GrpcRetryHandler, SparkConnectClient, WrappedCloseableIterator} +import org.apache.spark.sql.connect.common.config.ConnectCommon +import org.apache.spark.sql.connect.config.Connect +import org.apache.spark.sql.connect.dsl.MockRemoteSession +import org.apache.spark.sql.connect.dsl.plans._ +import org.apache.spark.sql.connect.service.{ExecuteHolder, SparkConnectService} +import org.apache.spark.sql.test.SharedSparkSession + +/** + * Base class and utilities for a test suite that starts and tests the real SparkConnectService + * with a real SparkConnectClient, communicating over RPC, but both in-process. + */ +class SparkConnectServerTest extends SharedSparkSession { + + // Server port + val serverPort: Int = + ConnectCommon.CONNECT_GRPC_BINDING_PORT + util.Random.nextInt(1000) + + val eventuallyTimeout = 30.seconds + + override def beforeAll(): Unit = { + super.beforeAll() + // Other suites using mocks leave a mess in the global executionManager, + // shut it down so that it's cleared before starting server. + SparkConnectService.executionManager.shutdown() + // Start the real service. + withSparkEnvConfs((Connect.CONNECT_GRPC_BINDING_PORT.key, serverPort.toString)) { + SparkConnectService.start(spark.sparkContext) + } + // register udf directly on the server, we're not testing client UDFs here... + val serverSession = + SparkConnectService.getOrCreateIsolatedSession(defaultUserId, defaultSessionId).session + serverSession.udf.register("sleep", ((ms: Int) => { Thread.sleep(ms); ms })) + } + + override def afterAll(): Unit = { + SparkConnectService.stop() + super.afterAll() + } + + override def beforeEach(): Unit = { + super.beforeEach() + clearAllExecutions() + } + + override def afterEach(): Unit = { + clearAllExecutions() + super.afterEach() + } + + protected def clearAllExecutions(): Unit = { + SparkConnectService.executionManager.listExecuteHolders.foreach(_.close()) + SparkConnectService.executionManager.periodicMaintenance(0) + assertNoActiveExecutions() + } + + protected val defaultSessionId = UUID.randomUUID.toString() + protected val defaultUserId = UUID.randomUUID.toString() + + // We don't have the real SparkSession/Dataset api available, + // so use mock for generating simple query plans. + protected val dsl = new MockRemoteSession() + + protected val userContext = proto.UserContext + .newBuilder() + .setUserId(defaultUserId) + .build() + + protected def buildExecutePlanRequest( + plan: proto.Plan, + sessionId: String = defaultSessionId, + operationId: String = UUID.randomUUID.toString) = { + proto.ExecutePlanRequest + .newBuilder() + .setUserContext(userContext) + .setSessionId(sessionId) + .setOperationId(operationId) + .setPlan(plan) + .addRequestOptions( + proto.ExecutePlanRequest.RequestOption + .newBuilder() + .setReattachOptions(proto.ReattachOptions.newBuilder().setReattachable(true).build()) + .build()) + .build() + } + + protected def buildReattachExecuteRequest(operationId: String, responseId: Option[String]) = { + val req = proto.ReattachExecuteRequest + .newBuilder() + .setUserContext(userContext) + .setSessionId(defaultSessionId) + .setOperationId(operationId) + + if (responseId.isDefined) { + req.setLastResponseId(responseId.get) + } + + req.build() + } + + protected def buildPlan(query: String) = { + proto.Plan.newBuilder().setRoot(dsl.sql(query)).build() + } + + protected def getReattachableIterator( + stubIterator: CloseableIterator[proto.ExecutePlanResponse]) = { + // This depends on the wrapping in CustomSparkConnectBlockingStub.executePlanReattachable: + // GrpcExceptionConverter.convertIterator + stubIterator + .asInstanceOf[WrappedCloseableIterator[proto.ExecutePlanResponse]] + // ExecutePlanResponseReattachableIterator + .innerIterator + .asInstanceOf[ExecutePlanResponseReattachableIterator] + } + + protected def assertNoActiveRpcs(): Unit = { + SparkConnectService.executionManager.listActiveExecutions match { + case Left(_) => // nothing running, good + case Right(executions) => + // all rpc detached. + assert( + executions.forall(_.lastAttachedRpcTime.isDefined), + s"Expected no RPCs, but got $executions") + } + } + + protected def assertEventuallyNoActiveRpcs(): Unit = { + Eventually.eventually(timeout(eventuallyTimeout)) { + assertNoActiveRpcs() + } + } + + protected def assertNoActiveExecutions(): Unit = { + SparkConnectService.executionManager.listActiveExecutions match { + case Left(_) => // cleaned up + case Right(executions) => fail(s"Expected empty, but got $executions") + } + } + + protected def assertEventuallyNoActiveExecutions(): Unit = { + Eventually.eventually(timeout(eventuallyTimeout)) { + assertNoActiveExecutions() + } + } + + protected def assertExecutionReleased(operationId: String): Unit = { + SparkConnectService.executionManager.listActiveExecutions match { + case Left(_) => // cleaned up + case Right(executions) => assert(!executions.exists(_.operationId == operationId)) + } + } + + protected def assertEventuallyExecutionReleased(operationId: String): Unit = { + Eventually.eventually(timeout(eventuallyTimeout)) { + assertExecutionReleased(operationId) + } + } + + // Get ExecutionHolder, assuming that only one execution is active + protected def getExecutionHolder: ExecuteHolder = { + val executions = SparkConnectService.executionManager.listExecuteHolders + assert(executions.length == 1) + executions.head + } + + protected def withClient(f: SparkConnectClient => Unit): Unit = { + val client = SparkConnectClient + .builder() + .port(serverPort) + .sessionId(defaultSessionId) + .userId(defaultUserId) + .enableReattachableExecute() + .build() + try f(client) + finally { + client.shutdown() + } + } + + protected def withRawBlockingStub( + f: proto.SparkConnectServiceGrpc.SparkConnectServiceBlockingStub => Unit): Unit = { + val conf = SparkConnectClient.Configuration(port = serverPort) + val channel = conf.createChannel() + val bstub = proto.SparkConnectServiceGrpc.newBlockingStub(channel) + try f(bstub) + finally { + channel.shutdownNow() + } + } + + protected def withCustomBlockingStub( + retryPolicy: GrpcRetryHandler.RetryPolicy = GrpcRetryHandler.RetryPolicy())( + f: CustomSparkConnectBlockingStub => Unit): Unit = { + val conf = SparkConnectClient.Configuration(port = serverPort) + val channel = conf.createChannel() + val bstub = new CustomSparkConnectBlockingStub(channel, retryPolicy) + try f(bstub) + finally { + channel.shutdownNow() + } + } + + protected def runQuery(plan: proto.Plan, queryTimeout: Span, iterSleep: Long): Unit = { + withClient { client => + TimeLimits.failAfter(queryTimeout) { + val iter = client.execute(plan) + var operationId: Option[String] = None + var r: proto.ExecutePlanResponse = null + val reattachableIter = getReattachableIterator(iter) + while (iter.hasNext) { + r = iter.next() + operationId match { + case None => operationId = Some(r.getOperationId) + case Some(id) => assert(r.getOperationId == id) + } + if (iterSleep > 0) { + Thread.sleep(iterSleep) + } + } + // Check that last response had ResultComplete indicator + assert(r != null) + assert(r.hasResultComplete) + // ... that client sent ReleaseExecute based on it + assert(reattachableIter.resultComplete) + // ... and that the server released the execution. + assert(operationId.isDefined) + assertEventuallyExecutionReleased(operationId.get) + } + } + } + + protected def runQuery(query: String, queryTimeout: Span, iterSleep: Long = 0): Unit = { + val plan = buildPlan(query) + runQuery(plan, queryTimeout, iterSleep) + } +} diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala new file mode 100644 index 0000000000000..169b15582b698 --- /dev/null +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala @@ -0,0 +1,352 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connect.execution + +import java.util.UUID + +import io.grpc.StatusRuntimeException +import org.scalatest.concurrent.Eventually +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.SparkException +import org.apache.spark.sql.connect.SparkConnectServerTest +import org.apache.spark.sql.connect.config.Connect +import org.apache.spark.sql.connect.service.SparkConnectService + +class ReattachableExecuteSuite extends SparkConnectServerTest { + + // Tests assume that this query will result in at least a couple ExecutePlanResponses on the + // stream. If this is no longer the case because of changes in how much is returned in a single + // ExecutePlanResponse, it may need to be adjusted. + val MEDIUM_RESULTS_QUERY = "select * from range(1000000)" + + test("reattach after initial RPC ends") { + withClient { client => + val iter = client.execute(buildPlan(MEDIUM_RESULTS_QUERY)) + val reattachableIter = getReattachableIterator(iter) + val initialInnerIter = reattachableIter.innerIterator + + // open the iterator + iter.next() + // expire all RPCs on server + SparkConnectService.executionManager.setAllRPCsDeadline(System.currentTimeMillis() - 1) + assertEventuallyNoActiveRpcs() + // iterator should reattach + // (but not necessarily at first next, as there might have been messages buffered client side) + while (iter.hasNext && (reattachableIter.innerIterator eq initialInnerIter)) { + iter.next() + } + assert( + reattachableIter.innerIterator ne initialInnerIter + ) // reattach changed the inner iter + } + } + + test("raw interrupted RPC results in INVALID_CURSOR.DISCONNECTED error") { + withRawBlockingStub { stub => + val iter = stub.executePlan(buildExecutePlanRequest(buildPlan(MEDIUM_RESULTS_QUERY))) + iter.next() // open the iterator + // interrupt all RPCs on server + SparkConnectService.executionManager.interruptAllRPCs() + assertEventuallyNoActiveRpcs() + val e = intercept[StatusRuntimeException] { + while (iter.hasNext) iter.next() + } + assert(e.getMessage.contains("INVALID_CURSOR.DISCONNECTED")) + } + } + + test("raw new RPC interrupts previous RPC with INVALID_CURSOR.DISCONNECTED error") { + // Raw stub does not have retries, auto reattach etc. + withRawBlockingStub { stub => + val operationId = UUID.randomUUID().toString + val iter = stub.executePlan( + buildExecutePlanRequest(buildPlan(MEDIUM_RESULTS_QUERY), operationId = operationId)) + iter.next() // open the iterator + + // send reattach + val iter2 = stub.reattachExecute(buildReattachExecuteRequest(operationId, None)) + iter2.next() // open the iterator + + // should result in INVALID_CURSOR.DISCONNECTED error on the original iterator + val e = intercept[StatusRuntimeException] { + while (iter.hasNext) iter.next() + } + assert(e.getMessage.contains("INVALID_CURSOR.DISCONNECTED")) + + // send another reattach + val iter3 = stub.reattachExecute(buildReattachExecuteRequest(operationId, None)) + assert(iter3.hasNext) + iter3.next() // open the iterator + + // should result in INVALID_CURSOR.DISCONNECTED error on the previous reattach iterator + val e2 = intercept[StatusRuntimeException] { + while (iter2.hasNext) iter2.next() + } + assert(e2.getMessage.contains("INVALID_CURSOR.DISCONNECTED")) + } + } + + test("client INVALID_CURSOR.DISCONNECTED error is retried when rpc sender gets interrupted") { + withClient { client => + val iter = client.execute(buildPlan(MEDIUM_RESULTS_QUERY)) + val reattachableIter = getReattachableIterator(iter) + val initialInnerIter = reattachableIter.innerIterator + val operationId = getReattachableIterator(iter).operationId + + // open the iterator + iter.next() + + // interrupt all RPCs on server + SparkConnectService.executionManager.interruptAllRPCs() + assertEventuallyNoActiveRpcs() + + // Nevertheless, the original iterator will handle the INVALID_CURSOR.DISCONNECTED error + iter.next() + // iterator changed because it had to reconnect + assert(reattachableIter.innerIterator ne initialInnerIter) + } + } + + test("client INVALID_CURSOR.DISCONNECTED error is retried when other RPC preempts this one") { + withClient { client => + val iter = client.execute(buildPlan(MEDIUM_RESULTS_QUERY)) + val reattachableIter = getReattachableIterator(iter) + val initialInnerIter = reattachableIter.innerIterator + val operationId = getReattachableIterator(iter).operationId + + // open the iterator + val response = iter.next() + + // Send another Reattach request, it should preempt this request with an + // INVALID_CURSOR.DISCONNECTED error. + withRawBlockingStub { stub => + val reattachIter = stub.reattachExecute( + buildReattachExecuteRequest(operationId, Some(response.getResponseId))) + assert(reattachIter.hasNext) + reattachIter.next() + + // Nevertheless, the original iterator will handle the INVALID_CURSOR.DISCONNECTED error + iter.next() + // iterator changed because it had to reconnect + assert(reattachableIter.innerIterator ne initialInnerIter) + } + } + } + + test("abandoned query gets INVALID_HANDLE.OPERATION_ABANDONED error") { + withClient { client => + val plan = buildPlan("select * from range(100000)") + val iter = client.execute(buildPlan(MEDIUM_RESULTS_QUERY)) + val operationId = getReattachableIterator(iter).operationId + // open the iterator + iter.next() + // disconnect and remove on server + SparkConnectService.executionManager.setAllRPCsDeadline(System.currentTimeMillis() - 1) + assertEventuallyNoActiveRpcs() + SparkConnectService.executionManager.periodicMaintenance(0) + assertNoActiveExecutions() + // check that it throws abandoned error + val e = intercept[SparkException] { + while (iter.hasNext) iter.next() + } + assert(e.getMessage.contains("INVALID_HANDLE.OPERATION_ABANDONED")) + // check that afterwards, new operation can't be created with the same operationId. + withCustomBlockingStub() { stub => + val executePlanReq = buildExecutePlanRequest(plan, operationId = operationId) + + val iterNonReattachable = stub.executePlan(executePlanReq) + val eNonReattachable = intercept[SparkException] { + iterNonReattachable.hasNext + } + assert(eNonReattachable.getMessage.contains("INVALID_HANDLE.OPERATION_ABANDONED")) + + val iterReattachable = stub.executePlanReattachable(executePlanReq) + val eReattachable = intercept[SparkException] { + iterReattachable.hasNext + } + assert(eReattachable.getMessage.contains("INVALID_HANDLE.OPERATION_ABANDONED")) + } + } + } + + test("client releases responses directly after consuming them") { + withClient { client => + val iter = client.execute(buildPlan(MEDIUM_RESULTS_QUERY)) + val reattachableIter = getReattachableIterator(iter) + val initialInnerIter = reattachableIter.innerIterator + val operationId = getReattachableIterator(iter).operationId + + assert(iter.hasNext) // open iterator + val execution = getExecutionHolder + assert(execution.responseObserver.releasedUntilIndex == 0) + + // get two responses, check on the server that ReleaseExecute releases them afterwards + val response1 = iter.next() + Eventually.eventually(timeout(eventuallyTimeout)) { + assert(execution.responseObserver.releasedUntilIndex == 1) + } + + val response2 = iter.next() + Eventually.eventually(timeout(eventuallyTimeout)) { + assert(execution.responseObserver.releasedUntilIndex == 2) + } + + withRawBlockingStub { stub => + // Reattach after response1 should fail with INVALID_CURSOR.POSITION_NOT_AVAILABLE + val reattach1 = stub.reattachExecute( + buildReattachExecuteRequest(operationId, Some(response1.getResponseId))) + val e = intercept[StatusRuntimeException] { + reattach1.hasNext() + } + assert(e.getMessage.contains("INVALID_CURSOR.POSITION_NOT_AVAILABLE")) + + // Reattach after response2 should work + val reattach2 = stub.reattachExecute( + buildReattachExecuteRequest(operationId, Some(response2.getResponseId))) + val response3 = reattach2.next() + val response4 = reattach2.next() + val response5 = reattach2.next() + + // The original client iterator will handle the INVALID_CURSOR.DISCONNECTED error, + // and reconnect back. Since the raw iterator was not releasing responses, client iterator + // should be able to continue where it left off (server shouldn't have released yet) + assert(execution.responseObserver.releasedUntilIndex == 2) + assert(iter.hasNext) + + val r3 = iter.next() + assert(r3.getResponseId == response3.getResponseId) + val r4 = iter.next() + assert(r4.getResponseId == response4.getResponseId) + val r5 = iter.next() + assert(r5.getResponseId == response5.getResponseId) + // inner iterator changed because it had to reconnect + assert(reattachableIter.innerIterator ne initialInnerIter) + } + } + } + + test("server releases responses automatically when client moves ahead") { + withRawBlockingStub { stub => + val operationId = UUID.randomUUID().toString + val iter = stub.executePlan( + buildExecutePlanRequest(buildPlan(MEDIUM_RESULTS_QUERY), operationId = operationId)) + var lastSeenResponse: String = null + + iter.hasNext // open iterator + val execution = getExecutionHolder + + // after consuming enough from the iterator, server should automatically start releasing + var lastSeenIndex = 0 + while (iter.hasNext && execution.responseObserver.releasedUntilIndex == 0) { + val r = iter.next() + lastSeenResponse = r.getResponseId() + lastSeenIndex += 1 + } + assert(iter.hasNext) + assert(execution.responseObserver.releasedUntilIndex > 0) + + // Reattach from the beginning is not available. + val reattach = stub.reattachExecute(buildReattachExecuteRequest(operationId, None)) + val e = intercept[StatusRuntimeException] { + reattach.hasNext() + } + assert(e.getMessage.contains("INVALID_CURSOR.POSITION_NOT_AVAILABLE")) + + // Original iterator got disconnected by the reattach and gets INVALID_CURSOR.DISCONNECTED + val e2 = intercept[StatusRuntimeException] { + while (iter.hasNext) iter.next() + } + assert(e2.getMessage.contains("INVALID_CURSOR.DISCONNECTED")) + + Eventually.eventually(timeout(eventuallyTimeout)) { + // Even though we didn't consume more from the iterator, the server thinks that + // it sent more, because GRPC stream onNext() can push into internal GRPC buffer without + // client picking it up. + assert(execution.responseObserver.highestConsumedIndex > lastSeenIndex) + } + // but CONNECT_EXECUTE_REATTACHABLE_OBSERVER_RETRY_BUFFER_SIZE is big enough that the last + // response we've seen is still in range + assert(execution.responseObserver.releasedUntilIndex < lastSeenIndex) + + // and a new reattach can continue after what there. + val reattach2 = + stub.reattachExecute(buildReattachExecuteRequest(operationId, Some(lastSeenResponse))) + assert(reattach2.hasNext) + while (reattach2.hasNext) reattach2.next() + } + } + + // A few integration tests with large results. + // They should run significantly faster than the LARGE_QUERY_TIMEOUT + // - big query (4 seconds, 871 milliseconds) + // - big query and slow client (7 seconds, 288 milliseconds) + // - big query with frequent reattach (1 second, 527 milliseconds) + // - big query with frequent reattach and slow client (7 seconds, 365 milliseconds) + // - long sleeping query (10 seconds, 805 milliseconds) + + // intentionally smaller than CONNECT_EXECUTE_REATTACHABLE_SENDER_MAX_STREAM_DURATION, + // so that reattach deadline doesn't "unstuck" if something got stuck. + val LARGE_QUERY_TIMEOUT = 100.seconds + + val LARGE_RESULTS_QUERY = s"select id, " + + (1 to 20).map(i => s"cast(id as string) c$i").mkString(", ") + + s" from range(1000000)" + + test("big query") { + // regular query with large results + runQuery(LARGE_RESULTS_QUERY, LARGE_QUERY_TIMEOUT) + // Check that execution is released on the server. + assertEventuallyNoActiveExecutions() + } + + test("big query and slow client") { + // regular query with large results, but client is slow so sender will need to control flow + runQuery(LARGE_RESULTS_QUERY, LARGE_QUERY_TIMEOUT, iterSleep = 50) + // Check that execution is released on the server. + assertEventuallyNoActiveExecutions() + } + + test("big query with frequent reattach") { + // will reattach every 100kB + withSparkEnvConfs((Connect.CONNECT_EXECUTE_REATTACHABLE_SENDER_MAX_STREAM_SIZE.key, "100k")) { + runQuery(LARGE_RESULTS_QUERY, LARGE_QUERY_TIMEOUT) + // Check that execution is released on the server. + assertEventuallyNoActiveExecutions() + } + } + + test("big query with frequent reattach and slow client") { + // will reattach every 100kB, and in addition the client is slow, + // so sender will need to control flow + withSparkEnvConfs((Connect.CONNECT_EXECUTE_REATTACHABLE_SENDER_MAX_STREAM_SIZE.key, "100k")) { + runQuery(LARGE_RESULTS_QUERY, LARGE_QUERY_TIMEOUT, iterSleep = 50) + // Check that execution is released on the server. + assertEventuallyNoActiveExecutions() + } + } + + test("long sleeping query") { + // query will be sleeping and not returning results, while having multiple reattach + withSparkEnvConfs( + (Connect.CONNECT_EXECUTE_REATTACHABLE_SENDER_MAX_STREAM_DURATION.key, "1s")) { + runQuery("select sleep(10000) as s", 30.seconds) + // Check that execution is released on the server. + assertEventuallyNoActiveExecutions() + } + } +} diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index f5819b9508777..1163088c82aa8 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -302,6 +302,30 @@ abstract class SparkFunSuite } } + /** + * Sets all configurations specified in `pairs` in SparkEnv SparkConf, calls `f`, and then + * restores all configurations. + */ + protected def withSparkEnvConfs(pairs: (String, String)*)(f: => Unit): Unit = { + val conf = SparkEnv.get.conf + val (keys, values) = pairs.unzip + val currentValues = keys.map { key => + if (conf.getOption(key).isDefined) { + Some(conf.get(key)) + } else { + None + } + } + pairs.foreach { kv => conf.set(kv._1, kv._2) } + try f + finally { + keys.zip(currentValues).foreach { + case (key, Some(value)) => conf.set(key, value) + case (key, None) => conf.remove(key) + } + } + } + /** * Checks an exception with an error class against expected results. * @param exception The exception to check From 151f88b53e67944d6ca5c635466f50958019c8b4 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 13 Sep 2023 21:20:19 +0900 Subject: [PATCH 010/521] [SPARK-45142][INFRA] Specify the range for Spark Connect dependencies in pyspark base image This PR proposes to pin the dependencies related to Spark Connect in its base image according to the range we support. See also https://github.com/apache/spark/blob/master/python/docs/source/getting_started/install.rst#dependencies To properly test the dependency versions we support. No, dev-only. In this PR, it will be tested. No. Closes #42898 from HyukjinKwon/SPARK-45142. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit 61435b42fdc4071f35aba6af9248ff9ad8fc8514) Signed-off-by: Hyukjin Kwon --- dev/infra/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/infra/Dockerfile b/dev/infra/Dockerfile index af8e1a980f93c..d3bae836cc631 100644 --- a/dev/infra/Dockerfile +++ b/dev/infra/Dockerfile @@ -68,7 +68,7 @@ RUN pypy3 -m pip install numpy 'pandas<=2.0.3' scipy coverage matplotlib RUN python3.9 -m pip install numpy pyarrow 'pandas<=2.0.3' scipy unittest-xml-reporting plotly>=4.8 'mlflow>=2.3.1' coverage matplotlib openpyxl 'memory-profiler==0.60.0' 'scikit-learn==1.1.*' # Add Python deps for Spark Connect. -RUN python3.9 -m pip install grpcio protobuf googleapis-common-protos grpcio-status +RUN python3.9 -m pip install 'grpcio>=1.48,<1.57' 'grpcio-status>=1.48,<1.57' 'protobuf==3.20.3' 'googleapis-common-protos==1.56.4' # Add torch as a testing dependency for TorchDistributor RUN python3.9 -m pip install torch torchvision torcheval From e72ae794e69d8182291655d023aee903a913571b Mon Sep 17 00:00:00 2001 From: chenyu-opensource <119398199+chenyu-opensource@users.noreply.github.com> Date: Wed, 13 Sep 2023 08:48:14 -0500 Subject: [PATCH 011/521] [SPARK-45146][DOCS] Update the default value of 'spark.submit.deployMode' **What changes were proposed in this pull request?** The PR updates the default value of 'spark.submit.deployMode' in configuration.html on the website **Why are the changes needed?** The default value of 'spark.submit.deployMode' is 'client', but the website is wrong. **Does this PR introduce any user-facing change?** No **How was this patch tested?** It doesn't need to. **Was this patch authored or co-authored using generative AI tooling?** No Closes #42902 from chenyu-opensource/branch-SPARK-45146. Authored-by: chenyu-opensource <119398199+chenyu-opensource@users.noreply.github.com> Signed-off-by: Sean Owen (cherry picked from commit 076cb7aabac2f0ff11ca77ca530b7b8db5310a5e) Signed-off-by: Sean Owen --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index dfded480c99d3..1139beb66462f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -394,7 +394,7 @@ of the most common options to set are: spark.submit.deployMode - (none) + client The deploy mode of Spark driver program, either "client" or "cluster", Which means to launch driver program locally ("client") From 0e1a9b65d48389e2bbed11dabfa6c61cca5f41f0 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 14 Sep 2023 18:23:38 +0800 Subject: [PATCH 012/521] [MINOR][PYTHON][DOCS] Fix default value of parameter `barrier` in MapInXXX ### What changes were proposed in this pull request? Fix default value of parameter `barrier` ### Why are the changes needed? they default to `False` ### Does this PR introduce _any_ user-facing change? yes ### How was this patch tested? CI ### Was this patch authored or co-authored using generative AI tooling? NO Closes #42923 from zhengruifeng/45114_followup. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng (cherry picked from commit e1d2372b8916741fe199ee7b154e53af1eb1ba5a) Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/pandas/map_ops.py | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/pandas/map_ops.py b/python/pyspark/sql/pandas/map_ops.py index bc26fdede2888..710fc8a9a370a 100644 --- a/python/pyspark/sql/pandas/map_ops.py +++ b/python/pyspark/sql/pandas/map_ops.py @@ -60,11 +60,10 @@ def mapInPandas( schema : :class:`pyspark.sql.types.DataType` or str the return type of the `func` in PySpark. The value can be either a :class:`pyspark.sql.types.DataType` object or a DDL-formatted type string. - barrier : bool, optional, default True + barrier : bool, optional, default False Use barrier mode execution. - .. versionchanged: 3.5.0 - Added ``barrier`` argument. + .. versionadded: 3.5.0 Examples -------- @@ -139,11 +138,10 @@ def mapInArrow( schema : :class:`pyspark.sql.types.DataType` or str the return type of the `func` in PySpark. The value can be either a :class:`pyspark.sql.types.DataType` object or a DDL-formatted type string. - barrier : bool, optional, default True + barrier : bool, optional, default False Use barrier mode execution. - .. versionchanged: 3.5.0 - Added ``barrier`` argument. + .. versionadded: 3.5.0 Examples -------- From 9c0b803ba124a6e70762aec1e5559b0d66529f4d Mon Sep 17 00:00:00 2001 From: Bruce Robbins Date: Fri, 15 Sep 2023 13:22:40 +0900 Subject: [PATCH 013/521] [SPARK-45171][SQL] Initialize non-deterministic expressions in `GenerateExec` ### What changes were proposed in this pull request? Before evaluating the generator function in `GenerateExec`, initialize non-deterministic expressions. ### Why are the changes needed? The following query fails: ``` select * from explode( transform(sequence(0, cast(rand()*1000 as int) + 1), x -> x * 22) ); 23/09/14 09:27:25 ERROR Executor: Exception in task 0.0 in stage 3.0 (TID 3) java.lang.IllegalArgumentException: requirement failed: Nondeterministic expression org.apache.spark.sql.catalyst.expressions.Rand should be initialized before eval. at scala.Predef$.require(Predef.scala:281) at org.apache.spark.sql.catalyst.expressions.Nondeterministic.eval(Expression.scala:497) at org.apache.spark.sql.catalyst.expressions.Nondeterministic.eval$(Expression.scala:495) at org.apache.spark.sql.catalyst.expressions.RDG.eval(randomExpressions.scala:35) at org.apache.spark.sql.catalyst.expressions.BinaryArithmetic.eval(arithmetic.scala:384) at org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:543) at org.apache.spark.sql.catalyst.expressions.BinaryArithmetic.eval(arithmetic.scala:384) at org.apache.spark.sql.catalyst.expressions.Sequence.eval(collectionOperations.scala:3062) at org.apache.spark.sql.catalyst.expressions.SimpleHigherOrderFunction.eval(higherOrderFunctions.scala:275) at org.apache.spark.sql.catalyst.expressions.SimpleHigherOrderFunction.eval$(higherOrderFunctions.scala:274) at org.apache.spark.sql.catalyst.expressions.ArrayTransform.eval(higherOrderFunctions.scala:308) at org.apache.spark.sql.catalyst.expressions.ExplodeBase.eval(generators.scala:375) at org.apache.spark.sql.execution.GenerateExec.$anonfun$doExecute$8(GenerateExec.scala:108) ... ``` However, this query succeeds: ``` select * from explode( sequence(0, cast(rand()*1000 as int) + 1) ); 0 1 2 3 ... 801 802 803 ``` The difference is that `transform` turns off whole-stage codegen, which exposes a bug in `GenerateExec` in which the non-deterministic expression passed to the generator function is not initialized before being used. This PR fixes the bug in `GenerateExec`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New unit test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #42933 from bersprockets/nondeterm_issue. Lead-authored-by: Bruce Robbins Co-authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit e097f916a2769dfe82bfd216fedcd6962e8280c8) Signed-off-by: Hyukjin Kwon --- .../org/apache/spark/sql/execution/GenerateExec.scala | 4 ++++ .../org/apache/spark/sql/GeneratorFunctionSuite.scala | 7 +++++++ 2 files changed, 11 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala index f6dbf5fda1816..b99361437e0d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala @@ -78,6 +78,10 @@ case class GenerateExec( // boundGenerator.terminate() should be triggered after all of the rows in the partition val numOutputRows = longMetric("numOutputRows") child.execute().mapPartitionsWithIndexInternal { (index, iter) => + boundGenerator.foreach { + case n: Nondeterministic => n.initialize(index) + case _ => + } val generatorNullRow = new GenericInternalRow(generator.elementSchema.length) val rows = if (requiredChildOutput.nonEmpty) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala index abec582d43a30..0746a4b92af29 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala @@ -536,6 +536,13 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { checkAnswer(df, Row(1, 1) :: Row(1, 2) :: Row(2, 2) :: Row(2, 3) :: Row(3, null) :: Nil) } + + test("SPARK-45171: Handle evaluated nondeterministic expression") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + val df = sql("select explode(array(rand(0)))") + checkAnswer(df, Row(0.7604953758285915d)) + } + } } case class EmptyGenerator() extends Generator with LeafLike[Expression] { From a3f50e742506e07473c281255d1b13ab8ae78cd6 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Sat, 16 Sep 2023 09:04:38 -0500 Subject: [PATCH 014/521] [SPARK-45127][DOCS] Exclude README.md from document build ### What changes were proposed in this pull request? The pr aims to exclude `README.md` from document build. ### Why are the changes needed? - Currently, our document `README.html` does not have any CSS style applied to it, as shown below: https://spark.apache.org/docs/latest/README.html image **If we do not intend to display the above page to users, we should remove it during the document build process.** - As we saw in the project `spark-website`, it has already set the following configuration: https://github.com/apache/spark-website/blob/642d1fb834817014e1799e73882d53650c1c1662/_config.yml#L7 image Let's stay consistent. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Manually test. After this pr, the README.html file will no longer be generated ``` (base) panbingkun:~/Developer/spark/spark-community/docs/_site$ls -al README.html ls: README.html: No such file or directory ``` - Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #42883 from panbingkun/SPARK-45127. Authored-by: panbingkun Signed-off-by: Sean Owen (cherry picked from commit 804f741453fb146b5261084fa3baf26631badb79) Signed-off-by: Sean Owen --- docs/_config.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/_config.yml b/docs/_config.yml index afe015b2972da..e346833722b93 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -46,3 +46,5 @@ DOCSEARCH_SCRIPT: | }); permalink: 404.html + +exclude: ['README.md'] From 723a85eb2dffa69571cba841380eb759a9b89321 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Sun, 17 Sep 2023 11:16:24 +0300 Subject: [PATCH 015/521] [SPARK-45078][SQL] Fix `array_insert` ImplicitCastInputTypes not work ### What changes were proposed in this pull request? This PR fix call `array_insert` with different type between array and insert column, will throw exception. Sometimes it should be execute successed. eg: ```sql select array_insert(array(1), 2, cast(2 as tinyint)) ``` The `ImplicitCastInputTypes` in `ArrayInsert` always return empty array at now. So that Spark can not convert `tinyint` to `int`. ### Why are the changes needed? Fix error behavior in `array_insert` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Add new test. ### Was this patch authored or co-authored using generative AI tooling? No Closes #42951 from Hisoka-X/SPARK-45078_arrayinsert_type_mismatch. Authored-by: Jia Fan Signed-off-by: Max Gekk (cherry picked from commit e84c66db60c78476806161479344cd32a7606ab1) Signed-off-by: Max Gekk --- .../sql/catalyst/expressions/collectionOperations.scala | 1 - .../sql-tests/analyzer-results/ansi/array.sql.out | 7 +++++++ .../resources/sql-tests/analyzer-results/array.sql.out | 7 +++++++ sql/core/src/test/resources/sql-tests/inputs/array.sql | 1 + .../test/resources/sql-tests/results/ansi/array.sql.out | 8 ++++++++ .../src/test/resources/sql-tests/results/array.sql.out | 8 ++++++++ 6 files changed, 31 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index fe9c4015c15ec..ade4a6c5be722 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -4711,7 +4711,6 @@ case class ArrayInsert( } case (e1, e2, e3) => Seq.empty } - Seq.empty } override def checkInputDataTypes(): TypeCheckResult = { diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out index cd101c7a524a1..6fc308157933f 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out @@ -531,6 +531,13 @@ Project [array_insert(array(2, 3, cast(null as int), 4), -5, 1, false) AS array_ +- OneRowRelation +-- !query +select array_insert(array(1), 2, cast(2 as tinyint)) +-- !query analysis +Project [array_insert(array(1), 2, cast(cast(2 as tinyint) as int), false) AS array_insert(array(1), 2, CAST(2 AS TINYINT))#x] ++- OneRowRelation + + -- !query set spark.sql.legacy.negativeIndexInArrayInsert=true -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out index 8279fb3362e54..e0585b77cb6bd 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out @@ -531,6 +531,13 @@ Project [array_insert(array(2, 3, cast(null as int), 4), -5, 1, false) AS array_ +- OneRowRelation +-- !query +select array_insert(array(1), 2, cast(2 as tinyint)) +-- !query analysis +Project [array_insert(array(1), 2, cast(cast(2 as tinyint) as int), false) AS array_insert(array(1), 2, CAST(2 AS TINYINT))#x] ++- OneRowRelation + + -- !query set spark.sql.legacy.negativeIndexInArrayInsert=true -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/array.sql b/sql/core/src/test/resources/sql-tests/inputs/array.sql index 48edc6b474254..52a0906ea7392 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/array.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/array.sql @@ -141,6 +141,7 @@ select array_insert(array(1, 2, 3, NULL), cast(NULL as INT), 4); select array_insert(array(1, 2, 3, NULL), 4, cast(NULL as INT)); select array_insert(array(2, 3, NULL, 4), 5, 5); select array_insert(array(2, 3, NULL, 4), -5, 1); +select array_insert(array(1), 2, cast(2 as tinyint)); set spark.sql.legacy.negativeIndexInArrayInsert=true; select array_insert(array(1, 3, 4), -2, 2); diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out index 03be0f9d84b1b..49e18411ffa37 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out @@ -659,6 +659,14 @@ struct> [1,2,3,null,4] +-- !query +select array_insert(array(1), 2, cast(2 as tinyint)) +-- !query schema +struct> +-- !query output +[1,2] + + -- !query set spark.sql.legacy.negativeIndexInArrayInsert=true -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/array.sql.out b/sql/core/src/test/resources/sql-tests/results/array.sql.out index 9dbf4fbebc20b..e568f5fa7796d 100644 --- a/sql/core/src/test/resources/sql-tests/results/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/array.sql.out @@ -540,6 +540,14 @@ struct> [1,2,3,null,4] +-- !query +select array_insert(array(1), 2, cast(2 as tinyint)) +-- !query schema +struct> +-- !query output +[1,2] + + -- !query set spark.sql.legacy.negativeIndexInArrayInsert=true -- !query schema From 84a053e72ac9d9cfc91bab777cea94958d3a91da Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 17 Sep 2023 10:34:23 -0700 Subject: [PATCH 016/521] [SPARK-45187][CORE] Fix `WorkerPage` to use the same pattern for `logPage` urls ### What changes were proposed in this pull request? This PR aims to use the same pattern for `logPage` urls of `WorkerPage` to make it work consistently when `spark.ui.reverseProxy=true`. ### Why are the changes needed? Since Apache Spark 3.2.0 (SPARK-34635, #31753), Apache Spark adds trailing slashes to reduce redirections for `logPage`. ```scala s"$workerUrlRef/logPage?driverId=$driverId&logType=stdout") s"$workerUrlRef/logPage/?driverId=$driverId&logType=stdout") ... stdout - stderr + stdout + stderr ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual tests because it requires a reverse proxy. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #42959 from dongjoon-hyun/SPARK-45187. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit f8f2735426ee7ad3d7a1f5bd07e72643516f4a35) Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala index 3171d3f16e8a0..e740b328dd7b9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala @@ -212,8 +212,8 @@ private[ui] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { {formatResourcesAddresses(driver.resources)} - stdout - stderr + stdout + stderr {driver.finalException.getOrElse("")} From 60073f318313ab2329ea1504ef7538641433852e Mon Sep 17 00:00:00 2001 From: Martin Grund Date: Tue, 19 Sep 2023 08:32:21 +0900 Subject: [PATCH 017/521] [SPARK-45167][CONNECT][PYTHON][3.5] Python client must call `release_all` ### What changes were proposed in this pull request? Cherry-pick of https://github.com/apache/spark/pull/42929 Previously the Python client would not call `release_all` after fetching all results and leaving the query dangling. The query would then be removed after the five minute timeout. This patch adds proper testing for calling release all and release until. In addition it fixes a test race condition where we would close the SparkSession which would in turn close the GRPC channel which might have dangling async release calls hanging. ### Why are the changes needed? Stability ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? new UT ### Was this patch authored or co-authored using generative AI tooling? No Closes #42973 from grundprinzip/SPARK-45167-3.5. Authored-by: Martin Grund Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/connect/client/core.py | 1 + python/pyspark/sql/connect/client/reattach.py | 37 +++- .../sql/tests/connect/client/test_client.py | 195 +++++++++++++++++- 3 files changed, 226 insertions(+), 7 deletions(-) diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index 7b3299d123b97..7b1aafbefebbe 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -1005,6 +1005,7 @@ def close(self) -> None: """ Close the channel. """ + ExecutePlanResponseReattachableIterator.shutdown() self._channel.close() self._closed = True diff --git a/python/pyspark/sql/connect/client/reattach.py b/python/pyspark/sql/connect/client/reattach.py index 7e1e722d5fd8a..e58864b965bd9 100644 --- a/python/pyspark/sql/connect/client/reattach.py +++ b/python/pyspark/sql/connect/client/reattach.py @@ -21,7 +21,9 @@ import warnings import uuid from collections.abc import Generator -from typing import Optional, Dict, Any, Iterator, Iterable, Tuple, Callable, cast +from typing import Optional, Dict, Any, Iterator, Iterable, Tuple, Callable, cast, Type, ClassVar +from multiprocessing import RLock +from multiprocessing.synchronize import RLock as RLockBase from multiprocessing.pool import ThreadPool import os @@ -53,7 +55,30 @@ class ExecutePlanResponseReattachableIterator(Generator): ReleaseExecute RPCs that instruct the server to release responses that it already processed. """ - _release_thread_pool = ThreadPool(os.cpu_count() if os.cpu_count() else 8) + # Lock to manage the pool + _lock: ClassVar[RLockBase] = RLock() + _release_thread_pool: Optional[ThreadPool] = ThreadPool(os.cpu_count() if os.cpu_count() else 8) + + @classmethod + def shutdown(cls: Type["ExecutePlanResponseReattachableIterator"]) -> None: + """ + When the channel is closed, this method will be called before, to make sure all + outstanding calls are closed. + """ + with cls._lock: + if cls._release_thread_pool is not None: + cls._release_thread_pool.close() + cls._release_thread_pool.join() + cls._release_thread_pool = None + + @classmethod + def _initialize_pool_if_necessary(cls: Type["ExecutePlanResponseReattachableIterator"]) -> None: + """ + If the processing pool for the release calls is None, initialize the pool exactly once. + """ + with cls._lock: + if cls._release_thread_pool is None: + cls._release_thread_pool = ThreadPool(os.cpu_count() if os.cpu_count() else 8) def __init__( self, @@ -62,6 +87,7 @@ def __init__( retry_policy: Dict[str, Any], metadata: Iterable[Tuple[str, str]], ): + ExecutePlanResponseReattachableIterator._initialize_pool_if_necessary() self._request = request self._retry_policy = retry_policy if request.operation_id: @@ -111,7 +137,6 @@ def send(self, value: Any) -> pb2.ExecutePlanResponse: self._last_returned_response_id = ret.response_id if ret.HasField("result_complete"): - self._result_complete = True self._release_all() else: self._release_until(self._last_returned_response_id) @@ -190,7 +215,8 @@ def target() -> None: except Exception as e: warnings.warn(f"ReleaseExecute failed with exception: {e}.") - ExecutePlanResponseReattachableIterator._release_thread_pool.apply_async(target) + if ExecutePlanResponseReattachableIterator._release_thread_pool is not None: + ExecutePlanResponseReattachableIterator._release_thread_pool.apply_async(target) def _release_all(self) -> None: """ @@ -218,7 +244,8 @@ def target() -> None: except Exception as e: warnings.warn(f"ReleaseExecute failed with exception: {e}.") - ExecutePlanResponseReattachableIterator._release_thread_pool.apply_async(target) + if ExecutePlanResponseReattachableIterator._release_thread_pool is not None: + ExecutePlanResponseReattachableIterator._release_thread_pool.apply_async(target) self._result_complete = True def _call_iter(self, iter_fun: Callable) -> Any: diff --git a/python/pyspark/sql/tests/connect/client/test_client.py b/python/pyspark/sql/tests/connect/client/test_client.py index 98f68767b8bca..cf43fb16df7a7 100644 --- a/python/pyspark/sql/tests/connect/client/test_client.py +++ b/python/pyspark/sql/tests/connect/client/test_client.py @@ -17,14 +17,20 @@ import unittest import uuid -from typing import Optional +from collections.abc import Generator +from typing import Optional, Any + +import grpc from pyspark.sql.connect.client import SparkConnectClient, ChannelBuilder import pyspark.sql.connect.proto as proto from pyspark.testing.connectutils import should_test_connect, connect_requirement_message from pyspark.sql.connect.client.core import Retrying -from pyspark.sql.connect.client.reattach import RetryException +from pyspark.sql.connect.client.reattach import ( + RetryException, + ExecutePlanResponseReattachableIterator, +) if should_test_connect: import pandas as pd @@ -120,6 +126,191 @@ def test_channel_builder_with_session(self): self.assertEqual(client._session_id, chan.session_id) +@unittest.skipIf(not should_test_connect, connect_requirement_message) +class SparkConnectClientReattachTestCase(unittest.TestCase): + def setUp(self) -> None: + self.request = proto.ExecutePlanRequest() + self.policy = { + "max_retries": 3, + "backoff_multiplier": 4.0, + "initial_backoff": 10, + "max_backoff": 10, + "jitter": 10, + "min_jitter_threshold": 10, + } + self.response = proto.ExecutePlanResponse( + response_id="1", + ) + self.finished = proto.ExecutePlanResponse( + result_complete=proto.ExecutePlanResponse.ResultComplete(), + response_id="2", + ) + + def _stub_with(self, execute=None, attach=None): + return MockSparkConnectStub( + execute_ops=ResponseGenerator(execute) if execute is not None else None, + attach_ops=ResponseGenerator(attach) if attach is not None else None, + ) + + def assertEventually(self, callable, timeout_ms=1000): + """Helper method that will continuously evaluate the callable to not raise an + exception.""" + import time + + limit = time.monotonic_ns() + timeout_ms * 1000 * 1000 + while time.monotonic_ns() < limit: + try: + callable() + break + except Exception: + time.sleep(0.1) + callable() + + def test_basic_flow(self): + stub = self._stub_with([self.response, self.finished]) + ite = ExecutePlanResponseReattachableIterator(self.request, stub, self.policy, []) + for b in ite: + pass + + def check_all(): + self.assertEqual(0, stub.attach_calls) + self.assertEqual(1, stub.release_until_calls) + self.assertEqual(1, stub.release_calls) + self.assertEqual(1, stub.execute_calls) + + self.assertEventually(check_all, timeout_ms=1000) + + def test_fail_during_execute(self): + def fatal(): + raise TestException("Fatal") + + stub = self._stub_with([self.response, fatal]) + with self.assertRaises(TestException): + ite = ExecutePlanResponseReattachableIterator(self.request, stub, self.policy, []) + for b in ite: + pass + + def check(): + self.assertEqual(0, stub.attach_calls) + self.assertEqual(1, stub.release_calls) + self.assertEqual(1, stub.release_until_calls) + self.assertEqual(1, stub.execute_calls) + + self.assertEventually(check, timeout_ms=1000) + + def test_fail_and_retry_during_execute(self): + def non_fatal(): + raise TestException("Non Fatal", grpc.StatusCode.UNAVAILABLE) + + stub = self._stub_with( + [self.response, non_fatal], [self.response, self.response, self.finished] + ) + ite = ExecutePlanResponseReattachableIterator(self.request, stub, self.policy, []) + for b in ite: + pass + + def check(): + self.assertEqual(1, stub.attach_calls) + self.assertEqual(1, stub.release_calls) + self.assertEqual(3, stub.release_until_calls) + self.assertEqual(1, stub.execute_calls) + + self.assertEventually(check, timeout_ms=1000) + + def test_fail_and_retry_during_reattach(self): + count = 0 + + def non_fatal(): + nonlocal count + if count < 2: + count += 1 + raise TestException("Non Fatal", grpc.StatusCode.UNAVAILABLE) + else: + return proto.ExecutePlanResponse() + + stub = self._stub_with( + [self.response, non_fatal], [self.response, non_fatal, self.response, self.finished] + ) + ite = ExecutePlanResponseReattachableIterator(self.request, stub, self.policy, []) + for b in ite: + pass + + def check(): + self.assertEqual(2, stub.attach_calls) + self.assertEqual(3, stub.release_until_calls) + self.assertEqual(1, stub.release_calls) + self.assertEqual(1, stub.execute_calls) + + self.assertEventually(check, timeout_ms=1000) + + +class TestException(grpc.RpcError, grpc.Call): + """Exception mock to test retryable exceptions.""" + + def __init__(self, msg, code=grpc.StatusCode.INTERNAL): + self.msg = msg + self._code = code + + def code(self): + return self._code + + def __str__(self): + return self.msg + + def trailing_metadata(self): + return () + + +class ResponseGenerator(Generator): + """This class is used to generate values that are returned by the streaming + iterator of the GRPC stub.""" + + def __init__(self, funs): + self._funs = funs + self._iterator = iter(self._funs) + + def send(self, value: Any) -> proto.ExecutePlanResponse: + val = next(self._iterator) + if callable(val): + return val() + else: + return val + + def throw(self, type: Any = None, value: Any = None, traceback: Any = None) -> Any: + super().throw(type, value, traceback) + + def close(self) -> None: + return super().close() + + +class MockSparkConnectStub: + """Simple mock class for the GRPC stub used by the re-attachable execution.""" + + def __init__(self, execute_ops=None, attach_ops=None): + self._execute_ops = execute_ops + self._attach_ops = attach_ops + # Call counters + self.execute_calls = 0 + self.release_calls = 0 + self.release_until_calls = 0 + self.attach_calls = 0 + + def ExecutePlan(self, *args, **kwargs): + self.execute_calls += 1 + return self._execute_ops + + def ReattachExecute(self, *args, **kwargs): + self.attach_calls += 1 + return self._attach_ops + + def ReleaseExecute(self, req: proto.ReleaseExecuteRequest, *args, **kwargs): + if req.HasField("release_all"): + self.release_calls += 1 + elif req.HasField("release_until"): + print("increment") + self.release_until_calls += 1 + + class MockService: # Simplest mock of the SparkConnectService. # If this needs more complex logic, it needs to be replaced with Python mocking. From 2a9dd2b3968da7c2e96c502aaf4c158ee782e5f4 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Mon, 18 Sep 2023 13:46:34 +0900 Subject: [PATCH 018/521] [SPARK-45167][CONNECT][PYTHON][FOLLOW-UP] Use lighter threading Rlock, and use the existing eventually util function This PR is a followup of https://github.com/apache/spark/pull/42929 that: - Use lighter threading `Rlock` instead of multithreading `Rlock`. Multiprocessing does not work with PySpark due to the ser/de problem for socket connections, and many others. - Use the existing eventually util function `pyspark.testing.eventually` instead of `assertEventually` to deduplicate code. Mainly for code clean-up. No. Existing tests should pass them. No. Closes #42965 from HyukjinKwon/SPARK-45167-followup. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit d5ff04da217df483d27011f6e38417df2eaa42bd) Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/connect/client/reattach.py | 5 ++-- .../sql/tests/connect/client/test_client.py | 23 ++++--------------- 2 files changed, 7 insertions(+), 21 deletions(-) diff --git a/python/pyspark/sql/connect/client/reattach.py b/python/pyspark/sql/connect/client/reattach.py index e58864b965bd9..6addb5bd2c652 100644 --- a/python/pyspark/sql/connect/client/reattach.py +++ b/python/pyspark/sql/connect/client/reattach.py @@ -18,12 +18,11 @@ check_dependencies(__name__) +from threading import RLock import warnings import uuid from collections.abc import Generator from typing import Optional, Dict, Any, Iterator, Iterable, Tuple, Callable, cast, Type, ClassVar -from multiprocessing import RLock -from multiprocessing.synchronize import RLock as RLockBase from multiprocessing.pool import ThreadPool import os @@ -56,7 +55,7 @@ class ExecutePlanResponseReattachableIterator(Generator): """ # Lock to manage the pool - _lock: ClassVar[RLockBase] = RLock() + _lock: ClassVar[RLock] = RLock() _release_thread_pool: Optional[ThreadPool] = ThreadPool(os.cpu_count() if os.cpu_count() else 8) @classmethod diff --git a/python/pyspark/sql/tests/connect/client/test_client.py b/python/pyspark/sql/tests/connect/client/test_client.py index cf43fb16df7a7..93b7006799b30 100644 --- a/python/pyspark/sql/tests/connect/client/test_client.py +++ b/python/pyspark/sql/tests/connect/client/test_client.py @@ -25,6 +25,7 @@ from pyspark.sql.connect.client import SparkConnectClient, ChannelBuilder import pyspark.sql.connect.proto as proto from pyspark.testing.connectutils import should_test_connect, connect_requirement_message +from pyspark.testing.utils import eventually from pyspark.sql.connect.client.core import Retrying from pyspark.sql.connect.client.reattach import ( @@ -152,20 +153,6 @@ def _stub_with(self, execute=None, attach=None): attach_ops=ResponseGenerator(attach) if attach is not None else None, ) - def assertEventually(self, callable, timeout_ms=1000): - """Helper method that will continuously evaluate the callable to not raise an - exception.""" - import time - - limit = time.monotonic_ns() + timeout_ms * 1000 * 1000 - while time.monotonic_ns() < limit: - try: - callable() - break - except Exception: - time.sleep(0.1) - callable() - def test_basic_flow(self): stub = self._stub_with([self.response, self.finished]) ite = ExecutePlanResponseReattachableIterator(self.request, stub, self.policy, []) @@ -178,7 +165,7 @@ def check_all(): self.assertEqual(1, stub.release_calls) self.assertEqual(1, stub.execute_calls) - self.assertEventually(check_all, timeout_ms=1000) + eventually(timeout=1, catch_assertions=True)(check_all)() def test_fail_during_execute(self): def fatal(): @@ -196,7 +183,7 @@ def check(): self.assertEqual(1, stub.release_until_calls) self.assertEqual(1, stub.execute_calls) - self.assertEventually(check, timeout_ms=1000) + eventually(timeout=1, catch_assertions=True)(check)() def test_fail_and_retry_during_execute(self): def non_fatal(): @@ -215,7 +202,7 @@ def check(): self.assertEqual(3, stub.release_until_calls) self.assertEqual(1, stub.execute_calls) - self.assertEventually(check, timeout_ms=1000) + eventually(timeout=1, catch_assertions=True)(check)() def test_fail_and_retry_during_reattach(self): count = 0 @@ -241,7 +228,7 @@ def check(): self.assertEqual(1, stub.release_calls) self.assertEqual(1, stub.execute_calls) - self.assertEventually(check, timeout_ms=1000) + eventually(timeout=1, catch_assertions=True)(check)() class TestException(grpc.RpcError, grpc.Call): From 555c8def51e5951c7bf5165a332795e9e330ec9d Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Tue, 19 Sep 2023 10:18:18 +0900 Subject: [PATCH 019/521] Revert "Revert "[SPARK-44742][PYTHON][DOCS] Add Spark version drop down to the PySpark doc site"" This reverts commit bbe12e148eb1f289cfb1f4412525f4c4381c10a9. --- python/docs/source/_static/css/pyspark.css | 13 ++++ python/docs/source/_static/versions.json | 22 ++++++ .../source/_templates/version-switcher.html | 77 +++++++++++++++++++ python/docs/source/conf.py | 9 ++- 4 files changed, 120 insertions(+), 1 deletion(-) create mode 100644 python/docs/source/_static/versions.json create mode 100644 python/docs/source/_templates/version-switcher.html diff --git a/python/docs/source/_static/css/pyspark.css b/python/docs/source/_static/css/pyspark.css index 89b7c65f27a51..ccfe60f2bca64 100644 --- a/python/docs/source/_static/css/pyspark.css +++ b/python/docs/source/_static/css/pyspark.css @@ -95,3 +95,16 @@ u.bd-sidebar .nav>li>ul>.active:hover>a,.bd-sidebar .nav>li>ul>.active>a { .spec_table tr, td, th { border-top: none!important; } + +/* Styling to the version dropdown */ +#version-button { + padding-left: 0.2rem; + padding-right: 3.2rem; +} + +#version_switcher { + height: auto; + max-height: 300px; + width: 165px; + overflow-y: auto; +} diff --git a/python/docs/source/_static/versions.json b/python/docs/source/_static/versions.json new file mode 100644 index 0000000000000..3d0bd14818064 --- /dev/null +++ b/python/docs/source/_static/versions.json @@ -0,0 +1,22 @@ +[ + { + "name": "3.4.1", + "version": "3.4.1" + }, + { + "name": "3.4.0", + "version": "3.4.0" + }, + { + "name": "3.3.2", + "version": "3.3.2" + }, + { + "name": "3.3.1", + "version": "3.3.1" + }, + { + "name": "3.3.0", + "version": "3.3.0" + } +] diff --git a/python/docs/source/_templates/version-switcher.html b/python/docs/source/_templates/version-switcher.html new file mode 100644 index 0000000000000..16c443229f4be --- /dev/null +++ b/python/docs/source/_templates/version-switcher.html @@ -0,0 +1,77 @@ + + + + + diff --git a/python/docs/source/conf.py b/python/docs/source/conf.py index 38c331048e7b6..0f57cb37ceeb1 100644 --- a/python/docs/source/conf.py +++ b/python/docs/source/conf.py @@ -177,10 +177,17 @@ # a list of builtin themes. html_theme = 'pydata_sphinx_theme' +html_context = { + "switcher_json_url": "_static/versions.json", + "switcher_template_url": "https://spark.apache.org/docs/{version}/api/python/index.html", +} + # Theme options are theme-specific and customize the look and feel of a theme # further. For a list of options available for each theme, see the # documentation. -#html_theme_options = {} +html_theme_options = { + "navbar_end": ["version-switcher"] +} # Add any paths that contain custom themes here, relative to this directory. #html_theme_path = [] From 6a498087361ecbd653821fc283b9ea0fa703c820 Mon Sep 17 00:00:00 2001 From: Giambattista Bloisi Date: Mon, 18 Sep 2023 21:37:09 -0700 Subject: [PATCH 020/521] [SPARK-44910][SQL] Encoders.bean does not support superclasses with generic type arguments ### What changes were proposed in this pull request? This pull request adds Encoders.bean support for beans having a superclass declared with generic type arguments. For example: ``` class JavaBeanWithGenericsA { public T getPropertyA() { return null; } public void setPropertyA(T a) { } } class JavaBeanWithGenericBase extends JavaBeanWithGenericsA { } Encoders.bean(JavaBeanWithGenericBase.class); // Exception ``` That feature had to be part of [PR 42327](https://github.com/apache/spark/commit/1f5d78b5952fcc6c7d36d3338a5594070e3a62dd) but was missing as I was focusing on nested beans only (hvanhovell ) ### Why are the changes needed? JavaTypeInference.encoderFor did not solve TypeVariable objects for superclasses so when managing a case like in the example above an exception was thrown. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests have been extended, new specific tests have been added ### Was this patch authored or co-authored using generative AI tooling? No Closes #42634 from gbloisi-openaire/SPARK-44910. Lead-authored-by: Giambattista Bloisi Co-authored-by: gbloisi-openaire <141144100+gbloisi-openaire@users.noreply.github.com> Signed-off-by: Dongjoon Hyun (cherry picked from commit 7e14c8cc33f0ed0a9c53a888e8a3b17dd2a5d493) Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/JavaTypeInference.scala | 5 +- ...erics.java => JavaTypeInferenceBeans.java} | 51 +++++++++++++++++-- .../sql/catalyst/JavaTypeInferenceSuite.scala | 41 +++++++++++++-- 3 files changed, 88 insertions(+), 9 deletions(-) rename sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/{JavaBeanWithGenerics.java => JavaTypeInferenceBeans.java} (54%) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index 3d536b735db59..191ccc5254404 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -130,10 +130,13 @@ object JavaTypeInference { // TODO: we should only collect properties that have getter and setter. However, some tests // pass in scala case class as java bean class which doesn't have getter and setter. val properties = getJavaBeanReadableProperties(c) + // add type variables from inheritance hierarchy of the class + val classTV = JavaTypeUtils.getTypeArguments(c, classOf[Object]).asScala.toMap ++ + typeVariables // Note that the fields are ordered by name. val fields = properties.map { property => val readMethod = property.getReadMethod - val encoder = encoderFor(readMethod.getGenericReturnType, seenTypeSet + c, typeVariables) + val encoder = encoderFor(readMethod.getGenericReturnType, seenTypeSet + c, classTV) // The existence of `javax.annotation.Nonnull`, means this field is not nullable. val hasNonNull = readMethod.isAnnotationPresent(classOf[Nonnull]) EncoderField( diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/JavaBeanWithGenerics.java b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/JavaTypeInferenceBeans.java similarity index 54% rename from sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/JavaBeanWithGenerics.java rename to sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/JavaTypeInferenceBeans.java index b84a3122cf84c..cc3540717ee7d 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/JavaBeanWithGenerics.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/JavaTypeInferenceBeans.java @@ -17,25 +17,66 @@ package org.apache.spark.sql.catalyst; -class JavaBeanWithGenerics { +public class JavaTypeInferenceBeans { + + static class JavaBeanWithGenericsA { + public T getPropertyA() { + return null; + } + + public void setPropertyA(T a) { + + } + } + + static class JavaBeanWithGenericsAB extends JavaBeanWithGenericsA { + public T getPropertyB() { + return null; + } + + public void setPropertyB(T a) { + + } + } + + static class JavaBeanWithGenericsABC extends JavaBeanWithGenericsAB { + public T getPropertyC() { + return null; + } + + public void setPropertyC(T a) { + + } + } + + static class JavaBeanWithGenerics { private A attribute; private T value; public A getAttribute() { - return attribute; + return attribute; } public void setAttribute(A attribute) { - this.attribute = attribute; + this.attribute = attribute; } public T getValue() { - return value; + return value; } public void setValue(T value) { - this.value = value; + this.value = value; } + } + + static class JavaBeanWithGenericBase extends JavaBeanWithGenerics { + + } + + static class JavaBeanWithGenericHierarchy extends JavaBeanWithGenericsABC { + + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/JavaTypeInferenceSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/JavaTypeInferenceSuite.scala index 6439997609766..f7c1043d1cb8f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/JavaTypeInferenceSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/JavaTypeInferenceSuite.scala @@ -24,6 +24,7 @@ import scala.beans.{BeanProperty, BooleanBeanProperty} import scala.reflect.{classTag, ClassTag} import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.JavaTypeInferenceBeans.{JavaBeanWithGenericBase, JavaBeanWithGenericHierarchy, JavaBeanWithGenericsABC} import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, UDTCaseClass, UDTForCaseClass} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._ import org.apache.spark.sql.types.{DecimalType, MapType, Metadata, StringType, StructField, StructType} @@ -66,7 +67,8 @@ class LeafBean { @BeanProperty var period: java.time.Period = _ @BeanProperty var enum: java.time.Month = _ @BeanProperty val readOnlyString = "read-only" - @BeanProperty var genericNestedBean: JavaBeanWithGenerics[String, String] = _ + @BeanProperty var genericNestedBean: JavaBeanWithGenericBase = _ + @BeanProperty var genericNestedBean2: JavaBeanWithGenericsABC[Integer] = _ var nonNullString: String = "value" @javax.annotation.Nonnull @@ -186,8 +188,18 @@ class JavaTypeInferenceSuite extends SparkFunSuite { encoderField("duration", DayTimeIntervalEncoder), encoderField("enum", JavaEnumEncoder(classTag[java.time.Month])), encoderField("genericNestedBean", JavaBeanEncoder( - ClassTag(classOf[JavaBeanWithGenerics[String, String]]), - Seq(encoderField("attribute", StringEncoder), encoderField("value", StringEncoder)))), + ClassTag(classOf[JavaBeanWithGenericBase]), + Seq( + encoderField("attribute", StringEncoder), + encoderField("value", StringEncoder) + ))), + encoderField("genericNestedBean2", JavaBeanEncoder( + ClassTag(classOf[JavaBeanWithGenericsABC[Integer]]), + Seq( + encoderField("propertyA", StringEncoder), + encoderField("propertyB", BoxedLongEncoder), + encoderField("propertyC", BoxedIntEncoder) + ))), encoderField("instant", STRICT_INSTANT_ENCODER), encoderField("localDate", STRICT_LOCAL_DATE_ENCODER), encoderField("localDateTime", LocalDateTimeEncoder), @@ -224,4 +236,27 @@ class JavaTypeInferenceSuite extends SparkFunSuite { )) assert(encoder === expected) } + + test("SPARK-44910: resolve bean with generic base class") { + val encoder = + JavaTypeInference.encoderFor(classOf[JavaBeanWithGenericBase]) + val expected = + JavaBeanEncoder(ClassTag(classOf[JavaBeanWithGenericBase]), Seq( + encoderField("attribute", StringEncoder), + encoderField("value", StringEncoder) + )) + assert(encoder === expected) + } + + test("SPARK-44910: resolve bean with hierarchy of generic classes") { + val encoder = + JavaTypeInference.encoderFor(classOf[JavaBeanWithGenericHierarchy]) + val expected = + JavaBeanEncoder(ClassTag(classOf[JavaBeanWithGenericHierarchy]), Seq( + encoderField("propertyA", StringEncoder), + encoderField("propertyB", BoxedLongEncoder), + encoderField("propertyC", BoxedIntEncoder) + )) + assert(encoder === expected) + } } From f357f93fa2cf941c1f1e2745a10865bb12d5ab56 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Tue, 19 Sep 2023 16:56:38 +0800 Subject: [PATCH 021/521] [SPARK-45211][CONNECT] Eliminated ambiguous references in `CloseableIterator#apply` to fix Scala 2.13 daily test ### What changes were proposed in this pull request? This pr eliminated an ambiguous references in `org.apache.spark.sql.connect.client.CloseableIterator#apply` function to make the test case `abandoned query gets INVALID_HANDLE.OPERATION_ABANDONED error` can test pass with Scala 2.13. ### Why are the changes needed? `abandoned query gets INVALID_HANDLE.OPERATION_ABANDONED error` failed in the daily test of Scala 2.13: - https://github.com/apache/spark/actions/runs/6215331575/job/16868131377 image ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Pass GitHub Actions - Manual check run ``` dev/change-scala-version.sh 2.13 build/sbt "connect/testOnly org.apache.spark.sql.connect.execution.ReattachableExecuteSuite" -Pscala-2.13 ``` **Before** ``` [info] ReattachableExecuteSuite: [info] - reattach after initial RPC ends (2 seconds, 258 milliseconds) [info] - raw interrupted RPC results in INVALID_CURSOR.DISCONNECTED error (30 milliseconds) [info] - raw new RPC interrupts previous RPC with INVALID_CURSOR.DISCONNECTED error (21 milliseconds) [info] - client INVALID_CURSOR.DISCONNECTED error is retried when rpc sender gets interrupted (602 milliseconds) [info] - client INVALID_CURSOR.DISCONNECTED error is retried when other RPC preempts this one (637 milliseconds) [info] - abandoned query gets INVALID_HANDLE.OPERATION_ABANDONED error *** FAILED *** (70 milliseconds) [info] Expected exception org.apache.spark.SparkException to be thrown, but java.lang.StackOverflowError was thrown (ReattachableExecuteSuite.scala:172) [info] org.scalatest.exceptions.TestFailedException: [info] at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472) [info] at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471) [info] at org.scalatest.funsuite.AnyFunSuite.newAssertionFailedException(AnyFunSuite.scala:1564) [info] at org.scalatest.Assertions.intercept(Assertions.scala:756) [info] at org.scalatest.Assertions.intercept$(Assertions.scala:746) [info] at org.scalatest.funsuite.AnyFunSuite.intercept(AnyFunSuite.scala:1564) [info] at org.apache.spark.sql.connect.execution.ReattachableExecuteSuite.$anonfun$new$18(ReattachableExecuteSuite.scala:172) [info] at org.apache.spark.sql.connect.execution.ReattachableExecuteSuite.$anonfun$new$18$adapted(ReattachableExecuteSuite.scala:168) [info] at org.apache.spark.sql.connect.SparkConnectServerTest.withCustomBlockingStub(SparkConnectServerTest.scala:222) [info] at org.apache.spark.sql.connect.SparkConnectServerTest.withCustomBlockingStub$(SparkConnectServerTest.scala:216) [info] at org.apache.spark.sql.connect.execution.ReattachableExecuteSuite.withCustomBlockingStub(ReattachableExecuteSuite.scala:30) [info] at org.apache.spark.sql.connect.execution.ReattachableExecuteSuite.$anonfun$new$16(ReattachableExecuteSuite.scala:168) [info] at org.apache.spark.sql.connect.execution.ReattachableExecuteSuite.$anonfun$new$16$adapted(ReattachableExecuteSuite.scala:151) [info] at org.apache.spark.sql.connect.SparkConnectServerTest.withClient(SparkConnectServerTest.scala:199) [info] at org.apache.spark.sql.connect.SparkConnectServerTest.withClient$(SparkConnectServerTest.scala:191) [info] at org.apache.spark.sql.connect.execution.ReattachableExecuteSuite.withClient(ReattachableExecuteSuite.scala:30) [info] at org.apache.spark.sql.connect.execution.ReattachableExecuteSuite.$anonfun$new$15(ReattachableExecuteSuite.scala:151) [info] at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.scala:18) [info] at org.scalatest.enablers.Timed$$anon$1.timeoutAfter(Timed.scala:127) [info] at org.scalatest.concurrent.TimeLimits$.failAfterImpl(TimeLimits.scala:282) [info] at org.scalatest.concurrent.TimeLimits.failAfter(TimeLimits.scala:231) [info] at org.scalatest.concurrent.TimeLimits.failAfter$(TimeLimits.scala:230) [info] at org.apache.spark.SparkFunSuite.failAfter(SparkFunSuite.scala:69) [info] at org.apache.spark.SparkFunSuite.$anonfun$test$2(SparkFunSuite.scala:155) [info] at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85) [info] at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83) [info] at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) [info] at org.scalatest.Transformer.apply(Transformer.scala:22) [info] at org.scalatest.Transformer.apply(Transformer.scala:20) [info] at org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226) [info] at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:227) [info] at org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224) [info] at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236) [info] at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306) [info] at org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236) [info] at org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218) [info] at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:69) [info] at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234) [info] at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227) [info] at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:69) [info] at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269) [info] at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413) [info] at scala.collection.immutable.List.foreach(List.scala:333) [info] at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401) [info] at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:396) [info] at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:475) [info] at org.scalatest.funsuite.AnyFunSuiteLike.runTests(AnyFunSuiteLike.scala:269) [info] at org.scalatest.funsuite.AnyFunSuiteLike.runTests$(AnyFunSuiteLike.scala:268) [info] at org.scalatest.funsuite.AnyFunSuite.runTests(AnyFunSuite.scala:1564) [info] at org.scalatest.Suite.run(Suite.scala:1114) [info] at org.scalatest.Suite.run$(Suite.scala:1096) [info] at org.scalatest.funsuite.AnyFunSuite.org$scalatest$funsuite$AnyFunSuiteLike$$super$run(AnyFunSuite.scala:1564) [info] at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$run$1(AnyFunSuiteLike.scala:273) [info] at org.scalatest.SuperEngine.runImpl(Engine.scala:535) [info] at org.scalatest.funsuite.AnyFunSuiteLike.run(AnyFunSuiteLike.scala:273) [info] at org.scalatest.funsuite.AnyFunSuiteLike.run$(AnyFunSuiteLike.scala:272) [info] at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:69) [info] at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213) [info] at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210) [info] at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208) [info] at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:69) [info] at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:321) [info] at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:517) [info] at sbt.ForkMain$Run.lambda$runTest$1(ForkMain.java:414) [info] at java.util.concurrent.FutureTask.run(FutureTask.java:266) [info] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [info] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [info] at java.lang.Thread.run(Thread.java:750) [info] Cause: java.lang.StackOverflowError: [info] at org.apache.spark.sql.connect.client.WrappedCloseableIterator.hasNext(CloseableIterator.scala:36) [info] at org.apache.spark.sql.connect.client.WrappedCloseableIterator.hasNext(CloseableIterator.scala:36) [info] at org.apache.spark.sql.connect.client.WrappedCloseableIterator.hasNext(CloseableIterator.scala:36) [info] at org.apache.spark.sql.connect.client.WrappedCloseableIterator.hasNext(CloseableIterator.scala:36) [info] at org.apache.spark.sql.connect.client.WrappedCloseableIterator.hasNext(CloseableIterator.scala:36) [info] at org.apache.spark.sql.connect.client.WrappedCloseableIterator.hasNext(CloseableIterator.scala:36) [info] at org.apache.spark.sql.connect.client.WrappedCloseableIterator.hasNext(CloseableIterator.scala:36) [info] at org.apache.spark.sql.connect.client.WrappedCloseableIterator.hasNext(CloseableIterator.scala:36) [info] at org.apache.spark.sql.connect.client.WrappedCloseableIterator.hasNext(CloseableIterator.scala:36) [info] at org.apache.spark.sql.connect.client.WrappedCloseableIterator.hasNext(CloseableIterator.scala:36) [info] at org.apache.spark.sql.connect.client.WrappedCloseableIterator.hasNext(CloseableIterator.scala:36) [info] at org.apache.spark.sql.connect.client.WrappedCloseableIterator.hasNext(CloseableIterator.scala:36) [info] at org.apache.spark.sql.connect.client.WrappedCloseableIterator.hasNext(CloseableIterator.scala:36) [info] at org.apache.spark.sql.connect.client.WrappedCloseableIterator.hasNext(CloseableIterator.scala:36) [info] at org.apache.spark.sql.connect.client.WrappedCloseableIterator.hasNext(CloseableIterator.scala:36) [info] at org.apache.spark.sql.connect.client.WrappedCloseableIterator.hasNext(CloseableIterator.scala:36) ... [info] - client releases responses directly after consuming them (236 milliseconds) [info] - server releases responses automatically when client moves ahead (336 milliseconds) [info] - big query (863 milliseconds) [info] - big query and slow client (7 seconds, 14 milliseconds) [info] - big query with frequent reattach (735 milliseconds) [info] - big query with frequent reattach and slow client (7 seconds, 606 milliseconds) [info] - long sleeping query (10 seconds, 156 milliseconds) [info] Run completed in 34 seconds, 522 milliseconds. [info] Total number of tests run: 13 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 12, failed 1, canceled 0, ignored 0, pending 0 [info] *** 1 TEST FAILED *** [error] Failed tests: [error] org.apache.spark.sql.connect.execution.ReattachableExecuteSuite ``` **After** ``` [info] ReattachableExecuteSuite: [info] - reattach after initial RPC ends (2 seconds, 134 milliseconds) [info] - raw interrupted RPC results in INVALID_CURSOR.DISCONNECTED error (26 milliseconds) [info] - raw new RPC interrupts previous RPC with INVALID_CURSOR.DISCONNECTED error (19 milliseconds) [info] - client INVALID_CURSOR.DISCONNECTED error is retried when rpc sender gets interrupted (328 milliseconds) [info] - client INVALID_CURSOR.DISCONNECTED error is retried when other RPC preempts this one (562 milliseconds) [info] - abandoned query gets INVALID_HANDLE.OPERATION_ABANDONED error (46 milliseconds) [info] - client releases responses directly after consuming them (231 milliseconds) [info] - server releases responses automatically when client moves ahead (359 milliseconds) [info] - big query (978 milliseconds) [info] - big query and slow client (7 seconds, 50 milliseconds) [info] - big query with frequent reattach (703 milliseconds) [info] - big query with frequent reattach and slow client (7 seconds, 626 milliseconds) [info] - long sleeping query (10 seconds, 141 milliseconds) [info] Run completed in 33 seconds, 844 milliseconds. [info] Total number of tests run: 13 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 13, failed 0, canceled 0, ignored 0, pending 0 [info] All tests passed. ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #42981 from LuciferYang/CloseableIterator-apply. Authored-by: yangjie01 Signed-off-by: yangjie01 (cherry picked from commit eec090755aa5b7e6048fc004264a8f5d3591df1a) Signed-off-by: yangjie01 --- .../apache/spark/sql/connect/client/CloseableIterator.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala index d3fc9963edc7a..810158b2ac8b3 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala @@ -48,9 +48,9 @@ private[sql] object CloseableIterator { */ def apply[T](iterator: Iterator[T]): CloseableIterator[T] = iterator match { case closeable: CloseableIterator[T] => closeable - case _ => + case iter => new WrappedCloseableIterator[T] { - override def innerIterator = iterator + override def innerIterator: Iterator[T] = iter } } } From b2aead9f98d900d139cff41d53f79a37e1e09e81 Mon Sep 17 00:00:00 2001 From: Juliusz Sompolski Date: Fri, 15 Sep 2023 19:06:58 -0700 Subject: [PATCH 022/521] [SPARK-44872][CONNECT][FOLLOWUP] Deflake ReattachableExecuteSuite and increase retry buffer ### What changes were proposed in this pull request? Deflake tests in ReattachableExecuteSuite and increase CONNECT_EXECUTE_REATTACHABLE_OBSERVER_RETRY_BUFFER_SIZE. ### Why are the changes needed? Two tests could be flaky with errors `INVALID_CURSOR.POSITION_NOT_AVAILABLE`. This is caused when a server releases the response when it falls more than CONNECT_EXECUTE_REATTACHABLE_OBSERVER_RETRY_BUFFER_SIZE behind the latest response it sent. However, because of HTTP2 flow control, the responses could still be in transit. In the test suite, we were explicitly disconnecting the iterators and later reconnect... In some cases they could not reconnect, because the response they last seen have fallen too fare behind. This not only changes the suite, but also adjust the default config. This potentially makes the reconnecting more robust. In normal situation, it should not lead to increased memory pressure, because the clients also release the responses using ReleaseExecute as soon as they are received. Normally, buffered responses should be freed by ReleaseExecute and this retry buffer is only a fallback mechanism. Therefore, it is safe to increase the default. In practice, this would only have effect in cases where there are actual network errors, and the increased buffer size should make the reconnects more robust in these cases. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? ReattachableExecuteSuite. Did more manual experiments of how far the response sent by client can be behind the response sent by server (because of HTTP2 flow control window) ### Was this patch authored or co-authored using generative AI tooling? No. Closes #42908 from juliuszsompolski/SPARK-44872-followup. Authored-by: Juliusz Sompolski Signed-off-by: Dongjoon Hyun --- .../spark/sql/connect/config/Connect.scala | 2 +- .../sql/connect/SparkConnectServerTest.scala | 2 +- .../execution/ReattachableExecuteSuite.scala | 26 ++++++++++++------- 3 files changed, 18 insertions(+), 12 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala index 7b8b05ce11a82..253ac38f9cf9e 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala @@ -133,7 +133,7 @@ object Connect { "With any value greater than 0, the last sent response will always be buffered.") .version("3.5.0") .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("1m") + .createWithDefaultString("10m") val CONNECT_EXTENSIONS_RELATION_CLASSES = buildStaticConf("spark.connect.extensions.relation.classes") diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala index 488858d33ea12..eddd1c6be72b1 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.test.SharedSparkSession * Base class and utilities for a test suite that starts and tests the real SparkConnectService * with a real SparkConnectClient, communicating over RPC, but both in-process. */ -class SparkConnectServerTest extends SharedSparkSession { +trait SparkConnectServerTest extends SharedSparkSession { // Server port val serverPort: Int = diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala index 169b15582b698..0e29a07b719af 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala @@ -22,7 +22,7 @@ import io.grpc.StatusRuntimeException import org.scalatest.concurrent.Eventually import org.scalatest.time.SpanSugar._ -import org.apache.spark.SparkException +import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.sql.connect.SparkConnectServerTest import org.apache.spark.sql.connect.config.Connect import org.apache.spark.sql.connect.service.SparkConnectService @@ -32,7 +32,7 @@ class ReattachableExecuteSuite extends SparkConnectServerTest { // Tests assume that this query will result in at least a couple ExecutePlanResponses on the // stream. If this is no longer the case because of changes in how much is returned in a single // ExecutePlanResponse, it may need to be adjusted. - val MEDIUM_RESULTS_QUERY = "select * from range(1000000)" + val MEDIUM_RESULTS_QUERY = "select * from range(10000000)" test("reattach after initial RPC ends") { withClient { client => @@ -138,13 +138,12 @@ class ReattachableExecuteSuite extends SparkConnectServerTest { val reattachIter = stub.reattachExecute( buildReattachExecuteRequest(operationId, Some(response.getResponseId))) assert(reattachIter.hasNext) - reattachIter.next() - - // Nevertheless, the original iterator will handle the INVALID_CURSOR.DISCONNECTED error - iter.next() - // iterator changed because it had to reconnect - assert(reattachableIter.innerIterator ne initialInnerIter) } + + // Nevertheless, the original iterator will handle the INVALID_CURSOR.DISCONNECTED error + iter.next() + // iterator changed because it had to reconnect + assert(reattachableIter.innerIterator ne initialInnerIter) } } @@ -246,19 +245,26 @@ class ReattachableExecuteSuite extends SparkConnectServerTest { val iter = stub.executePlan( buildExecutePlanRequest(buildPlan(MEDIUM_RESULTS_QUERY), operationId = operationId)) var lastSeenResponse: String = null + val serverRetryBuffer = SparkEnv.get.conf + .get(Connect.CONNECT_EXECUTE_REATTACHABLE_OBSERVER_RETRY_BUFFER_SIZE) + .toLong iter.hasNext // open iterator val execution = getExecutionHolder // after consuming enough from the iterator, server should automatically start releasing var lastSeenIndex = 0 - while (iter.hasNext && execution.responseObserver.releasedUntilIndex == 0) { + var totalSizeSeen = 0 + while (iter.hasNext && totalSizeSeen <= 1.1 * serverRetryBuffer) { val r = iter.next() lastSeenResponse = r.getResponseId() + totalSizeSeen += r.getSerializedSize lastSeenIndex += 1 } assert(iter.hasNext) - assert(execution.responseObserver.releasedUntilIndex > 0) + Eventually.eventually(timeout(eventuallyTimeout)) { + assert(execution.responseObserver.releasedUntilIndex > 0) + } // Reattach from the beginning is not available. val reattach = stub.reattachExecute(buildReattachExecuteRequest(operationId, None)) From 71d5b110660f028bf3b097e6b5805fe68126b4cd Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Tue, 19 Sep 2023 22:56:44 -0700 Subject: [PATCH 023/521] [SPARK-45189][3.5][SQL] Creating UnresolvedRelation from TableIdentifier should include the catalog field ### What changes were proposed in this pull request? Creating UnresolvedRelation from TableIdentifier should include the catalog field ### Why are the changes needed? Fix a issue in a utility method for UnresolvedRelation ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? New unit test ### Was this patch authored or co-authored using generative AI tooling? No Closes #42998 from gengliangwang/backportTableId. Authored-by: Gengliang Wang Signed-off-by: Gengliang Wang --- .../spark/sql/catalyst/analysis/unresolved.scala | 5 ++--- .../spark/sql/catalyst/parser/PlanParserSuite.scala | 12 ++++++++++++ 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 1c72ec0d69980..b1dcb465b4778 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -104,12 +104,11 @@ object UnresolvedRelation { tableIdentifier: TableIdentifier, extraOptions: CaseInsensitiveStringMap, isStreaming: Boolean): UnresolvedRelation = { - UnresolvedRelation( - tableIdentifier.database.toSeq :+ tableIdentifier.table, extraOptions, isStreaming) + UnresolvedRelation(tableIdentifier.nameParts, extraOptions, isStreaming) } def apply(tableIdentifier: TableIdentifier): UnresolvedRelation = - UnresolvedRelation(tableIdentifier.database.toSeq :+ tableIdentifier.table) + UnresolvedRelation(tableIdentifier.nameParts) } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 4a5d0a0ae29fa..13474fe29de98 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{Decimal, DecimalType, IntegerType, LongType, StringType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap /** * Parser test cases for rules defined in [[CatalystSqlParser]] / [[AstBuilder]]. @@ -1758,4 +1759,15 @@ class PlanParserSuite extends AnalysisTest { parsePlan("SELECT * FROM a LIMIT ?"), table("a").select(star()).limit(PosParameter(22))) } + + test("SPARK-45189: Creating UnresolvedRelation from TableIdentifier should include the" + + " catalog field") { + val tableId = TableIdentifier("t", Some("db"), Some("cat")) + val unresolvedRelation = UnresolvedRelation(tableId) + assert(unresolvedRelation.multipartIdentifier == Seq("cat", "db", "t")) + val unresolvedRelation2 = UnresolvedRelation(tableId, CaseInsensitiveStringMap.empty, true) + assert(unresolvedRelation2.multipartIdentifier == Seq("cat", "db", "t")) + assert(unresolvedRelation2.options == CaseInsensitiveStringMap.empty) + assert(unresolvedRelation2.isStreaming) + } } From 6ab870f6a9915da2d9f231586b9b85b8faf94e2e Mon Sep 17 00:00:00 2001 From: Rui Wang Date: Wed, 20 Sep 2023 17:45:22 +0800 Subject: [PATCH 024/521] [SPARK-43979][SQL][FOLLOWUP] Handle non alias-only project case ### What changes were proposed in this pull request? `simplifyPlanForCollectedMetrics ` still could need to handle non alias-only project case where the project contains a mixed of aliases and attributes. In such case `simplifyPlanForCollectedMetrics` should also drop the extra project for the plan check when it contains CollectedMetrics. ### Why are the changes needed? Improve `simplifyPlanForCollectedMetrics` so it handles more plan pattern. ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? UT ### Was this patch authored or co-authored using generative AI tooling? NO Closes #42971 from amaliujia/improve_simplification. Authored-by: Rui Wang Signed-off-by: Wenchen Fan (cherry picked from commit d92d6f60342ca4d005cc2c1db94dc3b107f5d89b) Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/CheckAnalysis.scala | 4 +++- .../sql/catalyst/analysis/AnalysisSuite.scala | 14 ++++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 43546bcaa421a..139fa34a1dfcf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -1109,7 +1109,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB * remove extra project which only re-assign expr ids from the plan so that we can identify exact * duplicates metric definition. */ - private def simplifyPlanForCollectedMetrics(plan: LogicalPlan): LogicalPlan = { + def simplifyPlanForCollectedMetrics(plan: LogicalPlan): LogicalPlan = { plan.resolveOperators { case p: Project if p.projectList.size == p.child.output.size => val assignExprIdOnly = p.projectList.zipWithIndex.forall { @@ -1118,6 +1118,8 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB // ordinal of this attribute in the child outputs. So an alias-only Project means the // the id of the aliased attribute is the same as its index in the project list. attr.exprId.id == index + case (left: AttributeReference, index) => + left.exprId.id == index case _ => false } if (assignExprIdOnly) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 06c3e3eb0405a..57b37e67b32b8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -1667,4 +1667,18 @@ class AnalysisSuite extends AnalysisTest with Matchers { checkAnalysis(ident2.select($"a"), testRelation.select($"a").analyze) } } + + test("simplifyPlanForCollectedMetrics should handle non alias-only project case") { + val inner = Project( + Seq( + Alias(testRelation2.output(0), "a")(), + testRelation2.output(1), + Alias(testRelation2.output(2), "c")(), + testRelation2.output(3), + testRelation2.output(4) + ), + testRelation2) + val actualPlan = getAnalyzer.simplifyPlanForCollectedMetrics(inner.canonicalized) + assert(actualPlan == testRelation2.canonicalized) + } } From 326f8297224a0c02b2db8c8cb3f4b92cdc0dafb4 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Wed, 20 Sep 2023 08:46:53 -0700 Subject: [PATCH 025/521] [SPARK-45237][DOCS] Change the default value of `spark.history.store.hybridStore.diskBackend` in `monitoring.md` to `ROCKSDB` ### What changes were proposed in this pull request? This pr change the default value of `spark.history.store.hybridStore.diskBackend` in `monitoring.md` to `ROCKSDB` ### Why are the changes needed? SPARK-42277 change to use `RocksDB` for `spark.history.store.hybridStore.diskBackend` by default, but in `monitoring.md`, the default value is still set as `LEVELDB`. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #43015 from LuciferYang/SPARK-45237. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun (cherry picked from commit f1bc0f938162485a96de5788f53f9fa4fb37a3b1) Signed-off-by: Dongjoon Hyun --- docs/monitoring.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/monitoring.md b/docs/monitoring.md index ebd8781fd0071..91b158bf85d26 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -414,7 +414,7 @@ Security options for the Spark History Server are covered more detail in the spark.history.store.hybridStore.diskBackend - LEVELDB + ROCKSDB Specifies a disk-based store used in hybrid store; LEVELDB or ROCKSDB. From 5af0819654aca896d73c16875b07b2143cb1132c Mon Sep 17 00:00:00 2001 From: Rui Wang Date: Fri, 22 Sep 2023 11:07:25 +0800 Subject: [PATCH 026/521] [SPARK-41086][SQL] Use DataFrame ID to semantically validate CollectMetrics ### What changes were proposed in this pull request? In existing code, plan matching is used to validate if two CollectMetrics have the same name but different semantic. However, plan matching approach is fragile. A better way to tackle this is to just utilize the unique DataFrame Id. This is because observe API is only supported by DataFrame API. SQL does not have such syntax. So two CollectMetric are semantic the same if and only if they have same name and same DataFrame id. ### Why are the changes needed? This is to use a more stable approach to replace a fragile approach. ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? UT ### Was this patch authored or co-authored using generative AI tooling? NO Closes #43010 from amaliujia/another_approch_for_collect_metrics. Authored-by: Rui Wang Signed-off-by: Wenchen Fan (cherry picked from commit 7c3c7c5a4bd94c9e05b5e680a5242c2485875633) Signed-off-by: Wenchen Fan --- .../connect/planner/SparkConnectPlanner.scala | 6 +- python/pyspark/sql/connect/plan.py | 1 + .../sql/catalyst/analysis/Analyzer.scala | 4 +- .../sql/catalyst/analysis/CheckAnalysis.scala | 36 ++---------- .../plans/logical/basicLogicalOperators.scala | 3 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 55 +++++++------------ .../scala/org/apache/spark/sql/Dataset.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 2 +- 8 files changed, 35 insertions(+), 74 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 641dfc5dcd3c8..50a55f5e6411d 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -164,7 +164,7 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { case proto.Relation.RelTypeCase.CACHED_REMOTE_RELATION => transformCachedRemoteRelation(rel.getCachedRemoteRelation) case proto.Relation.RelTypeCase.COLLECT_METRICS => - transformCollectMetrics(rel.getCollectMetrics) + transformCollectMetrics(rel.getCollectMetrics, rel.getCommon.getPlanId) case proto.Relation.RelTypeCase.PARSE => transformParse(rel.getParse) case proto.Relation.RelTypeCase.RELTYPE_NOT_SET => throw new IndexOutOfBoundsException("Expected Relation to be set, but is empty.") @@ -1054,12 +1054,12 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { numPartitionsOpt) } - private def transformCollectMetrics(rel: proto.CollectMetrics): LogicalPlan = { + private def transformCollectMetrics(rel: proto.CollectMetrics, planId: Long): LogicalPlan = { val metrics = rel.getMetricsList.asScala.toSeq.map { expr => Column(transformExpression(expr)) } - CollectMetrics(rel.getName, metrics.map(_.named), transformRelation(rel.getInput)) + CollectMetrics(rel.getName, metrics.map(_.named), transformRelation(rel.getInput), planId) } private def transformDeduplicate(rel: proto.Deduplicate): LogicalPlan = { diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 196b1f119ba41..b7ea1f9499354 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -1196,6 +1196,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: assert self._child is not None plan = proto.Relation() + plan.common.plan_id = self._child._plan_id plan.collect_metrics.input.CopyFrom(self._child.plan(session)) plan.collect_metrics.name = self._name plan.collect_metrics.metrics.extend([self.col_to_expr(x, session) for x in self._exprs]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 6c5d19f58ac25..8e3c9b30c61bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -3796,9 +3796,9 @@ object CleanupAliases extends Rule[LogicalPlan] with AliasHelper { Window(cleanedWindowExprs, partitionSpec.map(trimAliases), orderSpec.map(trimAliases(_).asInstanceOf[SortOrder]), child) - case CollectMetrics(name, metrics, child) => + case CollectMetrics(name, metrics, child, dataframeId) => val cleanedMetrics = metrics.map(trimNonTopLevelAliases) - CollectMetrics(name, cleanedMetrics, child) + CollectMetrics(name, cleanedMetrics, child, dataframeId) case Unpivot(ids, values, aliases, variableColumnName, valueColumnNames, child) => val cleanedIds = ids.map(_.map(trimNonTopLevelAliases)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 139fa34a1dfcf..511f3622e7e35 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -484,7 +484,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB groupingExprs.foreach(checkValidGroupingExprs) aggregateExprs.foreach(checkValidAggregateExpression) - case CollectMetrics(name, metrics, _) => + case CollectMetrics(name, metrics, _, _) => if (name == null || name.isEmpty) { operator.failAnalysis( errorClass = "INVALID_OBSERVED_METRICS.MISSING_NAME", @@ -1075,17 +1075,15 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB * are allowed (e.g. self-joins). */ private def checkCollectedMetrics(plan: LogicalPlan): Unit = { - val metricsMap = mutable.Map.empty[String, LogicalPlan] + val metricsMap = mutable.Map.empty[String, CollectMetrics] def check(plan: LogicalPlan): Unit = plan.foreach { node => node match { - case metrics @ CollectMetrics(name, _, _) => - val simplifiedMetrics = simplifyPlanForCollectedMetrics(metrics.canonicalized) + case metrics @ CollectMetrics(name, _, _, dataframeId) => metricsMap.get(name) match { case Some(other) => - val simplifiedOther = simplifyPlanForCollectedMetrics(other.canonicalized) // Exact duplicates are allowed. They can be the result // of a CTE that is used multiple times or a self join. - if (simplifiedMetrics != simplifiedOther) { + if (dataframeId != other.dataframeId) { failAnalysis( errorClass = "DUPLICATED_METRICS_NAME", messageParameters = Map("metricName" -> name)) @@ -1104,32 +1102,6 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB check(plan) } - /** - * This method is only used for checking collected metrics. This method tries to - * remove extra project which only re-assign expr ids from the plan so that we can identify exact - * duplicates metric definition. - */ - def simplifyPlanForCollectedMetrics(plan: LogicalPlan): LogicalPlan = { - plan.resolveOperators { - case p: Project if p.projectList.size == p.child.output.size => - val assignExprIdOnly = p.projectList.zipWithIndex.forall { - case (Alias(attr: AttributeReference, _), index) => - // The input plan of this method is already canonicalized. The attribute id becomes the - // ordinal of this attribute in the child outputs. So an alias-only Project means the - // the id of the aliased attribute is the same as its index in the project list. - attr.exprId.id == index - case (left: AttributeReference, index) => - left.exprId.id == index - case _ => false - } - if (assignExprIdOnly) { - p.child - } else { - p - } - } - } - /** * Validates to make sure the outer references appearing inside the subquery * are allowed. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 4bb830662a33f..96b67fc52e0d7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -1952,7 +1952,8 @@ trait SupportsSubquery extends LogicalPlan case class CollectMetrics( name: String, metrics: Seq[NamedExpression], - child: LogicalPlan) + child: LogicalPlan, + dataframeId: Long) extends UnaryNode { override lazy val resolved: Boolean = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 57b37e67b32b8..802b6d471a65c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -771,34 +771,35 @@ class AnalysisSuite extends AnalysisTest with Matchers { val literal = Literal(1).as("lit") // Ok - assert(CollectMetrics("event", literal :: sum :: random_sum :: Nil, testRelation).resolved) + assert(CollectMetrics("event", literal :: sum :: random_sum :: Nil, testRelation, 0).resolved) // Bad name - assert(!CollectMetrics("", sum :: Nil, testRelation).resolved) + assert(!CollectMetrics("", sum :: Nil, testRelation, 0).resolved) assertAnalysisErrorClass( - CollectMetrics("", sum :: Nil, testRelation), + CollectMetrics("", sum :: Nil, testRelation, 0), expectedErrorClass = "INVALID_OBSERVED_METRICS.MISSING_NAME", expectedMessageParameters = Map( - "operator" -> "'CollectMetrics , [sum(a#x) AS sum#xL]\n+- LocalRelation , [a#x]\n") + "operator" -> + "'CollectMetrics , [sum(a#x) AS sum#xL], 0\n+- LocalRelation , [a#x]\n") ) // No columns - assert(!CollectMetrics("evt", Nil, testRelation).resolved) + assert(!CollectMetrics("evt", Nil, testRelation, 0).resolved) def checkAnalysisError(exprs: Seq[NamedExpression], errors: String*): Unit = { - assertAnalysisError(CollectMetrics("event", exprs, testRelation), errors) + assertAnalysisError(CollectMetrics("event", exprs, testRelation, 0), errors) } // Unwrapped attribute assertAnalysisErrorClass( - CollectMetrics("event", a :: Nil, testRelation), + CollectMetrics("event", a :: Nil, testRelation, 0), expectedErrorClass = "INVALID_OBSERVED_METRICS.NON_AGGREGATE_FUNC_ARG_IS_ATTRIBUTE", expectedMessageParameters = Map("expr" -> "\"a\"") ) // Unwrapped non-deterministic expression assertAnalysisErrorClass( - CollectMetrics("event", Rand(10).as("rnd") :: Nil, testRelation), + CollectMetrics("event", Rand(10).as("rnd") :: Nil, testRelation, 0), expectedErrorClass = "INVALID_OBSERVED_METRICS.NON_AGGREGATE_FUNC_ARG_IS_NON_DETERMINISTIC", expectedMessageParameters = Map("expr" -> "\"rand(10) AS rnd\"") ) @@ -808,7 +809,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { CollectMetrics( "event", Sum(a).toAggregateExpression(isDistinct = true).as("sum") :: Nil, - testRelation), + testRelation, 0), expectedErrorClass = "INVALID_OBSERVED_METRICS.AGGREGATE_EXPRESSION_WITH_DISTINCT_UNSUPPORTED", expectedMessageParameters = Map("expr" -> "\"sum(DISTINCT a) AS sum\"") @@ -819,7 +820,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { CollectMetrics( "event", Sum(Sum(a).toAggregateExpression()).toAggregateExpression().as("sum") :: Nil, - testRelation), + testRelation, 0), expectedErrorClass = "INVALID_OBSERVED_METRICS.NESTED_AGGREGATES_UNSUPPORTED", expectedMessageParameters = Map("expr" -> "\"sum(sum(a)) AS sum\"") ) @@ -830,7 +831,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { WindowSpecDefinition(Nil, a.asc :: Nil, SpecifiedWindowFrame(RowFrame, UnboundedPreceding, CurrentRow))) assertAnalysisErrorClass( - CollectMetrics("event", windowExpr.as("rn") :: Nil, testRelation), + CollectMetrics("event", windowExpr.as("rn") :: Nil, testRelation, 0), expectedErrorClass = "INVALID_OBSERVED_METRICS.WINDOW_EXPRESSIONS_UNSUPPORTED", expectedMessageParameters = Map( "expr" -> @@ -848,14 +849,14 @@ class AnalysisSuite extends AnalysisTest with Matchers { // Same result - duplicate names are allowed assertAnalysisSuccess(Union( - CollectMetrics("evt1", count :: Nil, testRelation) :: - CollectMetrics("evt1", count :: Nil, testRelation) :: Nil)) + CollectMetrics("evt1", count :: Nil, testRelation, 0) :: + CollectMetrics("evt1", count :: Nil, testRelation, 0) :: Nil)) // Same children, structurally different metrics - fail assertAnalysisErrorClass( Union( - CollectMetrics("evt1", count :: Nil, testRelation) :: - CollectMetrics("evt1", sum :: Nil, testRelation) :: Nil), + CollectMetrics("evt1", count :: Nil, testRelation, 0) :: + CollectMetrics("evt1", sum :: Nil, testRelation, 1) :: Nil), expectedErrorClass = "DUPLICATED_METRICS_NAME", expectedMessageParameters = Map("metricName" -> "evt1") ) @@ -865,17 +866,17 @@ class AnalysisSuite extends AnalysisTest with Matchers { val tblB = LocalRelation(b) assertAnalysisErrorClass( Union( - CollectMetrics("evt1", count :: Nil, testRelation) :: - CollectMetrics("evt1", count :: Nil, tblB) :: Nil), + CollectMetrics("evt1", count :: Nil, testRelation, 0) :: + CollectMetrics("evt1", count :: Nil, tblB, 1) :: Nil), expectedErrorClass = "DUPLICATED_METRICS_NAME", expectedMessageParameters = Map("metricName" -> "evt1") ) // Subquery different tree - fail - val subquery = Aggregate(Nil, sum :: Nil, CollectMetrics("evt1", count :: Nil, testRelation)) + val subquery = Aggregate(Nil, sum :: Nil, CollectMetrics("evt1", count :: Nil, testRelation, 0)) val query = Project( b :: ScalarSubquery(subquery, Nil).as("sum") :: Nil, - CollectMetrics("evt1", count :: Nil, tblB)) + CollectMetrics("evt1", count :: Nil, tblB, 1)) assertAnalysisErrorClass( query, expectedErrorClass = "DUPLICATED_METRICS_NAME", @@ -887,7 +888,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { case a: AggregateExpression => a.copy(filter = Some(true)) }.asInstanceOf[NamedExpression] assertAnalysisErrorClass( - CollectMetrics("evt1", sumWithFilter :: Nil, testRelation), + CollectMetrics("evt1", sumWithFilter :: Nil, testRelation, 0), expectedErrorClass = "INVALID_OBSERVED_METRICS.AGGREGATE_EXPRESSION_WITH_FILTER_UNSUPPORTED", expectedMessageParameters = Map("expr" -> "\"sum(a) FILTER (WHERE true) AS sum\"") @@ -1667,18 +1668,4 @@ class AnalysisSuite extends AnalysisTest with Matchers { checkAnalysis(ident2.select($"a"), testRelation.select($"a").analyze) } } - - test("simplifyPlanForCollectedMetrics should handle non alias-only project case") { - val inner = Project( - Seq( - Alias(testRelation2.output(0), "a")(), - testRelation2.output(1), - Alias(testRelation2.output(2), "c")(), - testRelation2.output(3), - testRelation2.output(4) - ), - testRelation2) - val actualPlan = getAnalyzer.simplifyPlanForCollectedMetrics(inner.canonicalized) - assert(actualPlan == testRelation2.canonicalized) - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index fd8421fa096cc..e047b927b9057 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2189,7 +2189,7 @@ class Dataset[T] private[sql]( */ @varargs def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = withTypedPlan { - CollectMetrics(name, (expr +: exprs).map(_.named), logicalPlan) + CollectMetrics(name, (expr +: exprs).map(_.named), logicalPlan, id) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 903565a6d591b..d851eacd5ab92 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -935,7 +935,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { throw QueryExecutionErrors.ddlUnsupportedTemporarilyError("UPDATE TABLE") case _: MergeIntoTable => throw QueryExecutionErrors.ddlUnsupportedTemporarilyError("MERGE INTO TABLE") - case logical.CollectMetrics(name, metrics, child) => + case logical.CollectMetrics(name, metrics, child, _) => execution.CollectMetricsExec(name, metrics, planLater(child)) :: Nil case WriteFiles(child, fileFormat, partitionColumns, bucket, options, staticPartitions) => WriteFilesExec(planLater(child), fileFormat, partitionColumns, bucket, options, From 609306ff5daa8ff7c2212088d33c0911ad0f4989 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 24 Sep 2023 14:17:55 -0500 Subject: [PATCH 027/521] [SPARK-45286][DOCS] Add back Matomo analytics ### What changes were proposed in this pull request? Add analytics to doc pages using the ASF's Matomo service ### Why are the changes needed? We had previously removed Google Analytics from the website and release docs, per ASF policy: https://github.com/apache/spark/pull/36310 We just restored analytics using the ASF-hosted Matomo service on the website: https://github.com/apache/spark-website/commit/a1548627b48a62c2e51870d1488ca3e09397bd30 This change would put the same new tracking code back into the release docs. It would let us see what docs and resources are most used, I suppose. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? N/A ### Was this patch authored or co-authored using generative AI tooling? No Closes #43063 from srowen/SPARK-45286. Authored-by: Sean Owen Signed-off-by: Sean Owen (cherry picked from commit a881438114ea3e8e918d981ef89ed1ab956d6fca) Signed-off-by: Sean Owen --- docs/_layouts/global.html | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 9b7c469246165..8c4435fdf31d9 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -32,6 +32,25 @@ + {% production %} + + + + {% endproduction %} + Shuffle 2 ----> ResultStage - ShuffleMapStage1 is complete - A task from ShuffleMapStage2 fails with FetchFailed. Other tasks are still running - Both ShuffleMapStage1 and ShuffleMapStage2 are retried - ShuffleMapStage1 is retried and completes - ShuffleMapStage2 reattempt is scheduled for execution - Before all tasks of ShuffleMapStage2 reattempt could finish, one/more laggard tasks from the original attempt of ShuffleMapStage2 finish and ShuffleMapStage2 also gets marked as complete - Result Stage gets scheduled and finishes After this change, such laggard tasks from the old attempt of the indeterminate stage will be ignored ### Why are the changes needed? This can give wrong result when indeterminate stages needs to be retried under the circumstances mentioned above ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? A new test case ### Was this patch authored or co-authored using generative AI tooling? No Closes #42950 from mayurdb/rollbackFix. Authored-by: mayurb Signed-off-by: Wenchen Fan (cherry picked from commit 7ffc0b71aa3e416a9b21e0975a169b2a8a8403a8) Signed-off-by: Wenchen Fan --- .../apache/spark/scheduler/DAGScheduler.scala | 29 +++-- .../spark/scheduler/DAGSchedulerSuite.scala | 104 ++++++++++++++++++ 2 files changed, 122 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index fc83439454dcf..d73bb6339015b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1903,19 +1903,26 @@ private[spark] class DAGScheduler( case smt: ShuffleMapTask => val shuffleStage = stage.asInstanceOf[ShuffleMapStage] - shuffleStage.pendingPartitions -= task.partitionId - val status = event.result.asInstanceOf[MapStatus] - val execId = status.location.executorId - logDebug("ShuffleMapTask finished on " + execId) - if (executorFailureEpoch.contains(execId) && + // Ignore task completion for old attempt of indeterminate stage + val ignoreIndeterminate = stage.isIndeterminate && + task.stageAttemptId < stage.latestInfo.attemptNumber() + if (!ignoreIndeterminate) { + shuffleStage.pendingPartitions -= task.partitionId + val status = event.result.asInstanceOf[MapStatus] + val execId = status.location.executorId + logDebug("ShuffleMapTask finished on " + execId) + if (executorFailureEpoch.contains(execId) && smt.epoch <= executorFailureEpoch(execId)) { - logInfo(s"Ignoring possibly bogus $smt completion from executor $execId") + logInfo(s"Ignoring possibly bogus $smt completion from executor $execId") + } else { + // The epoch of the task is acceptable (i.e., the task was launched after the most + // recent failure we're aware of for the executor), so mark the task's output as + // available. + mapOutputTracker.registerMapOutput( + shuffleStage.shuffleDep.shuffleId, smt.partitionId, status) + } } else { - // The epoch of the task is acceptable (i.e., the task was launched after the most - // recent failure we're aware of for the executor), so mark the task's output as - // available. - mapOutputTracker.registerMapOutput( - shuffleStage.shuffleDep.shuffleId, smt.partitionId, status) + logInfo(s"Ignoring $smt completion from an older attempt of indeterminate stage") } if (runningStages.contains(shuffleStage) && shuffleStage.pendingPartitions.isEmpty) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index c7e4994e328f4..e351f8b95bbb0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -3041,6 +3041,27 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti (shuffleId1, shuffleId2) } + private def constructTwoIndeterminateStage(): (Int, Int) = { + val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, indeterminate = true) + + val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, new HashPartitioner(2)) + val shuffleId1 = shuffleDep1.shuffleId + val shuffleMapRdd2 = new MyRDD(sc, 2, List(shuffleDep1), tracker = mapOutputTracker, + indeterminate = true) + + val shuffleDep2 = new ShuffleDependency(shuffleMapRdd2, new HashPartitioner(2)) + val shuffleId2 = shuffleDep2.shuffleId + val finalRdd = new MyRDD(sc, 2, List(shuffleDep2), tracker = mapOutputTracker) + + submit(finalRdd, Array(0, 1)) + + // Finish the first shuffle map stage. + completeShuffleMapStageSuccessfully(0, 0, 2) + assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) + + (shuffleId1, shuffleId2) + } + test("SPARK-25341: abort stage while using old fetch protocol") { conf.set(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL.key, "true") // Construct the scenario of indeterminate stage fetch failed. @@ -3099,6 +3120,89 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti assertDataStructuresEmpty() } + test("SPARK-45182: Ignore task completion from old stage after retrying indeterminate stages") { + val (shuffleId1, shuffleId2) = constructTwoIndeterminateStage() + + // shuffleMapStage0 -> shuffleId1 -> shuffleMapStage1 -> shuffleId2 -> resultStage + val shuffleMapStage1 = scheduler.stageIdToStage(1).asInstanceOf[ShuffleMapStage] + val resultStage = scheduler.stageIdToStage(2).asInstanceOf[ResultStage] + + // Shuffle map stage 0 is done + assert(mapOutputTracker.findMissingPartitions(shuffleId1) == Some(Seq.empty)) + // Shuffle map stage 1 is still waiting for its 2 tasks to complete + assert(mapOutputTracker.findMissingPartitions(shuffleId2) == Some(Seq(0, 1))) + // The result stage is still waiting for its 2 tasks to complete + assert(resultStage.findMissingPartitions() == Seq(0, 1)) + + scheduler.resubmitFailedStages() + + // The first task of the shuffle map stage 1 fails with fetch failure + runEvent(makeCompletionEvent( + taskSets(1).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0L, 0, 0, "ignored"), + null)) + + // Both the stages should have been resubmitted + val newFailedStages = scheduler.failedStages.toSeq + assert(newFailedStages.map(_.id) == Seq(0, 1)) + + scheduler.resubmitFailedStages() + + // Since shuffleId1 is indeterminate, all tasks of shuffle map stage 0 should be ran + assert(taskSets(2).stageId == 0) + assert(taskSets(2).stageAttemptId == 1) + assert(taskSets(2).tasks.length == 2) + + // Complete the re-attempt of shuffle map stage 0 + completeShuffleMapStageSuccessfully(0, 1, 2) + assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) + + // Since shuffleId2 is indeterminate, all tasks of shuffle map stage 1 should be ran + assert(taskSets(3).stageId == 1) + assert(taskSets(3).stageAttemptId == 1) + assert(taskSets(3).tasks.length == 2) + + // The first task of the shuffle map stage 1 from 2nd attempt succeeds + runEvent(makeCompletionEvent( + taskSets(3).tasks(0), + Success, + makeMapStatus("hostB", + 2))) + + // The second task of the shuffle map stage 1 from 1st attempt succeeds + runEvent(makeCompletionEvent( + taskSets(1).tasks(1), + Success, + makeMapStatus("hostC", + 2))) + + // This task completion should get ignored and partition 1 should be missing + // for shuffle map stage 1 + assert(mapOutputTracker.findMissingPartitions(shuffleId2) == Some(Seq(1))) + + // The second task of the shuffle map stage 1 from 2nd attempt succeeds + runEvent(makeCompletionEvent( + taskSets(3).tasks(1), + Success, + makeMapStatus("hostD", + 2))) + + // The shuffle map stage 1 should be done + assert(mapOutputTracker.findMissingPartitions(shuffleId2) === Some(Seq.empty)) + + // The shuffle map outputs for shuffleId1 should be from latest attempt of shuffle map stage 1 + assert(mapOutputTracker.getMapLocation(shuffleMapStage1.shuffleDep, 0, 2) + === Seq("hostB", "hostD")) + + // Complete result stage + complete(taskSets(4), Seq((Success, 11), (Success, 12))) + + // Job successfully ended + assert(results === Map(0 -> 11, 1 -> 12)) + results.clear() + assertDataStructuresEmpty() + } + test("SPARK-25341: continuous indeterminate stage roll back") { // shuffleMapRdd1/2/3 are all indeterminate. val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, indeterminate = true) From eb6b68daa4ef237d92575f799bafd97b1c3615b5 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 27 Sep 2023 16:00:11 +0800 Subject: [PATCH 031/521] [SPARK-45346][SQL] Parquet schema inference should respect case sensitive flag when merging schema ### What changes were proposed in this pull request? Currently, when we infer schema from parquet files and try to merge the schema, it's always case-sensitive. Then a check fails later which tries to make sure the data schema of parquet fields does not have duplicated columns, in a case-insensitive way (the default). This PR fixes the problem and make the schema merging respect the case sensitivity flag. ### Why are the changes needed? bug fix ### Does this PR introduce _any_ user-facing change? yes, spark can read some parquet files now. ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #43134 from cloud-fan/merge-schema. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit 1cbc424ae2acaf4d82f928cfea2767c81425305e) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/types/StructType.scala | 28 +++++++++++++------ .../datasources/SchemaMergeUtils.scala | 5 ++-- .../parquet/ParquetSchemaSuite.scala | 21 ++++++++++++++ 3 files changed, 43 insertions(+), 11 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala index 8edc7cf370b7d..8fd7f47b34624 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.types +import java.util.Locale + import scala.collection.{mutable, Map} import scala.util.Try import scala.util.control.NonFatal @@ -476,8 +478,8 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru * 4. Otherwise, `this` and `that` are considered as conflicting schemas and an exception would be * thrown. */ - private[sql] def merge(that: StructType): StructType = - StructType.merge(this, that).asInstanceOf[StructType] + private[sql] def merge(that: StructType, caseSensitive: Boolean = true): StructType = + StructType.merge(this, that, caseSensitive).asInstanceOf[StructType] override private[spark] def asNullable: StructType = { val newFields = fields.map { @@ -561,16 +563,20 @@ object StructType extends AbstractDataType { StructType(newFields) }) - private[sql] def merge(left: DataType, right: DataType): DataType = + private[sql] def merge(left: DataType, right: DataType, caseSensitive: Boolean = true): DataType = mergeInternal(left, right, (s1: StructType, s2: StructType) => { val leftFields = s1.fields val rightFields = s2.fields val newFields = mutable.ArrayBuffer.empty[StructField] - val rightMapped = fieldsMap(rightFields) + def normalize(name: String): String = { + if (caseSensitive) name else name.toLowerCase(Locale.ROOT) + } + + val rightMapped = fieldsMap(rightFields, caseSensitive) leftFields.foreach { case leftField @ StructField(leftName, leftType, leftNullable, _) => - rightMapped.get(leftName) + rightMapped.get(normalize(leftName)) .map { case rightField @ StructField(rightName, rightType, rightNullable, _) => try { leftField.copy( @@ -588,9 +594,9 @@ object StructType extends AbstractDataType { .foreach(newFields += _) } - val leftMapped = fieldsMap(leftFields) + val leftMapped = fieldsMap(leftFields, caseSensitive) rightFields - .filterNot(f => leftMapped.get(f.name).nonEmpty) + .filterNot(f => leftMapped.contains(normalize(f.name))) .foreach { f => newFields += f } @@ -643,11 +649,15 @@ object StructType extends AbstractDataType { throw DataTypeErrors.cannotMergeIncompatibleDataTypesError(left, right) } - private[sql] def fieldsMap(fields: Array[StructField]): Map[String, StructField] = { + private[sql] def fieldsMap( + fields: Array[StructField], + caseSensitive: Boolean = true): Map[String, StructField] = { // Mimics the optimization of breakOut, not present in Scala 2.13, while working in 2.12 val map = mutable.Map[String, StructField]() map.sizeHint(fields.length) - fields.foreach(s => map.put(s.name, s)) + fields.foreach { s => + if (caseSensitive) map.put(s.name, s) else map.put(s.name.toLowerCase(Locale.ROOT), s) + } map } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaMergeUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaMergeUtils.scala index 35d9b5d60348d..cf0e67ecc30fa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaMergeUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaMergeUtils.scala @@ -64,6 +64,7 @@ object SchemaMergeUtils extends Logging { val ignoreCorruptFiles = new FileSourceOptions(CaseInsensitiveMap(parameters)).ignoreCorruptFiles + val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis // Issues a Spark job to read Parquet/ORC schema in parallel. val partiallyMergedSchemas = @@ -84,7 +85,7 @@ object SchemaMergeUtils extends Logging { var mergedSchema = schemas.head schemas.tail.foreach { schema => try { - mergedSchema = mergedSchema.merge(schema) + mergedSchema = mergedSchema.merge(schema, caseSensitive) } catch { case cause: SparkException => throw QueryExecutionErrors.failedMergingSchemaError(mergedSchema, schema, cause) } @@ -99,7 +100,7 @@ object SchemaMergeUtils extends Logging { var finalSchema = partiallyMergedSchemas.head partiallyMergedSchemas.tail.foreach { schema => try { - finalSchema = finalSchema.merge(schema) + finalSchema = finalSchema.merge(schema, caseSensitive) } catch { case cause: SparkException => throw QueryExecutionErrors.failedMergingSchemaError(finalSchema, schema, cause) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index 30f46a3cac2d3..facc9b90ff778 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -996,6 +996,27 @@ class ParquetSchemaSuite extends ParquetSchemaTest { } } + test("SPARK-45346: merge schema should respect case sensitivity") { + import testImplicits._ + Seq(true, false).foreach { caseSensitive => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + withTempPath { path => + Seq(1).toDF("col").write.mode("append").parquet(path.getCanonicalPath) + Seq(2).toDF("COL").write.mode("append").parquet(path.getCanonicalPath) + val df = spark.read.option("mergeSchema", "true").parquet(path.getCanonicalPath) + if (caseSensitive) { + assert(df.columns.toSeq.sorted == Seq("COL", "col")) + assert(df.collect().length == 2) + } else { + // The final column name depends on which file is listed first, and is a bit random. + assert(df.columns.toSeq.map(_.toLowerCase(java.util.Locale.ROOT)) == Seq("col")) + assert(df.collect().length == 2) + } + } + } + } + } + // ======================================= // Tests for parquet schema mismatch error // ======================================= From e6e0c074636c1a43fa5957f906a881469515393a Mon Sep 17 00:00:00 2001 From: zeruibao Date: Wed, 27 Sep 2023 16:42:35 +0800 Subject: [PATCH 032/521] [SPARK-43380][SQL] Fix Avro data type conversion issues without causing performance regression ### What changes were proposed in this pull request? My last PR https://github.com/apache/spark/pull/41052 causes AVRO read performance regression since I change the code structure. I turn one match case into a nested match case. So I fix the Avro data type conversion issues in anther way to avoid this regression. Original Change: We introduce the SQLConf `spark.sql.legacy.avro.allowReadingWithIncompatibleSchema` to prevent reading interval types as date or timestamp types to avoid getting corrupt dates as well as reading decimal types with incorrect precision. ### Why are the changes needed? We found the following issues with open source Avro: - Interval types can be read as date or timestamp types that would lead to wildly different results For example, `Duration.ofDays(1).plusSeconds(1)` will be read as `1972-09-27`, which is weird. - Decimal types can be read with lower precision, that leads to data being read as `null` instead of suggesting that a wider decimal format should be provided ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Old unit test Closes #42503 from zeruibao/SPARK-4380-real-fix-regression. Lead-authored-by: zeruibao Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit f8c87f03297e2770e2944e8e8fe097b75f9e8fea) Signed-off-by: Wenchen Fan --- .../main/resources/error/error-classes.json | 5 + .../spark/sql/avro/AvroDeserializer.scala | 46 ++++- .../org/apache/spark/sql/avro/AvroSuite.scala | 158 ++++++++++++++++++ docs/sql-error-conditions.md | 6 + docs/sql-migration-guide.md | 1 + .../sql/errors/QueryCompilationErrors.scala | 16 ++ .../apache/spark/sql/internal/SQLConf.scala | 12 ++ 7 files changed, 235 insertions(+), 9 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 477fe9b3f614e..9bc65ae32a276 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -69,6 +69,11 @@ } } }, + "AVRO_INCOMPATIBLE_READ_TYPE" : { + "message" : [ + "Cannot convert Avro to SQL because the original encoded data type is , however you're trying to read the field as , which would lead to an incorrect answer. To allow reading this field, enable the SQL configuration: \"spark.sql.legacy.avro.allowIncompatibleSchema\"." + ] + }, "BATCH_METADATA_NOT_FOUND" : { "message" : [ "Unable to find batch ." diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala index a78ee89a3e933..e82116eec1e9c 100644 --- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala @@ -35,8 +35,9 @@ import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeArr import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.catalyst.util.DateTimeConstants.MILLIS_PER_DAY import org.apache.spark.sql.catalyst.util.RebaseDateTime.RebaseSpec +import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.datasources.DataSourceUtils -import org.apache.spark.sql.internal.LegacyBehaviorPolicy +import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -117,6 +118,10 @@ private[sql] class AvroDeserializer( val incompatibleMsg = errorPrefix + s"schema is incompatible (avroType = $avroType, sqlType = ${catalystType.sql})" + val realDataType = SchemaConverters.toSqlType(avroType).dataType + val confKey = SQLConf.LEGACY_AVRO_ALLOW_INCOMPATIBLE_SCHEMA + val preventReadingIncorrectType = !SQLConf.get.getConf(confKey) + (avroType.getType, catalystType) match { case (NULL, NullType) => (updater, ordinal, _) => updater.setNullAt(ordinal) @@ -128,9 +133,19 @@ private[sql] class AvroDeserializer( case (INT, IntegerType) => (updater, ordinal, value) => updater.setInt(ordinal, value.asInstanceOf[Int]) + case (INT, dt: DatetimeType) + if preventReadingIncorrectType && realDataType.isInstanceOf[YearMonthIntervalType] => + throw QueryCompilationErrors.avroIncompatibleReadError(toFieldStr(avroPath), + toFieldStr(catalystPath), realDataType.catalogString, dt.catalogString) + case (INT, DateType) => (updater, ordinal, value) => updater.setInt(ordinal, dateRebaseFunc(value.asInstanceOf[Int])) + case (LONG, dt: DatetimeType) + if preventReadingIncorrectType && realDataType.isInstanceOf[DayTimeIntervalType] => + throw QueryCompilationErrors.avroIncompatibleReadError(toFieldStr(avroPath), + toFieldStr(catalystPath), realDataType.catalogString, dt.catalogString) + case (LONG, LongType) => (updater, ordinal, value) => updater.setLong(ordinal, value.asInstanceOf[Long]) @@ -204,17 +219,30 @@ private[sql] class AvroDeserializer( } updater.set(ordinal, bytes) - case (FIXED, _: DecimalType) => (updater, ordinal, value) => + case (FIXED, dt: DecimalType) => val d = avroType.getLogicalType.asInstanceOf[LogicalTypes.Decimal] - val bigDecimal = decimalConversions.fromFixed(value.asInstanceOf[GenericFixed], avroType, d) - val decimal = createDecimal(bigDecimal, d.getPrecision, d.getScale) - updater.setDecimal(ordinal, decimal) + if (preventReadingIncorrectType && + d.getPrecision - d.getScale > dt.precision - dt.scale) { + throw QueryCompilationErrors.avroIncompatibleReadError(toFieldStr(avroPath), + toFieldStr(catalystPath), realDataType.catalogString, dt.catalogString) + } + (updater, ordinal, value) => + val bigDecimal = + decimalConversions.fromFixed(value.asInstanceOf[GenericFixed], avroType, d) + val decimal = createDecimal(bigDecimal, d.getPrecision, d.getScale) + updater.setDecimal(ordinal, decimal) - case (BYTES, _: DecimalType) => (updater, ordinal, value) => + case (BYTES, dt: DecimalType) => val d = avroType.getLogicalType.asInstanceOf[LogicalTypes.Decimal] - val bigDecimal = decimalConversions.fromBytes(value.asInstanceOf[ByteBuffer], avroType, d) - val decimal = createDecimal(bigDecimal, d.getPrecision, d.getScale) - updater.setDecimal(ordinal, decimal) + if (preventReadingIncorrectType && + d.getPrecision - d.getScale > dt.precision - dt.scale) { + throw QueryCompilationErrors.avroIncompatibleReadError(toFieldStr(avroPath), + toFieldStr(catalystPath), realDataType.catalogString, dt.catalogString) + } + (updater, ordinal, value) => + val bigDecimal = decimalConversions.fromBytes(value.asInstanceOf[ByteBuffer], avroType, d) + val decimal = createDecimal(bigDecimal, d.getPrecision, d.getScale) + updater.setDecimal(ordinal, decimal) case (RECORD, st: StructType) => // Avro datasource doesn't accept filters with nested attributes. See SPARK-32328. diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index d22a2d3697579..ffb0a49641b59 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -32,6 +32,7 @@ import org.apache.avro.file.{DataFileReader, DataFileWriter} import org.apache.avro.generic.{GenericData, GenericDatumReader, GenericDatumWriter, GenericRecord} import org.apache.avro.generic.GenericData.{EnumSymbol, Fixed} import org.apache.commons.io.FileUtils +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.spark.{SPARK_VERSION_SHORT, SparkConf, SparkException, SparkUpgradeException} import org.apache.spark.TestUtils.assertExceptionMsg @@ -814,6 +815,163 @@ abstract class AvroSuite } } + test("SPARK-43380: Fix Avro data type conversion" + + " of decimal type to avoid producing incorrect results") { + withTempPath { path => + val confKey = SQLConf.LEGACY_AVRO_ALLOW_INCOMPATIBLE_SCHEMA.key + sql("SELECT 13.1234567890 a").write.format("avro").save(path.toString) + // With the flag disabled, we will throw an exception if there is a mismatch + withSQLConf(confKey -> "false") { + val e = intercept[SparkException] { + spark.read.schema("a DECIMAL(4, 3)").format("avro").load(path.toString).collect() + } + ExceptionUtils.getRootCause(e) match { + case ex: AnalysisException => + checkError( + exception = ex, + errorClass = "AVRO_INCOMPATIBLE_READ_TYPE", + parameters = Map("avroPath" -> "field 'a'", + "sqlPath" -> "field 'a'", + "avroType" -> "decimal\\(12,10\\)", + "sqlType" -> "\"DECIMAL\\(4,3\\)\""), + matchPVals = true + ) + case other => + fail(s"Received unexpected exception", other) + } + } + // The following used to work, so it should still work with the flag enabled + checkAnswer( + spark.read.schema("a DECIMAL(5, 3)").format("avro").load(path.toString), + Row(new java.math.BigDecimal("13.123")) + ) + withSQLConf(confKey -> "true") { + // With the flag enabled, we return a null silently, which isn't great + checkAnswer( + spark.read.schema("a DECIMAL(4, 3)").format("avro").load(path.toString), + Row(null) + ) + checkAnswer( + spark.read.schema("a DECIMAL(5, 3)").format("avro").load(path.toString), + Row(new java.math.BigDecimal("13.123")) + ) + } + } + } + + test("SPARK-43380: Fix Avro data type conversion" + + " of DayTimeIntervalType to avoid producing incorrect results") { + withTempPath { path => + val confKey = SQLConf.LEGACY_AVRO_ALLOW_INCOMPATIBLE_SCHEMA.key + val schema = StructType(Array(StructField("a", DayTimeIntervalType(), false))) + val data = Seq(Row(java.time.Duration.ofDays(1).plusSeconds(1))) + + val df = spark.createDataFrame(sparkContext.parallelize(data), schema) + df.write.format("avro").save(path.getCanonicalPath) + + withSQLConf(confKey -> "false") { + Seq("DATE", "TIMESTAMP", "TIMESTAMP_NTZ").foreach { sqlType => + val e = intercept[SparkException] { + spark.read.schema(s"a $sqlType").format("avro").load(path.toString).collect() + } + + ExceptionUtils.getRootCause(e) match { + case ex: AnalysisException => + checkError( + exception = ex, + errorClass = "AVRO_INCOMPATIBLE_READ_TYPE", + parameters = Map("avroPath" -> "field 'a'", + "sqlPath" -> "field 'a'", + "avroType" -> "interval day to second", + "sqlType" -> s""""$sqlType""""), + matchPVals = true + ) + case other => + fail(s"Received unexpected exception", other) + } + } + } + + withSQLConf(confKey -> "true") { + // Allow conversion and do not need to check result + spark.read.schema("a Date").format("avro").load(path.toString) + spark.read.schema("a timestamp").format("avro").load(path.toString) + spark.read.schema("a timestamp_ntz").format("avro").load(path.toString) + } + } + } + + test("SPARK-43380: Fix Avro data type conversion" + + " of YearMonthIntervalType to avoid producing incorrect results") { + withTempPath { path => + val confKey = SQLConf.LEGACY_AVRO_ALLOW_INCOMPATIBLE_SCHEMA.key + val schema = StructType(Array(StructField("a", YearMonthIntervalType(), false))) + val data = Seq(Row(java.time.Period.of(1, 1, 0))) + + val df = spark.createDataFrame(sparkContext.parallelize(data), schema) + df.write.format("avro").save(path.getCanonicalPath) + + withSQLConf(confKey -> "false") { + Seq("DATE", "TIMESTAMP", "TIMESTAMP_NTZ").foreach { sqlType => + val e = intercept[SparkException] { + spark.read.schema(s"a $sqlType").format("avro").load(path.toString).collect() + } + + ExceptionUtils.getRootCause(e) match { + case ex: AnalysisException => + checkError( + exception = ex, + errorClass = "AVRO_INCOMPATIBLE_READ_TYPE", + parameters = Map("avroPath" -> "field 'a'", + "sqlPath" -> "field 'a'", + "avroType" -> "interval year to month", + "sqlType" -> s""""$sqlType""""), + matchPVals = true + ) + case other => + fail(s"Received unexpected exception", other) + } + } + } + + withSQLConf(confKey -> "true") { + // Allow conversion and do not need to check result + spark.read.schema("a Date").format("avro").load(path.toString) + spark.read.schema("a timestamp").format("avro").load(path.toString) + spark.read.schema("a timestamp_ntz").format("avro").load(path.toString) + } + } + } + + Seq( + "time-millis", + "time-micros", + "timestamp-micros", + "timestamp-millis", + "local-timestamp-millis", + "local-timestamp-micros" + ).foreach { timeLogicalType => + test(s"converting $timeLogicalType type to long in avro") { + withTempPath { path => + val df = Seq(100L) + .toDF("dt") + val avroSchema = + s""" + |{ + | "type" : "record", + | "name" : "test_schema", + | "fields" : [ + | {"name": "dt", "type": {"type": "long", "logicalType": "$timeLogicalType"}} + | ] + |}""".stripMargin + df.write.format("avro").option("avroSchema", avroSchema).save(path.getCanonicalPath) + checkAnswer( + spark.read.schema(s"dt long").format("avro").load(path.toString), + Row(100L)) + } + } + } + test("converting some specific sparkSQL types to avro") { withTempPath { tempDir => val testSchema = StructType(Seq( diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index e7df1aa9a4f9c..90d21f9758573 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -87,6 +87,12 @@ Invalid as-of join. For more details see [AS_OF_JOIN](sql-error-conditions-as-of-join-error-class.html) +### AVRO_INCOMPATIBLE_READ_TYPE + +SQLSTATE: none assigned + +Cannot convert Avro `` to SQL `` because the original encoded data type is ``, however you're trying to read the field as ``, which would lead to an incorrect answer. To allow reading this field, enable the SQL configuration: "spark.sql.legacy.avro.allowIncompatibleSchema". + ### BATCH_METADATA_NOT_FOUND [SQLSTATE: 42K03](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 5fc323ec1b0ea..5cf0b28982c24 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -30,6 +30,7 @@ license: | - Since Spark 3.5, the `plan` field is moved from `AnalysisException` to `EnhancedAnalysisException`. - Since Spark 3.5, `spark.sql.optimizer.canChangeCachedPlanOutputPartitioning` is enabled by default. To restore the previous behavior, set `spark.sql.optimizer.canChangeCachedPlanOutputPartitioning` to `false`. - Since Spark 3.5, the `array_insert` function is 1-based for negative indexes. It inserts new element at the end of input arrays for the index -1. To restore the previous behavior, set `spark.sql.legacy.negativeIndexInArrayInsert` to `true`. +- Since Spark 3.5, the Avro will throw `AnalysisException` when reading Interval types as Date or Timestamp types, or reading Decimal types with lower precision. To restore the legacy behavior, set `spark.sql.legacy.avro.allowIncompatibleSchema` to `true` ## Upgrading from Spark SQL 3.3 to 3.4 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 14882a7006173..9dca2c5f2822e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -3672,6 +3672,22 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat ) } + def avroIncompatibleReadError( + avroPath: String, + sqlPath: String, + avroType: String, + sqlType: String): Throwable = { + new AnalysisException( + errorClass = "AVRO_INCOMPATIBLE_READ_TYPE", + messageParameters = Map( + "avroPath" -> avroPath, + "sqlPath" -> sqlPath, + "avroType" -> avroType, + "sqlType" -> toSQLType(sqlType) + ) + ) + } + def optionMustBeLiteralString(key: String): Throwable = { new AnalysisException( errorClass = "INVALID_SQL_SYNTAX.OPTION_IS_INVALID", diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 2e0ce7c4dea9d..73d3756ef6b93 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -4251,6 +4251,18 @@ object SQLConf { .booleanConf .createWithDefault(false) + val LEGACY_AVRO_ALLOW_INCOMPATIBLE_SCHEMA = + buildConf("spark.sql.legacy.avro.allowIncompatibleSchema") + .internal() + .doc("When set to false, if types in Avro are encoded in the same format, but " + + "the type in the Avro schema explicitly says that the data types are different, " + + "reject reading the data type in the format to avoid returning incorrect results. " + + "When set to true, it restores the legacy behavior of allow reading the data in the" + + " format, which may return incorrect results.") + .version("3.5.1") + .booleanConf + .createWithDefault(false) + val LEGACY_NON_IDENTIFIER_OUTPUT_CATALOG_NAME = buildConf("spark.sql.legacy.v1IdentifierNoCatalog") .internal() From 917bc8cb92728267fb93891f4ef9da13c06e4589 Mon Sep 17 00:00:00 2001 From: Yihong He Date: Thu, 28 Sep 2023 12:58:07 -0400 Subject: [PATCH 033/521] [SPARK-45360][SQL][CONNECT] Initialize spark session builder configuration from SPARK_REMOTE ### What changes were proposed in this pull request? - Initialize spark session builder configuration from SPARK_REMOTE ### Why are the changes needed? - `SparkSession.builder().getOrCreate()` should follow the behavior documents [here](https://github.com/apache/spark/blob/2cc1ee4d3a05a641d7a245f015ef824d8f7bae8b/docs/spark-connect-overview.md?plain=1#L241-L244) and support initialization from SPARK_REMOTE ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - `build/sbt "connect-client-jvm/testOnly *SparkConnectClientSuite"` ### Was this patch authored or co-authored using generative AI tooling? Closes #43153 from heyihong/SPARK-45360. Authored-by: Yihong He Signed-off-by: Herman van Hovell (cherry picked from commit 183a3d761f36d35572cfb37ab921b6a86f8f28ed) Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/SparkSession.scala | 5 +- .../client/SparkConnectClientSuite.scala | 61 +++++++++++++++++++ 2 files changed, 65 insertions(+), 1 deletion(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 7bd8fa59aea8f..421f37b9e8a62 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -783,7 +783,10 @@ object SparkSession extends Logging { } class Builder() extends Logging { - private val builder = SparkConnectClient.builder() + // Initialize the connection string of the Spark Connect client builder from SPARK_REMOTE + // by default, if it exists. The connection string can be overridden using + // the remote() function, as it takes precedence over the SPARK_REMOTE environment variable. + private val builder = SparkConnectClient.builder().loadFromEnvironment() private var client: SparkConnectClient = _ private[this] val options = new scala.collection.mutable.HashMap[String, String] diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala index 80e245ec78b7d..89acc2c60ac21 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala @@ -86,6 +86,24 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach { assert(response.getSessionId === "abc123") } + private def withEnvs(pairs: (String, String)*)(f: => Unit): Unit = { + val readonlyEnv = System.getenv() + val field = readonlyEnv.getClass.getDeclaredField("m") + field.setAccessible(true) + val modifiableEnv = field.get(readonlyEnv).asInstanceOf[java.util.Map[String, String]] + try { + for ((k, v) <- pairs) { + assert(!modifiableEnv.containsKey(k)) + modifiableEnv.put(k, v) + } + f + } finally { + for ((k, _) <- pairs) { + modifiableEnv.remove(k) + } + } + } + test("Test connection") { testClientConnection() { testPort => SparkConnectClient.builder().port(testPort).build() } } @@ -112,6 +130,49 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach { } } + test("SparkSession create with SPARK_REMOTE") { + startDummyServer(0) + + withEnvs("SPARK_REMOTE" -> s"sc://localhost:${server.getPort}") { + val session = SparkSession.builder().create() + val df = session.range(10) + df.analyze // Trigger RPC + assert(df.plan === service.getAndClearLatestInputPlan()) + + val session2 = SparkSession.builder().create() + assert(session != session2) + } + } + + test("SparkSession getOrCreate with SPARK_REMOTE") { + startDummyServer(0) + + withEnvs("SPARK_REMOTE" -> s"sc://localhost:${server.getPort}") { + val session = SparkSession.builder().getOrCreate() + + val df = session.range(10) + df.analyze // Trigger RPC + assert(df.plan === service.getAndClearLatestInputPlan()) + + val session2 = SparkSession.builder().getOrCreate() + assert(session === session2) + } + } + + test("Builder.remote takes precedence over SPARK_REMOTE") { + startDummyServer(0) + val incorrectUrl = s"sc://localhost:${server.getPort + 1}" + + withEnvs("SPARK_REMOTE" -> incorrectUrl) { + val session = + SparkSession.builder().remote(s"sc://localhost:${server.getPort}").getOrCreate() + + val df = session.range(10) + df.analyze // Trigger RPC + assert(df.plan === service.getAndClearLatestInputPlan()) + } + } + test("SparkSession initialisation with connection string") { startDummyServer(0) client = SparkConnectClient From 3a723a171b1b77a8a8b2ccce2bd489acb8db00a3 Mon Sep 17 00:00:00 2001 From: Hasnain Lakhani Date: Thu, 28 Sep 2023 18:16:49 -0500 Subject: [PATCH 034/521] [SPARK-44937][CORE] Mark connection as timedOut in TransportClient.close ### What changes were proposed in this pull request? This PR avoids a race condition where a connection which is in the process of being closed could be returned by the TransportClientFactory only to be immediately closed and cause errors upon use. This race condition is rare and not easily triggered, but with the upcoming changes to introduce SSL connection support, connection closing can take just a slight bit longer and it's much easier to trigger this issue. Looking at the history of the code I believe this was an oversight in https://github.com/apache/spark/pull/9853. ### Why are the changes needed? Without this change, some of the new tests added in https://github.com/apache/spark/pull/42685 would fail ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests were run in CI. Without this change, some of the new tests added in https://github.com/apache/spark/pull/42685 fail ### Was this patch authored or co-authored using generative AI tooling? No Closes #43162 from hasnain-db/spark-tls-timeout. Authored-by: Hasnain Lakhani Signed-off-by: Mridul Muralidharan gmail.com> (cherry picked from commit 2a88feadd4b7cec9e01bc744e589783e3390e5ce) Signed-off-by: Mridul Muralidharan --- .../org/apache/spark/network/client/TransportClient.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java index 4a0a156699852..40825e06b82fd 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java @@ -325,7 +325,10 @@ public TransportResponseHandler getHandler() { @Override public void close() { - // close is a local operation and should finish with milliseconds; timeout just to be safe + // Mark the connection as timed out, so we do not return a connection that's being closed + // from the TransportClientFactory if closing takes some time (e.g. with SSL) + this.timedOut = true; + // close should not take this long; use a timeout just to be safe channel.close().awaitUninterruptibly(10, TimeUnit.SECONDS); } From aeba488ccd9213d28e6401d1bf7eadfdb9d955c3 Mon Sep 17 00:00:00 2001 From: Warren Zhu Date: Thu, 28 Sep 2023 18:51:33 -0500 Subject: [PATCH 035/521] [SPARK-45057][CORE] Avoid acquire read lock when keepReadLock is false ### What changes were proposed in this pull request? Add `keepReadLock` parameter in `lockNewBlockForWriting()`. When `keepReadLock` is `false`, skip `lockForReading()` to avoid block on read Lock or potential deadlock issue. When 2 tasks try to compute same rdd with replication level of 2 and running on only 2 executors. Deadlock will happen. Details refer [SPARK-45057] Task thread hold write lock and waiting for replication to remote executor while shuffle server thread which handling block upload request waiting on `lockForReading` in [BlockInfoManager.scala](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala#L457C24-L457C24) ### Why are the changes needed? This could save unnecessary read lock acquire and avoid deadlock issue mention above. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added UT in BlockInfoManagerSuite ### Was this patch authored or co-authored using generative AI tooling? No Closes #43067 from warrenzhu25/deadlock. Authored-by: Warren Zhu Signed-off-by: Mridul Muralidharan gmail.com> (cherry picked from commit 0d6fda5bbee99f9d1821952195efc6764816ec2f) Signed-off-by: Mridul Muralidharan --- .../apache/spark/storage/BlockInfoManager.scala | 11 +++++++---- .../org/apache/spark/storage/BlockManager.scala | 6 +----- .../spark/storage/BlockInfoManagerSuite.scala | 14 ++++++++++++++ 3 files changed, 22 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index 45ebb6eafa69f..ab4073fe8c05c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -415,13 +415,14 @@ private[storage] class BlockInfoManager(trackingCacheVisibility: Boolean = false * then just go ahead and acquire the write lock. Otherwise, if another thread is already * writing the block, then we wait for the write to finish before acquiring the read lock. * - * @return true if the block did not already exist, false otherwise. If this returns false, then - * a read lock on the existing block will be held. If this returns true, a write lock on - * the new block will be held. + * @return true if the block did not already exist, false otherwise. + * If this returns true, a write lock on the new block will be held. + * If this returns false then a read lock will be held iff keepReadLock == true. */ def lockNewBlockForWriting( blockId: BlockId, - newBlockInfo: BlockInfo): Boolean = { + newBlockInfo: BlockInfo, + keepReadLock: Boolean = true): Boolean = { logTrace(s"Task $currentTaskAttemptId trying to put $blockId") // Get the lock that will be associated with the to-be written block and lock it for the entire // duration of this operation. This way we prevent race conditions when two threads try to write @@ -449,6 +450,8 @@ private[storage] class BlockInfoManager(trackingCacheVisibility: Boolean = false val result = lockForWriting(blockId, blocking = false) assert(result.isDefined) return true + } else if (!keepReadLock) { + return false } else { // Block already exists. This could happen if another thread races with us to compute // the same block. In this case we try to acquire a read lock, if the locking succeeds diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 05d57c67576a5..6de6069d2fea5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1510,14 +1510,10 @@ private[spark] class BlockManager( val putBlockInfo = { val newInfo = new BlockInfo(level, classTag, tellMaster) - if (blockInfoManager.lockNewBlockForWriting(blockId, newInfo)) { + if (blockInfoManager.lockNewBlockForWriting(blockId, newInfo, keepReadLock)) { newInfo } else { logWarning(s"Block $blockId already exists on this machine; not re-adding it") - if (!keepReadLock) { - // lockNewBlockForWriting returned a read lock on the existing block, so we must free it: - releaseLock(blockId) - } return None } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala index 3708f0aa67223..f133a38269d71 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -166,6 +166,20 @@ class BlockInfoManagerSuite extends SparkFunSuite { assert(blockInfoManager.get("block").get.readerCount === 1) } + test("lockNewBlockForWriting should not block when keepReadLock is false") { + withTaskId(0) { + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + } + val lock1Future = Future { + withTaskId(1) { + blockInfoManager.lockNewBlockForWriting("block", newBlockInfo(), false) + } + } + + assert(!ThreadUtils.awaitResult(lock1Future, 1.seconds)) + assert(blockInfoManager.get("block").get.readerCount === 0) + } + test("read locks are reentrant") { withTaskId(1) { assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) From 5cdb4ab6f0b7aea7d890ee7dff61350671a09e79 Mon Sep 17 00:00:00 2001 From: Bo Xiong Date: Thu, 28 Sep 2023 22:53:37 -0500 Subject: [PATCH 036/521] [SPARK-45227][CORE] Fix a subtle thread-safety issue with CoarseGrainedExecutorBackend ### What changes were proposed in this pull request? Fix a subtle thread-safety issue with CoarseGrainedExecutorBackend where an executor process randomly gets stuck ### Why are the changes needed? For each executor, the single-threaded dispatcher can run into an "infinite loop" (as explained in the SPARK-45227). Once an executor process runs into a state, it'd stop launching tasks from the driver or reporting task status back. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? ``` $ build/mvn package -DskipTests -pl core $ build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.executor.CoarseGrainedExecutorBackendSuite test ``` ### Was this patch authored or co-authored using generative AI tooling? No ****************************************************************************** **_Please feel free to skip reading unless you're interested in details_** ****************************************************************************** ### Symptom Our Spark 3 app running on EMR 6.10.0 with Spark 3.3.1 got stuck in the very last step of writing a data frame to S3 by calling `df.write`. Looking at Spark UI, we saw that an executor process hung over 1 hour. After we manually killed the executor process, the app succeeded. Note that the same EMR cluster with two worker nodes was able to run the same app without any issue before and after the incident. Below is what's observed from relevant container logs and thread dump. - A regular task that's sent to the executor, which also reported back to the driver upon the task completion. ``` $zgrep 'task 150' container_1694029806204_12865_01_000001/stderr.gz 23/09/12 18:13:55 INFO TaskSetManager: Starting task 150.0 in stage 23.0 (TID 923) (ip-10-0-185-107.ec2.internal, executor 3, partition 150, NODE_LOCAL, 4432 bytes) taskResourceAssignments Map() 23/09/12 18:13:55 INFO TaskSetManager: Finished task 150.0 in stage 23.0 (TID 923) in 126 ms on ip-10-0-185-107.ec2.internal (executor 3) (16/200) $zgrep ' 923' container_1694029806204_12865_01_000004/stderr.gz 23/09/12 18:13:55 INFO YarnCoarseGrainedExecutorBackend: Got assigned task 923 $zgrep 'task 150' container_1694029806204_12865_01_000004/stderr.gz 23/09/12 18:13:55 INFO Executor: Running task 150.0 in stage 23.0 (TID 923) 23/09/12 18:13:55 INFO Executor: Finished task 150.0 in stage 23.0 (TID 923). 4495 bytes result sent to driver ``` - Another task that's sent to the executor but didn't get launched since the single-threaded dispatcher was stuck (presumably in an "infinite loop" as explained later). ``` $zgrep 'task 153' container_1694029806204_12865_01_000001/stderr.gz 23/09/12 18:13:55 INFO TaskSetManager: Starting task 153.0 in stage 23.0 (TID 924) (ip-10-0-185-107.ec2.internal, executor 3, partition 153, NODE_LOCAL, 4432 bytes) taskResourceAssignments Map() $zgrep ' 924' container_1694029806204_12865_01_000004/stderr.gz 23/09/12 18:13:55 INFO YarnCoarseGrainedExecutorBackend: Got assigned task 924 $zgrep 'task 153' container_1694029806204_12865_01_000004/stderr.gz >> note that the above command has no matching result, indicating that task 153.0 in stage 23.0 (TID 924) was never launched ``` - Thread dump shows that the dispatcher-Executor thread has the following stack trace. ``` "dispatcher-Executor" #40 daemon prio=5 os_prio=0 tid=0x0000ffff98e37800 nid=0x1aff runnable [0x0000ffff73bba000] java.lang.Thread.State: RUNNABLE at scala.runtime.BoxesRunTime.equalsNumObject(BoxesRunTime.java:142) at scala.runtime.BoxesRunTime.equals2(BoxesRunTime.java:131) at scala.runtime.BoxesRunTime.equals(BoxesRunTime.java:123) at scala.collection.mutable.HashTable.elemEquals(HashTable.scala:365) at scala.collection.mutable.HashTable.elemEquals$(HashTable.scala:365) at scala.collection.mutable.HashMap.elemEquals(HashMap.scala:44) at scala.collection.mutable.HashTable.findEntry0(HashTable.scala:140) at scala.collection.mutable.HashTable.findOrAddEntry(HashTable.scala:169) at scala.collection.mutable.HashTable.findOrAddEntry$(HashTable.scala:167) at scala.collection.mutable.HashMap.findOrAddEntry(HashMap.scala:44) at scala.collection.mutable.HashMap.put(HashMap.scala:126) at scala.collection.mutable.HashMap.update(HashMap.scala:131) at org.apache.spark.executor.CoarseGrainedExecutorBackend$$anonfun$receive$1.applyOrElse(CoarseGrainedExecutorBackend.scala:200) at org.apache.spark.rpc.netty.Inbox.$anonfun$process$1(Inbox.scala:115) at org.apache.spark.rpc.netty.Inbox$$Lambda$323/1930826709.apply$mcV$sp(Unknown Source) at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:213) at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:100) at org.apache.spark.rpc.netty.MessageLoop.org$apache$spark$rpc$netty$MessageLoop$$receiveLoop(MessageLoop.scala:75) at org.apache.spark.rpc.netty.MessageLoop$$anon$1.run(MessageLoop.scala:41) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:750) ``` ### Relevant code paths Within an executor process, there's a [dispatcher thread](https://github.com/apache/spark/blob/1fdd46f173f7bc90e0523eb0a2d5e8e27e990102/core/src/main/scala/org/apache/spark/rpc/netty/MessageLoop.scala#L170) dedicated to CoarseGrainedExecutorBackend(a single RPC endpoint) that launches tasks scheduled by the driver. Each task is run on a TaskRunner thread backed by a thread pool created for the executor. The TaskRunner thread and the dispatcher thread are different. However, they read and write a common object (i.e., taskResources) that's a mutable hashmap without thread-safety, in [Executor](https://github.com/apache/spark/blob/1fdd46f173f7bc90e0523eb0a2d5e8e27e990102/core/src/main/scala/org/apache/spark/executor/Executor.scala#L561) and [CoarseGrainedExecutorBackend](https://github.com/apache/spark/blob/1fdd46f173f7bc90e0523eb0a2d5e8e27e990102/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala#L189), respectively. ### What's going on? Based on the above observations, our hypothesis is that the dispatcher thread runs into an "infinite loop" due to a race condition when two threads access the same hashmap object. For illustration purpose, let's consider the following scenario where two threads (Thread 1 and Thread 2) access a hash table without thread-safety - Thread 1 sees A.next = B, but then yields execution to Thread 2 - Thread 2 triggers a resize operation resulting in B.next = A (Note that hashmap doesn't care about ordering), and then yields execution to Thread 1. - After taking over CPU, Thread 1 would run into an "infinite loop" when traversing the list in the last bucket, given A.next = B and B.next = A in its view. Closes #43021 from xiongbo-sjtu/master. Authored-by: Bo Xiong Signed-off-by: Mridul Muralidharan gmail.com> (cherry picked from commit 8e6b1603a66706ee27a0b16d850f5ee56d633354) Signed-off-by: Mridul Muralidharan --- .../spark/executor/CoarseGrainedExecutorBackend.scala | 11 +++++++---- .../executor/CoarseGrainedExecutorBackendSuite.scala | 6 +++--- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index ab238626efe9b..c695a9ec2851b 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -20,9 +20,9 @@ package org.apache.spark.executor import java.net.URL import java.nio.ByteBuffer import java.util.Locale +import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicBoolean -import scala.collection.mutable import scala.util.{Failure, Success} import scala.util.control.NonFatal @@ -71,9 +71,12 @@ private[spark] class CoarseGrainedExecutorBackend( /** * Map each taskId to the information about the resource allocated to it, Please refer to * [[ResourceInformation]] for specifics. + * CHM is used to ensure thread-safety (https://issues.apache.org/jira/browse/SPARK-45227) * Exposed for testing only. */ - private[executor] val taskResources = new mutable.HashMap[Long, Map[String, ResourceInformation]] + private[executor] val taskResources = new ConcurrentHashMap[ + Long, Map[String, ResourceInformation] + ] private var decommissioned = false @@ -184,7 +187,7 @@ private[spark] class CoarseGrainedExecutorBackend( } else { val taskDesc = TaskDescription.decode(data.value) logInfo("Got assigned task " + taskDesc.taskId) - taskResources(taskDesc.taskId) = taskDesc.resources + taskResources.put(taskDesc.taskId, taskDesc.resources) executor.launchTask(this, taskDesc) } @@ -261,7 +264,7 @@ private[spark] class CoarseGrainedExecutorBackend( } override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer): Unit = { - val resources = taskResources.getOrElse(taskId, Map.empty[String, ResourceInformation]) + val resources = taskResources.getOrDefault(taskId, Map.empty[String, ResourceInformation]) val cpus = executor.runningTasks.get(taskId).taskDescription.cpus val msg = StatusUpdate(executorId, taskId, state, data, cpus, resources) if (TaskState.isFinished(state)) { diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index 0dcc7c7f9b4cf..909d605442575 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -302,7 +302,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite resourceProfile = ResourceProfile.getOrCreateDefaultProfile(conf)) assert(backend.taskResources.isEmpty) - val taskId = 1000000 + val taskId = 1000000L // We don't really verify the data, just pass it around. val data = ByteBuffer.wrap(Array[Byte](1, 2, 3, 4)) val taskDescription = new TaskDescription(taskId, 2, "1", "TASK 1000000", 19, @@ -339,14 +339,14 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite backend.self.send(LaunchTask(new SerializableBuffer(serializedTaskDescription))) eventually(timeout(10.seconds)) { assert(backend.taskResources.size == 1) - val resources = backend.taskResources(taskId) + val resources = backend.taskResources.get(taskId) assert(resources(GPU).addresses sameElements Array("0", "1")) } // Update the status of a running task shall not affect `taskResources` map. backend.statusUpdate(taskId, TaskState.RUNNING, data) assert(backend.taskResources.size == 1) - val resources = backend.taskResources(taskId) + val resources = backend.taskResources.get(taskId) assert(resources(GPU).addresses sameElements Array("0", "1")) // Update the status of a finished task shall remove the entry from `taskResources` map. From 845e4f6c5bcf3a368ee78757f3a74b390cdce5c0 Mon Sep 17 00:00:00 2001 From: Peter Kaszt Date: Mon, 2 Oct 2023 07:48:56 -0500 Subject: [PATCH 037/521] [MINOR][DOCS] Fix Python code sample for StreamingQueryListener: Reporting Metrics programmatically using Asynchronous APIs Fix Python language code sample in the docs for _StreamingQueryListener_: Reporting Metrics programmatically using Asynchronous APIs section. ### What changes were proposed in this pull request? The code sample in the [Reporting Metrics programmatically using Asynchronous APIs](https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html#reporting-metrics-programmatically-using-asynchronous-apis) section was this: ``` spark = ... class Listener(StreamingQueryListener): def onQueryStarted(self, event): print("Query started: " + queryStarted.id) def onQueryProgress(self, event): println("Query terminated: " + queryTerminated.id) def onQueryTerminated(self, event): println("Query made progress: " + queryProgress.progress) spark.streams.addListener(Listener()) ``` Which is not a proper Python code, and has QueryProgress and QueryTerminated prints mixed. Proposed change/fix: ``` spark = ... class Listener(StreamingQueryListener): def onQueryStarted(self, event): print("Query started: " + queryStarted.id) def onQueryProgress(self, event): print("Query made progress: " + queryProgress.progress) def onQueryTerminated(self, event): print("Query terminated: " + queryTerminated.id) spark.streams.addListener(Listener()) ``` ### Why are the changes needed? To fix docimentation errors. ### Does this PR introduce _any_ user-facing change? Yes. -> Sample python code snippet is fixed in docs (see above). ### How was this patch tested? Checked with github's .md preview, and built the docs according to the readme. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43190 from kasztp/master. Authored-by: Peter Kaszt Signed-off-by: Sean Owen (cherry picked from commit d708fd7b68bf0c9964e861cb2c81818d17d7136e) Signed-off-by: Sean Owen --- docs/structured-streaming-programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 76a22621a0e32..3e87c45a34915 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -3831,10 +3831,10 @@ class Listener(StreamingQueryListener): print("Query started: " + queryStarted.id) def onQueryProgress(self, event): - println("Query terminated: " + queryTerminated.id) + print("Query made progress: " + queryProgress.progress) def onQueryTerminated(self, event): - println("Query made progress: " + queryProgress.progress) + print("Query terminated: " + queryTerminated.id) spark.streams.addListener(Listener()) From c5203abcbd191423071ef3603e95a7941bb1eec2 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Mon, 2 Oct 2023 13:03:06 -0400 Subject: [PATCH 038/521] [SPARK-45371][CONNECT] Fix shading issues in the Spark Connect Scala Client ### What changes were proposed in this pull request? This PR fixes shading for the Spark Connect Scala Client maven build. The following things are addressed: - Guava and protobuf are included in the shaded jars. These were missing, and were causing users to see `ClassNotFoundException`s. - Fixed duplicate shading of guava. We use the parent pom's location now. - Fixed duplicate Netty dependency (shaded and transitive). One was used for GRPC and the other was needed by Arrow. This was fixed by pulling arrow into the shaded jar. - Use the same package as the shading defined in the parent package. ### Why are the changes needed? The maven artifacts for the Spark Connect Scala Client are currently broken. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual tests. #### Step 1: Build new shaded library and install it in local maven repository `build/mvn clean install -pl connector/connect/client/jvm -am -DskipTests` #### Step 2: Start Connect Server `connector/connect/bin/spark-connect` #### Step 3: Launch REPL using the newly created library This step requires [coursier](https://get-coursier.io/) to be installed. `cs launch --jvm zulu:17.0.8 --scala 2.13.9 -r m2Local com.lihaoyi:::ammonite:2.5.11 org.apache.spark::spark-connect-client-jvm:4.0.0-SNAPSHOT --java-opt --add-opens=java.base/java.nio=ALL-UNNAMED -M org.apache.spark.sql.application.ConnectRepl` #### Step 4: Run a bunch of commands: ```scala // Check version spark.version // Run a simple query { spark.range(1, 10000, 1) .select($"id", $"id" % 5 as "group", rand(1).as("v1"), rand(2).as("v2")) .groupBy($"group") .agg( avg($"v1").as("v1_avg"), avg($"v2").as("v2_avg")) .show() } // Run a streaming query { import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger val query_name = "simple_streaming" val stream = spark.readStream .format("rate") .option("numPartitions", "1") .option("rowsPerSecond", "10") .load() .withWatermark("timestamp", "10 milliseconds") .groupBy(window(col("timestamp"), "10 milliseconds")) .count() .selectExpr("window.start as timestamp", "count as num_events") .writeStream .format("memory") .queryName(query_name) .trigger(ProcessingTimeTrigger.create("10 milliseconds")) // run for 20 seconds val query = stream.start() val start = System.currentTimeMillis() val end = System.currentTimeMillis() + 20 * 1000 while (System.currentTimeMillis() < end) { println(s"time: ${System.currentTimeMillis() - start} ms") println(query.status) spark.sql(s"select * from ${query_name}").show() Thread.sleep(500) } query.stop() } ``` Closes #43195 from hvanhovell/SPARK-45371. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell (cherry picked from commit e53abbbceaa2c41babaa23fe4c2f282f559b4c03) Signed-off-by: Herman van Hovell --- connector/connect/client/jvm/pom.xml | 39 +++++++++++++++++++++------- 1 file changed, 30 insertions(+), 9 deletions(-) diff --git a/connector/connect/client/jvm/pom.xml b/connector/connect/client/jvm/pom.xml index 67227ef38eb8f..236e5850b762f 100644 --- a/connector/connect/client/jvm/pom.xml +++ b/connector/connect/client/jvm/pom.xml @@ -50,10 +50,20 @@ spark-sketch_${scala.binary.version} ${project.version} + com.google.guava guava ${connect.guava.version} + compile + + + com.google.protobuf + protobuf-java + compile com.lihaoyi @@ -85,6 +95,7 @@ maven-shade-plugin false + true com.google.android:* @@ -92,52 +103,62 @@ com.google.code.findbugs:* com.google.code.gson:* com.google.errorprone:* - com.google.guava:* com.google.j2objc:* com.google.protobuf:* + com.google.flatbuffers:* io.grpc:* io.netty:* io.perfmark:* + org.apache.arrow:* org.codehaus.mojo:* org.checkerframework:* org.apache.spark:spark-connect-common_${scala.binary.version} + org.apache.spark:spark-sql-api_${scala.binary.version} io.grpc - ${spark.shade.packageName}.connect.client.io.grpc + ${spark.shade.packageName}.io.grpc io.grpc.** com.google - ${spark.shade.packageName}.connect.client.com.google + ${spark.shade.packageName}.com.google + + + com.google.common.** + io.netty - ${spark.shade.packageName}.connect.client.io.netty + ${spark.shade.packageName}.io.netty org.checkerframework - ${spark.shade.packageName}.connect.client.org.checkerframework + ${spark.shade.packageName}.org.checkerframework javax.annotation - ${spark.shade.packageName}.connect.client.javax.annotation + ${spark.shade.packageName}.javax.annotation io.perfmark - ${spark.shade.packageName}.connect.client.io.perfmark + ${spark.shade.packageName}.io.perfmark org.codehaus - ${spark.shade.packageName}.connect.client.org.codehaus + ${spark.shade.packageName}.org.codehaus + + + org.apache.arrow + ${spark.shade.packageName}.org.apache.arrow android.annotation - ${spark.shade.packageName}.connect.client.android.annotation + ${spark.shade.packageName}.android.annotation From 522af69713502d33d34b330bce6166e3d15dba8a Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Thu, 5 Oct 2023 09:39:36 +0900 Subject: [PATCH 039/521] Revert "[SPARK-45167][CONNECT][PYTHON][FOLLOW-UP] Use lighter threading Rlock, and use the existing eventually util function" This reverts commit 2a9dd2b3968da7c2e96c502aaf4c158ee782e5f4. --- python/pyspark/sql/connect/client/reattach.py | 5 ++-- .../sql/tests/connect/client/test_client.py | 23 +++++++++++++++---- 2 files changed, 21 insertions(+), 7 deletions(-) diff --git a/python/pyspark/sql/connect/client/reattach.py b/python/pyspark/sql/connect/client/reattach.py index 6addb5bd2c652..e58864b965bd9 100644 --- a/python/pyspark/sql/connect/client/reattach.py +++ b/python/pyspark/sql/connect/client/reattach.py @@ -18,11 +18,12 @@ check_dependencies(__name__) -from threading import RLock import warnings import uuid from collections.abc import Generator from typing import Optional, Dict, Any, Iterator, Iterable, Tuple, Callable, cast, Type, ClassVar +from multiprocessing import RLock +from multiprocessing.synchronize import RLock as RLockBase from multiprocessing.pool import ThreadPool import os @@ -55,7 +56,7 @@ class ExecutePlanResponseReattachableIterator(Generator): """ # Lock to manage the pool - _lock: ClassVar[RLock] = RLock() + _lock: ClassVar[RLockBase] = RLock() _release_thread_pool: Optional[ThreadPool] = ThreadPool(os.cpu_count() if os.cpu_count() else 8) @classmethod diff --git a/python/pyspark/sql/tests/connect/client/test_client.py b/python/pyspark/sql/tests/connect/client/test_client.py index 93b7006799b30..cf43fb16df7a7 100644 --- a/python/pyspark/sql/tests/connect/client/test_client.py +++ b/python/pyspark/sql/tests/connect/client/test_client.py @@ -25,7 +25,6 @@ from pyspark.sql.connect.client import SparkConnectClient, ChannelBuilder import pyspark.sql.connect.proto as proto from pyspark.testing.connectutils import should_test_connect, connect_requirement_message -from pyspark.testing.utils import eventually from pyspark.sql.connect.client.core import Retrying from pyspark.sql.connect.client.reattach import ( @@ -153,6 +152,20 @@ def _stub_with(self, execute=None, attach=None): attach_ops=ResponseGenerator(attach) if attach is not None else None, ) + def assertEventually(self, callable, timeout_ms=1000): + """Helper method that will continuously evaluate the callable to not raise an + exception.""" + import time + + limit = time.monotonic_ns() + timeout_ms * 1000 * 1000 + while time.monotonic_ns() < limit: + try: + callable() + break + except Exception: + time.sleep(0.1) + callable() + def test_basic_flow(self): stub = self._stub_with([self.response, self.finished]) ite = ExecutePlanResponseReattachableIterator(self.request, stub, self.policy, []) @@ -165,7 +178,7 @@ def check_all(): self.assertEqual(1, stub.release_calls) self.assertEqual(1, stub.execute_calls) - eventually(timeout=1, catch_assertions=True)(check_all)() + self.assertEventually(check_all, timeout_ms=1000) def test_fail_during_execute(self): def fatal(): @@ -183,7 +196,7 @@ def check(): self.assertEqual(1, stub.release_until_calls) self.assertEqual(1, stub.execute_calls) - eventually(timeout=1, catch_assertions=True)(check)() + self.assertEventually(check, timeout_ms=1000) def test_fail_and_retry_during_execute(self): def non_fatal(): @@ -202,7 +215,7 @@ def check(): self.assertEqual(3, stub.release_until_calls) self.assertEqual(1, stub.execute_calls) - eventually(timeout=1, catch_assertions=True)(check)() + self.assertEventually(check, timeout_ms=1000) def test_fail_and_retry_during_reattach(self): count = 0 @@ -228,7 +241,7 @@ def check(): self.assertEqual(1, stub.release_calls) self.assertEqual(1, stub.execute_calls) - eventually(timeout=1, catch_assertions=True)(check)() + self.assertEventually(check, timeout_ms=1000) class TestException(grpc.RpcError, grpc.Call): From 6468f96ea42f6efe42033507c4e26600b751bfcc Mon Sep 17 00:00:00 2001 From: Emil Ejbyfeldt Date: Thu, 5 Oct 2023 09:41:08 +0900 Subject: [PATCH 040/521] [SPARK-45386][SQL][3.5] Fix correctness issue with persist using StorageLevel.NONE on Dataset ### What changes were proposed in this pull request? Support for InMememoryTableScanExec in AQE was added in #39624, but this patch contained a bug when a Dataset is persisted using `StorageLevel.NONE`. Before that patch a query like: ``` import org.apache.spark.storage.StorageLevel spark.createDataset(Seq(1, 2)).persist(StorageLevel.NONE).count() ``` would correctly return 2. But after that patch it incorrectly returns 0. This is because AQE incorrectly determines based on the runtime statistics that are collected here: https://github.com/apache/spark/blob/eac5a8c7e6da94bb27e926fc9a681aed6582f7d3/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala#L294 that the input is empty. The problem is that the action that should make sure the statistics are collected here https://github.com/apache/spark/blob/eac5a8c7e6da94bb27e926fc9a681aed6582f7d3/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala#L285-L291 never use the iterator and when we have `StorageLevel.NONE` the persisting will also not use the iterator and we will not gather the correct statistics. The proposed fix in the patch just make calling persist with StorageLevel.NONE a no-op. Changing the action since it always "emptied" the iterator would also work but seems like that would be unnecessary work in a lot of normal circumstances. ### Why are the changes needed? The current code has a correctness issue. ### Does this PR introduce _any_ user-facing change? Yes, fixes the correctness issue. ### How was this patch tested? New and existing unit tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43213 from eejbyfeldt/SPARK-45386-branch-3.5. Authored-by: Emil Ejbyfeldt Signed-off-by: Hyukjin Kwon --- .../scala/org/apache/spark/sql/execution/CacheManager.scala | 4 +++- .../src/test/scala/org/apache/spark/sql/DatasetSuite.scala | 6 ++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 064819275e004..e906c74f8a5ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -113,7 +113,9 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { planToCache: LogicalPlan, tableName: Option[String], storageLevel: StorageLevel): Unit = { - if (lookupCachedData(planToCache).nonEmpty) { + if (storageLevel == StorageLevel.NONE) { + // Do nothing for StorageLevel.NONE since it will not actually cache any data. + } else if (lookupCachedData(planToCache).nonEmpty) { logWarning("Asked to cache already cached data.") } else { val sessionWithConfigsOff = getOrCloneSessionWithConfigsOff(spark) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index c967540541a5c..6d9c43f866a0c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -45,6 +45,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ +import org.apache.spark.storage.StorageLevel case class TestDataPoint(x: Int, y: Double, s: String, t: TestDataPoint2) case class TestDataPoint2(x: Int, s: String) @@ -2535,6 +2536,11 @@ class DatasetSuite extends QueryTest checkDataset(ds.filter(f(col("_1"))), Tuple1(ValueClass(2))) } + + test("SPARK-45386: persist with StorageLevel.NONE should give correct count") { + val ds = Seq(1, 2).toDS().persist(StorageLevel.NONE) + assert(ds.count() == 2) + } } class DatasetLargeResultCollectingSuite extends QueryTest From d60a40e5d30e565536e0833fc650dc581439ee89 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Thu, 5 Oct 2023 13:00:02 +0900 Subject: [PATCH 041/521] [SPARK-45167][CONNECT][PYTHON][FOLLOW-UP][3.5] Use lighter threading Rlock ### What changes were proposed in this pull request? This PR reverts the revert: https://github.com/apache/spark/commit/522af69713502d33d34b330bce6166e3d15dba8a. It only partially ports the real change within main code. It excludes the testing side which depends on https://github.com/apache/spark/commit/9798244ca647ec68d36f4b9b21356a6de5f73f77 that does not exist in `branch-3.5`. ### Why are the changes needed? Mainly for code clean-up. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests should cover this. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43221 from HyukjinKwon/SPARK-45167-followup2. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/connect/client/reattach.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/connect/client/reattach.py b/python/pyspark/sql/connect/client/reattach.py index e58864b965bd9..6addb5bd2c652 100644 --- a/python/pyspark/sql/connect/client/reattach.py +++ b/python/pyspark/sql/connect/client/reattach.py @@ -18,12 +18,11 @@ check_dependencies(__name__) +from threading import RLock import warnings import uuid from collections.abc import Generator from typing import Optional, Dict, Any, Iterator, Iterable, Tuple, Callable, cast, Type, ClassVar -from multiprocessing import RLock -from multiprocessing.synchronize import RLock as RLockBase from multiprocessing.pool import ThreadPool import os @@ -56,7 +55,7 @@ class ExecutePlanResponseReattachableIterator(Generator): """ # Lock to manage the pool - _lock: ClassVar[RLockBase] = RLock() + _lock: ClassVar[RLock] = RLock() _release_thread_pool: Optional[ThreadPool] = ThreadPool(os.cpu_count() if os.cpu_count() else 8) @classmethod From c50e371c2d3ccba9340bc8980add0753f2d7a86b Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Mon, 2 Oct 2023 23:00:56 -0500 Subject: [PATCH 042/521] [SPARK-45250][CORE] Support stage level task resource profile for yarn cluster when dynamic allocation disabled ### What changes were proposed in this pull request? This PR is a follow-up of https://github.com/apache/spark/pull/37268 which supports stage level task resource profile for standalone cluster when dynamic allocation disabled. This PR enables stage-level task resource profile for yarn cluster. ### Why are the changes needed? Users who work on spark ML/DL cases running on Yarn would expect stage-level task resource profile feature. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? The current tests of https://github.com/apache/spark/pull/37268 can also cover this PR since both yarn and standalone cluster share the same TaskSchedulerImpl class which implements this feature. Apart from that, modifying the existing test to cover yarn cluster. Apart from that, I also performed some manual tests which have been updated in the comments. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43030 from wbo4958/yarn-task-resoure-profile. Authored-by: Bobby Wang Signed-off-by: Mridul Muralidharan gmail.com> (cherry picked from commit 5b80639e643b6dd09dd64c3f43ec039b2ef2f9fd) Signed-off-by: Thomas Graves --- .../spark/resource/ResourceProfileManager.scala | 6 +++--- .../resource/ResourceProfileManagerSuite.scala | 15 +++++++++++++-- docs/configuration.md | 2 +- docs/running-on-yarn.md | 6 +++++- 4 files changed, 22 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala index 9f98d4d9c9c79..cd7124a572464 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala @@ -67,9 +67,9 @@ private[spark] class ResourceProfileManager(sparkConf: SparkConf, */ private[spark] def isSupported(rp: ResourceProfile): Boolean = { if (rp.isInstanceOf[TaskResourceProfile] && !dynamicEnabled) { - if ((notRunningUnitTests || testExceptionThrown) && !isStandaloneOrLocalCluster) { - throw new SparkException("TaskResourceProfiles are only supported for Standalone " + - "cluster for now when dynamic allocation is disabled.") + if ((notRunningUnitTests || testExceptionThrown) && !(isStandaloneOrLocalCluster || isYarn)) { + throw new SparkException("TaskResourceProfiles are only supported for Standalone and " + + "Yarn cluster for now when dynamic allocation is disabled.") } } else { val isNotDefaultProfile = rp.id != ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala index e97d5c7883aa8..77dc7bcb4c56e 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala @@ -126,18 +126,29 @@ class ResourceProfileManagerSuite extends SparkFunSuite { val defaultProf = rpmanager.defaultResourceProfile assert(rpmanager.isSupported(defaultProf)) - // task resource profile. + // Standalone: supports task resource profile. val gpuTaskReq = new TaskResourceRequests().resource("gpu", 1) val taskProf = new TaskResourceProfile(gpuTaskReq.requests) assert(rpmanager.isSupported(taskProf)) + // Local: doesn't support task resource profile. conf.setMaster("local") rpmanager = new ResourceProfileManager(conf, listenerBus) val error = intercept[SparkException] { rpmanager.isSupported(taskProf) }.getMessage assert(error === "TaskResourceProfiles are only supported for Standalone " + - "cluster for now when dynamic allocation is disabled.") + "and Yarn cluster for now when dynamic allocation is disabled.") + + // Local cluster: supports task resource profile. + conf.setMaster("local-cluster[1, 1, 1024]") + rpmanager = new ResourceProfileManager(conf, listenerBus) + assert(rpmanager.isSupported(taskProf)) + + // Yarn: supports task resource profile. + conf.setMaster("yarn") + rpmanager = new ResourceProfileManager(conf, listenerBus) + assert(rpmanager.isSupported(taskProf)) } test("isSupported task resource profiles with dynamic allocation enabled") { diff --git a/docs/configuration.md b/docs/configuration.md index 1139beb66462f..74ddd6df0233c 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -3670,7 +3670,7 @@ See your cluster manager specific page for requirements and details on each of - # Stage Level Scheduling Overview The stage level scheduling feature allows users to specify task and executor resource requirements at the stage level. This allows for different stages to run with executors that have different resources. A prime example of this is one ETL stage runs with executors with just CPUs, the next stage is an ML stage that needs GPUs. Stage level scheduling allows for user to request different executors that have GPUs when the ML stage runs rather then having to acquire executors with GPUs at the start of the application and them be idle while the ETL stage is being run. -This is only available for the RDD API in Scala, Java, and Python. It is available on YARN, Kubernetes and Standalone when dynamic allocation is enabled. When dynamic allocation is disabled, it allows users to specify different task resource requirements at stage level, and this is supported on Standalone cluster right now. See the [YARN](running-on-yarn.html#stage-level-scheduling-overview) page or [Kubernetes](running-on-kubernetes.html#stage-level-scheduling-overview) page or [Standalone](spark-standalone.html#stage-level-scheduling-overview) page for more implementation details. +This is only available for the RDD API in Scala, Java, and Python. It is available on YARN, Kubernetes and Standalone when dynamic allocation is enabled. When dynamic allocation is disabled, it allows users to specify different task resource requirements at stage level, and this is supported on YARN and Standalone cluster right now. See the [YARN](running-on-yarn.html#stage-level-scheduling-overview) page or [Kubernetes](running-on-kubernetes.html#stage-level-scheduling-overview) page or [Standalone](spark-standalone.html#stage-level-scheduling-overview) page for more implementation details. See the `RDD.withResources` and `ResourceProfileBuilder` API's for using this feature. When dynamic allocation is disabled, tasks with different task resource requirements will share executors with `DEFAULT_RESOURCE_PROFILE`. While when dynamic allocation is enabled, the current implementation acquires new executors for each `ResourceProfile` created and currently has to be an exact match. Spark does not try to fit tasks into an executor that require a different ResourceProfile than the executor was created with. Executors that are not in use will idle timeout with the dynamic allocation logic. The default configuration for this feature is to only allow one ResourceProfile per stage. If the user associates more then 1 ResourceProfile to an RDD, Spark will throw an exception by default. See config `spark.scheduler.resource.profileMergeConflicts` to control that behavior. The current merge strategy Spark implements when `spark.scheduler.resource.profileMergeConflicts` is enabled is a simple max of each resource within the conflicting ResourceProfiles. Spark will create a new ResourceProfile with the max of each of the resources. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 5eec6c490cb1f..97cc9ac135af1 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -759,7 +759,11 @@ YARN does not tell Spark the addresses of the resources allocated to each contai # Stage Level Scheduling Overview -Stage level scheduling is supported on YARN when dynamic allocation is enabled. One thing to note that is YARN specific is that each ResourceProfile requires a different container priority on YARN. The mapping is simply the ResourceProfile id becomes the priority, on YARN lower numbers are higher priority. This means that profiles created earlier will have a higher priority in YARN. Normally this won't matter as Spark finishes one stage before starting another one, the only case this might have an affect is in a job server type scenario, so its something to keep in mind. +Stage level scheduling is supported on YARN: +- When dynamic allocation is disabled: It allows users to specify different task resource requirements at the stage level and will use the same executors requested at startup. +- When dynamic allocation is enabled: It allows users to specify task and executor resource requirements at the stage level and will request the extra executors. + +One thing to note that is YARN specific is that each ResourceProfile requires a different container priority on YARN. The mapping is simply the ResourceProfile id becomes the priority, on YARN lower numbers are higher priority. This means that profiles created earlier will have a higher priority in YARN. Normally this won't matter as Spark finishes one stage before starting another one, the only case this might have an affect is in a job server type scenario, so its something to keep in mind. Note there is a difference in the way custom resources are handled between the base default profile and custom ResourceProfiles. To allow for the user to request YARN containers with extra resources without Spark scheduling on them, the user can specify resources via the spark.yarn.executor.resource. config. Those configs are only used in the base default profile though and do not get propagated into any other custom ResourceProfiles. This is because there would be no way to remove them if you wanted a stage to not have them. This results in your default profile getting custom resources defined in spark.yarn.executor.resource. plus spark defined resources of GPU or FPGA. Spark converts GPU and FPGA resources into the YARN built in types yarn.io/gpu) and yarn.io/fpga, but does not know the mapping of any other resources. Any other Spark custom resources are not propagated to YARN for the default profile. So if you want Spark to schedule based off a custom resource and have it requested from YARN, you must specify it in both YARN (spark.yarn.{driver/executor}.resource.) and Spark (spark.{driver/executor}.resource.) configs. Leave the Spark config off if you only want YARN containers with the extra resources but Spark not to schedule using them. Now for custom ResourceProfiles, it doesn't currently have a way to only specify YARN resources without Spark scheduling off of them. This means for custom ResourceProfiles we propagate all the resources defined in the ResourceProfile to YARN. We still convert GPU and FPGA to the YARN build in types as well. This requires that the name of any custom resources you specify match what they are defined as in YARN. # Important notes From 64e2b22f6b4023197871a60eb08b055688e9fdd2 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Thu, 5 Oct 2023 08:38:54 +0900 Subject: [PATCH 043/521] [SPARK-45396][PYTHON] Add doc entry for `pyspark.ml.connect` module, and adds `Evaluator` to `__all__` at `ml.connect` This PR documents MLlib's Spark Connect support at API reference. This PR also piggies back a fix in `__all__` at `python/pyspark/ml/connect/__init__.py` so `from pyspark.sql.commect import Evaluator` works. With this this, user cannot see `pyspark.ml.connect` Python APIs on doc website. Yes it adds the new page into your facing documentation ([PySpark API reference](https://spark.apache.org/docs/latest/api/python/reference/index.html)). Manually tested via: ```bash cd python/docs make clean html ``` No. Closes #43210 from HyukjinKwon/SPARK-45396-followup. Lead-authored-by: Weichen Xu Co-authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit 35b627a934b1ab28be7d6ba88fdad63dc129525a) Signed-off-by: Hyukjin Kwon --- python/docs/source/reference/index.rst | 1 + .../source/reference/pyspark.ml.connect.rst | 122 ++++++++++++++++++ python/pyspark/ml/connect/__init__.py | 3 +- 3 files changed, 125 insertions(+), 1 deletion(-) create mode 100644 python/docs/source/reference/pyspark.ml.connect.rst diff --git a/python/docs/source/reference/index.rst b/python/docs/source/reference/index.rst index ed3eb4d07dac6..6330636839cdf 100644 --- a/python/docs/source/reference/index.rst +++ b/python/docs/source/reference/index.rst @@ -31,6 +31,7 @@ Pandas API on Spark follows the API specifications of latest pandas release. pyspark.pandas/index pyspark.ss/index pyspark.ml + pyspark.ml.connect pyspark.streaming pyspark.mllib pyspark diff --git a/python/docs/source/reference/pyspark.ml.connect.rst b/python/docs/source/reference/pyspark.ml.connect.rst new file mode 100644 index 0000000000000..1a3e6a593980f --- /dev/null +++ b/python/docs/source/reference/pyspark.ml.connect.rst @@ -0,0 +1,122 @@ +.. 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. + + +MLlib (DataFrame-based) for Spark Connect +========================================= + +.. warning:: + The namespace for this package can change in the future Spark version. + + +Pipeline APIs +------------- + +.. currentmodule:: pyspark.ml.connect + +.. autosummary:: + :template: autosummary/class_with_docs.rst + :toctree: api/ + + Transformer + Estimator + Model + Evaluator + Pipeline + PipelineModel + + +Feature +------- + +.. currentmodule:: pyspark.ml.connect.feature + +.. autosummary:: + :template: autosummary/class_with_docs.rst + :toctree: api/ + + MaxAbsScaler + MaxAbsScalerModel + StandardScaler + StandardScalerModel + + +Classification +-------------- + +.. currentmodule:: pyspark.ml.connect.classification + +.. autosummary:: + :template: autosummary/class_with_docs.rst + :toctree: api/ + + LogisticRegression + LogisticRegressionModel + + +Functions +--------- + +.. currentmodule:: pyspark.ml.connect.functions + +.. autosummary:: + :toctree: api/ + + array_to_vector + vector_to_array + + +Tuning +------ + +.. currentmodule:: pyspark.ml.connect.tuning + +.. autosummary:: + :template: autosummary/class_with_docs.rst + :toctree: api/ + + CrossValidator + CrossValidatorModel + + +Evaluation +---------- + +.. currentmodule:: pyspark.ml.connect.evaluation + +.. autosummary:: + :template: autosummary/class_with_docs.rst + :toctree: api/ + + RegressionEvaluator + BinaryClassificationEvaluator + MulticlassClassificationEvaluator + + +Utilities +--------- + +.. currentmodule:: pyspark.ml.connect.io_utils + +.. autosummary:: + :template: autosummary/class_with_docs.rst + :toctree: api/ + + ParamsReadWrite + CoreModelReadWrite + MetaAlgorithmReadWrite + diff --git a/python/pyspark/ml/connect/__init__.py b/python/pyspark/ml/connect/__init__.py index 2ee152f6a38a3..e6115a62ccfe8 100644 --- a/python/pyspark/ml/connect/__init__.py +++ b/python/pyspark/ml/connect/__init__.py @@ -28,13 +28,14 @@ evaluation, tuning, ) +from pyspark.ml.connect.evaluation import Evaluator from pyspark.ml.connect.pipeline import Pipeline, PipelineModel __all__ = [ "Estimator", "Transformer", - "Estimator", + "Evaluator", "Model", "feature", "evaluation", From 1b5b2a15dec3d17fa1e7be9a78b37dc600eeb8ab Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Sat, 7 Oct 2023 15:26:17 -0700 Subject: [PATCH 044/521] [SPARK-45389][SQL][3.5] Correct MetaException matching rule on getting partition metadata This is the backport of https://github.com/apache/spark/pull/43191 for `branch-3.5`, it should also be applicable for `branch-3.3` and `branch-3.4` ### What changes were proposed in this pull request? This PR aims to fix the HMS call fallback logic introduced in SPARK-35437. ```patch try { ... hive.getPartitionNames ... hive.getPartitionsByNames } catch { - case ex: InvocationTargetException if ex.getCause.isInstanceOf[MetaException] => + case ex: HiveException if ex.getCause.isInstanceOf[MetaException] => ... } ``` ### Why are the changes needed? Directly method call won't throw `InvocationTargetException`, and check the code of `hive.getPartitionNames` and `hive.getPartitionsByNames`, both of them will wrap a `HiveException` if `MetaException` throws. ### Does this PR introduce _any_ user-facing change? Yes, it should be a bug fix. ### How was this patch tested? Pass GA and code review. (I'm not sure how to construct/simulate a MetaException during the HMS thrift call with the current HMS testing infrastructure) ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43260 from pan3793/SPARK-45389-3.5. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/sql/hive/client/HiveShim.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 63f672b22bad2..60ff9ec42f29d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.metastore.TableType import org.apache.hadoop.hive.metastore.api.{Database, EnvironmentContext, Function => HiveFunction, FunctionType, Index, MetaException, PrincipalType, ResourceType, ResourceUri} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.io.AcidUtils -import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} +import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition, Table} import org.apache.hadoop.hive.ql.plan.AddPartitionDesc import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorFactory} import org.apache.hadoop.hive.ql.session.SessionState @@ -1190,7 +1190,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { recordHiveCall() hive.getPartitionsByNames(table, partNames.asJava) } catch { - case ex: InvocationTargetException if ex.getCause.isInstanceOf[MetaException] => + case ex: HiveException if ex.getCause.isInstanceOf[MetaException] => logWarning("Caught Hive MetaException attempting to get partition metadata by " + "filter from client side. Falling back to fetching all partition metadata", ex) recordHiveCall() From 8105e0c9f93bd4c93d364646766d5ee5e057d19a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 8 Oct 2023 04:44:55 -0700 Subject: [PATCH 045/521] [MINOR][DOCS] Update `CTAS` with `LOCATION` behavior with Spark 3.2+ ### What changes were proposed in this pull request? This PR aims to update `CTAS` with `LOCATION` behavior according to Spark 3.2+. ### Why are the changes needed? SPARK-28551 changed the behavior at Apache Spark 3.2.0. https://github.com/apache/spark/blob/24b82dfd6cfb9a658af615446be5423695830dd9/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L2306-L2313 ### Does this PR introduce _any_ user-facing change? No. This is a documentation fix. ### How was this patch tested? N/A ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43277 from dongjoon-hyun/minor_ctas. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 2d6d09b71e77b362a4c774170e2ca992a31fb1ea) Signed-off-by: Dongjoon Hyun --- docs/sql-ref-syntax-ddl-create-table-datasource.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-ddl-create-table-datasource.md b/docs/sql-ref-syntax-ddl-create-table-datasource.md index 7920a8a558e3d..f645732a15df9 100644 --- a/docs/sql-ref-syntax-ddl-create-table-datasource.md +++ b/docs/sql-ref-syntax-ddl-create-table-datasource.md @@ -104,7 +104,9 @@ In general CREATE TABLE is creating a "pointer", and you need to make sure it po existing. An exception is file source such as parquet, json. If you don't specify the LOCATION, Spark will create a default table location for you. -For CREATE TABLE AS SELECT, Spark will overwrite the underlying data source with the data of the +For CREATE TABLE AS SELECT with LOCATION, Spark throws analysis exceptions if the given location +exists as a non-empty directory. If `spark.sql.legacy.allowNonEmptyLocationInCTAS` is set to true, +Spark overwrites the underlying data source with the data of the input query, to make sure the table gets created contains exactly the same data as the input query. ### Examples From 5f8ae9a3dbd2c7624bffd588483c9916c302c081 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Mon, 9 Oct 2023 12:30:20 +0300 Subject: [PATCH 046/521] [SPARK-45424][SQL] Fix TimestampFormatter return optional parse results when only prefix match ### What changes were proposed in this pull request? When use custom pattern to parse timestamp, if there have matched prefix, not matched all. The `Iso8601TimestampFormatter::parseOptional` and `Iso8601TimestampFormatter::parseWithoutTimeZoneOptional` should not return not empty result. eg: pattern = `yyyy-MM-dd HH:mm:ss`, value = `9999-12-31 23:59:59.999`. If fact, `yyyy-MM-dd HH:mm:ss` can parse `9999-12-31 23:59:59` normally, but value have suffix `.999`. so we can't return not empty result. This bug will affect inference the schema in CSV/JSON. ### Why are the changes needed? Fix inference the schema bug. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? add new test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43245 from Hisoka-X/SPARK-45424-inference-schema-unresolved. Authored-by: Jia Fan Signed-off-by: Max Gekk (cherry picked from commit 4493b431192fcdbab1379b7ffb89eea0cdaa19f1) Signed-off-by: Max Gekk --- .../spark/sql/catalyst/util/TimestampFormatter.scala | 10 ++++++---- .../sql/catalyst/util/TimestampFormatterSuite.scala | 10 ++++++++++ 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index 8a288d0e9f3a3..55eee41c14ca5 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -167,8 +167,9 @@ class Iso8601TimestampFormatter( override def parseOptional(s: String): Option[Long] = { try { - val parsed = formatter.parseUnresolved(s, new ParsePosition(0)) - if (parsed != null) { + val parsePosition = new ParsePosition(0) + val parsed = formatter.parseUnresolved(s, parsePosition) + if (parsed != null && s.length == parsePosition.getIndex) { Some(extractMicros(parsed)) } else { None @@ -196,8 +197,9 @@ class Iso8601TimestampFormatter( override def parseWithoutTimeZoneOptional(s: String, allowTimeZone: Boolean): Option[Long] = { try { - val parsed = formatter.parseUnresolved(s, new ParsePosition(0)) - if (parsed != null) { + val parsePosition = new ParsePosition(0) + val parsed = formatter.parseUnresolved(s, parsePosition) + if (parsed != null && s.length == parsePosition.getIndex) { Some(extractMicrosNTZ(s, parsed, allowTimeZone)) } else { None diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala index eb173bc7f8c87..2134a0d6ecd36 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala @@ -507,4 +507,14 @@ class TimestampFormatterSuite extends DatetimeFormatterSuite { assert(simpleFormatter.parseOptional("abc").isEmpty) } + + test("SPARK-45424: do not return optional parse results when only prefix match") { + val formatter = new Iso8601TimestampFormatter( + "yyyy-MM-dd HH:mm:ss", + locale = DateFormatter.defaultLocale, + legacyFormat = LegacyDateFormats.SIMPLE_DATE_FORMAT, + isParsing = true, zoneId = DateTimeTestUtils.LA) + assert(formatter.parseOptional("9999-12-31 23:59:59.999").isEmpty) + assert(formatter.parseWithoutTimeZoneOptional("9999-12-31 23:59:59.999", true).isEmpty) + } } From 4841a404be3c37fc16031a0119b321eefcb2faab Mon Sep 17 00:00:00 2001 From: panbingkun Date: Mon, 9 Oct 2023 12:32:14 +0300 Subject: [PATCH 047/521] [SPARK-45459][SQL][TESTS][DOCS] Remove the last 2 extra spaces in the automatically generated `sql-error-conditions.md` file ### What changes were proposed in this pull request? The pr aims to remove the last 2 extra spaces in the automatically generated `sql-error-conditions.md` file. ### Why are the changes needed? - When I am work on another PR, I use the following command: ``` SPARK_GENERATE_GOLDEN_FILES=1 build/sbt \ "core/testOnly *SparkThrowableSuite -- -t \"Error classes match with document\"" ``` I found that in the automatically generated `sql-error-conditions.md` file, there are 2 extra spaces added at the end, Obviously, this is not what we expected, otherwise we would need to manually remove it, which is not in line with automation. - The git tells us this difference, as follows: image ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Pass GA. - Manually test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43274 from panbingkun/SPARK-45459. Authored-by: panbingkun Signed-off-by: Max Gekk (cherry picked from commit af800b505956ff26e03c5fc56b6cb4ac5c0efe2f) Signed-off-by: Max Gekk --- .../test/scala/org/apache/spark/SparkThrowableSuite.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala index 0249cde54884b..299bcea3f9e23 100644 --- a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala @@ -253,8 +253,7 @@ class SparkThrowableSuite extends SparkFunSuite { | |Also see [SQLSTATE Codes](sql-error-conditions-sqlstates.html). | - |$sqlErrorParentDocContent - |""".stripMargin + |$sqlErrorParentDocContent""".stripMargin errors.filter(_._2.subClass.isDefined).foreach(error => { val name = error._1 @@ -316,7 +315,7 @@ class SparkThrowableSuite extends SparkFunSuite { } FileUtils.writeStringToFile( parentDocPath.toFile, - sqlErrorParentDoc + lineSeparator, + sqlErrorParentDoc, StandardCharsets.UTF_8) } } else { From 8bf5a5bca3f9f7db78182d14e56476d384f442fa Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 9 Oct 2023 22:15:45 +0300 Subject: [PATCH 048/521] [SPARK-45383][SQL] Fix error message for time travel with non-existing table ### What changes were proposed in this pull request? Fixes a small bug to report `TABLE_OR_VIEW_NOT_FOUND` error correctly for time travel. It was missed before because `RelationTimeTravel` is a leaf node but it may contain `UnresolvedRelation`. ### Why are the changes needed? bug fix ### Does this PR introduce _any_ user-facing change? Yes, the error message becomes reasonable ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #43298 from cloud-fan/time-travel. Authored-by: Wenchen Fan Signed-off-by: Max Gekk (cherry picked from commit ced321c8b5a32c69dfb2841d4bec8a03f21b8038) Signed-off-by: Max Gekk --- .../spark/sql/catalyst/analysis/CheckAnalysis.scala | 4 ++++ .../spark/sql/connector/DataSourceV2SQLSuite.scala | 11 +++++++++++ 2 files changed, 15 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 511f3622e7e35..533ea8a2b7998 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -365,6 +365,9 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB }) operator match { + case RelationTimeTravel(u: UnresolvedRelation, _, _) => + u.tableNotFound(u.multipartIdentifier) + case etw: EventTimeWatermark => etw.eventTime.dataType match { case s: StructType @@ -377,6 +380,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB "eventName" -> toSQLId(etw.eventTime.name), "eventType" -> toSQLType(etw.eventTime.dataType))) } + case f: Filter if f.condition.dataType != BooleanType => f.failAnalysis( errorClass = "DATATYPE_MISMATCH.FILTER_NOT_BOOLEAN", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 06f5600e0d199..7745e9c0a4ee7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -3014,6 +3014,17 @@ class DataSourceV2SQLSuiteV1Filter sqlState = None, parameters = Map("relationId" -> "`x`")) + checkError( + exception = intercept[AnalysisException] { + sql("SELECT * FROM non_exist VERSION AS OF 1") + }, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`non_exist`"), + context = ExpectedContext( + fragment = "non_exist", + start = 14, + stop = 22)) + val subquery1 = "SELECT 1 FROM non_exist" checkError( exception = intercept[AnalysisException] { From ac4b9154b5822779023e66f2efb24d05e20b1cca Mon Sep 17 00:00:00 2001 From: Chaoqin Li Date: Tue, 10 Oct 2023 11:03:19 +0900 Subject: [PATCH 049/521] [SPARK-45419][SS] Avoid reusing rocksdb sst files in a dfferent rocksdb instance ### What changes were proposed in this pull request? When loading a rocksdb instance, remove file version map entry of larger versions to avoid rocksdb sst file unique id mismatch exception. The SST files in larger versions can't be reused even if they have the same size and name because they belong to another rocksdb instance. ### Why are the changes needed? Avoid rocksdb file mismatch exception that may occur in runtime. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Add rocksdb unit test. Closes #43174 from chaoqin-li1123/rocksdb_mismatch. Authored-by: Chaoqin Li Signed-off-by: Jungtaek Lim --- .../streaming/state/RocksDBFileManager.scala | 4 +++ .../streaming/state/RocksDBSuite.scala | 29 +++++++++++++++++++ 2 files changed, 33 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala index 0891d7737135a..faf9cd701aeca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala @@ -207,6 +207,10 @@ class RocksDBFileManager( */ def loadCheckpointFromDfs(version: Long, localDir: File): RocksDBCheckpointMetadata = { logInfo(s"Loading checkpoint files for version $version") + // The unique ids of SST files are checked when opening a rocksdb instance. The SST files + // in larger versions can't be reused even if they have the same size and name because + // they belong to another rocksdb instance. + versionToRocksDBFiles.keySet().removeIf(_ >= version) val metadata = if (version == 0) { if (localDir.exists) Utils.deleteRecursively(localDir) localDir.mkdirs() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index e31b05c362f6a..91dd858220717 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -214,6 +214,35 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } + testWithChangelogCheckpointingEnabled("SPARK-45419: Do not reuse SST files" + + " in different RocksDB instances") { + val remoteDir = Utils.createTempDir().toString + val conf = dbConf.copy(minDeltasForSnapshot = 0, compactOnCommit = false) + new File(remoteDir).delete() // to make sure that the directory gets created + withDB(remoteDir, conf = conf) { db => + for (version <- 0 to 2) { + db.load(version) + db.put(version.toString, version.toString) + db.commit() + } + // upload snapshot 3.zip + db.doMaintenance() + // Roll back to version 1 and start to process data. + for (version <- 1 to 3) { + db.load(version) + db.put(version.toString, version.toString) + db.commit() + } + // Upload snapshot 4.zip, should not reuse the SST files in 3.zip + db.doMaintenance() + } + + withDB(remoteDir, conf = conf) { db => + // Open the db to verify that the state in 4.zip is no corrupted. + db.load(4) + } + } + // A rocksdb instance with changelog checkpointing enabled should be able to load // an existing checkpoint without changelog. testWithChangelogCheckpointingEnabled( From 24f88b319c88bfe55e8b2b683193a85842bdad88 Mon Sep 17 00:00:00 2001 From: yorksity Date: Tue, 10 Oct 2023 14:36:23 +0800 Subject: [PATCH 050/521] [SPARK-45205][SQL] CommandResultExec to override iterator methods to avoid triggering multiple jobs MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? After SPARK-35378 was changed, the execution of statements such as ‘show parititions test' became slower. The change point is that the execution process changes from ExecutedCommandEnec to CommandResultExec, but ExecutedCommandExec originally implemented the following method override def executeToIterator(): Iterator[InternalRow] = sideEffectResult.iterator CommandResultExec is not rewritten, so when the hasNext method is executed, a job process is created, resulting in increased time-consuming ### Why are the changes needed? Improve performance when show partitions/tables. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests should cover this. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43270 from yorksity/SPARK-45205. Authored-by: yorksity Signed-off-by: Wenchen Fan (cherry picked from commit c9c99222e828d556552694dfb48c75bf0703a2c4) Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/execution/CommandResultExec.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CommandResultExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CommandResultExec.scala index 5f38278d2dc67..45e3e41ab053d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CommandResultExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CommandResultExec.scala @@ -81,6 +81,8 @@ case class CommandResultExec( unsafeRows } + override def executeToIterator(): Iterator[InternalRow] = unsafeRows.iterator + override def executeTake(limit: Int): Array[InternalRow] = { val taken = unsafeRows.take(limit) longMetric("numOutputRows").add(taken.size) From 81a7f8f184cd597208fcad72130354288a0c9f79 Mon Sep 17 00:00:00 2001 From: liangyongyuan Date: Tue, 10 Oct 2023 14:40:33 +0800 Subject: [PATCH 051/521] [SPARK-45449][SQL] Cache Invalidation Issue with JDBC Table ### What changes were proposed in this pull request? Add an equals method to `JDBCOptions` that considers two instances equal if their `JDBCOptions.parameters` are the same. ### Why are the changes needed? We have identified a cache invalidation issue when caching JDBC tables in Spark SQL. The cached table is unexpectedly invalidated when queried, leading to a re-read from the JDBC table instead of retrieving data from the cache. Example SQL: ``` CACHE TABLE cache_t SELECT * FROM mysql.test.test1; SELECT * FROM cache_t; ``` Expected Behavior: The expectation is that querying the cached table (cache_t) should retrieve the result from the cache without re-evaluating the execution plan. Actual Behavior: However, the cache is invalidated, and the content is re-read from the JDBC table. Root Cause: The issue lies in the `CacheData` class, where the comparison involves `JDBCTable`. The `JDBCTable` is a case class: `case class JDBCTable(ident: Identifier, schema: StructType, jdbcOptions: JDBCOptions)` The comparison of non-case class components, such as `jdbcOptions`, involves pointer comparison. This leads to unnecessary cache invalidation. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Add uts ### Was this patch authored or co-authored using generative AI tooling? No Closes #43258 from lyy-pineapple/spark-git-cache. Authored-by: liangyongyuan Signed-off-by: Wenchen Fan (cherry picked from commit d073f2d3e2f67a4b612e020a583e23dc1fa63aab) Signed-off-by: Wenchen Fan --- .../execution/datasources/jdbc/JDBCOptions.scala | 8 ++++++++ .../v2/jdbc/JDBCTableCatalogSuite.scala | 15 +++++++++++++++ 2 files changed, 23 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala index 268a65b81ff68..57651684070f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -239,6 +239,14 @@ class JDBCOptions( .get(JDBC_PREFER_TIMESTAMP_NTZ) .map(_.toBoolean) .getOrElse(SQLConf.get.timestampType == TimestampNTZType) + + override def hashCode: Int = this.parameters.hashCode() + + override def equals(other: Any): Boolean = other match { + case otherOption: JDBCOptions => + otherOption.parameters.equals(this.parameters) + case _ => false + } } class JdbcOptionsInWrite( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala index 6b85911dca773..eed64b873c451 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.util.CharVarcharUtils +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -512,4 +513,18 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { assert(t.schema === replaced) } } + + test("SPARK-45449: Cache Invalidation Issue with JDBC Table") { + withTable("h2.test.cache_t") { + withConnection { conn => + conn.prepareStatement( + """CREATE TABLE "test"."cache_t" (id decimal(25) PRIMARY KEY NOT NULL, + |name TEXT(32) NOT NULL)""".stripMargin).executeUpdate() + } + sql("INSERT OVERWRITE h2.test.cache_t SELECT 1 AS id, 'a' AS name") + sql("CACHE TABLE t1 SELECT id, name FROM h2.test.cache_t") + val plan = sql("select * from t1").queryExecution.sparkPlan + assert(plan.isInstanceOf[InMemoryTableScanExec]) + } + } } From 40d44a3fa4a59f6e98d076a3b3f006851f25be3a Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Tue, 10 Oct 2023 17:16:11 +0900 Subject: [PATCH 052/521] [SPARK-45475][SQL] Uses DataFrame.foreachPartition instead of RDD.foreachPartition in JdbcUtils This PR is kind of a followup for https://github.com/apache/spark/pull/39976 that addresses https://github.com/apache/spark/pull/39976#issuecomment-1752930380 comment. In order to probably assign the SQL execution ID so `df.observe` works with this. Yes. `df.observe` will work with JDBC connectors. Manually tested. Unit test was added. Closes #43304 from HyukjinKwon/foreachbatch. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit 39cc4abaff73cb49f9d79d1d844fe5c9fa14c917) Signed-off-by: Hyukjin Kwon --- .../execution/datasources/jdbc/JdbcUtils.scala | 2 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 17 ++++++++++++++++- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index b7019c1dcbe53..aae4be3f09483 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -898,7 +898,7 @@ object JdbcUtils extends Logging with SQLConfHelper { case Some(n) if n < df.rdd.getNumPartitions => df.coalesce(n) case _ => df } - repartitionedDF.rdd.foreachPartition { iterator => savePartition( + repartitionedDF.foreachPartition { iterator => savePartition( table, iterator, rddSchema, insertStmt, batchSize, dialect, isolationLevel, options) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 93b6652d516cc..eae171e20b706 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -29,7 +29,7 @@ import org.mockito.ArgumentMatchers._ import org.mockito.Mockito._ import org.apache.spark.{SparkException, SparkSQLException} -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} +import org.apache.spark.sql.{AnalysisException, DataFrame, Observation, QueryTest, Row} import org.apache.spark.sql.catalyst.{analysis, TableIdentifier} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.ShowCreateTable @@ -39,6 +39,7 @@ import org.apache.spark.sql.execution.command.{ExplainCommand, ShowCreateTableCo import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JDBCPartition, JDBCRelation, JdbcUtils} import org.apache.spark.sql.execution.metric.InputOutputMetricsHelper +import org.apache.spark.sql.functions.{lit, percentile_approx} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.test.SharedSparkSession @@ -2057,4 +2058,18 @@ class JDBCSuite extends QueryTest with SharedSparkSession { val df = sql("SELECT * FROM composite_name WHERE `last name` = 'smith'") assert(df.collect.toSet === Set(Row("smith", 1))) } + + test("SPARK-45475: saving a table via JDBC should work with observe API") { + val tableName = "test_table" + val namedObservation = Observation("named") + val observed_df = spark.range(100).observe( + namedObservation, percentile_approx($"id", lit(0.5), lit(100)).as("percentile_approx_val")) + + observed_df.write.format("jdbc") + .option("url", urlWithUserAndPass) + .option("dbtable", tableName).save() + + val expected = Map("percentile_approx_val" -> 49) + assert(namedObservation.get === expected) + } } From 80c166092eb691586eeaf7adb43f818f50c6cdea Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 11 Oct 2023 01:03:27 +0900 Subject: [PATCH 053/521] Revert "[SPARK-45475][SQL] Uses DataFrame.foreachPartition instead of RDD.foreachPartition in JdbcUtils" This reverts commit 40d44a3fa4a59f6e98d076a3b3f006851f25be3a. --- .../execution/datasources/jdbc/JdbcUtils.scala | 2 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 17 +---------------- 2 files changed, 2 insertions(+), 17 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index aae4be3f09483..b7019c1dcbe53 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -898,7 +898,7 @@ object JdbcUtils extends Logging with SQLConfHelper { case Some(n) if n < df.rdd.getNumPartitions => df.coalesce(n) case _ => df } - repartitionedDF.foreachPartition { iterator => savePartition( + repartitionedDF.rdd.foreachPartition { iterator => savePartition( table, iterator, rddSchema, insertStmt, batchSize, dialect, isolationLevel, options) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index eae171e20b706..93b6652d516cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -29,7 +29,7 @@ import org.mockito.ArgumentMatchers._ import org.mockito.Mockito._ import org.apache.spark.{SparkException, SparkSQLException} -import org.apache.spark.sql.{AnalysisException, DataFrame, Observation, QueryTest, Row} +import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.{analysis, TableIdentifier} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.ShowCreateTable @@ -39,7 +39,6 @@ import org.apache.spark.sql.execution.command.{ExplainCommand, ShowCreateTableCo import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JDBCPartition, JDBCRelation, JdbcUtils} import org.apache.spark.sql.execution.metric.InputOutputMetricsHelper -import org.apache.spark.sql.functions.{lit, percentile_approx} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.test.SharedSparkSession @@ -2058,18 +2057,4 @@ class JDBCSuite extends QueryTest with SharedSparkSession { val df = sql("SELECT * FROM composite_name WHERE `last name` = 'smith'") assert(df.collect.toSet === Set(Row("smith", 1))) } - - test("SPARK-45475: saving a table via JDBC should work with observe API") { - val tableName = "test_table" - val namedObservation = Observation("named") - val observed_df = spark.range(100).observe( - namedObservation, percentile_approx($"id", lit(0.5), lit(100)).as("percentile_approx_val")) - - observed_df.write.format("jdbc") - .option("url", urlWithUserAndPass) - .option("dbtable", tableName).save() - - val expected = Map("percentile_approx_val" -> 49) - assert(namedObservation.get === expected) - } } From 22e924778eef9a36c360e3e2a479c9401e6c58a1 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 10 Oct 2023 19:49:20 -0700 Subject: [PATCH 054/521] [SPARK-45473][SQL][3.5] Fix incorrect error message for RoundBase ### What changes were proposed in this pull request? This minor patch fixes incorrect error message of `RoundBase`. ### Why are the changes needed? Fix incorrect error message. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit test ### Was this patch authored or co-authored using generative AI tooling? No Closes #43315 from viirya/minor_fix-3.5. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/expressions/mathExpressions.scala | 2 +- .../sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala index add59a38b7201..b9a2cb348e380 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala @@ -1509,7 +1509,7 @@ abstract class RoundBase(child: Expression, scale: Expression, DataTypeMismatch( errorSubClass = "NON_FOLDABLE_INPUT", messageParameters = Map( - "inputName" -> "scala", + "inputName" -> "scale", "inputType" -> toSQLType(scale.dataType), "inputExpr" -> toSQLExpr(scale))) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala index 665204cd0c58e..08be4c8acc4b5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala @@ -602,7 +602,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer errorClass = "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", parameters = Map( "sqlExpr" -> "\"round(intField, intField)\"", - "inputName" -> "scala", + "inputName" -> "scale", "inputType" -> "\"INT\"", "inputExpr" -> "\"intField\"")) @@ -649,7 +649,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite with SQLHelper with Quer errorClass = "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", parameters = Map( "sqlExpr" -> "\"bround(intField, intField)\"", - "inputName" -> "scala", + "inputName" -> "scale", "inputType" -> "\"INT\"", "inputExpr" -> "\"intField\"")) checkError( From 04e6b713ab5e4a607254f331a0ea1e331ae6c857 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 11 Oct 2023 14:15:40 +0900 Subject: [PATCH 055/521] [SPARK-45475][SQL][3.5] Uses DataFrame.foreachPartition instead of RDD.foreachPartition in JdbcUtils This PR cherry-picks https://github.com/apache/spark/pull/43304 to branch-3.5 --- ### What changes were proposed in this pull request? This PR is kind of a followup for https://github.com/apache/spark/pull/39976 that addresses https://github.com/apache/spark/pull/39976#issuecomment-1752930380 comment. ### Why are the changes needed? In order to probably assign the SQL execution ID so `df.observe` works with this. ### Does this PR introduce _any_ user-facing change? Yes. `df.observe` will work with JDBC connectors. ### How was this patch tested? Manually tested. ### Was this patch authored or co-authored using generative AI tooling? Unit test was added. Closes #43322 from HyukjinKwon/SPARK-45475-3.5. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .../execution/datasources/jdbc/JdbcUtils.scala | 2 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 17 ++++++++++++++++- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index b7019c1dcbe53..6e7298710a5d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -898,7 +898,7 @@ object JdbcUtils extends Logging with SQLConfHelper { case Some(n) if n < df.rdd.getNumPartitions => df.coalesce(n) case _ => df } - repartitionedDF.rdd.foreachPartition { iterator => savePartition( + repartitionedDF.foreachPartition { iterator: Iterator[Row] => savePartition( table, iterator, rddSchema, insertStmt, batchSize, dialect, isolationLevel, options) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 93b6652d516cc..eae171e20b706 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -29,7 +29,7 @@ import org.mockito.ArgumentMatchers._ import org.mockito.Mockito._ import org.apache.spark.{SparkException, SparkSQLException} -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} +import org.apache.spark.sql.{AnalysisException, DataFrame, Observation, QueryTest, Row} import org.apache.spark.sql.catalyst.{analysis, TableIdentifier} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.ShowCreateTable @@ -39,6 +39,7 @@ import org.apache.spark.sql.execution.command.{ExplainCommand, ShowCreateTableCo import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JDBCPartition, JDBCRelation, JdbcUtils} import org.apache.spark.sql.execution.metric.InputOutputMetricsHelper +import org.apache.spark.sql.functions.{lit, percentile_approx} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.test.SharedSparkSession @@ -2057,4 +2058,18 @@ class JDBCSuite extends QueryTest with SharedSparkSession { val df = sql("SELECT * FROM composite_name WHERE `last name` = 'smith'") assert(df.collect.toSet === Set(Row("smith", 1))) } + + test("SPARK-45475: saving a table via JDBC should work with observe API") { + val tableName = "test_table" + val namedObservation = Observation("named") + val observed_df = spark.range(100).observe( + namedObservation, percentile_approx($"id", lit(0.5), lit(100)).as("percentile_approx_val")) + + observed_df.write.format("jdbc") + .option("url", urlWithUserAndPass) + .option("dbtable", tableName).save() + + val expected = Map("percentile_approx_val" -> 49) + assert(namedObservation.get === expected) + } } From 7e3ddc1e582a6e4fa96bab608c4c2bbc2c93b449 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Wed, 11 Oct 2023 19:33:23 +0300 Subject: [PATCH 056/521] [SPARK-45433][SQL] Fix CSV/JSON schema inference when timestamps do not match specified timestampFormat ### What changes were proposed in this pull request? This PR fix CSV/JSON schema inference when timestamps do not match specified timestampFormat will report error. ```scala //eg val csv = spark.read.option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss") .option("inferSchema", true).csv(Seq("2884-06-24T02:45:51.138").toDS()) csv.show() //error Caused by: java.time.format.DateTimeParseException: Text '2884-06-24T02:45:51.138' could not be parsed, unparsed text found at index 19 ``` This bug only happend when partition had one row. The data type should be `StringType` not `TimestampType` because the value not match `timestampFormat`. Use csv as eg, in `CSVInferSchema::tryParseTimestampNTZ`, first, use `timestampNTZFormatter.parseWithoutTimeZoneOptional` to inferring return `TimestampType`, if same partition had another row, it will use `tryParseTimestamp` to parse row with user defined `timestampFormat`, then found it can't be convert to timestamp with `timestampFormat`. Finally return `StringType`. But when only one row, we use `timestampNTZFormatter.parseWithoutTimeZoneOptional` to parse normally timestamp not right. We should only parse it when `spark.sql.timestampType` is `TIMESTAMP_NTZ`. If `spark.sql.timestampType` is `TIMESTAMP_LTZ`, we should directly parse it use `tryParseTimestamp`. To avoid return `TimestampType` when timestamps do not match specified timestampFormat. ### Why are the changes needed? Fix schema inference bug. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? add new test. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43243 from Hisoka-X/SPARK-45433-inference-mismatch-timestamp-one-row. Authored-by: Jia Fan Signed-off-by: Max Gekk (cherry picked from commit eae5c0e1efce83c2bb08754784db070be285285a) Signed-off-by: Max Gekk --- .../apache/spark/sql/catalyst/csv/CSVInferSchema.scala | 9 ++++++--- .../spark/sql/catalyst/json/JsonInferSchema.scala | 8 +++++--- .../spark/sql/catalyst/csv/CSVInferSchemaSuite.scala | 10 ++++++++++ .../spark/sql/catalyst/json/JsonInferSchemaSuite.scala | 8 ++++++++ 4 files changed, 29 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala index 51586a0065e95..ec01b56f9eb7c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.util.{DateFormatter, TimestampFormatter} import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} import org.apache.spark.sql.types._ class CSVInferSchema(val options: CSVOptions) extends Serializable { @@ -202,8 +202,11 @@ class CSVInferSchema(val options: CSVOptions) extends Serializable { // We can only parse the value as TimestampNTZType if it does not have zone-offset or // time-zone component and can be parsed with the timestamp formatter. // Otherwise, it is likely to be a timestamp with timezone. - if (timestampNTZFormatter.parseWithoutTimeZoneOptional(field, false).isDefined) { - SQLConf.get.timestampType + val timestampType = SQLConf.get.timestampType + if ((SQLConf.get.legacyTimeParserPolicy == LegacyBehaviorPolicy.LEGACY || + timestampType == TimestampNTZType) && + timestampNTZFormatter.parseWithoutTimeZoneOptional(field, false).isDefined) { + timestampType } else { tryParseTimestamp(field) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala index 5385afe8c9353..4123c5290b6a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.json.JacksonUtils.nextUntil import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -148,11 +148,13 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable { val bigDecimal = decimalParser(field) DecimalType(bigDecimal.precision, bigDecimal.scale) } + val timestampType = SQLConf.get.timestampType if (options.prefersDecimal && decimalTry.isDefined) { decimalTry.get - } else if (options.inferTimestamp && + } else if (options.inferTimestamp && (SQLConf.get.legacyTimeParserPolicy == + LegacyBehaviorPolicy.LEGACY || timestampType == TimestampNTZType) && timestampNTZFormatter.parseWithoutTimeZoneOptional(field, false).isDefined) { - SQLConf.get.timestampType + timestampType } else if (options.inferTimestamp && timestampFormatter.parseOptional(field).isDefined) { TimestampType diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala index acedf7998c2d5..fb91200557a65 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala @@ -263,4 +263,14 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { inferSchema = new CSVInferSchema(options) assert(inferSchema.inferField(DateType, "2012_12_12") == DateType) } + + test("SPARK-45433: inferring the schema when timestamps do not match specified timestampFormat" + + " with only one row") { + val options = new CSVOptions( + Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss"), + columnPruning = false, + defaultTimeZoneId = "UTC") + val inferSchema = new CSVInferSchema(options) + assert(inferSchema.inferField(NullType, "2884-06-24T02:45:51.138") == StringType) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala index 8290b38e33934..81a4858dce82a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala @@ -112,4 +112,12 @@ class JsonInferSchemaSuite extends SparkFunSuite with SQLHelper { checkType(Map("inferTimestamp" -> "true"), json, TimestampType) checkType(Map("inferTimestamp" -> "false"), json, StringType) } + + test("SPARK-45433: inferring the schema when timestamps do not match specified timestampFormat" + + " with only one row") { + checkType( + Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss", "inferTimestamp" -> "true"), + """{"a": "2884-06-24T02:45:51.138"}""", + StringType) + } } From 249533bcc8c7fa7f578961ce21d4d7118565dfc1 Mon Sep 17 00:00:00 2001 From: srielau Date: Thu, 12 Oct 2023 21:34:49 +0800 Subject: [PATCH 057/521] [SPARK-45132][SQL] Fix IDENTIFIER for function invocation ### What changes were proposed in this pull request? Due to a quirk in the parser, in some cases, IDENTIFIER()() is not properly recognized as a function invocation. The change is to remove the explicit IDENTIFIER-clause rule in the function invocation grammar and instead recognize IDENTIFIER() within visitFunctionCall. ### Why are the changes needed? Function invocation support for IDENTIFIER is incomplete otherwise ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added new testcases to identifier-clause.sql ### Was this patch authored or co-authored using generative AI tooling? No Closes #42888 from srielau/SPARK-45132. Lead-authored-by: srielau Co-authored-by: Wenchen Fan Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit f0b2e6da52113802f64f7879f207064d3bdbc7b0) Signed-off-by: Wenchen Fan --- .../sql/catalyst/parser/SqlBaseParser.g4 | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 43 +++++++++++-------- .../identifier-clause.sql.out | 28 ++++++++++-- .../sql-tests/inputs/identifier-clause.sql | 3 +- .../results/identifier-clause.sql.out | 27 +++++++++++- 5 files changed, 77 insertions(+), 26 deletions(-) diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 85dbc499fbde5..04128216be073 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -951,7 +951,6 @@ primaryExpression | qualifiedName DOT ASTERISK #star | LEFT_PAREN namedExpression (COMMA namedExpression)+ RIGHT_PAREN #rowConstructor | LEFT_PAREN query RIGHT_PAREN #subqueryExpression - | IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN #identifierClause | functionName LEFT_PAREN (setQuantifier? argument+=functionArgument (COMMA argument+=functionArgument)*)? RIGHT_PAREN (FILTER LEFT_PAREN WHERE where=booleanExpression RIGHT_PAREN)? @@ -1176,6 +1175,7 @@ qualifiedNameList functionName : IDENTIFIER_KW LEFT_PAREN expression RIGHT_PAREN + | identFunc=IDENTIFIER_KW // IDENTIFIER itself is also a valid function name. | qualifiedName | FILTER | LEFT diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 83938632e534f..b80ea8fddcfe2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2223,13 +2223,6 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { } } - /** - * Create an expression for the IDENTIFIER() clause. - */ - override def visitIdentifierClause(ctx: IdentifierClauseContext): Expression = withOrigin(ctx) { - ExpressionWithUnresolvedIdentifier(expression(ctx.expression), UnresolvedAttribute(_)) - } - /** * Create a (windowed) Function expression. */ @@ -2251,19 +2244,31 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { val filter = Option(ctx.where).map(expression(_)) val ignoreNulls = Option(ctx.nullsOption).map(_.getType == SqlBaseParser.IGNORE).getOrElse(false) - val funcCtx = ctx.functionName - val func = withFuncIdentClause( - funcCtx, - ident => UnresolvedFunction(ident, arguments, isDistinct, filter, ignoreNulls) - ) - // Check if the function is evaluated in a windowed context. - ctx.windowSpec match { - case spec: WindowRefContext => - UnresolvedWindowExpression(func, visitWindowRef(spec)) - case spec: WindowDefContext => - WindowExpression(func, visitWindowDef(spec)) - case _ => func + // Is this an IDENTIFIER clause instead of a function call? + if (ctx.functionName.identFunc != null && + arguments.length == 1 && // One argument + ctx.setQuantifier == null && // No other clause + ctx.where == null && + ctx.nullsOption == null && + ctx.windowSpec == null) { + ExpressionWithUnresolvedIdentifier(arguments.head, UnresolvedAttribute(_)) + } else { + // It's a function call + val funcCtx = ctx.functionName + val func = withFuncIdentClause( + funcCtx, + ident => UnresolvedFunction(ident, arguments, isDistinct, filter, ignoreNulls) + ) + + // Check if the function is evaluated in a windowed context. + ctx.windowSpec match { + case spec: WindowRefContext => + UnresolvedWindowExpression(func, visitWindowRef(spec)) + case spec: WindowDefContext => + WindowExpression(func, visitWindowDef(spec)) + case _ => func + } } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out index 00e2d8ff8ae75..f91d0a26cf8a4 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out @@ -187,10 +187,11 @@ Project [coalesce(cast(null as int), 1) AS coalesce(NULL, 1)#x] -- !query -SELECT IDENTIFIER('abs')(-1) +SELECT IDENTIFIER('abs')(c1) FROM VALUES(-1) AS T(c1) -- !query analysis -Project [abs(-1) AS abs(-1)#x] -+- OneRowRelation +Project [abs(c1#x) AS abs(c1)#x] ++- SubqueryAlias T + +- LocalRelation [c1#x] -- !query @@ -664,6 +665,27 @@ org.apache.spark.sql.AnalysisException } +-- !query +SELECT `IDENTIFIER`('abs')(c1) FROM VALUES(-1) AS T(c1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`IDENTIFIER`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "`IDENTIFIER`('abs')" + } ] +} + + -- !query CREATE TABLE IDENTIFIER(1)(c1 INT) -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql index a1bd500455de9..07ae157072938 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql @@ -36,7 +36,7 @@ DROP SCHEMA s; -- Function reference SELECT IDENTIFIER('COAL' || 'ESCE')(NULL, 1); -SELECT IDENTIFIER('abs')(-1); +SELECT IDENTIFIER('abs')(c1) FROM VALUES(-1) AS T(c1); SELECT * FROM IDENTIFIER('ra' || 'nge')(0, 1); -- Table DDL @@ -107,6 +107,7 @@ SELECT IDENTIFIER('') FROM VALUES(1) AS T(``); VALUES(IDENTIFIER(CAST(NULL AS STRING))); VALUES(IDENTIFIER(1)); VALUES(IDENTIFIER(SUBSTR('HELLO', 1, RAND() + 1))); +SELECT `IDENTIFIER`('abs')(c1) FROM VALUES(-1) AS T(c1); CREATE TABLE IDENTIFIER(1)(c1 INT); CREATE TABLE IDENTIFIER('a.b.c')(c1 INT); diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out index 8eabb74da97ba..ed87f69fc5e6b 100644 --- a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out @@ -205,9 +205,9 @@ struct -- !query -SELECT IDENTIFIER('abs')(-1) +SELECT IDENTIFIER('abs')(c1) FROM VALUES(-1) AS T(c1) -- !query schema -struct +struct -- !query output 1 @@ -770,6 +770,29 @@ org.apache.spark.sql.AnalysisException } +-- !query +SELECT `IDENTIFIER`('abs')(c1) FROM VALUES(-1) AS T(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`IDENTIFIER`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "`IDENTIFIER`('abs')" + } ] +} + + -- !query CREATE TABLE IDENTIFIER(1)(c1 INT) -- !query schema From b5f3dc9e76082a81357555ace0c489df97e6f81a Mon Sep 17 00:00:00 2001 From: mayurb Date: Fri, 13 Oct 2023 10:17:56 +0800 Subject: [PATCH 058/521] =?UTF-8?q?[SPARK-45498][CORE]=20Followup:=20Ignor?= =?UTF-8?q?e=20task=20completion=20from=20old=20stage=20a=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? With [SPARK-45182](https://issues.apache.org/jira/browse/SPARK-45182), we added a fix for not letting laggard tasks of the older attempts of the indeterminate stage from marking the partition has completed in the map output tracker. When a task is completed, the DAG scheduler also notifies all the task sets of the stage about that partition being completed. Tasksets would not schedule such tasks if they are not already scheduled. This is not correct for the indeterminate stage, since we want to re-run all the tasks on a re-attempt ### Why are the changes needed? Since the partition is not completed by older attempts and the partition from the newer attempt also doesn't get scheduled, the stage will have to be rescheduled to complete that partition. Since the stage is indeterminate, all the partitions will be recomputed ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added check in existing unit test ### Was this patch authored or co-authored using generative AI tooling? No Closes #43326 from mayurdb/indeterminateFix. Authored-by: mayurb Signed-off-by: Wenchen Fan (cherry picked from commit fb3b707bc1c875c14ff7c6e7a3f39b5c4b852c86) Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 6 +++--- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 5 ++++- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index d73bb6339015b..d8adaae19b90d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1847,9 +1847,9 @@ private[spark] class DAGScheduler( case Success => // An earlier attempt of a stage (which is zombie) may still have running tasks. If these // tasks complete, they still count and we can mark the corresponding partitions as - // finished. Here we notify the task scheduler to skip running tasks for the same partition, - // to save resource. - if (task.stageAttemptId < stage.latestInfo.attemptNumber()) { + // finished if the stage is determinate. Here we notify the task scheduler to skip running + // tasks for the same partition to save resource. + if (!stage.isIndeterminate && task.stageAttemptId < stage.latestInfo.attemptNumber()) { taskScheduler.notifyPartitionCompletion(stageId, task.partitionId) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index e351f8b95bbb0..9b7c5d5ace314 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -3169,13 +3169,16 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti makeMapStatus("hostB", 2))) - // The second task of the shuffle map stage 1 from 1st attempt succeeds + // The second task of the shuffle map stage 1 from 1st attempt succeeds runEvent(makeCompletionEvent( taskSets(1).tasks(1), Success, makeMapStatus("hostC", 2))) + // Above task completion should not mark the partition 1 complete from 2nd attempt + assert(!tasksMarkedAsCompleted.contains(taskSets(3).tasks(1))) + // This task completion should get ignored and partition 1 should be missing // for shuffle map stage 1 assert(mapOutputTracker.findMissingPartitions(shuffleId2) == Some(Seq(1))) From 370717702c9e7236aab4ea7135d8085cd4792e99 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 13 Oct 2023 14:05:23 +0800 Subject: [PATCH 059/521] [SPARK-45532][DOCS] Restore codetabs for the Protobuf Data Source Guide ### What changes were proposed in this pull request? This PR restores the [Protobuf Data Source Guide](https://spark.apache.org/docs/latest/sql-data-sources-protobuf.html#python)'s code tabs which https://github.com/apache/spark/pull/40614 removed for markdown syntax fixes In this PR, we introduce a hidden div to hold the code-block marker of markdown, then make both the liquid and markdown happy. ### Why are the changes needed? improve doc readability and consistency. ### Does this PR introduce _any_ user-facing change? yes, doc change ### How was this patch tested? #### Doc build ![image](https://github.com/apache/spark/assets/8326978/8aefeee0-92b2-4048-a3f6-108e4c3f309d) #### markdown editor and view ![image](https://github.com/apache/spark/assets/8326978/283b0820-390a-4540-8713-647c40f956ac) ### Was this patch authored or co-authored using generative AI tooling? no Closes #43361 from yaooqinn/SPARK-45532. Authored-by: Kent Yao Signed-off-by: Kent Yao (cherry picked from commit 0257b77528a3a0d0ba08df5363470e4bc5928b06) Signed-off-by: Kent Yao --- docs/sql-data-sources-protobuf.md | 243 ++++++++++++++++++------------ 1 file changed, 150 insertions(+), 93 deletions(-) diff --git a/docs/sql-data-sources-protobuf.md b/docs/sql-data-sources-protobuf.md index f92a8f20b3570..c8ee139e344fe 100644 --- a/docs/sql-data-sources-protobuf.md +++ b/docs/sql-data-sources-protobuf.md @@ -18,7 +18,10 @@ license: | limitations under the License. --- -Since Spark 3.4.0 release, [Spark SQL](https://spark.apache.org/docs/latest/sql-programming-guide.html) provides built-in support for reading and writing protobuf data. +* This will become a table of contents (this text will be scraped). +{:toc} + +Since Spark 3.4.0 release, [Spark SQL](sql-programming-guide.html) provides built-in support for reading and writing protobuf data. ## Deploying The `spark-protobuf` module is external and not included in `spark-submit` or `spark-shell` by default. @@ -46,45 +49,53 @@ Kafka key-value record will be augmented with some metadata, such as the ingesti Spark SQL schema is generated based on the protobuf descriptor file or protobuf class passed to `from_protobuf` and `to_protobuf`. The specified protobuf class or protobuf descriptor file must match the data, otherwise, the behavior is undefined: it may fail or return arbitrary results. -### Python +
+ +
+ +
+This div is only used to make markdown editor/viewer happy and does not display on web + ```python +
+ +{% highlight python %} + from pyspark.sql.protobuf.functions import from_protobuf, to_protobuf -# `from_protobuf` and `to_protobuf` provides two schema choices. Via Protobuf descriptor file, +# from_protobuf and to_protobuf provide two schema choices. Via Protobuf descriptor file, # or via shaded Java class. # give input .proto protobuf schema -# syntax = "proto3" +# syntax = "proto3" # message AppEvent { -# string name = 1; -# int64 id = 2; -# string context = 3; +# string name = 1; +# int64 id = 2; +# string context = 3; # } - -df = spark\ -.readStream\ -.format("kafka")\ -.option("kafka.bootstrap.servers", "host1:port1,host2:port2")\ -.option("subscribe", "topic1")\ -.load() +df = spark + .readStream + .format("kafka")\ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load() # 1. Decode the Protobuf data of schema `AppEvent` into a struct; # 2. Filter by column `name`; # 3. Encode the column `event` in Protobuf format. # The Protobuf protoc command can be used to generate a protobuf descriptor file for give .proto file. -output = df\ -.select(from_protobuf("value", "AppEvent", descriptorFilePath).alias("event"))\ -.where('event.name == "alice"')\ -.select(to_protobuf("event", "AppEvent", descriptorFilePath).alias("event")) +output = df + .select(from_protobuf("value", "AppEvent", descriptorFilePath).alias("event")) + .where('event.name == "alice"') + .select(to_protobuf("event", "AppEvent", descriptorFilePath).alias("event")) # Alternatively, you can decode and encode the SQL columns into protobuf format using protobuf # class name. The specified Protobuf class must match the data, otherwise the behavior is undefined: # it may fail or return arbitrary result. To avoid conflicts, the jar file containing the # 'com.google.protobuf.*' classes should be shaded. An example of shading can be found at # https://github.com/rangadi/shaded-protobuf-classes. - -output = df\ -.select(from_protobuf("value", "org.sparkproject.spark_protobuf.protobuf.AppEvent").alias("event"))\ -.where('event.name == "alice"') +output = df + .select(from_protobuf("value", "org.sparkproject.spark_protobuf.protobuf.AppEvent").alias("event")) + .where('event.name == "alice"') output.printSchema() # root @@ -94,52 +105,66 @@ output.printSchema() # | |-- context: string (nullable = true) output = output -.select(to_protobuf("event", "org.sparkproject.spark_protobuf.protobuf.AppEvent").alias("event")) - -query = output\ -.writeStream\ -.format("kafka")\ -.option("kafka.bootstrap.servers", "host1:port1,host2:port2")\ -.option("topic", "topic2")\ -.start() + .select(to_protobuf("event", "org.sparkproject.spark_protobuf.protobuf.AppEvent").alias("event")) + +query = output + .writeStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2")\ + .option("topic", "topic2") + .start() + +{% endhighlight %} + +
``` +
+ +
+ +
+ +
+This div is only used to make markdown editor/viewer happy and does not display on web -### Scala ```scala +
+ +{% highlight scala %} import org.apache.spark.sql.protobuf.functions._ -// `from_protobuf` and `to_protobuf` provides two schema choices. Via Protobuf descriptor file, +// `from_protobuf` and `to_protobuf` provides two schema choices. Via the protobuf descriptor file, // or via shaded Java class. // give input .proto protobuf schema -// syntax = "proto3" +// syntax = "proto3" // message AppEvent { -// string name = 1; -// int64 id = 2; -// string context = 3; +// string name = 1; +// int64 id = 2; +// string context = 3; // } val df = spark -.readStream -.format("kafka") -.option("kafka.bootstrap.servers", "host1:port1,host2:port2") -.option("subscribe", "topic1") -.load() + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load() // 1. Decode the Protobuf data of schema `AppEvent` into a struct; // 2. Filter by column `name`; // 3. Encode the column `event` in Protobuf format. // The Protobuf protoc command can be used to generate a protobuf descriptor file for give .proto file. val output = df -.select(from_protobuf($"value", "AppEvent", descriptorFilePath) as $"event") -.where("event.name == \"alice\"") -.select(to_protobuf($"user", "AppEvent", descriptorFilePath) as $"event") + .select(from_protobuf($"value", "AppEvent", descriptorFilePath) as $"event") + .where("event.name == \"alice\"") + .select(to_protobuf($"user", "AppEvent", descriptorFilePath) as $"event") val query = output -.writeStream -.format("kafka") -.option("kafka.bootstrap.servers", "host1:port1,host2:port2") -.option("topic", "topic2") -.start() + .writeStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("topic", "topic2") + .start() // Alternatively, you can decode and encode the SQL columns into protobuf format using protobuf // class name. The specified Protobuf class must match the data, otherwise the behavior is undefined: @@ -147,8 +172,8 @@ val query = output // 'com.google.protobuf.*' classes should be shaded. An example of shading can be found at // https://github.com/rangadi/shaded-protobuf-classes. var output = df -.select(from_protobuf($"value", "org.example.protos..AppEvent") as $"event") -.where("event.name == \"alice\"") + .select(from_protobuf($"value", "org.example.protos..AppEvent") as $"event") + .where("event.name == \"alice\"") output.printSchema() // root @@ -160,43 +185,56 @@ output.printSchema() output = output.select(to_protobuf($"event", "org.sparkproject.spark_protobuf.protobuf.AppEvent") as $"event") val query = output -.writeStream -.format("kafka") -.option("kafka.bootstrap.servers", "host1:port1,host2:port2") -.option("topic", "topic2") -.start() + .writeStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("topic", "topic2") + .start() + +{% endhighlight %} + +
``` +
+
+ +
+ +
+This div is only used to make markdown editor/viewer happy and does not display on web -### Java ```java +
+ +{% highlight java %} import static org.apache.spark.sql.functions.col; import static org.apache.spark.sql.protobuf.functions.*; -// `from_protobuf` and `to_protobuf` provides two schema choices. Via Protobuf descriptor file, +// `from_protobuf` and `to_protobuf` provides two schema choices. Via the protobuf descriptor file, // or via shaded Java class. // give input .proto protobuf schema -// syntax = "proto3" +// syntax = "proto3" // message AppEvent { -// string name = 1; -// int64 id = 2; -// string context = 3; +// string name = 1; +// int64 id = 2; +// string context = 3; // } Dataset df = spark -.readStream() -.format("kafka") -.option("kafka.bootstrap.servers", "host1:port1,host2:port2") -.option("subscribe", "topic1") -.load(); + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load(); // 1. Decode the Protobuf data of schema `AppEvent` into a struct; // 2. Filter by column `name`; // 3. Encode the column `event` in Protobuf format. // The Protobuf protoc command can be used to generate a protobuf descriptor file for give .proto file. Dataset output = df -.select(from_protobuf(col("value"), "AppEvent", descriptorFilePath).as("event")) -.where("event.name == \"alice\"") -.select(to_protobuf(col("event"), "AppEvent", descriptorFilePath).as("event")); + .select(from_protobuf(col("value"), "AppEvent", descriptorFilePath).as("event")) + .where("event.name == \"alice\"") + .select(to_protobuf(col("event"), "AppEvent", descriptorFilePath).as("event")); // Alternatively, you can decode and encode the SQL columns into protobuf format using protobuf // class name. The specified Protobuf class must match the data, otherwise the behavior is undefined: @@ -204,10 +242,10 @@ Dataset output = df // 'com.google.protobuf.*' classes should be shaded. An example of shading can be found at // https://github.com/rangadi/shaded-protobuf-classes. Dataset output = df -.select( - from_protobuf(col("value"), - "org.sparkproject.spark_protobuf.protobuf.AppEvent").as("event")) -.where("event.name == \"alice\"") + .select( + from_protobuf(col("value"), + "org.sparkproject.spark_protobuf.protobuf.AppEvent").as("event")) + .where("event.name == \"alice\"") output.printSchema() // root @@ -221,19 +259,28 @@ output = output.select( "org.sparkproject.spark_protobuf.protobuf.AppEvent").as("event")); StreamingQuery query = output -.writeStream() -.format("kafka") -.option("kafka.bootstrap.servers", "host1:port1,host2:port2") -.option("topic", "topic2") -.start(); + .writeStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("topic", "topic2") + .start(); + +{% endhighlight %} + +
``` +
+
+ +
## Supported types for Protobuf -> Spark SQL conversion + Currently Spark supports reading [protobuf scalar types](https://developers.google.com/protocol-buffers/docs/proto3#scalar), [enum types](https://developers.google.com/protocol-buffers/docs/proto3#enum), [nested type](https://developers.google.com/protocol-buffers/docs/proto3#nested), and [maps type](https://developers.google.com/protocol-buffers/docs/proto3#maps) under messages of Protobuf. In addition to the these types, `spark-protobuf` also introduces support for Protobuf `OneOf` fields. which allows you to handle messages that can have multiple possible sets of fields, but only one set can be present at a time. This is useful for situations where the data you are working with is not always in the same format, and you need to be able to handle messages with different sets of fields without encountering errors. - - +
Protobuf typeSpark SQL type
+ @@ -282,16 +329,12 @@ In addition to the these types, `spark-protobuf` also introduces support for Pro - - - -
Protobuf typeSpark SQL type
boolean BooleanTypeOneOf Struct
AnyStructType
It also supports reading the following Protobuf types [Timestamp](https://developers.google.com/protocol-buffers/docs/reference/google.protobuf#timestamp) and [Duration](https://developers.google.com/protocol-buffers/docs/reference/google.protobuf#duration) - - +
Protobuf logical typeProtobuf schemaSpark SQL type
+ @@ -305,10 +348,11 @@ It also supports reading the following Protobuf types [Timestamp](https://develo
Protobuf logical typeProtobuf schemaSpark SQL type
duration MessageType{seconds: Long, nanos: Int}
## Supported types for Spark SQL -> Protobuf conversion + Spark supports the writing of all Spark SQL types into Protobuf. For most types, the mapping from Spark types to Protobuf types is straightforward (e.g. IntegerType gets converted to int); - - +
Spark SQL typeProtobuf type
+ @@ -356,15 +400,23 @@ Spark supports the writing of all Spark SQL types into Protobuf. For most types,
Spark SQL typeProtobuf type
BooleanType boolean
## Handling circular references protobuf fields + One common issue that can arise when working with Protobuf data is the presence of circular references. In Protobuf, a circular reference occurs when a field refers back to itself or to another field that refers back to the original field. This can cause issues when parsing the data, as it can result in infinite loops or other unexpected behavior. -To address this issue, the latest version of spark-protobuf introduces a new feature: the ability to check for circular references through field types. This allows users use the `recursive.fields.max.depth` option to specify the maximum number of levels of recursion to allow when parsing the schema. By default, `spark-protobuf` will not permit recursive fields by setting `recursive.fields.max.depth` to -1. However, you can set this option to 0 to 10 if needed. +To address this issue, the latest version of spark-protobuf introduces a new feature: the ability to check for circular references through field types. This allows users use the `recursive.fields.max.depth` option to specify the maximum number of levels of recursion to allow when parsing the schema. By default, `spark-protobuf` will not permit recursive fields by setting `recursive.fields.max.depth` to -1. However, you can set this option to 0 to 10 if needed. Setting `recursive.fields.max.depth` to 0 drops all recursive fields, setting it to 1 allows it to be recursed once, and setting it to 2 allows it to be recursed twice. A `recursive.fields.max.depth` value greater than 10 is not allowed, as it can lead to performance issues and even stack overflows. SQL Schema for the below protobuf message will vary based on the value of `recursive.fields.max.depth`. -```proto -syntax = "proto3" +
+
+This div is only used to make markdown editor/viewer happy and does not display on web + +```protobuf +
+ +{% highlight protobuf %} +syntax = "proto3" message Person { string name = 1; Person bff = 2 @@ -376,4 +428,9 @@ message Person { 0: struct 1: struct> 2: struct>> ... -``` \ No newline at end of file + +{% endhighlight %} +
+``` +
+
\ No newline at end of file From e8d7497abadd0bccb4bd8e615aadc77fc2038566 Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Fri, 13 Oct 2023 10:50:18 -0500 Subject: [PATCH 060/521] [SPARK-45495][CORE] Support stage level task resource profile for k8s cluster when dynamic allocation disabled ### What changes were proposed in this pull request? This PR is a follow-up of https://github.com/apache/spark/pull/37268 which supports stage-level task resource profile for standalone cluster when dynamic allocation is disabled. This PR enables stage-level task resource profile for the Kubernetes cluster. ### Why are the changes needed? Users who work on spark ML/DL cases running on Kubernetes would expect stage-level task resource profile feature. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? The current tests of https://github.com/apache/spark/pull/37268 can also cover this PR since both Kubernetes and standalone cluster share the same TaskSchedulerImpl class which implements this feature. Apart from that, modifying the existing test to cover the Kubernetes cluster. Apart from that, I also performed some manual tests which have been updated in the comments. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43323 from wbo4958/k8s-stage-level. Authored-by: Bobby Wang Signed-off-by: Thomas Graves (cherry picked from commit 632eabdb6dfb78c0a5dc84c01806548e1dc6dd0a) Signed-off-by: Thomas Graves --- .../apache/spark/resource/ResourceProfileManager.scala | 7 ++++--- .../spark/resource/ResourceProfileManagerSuite.scala | 9 +++++++-- docs/configuration.md | 2 +- docs/running-on-kubernetes.md | 4 +++- 4 files changed, 15 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala index cd7124a572464..afbacb8013645 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala @@ -67,9 +67,10 @@ private[spark] class ResourceProfileManager(sparkConf: SparkConf, */ private[spark] def isSupported(rp: ResourceProfile): Boolean = { if (rp.isInstanceOf[TaskResourceProfile] && !dynamicEnabled) { - if ((notRunningUnitTests || testExceptionThrown) && !(isStandaloneOrLocalCluster || isYarn)) { - throw new SparkException("TaskResourceProfiles are only supported for Standalone and " + - "Yarn cluster for now when dynamic allocation is disabled.") + if ((notRunningUnitTests || testExceptionThrown) && + !(isStandaloneOrLocalCluster || isYarn || isK8s)) { + throw new SparkException("TaskResourceProfiles are only supported for Standalone, " + + "Yarn and Kubernetes cluster for now when dynamic allocation is disabled.") } } else { val isNotDefaultProfile = rp.id != ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala index 77dc7bcb4c56e..7149267583bc5 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala @@ -137,8 +137,8 @@ class ResourceProfileManagerSuite extends SparkFunSuite { val error = intercept[SparkException] { rpmanager.isSupported(taskProf) }.getMessage - assert(error === "TaskResourceProfiles are only supported for Standalone " + - "and Yarn cluster for now when dynamic allocation is disabled.") + assert(error === "TaskResourceProfiles are only supported for Standalone, " + + "Yarn and Kubernetes cluster for now when dynamic allocation is disabled.") // Local cluster: supports task resource profile. conf.setMaster("local-cluster[1, 1, 1024]") @@ -149,6 +149,11 @@ class ResourceProfileManagerSuite extends SparkFunSuite { conf.setMaster("yarn") rpmanager = new ResourceProfileManager(conf, listenerBus) assert(rpmanager.isSupported(taskProf)) + + // K8s: supports task resource profile. + conf.setMaster("k8s://foo") + rpmanager = new ResourceProfileManager(conf, listenerBus) + assert(rpmanager.isSupported(taskProf)) } test("isSupported task resource profiles with dynamic allocation enabled") { diff --git a/docs/configuration.md b/docs/configuration.md index 74ddd6df0233c..4b0b9b3e3c260 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -3670,7 +3670,7 @@ See your cluster manager specific page for requirements and details on each of - # Stage Level Scheduling Overview The stage level scheduling feature allows users to specify task and executor resource requirements at the stage level. This allows for different stages to run with executors that have different resources. A prime example of this is one ETL stage runs with executors with just CPUs, the next stage is an ML stage that needs GPUs. Stage level scheduling allows for user to request different executors that have GPUs when the ML stage runs rather then having to acquire executors with GPUs at the start of the application and them be idle while the ETL stage is being run. -This is only available for the RDD API in Scala, Java, and Python. It is available on YARN, Kubernetes and Standalone when dynamic allocation is enabled. When dynamic allocation is disabled, it allows users to specify different task resource requirements at stage level, and this is supported on YARN and Standalone cluster right now. See the [YARN](running-on-yarn.html#stage-level-scheduling-overview) page or [Kubernetes](running-on-kubernetes.html#stage-level-scheduling-overview) page or [Standalone](spark-standalone.html#stage-level-scheduling-overview) page for more implementation details. +This is only available for the RDD API in Scala, Java, and Python. It is available on YARN, Kubernetes and Standalone when dynamic allocation is enabled. When dynamic allocation is disabled, it allows users to specify different task resource requirements at stage level, and this is supported on YARN, Kubernetes and Standalone cluster right now. See the [YARN](running-on-yarn.html#stage-level-scheduling-overview) page or [Kubernetes](running-on-kubernetes.html#stage-level-scheduling-overview) page or [Standalone](spark-standalone.html#stage-level-scheduling-overview) page for more implementation details. See the `RDD.withResources` and `ResourceProfileBuilder` API's for using this feature. When dynamic allocation is disabled, tasks with different task resource requirements will share executors with `DEFAULT_RESOURCE_PROFILE`. While when dynamic allocation is enabled, the current implementation acquires new executors for each `ResourceProfile` created and currently has to be an exact match. Spark does not try to fit tasks into an executor that require a different ResourceProfile than the executor was created with. Executors that are not in use will idle timeout with the dynamic allocation logic. The default configuration for this feature is to only allow one ResourceProfile per stage. If the user associates more then 1 ResourceProfile to an RDD, Spark will throw an exception by default. See config `spark.scheduler.resource.profileMergeConflicts` to control that behavior. The current merge strategy Spark implements when `spark.scheduler.resource.profileMergeConflicts` is enabled is a simple max of each resource within the conflicting ResourceProfiles. Spark will create a new ResourceProfile with the max of each of the resources. diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 707a76196f3ab..38a745f1afca3 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -1936,5 +1936,7 @@ With the above configuration, the job will be scheduled by YuniKorn scheduler in ### Stage Level Scheduling Overview -Stage level scheduling is supported on Kubernetes when dynamic allocation is enabled. This also requires spark.dynamicAllocation.shuffleTracking.enabled to be enabled since Kubernetes doesn't support an external shuffle service at this time. The order in which containers for different profiles is requested from Kubernetes is not guaranteed. Note that since dynamic allocation on Kubernetes requires the shuffle tracking feature, this means that executors from previous stages that used a different ResourceProfile may not idle timeout due to having shuffle data on them. This could result in using more cluster resources and in the worst case if there are no remaining resources on the Kubernetes cluster then Spark could potentially hang. You may consider looking at config spark.dynamicAllocation.shuffleTracking.timeout to set a timeout, but that could result in data having to be recomputed if the shuffle data is really needed. +Stage level scheduling is supported on Kubernetes: +- When dynamic allocation is disabled: It allows users to specify different task resource requirements at the stage level and will use the same executors requested at startup. +- When dynamic allocation is enabled: It allows users to specify task and executor resource requirements at the stage level and will request the extra executors. This also requires spark.dynamicAllocation.shuffleTracking.enabled to be enabled since Kubernetes doesn't support an external shuffle service at this time. The order in which containers for different profiles is requested from Kubernetes is not guaranteed. Note that since dynamic allocation on Kubernetes requires the shuffle tracking feature, this means that executors from previous stages that used a different ResourceProfile may not idle timeout due to having shuffle data on them. This could result in using more cluster resources and in the worst case if there are no remaining resources on the Kubernetes cluster then Spark could potentially hang. You may consider looking at config spark.dynamicAllocation.shuffleTracking.timeout to set a timeout, but that could result in data having to be recomputed if the shuffle data is really needed. Note, there is a difference in the way pod template resources are handled between the base default profile and custom ResourceProfiles. Any resources specified in the pod template file will only be used with the base default profile. If you create custom ResourceProfiles be sure to include all necessary resources there since the resources from the template file will not be propagated to custom ResourceProfiles. From 146fba1a22e3f1555f3e4494522810030f9a7854 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 13 Oct 2023 13:29:45 +0800 Subject: [PATCH 061/521] [SPARK-45508][CORE] Add "--add-opens=java.base/jdk.internal.ref=ALL-UNNAMED" so Platform can access Cleaner on Java 9+ This PR adds `--add-opens=java.base/jdk.internal.ref=ALL-UNNAMED` to our JVM flags so that we can access `jdk.internal.ref.Cleaner` in JDK 9+. This allows Spark to allocate direct memory while ignoring the JVM's MaxDirectMemorySize limit. Spark uses JDK internal APIs to directly construct DirectByteBuffers while bypassing that limit, but there is a fallback path at https://github.com/apache/spark/blob/v3.5.0/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java#L213 that is used if we cannot reflectively access the `Cleaner` API. No. Added a unit test in `PlatformUtilSuite`. No. Closes #43344 from JoshRosen/SPARK-45508. Authored-by: Josh Rosen Signed-off-by: yangjie01 (cherry picked from commit 96bac6c033b5bb37101ebcd8436ab9a84db8e092) Signed-off-by: Dongjoon Hyun --- .../src/main/java/org/apache/spark/unsafe/Platform.java | 7 ++++++- .../java/org/apache/spark/unsafe/PlatformUtilSuite.java | 7 +++++++ .../java/org/apache/spark/launcher/JavaModuleOptions.java | 1 + pom.xml | 1 + project/SparkBuild.scala | 1 + 5 files changed, 16 insertions(+), 1 deletion(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java index a91ea2ee6b5a8..e02346c477375 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java @@ -96,7 +96,7 @@ public final class Platform { Method createMethod = cleanerClass.getMethod("create", Object.class, Runnable.class); // Accessing jdk.internal.ref.Cleaner should actually fail by default in JDK 9+, // unfortunately, unless the user has allowed access with something like - // --add-opens java.base/java.lang=ALL-UNNAMED If not, we can't really use the Cleaner + // --add-opens java.base/jdk.internal.ref=ALL-UNNAMED If not, we can't use the Cleaner // hack below. It doesn't break, just means the user might run into the default JVM limit // on off-heap memory and increase it or set the flag above. This tests whether it's // available: @@ -118,6 +118,11 @@ public final class Platform { } } + // Visible for testing + public static boolean cleanerCreateMethodIsDefined() { + return CLEANER_CREATE_METHOD != null; + } + /** * @return true when running JVM is having sun's Unsafe package available in it and underlying * system having unaligned-access capability. diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java index c59878fea9941..c99f2d85f4e54 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java @@ -157,4 +157,11 @@ public void heapMemoryReuse() { Assert.assertEquals(1024 * 1024 + 7, onheap4.size()); Assert.assertEquals(obj3, onheap4.getBaseObject()); } + + @Test + public void cleanerCreateMethodIsDefined() { + // Regression test for SPARK-45508: we don't expect the "no cleaner" fallback + // path to be hit in normal usage. + Assert.assertTrue(Platform.cleanerCreateMethodIsDefined()); + } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/JavaModuleOptions.java b/launcher/src/main/java/org/apache/spark/launcher/JavaModuleOptions.java index 013dde2766f49..f6a9607e7c5d3 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/JavaModuleOptions.java +++ b/launcher/src/main/java/org/apache/spark/launcher/JavaModuleOptions.java @@ -36,6 +36,7 @@ public class JavaModuleOptions { "--add-opens=java.base/java.util=ALL-UNNAMED", "--add-opens=java.base/java.util.concurrent=ALL-UNNAMED", "--add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED", + "--add-opens=java.base/jdk.internal.ref=ALL-UNNAMED", "--add-opens=java.base/sun.nio.ch=ALL-UNNAMED", "--add-opens=java.base/sun.nio.cs=ALL-UNNAMED", "--add-opens=java.base/sun.security.action=ALL-UNNAMED", diff --git a/pom.xml b/pom.xml index 8fc4b89a78cc2..be8400c33bf2b 100644 --- a/pom.xml +++ b/pom.xml @@ -308,6 +308,7 @@ --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED + --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 563d53577548e..718f2bb28cec4 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1619,6 +1619,7 @@ object TestSettings { "--add-opens=java.base/java.util=ALL-UNNAMED", "--add-opens=java.base/java.util.concurrent=ALL-UNNAMED", "--add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED", + "--add-opens=java.base/jdk.internal.ref=ALL-UNNAMED", "--add-opens=java.base/sun.nio.ch=ALL-UNNAMED", "--add-opens=java.base/sun.nio.cs=ALL-UNNAMED", "--add-opens=java.base/sun.security.action=ALL-UNNAMED", From daa3281e6a68845943fcf61ba7ad1d2d3c8be28f Mon Sep 17 00:00:00 2001 From: xieshuaihu Date: Mon, 16 Oct 2023 17:01:18 +0900 Subject: [PATCH 062/521] [SPARK-45538][PYTHON][CONNECT] pyspark connect overwrite_partitions bug Fix a bug in pyspark connect. DataFrameWriterV2.overwritePartitions set mode as overwrite_partitions [pyspark/sql/connect/readwriter.py, line 825], but WirteOperationV2 take it as overwrite_partition [pyspark/sql/connect/plan.py, line 1660] make dataframe.writeTo(table).overwritePartitions() work No No test. This bug is very obvious. No Closes #43367 from xieshuaihu/python_connect_overwrite. Authored-by: xieshuaihu Signed-off-by: Hyukjin Kwon (cherry picked from commit 9bdad31039134b492caeeba430120d5978a085ee) Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/connect/plan.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index b7ea1f9499354..9af5823dd8b84 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -1655,7 +1655,7 @@ def command(self, session: "SparkConnectClient") -> proto.Command: plan.write_operation_v2.mode = proto.WriteOperationV2.Mode.MODE_CREATE elif wm == "overwrite": plan.write_operation_v2.mode = proto.WriteOperationV2.Mode.MODE_OVERWRITE - elif wm == "overwrite_partition": + elif wm == "overwrite_partitions": plan.write_operation_v2.mode = proto.WriteOperationV2.Mode.MODE_OVERWRITE_PARTITIONS elif wm == "append": plan.write_operation_v2.mode = proto.WriteOperationV2.Mode.MODE_APPEND From 0dc1962374dceea29a0fa7802881dfeff335d3c9 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 16 Oct 2023 17:10:26 +0800 Subject: [PATCH 063/521] [SPARK-44619][INFRA][3.5] Free up disk space for container jobs ### What changes were proposed in this pull request? Free up disk space for container jobs ### Why are the changes needed? increase the available disk space before this PR ![image](https://github.com/apache/spark/assets/7322292/64230324-607b-4c1d-ac2d-84b9bcaab12a) after this PR ![image](https://github.com/apache/spark/assets/7322292/aafed2d6-5d26-4f7f-b020-1efe4f551a8f) ### Does this PR introduce _any_ user-facing change? No, infra-only ### How was this patch tested? updated CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #43381 from LuciferYang/SPARK-44619-35. Authored-by: Ruifeng Zheng Signed-off-by: yangjie01 --- .github/workflows/build_and_test.yml | 6 +++++ dev/free_disk_space_container | 33 ++++++++++++++++++++++++++++ 2 files changed, 39 insertions(+) create mode 100755 dev/free_disk_space_container diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 1fcca7e4c3984..674e59508510c 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -407,6 +407,8 @@ jobs: key: pyspark-coursier-${{ hashFiles('**/pom.xml', '**/plugins.sbt') }} restore-keys: | pyspark-coursier- + - name: Free up disk space + run: ./dev/free_disk_space_container - name: Install Java ${{ matrix.java }} uses: actions/setup-java@v3 with: @@ -504,6 +506,8 @@ jobs: key: sparkr-coursier-${{ hashFiles('**/pom.xml', '**/plugins.sbt') }} restore-keys: | sparkr-coursier- + - name: Free up disk space + run: ./dev/free_disk_space_container - name: Install Java ${{ inputs.java }} uses: actions/setup-java@v3 with: @@ -612,6 +616,8 @@ jobs: key: docs-maven-${{ hashFiles('**/pom.xml') }} restore-keys: | docs-maven- + - name: Free up disk space + run: ./dev/free_disk_space_container - name: Install Java 8 uses: actions/setup-java@v3 with: diff --git a/dev/free_disk_space_container b/dev/free_disk_space_container new file mode 100755 index 0000000000000..cc3b74643e4fa --- /dev/null +++ b/dev/free_disk_space_container @@ -0,0 +1,33 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +echo "==================================" +echo "Free up disk space on CI system" +echo "==================================" + +echo "Listing 100 largest packages" +dpkg-query -Wf '${Installed-Size}\t${Package}\n' | sort -n | tail -n 100 +df -h + +echo "Removing large packages" +rm -rf /__t/CodeQL +rm -rf /__t/go +rm -rf /__t/node + +df -h From b2103731bcfe7e0bee3b1302c773e46f80badcc9 Mon Sep 17 00:00:00 2001 From: Jiaan Geng Date: Tue, 17 Oct 2023 09:50:39 +0800 Subject: [PATCH 064/521] [SPARK-45484][SQL][3.5] Deprecated the incorrect parquet compression codec lz4raw ### What changes were proposed in this pull request? According to the discussion at https://github.com/apache/spark/pull/43310#issuecomment-1757139681, this PR want deprecates the incorrect parquet compression codec `lz4raw` at Spark 3.5.1 and adds a warning log. The warning log prompts users that `lz4raw` will be removed it at Apache Spark 4.0.0. ### Why are the changes needed? Deprecated the incorrect parquet compression codec `lz4raw`. ### Does this PR introduce _any_ user-facing change? 'Yes'. Users will see the waring log below. `Parquet compression codec 'lz4raw' is deprecated, please use 'lz4_raw'` ### How was this patch tested? Exists test cases and new test cases. ### Was this patch authored or co-authored using generative AI tooling? 'No'. Closes #43330 from beliefer/SPARK-45484_3.5. Authored-by: Jiaan Geng Signed-off-by: Jiaan Geng --- .../apache/spark/sql/internal/SQLConf.scala | 14 +++++++++-- .../datasources/parquet/ParquetOptions.scala | 8 +++++- .../datasources/FileSourceCodecSuite.scala | 2 +- ...rquetCompressionCodecPrecedenceSuite.scala | 25 ++++++++++++++++--- 4 files changed, 41 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 73d3756ef6b93..427d04801902f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -995,12 +995,22 @@ object SQLConf { "`parquet.compression` is specified in the table-specific options/properties, the " + "precedence would be `compression`, `parquet.compression`, " + "`spark.sql.parquet.compression.codec`. Acceptable values include: none, uncompressed, " + - "snappy, gzip, lzo, brotli, lz4, lz4raw, zstd.") + "snappy, gzip, lzo, brotli, lz4, lz4raw, lz4_raw, zstd.") .version("1.1.1") .stringConf .transform(_.toLowerCase(Locale.ROOT)) .checkValues( - Set("none", "uncompressed", "snappy", "gzip", "lzo", "brotli", "lz4", "lz4raw", "zstd")) + Set( + "none", + "uncompressed", + "snappy", + "gzip", + "lzo", + "brotli", + "lz4", + "lz4raw", + "lz4_raw", + "zstd")) .createWithDefault("snappy") val PARQUET_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.parquet.filterPushdown") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala index 023d2460959cd..95869b6fbb9d5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala @@ -22,6 +22,7 @@ import java.util.Locale import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.parquet.hadoop.metadata.CompressionCodecName +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.{DataSourceOptions, FileSourceOptions} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.internal.SQLConf @@ -32,7 +33,7 @@ import org.apache.spark.sql.internal.SQLConf class ParquetOptions( @transient private val parameters: CaseInsensitiveMap[String], @transient private val sqlConf: SQLConf) - extends FileSourceOptions(parameters) { + extends FileSourceOptions(parameters) with Logging { import ParquetOptions._ @@ -59,6 +60,9 @@ class ParquetOptions( throw new IllegalArgumentException(s"Codec [$codecName] " + s"is not available. Available codecs are ${availableCodecs.mkString(", ")}.") } + if (codecName == "lz4raw") { + log.warn("Parquet compression codec 'lz4raw' is deprecated, please use 'lz4_raw'") + } shortParquetCompressionCodecNames(codecName).name() } @@ -96,7 +100,9 @@ object ParquetOptions extends DataSourceOptions { "lzo" -> CompressionCodecName.LZO, "brotli" -> CompressionCodecName.BROTLI, "lz4" -> CompressionCodecName.LZ4, + // Deprecated, to be removed at Spark 4.0.0, please use 'lz4_raw' instead. "lz4raw" -> CompressionCodecName.LZ4_RAW, + "lz4_raw" -> CompressionCodecName.LZ4_RAW, "zstd" -> CompressionCodecName.ZSTD) def getParquetCompressionCodecName(name: String): String = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCodecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCodecSuite.scala index 09a348cd29451..9f3d6ff48d477 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCodecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceCodecSuite.scala @@ -59,7 +59,7 @@ class ParquetCodecSuite extends FileSourceCodecSuite { // Exclude "brotli" because the com.github.rdblue:brotli-codec dependency is not available // on Maven Central. override protected def availableCodecs: Seq[String] = { - Seq("none", "uncompressed", "snappy", "gzip", "zstd", "lz4", "lz4raw") + Seq("none", "uncompressed", "snappy", "gzip", "zstd", "lz4", "lz4raw", "lz4_raw") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala index ac0aad16f1eba..27e2816ce9d94 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala @@ -29,9 +29,23 @@ import org.apache.spark.sql.test.SharedSparkSession class ParquetCompressionCodecPrecedenceSuite extends ParquetTest with SharedSparkSession { test("Test `spark.sql.parquet.compression.codec` config") { - Seq("NONE", "UNCOMPRESSED", "SNAPPY", "GZIP", "LZO", "LZ4", "BROTLI", "ZSTD").foreach { c => + Seq( + "NONE", + "UNCOMPRESSED", + "SNAPPY", + "GZIP", + "LZO", + "LZ4", + "BROTLI", + "ZSTD", + "LZ4RAW", + "LZ4_RAW").foreach { c => withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> c) { - val expected = if (c == "NONE") "UNCOMPRESSED" else c + val expected = c match { + case "NONE" => "UNCOMPRESSED" + case "LZ4RAW" => "LZ4_RAW" + case other => other + } val option = new ParquetOptions(Map.empty[String, String], spark.sessionState.conf) assert(option.compressionCodecClassName == expected) } @@ -97,7 +111,10 @@ class ParquetCompressionCodecPrecedenceSuite extends ParquetTest with SharedSpar createTableWithCompression(tempTableName, isPartitioned, compressionCodec, tmpDir) val partitionPath = if (isPartitioned) "p=2" else "" val path = s"${tmpDir.getPath.stripSuffix("/")}/$tempTableName/$partitionPath" - val realCompressionCodecs = getTableCompressionCodec(path) + val realCompressionCodecs = getTableCompressionCodec(path).map { + case "LZ4_RAW" if compressionCodec == "LZ4RAW" => "LZ4RAW" + case other => other + } assert(realCompressionCodecs.forall(_ == compressionCodec)) } } @@ -105,7 +122,7 @@ class ParquetCompressionCodecPrecedenceSuite extends ParquetTest with SharedSpar test("Create parquet table with compression") { Seq(true, false).foreach { isPartitioned => - val codecs = Seq("UNCOMPRESSED", "SNAPPY", "GZIP", "ZSTD", "LZ4") + val codecs = Seq("UNCOMPRESSED", "SNAPPY", "GZIP", "ZSTD", "LZ4", "LZ4RAW", "LZ4_RAW") codecs.foreach { compressionCodec => checkCompressionCodec(compressionCodec, isPartitioned) } From 22a83caa4896a8d03ec7e76b3e7a3bd08930adcb Mon Sep 17 00:00:00 2001 From: Bo Xiong Date: Tue, 17 Oct 2023 18:05:23 +0800 Subject: [PATCH 065/521] [SPARK-45283][CORE][TESTS][3.5] Make StatusTrackerSuite less fragile ### Why are the changes needed? It's discovered from [Github Actions](https://github.com/xiongbo-sjtu/spark/actions/runs/6270601155/job/17028788767) that StatusTrackerSuite can run into random failures, as shown by the following error message. The proposed fix is to update the unit test to remove the nondeterministic behavior. The fix has been made to the master branch in https://github.com/apache/spark/pull/43194. This PR is meant to patch branch-3.5 only. ``` [info] StatusTrackerSuite: [info] - basic status API usage (99 milliseconds) [info] - getJobIdsForGroup() (56 milliseconds) [info] - getJobIdsForGroup() with takeAsync() (48 milliseconds) [info] - getJobIdsForGroup() with takeAsync() across multiple partitions (58 milliseconds) [info] - getJobIdsForTag() *** FAILED *** (10 seconds, 77 milliseconds) [info] The code passed to eventually never returned normally. Attempted 651 times over 10.005059944000001 seconds. Last failure message: Set(3, 2, 1) was not equal to Set(1, 2). (StatusTrackerSuite.scala:148) ``` Full trace can be found [here](https://issues.apache.org/jira/browse/SPARK-45283). ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? ``` build/mvn package -DskipTests -pl core build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.StatusTrackerSuite test ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #43388 from xiongbo-sjtu/branch-3.5. Authored-by: Bo Xiong Signed-off-by: yangjie01 --- .../test/scala/org/apache/spark/StatusTrackerSuite.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala index 0817abbc6a328..9019ea484b3f3 100644 --- a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala @@ -140,16 +140,19 @@ class StatusTrackerSuite extends SparkFunSuite with Matchers with LocalSparkCont } sc.removeJobTag("tag1") + // takeAsync() across multiple partitions val thirdJobFuture = sc.parallelize(1 to 1000, 2).takeAsync(999) - val thirdJobId = eventually(timeout(10.seconds)) { - thirdJobFuture.jobIds.head + val thirdJobIds = eventually(timeout(10.seconds)) { + // Wait for the two jobs triggered by takeAsync + thirdJobFuture.jobIds.size should be(2) + thirdJobFuture.jobIds } eventually(timeout(10.seconds)) { sc.statusTracker.getJobIdsForTag("tag1").toSet should be ( Set(firstJobId, secondJobId)) sc.statusTracker.getJobIdsForTag("tag2").toSet should be ( - Set(secondJobId, thirdJobId)) + Set(secondJobId) ++ thirdJobIds) } } } From 6a5747d66e53ed0d934cdd9ca5c9bd9fde6868e6 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 17 Oct 2023 22:19:18 +0800 Subject: [PATCH 066/521] [SPARK-45568][TESTS] Fix flaky WholeStageCodegenSparkSubmitSuite ### What changes were proposed in this pull request? WholeStageCodegenSparkSubmitSuite is [flaky](https://github.com/apache/spark/actions/runs/6479534195/job/17593342589) because SHUFFLE_PARTITIONS(200) creates 200 reducers for one total core and improper stop progress causes executor launcher reties. The heavy load and reties might result in timeout test failures. ### Why are the changes needed? CI robustness ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? existing WholeStageCodegenSparkSubmitSuite ### Was this patch authored or co-authored using generative AI tooling? no Closes #43394 from yaooqinn/SPARK-45568. Authored-by: Kent Yao Signed-off-by: Kent Yao (cherry picked from commit f00ec39542a5f9ac75d8c24f0f04a7be703c8d7c) Signed-off-by: Kent Yao --- .../WholeStageCodegenSparkSubmitSuite.scala | 57 ++++++++++--------- 1 file changed, 30 insertions(+), 27 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala index e253de76221ad..69145d890fc19 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.deploy.SparkSubmitTestUtils import org.apache.spark.internal.Logging import org.apache.spark.sql.{QueryTest, Row, SparkSession} import org.apache.spark.sql.functions.{array, col, count, lit} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.IntegerType import org.apache.spark.tags.ExtendedSQLTest import org.apache.spark.unsafe.Platform @@ -70,39 +71,41 @@ class WholeStageCodegenSparkSubmitSuite extends SparkSubmitTestUtils object WholeStageCodegenSparkSubmitSuite extends Assertions with Logging { - var spark: SparkSession = _ - def main(args: Array[String]): Unit = { TestUtils.configTestLog4j2("INFO") - spark = SparkSession.builder().getOrCreate() + val spark = SparkSession.builder() + .config(SQLConf.SHUFFLE_PARTITIONS.key, "2") + .getOrCreate() + + try { + // Make sure the test is run where the driver and the executors uses different object layouts + val driverArrayHeaderSize = Platform.BYTE_ARRAY_OFFSET + val executorArrayHeaderSize = + spark.sparkContext.range(0, 1).map(_ => Platform.BYTE_ARRAY_OFFSET).collect().head + assert(driverArrayHeaderSize > executorArrayHeaderSize) - // Make sure the test is run where the driver and the executors uses different object layouts - val driverArrayHeaderSize = Platform.BYTE_ARRAY_OFFSET - val executorArrayHeaderSize = - spark.sparkContext.range(0, 1).map(_ => Platform.BYTE_ARRAY_OFFSET).collect.head.toInt - assert(driverArrayHeaderSize > executorArrayHeaderSize) + val df = spark.range(71773).select((col("id") % lit(10)).cast(IntegerType) as "v") + .groupBy(array(col("v"))).agg(count(col("*"))) + val plan = df.queryExecution.executedPlan + assert(plan.exists(_.isInstanceOf[WholeStageCodegenExec])) - val df = spark.range(71773).select((col("id") % lit(10)).cast(IntegerType) as "v") - .groupBy(array(col("v"))).agg(count(col("*"))) - val plan = df.queryExecution.executedPlan - assert(plan.exists(_.isInstanceOf[WholeStageCodegenExec])) + val expectedAnswer = + Row(Array(0), 7178) :: + Row(Array(1), 7178) :: + Row(Array(2), 7178) :: + Row(Array(3), 7177) :: + Row(Array(4), 7177) :: + Row(Array(5), 7177) :: + Row(Array(6), 7177) :: + Row(Array(7), 7177) :: + Row(Array(8), 7177) :: + Row(Array(9), 7177) :: Nil - val expectedAnswer = - Row(Array(0), 7178) :: - Row(Array(1), 7178) :: - Row(Array(2), 7178) :: - Row(Array(3), 7177) :: - Row(Array(4), 7177) :: - Row(Array(5), 7177) :: - Row(Array(6), 7177) :: - Row(Array(7), 7177) :: - Row(Array(8), 7177) :: - Row(Array(9), 7177) :: Nil - val result = df.collect - QueryTest.sameRows(result.toSeq, expectedAnswer) match { - case Some(errMsg) => fail(errMsg) - case _ => + QueryTest.checkAnswer(df, expectedAnswer) + } finally { + spark.stop() } + } } From 18599ea750f50e07a910487fb3a871ed69fb9cab Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 17 Oct 2023 16:10:56 -0700 Subject: [PATCH 067/521] [MINOR][SQL] Remove signature from Hive thriftserver exception ### What changes were proposed in this pull request? Don't return expected signature to caller in Hive thriftserver exception ### Why are the changes needed? Please see private discussion ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #43402 from srowen/HiveCookieSigner. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun (cherry picked from commit cf59b1f51c16301f689b4e0f17ba4dbd140e1b19) Signed-off-by: Dongjoon Hyun --- .../src/main/java/org/apache/hive/service/CookieSigner.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java index 782e47a6cd902..4b8d2cb1536cd 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java @@ -81,8 +81,7 @@ public String verifyAndExtract(String signedStr) { LOG.debug("Signature generated for " + rawValue + " inside verify is " + currentSignature); } if (!MessageDigest.isEqual(originalSignature.getBytes(), currentSignature.getBytes())) { - throw new IllegalArgumentException("Invalid sign, original = " + originalSignature + - " current = " + currentSignature); + throw new IllegalArgumentException("Invalid sign"); } return rawValue; } From ed2a4cc6033ac35faa7b19eb236a4c953543d519 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 18 Oct 2023 11:43:59 +0900 Subject: [PATCH 068/521] [MINOR][DOCS] Update the docs for spark.sql.optimizer.canChangeCachedPlanOutputPartitioning configuration ### What changes were proposed in this pull request? This PR fixes the documentation for `spark.sql.optimizer.canChangeCachedPlanOutputPartitioning` configuration by saying this is enabled by default. This is a followup of https://github.com/apache/spark/pull/40390 (but did not use a JIRA due to fixed versions properties in the JIRA). ### Why are the changes needed? To mention that this is enabled, to the end users. ### Does this PR introduce _any_ user-facing change? No, it's an internal conf, not documented. ### How was this patch tested? CI in this PR. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43411 from HyukjinKwon/fix-docs. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit 0cb4a84f6ab0c1bd101e6bc72be82987bbc02e9b) Signed-off-by: Hyukjin Kwon --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 427d04801902f..4ea0cd5bcc126 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1529,7 +1529,7 @@ object SQLConf { .doc("Whether to forcibly enable some optimization rules that can change the output " + "partitioning of a cached query when executing it for caching. If it is set to true, " + "queries may need an extra shuffle to read the cached data. This configuration is " + - "disabled by default. Currently, the optimization rules enabled by this configuration " + + "enabled by default. The optimization rules enabled by this configuration " + s"are ${ADAPTIVE_EXECUTION_ENABLED.key} and ${AUTO_BUCKETED_SCAN_ENABLED.key}.") .version("3.2.0") .booleanConf From 2f66851972c2fc66b053a8c78bc2814b7bb4257f Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 19 Oct 2023 09:43:13 +0800 Subject: [PATCH 069/521] [SPARK-45585][TEST] Fix time format and redirection issues in SparkSubmit tests ### What changes were proposed in this pull request? This PR fixes: - The deviation from `new Timestamp(new Date().getTime)` and log4j2 date format pattern from sub spark-submit progress ``` 2023-10-17 03:58:48.275 - stderr> 23/10/17 18:58:48 INFO StandaloneSchedulerBackend: Connected to Spark cluster with app ID app-20231017185848-0000 2023-10-17 03:58:48.278 - stderr> 23/10/17 18:58:48 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 57637. ``` - The duplication of `new Timestamp(new Date().getTime)` when using logInfo instead of println ``` 23/10/17 19:02:34.392 Thread-5 INFO SparkShellSuite: 2023-10-17 04:02:34.392 - stderr> 23/10/17 19:02:34 WARN Utils: Your hostname, hulk.local resolves to a loopback address: 127.0.0.1; using 10.221.103.23 instead (on interface en0) 23/10/17 19:02:34.393 Thread-5 INFO SparkShellSuite: 2023-10-17 04:02:34.393 - stderr> 23/10/17 19:02:34 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address ``` - Correctly redirects sub spark-submit progress logs to unit-tests.log ### Why are the changes needed? test fixes ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? - WholeStageCodegenSparkSubmitSuite - before ``` 18:58:53.882 shutdown-hook-0 INFO ShutdownHookManager: Shutdown hook called 18:58:53.882 shutdown-hook-0 INFO ShutdownHookManager: Deleting directory /Users/hzyaoqin/spark/target/tmp/spark-ecd53d47-d109-4ddc-80dd-2d829f34371e 11:58:18.892 pool-1-thread-1 WARN Utils: Your hostname, hulk.local resolves to a loopback address: 127.0.0.1; using 10.221.103.23 instead (on interface en0) 11:58:18.893 pool-1-thread-1 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address 11:58:18.932 pool-1-thread-1-ScalaTest-running-WholeStageCodegenSparkSubmitSuite INFO WholeStageCodegenSparkSubmitSuite: ``` - WholeStageCodegenSparkSubmitSuite - after ``` ===== TEST OUTPUT FOR o.a.s.sql.execution.WholeStageCodegenSparkSubmitSuite: 'Generated code on driver should not embed platform-specific constant' ===== 11:58:19.882 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:19 WARN Utils: Your hostname, hulk.local resolves to a loopback address: 127.0.0.1; using 10.221.103.23 instead (on interface en0) 11:58:19.883 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:19 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address 11:58:20.195 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 INFO SparkContext: Running Spark version 4.0.0-SNAPSHOT 11:58:20.195 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 INFO SparkContext: OS info Mac OS X, 13.4, aarch64 11:58:20.195 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 INFO SparkContext: Java version 17.0.8 11:58:20.227 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable 11:58:20.253 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 INFO ResourceUtils: ============================================================== 11:58:20.253 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 INFO ResourceUtils: No custom resources configured for spark.driver. 11:58:20.253 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 INFO ResourceUtils: ============================================================== 11:58:20.254 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 INFO SparkContext: Submitted application: org.apache.spark.sql.execution.WholeStageCodegenSparkSubmitSuite 11:58:20.266 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 0, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) 11:58:20.268 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 INFO ResourceProfile: Limiting resource is cpu 11:58:20.268 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 INFO ResourceProfileManager: Added ResourceProfile id: 0 11:58:20.302 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 INFO SecurityManager: Changing view acls to: hzyaoqin 11:58:20.302 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 INFO SecurityManager: Changing modify acls to: hzyaoqin 11:58:20.303 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 INFO SecurityManager: Changing view acls groups to: 11:58:20.303 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 INFO SecurityManager: Changing modify acls groups to: 11:58:20.305 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: hzyaoqin; groups with view permissions: EMPTY; users with modify permissions: hzyaoqin; groups with modify permissions: EMPTY; RPC SSL disabled 11:58:20.448 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 INFO Utils: Successfully started service 'sparkDriver' on port 52173. 11:58:20.465 Thread-6 INFO WholeStageCodegenSparkSubmitSuite: stderr> 23/10/18 11:58:20 INFO SparkEnv: Registering MapOutputTracker ``` ### Was this patch authored or co-authored using generative AI tooling? no Closes #43421 from yaooqinn/SPARK-45585. Authored-by: Kent Yao Signed-off-by: Kent Yao (cherry picked from commit a14f90941caf06e2d77789a3952dd588e6900b90) Signed-off-by: Kent Yao --- .../spark/deploy/SparkSubmitTestUtils.scala | 15 ++------------- .../org/apache/spark/repl/SparkShellSuite.scala | 11 ++++------- .../spark/sql/hive/thriftserver/CliSuite.scala | 9 ++------- 3 files changed, 8 insertions(+), 27 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitTestUtils.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitTestUtils.scala index 2ab2e17df03a8..932e972374cae 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitTestUtils.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitTestUtils.scala @@ -18,8 +18,6 @@ package org.apache.spark.deploy import java.io.File -import java.sql.Timestamp -import java.util.Date import scala.collection.mutable.ArrayBuffer @@ -69,17 +67,8 @@ trait SparkSubmitTestUtils extends SparkFunSuite with TimeLimits { env.put("SPARK_HOME", sparkHome) def captureOutput(source: String)(line: String): Unit = { - // This test suite has some weird behaviors when executed on Jenkins: - // - // 1. Sometimes it gets extremely slow out of unknown reason on Jenkins. Here we add a - // timestamp to provide more diagnosis information. - // 2. Log lines are not correctly redirected to unit-tests.log as expected, so here we print - // them out for debugging purposes. - val logLine = s"${new Timestamp(new Date().getTime)} - $source> $line" - // scalastyle:off println - println(logLine) - // scalastyle:on println - history += logLine + logInfo(s"$source> $line") + history += line } val process = builder.start() diff --git a/repl/src/test/scala/org/apache/spark/repl/SparkShellSuite.scala b/repl/src/test/scala/org/apache/spark/repl/SparkShellSuite.scala index 39544beec4154..067f08cb67528 100644 --- a/repl/src/test/scala/org/apache/spark/repl/SparkShellSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/SparkShellSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.repl import java.io._ import java.nio.charset.StandardCharsets -import java.sql.Timestamp -import java.util.Date import scala.collection.mutable.ArrayBuffer import scala.concurrent.Promise @@ -70,10 +68,9 @@ class SparkShellSuite extends SparkFunSuite { val lock = new Object def captureOutput(source: String)(line: String): Unit = lock.synchronized { - // This test suite sometimes gets extremely slow out of unknown reason on Jenkins. Here we - // add a timestamp to provide more diagnosis information. - val newLine = s"${new Timestamp(new Date().getTime)} - $source> $line" - log.info(newLine) + val newLine = s"$source> $line" + + logInfo(newLine) buffer += newLine if (line.startsWith("Spark context available") && line.contains("app id")) { @@ -82,7 +79,7 @@ class SparkShellSuite extends SparkFunSuite { // If we haven't found all expected answers and another expected answer comes up... if (next < expectedAnswers.size && line.contains(expectedAnswers(next))) { - log.info(s"$source> found expected output line $next: '${expectedAnswers(next)}'") + logInfo(s"$source> found expected output line $next: '${expectedAnswers(next)}'") next += 1 // If all expected answers have been found... if (next == expectedAnswers.size) { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index d3a9a9f08411c..8ba9ea28a5a96 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.hive.thriftserver import java.io._ import java.nio.charset.StandardCharsets -import java.sql.Timestamp -import java.util.Date import java.util.concurrent.CountDownLatch import scala.collection.JavaConverters._ @@ -145,11 +143,8 @@ class CliSuite extends SparkFunSuite { val lock = new Object def captureOutput(source: String)(line: String): Unit = lock.synchronized { - // This test suite sometimes gets extremely slow out of unknown reason on Jenkins. Here we - // add a timestamp to provide more diagnosis information. - val newLine = s"${new Timestamp(new Date().getTime)} - $source> $line" - log.info(newLine) - buffer += newLine + logInfo(s"$source> $line") + buffer += line if (line.startsWith("Spark master: ") && line.contains("Application Id: ")) { foundMasterAndApplicationIdMessage.trySuccess(()) From 17d283990b64614828838afa718f48b855ab7842 Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Thu, 19 Oct 2023 13:57:01 +0900 Subject: [PATCH 070/521] [SPARK-45553][PS] Deprecate `assertPandasOnSparkEqual` ### What changes were proposed in this pull request? This PR proposes to deprecate `assertPandasOnSparkEqual`. ### Why are the changes needed? Now we have more pandas friendly testing utils such as `ps.testing.assert_frame_equal`, `ps.testing.assert_series_equal` and `ps.testing.assert_index_equal`. ### Does this PR introduce _any_ user-facing change? Not for now, but `assertPandasOnSparkEqual` will be removed in the future version. ### How was this patch tested? The existing CI should pass. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43426 from itholic/SPARK-45553. Authored-by: Haejoon Lee Signed-off-by: Hyukjin Kwon (cherry picked from commit f3e280b952da8b8ab6c78371f3715cc674a73bc1) Signed-off-by: Hyukjin Kwon --- python/pyspark/testing/pandasutils.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/python/pyspark/testing/pandasutils.py b/python/pyspark/testing/pandasutils.py index c80ffb7ee53cb..04a523bce7640 100644 --- a/python/pyspark/testing/pandasutils.py +++ b/python/pyspark/testing/pandasutils.py @@ -365,6 +365,11 @@ def assertPandasOnSparkEqual( .. versionadded:: 3.5.0 + .. deprecated:: 3.5.1 + `assertPandasOnSparkEqual` will be removed in Spark 4.0.0. + Use `ps.testing.assert_frame_equal`, `ps.testing.assert_series_equal` + and `ps.testing.assert_index_equal` instead. + Parameters ---------- actual: pandas-on-Spark DataFrame, Series, or Index @@ -417,6 +422,12 @@ def assertPandasOnSparkEqual( >>> s2 = ps.Index([212.3, 100.0001]) >>> assertPandasOnSparkEqual(s1, s2, almost=True) # pass, ps.Index obj are almost equal """ + warnings.warn( + "`assertPandasOnSparkEqual` will be removed in Spark 4.0.0. " + "Use `ps.testing.assert_frame_equal`, `ps.testing.assert_series_equal` " + "and `ps.testing.assert_index_equal` instead.", + FutureWarning, + ) if actual is None and expected is None: return True elif actual is None or expected is None: From feb48dc146d8a89882875f25115af52e8295dfcc Mon Sep 17 00:00:00 2001 From: Jiaan Geng Date: Thu, 19 Oct 2023 20:16:21 +0800 Subject: [PATCH 071/521] [SPARK-45543][SQL] `InferWindowGroupLimit` causes bug if the other window functions haven't the same window frame as the rank-like functions ### What changes were proposed in this pull request? https://github.com/apache/spark/pull/38799 Introduce the group limit of Window for rank-based filter to optimize top-k computation. But it causes a bug if window expressions exists non-rank function which has the window frame is not the same as `(UnboundedPreceding, CurrentRow)`. Please see the detail at https://issues.apache.org/jira/browse/SPARK-45543. ### Why are the changes needed? Fix the bug. ### Does this PR introduce _any_ user-facing change? 'Yes'. ### How was this patch tested? New test cases. ### Was this patch authored or co-authored using generative AI tooling? 'No'. Closes #43385 from beliefer/SPARK-45543. Authored-by: Jiaan Geng Signed-off-by: Jiaan Geng (cherry picked from commit d6d4e52ecc3015b41c51bc7e4e122696c76b06ee) Signed-off-by: Jiaan Geng --- .../optimizer/InferWindowGroupLimit.scala | 18 ++- .../sql/DataFrameWindowFunctionsSuite.scala | 112 ++++++++++++++++++ 2 files changed, 126 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InferWindowGroupLimit.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InferWindowGroupLimit.scala index 261be2914630e..04204c6a2e108 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InferWindowGroupLimit.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InferWindowGroupLimit.scala @@ -52,23 +52,33 @@ object InferWindowGroupLimit extends Rule[LogicalPlan] with PredicateHelper { if (limits.nonEmpty) Some(limits.min) else None } - private def support( + /** + * All window expressions should use the same expanding window, so that + * we can safely do the early stop. + */ + private def isExpandingWindow( windowExpression: NamedExpression): Boolean = windowExpression match { - case Alias(WindowExpression(_: Rank | _: DenseRank | _: RowNumber, WindowSpecDefinition(_, _, + case Alias(WindowExpression(_, WindowSpecDefinition(_, _, SpecifiedWindowFrame(RowFrame, UnboundedPreceding, CurrentRow))), _) => true case _ => false } + private def support(windowFunction: Expression): Boolean = windowFunction match { + case _: Rank | _: DenseRank | _: RowNumber => true + case _ => false + } + def apply(plan: LogicalPlan): LogicalPlan = { if (conf.windowGroupLimitThreshold == -1) return plan plan.transformWithPruning(_.containsAllPatterns(FILTER, WINDOW), ruleId) { case filter @ Filter(condition, window @ Window(windowExpressions, partitionSpec, orderSpec, child)) - if !child.isInstanceOf[WindowGroupLimit] && windowExpressions.exists(support) && + if !child.isInstanceOf[WindowGroupLimit] && windowExpressions.forall(isExpandingWindow) && orderSpec.nonEmpty => val limits = windowExpressions.collect { - case alias @ Alias(WindowExpression(rankLikeFunction, _), _) if support(alias) => + case alias @ Alias(WindowExpression(rankLikeFunction, _), _) + if support(rankLikeFunction) => extractLimits(condition, alias.toAttribute).map((_, rankLikeFunction)) }.flatten diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index a57e927ba8427..47380db421734 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -1521,4 +1521,116 @@ class DataFrameWindowFunctionsSuite extends QueryTest assert(windows.size === 1) } } + + test("SPARK-45543: InferWindowGroupLimit causes bug " + + "if the other window functions haven't the same window frame as the rank-like functions") { + val df = Seq( + (1, "Dave", 1, 2020), + (2, "Dave", 1, 2021), + (3, "Dave", 2, 2022), + (4, "Dave", 3, 2023), + (5, "Dave", 3, 2024), + (6, "Mark", 2, 2022), + (7, "Mark", 3, 2023), + (8, "Mark", 3, 2024), + (9, "Amy", 6, 2021), + (10, "Amy", 5, 2022), + (11, "Amy", 6, 2023), + (12, "Amy", 7, 2024), + (13, "John", 7, 2024)).toDF("id", "name", "score", "year") + + val window = Window.partitionBy($"year").orderBy($"score".desc) + val window2 = window.rowsBetween(Window.unboundedPreceding, Window.currentRow) + val window3 = window.rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing) + + Seq(-1, 100).foreach { threshold => + withSQLConf(SQLConf.WINDOW_GROUP_LIMIT_THRESHOLD.key -> threshold.toString) { + // The other window functions have the same window frame as the rank-like functions. + // df2, df3 and df4 can apply InferWindowGroupLimit + val df2 = df + .withColumn("rn", row_number().over(window)) + .withColumn("all_scores", collect_list($"score").over(window2)) + .sort($"year") + + checkAnswer(df2.filter("rn=1"), Seq( + Row(1, "Dave", 1, 2020, 1, Array(1)), + Row(9, "Amy", 6, 2021, 1, Array(6)), + Row(10, "Amy", 5, 2022, 1, Array(5)), + Row(11, "Amy", 6, 2023, 1, Array(6)), + Row(12, "Amy", 7, 2024, 1, Array(7)) + )) + + val df3 = df + .withColumn("rank", rank().over(window)) + .withColumn("all_scores", collect_list($"score").over(window2)) + .sort($"year") + + checkAnswer(df3.filter("rank=2"), Seq( + Row(2, "Dave", 1, 2021, 2, Array(6, 1)), + Row(3, "Dave", 2, 2022, 2, Array(5, 2)), + Row(6, "Mark", 2, 2022, 2, Array(5, 2, 2)), + Row(4, "Dave", 3, 2023, 2, Array(6, 3)), + Row(7, "Mark", 3, 2023, 2, Array(6, 3, 3)) + )) + + val df4 = df + .withColumn("rank", dense_rank().over(window)) + .withColumn("all_scores", collect_list($"score").over(window2)) + .sort($"year") + + checkAnswer(df4.filter("rank=2"), Seq( + Row(2, "Dave", 1, 2021, 2, Array(6, 1)), + Row(3, "Dave", 2, 2022, 2, Array(5, 2)), + Row(6, "Mark", 2, 2022, 2, Array(5, 2, 2)), + Row(4, "Dave", 3, 2023, 2, Array(6, 3)), + Row(7, "Mark", 3, 2023, 2, Array(6, 3, 3)), + Row(5, "Dave", 3, 2024, 2, Array(7, 7, 3)), + Row(8, "Mark", 3, 2024, 2, Array(7, 7, 3, 3)) + )) + + // The other window functions haven't the same window frame as the rank-like functions. + // df5, df6 and df7 cannot apply InferWindowGroupLimit + val df5 = df + .withColumn("rn", row_number().over(window)) + .withColumn("all_scores", collect_list($"score").over(window3)) + .sort($"year") + + checkAnswer(df5.filter("rn=1"), Seq( + Row(1, "Dave", 1, 2020, 1, Array(1)), + Row(9, "Amy", 6, 2021, 1, Array(6, 1)), + Row(10, "Amy", 5, 2022, 1, Array(5, 2, 2)), + Row(11, "Amy", 6, 2023, 1, Array(6, 3, 3)), + Row(12, "Amy", 7, 2024, 1, Array(7, 7, 3, 3)) + )) + + val df6 = df + .withColumn("rank", rank().over(window)) + .withColumn("all_scores", collect_list($"score").over(window3)) + .sort($"year") + + checkAnswer(df6.filter("rank=2"), Seq( + Row(2, "Dave", 1, 2021, 2, Array(6, 1)), + Row(3, "Dave", 2, 2022, 2, Array(5, 2, 2)), + Row(6, "Mark", 2, 2022, 2, Array(5, 2, 2)), + Row(4, "Dave", 3, 2023, 2, Array(6, 3, 3)), + Row(7, "Mark", 3, 2023, 2, Array(6, 3, 3)) + )) + + val df7 = df + .withColumn("rank", dense_rank().over(window)) + .withColumn("all_scores", collect_list($"score").over(window3)) + .sort($"year") + + checkAnswer(df7.filter("rank=2"), Seq( + Row(2, "Dave", 1, 2021, 2, Array(6, 1)), + Row(3, "Dave", 2, 2022, 2, Array(5, 2, 2)), + Row(6, "Mark", 2, 2022, 2, Array(5, 2, 2)), + Row(4, "Dave", 3, 2023, 2, Array(6, 3, 3)), + Row(7, "Mark", 3, 2023, 2, Array(6, 3, 3)), + Row(5, "Dave", 3, 2024, 2, Array(7, 7, 3, 3)), + Row(8, "Mark", 3, 2024, 2, Array(7, 7, 3, 3)) + )) + } + } + } } From f47b63c6a62fb6f1fd894f64736847719af7a199 Mon Sep 17 00:00:00 2001 From: allisonwang-db Date: Fri, 20 Oct 2023 08:36:42 +0800 Subject: [PATCH 072/521] [SPARK-45584][SQL] Fix subquery execution failure with TakeOrderedAndProjectExec This PR fixes a bug when there are subqueries in `TakeOrderedAndProjectExec`. The executeCollect method does not wait for subqueries to finish and it can result in IllegalArgumentException when executing a simple query. For example this query: ``` WITH t2 AS ( SELECT * FROM t1 ORDER BY id ) SELECT *, (SELECT COUNT(*) FROM t2) FROM t2 LIMIT 10 ``` will fail with this error ``` java.lang.IllegalArgumentException: requirement failed: Subquery subquery#242, [id=#109] has not finished ``` To fix a bug. No New unit test No Closes #43419 from allisonwang-db/spark-45584-subquery-failure. Authored-by: allisonwang-db Signed-off-by: Wenchen Fan (cherry picked from commit 8fd915ffaba1cc99813cc8d6d2a28688d7fae39b) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/execution/limit.scala | 2 +- .../org/apache/spark/sql/SubquerySuite.scala | 24 +++++++++++++++++++ 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index 877f6508d963f..77135d21a26ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -282,7 +282,7 @@ case class TakeOrderedAndProjectExec( projectList.map(_.toAttribute) } - override def executeCollect(): Array[InternalRow] = { + override def executeCollect(): Array[InternalRow] = executeQuery { val orderingSatisfies = SortOrder.orderingSatisfies(child.outputOrdering, sortOrder) val ord = new LazilyGeneratedOrdering(sortOrder, child.output) val limited = if (orderingSatisfies) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index d235d2a15fea3..a7a0f6156cb1d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -2712,4 +2712,28 @@ class SubquerySuite extends QueryTest expected) } } + + test("SPARK-45584: subquery execution should not fail with ORDER BY and LIMIT") { + withTable("t1") { + sql( + """ + |CREATE TABLE t1 USING PARQUET + |AS SELECT * FROM VALUES + |(1, "a"), + |(2, "a"), + |(3, "a") t(id, value) + |""".stripMargin) + val df = sql( + """ + |WITH t2 AS ( + | SELECT * FROM t1 ORDER BY id + |) + |SELECT *, (SELECT COUNT(*) FROM t2) FROM t2 LIMIT 10 + |""".stripMargin) + // This should not fail with IllegalArgumentException. + checkAnswer( + df, + Row(1, "a", 3) :: Row(2, "a", 3) :: Row(3, "a", 3) :: Nil) + } + } } From 6a3a8f5750e1db3d4dea3a9d12795764035b953a Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Fri, 20 Oct 2023 15:01:15 +0900 Subject: [PATCH 073/521] [SPARK-45553][3.5][PS] Correct warning messages ### What changes were proposed in this pull request? This followups for https://github.com/apache/spark/pull/43426. ### Why are the changes needed? To remove incorrect context from the warning message. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The existing CI should pass ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43459 from itholic/45553-remove. Authored-by: Haejoon Lee Signed-off-by: Hyukjin Kwon --- python/pyspark/testing/pandasutils.py | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/python/pyspark/testing/pandasutils.py b/python/pyspark/testing/pandasutils.py index 04a523bce7640..2463289d59f71 100644 --- a/python/pyspark/testing/pandasutils.py +++ b/python/pyspark/testing/pandasutils.py @@ -367,8 +367,6 @@ def assertPandasOnSparkEqual( .. deprecated:: 3.5.1 `assertPandasOnSparkEqual` will be removed in Spark 4.0.0. - Use `ps.testing.assert_frame_equal`, `ps.testing.assert_series_equal` - and `ps.testing.assert_index_equal` instead. Parameters ---------- @@ -423,9 +421,7 @@ def assertPandasOnSparkEqual( >>> assertPandasOnSparkEqual(s1, s2, almost=True) # pass, ps.Index obj are almost equal """ warnings.warn( - "`assertPandasOnSparkEqual` will be removed in Spark 4.0.0. " - "Use `ps.testing.assert_frame_equal`, `ps.testing.assert_series_equal` " - "and `ps.testing.assert_index_equal` instead.", + "`assertPandasOnSparkEqual` will be removed in Spark 4.0.0. ", FutureWarning, ) if actual is None and expected is None: From 6c55a6c0c680f80a6cdef7f1a83045b6400b4d09 Mon Sep 17 00:00:00 2001 From: Zamil Majdy Date: Sun, 22 Oct 2023 10:53:22 +0500 Subject: [PATCH 074/521] [SPARK-45604][SQL] Add LogicalType checking on INT64 -> DateTime conversion on Parquet Vectorized Reader ### What changes were proposed in this pull request? Currently, the read logical type is not checked while converting physical types INT64 into DateTime. One valid scenario where this can break is where the physical type is `timestamp_ntz`, and the logical type is `array`, since the logical type check does not happen, this conversion is allowed. However, the vectorized reader does not support this and will produce NPE on on-heap memory mode and SEGFAULT on off-heap memory mode. Segmentation fault on off-heap memory mode can be prevented by having an explicit boundary check on OffHeapColumnVector, but this is outside of the scope of this PR, and will be done here: https://github.com/apache/spark/pull/43452. ### Why are the changes needed? Prevent NPE or Segfault from happening. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? A new test is added in `ParquetSchemaSuite`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43451 from majdyz/SPARK-45604. Lead-authored-by: Zamil Majdy Co-authored-by: Zamil Majdy Signed-off-by: Max Gekk (cherry picked from commit 13b67ee8cc377a5cc47d02b9addbc00eabfc8b6c) Signed-off-by: Max Gekk --- .../parquet/ParquetVectorUpdaterFactory.java | 10 +++++++-- .../parquet/ParquetSchemaSuite.scala | 21 +++++++++++++++++++ 2 files changed, 29 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java index 15d58f0c7572a..42442cf8ea8a4 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java @@ -109,7 +109,8 @@ public ParquetVectorUpdater getUpdater(ColumnDescriptor descriptor, DataType spa // For unsigned int64, it stores as plain signed int64 in Parquet when dictionary // fallbacks. We read them as decimal values. return new UnsignedLongUpdater(); - } else if (isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MICROS)) { + } else if (isTimestamp(sparkType) && + isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MICROS)) { validateTimestampType(sparkType); if ("CORRECTED".equals(datetimeRebaseMode)) { return new LongUpdater(); @@ -117,7 +118,8 @@ public ParquetVectorUpdater getUpdater(ColumnDescriptor descriptor, DataType spa boolean failIfRebase = "EXCEPTION".equals(datetimeRebaseMode); return new LongWithRebaseUpdater(failIfRebase, datetimeRebaseTz); } - } else if (isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MILLIS)) { + } else if (isTimestamp(sparkType) && + isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MILLIS)) { validateTimestampType(sparkType); if ("CORRECTED".equals(datetimeRebaseMode)) { return new LongAsMicrosUpdater(); @@ -1150,6 +1152,10 @@ private static boolean isLongDecimal(DataType dt) { return false; } + private static boolean isTimestamp(DataType dt) { + return dt == DataTypes.TimestampType || dt == DataTypes.TimestampNTZType; + } + private static boolean isDecimalTypeMatched(ColumnDescriptor descriptor, DataType dt) { DecimalType d = (DecimalType) dt; LogicalTypeAnnotation typeAnnotation = descriptor.getPrimitiveType().getLogicalTypeAnnotation(); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index facc9b90ff778..3f47c5e506ffd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -1087,6 +1087,27 @@ class ParquetSchemaSuite extends ParquetSchemaTest { } } + test("SPARK-45604: schema mismatch failure error on timestamp_ntz to array") { + import testImplicits._ + + withTempPath { dir => + val path = dir.getCanonicalPath + val timestamp = java.time.LocalDateTime.of(1, 2, 3, 4, 5) + val df1 = Seq((1, timestamp)).toDF() + val df2 = Seq((2, Array(timestamp))).toDF() + df1.write.mode("overwrite").parquet(s"$path/parquet") + df2.write.mode("append").parquet(s"$path/parquet") + + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { + val e = intercept[SparkException] { + spark.read.schema(df2.schema).parquet(s"$path/parquet").collect() + } + assert(e.getCause.isInstanceOf[SparkException]) + assert(e.getCause.getCause.isInstanceOf[SchemaColumnConvertNotSupportedException]) + } + } + } + test("SPARK-40819: parquet file with TIMESTAMP(NANOS, true) (with nanosAsLong=true)") { val tsAttribute = "birthday" withSQLConf(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key -> "true") { From 75a38b9024af3c9cfd85e916c46359f7e7315c87 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 23 Oct 2023 10:47:42 +0800 Subject: [PATCH 075/521] [SPARK-45616][CORE] Avoid ParVector, which does not propagate ThreadLocals or SparkSession MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? `CastSuiteBase` and `ExpressionInfoSuite` use `ParVector.foreach()` to run Spark SQL queries in parallel. They incorrectly assume that each parallel operation will inherit the main thread’s active SparkSession. This is only true when these parallel operations run in freshly-created threads. However, when other code has already run some parallel operations before Spark was started, then there may be existing threads that do not have an active SparkSession. In that case, these tests fail with NullPointerExceptions when creating SparkPlans or running SQL queries. The fix is to use the existing method `ThreadUtils.parmap()`. This method creates fresh threads that inherit the current active SparkSession, and it propagates the Spark ThreadLocals. This PR also adds a scalastyle warning against use of ParVector. ### Why are the changes needed? This change makes `CastSuiteBase` and `ExpressionInfoSuite` less brittle to future changes that may run parallel operations during test startup. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Reproduced the test failures by running a ParVector operation before Spark starts. Verified that this PR fixes the test failures in this condition. ```scala protected override def beforeAll(): Unit = { // Run a ParVector operation before initializing the SparkSession. This starts some Scala // execution context threads that have no active SparkSession. These threads will be reused for // later ParVector operations, reproducing SPARK-45616. new ParVector((0 until 100).toVector).foreach { _ => } super.beforeAll() } ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43466 from ankurdave/SPARK-45616. Authored-by: Ankur Dave Signed-off-by: Wenchen Fan (cherry picked from commit 376de8a502fca6b46d7f21560a60024d643144ea) Signed-off-by: Wenchen Fan --- .../main/scala/org/apache/spark/rdd/UnionRDD.scala | 2 ++ .../scala/org/apache/spark/util/ThreadUtils.scala | 4 ++++ scalastyle-config.xml | 12 ++++++++++++ .../sql/catalyst/expressions/CastSuiteBase.scala | 9 ++++++--- .../org/apache/spark/sql/execution/command/ddl.scala | 2 ++ .../spark/sql/expressions/ExpressionInfoSuite.scala | 11 ++++++----- .../org/apache/spark/streaming/DStreamGraph.scala | 4 ++++ .../streaming/util/FileBasedWriteAheadLog.scala | 2 ++ 8 files changed, 38 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 0a93023443704..3c1451a01850d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -76,8 +76,10 @@ class UnionRDD[T: ClassTag]( override def getPartitions: Array[Partition] = { val parRDDs = if (isPartitionListingParallel) { + // scalastyle:off parvector val parArray = new ParVector(rdds.toVector) parArray.tasksupport = UnionRDD.partitionEvalTaskSupport + // scalastyle:on parvector parArray } else { rdds diff --git a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala index 16d7de56c39eb..2d3d6ec89ffbd 100644 --- a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala @@ -363,6 +363,10 @@ private[spark] object ThreadUtils { * Comparing to the map() method of Scala parallel collections, this method can be interrupted * at any time. This is useful on canceling of task execution, for example. * + * Functions are guaranteed to be executed in freshly-created threads that inherit the calling + * thread's Spark thread-local variables. These threads also inherit the calling thread's active + * SparkSession. + * * @param in - the input collection which should be transformed in parallel. * @param prefix - the prefix assigned to the underlying thread pool. * @param maxThreads - maximum number of thread can be created during execution. diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 74e8480deaff7..0ccd937e72e88 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -227,6 +227,18 @@ This file is divided into 3 sections: ]]> + + new.*ParVector + + + (\.toUpperCase|\.toLowerCase)(?!(\(|\(Locale.ROOT\))) + ThreadUtils.parmap( + ALL_TIMEZONES, + prefix = "CastSuiteBase-cast-string-to-timestamp", + maxThreads = Runtime.getRuntime.availableProcessors + ) { zid => def checkCastStringToTimestamp(str: String, expected: Timestamp): Unit = { checkEvaluation(cast(Literal(str), TimestampType, Option(zid.getId)), expected) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index a8f7cdb260010..bb8fea71019fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -755,8 +755,10 @@ case class RepairTableCommand( val statusPar: Seq[FileStatus] = if (partitionNames.length > 1 && statuses.length > threshold || partitionNames.length > 2) { // parallelize the list of partitions here, then we can have better parallelism later. + // scalastyle:off parvector val parArray = new ParVector(statuses.toVector) parArray.tasksupport = evalTaskSupport + // scalastyle:on parvector parArray.seq } else { statuses diff --git a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala index 4dd93983e87e3..a02137a56aacc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.expressions -import scala.collection.parallel.immutable.ParVector - import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow} import org.apache.spark.sql.catalyst.expressions._ @@ -26,7 +24,7 @@ import org.apache.spark.sql.execution.HiveResult.hiveResultString import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.tags.SlowSQLTest -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} @SlowSQLTest class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { @@ -197,8 +195,11 @@ class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { // The encrypt expression includes a random initialization vector to its encrypted result classOf[AesEncrypt].getName) - val parFuncs = new ParVector(spark.sessionState.functionRegistry.listFunction().toVector) - parFuncs.foreach { funcId => + ThreadUtils.parmap( + spark.sessionState.functionRegistry.listFunction(), + prefix = "ExpressionInfoSuite-check-outputs-of-expression-examples", + maxThreads = Runtime.getRuntime.availableProcessors + ) { funcId => // Examples can change settings. We clone the session to prevent tests clashing. val clonedSpark = spark.cloneSession() // Coalescing partitions can change result order, so disable it. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index 43aaa7e1eeaec..a8f55c8b4d641 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -52,7 +52,9 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { outputStreams.foreach(_.validateAtStart()) numReceivers = inputStreams.count(_.isInstanceOf[ReceiverInputDStream[_]]) inputStreamNameAndID = inputStreams.map(is => (is.name, is.id)).toSeq + // scalastyle:off parvector new ParVector(inputStreams.toVector).foreach(_.start()) + // scalastyle:on parvector } } @@ -62,7 +64,9 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { def stop(): Unit = { this.synchronized { + // scalastyle:off parvector new ParVector(inputStreams.toVector).foreach(_.stop()) + // scalastyle:on parvector } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala index d1f9dfb791355..4e65bc75e4395 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala @@ -314,8 +314,10 @@ private[streaming] object FileBasedWriteAheadLog { val groupSize = taskSupport.parallelismLevel.max(8) source.grouped(groupSize).flatMap { group => + // scalastyle:off parvector val parallelCollection = new ParVector(group.toVector) parallelCollection.tasksupport = taskSupport + // scalastyle:on parvector parallelCollection.map(handler) }.flatten } From 06f48855d7b9e0cd0b02b6f7884af39ce1a5f68c Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 23 Oct 2023 21:12:01 +0900 Subject: [PATCH 076/521] [SPARK-45631][SS][PYSPARK] Remove @abstractmethod from onQueryIdle in PySpark StreamingQueryListener ### What changes were proposed in this pull request? Credit to anish-db for the initial investigation and the fix. This PR proposes to remove `abstractmethod` annotation from `onQueryIdle` in PySpark StreamingQueryListener. The function `onQueryIdle` was added with the annotation `abstractmethod`, which does not pick up default implementation and enforces users to implement the new method. This breaks all existing streaming query listener implementations and enforces them to add the dummy function implementation at least. This PR re-allows existing implementations to work properly without explicitly adding a new function `onQueryIdle`. ### Why are the changes needed? We broke backward compatibility in [SPARK-43183](https://issues.apache.org/jira/browse/SPARK-43183) and we want to fix it. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Modified tests. Now tests are verifying two different implementations covering old interface vs new interface. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43483 from HeartSaVioR/SPARK-45631. Lead-authored-by: Jungtaek Lim Co-authored-by: Anish Shrigondekar Signed-off-by: Jungtaek Lim (cherry picked from commit 75bc5ac9f2b07bc894091b8b15682ee906a19356) Signed-off-by: Jungtaek Lim --- python/pyspark/sql/streaming/listener.py | 4 +- .../streaming/test_streaming_listener.py | 117 ++++++++++++------ 2 files changed, 82 insertions(+), 39 deletions(-) diff --git a/python/pyspark/sql/streaming/listener.py b/python/pyspark/sql/streaming/listener.py index 16f40396490c7..3a0f30872dc8c 100644 --- a/python/pyspark/sql/streaming/listener.py +++ b/python/pyspark/sql/streaming/listener.py @@ -107,7 +107,9 @@ def onQueryProgress(self, event: "QueryProgressEvent") -> None: """ pass - @abstractmethod + # NOTE: Do not mark this as abstract method, since we released this abstract class without + # this method in prior version and marking this as abstract method would break existing + # implementations. def onQueryIdle(self, event: "QueryIdleEvent") -> None: """ Called when the query is idle and waiting for new data to process. diff --git a/python/pyspark/sql/tests/streaming/test_streaming_listener.py b/python/pyspark/sql/tests/streaming/test_streaming_listener.py index 87d0dae00d8bd..05c1ec71675c2 100644 --- a/python/pyspark/sql/tests/streaming/test_streaming_listener.py +++ b/python/pyspark/sql/tests/streaming/test_streaming_listener.py @@ -251,7 +251,23 @@ def test_listener_events(self): progress_event = None terminated_event = None - class TestListener(StreamingQueryListener): + # V1: Initial interface of StreamingQueryListener containing methods `onQueryStarted`, + # `onQueryProgress`, `onQueryTerminated`. It is prior to Spark 3.5. + class TestListenerV1(StreamingQueryListener): + def onQueryStarted(self, event): + nonlocal start_event + start_event = event + + def onQueryProgress(self, event): + nonlocal progress_event + progress_event = event + + def onQueryTerminated(self, event): + nonlocal terminated_event + terminated_event = event + + # V2: The interface after the method `onQueryIdle` is added. It is Spark 3.5+. + class TestListenerV2(StreamingQueryListener): def onQueryStarted(self, event): nonlocal start_event start_event = event @@ -267,48 +283,71 @@ def onQueryTerminated(self, event): nonlocal terminated_event terminated_event = event - test_listener = TestListener() + def verify(test_listener): + nonlocal start_event + nonlocal progress_event + nonlocal terminated_event - try: - self.spark.streams.addListener(test_listener) + start_event = None + progress_event = None + terminated_event = None - df = self.spark.readStream.format("rate").option("rowsPerSecond", 10).load() + try: + self.spark.streams.addListener(test_listener) - # check successful stateful query - df_stateful = df.groupBy().count() # make query stateful - q = ( - df_stateful.writeStream.format("noop") - .queryName("test") - .outputMode("complete") - .start() - ) - self.assertTrue(q.isActive) - time.sleep(10) - q.stop() + df = self.spark.readStream.format("rate").option("rowsPerSecond", 10).load() - # Make sure all events are empty - self.spark.sparkContext._jsc.sc().listenerBus().waitUntilEmpty() + # check successful stateful query + df_stateful = df.groupBy().count() # make query stateful + q = ( + df_stateful.writeStream.format("noop") + .queryName("test") + .outputMode("complete") + .start() + ) + self.assertTrue(q.isActive) + time.sleep(10) + q.stop() - self.check_start_event(start_event) - self.check_progress_event(progress_event) - self.check_terminated_event(terminated_event) + # Make sure all events are empty + self.spark.sparkContext._jsc.sc().listenerBus().waitUntilEmpty() - # Check query terminated with exception - from pyspark.sql.functions import col, udf + self.check_start_event(start_event) + self.check_progress_event(progress_event) + self.check_terminated_event(terminated_event) - bad_udf = udf(lambda x: 1 / 0) - q = df.select(bad_udf(col("value"))).writeStream.format("noop").start() - time.sleep(5) - q.stop() - self.spark.sparkContext._jsc.sc().listenerBus().waitUntilEmpty() - self.check_terminated_event(terminated_event, "ZeroDivisionError") + # Check query terminated with exception + from pyspark.sql.functions import col, udf - finally: - self.spark.streams.removeListener(test_listener) + bad_udf = udf(lambda x: 1 / 0) + q = df.select(bad_udf(col("value"))).writeStream.format("noop").start() + time.sleep(5) + q.stop() + self.spark.sparkContext._jsc.sc().listenerBus().waitUntilEmpty() + self.check_terminated_event(terminated_event, "ZeroDivisionError") + + finally: + self.spark.streams.removeListener(test_listener) + + verify(TestListenerV1()) + verify(TestListenerV2()) def test_remove_listener(self): # SPARK-38804: Test StreamingQueryManager.removeListener - class TestListener(StreamingQueryListener): + # V1: Initial interface of StreamingQueryListener containing methods `onQueryStarted`, + # `onQueryProgress`, `onQueryTerminated`. It is prior to Spark 3.5. + class TestListenerV1(StreamingQueryListener): + def onQueryStarted(self, event): + pass + + def onQueryProgress(self, event): + pass + + def onQueryTerminated(self, event): + pass + + # V2: The interface after the method `onQueryIdle` is added. It is Spark 3.5+. + class TestListenerV2(StreamingQueryListener): def onQueryStarted(self, event): pass @@ -321,13 +360,15 @@ def onQueryIdle(self, event): def onQueryTerminated(self, event): pass - test_listener = TestListener() + def verify(test_listener): + num_listeners = len(self.spark.streams._jsqm.listListeners()) + self.spark.streams.addListener(test_listener) + self.assertEqual(num_listeners + 1, len(self.spark.streams._jsqm.listListeners())) + self.spark.streams.removeListener(test_listener) + self.assertEqual(num_listeners, len(self.spark.streams._jsqm.listListeners())) - num_listeners = len(self.spark.streams._jsqm.listListeners()) - self.spark.streams.addListener(test_listener) - self.assertEqual(num_listeners + 1, len(self.spark.streams._jsqm.listListeners())) - self.spark.streams.removeListener(test_listener) - self.assertEqual(num_listeners, len(self.spark.streams._jsqm.listListeners())) + verify(TestListenerV1()) + verify(TestListenerV2()) def test_query_started_event_fromJson(self): start_event = """ From da08382faf101aeada757e28ebca24ffe8f5bb8e Mon Sep 17 00:00:00 2001 From: Vitalii Li Date: Tue, 24 Oct 2023 15:13:00 +0800 Subject: [PATCH 077/521] [SPARK-45430] Fix for FramelessOffsetWindowFunction when IGNORE NULLS and offset > rowCount ### What changes were proposed in this pull request? This is a fix for the failure when function that utilized `FramelessOffsetWindowFunctionFrame` is used with `ignoreNulls = true` and `offset > rowCount`. e.g. ``` select x, lead(x, 5) IGNORE NULLS over (order by x) from (select explode(sequence(1, 3)) x) ``` ### Why are the changes needed? Fix existing bug ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Modify existing unit test to cover this case ### Was this patch authored or co-authored using generative AI tooling? No Closes #43236 from vitaliili-db/SPARK-45430. Authored-by: Vitalii Li Signed-off-by: Wenchen Fan (cherry picked from commit 32e1e58411913517c87d7e75942437f4e1c1d40e) Signed-off-by: Wenchen Fan --- .../window/WindowFunctionFrame.scala | 6 ++- .../sql/DataFrameWindowFunctionsSuite.scala | 40 ++++++++++--------- 2 files changed, 27 insertions(+), 19 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala index 2b7f702a7f20a..a849c3894f0d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala @@ -201,7 +201,11 @@ class FrameLessOffsetWindowFunctionFrame( override def prepare(rows: ExternalAppendOnlyUnsafeRowArray): Unit = { resetStates(rows) if (ignoreNulls) { - findNextRowWithNonNullInput() + if (Math.abs(offset) > rows.length) { + fillDefaultValue(EmptyRow) + } else { + findNextRowWithNonNullInput() + } } else { // drain the first few rows if offset is larger than zero while (inputIndex < offset) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index 47380db421734..47a311c71d55d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -819,6 +819,8 @@ class DataFrameWindowFunctionsSuite extends QueryTest lead($"value", 1, null, true).over(window), lead($"value", 2, null, true).over(window), lead($"value", 3, null, true).over(window), + // offset > rowCount: SPARK-45430 + lead($"value", 100, null, true).over(window), lead(concat($"value", $"key"), 1, null, true).over(window), lag($"value", 1).over(window), lag($"value", 2).over(window), @@ -826,27 +828,29 @@ class DataFrameWindowFunctionsSuite extends QueryTest lag($"value", 1, null, true).over(window), lag($"value", 2, null, true).over(window), lag($"value", 3, null, true).over(window), + // abs(offset) > rowCount: SPARK-45430 + lag($"value", -100, null, true).over(window), lag(concat($"value", $"key"), 1, null, true).over(window)) .orderBy($"order"), Seq( - Row("a", 0, null, "x", null, null, "x", "y", "z", "xa", - null, null, null, null, null, null, null), - Row("a", 1, "x", null, null, "x", "y", "z", "v", "ya", - null, null, "x", null, null, null, null), - Row("b", 2, null, null, "y", null, "y", "z", "v", "ya", - "x", null, null, "x", null, null, "xa"), - Row("c", 3, null, "y", null, null, "y", "z", "v", "ya", - null, "x", null, "x", null, null, "xa"), - Row("a", 4, "y", null, "z", "y", "z", "v", null, "za", - null, null, "y", "x", null, null, "xa"), - Row("b", 5, null, "z", "v", null, "z", "v", null, "za", - "y", null, null, "y", "x", null, "ya"), - Row("a", 6, "z", "v", null, "z", "v", null, null, "va", - null, "y", "z", "y", "x", null, "ya"), - Row("a", 7, "v", null, null, "v", null, null, null, null, - "z", null, "v", "z", "y", "x", "za"), - Row("a", 8, null, null, null, null, null, null, null, null, - "v", "z", null, "v", "z", "y", "va"))) + Row("a", 0, null, "x", null, null, "x", "y", "z", null, "xa", + null, null, null, null, null, null, null, null), + Row("a", 1, "x", null, null, "x", "y", "z", "v", null, "ya", + null, null, "x", null, null, null, null, null), + Row("b", 2, null, null, "y", null, "y", "z", "v", null, "ya", + "x", null, null, "x", null, null, null, "xa"), + Row("c", 3, null, "y", null, null, "y", "z", "v", null, "ya", + null, "x", null, "x", null, null, null, "xa"), + Row("a", 4, "y", null, "z", "y", "z", "v", null, null, "za", + null, null, "y", "x", null, null, null, "xa"), + Row("b", 5, null, "z", "v", null, "z", "v", null, null, "za", + "y", null, null, "y", "x", null, null, "ya"), + Row("a", 6, "z", "v", null, "z", "v", null, null, null, "va", + null, "y", "z", "y", "x", null, null, "ya"), + Row("a", 7, "v", null, null, "v", null, null, null, null, null, + "z", null, "v", "z", "y", "x", null, "za"), + Row("a", 8, null, null, null, null, null, null, null, null, null, + "v", "z", null, "v", "z", "y", null, "va"))) } test("lag - Offset expression must be a literal") { From ddccf5add8f5aa35693c4120f0b161a74379aec9 Mon Sep 17 00:00:00 2001 From: Michael Zhang Date: Tue, 24 Oct 2023 14:51:45 +0500 Subject: [PATCH 078/521] [SPARK-45561][SQL] Add proper conversions for TINYINT in MySQLDialect ### What changes were proposed in this pull request? Change MySql Dialect to convert catalyst TINYINT into MySQL TINYINT rather than BYTE and INTEGER. BYTE does not exist in MySQL. The same applies to MsSqlServerDialect. ### Why are the changes needed? Since BYTE type does not exist in MySQL, any casts that could be pushed down involving BYTE type would fail. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? UT pass. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43390 from michaelzhan-db/SPARK-45561. Lead-authored-by: Michael Zhang Co-authored-by: Wenchen Fan Signed-off-by: Max Gekk (cherry picked from commit 5092c8970246eb828a31154796c3b16f0b61bddd) Signed-off-by: Max Gekk --- .../org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala | 8 +++++--- .../scala/org/apache/spark/sql/jdbc/MySQLDialect.scala | 5 ++++- .../test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 2 ++ 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala index dc3acb66ff1f4..20fdc965874ff 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala @@ -56,10 +56,10 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { conn.prepareStatement("CREATE TABLE numbers (onebit BIT(1), tenbits BIT(10), " + "small SMALLINT, med MEDIUMINT, nor INT, big BIGINT, deci DECIMAL(40,20), flt FLOAT, " - + "dbl DOUBLE)").executeUpdate() + + "dbl DOUBLE, tiny TINYINT)").executeUpdate() conn.prepareStatement("INSERT INTO numbers VALUES (b'0', b'1000100101', " + "17, 77777, 123456789, 123456789012345, 123456789012345.123456789012345, " - + "42.75, 1.0000000000000002)").executeUpdate() + + "42.75, 1.0000000000000002, -128)").executeUpdate() conn.prepareStatement("CREATE TABLE dates (d DATE, t TIME, dt DATETIME, ts TIMESTAMP, " + "yr YEAR)").executeUpdate() @@ -89,7 +89,7 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { val rows = df.collect() assert(rows.length == 1) val types = rows(0).toSeq.map(x => x.getClass.toString) - assert(types.length == 9) + assert(types.length == 10) assert(types(0).equals("class java.lang.Boolean")) assert(types(1).equals("class java.lang.Long")) assert(types(2).equals("class java.lang.Integer")) @@ -99,6 +99,7 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { assert(types(6).equals("class java.math.BigDecimal")) assert(types(7).equals("class java.lang.Double")) assert(types(8).equals("class java.lang.Double")) + assert(types(9).equals("class java.lang.Byte")) assert(rows(0).getBoolean(0) == false) assert(rows(0).getLong(1) == 0x225) assert(rows(0).getInt(2) == 17) @@ -109,6 +110,7 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { assert(rows(0).getAs[BigDecimal](6).equals(bd)) assert(rows(0).getDouble(7) == 42.75) assert(rows(0).getDouble(8) == 1.0000000000000002) + assert(rows(0).getByte(9) == 0x80.toByte) } test("Date types") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala index a08c89318b660..c7e14cc78d5bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.connector.catalog.index.TableIndex import org.apache.spark.sql.connector.expressions.{Expression, FieldReference, NamedReference, NullOrdering, SortDirection} import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} -import org.apache.spark.sql.types.{BooleanType, DataType, FloatType, LongType, MetadataBuilder, StringType} +import org.apache.spark.sql.types.{BooleanType, ByteType, DataType, FloatType, LongType, MetadataBuilder, StringType} private case object MySQLDialect extends JdbcDialect with SQLConfHelper { @@ -102,6 +102,8 @@ private case object MySQLDialect extends JdbcDialect with SQLConfHelper { // Some MySQL JDBC drivers converts JSON type into Types.VARCHAR with a precision of -1. // Explicitly converts it into StringType here. Some(StringType) + } else if (sqlType == Types.TINYINT && typeName.equals("TINYINT")) { + Some(ByteType) } else None } @@ -184,6 +186,7 @@ private case object MySQLDialect extends JdbcDialect with SQLConfHelper { // We override getJDBCType so that FloatType is mapped to FLOAT instead case FloatType => Option(JdbcType("FLOAT", java.sql.Types.FLOAT)) case StringType => Option(JdbcType("LONGTEXT", java.sql.Types.LONGVARCHAR)) + case ByteType => Option(JdbcType("TINYINT", java.sql.Types.TINYINT)) case _ => JdbcUtils.getCommonJDBCType(dt) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index eae171e20b706..71c7245b06090 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -914,6 +914,8 @@ class JDBCSuite extends QueryTest with SharedSparkSession { assert(mySqlDialect.getCatalystType(java.sql.Types.VARBINARY, "BIT", 1, metadata) == None) assert(mySqlDialect.getCatalystType(java.sql.Types.BIT, "TINYINT", 1, metadata) == Some(BooleanType)) + assert(mySqlDialect.getCatalystType(java.sql.Types.TINYINT, "TINYINT", 1, metadata) == + Some(ByteType)) } test("SPARK-35446: MySQLDialect type mapping of float") { From 26f6663d76849e1aa16833398082ba9b4a2e73af Mon Sep 17 00:00:00 2001 From: panbingkun Date: Wed, 25 Oct 2023 15:59:00 +0900 Subject: [PATCH 079/521] [SPARK-45588][SPARK-45640][SQL][TESTS][3.5] Fix flaky ProtobufCatalystDataConversionSuite ### What changes were proposed in this pull request? The pr aims to fix flaky ProtobufCatalystDataConversionSuite, include: - Fix the type check (when the random value was empty array, we didn't skip it. Original intention is to skip default values for types.) [SPARK-45588] - When data.get(0) is null, data.get(0).asInstanceOf[Array[Byte]].isEmpty will be thrown java.lang.NullPointerException. [SPARK-45640] Backport above to branch 3.5. Master branch pr: https://github.com/apache/spark/pull/43424 & https://github.com/apache/spark/pull/43493 ### Why are the changes needed? Fix flaky ProtobufCatalystDataConversionSuite. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Pass GA - Manually test ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43521 from panbingkun/branch-3.5_SPARK-45640. Authored-by: panbingkun Signed-off-by: Hyukjin Kwon --- .../sql/protobuf/ProtobufCatalystDataConversionSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufCatalystDataConversionSuite.scala b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufCatalystDataConversionSuite.scala index d3e63a11a66bf..b7f17fece5fa6 100644 --- a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufCatalystDataConversionSuite.scala +++ b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufCatalystDataConversionSuite.scala @@ -137,7 +137,8 @@ class ProtobufCatalystDataConversionSuite while ( data != null && (data.get(0) == defaultValue || - (dt == BinaryType && + (dt.fields(0).dataType == BinaryType && + data.get(0) != null && data.get(0).asInstanceOf[Array[Byte]].isEmpty))) data = generator().asInstanceOf[Row] From 9e4411e2450d0503933626207b5e03308c30bc72 Mon Sep 17 00:00:00 2001 From: Paul Staab Date: Wed, 25 Oct 2023 07:36:15 -0500 Subject: [PATCH 080/521] [SPARK-40154][PYTHON][DOCS] Correct storage level in Dataframe.cache docstring ### What changes were proposed in this pull request? Corrects the docstring `DataFrame.cache` to give the correct storage level after it changed with Spark 3.0. It seems that the docstring of `DataFrame.persist` was updated, but `cache` was forgotten. ### Why are the changes needed? The doctoring claims that `cache` uses serialised storage, but it actually uses deserialised storage. I confirmed that this is still the case with Spark 3.5.0 using the example code from the Jira ticket. ### Does this PR introduce _any_ user-facing change? Yes, the docstring changes. ### How was this patch tested? The Github actions workflow succeeded. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43229 from paulstaab/SPARK-40154. Authored-by: Paul Staab Signed-off-by: Sean Owen (cherry picked from commit 94607dd001b133a25dc9865f25b3f9e7f5a5daa3) Signed-off-by: Sean Owen --- python/pyspark/sql/dataframe.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 30ed73d3c47b0..5707ae2a31fec 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -1485,7 +1485,7 @@ def foreachPartition(self, f: Callable[[Iterator[Row]], None]) -> None: self.rdd.foreachPartition(f) # type: ignore[arg-type] def cache(self) -> "DataFrame": - """Persists the :class:`DataFrame` with the default storage level (`MEMORY_AND_DISK`). + """Persists the :class:`DataFrame` with the default storage level (`MEMORY_AND_DISK_DESER`). .. versionadded:: 1.3.0 @@ -1494,7 +1494,7 @@ def cache(self) -> "DataFrame": Notes ----- - The default storage level has changed to `MEMORY_AND_DISK` to match Scala in 2.0. + The default storage level has changed to `MEMORY_AND_DISK_DESER` to match Scala in 3.0. Returns ------- From 1b77c6556060a921af2158896d2693c08e622624 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Thu, 26 Oct 2023 13:53:30 -0700 Subject: [PATCH 081/521] [SPARK-45652][SQL][3.5] SPJ: Handle empty input partitions after dynamic filtering This is a cherry-pick of https://github.com/apache/spark/pull/43531 to branch-3.5, with a few modifications. ### What changes were proposed in this pull request? Handle the case when input partitions become empty after V2 dynamic filtering, when SPJ is enabled. ### Why are the changes needed? Current in the situation when all input partitions are filtered out via dynamic filtering, SPJ doesn't work but instead will panic: ``` java.util.NoSuchElementException: None.get at scala.None$.get(Option.scala:529) at scala.None$.get(Option.scala:527) at org.apache.spark.sql.execution.datasources.v2.BatchScanExec.filteredPartitions$lzycompute(BatchScanExec.scala:108) at org.apache.spark.sql.execution.datasources.v2.BatchScanExec.filteredPartitions(BatchScanExec.scala:65) at org.apache.spark.sql.execution.datasources.v2.BatchScanExec.inputRDD$lzycompute(BatchScanExec.scala:136) at org.apache.spark.sql.execution.datasources.v2.BatchScanExec.inputRDD(BatchScanExec.scala:135) at org.apache.spark.sql.boson.BosonBatchScanExec.inputRDD$lzycompute(BosonBatchScanExec.scala:28) ``` This is because the `groupPartitions` method will return `None` in this scenario. We should handle the case. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added a test case for this. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43540 from sunchao/SPARK-45652-branch-3.5. Authored-by: Chao Sun Signed-off-by: Dongjoon Hyun --- .../datasources/v2/BatchScanExec.scala | 4 +- .../KeyGroupedPartitioningSuite.scala | 42 +++++++++++++++++++ 2 files changed, 44 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala index eba3c71f871e3..2a3a5cdeb82b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala @@ -101,7 +101,7 @@ case class BatchScanExec( "partition values that are not present in the original partitioning.") } - groupPartitions(newPartitions).get.map(_._2) + groupPartitions(newPartitions).getOrElse(Seq.empty).map(_._2) case _ => // no validation is needed as the data source did not report any specific partitioning @@ -145,7 +145,7 @@ case class BatchScanExec( "is enabled") val groupedPartitions = groupPartitions(finalPartitions.map(_.head), - groupSplits = true).get + groupSplits = true).getOrElse(Seq.empty) // This means the input partitions are not grouped by partition values. We'll need to // check `groupByPartitionValues` and decide whether to group and replicate splits diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala index 8461f528277c3..6b07c77aefb60 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala @@ -1095,4 +1095,46 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { } } } + + test("SPARK-45652: SPJ should handle empty partition after dynamic filtering") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "10") { + val items_partitions = Array(identity("id")) + createTable(items, items_schema, items_partitions) + sql(s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 10.5, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchases_schema, purchases_partitions) + sql(s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + + s"(1, 44.0, cast('2020-01-15' as timestamp)), " + + s"(1, 45.0, cast('2020-01-15' as timestamp)), " + + s"(2, 11.0, cast('2020-01-01' as timestamp)), " + + s"(3, 19.5, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { pushDownValues => + Seq(true, false).foreach { partiallyClustered => { + withSQLConf( + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> + partiallyClustered.toString, + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString) { + // The dynamic filtering effectively filtered out all the partitions + val df = sql(s"SELECT p.price from testcat.ns.$items i, testcat.ns.$purchases p " + + "WHERE i.id = p.item_id AND i.price > 50.0") + checkAnswer(df, Seq.empty) + } + } + } + } + } + } } From e2b92b8812ce6ce590eb5ef8f2661cd73547c6a9 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Fri, 27 Oct 2023 15:25:59 +0900 Subject: [PATCH 082/521] [SPARK-45670][CORE][3.5] SparkSubmit does not support `--total-executor-cores` when deploying on K8s This is the cherry-pick of https://github.com/apache/spark/pull/43536 for branch-3.5 ### What changes were proposed in this pull request? Remove Kubernetes from the support list of `--total-executor-cores` in SparkSubmit ### Why are the changes needed? `--total-executor-cores` does not take effect in Spark on K8s, [the comments from original PR](https://github.com/apache/spark/pull/19717#discussion_r154568773) also proves that ### Does this PR introduce _any_ user-facing change? The output of `spark-submit --help` changed ```patch ... - Spark standalone, Mesos and Kubernetes only: + Spark standalone and Mesos only: --total-executor-cores NUM Total cores for all executors. ... ``` ### How was this patch tested? Pass GA and review. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43550 from pan3793/SPARK-45670-3.5. Authored-by: Cheng Pan Signed-off-by: Hyukjin Kwon --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +- .../scala/org/apache/spark/deploy/SparkSubmitArguments.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 60253ed5fda1f..af35f451e3704 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -683,7 +683,7 @@ private[spark] class SparkSubmit extends Logging { confKey = EXECUTOR_CORES.key), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN | KUBERNETES, ALL_DEPLOY_MODES, confKey = EXECUTOR_MEMORY.key), - OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, + OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES, confKey = CORES_MAX.key), OptionAssigner(args.files, LOCAL | STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, confKey = FILES.key), diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index a3fe5153bee9f..93dd25db0937b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -567,7 +567,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | --kill SUBMISSION_ID If given, kills the driver specified. | --status SUBMISSION_ID If given, requests the status of the driver specified. | - | Spark standalone, Mesos and Kubernetes only: + | Spark standalone and Mesos only: | --total-executor-cores NUM Total cores for all executors. | | Spark standalone, YARN and Kubernetes only: From d393b50e6d5e64976747c9e84e3787366dbe4280 Mon Sep 17 00:00:00 2001 From: zeruibao Date: Fri, 27 Oct 2023 14:45:33 +0800 Subject: [PATCH 083/521] [SPARK-43380][SQL] Fix slowdown in Avro read ### What changes were proposed in this pull request? Fix slowdown in Avro read. There is a https://github.com/apache/spark/pull/42503 causes the performance regression. It seems that creating an `AvroOptions` inside `toSqlType` is very expensive. Try to pass this in the callstack. After regression ![image-20231024-193909](https://github.com/apache/spark/assets/125398515/c6af9376-e058-4da9-8f63-d9e8663b36ef) Before regression ![image-20231024-193650](https://github.com/apache/spark/assets/125398515/fd609c05-accb-4ce8-8020-2866328a52f7) ### Why are the changes needed? Need to fix the performance regression of Avro read. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing UT test ### Was this patch authored or co-authored using generative AI tooling? No Closes #43530 from zeruibao/SPARK-4380-real-fix-regression-2. Lead-authored-by: zeruibao Co-authored-by: Zerui Bao <125398515+zeruibao@users.noreply.github.com> Signed-off-by: Wenchen Fan (cherry picked from commit 7d94c5769a8b95a2811e73527fa6ea60f9087901) Signed-off-by: Wenchen Fan --- .../spark/sql/avro/AvroDataToCatalyst.scala | 3 +- .../spark/sql/avro/AvroDeserializer.scala | 11 ++++--- .../spark/sql/avro/AvroFileFormat.scala | 3 +- .../spark/sql/avro/SchemaConverters.scala | 32 ++++++++++++------- .../v2/avro/AvroPartitionReaderFactory.scala | 3 +- .../AvroCatalystDataConversionSuite.scala | 7 ++-- .../spark/sql/avro/AvroRowReaderSuite.scala | 3 +- .../spark/sql/avro/AvroSerdeSuite.scala | 3 +- .../org/apache/spark/sql/avro/AvroSuite.scala | 2 +- 9 files changed, 43 insertions(+), 24 deletions(-) diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala index 59f2999bdd395..2c2a45fc3f14f 100644 --- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala @@ -61,7 +61,8 @@ private[sql] case class AvroDataToCatalyst( @transient private lazy val reader = new GenericDatumReader[Any](actualSchema, expectedSchema) @transient private lazy val deserializer = - new AvroDeserializer(expectedSchema, dataType, avroOptions.datetimeRebaseModeInRead) + new AvroDeserializer(expectedSchema, dataType, + avroOptions.datetimeRebaseModeInRead, avroOptions.useStableIdForUnionType) @transient private var decoder: BinaryDecoder = _ diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala index e82116eec1e9c..fe0bd7392b636 100644 --- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala @@ -49,18 +49,21 @@ private[sql] class AvroDeserializer( rootCatalystType: DataType, positionalFieldMatch: Boolean, datetimeRebaseSpec: RebaseSpec, - filters: StructFilters) { + filters: StructFilters, + useStableIdForUnionType: Boolean) { def this( rootAvroType: Schema, rootCatalystType: DataType, - datetimeRebaseMode: String) = { + datetimeRebaseMode: String, + useStableIdForUnionType: Boolean) = { this( rootAvroType, rootCatalystType, positionalFieldMatch = false, RebaseSpec(LegacyBehaviorPolicy.withName(datetimeRebaseMode)), - new NoopFilters) + new NoopFilters, + useStableIdForUnionType) } private lazy val decimalConversions = new DecimalConversion() @@ -118,7 +121,7 @@ private[sql] class AvroDeserializer( val incompatibleMsg = errorPrefix + s"schema is incompatible (avroType = $avroType, sqlType = ${catalystType.sql})" - val realDataType = SchemaConverters.toSqlType(avroType).dataType + val realDataType = SchemaConverters.toSqlType(avroType, useStableIdForUnionType).dataType val confKey = SQLConf.LEGACY_AVRO_ALLOW_INCOMPATIBLE_SCHEMA val preventReadingIncorrectType = !SQLConf.get.getConf(confKey) diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala index 53562a3afdb5b..7b0292df43c2f 100755 --- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala @@ -141,7 +141,8 @@ private[sql] class AvroFileFormat extends FileFormat requiredSchema, parsedOptions.positionalFieldMatching, datetimeRebaseMode, - avroFilters) + avroFilters, + parsedOptions.useStableIdForUnionType) override val stopPosition = file.start + file.length override def hasNext: Boolean = hasNextRow diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala index 6f21639e28d68..06abe977e3b08 100644 --- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala @@ -46,16 +46,24 @@ object SchemaConverters { */ case class SchemaType(dataType: DataType, nullable: Boolean) + /** + * Converts an Avro schema to a corresponding Spark SQL schema. + * + * @since 4.0.0 + */ + def toSqlType(avroSchema: Schema, useStableIdForUnionType: Boolean): SchemaType = { + toSqlTypeHelper(avroSchema, Set.empty, useStableIdForUnionType) + } /** * Converts an Avro schema to a corresponding Spark SQL schema. * * @since 2.4.0 */ def toSqlType(avroSchema: Schema): SchemaType = { - toSqlTypeHelper(avroSchema, Set.empty, AvroOptions(Map())) + toSqlType(avroSchema, false) } def toSqlType(avroSchema: Schema, options: Map[String, String]): SchemaType = { - toSqlTypeHelper(avroSchema, Set.empty, AvroOptions(options)) + toSqlTypeHelper(avroSchema, Set.empty, AvroOptions(options).useStableIdForUnionType) } // The property specifies Catalyst type of the given field @@ -64,7 +72,7 @@ object SchemaConverters { private def toSqlTypeHelper( avroSchema: Schema, existingRecordNames: Set[String], - avroOptions: AvroOptions): SchemaType = { + useStableIdForUnionType: Boolean): SchemaType = { avroSchema.getType match { case INT => avroSchema.getLogicalType match { case _: Date => SchemaType(DateType, nullable = false) @@ -117,7 +125,7 @@ object SchemaConverters { } val newRecordNames = existingRecordNames + avroSchema.getFullName val fields = avroSchema.getFields.asScala.map { f => - val schemaType = toSqlTypeHelper(f.schema(), newRecordNames, avroOptions) + val schemaType = toSqlTypeHelper(f.schema(), newRecordNames, useStableIdForUnionType) StructField(f.name, schemaType.dataType, schemaType.nullable) } @@ -127,13 +135,14 @@ object SchemaConverters { val schemaType = toSqlTypeHelper( avroSchema.getElementType, existingRecordNames, - avroOptions) + useStableIdForUnionType) SchemaType( ArrayType(schemaType.dataType, containsNull = schemaType.nullable), nullable = false) case MAP => - val schemaType = toSqlTypeHelper(avroSchema.getValueType, existingRecordNames, avroOptions) + val schemaType = toSqlTypeHelper(avroSchema.getValueType, + existingRecordNames, useStableIdForUnionType) SchemaType( MapType(StringType, schemaType.dataType, valueContainsNull = schemaType.nullable), nullable = false) @@ -143,17 +152,18 @@ object SchemaConverters { // In case of a union with null, eliminate it and make a recursive call val remainingUnionTypes = AvroUtils.nonNullUnionBranches(avroSchema) if (remainingUnionTypes.size == 1) { - toSqlTypeHelper(remainingUnionTypes.head, existingRecordNames, avroOptions) + toSqlTypeHelper(remainingUnionTypes.head, existingRecordNames, useStableIdForUnionType) .copy(nullable = true) } else { toSqlTypeHelper( Schema.createUnion(remainingUnionTypes.asJava), existingRecordNames, - avroOptions).copy(nullable = true) + useStableIdForUnionType).copy(nullable = true) } } else avroSchema.getTypes.asScala.map(_.getType).toSeq match { case Seq(t1) => - toSqlTypeHelper(avroSchema.getTypes.get(0), existingRecordNames, avroOptions) + toSqlTypeHelper(avroSchema.getTypes.get(0), + existingRecordNames, useStableIdForUnionType) case Seq(t1, t2) if Set(t1, t2) == Set(INT, LONG) => SchemaType(LongType, nullable = false) case Seq(t1, t2) if Set(t1, t2) == Set(FLOAT, DOUBLE) => @@ -167,9 +177,9 @@ object SchemaConverters { val fieldNameSet : mutable.Set[String] = mutable.Set() val fields = avroSchema.getTypes.asScala.zipWithIndex.map { case (s, i) => - val schemaType = toSqlTypeHelper(s, existingRecordNames, avroOptions) + val schemaType = toSqlTypeHelper(s, existingRecordNames, useStableIdForUnionType) - val fieldName = if (avroOptions.useStableIdForUnionType) { + val fieldName = if (useStableIdForUnionType) { // Avro's field name may be case sensitive, so field names for two named type // could be "a" and "A" and we need to distinguish them. In this case, we throw // an exception. diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala b/connector/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala index cc7bd180e8477..2c85c1b067392 100644 --- a/connector/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala @@ -103,7 +103,8 @@ case class AvroPartitionReaderFactory( readDataSchema, options.positionalFieldMatching, datetimeRebaseMode, - avroFilters) + avroFilters, + options.useStableIdForUnionType) override val stopPosition = partitionedFile.start + partitionedFile.length override def next(): Boolean = hasNextRow diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala index 1cb34a0bc4dc5..250b5e0615ad8 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala @@ -59,7 +59,7 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite val expected = { val avroSchema = new Schema.Parser().parse(schema) - SchemaConverters.toSqlType(avroSchema).dataType match { + SchemaConverters.toSqlType(avroSchema, false).dataType match { case st: StructType => Row.fromSeq((0 until st.length).map(_ => null)) case _ => null } @@ -281,13 +281,14 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite data: GenericData.Record, expected: Option[Any], filters: StructFilters = new NoopFilters): Unit = { - val dataType = SchemaConverters.toSqlType(schema).dataType + val dataType = SchemaConverters.toSqlType(schema, false).dataType val deserializer = new AvroDeserializer( schema, dataType, false, RebaseSpec(LegacyBehaviorPolicy.CORRECTED), - filters) + filters, + false) val deserialized = deserializer.deserialize(data) expected match { case None => assert(deserialized == None) diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroRowReaderSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroRowReaderSuite.scala index 70d0bc6c0ad10..965e3a0c1cba6 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroRowReaderSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroRowReaderSuite.scala @@ -75,7 +75,8 @@ class AvroRowReaderSuite StructType(new StructField("value", IntegerType, true) :: Nil), false, RebaseSpec(CORRECTED), - new NoopFilters) + new NoopFilters, + false) override val stopPosition = fileSize override def hasNext: Boolean = hasNextRow diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSerdeSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSerdeSuite.scala index 7f99f3c737c86..a21f3f008fdc7 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSerdeSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSerdeSuite.scala @@ -226,7 +226,8 @@ object AvroSerdeSuite { sql, isPositional(matchType), RebaseSpec(CORRECTED), - new NoopFilters) + new NoopFilters, + false) } /** diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index ffb0a49641b59..1df99210a55ac 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -2137,7 +2137,7 @@ abstract class AvroSuite private def checkSchemaWithRecursiveLoop(avroSchema: String): Unit = { val message = intercept[IncompatibleSchemaException] { - SchemaConverters.toSqlType(new Schema.Parser().parse(avroSchema)) + SchemaConverters.toSqlType(new Schema.Parser().parse(avroSchema), false) }.getMessage assert(message.contains("Found recursive reference in Avro schema")) From de66d8f2ec78525ea73eba8a0d8fa6e4d4839ead Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Fri, 27 Oct 2023 21:20:40 +0900 Subject: [PATCH 084/521] [SPARK-45706][PYTHON][DOCS] Makes entire Binder build fails fast during setting up ### What changes were proposed in this pull request? This PR proposes to make entire Binder build fails fast during setting up to prevent the Binder image to be successfully built which it cannot be rebuilt later on the same commit. ### Why are the changes needed? Binder build is currently broken for Spark 3.5.0: https://mybinder.org/v2/gh/apache/spark/ce5ddad9903?filepath=python%2Fdocs%2Fsource%2Fgetting_started%2Fquickstart_df.ipynb Seems like we uploaded PySpark late into PyPI, and the installation steps just slightly ignored the failure (a user triggered the first docker image for Binder, and that's being reused at that time PySpark wasn't uploaded to PyPI). ![Screenshot 2023-10-27 at 5 42 26 PM](https://github.com/apache/spark/assets/6477701/9030e4a1-2afa-43a2-aee0-dda01abb46ce) ### Does this PR introduce _any_ user-facing change? Yes, it fixes the user-facing live notebooks (at https://spark.apache.org/docs/latest/api/python/index.html). ### How was this patch tested? Manually tested in my fork: - https://mybinder.org/v2/gh/HyukjinKwon/spark/ce5ddad9903?filepath=python%2Fdocs%2Fsource%2Fgetting_started%2Fquickstart_df.ipynb - https://mybinder.org/v2/gh/HyukjinKwon/spark/ce5ddad9903?filepath=python%2Fdocs%2Fsource%2Fgetting_started%2Fquickstart_connect.ipynb - https://mybinder.org/v2/gh/HyukjinKwon/spark/ce5ddad9903?filepath=python%2Fdocs%2Fsource%2Fgetting_started%2Fquickstart_ps.ipynb ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43553 from HyukjinKwon/SPARK-45706. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit f1d1dc1f87a7e4accd1c3c2d824f39df05465906) Signed-off-by: Hyukjin Kwon --- binder/postBuild | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/binder/postBuild b/binder/postBuild index 70ae23b393707..fec233f8c8ce9 100644 --- a/binder/postBuild +++ b/binder/postBuild @@ -20,6 +20,11 @@ # This file is used for Binder integration to install PySpark available in # Jupyter notebook. +# SPARK-45706: Should fail fast. Otherwise, the Binder image is successfully +# built, and it cannot be rebuilt. +set -o pipefail +set -e + VERSION=$(python -c "exec(open('python/pyspark/version.py').read()); print(__version__)") TAG=$(git describe --tags --exact-match 2>/dev/null) From 238630e3aca7c146201f9032a8294a06e7ee3dbd Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 27 Oct 2023 19:21:04 -0700 Subject: [PATCH 085/521] [SPARK-45678][CORE] Cover BufferReleasingInputStream.available/reset under tryOrFetchFailedException ### What changes were proposed in this pull request? This patch proposes to wrap `BufferReleasingInputStream.available/reset` under `tryOrFetchFailedException`. So `IOException` during `available`/`reset` call will be rethrown as `FetchFailedException`. ### Why are the changes needed? We have encountered shuffle data corruption issue: ``` Caused by: java.io.IOException: FAILED_TO_UNCOMPRESS(5) at org.xerial.snappy.SnappyNative.throw_error(SnappyNative.java:112) at org.xerial.snappy.SnappyNative.rawUncompress(Native Method) at org.xerial.snappy.Snappy.rawUncompress(Snappy.java:504) at org.xerial.snappy.Snappy.uncompress(Snappy.java:543) at org.xerial.snappy.SnappyInputStream.hasNextChunk(SnappyInputStream.java:450) at org.xerial.snappy.SnappyInputStream.available(SnappyInputStream.java:497) at org.apache.spark.storage.BufferReleasingInputStream.available(ShuffleBlockFetcherIterator.scala:1356) ``` Spark shuffle has capacity to detect corruption for a few stream op like `read` and `skip`, such `IOException` in the stack trace will be rethrown as `FetchFailedException` that will re-try the failed shuffle task. But in the stack trace it is `available` that is not covered by the mechanism. So no-retry has been happened and the Spark application just failed. As the `available`/`reset` op will also involve data decompression and throw `IOException`, we should be able to check it like `read` and `skip` do. ### Does this PR introduce _any_ user-facing change? Yes. Data corruption during `available`/`reset` op is now causing `FetchFailedException` like `read` and `skip` that can be retried instead of `IOException`. ### How was this patch tested? Added test. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43543 from viirya/add_available. Authored-by: Liang-Chi Hsieh Signed-off-by: Chao Sun --- .../storage/ShuffleBlockFetcherIterator.scala | 8 ++- .../ShuffleBlockFetcherIteratorSuite.scala | 64 ++++++++++++++++++- 2 files changed, 68 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index b21a2aa1c1791..b9365f45a11ae 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -1354,7 +1354,8 @@ private class BufferReleasingInputStream( } } - override def available(): Int = delegate.available() + override def available(): Int = + tryOrFetchFailedException(delegate.available()) override def mark(readlimit: Int): Unit = delegate.mark(readlimit) @@ -1369,12 +1370,13 @@ private class BufferReleasingInputStream( override def read(b: Array[Byte], off: Int, len: Int): Int = tryOrFetchFailedException(delegate.read(b, off, len)) - override def reset(): Unit = delegate.reset() + override def reset(): Unit = tryOrFetchFailedException(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. + * used by the `available`, `read` and `skip` methods inside `BufferReleasingInputStream` + * currently. */ private def tryOrFetchFailedException[T](block: => T): T = { try { diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index af37a72c9e3f8..f2d5f27a66cce 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -182,6 +182,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT blocksByAddress: Map[BlockManagerId, Seq[(BlockId, Long, Int)]], taskContext: Option[TaskContext] = None, streamWrapperLimitSize: Option[Long] = None, + corruptAtAvailableReset: Boolean = false, blockManager: Option[BlockManager] = None, maxBytesInFlight: Long = Long.MaxValue, maxReqsInFlight: Int = Int.MaxValue, @@ -201,7 +202,14 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT blockManager.getOrElse(createMockBlockManager()), mapOutputTracker, blocksByAddress.iterator, - (_, in) => streamWrapperLimitSize.map(new LimitedInputStream(in, _)).getOrElse(in), + (_, in) => { + val limited = streamWrapperLimitSize.map(new LimitedInputStream(in, _)).getOrElse(in) + if (corruptAtAvailableReset) { + new CorruptAvailableResetStream(limited) + } else { + limited + } + }, maxBytesInFlight, maxReqsInFlight, maxBlocksInFlightPerAddress, @@ -712,6 +720,16 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT corruptBuffer } + private class CorruptAvailableResetStream(in: InputStream) extends InputStream { + override def read(): Int = in.read() + + override def read(dest: Array[Byte], off: Int, len: Int): Int = in.read(dest, off, len) + + override def available(): Int = throw new IOException("corrupt at available") + + override def reset(): Unit = throw new IOException("corrupt at reset") + } + private class CorruptStream(corruptAt: Long = 0L) extends InputStream { var pos = 0 var closed = false @@ -1879,4 +1897,48 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT blockManager = Some(blockManager), streamWrapperLimitSize = Some(100)) verifyLocalBlocksFromFallback(iterator) } + + test("SPARK-45678: retry corrupt blocks on available() and reset()") { + val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2) + val blocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer() + ) + + // Semaphore to coordinate event sequence in two different threads. + val sem = new Semaphore(0) + + answerFetchBlocks { invocation => + val listener = invocation.getArgument[BlockFetchingListener](4) + Future { + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 0, 0).toString, createMockManagedBuffer()) + sem.release() + } + } + + val iterator = createShuffleBlockIteratorWithDefaults( + Map(remoteBmId -> toBlockList(blocks.keys, 1L, 0)), + streamWrapperLimitSize = Some(100), + detectCorruptUseExtraMemory = false, // Don't use `ChunkedByteBufferInputStream`. + corruptAtAvailableReset = true, + checksumEnabled = false + ) + + sem.acquire() + + val (id1, stream) = iterator.next() + assert(id1 === ShuffleBlockId(0, 0, 0)) + + val err1 = intercept[FetchFailedException] { + stream.available() + } + + assert(err1.getMessage.contains("corrupt at available")) + + val err2 = intercept[FetchFailedException] { + stream.reset() + } + + assert(err2.getMessage.contains("corrupt at reset")) + } } From 212f00462f2419039742a305cdc941a886e3a15a Mon Sep 17 00:00:00 2001 From: Emil Ejbyfeldt Date: Tue, 31 Oct 2023 11:19:32 +0800 Subject: [PATCH 086/521] [SPARK-45592][SQL] Correctness issue in AQE with InMemoryTableScanExec Fixes correctness issue in 3.5.0. The problem seems to be that when AQEShuffleRead does a coalesced read it can return a HashPartitioning with the coalesced number of partitions. This causes a correctness bug as the partitioning is not compatible for joins with other HashPartitioning even though the number of partitions matches. This is resolved in this patch by introducing CoalescedHashPartitioning and making AQEShuffleRead return that instead. The fix was suggested by cloud-fan > AQEShuffleRead should probably return a different partitioning, e.g. CoalescedHashPartitioning. It still satisfies ClusterDistribution, so Aggregate is fine and there will be no shuffle. For joins, two CoalescedHashPartitionings are compatible if they have the same original partition number and coalesce boundaries, and CoalescedHashPartitioning is not compatible with HashPartitioning. Correctness bug. Yes, fixed correctness issue. New and existing unit test. No Closes #43435 from eejbyfeldt/SPARK-45592. Authored-by: Emil Ejbyfeldt Signed-off-by: Wenchen Fan (cherry picked from commit 2be03d81cea34ab08c44426837260c22c67e092e) Signed-off-by: Wenchen Fan --- .../plans/physical/partitioning.scala | 49 +++ .../sql/catalyst/DistributionSuite.scala | 124 +++--- .../spark/sql/catalyst/ShuffleSpecSuite.scala | 401 ++++++++++-------- .../adaptive/AQEShuffleReadExec.scala | 11 +- .../org/apache/spark/sql/DatasetSuite.scala | 14 + .../WriteDistributionAndOrderingSuite.scala | 53 ++- 6 files changed, 386 insertions(+), 266 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index d2f9e9b5d5bf5..1eefe65859bdd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -306,6 +306,35 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) override protected def withNewChildrenInternal( newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren) + +} + +case class CoalescedBoundary(startReducerIndex: Int, endReducerIndex: Int) + +/** + * Represents a partitioning where partitions have been coalesced from a HashPartitioning into a + * fewer number of partitions. + */ +case class CoalescedHashPartitioning(from: HashPartitioning, partitions: Seq[CoalescedBoundary]) + extends Expression with Partitioning with Unevaluable { + + override def children: Seq[Expression] = from.expressions + override def nullable: Boolean = from.nullable + override def dataType: DataType = from.dataType + + override def satisfies0(required: Distribution): Boolean = from.satisfies0(required) + + override def createShuffleSpec(distribution: ClusteredDistribution): ShuffleSpec = + CoalescedHashShuffleSpec(from.createShuffleSpec(distribution), partitions) + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression]): CoalescedHashPartitioning = + copy(from = from.copy(expressions = newChildren)) + + override val numPartitions: Int = partitions.length + + override def toString: String = from.toString + override def sql: String = from.sql } /** @@ -661,6 +690,26 @@ case class HashShuffleSpec( override def numPartitions: Int = partitioning.numPartitions } +case class CoalescedHashShuffleSpec( + from: ShuffleSpec, + partitions: Seq[CoalescedBoundary]) extends ShuffleSpec { + + override def isCompatibleWith(other: ShuffleSpec): Boolean = other match { + case SinglePartitionShuffleSpec => + numPartitions == 1 + case CoalescedHashShuffleSpec(otherParent, otherPartitions) => + partitions == otherPartitions && from.isCompatibleWith(otherParent) + case ShuffleSpecCollection(specs) => + specs.exists(isCompatibleWith) + case _ => + false + } + + override def canCreatePartitioning: Boolean = false + + override def numPartitions: Int = partitions.length +} + case class KeyGroupedShuffleSpec( partitioning: KeyGroupedPartitioning, distribution: ClusteredDistribution) extends ShuffleSpec { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala index a924a9ed02e5d..7cb4d5f123253 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst import org.apache.spark.SparkFunSuite /* Implicit conversions */ import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.expressions.{Literal, Murmur3Hash, Pmod} +import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, Murmur3Hash, Pmod} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.types.IntegerType @@ -146,63 +146,75 @@ class DistributionSuite extends SparkFunSuite { false) } - test("HashPartitioning is the output partitioning") { - // HashPartitioning can satisfy ClusteredDistribution iff its hash expressions are a subset of - // the required clustering expressions. - checkSatisfied( - HashPartitioning(Seq($"a", $"b", $"c"), 10), - ClusteredDistribution(Seq($"a", $"b", $"c")), - true) - - checkSatisfied( - HashPartitioning(Seq($"b", $"c"), 10), - ClusteredDistribution(Seq($"a", $"b", $"c")), - true) - - checkSatisfied( - HashPartitioning(Seq($"a", $"b", $"c"), 10), - ClusteredDistribution(Seq($"b", $"c")), - false) - - checkSatisfied( - HashPartitioning(Seq($"a", $"b", $"c"), 10), - ClusteredDistribution(Seq($"d", $"e")), - false) - - // When ClusteredDistribution.requireAllClusterKeys is set to true, - // HashPartitioning can only satisfy ClusteredDistribution iff its hash expressions are - // exactly same as the required clustering expressions. - checkSatisfied( - HashPartitioning(Seq($"a", $"b", $"c"), 10), - ClusteredDistribution(Seq($"a", $"b", $"c"), requireAllClusterKeys = true), - true) - - checkSatisfied( - HashPartitioning(Seq($"b", $"c"), 10), - ClusteredDistribution(Seq($"a", $"b", $"c"), requireAllClusterKeys = true), - false) - - checkSatisfied( - HashPartitioning(Seq($"b", $"a", $"c"), 10), - ClusteredDistribution(Seq($"a", $"b", $"c"), requireAllClusterKeys = true), - false) - - // HashPartitioning cannot satisfy OrderedDistribution - checkSatisfied( - HashPartitioning(Seq($"a", $"b", $"c"), 10), - OrderedDistribution(Seq($"a".asc, $"b".asc, $"c".asc)), - false) + private def testHashPartitioningLike( + partitioningName: String, + create: (Seq[Expression], Int) => Partitioning): Unit = { + + test(s"$partitioningName is the output partitioning") { + // HashPartitioning can satisfy ClusteredDistribution iff its hash expressions are a subset of + // the required clustering expressions. + checkSatisfied( + create(Seq($"a", $"b", $"c"), 10), + ClusteredDistribution(Seq($"a", $"b", $"c")), + true) + + checkSatisfied( + create(Seq($"b", $"c"), 10), + ClusteredDistribution(Seq($"a", $"b", $"c")), + true) + + checkSatisfied( + create(Seq($"a", $"b", $"c"), 10), + ClusteredDistribution(Seq($"b", $"c")), + false) + + checkSatisfied( + create(Seq($"a", $"b", $"c"), 10), + ClusteredDistribution(Seq($"d", $"e")), + false) + + // When ClusteredDistribution.requireAllClusterKeys is set to true, + // HashPartitioning can only satisfy ClusteredDistribution iff its hash expressions are + // exactly same as the required clustering expressions. + checkSatisfied( + create(Seq($"a", $"b", $"c"), 10), + ClusteredDistribution(Seq($"a", $"b", $"c"), requireAllClusterKeys = true), + true) + + checkSatisfied( + create(Seq($"b", $"c"), 10), + ClusteredDistribution(Seq($"a", $"b", $"c"), requireAllClusterKeys = true), + false) + + checkSatisfied( + create(Seq($"b", $"a", $"c"), 10), + ClusteredDistribution(Seq($"a", $"b", $"c"), requireAllClusterKeys = true), + false) + + // HashPartitioning cannot satisfy OrderedDistribution + checkSatisfied( + create(Seq($"a", $"b", $"c"), 10), + OrderedDistribution(Seq($"a".asc, $"b".asc, $"c".asc)), + false) + + checkSatisfied( + create(Seq($"a", $"b", $"c"), 1), + OrderedDistribution(Seq($"a".asc, $"b".asc, $"c".asc)), + false) // TODO: this can be relaxed. + + checkSatisfied( + create(Seq($"b", $"c"), 10), + OrderedDistribution(Seq($"a".asc, $"b".asc, $"c".asc)), + false) + } + } - checkSatisfied( - HashPartitioning(Seq($"a", $"b", $"c"), 1), - OrderedDistribution(Seq($"a".asc, $"b".asc, $"c".asc)), - false) // TODO: this can be relaxed. + testHashPartitioningLike("HashPartitioning", + (expressions, numPartitions) => HashPartitioning(expressions, numPartitions)) - checkSatisfied( - HashPartitioning(Seq($"b", $"c"), 10), - OrderedDistribution(Seq($"a".asc, $"b".asc, $"c".asc)), - false) - } + testHashPartitioningLike("CoalescedHashPartitioning", (expressions, numPartitions) => + CoalescedHashPartitioning( + HashPartitioning(expressions, numPartitions), Seq(CoalescedBoundary(0, numPartitions)))) test("RangePartitioning is the output partitioning") { // RangePartitioning can satisfy OrderedDistribution iff its ordering is a prefix diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala index 51e7688732265..6b069d1c97363 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ShuffleSpecSuite.scala @@ -62,211 +62,254 @@ class ShuffleSpecSuite extends SparkFunSuite with SQLHelper { } } - test("compatibility: HashShuffleSpec on both sides") { - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - expected = true - ) - - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a"), 10), ClusteredDistribution(Seq($"a", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"a"), 10), ClusteredDistribution(Seq($"a", $"b"))), - expected = true - ) + private def testHashShuffleSpecLike( + shuffleSpecName: String, + create: (HashPartitioning, ClusteredDistribution) => ShuffleSpec): Unit = { - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"b"), 10), ClusteredDistribution(Seq($"a", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"d"), 10), ClusteredDistribution(Seq($"c", $"d"))), - expected = true - ) + test(s"compatibility: $shuffleSpecName on both sides") { + checkCompatible( + create(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + create(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + expected = true + ) - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a", $"a", $"b"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"c", $"c", $"d"), 10), - ClusteredDistribution(Seq($"c", $"d"))), - expected = true - ) + checkCompatible( + create(HashPartitioning(Seq($"a"), 10), ClusteredDistribution(Seq($"a", $"b"))), + create(HashPartitioning(Seq($"a"), 10), ClusteredDistribution(Seq($"a", $"b"))), + expected = true + ) - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), - ClusteredDistribution(Seq($"a", $"b", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"a", $"d"), 10), - ClusteredDistribution(Seq($"a", $"c", $"d"))), - expected = true - ) + checkCompatible( + create(HashPartitioning(Seq($"b"), 10), ClusteredDistribution(Seq($"a", $"b"))), + create(HashPartitioning(Seq($"d"), 10), ClusteredDistribution(Seq($"c", $"d"))), + expected = true + ) - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a", $"b", $"a"), 10), - ClusteredDistribution(Seq($"a", $"b", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"a", $"c", $"a"), 10), - ClusteredDistribution(Seq($"a", $"c", $"c"))), - expected = true - ) + checkCompatible( + create(HashPartitioning(Seq($"a", $"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + create(HashPartitioning(Seq($"c", $"c", $"d"), 10), + ClusteredDistribution(Seq($"c", $"d"))), + expected = true + ) - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a", $"b", $"a"), 10), - ClusteredDistribution(Seq($"a", $"b", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"a", $"c", $"a"), 10), - ClusteredDistribution(Seq($"a", $"c", $"d"))), - expected = true - ) + checkCompatible( + create(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b", $"b"))), + create(HashPartitioning(Seq($"a", $"d"), 10), + ClusteredDistribution(Seq($"a", $"c", $"d"))), + expected = true + ) - // negative cases - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"c"), 5), - ClusteredDistribution(Seq($"c", $"d"))), - expected = false - ) + checkCompatible( + create(HashPartitioning(Seq($"a", $"b", $"a"), 10), + ClusteredDistribution(Seq($"a", $"b", $"b"))), + create(HashPartitioning(Seq($"a", $"c", $"a"), 10), + ClusteredDistribution(Seq($"a", $"c", $"c"))), + expected = true + ) - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"b"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - expected = false - ) + checkCompatible( + create(HashPartitioning(Seq($"a", $"b", $"a"), 10), + ClusteredDistribution(Seq($"a", $"b", $"b"))), + create(HashPartitioning(Seq($"a", $"c", $"a"), 10), + ClusteredDistribution(Seq($"a", $"c", $"d"))), + expected = true + ) - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"b"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - expected = false - ) + // negative cases + checkCompatible( + create(HashPartitioning(Seq($"a"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + create(HashPartitioning(Seq($"c"), 5), + ClusteredDistribution(Seq($"c", $"d"))), + expected = false + ) - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"d"), 10), - ClusteredDistribution(Seq($"c", $"d"))), - expected = false - ) + checkCompatible( + create(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + create(HashPartitioning(Seq($"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + expected = false + ) - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"d"), 10), - ClusteredDistribution(Seq($"c", $"d"))), - expected = false - ) + checkCompatible( + create(HashPartitioning(Seq($"a"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + create(HashPartitioning(Seq($"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + expected = false + ) - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a", $"a", $"b"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"a", $"b", $"a"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - expected = false - ) + checkCompatible( + create(HashPartitioning(Seq($"a"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + create(HashPartitioning(Seq($"d"), 10), + ClusteredDistribution(Seq($"c", $"d"))), + expected = false + ) - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a", $"a", $"b"), 10), - ClusteredDistribution(Seq($"a", $"b", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"a", $"b", $"a"), 10), - ClusteredDistribution(Seq($"a", $"b", $"b"))), - expected = false - ) - } + checkCompatible( + create(HashPartitioning(Seq($"a"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + create(HashPartitioning(Seq($"d"), 10), + ClusteredDistribution(Seq($"c", $"d"))), + expected = false + ) - test("compatibility: Only one side is HashShuffleSpec") { - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - SinglePartitionShuffleSpec, - expected = false - ) + checkCompatible( + create(HashPartitioning(Seq($"a", $"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + create(HashPartitioning(Seq($"a", $"b", $"a"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + expected = false + ) - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 1), - ClusteredDistribution(Seq($"a", $"b"))), - SinglePartitionShuffleSpec, - expected = true - ) + checkCompatible( + create(HashPartitioning(Seq($"a", $"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b", $"b"))), + create(HashPartitioning(Seq($"a", $"b", $"a"), 10), + ClusteredDistribution(Seq($"a", $"b", $"b"))), + expected = false + ) + } - checkCompatible( - SinglePartitionShuffleSpec, - HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 1), - ClusteredDistribution(Seq($"a", $"b"))), - expected = true - ) + test(s"compatibility: Only one side is $shuffleSpecName") { + checkCompatible( + create(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + SinglePartitionShuffleSpec, + expected = false + ) - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - RangeShuffleSpec(10, ClusteredDistribution(Seq($"a", $"b"))), - expected = false - ) + checkCompatible( + create(HashPartitioning(Seq($"a", $"b"), 1), + ClusteredDistribution(Seq($"a", $"b"))), + SinglePartitionShuffleSpec, + expected = true + ) - checkCompatible( - RangeShuffleSpec(10, ClusteredDistribution(Seq($"a", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - expected = false - ) + checkCompatible( + SinglePartitionShuffleSpec, + create(HashPartitioning(Seq($"a", $"b"), 1), + ClusteredDistribution(Seq($"a", $"b"))), + expected = true + ) - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - ShuffleSpecCollection(Seq( - HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), - ClusteredDistribution(Seq($"a", $"b"))))), - expected = true - ) + checkCompatible( + create(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + RangeShuffleSpec(10, ClusteredDistribution(Seq($"a", $"b"))), + expected = false + ) - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - ShuffleSpecCollection(Seq( - HashShuffleSpec(HashPartitioning(Seq($"a"), 10), + checkCompatible( + RangeShuffleSpec(10, ClusteredDistribution(Seq($"a", $"b"))), + create(HashPartitioning(Seq($"a", $"b"), 10), ClusteredDistribution(Seq($"a", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), - ClusteredDistribution(Seq($"a", $"b"))))), - expected = true - ) + expected = false + ) - checkCompatible( - HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), - ClusteredDistribution(Seq($"a", $"b"))), - ShuffleSpecCollection(Seq( - HashShuffleSpec(HashPartitioning(Seq($"a"), 10), + checkCompatible( + create(HashPartitioning(Seq($"a", $"b"), 10), ClusteredDistribution(Seq($"a", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"a", $"b", $"c"), 10), - ClusteredDistribution(Seq($"a", $"b", $"c"))))), - expected = false - ) + ShuffleSpecCollection(Seq( + create(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))))), + expected = true + ) - checkCompatible( - ShuffleSpecCollection(Seq( - HashShuffleSpec(HashPartitioning(Seq($"b"), 10), + checkCompatible( + create(HashPartitioning(Seq($"a", $"b"), 10), ClusteredDistribution(Seq($"a", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), - ClusteredDistribution(Seq($"a", $"b"))))), - ShuffleSpecCollection(Seq( - HashShuffleSpec(HashPartitioning(Seq($"a", $"b", $"c"), 10), - ClusteredDistribution(Seq($"a", $"b", $"c"))), - HashShuffleSpec(HashPartitioning(Seq($"d"), 10), - ClusteredDistribution(Seq($"c", $"d"))))), - expected = true - ) + ShuffleSpecCollection(Seq( + create(HashPartitioning(Seq($"a"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + create(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))))), + expected = true + ) - checkCompatible( - ShuffleSpecCollection(Seq( - HashShuffleSpec(HashPartitioning(Seq($"b"), 10), + checkCompatible( + create(HashPartitioning(Seq($"a", $"b"), 10), ClusteredDistribution(Seq($"a", $"b"))), - HashShuffleSpec(HashPartitioning(Seq($"a", $"b"), 10), - ClusteredDistribution(Seq($"a", $"b"))))), - ShuffleSpecCollection(Seq( - HashShuffleSpec(HashPartitioning(Seq($"a", $"b", $"c"), 10), - ClusteredDistribution(Seq($"a", $"b", $"c"))), - HashShuffleSpec(HashPartitioning(Seq($"c"), 10), - ClusteredDistribution(Seq($"c", $"d"))))), - expected = false - ) + ShuffleSpecCollection(Seq( + create(HashPartitioning(Seq($"a"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + create(HashPartitioning(Seq($"a", $"b", $"c"), 10), + ClusteredDistribution(Seq($"a", $"b", $"c"))))), + expected = false + ) + + checkCompatible( + ShuffleSpecCollection(Seq( + create(HashPartitioning(Seq($"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + create(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))))), + ShuffleSpecCollection(Seq( + create(HashPartitioning(Seq($"a", $"b", $"c"), 10), + ClusteredDistribution(Seq($"a", $"b", $"c"))), + create(HashPartitioning(Seq($"d"), 10), + ClusteredDistribution(Seq($"c", $"d"))))), + expected = true + ) + + checkCompatible( + ShuffleSpecCollection(Seq( + create(HashPartitioning(Seq($"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))), + create(HashPartitioning(Seq($"a", $"b"), 10), + ClusteredDistribution(Seq($"a", $"b"))))), + ShuffleSpecCollection(Seq( + create(HashPartitioning(Seq($"a", $"b", $"c"), 10), + ClusteredDistribution(Seq($"a", $"b", $"c"))), + create(HashPartitioning(Seq($"c"), 10), + ClusteredDistribution(Seq($"c", $"d"))))), + expected = false + ) + } + } + + testHashShuffleSpecLike("HashShuffleSpec", + (partitioning, distribution) => HashShuffleSpec(partitioning, distribution)) + testHashShuffleSpecLike("CoalescedHashShuffleSpec", + (partitioning, distribution) => { + val partitions = if (partitioning.numPartitions == 1) { + Seq(CoalescedBoundary(0, 1)) + } else { + Seq(CoalescedBoundary(0, 1), CoalescedBoundary(0, partitioning.numPartitions)) + } + CoalescedHashShuffleSpec(HashShuffleSpec(partitioning, distribution), partitions) + }) + + test("compatibility: CoalescedHashShuffleSpec other specs") { + val hashShuffleSpec = HashShuffleSpec( + HashPartitioning(Seq($"a", $"b"), 10), ClusteredDistribution(Seq($"a", $"b"))) + checkCompatible( + hashShuffleSpec, + CoalescedHashShuffleSpec(hashShuffleSpec, Seq(CoalescedBoundary(0, 10))), + expected = false + ) + + checkCompatible( + CoalescedHashShuffleSpec(hashShuffleSpec, + Seq(CoalescedBoundary(0, 5), CoalescedBoundary(5, 10))), + CoalescedHashShuffleSpec(hashShuffleSpec, + Seq(CoalescedBoundary(0, 5), CoalescedBoundary(5, 10))), + expected = true + ) + + checkCompatible( + CoalescedHashShuffleSpec(hashShuffleSpec, + Seq(CoalescedBoundary(0, 4), CoalescedBoundary(4, 10))), + CoalescedHashShuffleSpec(hashShuffleSpec, + Seq(CoalescedBoundary(0, 5), CoalescedBoundary(5, 10))), + expected = false + ) } test("compatibility: other specs") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEShuffleReadExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEShuffleReadExec.scala index 46ec91dcc0ab2..6b39ac70a62ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEShuffleReadExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AQEShuffleReadExec.scala @@ -19,10 +19,11 @@ package org.apache.spark.sql.execution.adaptive import scala.collection.mutable.ArrayBuffer +import org.apache.spark.SparkException import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning, SinglePartition, UnknownPartitioning} +import org.apache.spark.sql.catalyst.plans.physical.{CoalescedBoundary, CoalescedHashPartitioning, HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning, SinglePartition, UnknownPartitioning} import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExchangeLike} @@ -75,7 +76,13 @@ case class AQEShuffleReadExec private( // partitions is changed. child.outputPartitioning match { case h: HashPartitioning => - CurrentOrigin.withOrigin(h.origin)(h.copy(numPartitions = partitionSpecs.length)) + val partitions = partitionSpecs.map { + case CoalescedPartitionSpec(start, end, _) => CoalescedBoundary(start, end) + // Can not happend due to isCoalescedRead + case unexpected => + throw SparkException.internalError(s"Unexpected ShufflePartitionSpec: $unexpected") + } + CurrentOrigin.withOrigin(h.origin)(CoalescedHashPartitioning(h, partitions)) case r: RangePartitioning => CurrentOrigin.withOrigin(r.origin)(r.copy(numPartitions = partitionSpecs.length)) // This can only happen for `REBALANCE_PARTITIONS_BY_NONE`, which uses diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 6d9c43f866a0c..207c66dc4d43b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -2541,6 +2541,20 @@ class DatasetSuite extends QueryTest val ds = Seq(1, 2).toDS().persist(StorageLevel.NONE) assert(ds.count() == 2) } + + test("SPARK-45592: Coaleasced shuffle read is not compatible with hash partitioning") { + val ee = spark.range(0, 1000000, 1, 5).map(l => (l, l)).toDF() + .persist(org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK) + ee.count() + + val minNbrs1 = ee + .groupBy("_1").agg(min(col("_2")).as("min_number")) + .persist(org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK) + + val join = ee.join(minNbrs1, "_1") + assert(join.count() == 1000000) + } + } class DatasetLargeResultCollectingSuite extends QueryTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala index 6cab0e0239dc4..40938eb642478 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.{catalyst, AnalysisException, DataFrame, Row} import org.apache.spark.sql.catalyst.expressions.{ApplyFunctionExpression, Cast, Literal} import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.catalyst.plans.physical -import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, RangePartitioning, UnknownPartitioning} +import org.apache.spark.sql.catalyst.plans.physical.{CoalescedBoundary, CoalescedHashPartitioning, HashPartitioning, RangePartitioning, UnknownPartitioning} import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.functions._ import org.apache.spark.sql.connector.distributions.{Distribution, Distributions} @@ -264,11 +264,8 @@ class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase ) ) val writePartitioningExprs = Seq(attr("data"), attr("id")) - val writePartitioning = if (!coalesce) { - clusteredWritePartitioning(writePartitioningExprs, targetNumPartitions) - } else { - clusteredWritePartitioning(writePartitioningExprs, Some(1)) - } + val writePartitioning = clusteredWritePartitioning( + writePartitioningExprs, targetNumPartitions, coalesce) checkWriteRequirements( tableDistribution, @@ -377,11 +374,8 @@ class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase ) ) val writePartitioningExprs = Seq(attr("data")) - val writePartitioning = if (!coalesce) { - clusteredWritePartitioning(writePartitioningExprs, targetNumPartitions) - } else { - clusteredWritePartitioning(writePartitioningExprs, Some(1)) - } + val writePartitioning = clusteredWritePartitioning( + writePartitioningExprs, targetNumPartitions, coalesce) checkWriteRequirements( tableDistribution, @@ -875,11 +869,8 @@ class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase ) ) val writePartitioningExprs = Seq(attr("data")) - val writePartitioning = if (!coalesce) { - clusteredWritePartitioning(writePartitioningExprs, targetNumPartitions) - } else { - clusteredWritePartitioning(writePartitioningExprs, Some(1)) - } + val writePartitioning = clusteredWritePartitioning( + writePartitioningExprs, targetNumPartitions, coalesce) checkWriteRequirements( tableDistribution, @@ -963,11 +954,8 @@ class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase ) ) val writePartitioningExprs = Seq(attr("data")) - val writePartitioning = if (!coalesce) { - clusteredWritePartitioning(writePartitioningExprs, targetNumPartitions) - } else { - clusteredWritePartitioning(writePartitioningExprs, Some(1)) - } + val writePartitioning = clusteredWritePartitioning( + writePartitioningExprs, targetNumPartitions, coalesce) checkWriteRequirements( tableDistribution, @@ -1154,11 +1142,8 @@ class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase ) val writePartitioningExprs = Seq(truncateExpr) - val writePartitioning = if (!coalesce) { - clusteredWritePartitioning(writePartitioningExprs, targetNumPartitions) - } else { - clusteredWritePartitioning(writePartitioningExprs, Some(1)) - } + val writePartitioning = clusteredWritePartitioning( + writePartitioningExprs, targetNumPartitions, coalesce) checkWriteRequirements( tableDistribution, @@ -1422,6 +1407,9 @@ class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase case p: physical.HashPartitioning => val resolvedExprs = p.expressions.map(resolveAttrs(_, plan)) p.copy(expressions = resolvedExprs) + case c: physical.CoalescedHashPartitioning => + val resolvedExprs = c.from.expressions.map(resolveAttrs(_, plan)) + c.copy(from = c.from.copy(expressions = resolvedExprs)) case _: UnknownPartitioning => // don't check partitioning if no particular one is expected actualPartitioning @@ -1480,9 +1468,16 @@ class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase private def clusteredWritePartitioning( writePartitioningExprs: Seq[catalyst.expressions.Expression], - targetNumPartitions: Option[Int]): physical.Partitioning = { - HashPartitioning(writePartitioningExprs, - targetNumPartitions.getOrElse(conf.numShufflePartitions)) + targetNumPartitions: Option[Int], + coalesce: Boolean): physical.Partitioning = { + val partitioning = HashPartitioning(writePartitioningExprs, + targetNumPartitions.getOrElse(conf.numShufflePartitions)) + if (coalesce) { + CoalescedHashPartitioning( + partitioning, Seq(CoalescedBoundary(0, partitioning.numPartitions))) + } else { + partitioning + } } private def partitionSizes(dataSkew: Boolean, coalesce: Boolean): Seq[Option[Long]] = { From 2a7e3fec1c8e3867afb9bdecf7a02d6ba7b36f90 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Tue, 31 Oct 2023 16:47:30 +0900 Subject: [PATCH 087/521] [SPARK-45735][PYTHON][CONNECT][TESTS] Reenable CatalogTests without Spark Connect ### What changes were proposed in this pull request? This PR is a followup of https://github.com/apache/spark/pull/39214 that restores the original Catalog tests in PySpark. That PR mistakenly disabled the tests without Spark Connect: https://github.com/apache/spark/blob/fc6a5cca06cf15c4a952cb56720f627efdba7cce/python/pyspark/sql/tests/test_catalog.py#L489 ### Why are the changes needed? To restore the test coverage. ### Does this PR introduce _any_ user-facing change? No, test-only. ### How was this patch tested? Reenabled unittests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43595 from HyukjinKwon/SPARK-45735. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit 76d9a70932df97d8ea4cc6e279933dee29a88571) Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/tests/test_catalog.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/tests/test_catalog.py b/python/pyspark/sql/tests/test_catalog.py index cafffdc9ae8b5..b72172a402bfc 100644 --- a/python/pyspark/sql/tests/test_catalog.py +++ b/python/pyspark/sql/tests/test_catalog.py @@ -486,7 +486,7 @@ def test_refresh_table(self): self.assertEqual(spark.table("my_tab").count(), 0) -class CatalogTests(ReusedSQLTestCase): +class CatalogTests(CatalogTestsMixin, ReusedSQLTestCase): pass From 64242bf6a6425274b83bc1191230437c2d3fbc71 Mon Sep 17 00:00:00 2001 From: zeruibao Date: Tue, 31 Oct 2023 16:46:40 -0700 Subject: [PATCH 088/521] [SPARK-43380][SQL][FOLLOW-UP] Fix slowdown in Avro read ### What changes were proposed in this pull request? Fix slowdown in Avro read. There is a https://github.com/apache/spark/pull/42503 that causes the performance regression. It seems that `SQLConf.get.getConf(confKey)` is very costly. Move it out of `newWriter` function. ### Why are the changes needed? Need to fix the performance regression of Avro read. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing UT test ### Was this patch authored or co-authored using generative AI tooling? No Closes #43606 from zeruibao/SPARK-43380-FIX-SLOWDOWN. Authored-by: zeruibao Signed-off-by: Gengliang Wang (cherry picked from commit 45f73bc69655a236323be1bcb2988341d2aa5203) Signed-off-by: Gengliang Wang --- .../scala/org/apache/spark/sql/avro/AvroDeserializer.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala index fe0bd7392b636..ec34d10a5ffe8 100644 --- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala @@ -105,6 +105,9 @@ private[sql] class AvroDeserializer( s"Cannot convert Avro type $rootAvroType to SQL type ${rootCatalystType.sql}.", ise) } + private lazy val preventReadingIncorrectType = !SQLConf.get + .getConf(SQLConf.LEGACY_AVRO_ALLOW_INCOMPATIBLE_SCHEMA) + def deserialize(data: Any): Option[Any] = converter(data) /** @@ -122,8 +125,6 @@ private[sql] class AvroDeserializer( s"schema is incompatible (avroType = $avroType, sqlType = ${catalystType.sql})" val realDataType = SchemaConverters.toSqlType(avroType, useStableIdForUnionType).dataType - val confKey = SQLConf.LEGACY_AVRO_ALLOW_INCOMPATIBLE_SCHEMA - val preventReadingIncorrectType = !SQLConf.get.getConf(confKey) (avroType.getType, catalystType) match { case (NULL, NullType) => (updater, ordinal, _) => From 1cf1c6a3a8a8cffc5048c584ca1cdae149843d42 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 1 Nov 2023 09:55:45 +0800 Subject: [PATCH 089/521] [SPARK-45749][CORE][WEBUI] Fix `Spark History Server` to sort `Duration` column properly MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR aims to fix an UI regression at Apache Spark 3.2.0 caused by SPARK-34123. From Apache Spark **3.2.0** to **3.5.0**, `Spark History Server` cannot sort `Duration` column. After this PR, Spark History Server can sort `Duration` column properly like Apache Spark 3.1.3 and before. ### Why are the changes needed? Before SPARK-34123, Apache Spark had the `title` attribute for sorting. - https://github.com/apache/spark/pull/31191 ``` {{duration}} ``` Without `title`, `title-numeric` doesn't work. ### Does this PR introduce _any_ user-facing change? No. This is a bug fix. ### How was this patch tested? Manual test. Please use `Safari Private Browsing ` or `Chrome Incognito` mode. Screenshot 2023-10-31 at 5 47 34 PM Screenshot 2023-10-31 at 5 47 29 PM ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43613 from dongjoon-hyun/SPARK-45749. Authored-by: Dongjoon Hyun Signed-off-by: Kent Yao (cherry picked from commit f72510ca9e04ae88660346de440b231fc8225698) Signed-off-by: Kent Yao --- .../resources/org/apache/spark/ui/static/historypage.js | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage.js b/core/src/main/resources/org/apache/spark/ui/static/historypage.js index b334bceb5a039..68dc8ba316dbf 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/historypage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage.js @@ -192,7 +192,12 @@ $(document).ready(function() { }, {name: startedColumnName, data: 'startTime' }, {name: completedColumnName, data: 'endTime' }, - {name: durationColumnName, type: "title-numeric", data: 'duration' }, + { + name: durationColumnName, + type: "title-numeric", + data: 'duration', + render: (id, type, row) => `${row.duration}` + }, {name: 'user', data: 'sparkUser' }, {name: 'lastUpdated', data: 'lastUpdated' }, { From 4ccadd67a87fb76e105ab527d01f27ff9fed95cc Mon Sep 17 00:00:00 2001 From: chenyu <119398199+chenyu-opensource@users.noreply.github.com> Date: Wed, 1 Nov 2023 17:12:45 +0800 Subject: [PATCH 090/521] [SPARK-45751][DOCS] Update the default value for spark.executor.logs.rolling.maxRetainedFile **What changes were proposed in this pull request?** The PR updates the default value of 'spark.executor.logs.rolling.maxRetainedFiles' in configuration.html on the website **Why are the changes needed?** The default value of 'spark.executor.logs.rolling.maxRetainedFiles' is -1, but the website is wrong. **Does this PR introduce any user-facing change?** No **How was this patch tested?** It doesn't need to. **Was this patch authored or co-authored using generative AI tooling?** No Closes #43618 from chenyu-opensource/branch-SPARK-45751. Authored-by: chenyu <119398199+chenyu-opensource@users.noreply.github.com> Signed-off-by: Kent Yao (cherry picked from commit e6b4fa835de3f6d0057bf3809ea369d785967bcd) Signed-off-by: Kent Yao --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 4b0b9b3e3c260..25080784f7374 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -658,7 +658,7 @@ Apart from these, the following properties are also available, and may be useful spark.executor.logs.rolling.maxRetainedFiles - (none) + -1 Sets the number of latest rolling log files that are going to be retained by the system. Older log files will be deleted. Disabled by default. From cec4e488dc7d6fa64dbf3b7bc004ade87a4e27d7 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Sat, 4 Nov 2023 01:16:32 +0800 Subject: [PATCH 091/521] [SPARK-44843][TESTS] Double streamingTimeout for StateStoreMetricsTest to make RocksDBStateStore related streaming tests reliable ### What changes were proposed in this pull request? This PR increases streamingTimeout and the check interval for StateStoreMetricsTest to make RocksDBStateStore-related streaming tests reliable, hopefully. ### Why are the changes needed? ``` SPARK-35896: metrics in StateOperatorProgress are output correctly (RocksDBStateStore with changelog checkpointing) *** FAILED *** (1 minute) [info] Timed out waiting for stream: The code passed to failAfter did not complete within 60 seconds. [info] java.base/java.lang.Thread.getStackTrace(Thread.java:1619) ``` The probability of these tests failing is close to 100%, which seriously affects the UX of making PRs for the contributors. https://github.com/yaooqinn/spark/actions/runs/6744173341/job/18333952141 ### Does this PR introduce _any_ user-facing change? no, test only ### How was this patch tested? this can be verified by `sql - slow test` job in CI ### Was this patch authored or co-authored using generative AI tooling? no Closes #43647 from yaooqinn/SPARK-44843. Authored-by: Kent Yao Signed-off-by: Kent Yao (cherry picked from commit afdce266f0ffeb068d47eca2f2af1bcba66b0e95) Signed-off-by: Kent Yao --- .../apache/spark/sql/streaming/StateStoreMetricsTest.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala index 57ced748cd9f0..07837f5c06473 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.streaming +import org.scalatest.time.SpanSugar._ + import org.apache.spark.sql.execution.streaming.StreamExecution trait StateStoreMetricsTest extends StreamTest { @@ -24,6 +26,8 @@ trait StateStoreMetricsTest extends StreamTest { private var lastCheckedRecentProgressIndex = -1 private var lastQuery: StreamExecution = null + override val streamingTimeout = 120.seconds + override def beforeEach(): Unit = { super.beforeEach() lastCheckedRecentProgressIndex = -1 @@ -106,7 +110,7 @@ trait StateStoreMetricsTest extends StreamTest { AssertOnQuery(s"Check operator progress metrics: operatorName = $operatorName, " + s"numShufflePartitions = $numShufflePartitions, " + s"numStateStoreInstances = $numStateStoreInstances") { q => - eventually(timeout(streamingTimeout)) { + eventually(timeout(streamingTimeout), interval(200.milliseconds)) { val (progressesSinceLastCheck, lastCheckedProgressIndex, numStateOperators) = retrieveProgressesSinceLastCheck(q) assert(operatorIndex < numStateOperators, s"Invalid operator Index: $operatorIndex") From 69a980984d001499b502d32bad2228c0e0f59ba1 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 4 Nov 2023 09:23:58 -0700 Subject: [PATCH 092/521] [SPARK-45791][CONNECT][TESTS] Rename `SparkConnectSessionHodlerSuite.scala` to `SparkConnectSessionHolderSuite.scala` ### What changes were proposed in this pull request? This PR aims to fix a typo `Hodler` in file name. - `SparkConnectSessionHodlerSuite.scala` (from) - `SparkConnectSessionHolderSuite.scala` (to) It's also unmatched with the class name in the file because class name itself is correct. https://github.com/apache/spark/blob/3363c2af3f6a59363135451d251f25e328a4fddf/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionHodlerSuite.scala#L37 ### Why are the changes needed? This is a typo from the original PR. - https://github.com/apache/spark/pull/41580 Since the original PR is shipped as Apache Spark 3.5.0, I created a JIRA instead of a follow-up. We need to backport this patch to `branch-3.5`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43657 from dongjoon-hyun/SPARK-45791. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 6d669fa957463851af463d0ba03d6e6ee76e2cda) Signed-off-by: Dongjoon Hyun --- ...sionHodlerSuite.scala => SparkConnectSessionHolderSuite.scala} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/{SparkConnectSessionHodlerSuite.scala => SparkConnectSessionHolderSuite.scala} (100%) diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionHodlerSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionHolderSuite.scala similarity index 100% rename from connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionHodlerSuite.scala rename to connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionHolderSuite.scala From d36e3e62c72ae121ebf3404db7c4cc51fe66066b Mon Sep 17 00:00:00 2001 From: Kazuyuki Tanimura Date: Tue, 7 Nov 2023 09:06:00 -0800 Subject: [PATCH 093/521] [SPARK-45786][SQL] Fix inaccurate Decimal multiplication and division results ### What changes were proposed in this pull request? This PR fixes inaccurate Decimal multiplication and division results. ### Why are the changes needed? Decimal multiplication and division results may be inaccurate due to rounding issues. #### Multiplication: ``` scala> sql("select -14120025096157587712113961295153.858047 * -0.4652").show(truncate=false) +----------------------------------------------------+ |(-14120025096157587712113961295153.858047 * -0.4652)| +----------------------------------------------------+ |6568635674732509803675414794505.574764 | +----------------------------------------------------+ ``` The correct answer is `6568635674732509803675414794505.574763` Please note that the last digit is `3` instead of `4` as ``` scala> java.math.BigDecimal("-14120025096157587712113961295153.858047").multiply(java.math.BigDecimal("-0.4652")) val res21: java.math.BigDecimal = 6568635674732509803675414794505.5747634644 ``` Since the factional part `.574763` is followed by `4644`, it should not be rounded up. #### Division: ``` scala> sql("select -0.172787979 / 533704665545018957788294905796.5").show(truncate=false) +-------------------------------------------------+ |(-0.172787979 / 533704665545018957788294905796.5)| +-------------------------------------------------+ |-3.237521E-31 | +-------------------------------------------------+ ``` The correct answer is `-3.237520E-31` Please note that the last digit is `0` instead of `1` as ``` scala> java.math.BigDecimal("-0.172787979").divide(java.math.BigDecimal("533704665545018957788294905796.5"), 100, java.math.RoundingMode.DOWN) val res22: java.math.BigDecimal = -3.237520489418037889998826491401059986665344697406144511563561222578738E-31 ``` Since the factional part `.237520` is followed by `4894...`, it should not be rounded up. ### Does this PR introduce _any_ user-facing change? Yes, users will see correct Decimal multiplication and division results. Directly multiplying and dividing with `org.apache.spark.sql.types.Decimal()` (not via SQL) will return 39 digit at maximum instead of 38 at maximum and round down instead of round half-up ### How was this patch tested? Test added ### Was this patch authored or co-authored using generative AI tooling? No Closes #43678 from kazuyukitanimura/SPARK-45786. Authored-by: Kazuyuki Tanimura Signed-off-by: Dongjoon Hyun (cherry picked from commit 5ef3a846f52ab90cb7183953cff3080449d0b57b) Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/types/Decimal.scala | 8 +- .../ArithmeticExpressionSuite.scala | 107 ++++++++++++++++++ .../ansi/decimalArithmeticOperations.sql.out | 14 +-- 3 files changed, 120 insertions(+), 9 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/Decimal.scala index afe73635a6824..77e9aa06c830c 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -499,7 +499,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { def / (that: Decimal): Decimal = if (that.isZero) null else Decimal(toJavaBigDecimal.divide(that.toJavaBigDecimal, - DecimalType.MAX_SCALE, MATH_CONTEXT.getRoundingMode)) + DecimalType.MAX_SCALE + 1, MATH_CONTEXT.getRoundingMode)) def % (that: Decimal): Decimal = if (that.isZero) null @@ -547,7 +547,11 @@ object Decimal { val POW_10 = Array.tabulate[Long](MAX_LONG_DIGITS + 1)(i => math.pow(10, i).toLong) - private val MATH_CONTEXT = new MathContext(DecimalType.MAX_PRECISION, RoundingMode.HALF_UP) + // SPARK-45786 Using RoundingMode.HALF_UP with MathContext may cause inaccurate SQL results + // because TypeCoercion later rounds again. Instead, always round down and use 1 digit longer + // precision than DecimalType.MAX_PRECISION. Then, TypeCoercion will properly round up/down + // the last extra digit. + private val MATH_CONTEXT = new MathContext(DecimalType.MAX_PRECISION + 1, RoundingMode.DOWN) private[sql] val ZERO = Decimal(0) private[sql] val ONE = Decimal(1) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala index e21793ab506c4..568dcd10d1166 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions +import java.math.RoundingMode import java.sql.{Date, Timestamp} import java.time.{Duration, Period} import java.time.temporal.ChronoUnit @@ -225,6 +226,112 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper } } + test("SPARK-45786: Decimal multiply, divide, remainder, quot") { + // Some known cases + checkEvaluation( + Multiply( + Literal(Decimal(BigDecimal("-14120025096157587712113961295153.858047"), 38, 6)), + Literal(Decimal(BigDecimal("-0.4652"), 4, 4)) + ), + Decimal(BigDecimal("6568635674732509803675414794505.574763")) + ) + checkEvaluation( + Multiply( + Literal(Decimal(BigDecimal("-240810500742726"), 15, 0)), + Literal(Decimal(BigDecimal("-5677.6988688550027099967697071"), 29, 25)) + ), + Decimal(BigDecimal("1367249507675382200.164877854336665327")) + ) + checkEvaluation( + Divide( + Literal(Decimal(BigDecimal("-0.172787979"), 9, 9)), + Literal(Decimal(BigDecimal("533704665545018957788294905796.5"), 31, 1)) + ), + Decimal(BigDecimal("-3.237520E-31")) + ) + checkEvaluation( + Divide( + Literal(Decimal(BigDecimal("-0.574302343618"), 12, 12)), + Literal(Decimal(BigDecimal("-795826820326278835912868.106"), 27, 3)) + ), + Decimal(BigDecimal("7.21642358550E-25")) + ) + + // Random tests + val rand = scala.util.Random + def makeNum(p: Int, s: Int): String = { + val int1 = rand.nextLong() + val int2 = rand.nextLong().abs + val frac1 = rand.nextLong().abs + val frac2 = rand.nextLong().abs + s"$int1$int2".take(p - s + (int1 >>> 63).toInt) + "." + s"$frac1$frac2".take(s) + } + + (0 until 100).foreach { _ => + val p1 = rand.nextInt(38) + 1 // 1 <= p1 <= 38 + val s1 = rand.nextInt(p1 + 1) // 0 <= s1 <= p1 + val p2 = rand.nextInt(38) + 1 + val s2 = rand.nextInt(p2 + 1) + + val n1 = makeNum(p1, s1) + val n2 = makeNum(p2, s2) + + val mulActual = Multiply( + Literal(Decimal(BigDecimal(n1), p1, s1)), + Literal(Decimal(BigDecimal(n2), p2, s2)) + ) + val mulExact = new java.math.BigDecimal(n1).multiply(new java.math.BigDecimal(n2)) + + val divActual = Divide( + Literal(Decimal(BigDecimal(n1), p1, s1)), + Literal(Decimal(BigDecimal(n2), p2, s2)) + ) + val divExact = new java.math.BigDecimal(n1) + .divide(new java.math.BigDecimal(n2), 100, RoundingMode.DOWN) + + val remActual = Remainder( + Literal(Decimal(BigDecimal(n1), p1, s1)), + Literal(Decimal(BigDecimal(n2), p2, s2)) + ) + val remExact = new java.math.BigDecimal(n1).remainder(new java.math.BigDecimal(n2)) + + val quotActual = IntegralDivide( + Literal(Decimal(BigDecimal(n1), p1, s1)), + Literal(Decimal(BigDecimal(n2), p2, s2)) + ) + val quotExact = + new java.math.BigDecimal(n1).divideToIntegralValue(new java.math.BigDecimal(n2)) + + Seq(true, false).foreach { allowPrecLoss => + withSQLConf(SQLConf.DECIMAL_OPERATIONS_ALLOW_PREC_LOSS.key -> allowPrecLoss.toString) { + val mulType = Multiply(null, null).resultDecimalType(p1, s1, p2, s2) + val mulResult = Decimal(mulExact.setScale(mulType.scale, RoundingMode.HALF_UP)) + val mulExpected = + if (mulResult.precision > DecimalType.MAX_PRECISION) null else mulResult + checkEvaluation(mulActual, mulExpected) + + val divType = Divide(null, null).resultDecimalType(p1, s1, p2, s2) + val divResult = Decimal(divExact.setScale(divType.scale, RoundingMode.HALF_UP)) + val divExpected = + if (divResult.precision > DecimalType.MAX_PRECISION) null else divResult + checkEvaluation(divActual, divExpected) + + val remType = Remainder(null, null).resultDecimalType(p1, s1, p2, s2) + val remResult = Decimal(remExact.setScale(remType.scale, RoundingMode.HALF_UP)) + val remExpected = + if (remResult.precision > DecimalType.MAX_PRECISION) null else remResult + checkEvaluation(remActual, remExpected) + + val quotType = IntegralDivide(null, null).resultDecimalType(p1, s1, p2, s2) + val quotResult = Decimal(quotExact.setScale(quotType.scale, RoundingMode.HALF_UP)) + val quotExpected = + if (quotResult.precision > DecimalType.MAX_PRECISION) null else quotResult + checkEvaluation(quotActual, quotExpected.toLong) + } + } + } + } + private def testDecimalAndDoubleType(testFunc: (Int => Any) => Unit): Unit = { testFunc(_.toDouble) testFunc(Decimal(_)) diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out index 699c916fd8fdb..9593291fae21d 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out @@ -155,7 +155,7 @@ org.apache.spark.SparkArithmeticException "config" : "\"spark.sql.ansi.enabled\"", "precision" : "38", "scale" : "6", - "value" : "1000000000000000000000000000000000000.00000000000000000000000000000000000000" + "value" : "1000000000000000000000000000000000000.000000000000000000000000000000000000000" }, "queryContext" : [ { "objectType" : "", @@ -204,7 +204,7 @@ org.apache.spark.SparkArithmeticException "config" : "\"spark.sql.ansi.enabled\"", "precision" : "38", "scale" : "6", - "value" : "10123456789012345678901234567890123456.00000000000000000000000000000000000000" + "value" : "10123456789012345678901234567890123456.000000000000000000000000000000000000000" }, "queryContext" : [ { "objectType" : "", @@ -229,7 +229,7 @@ org.apache.spark.SparkArithmeticException "config" : "\"spark.sql.ansi.enabled\"", "precision" : "38", "scale" : "6", - "value" : "101234567890123456789012345678901234.56000000000000000000000000000000000000" + "value" : "101234567890123456789012345678901234.560000000000000000000000000000000000000" }, "queryContext" : [ { "objectType" : "", @@ -254,7 +254,7 @@ org.apache.spark.SparkArithmeticException "config" : "\"spark.sql.ansi.enabled\"", "precision" : "38", "scale" : "6", - "value" : "10123456789012345678901234567890123.45600000000000000000000000000000000000" + "value" : "10123456789012345678901234567890123.456000000000000000000000000000000000000" }, "queryContext" : [ { "objectType" : "", @@ -279,7 +279,7 @@ org.apache.spark.SparkArithmeticException "config" : "\"spark.sql.ansi.enabled\"", "precision" : "38", "scale" : "6", - "value" : "1012345678901234567890123456789012.34560000000000000000000000000000000000" + "value" : "1012345678901234567890123456789012.345600000000000000000000000000000000000" }, "queryContext" : [ { "objectType" : "", @@ -304,7 +304,7 @@ org.apache.spark.SparkArithmeticException "config" : "\"spark.sql.ansi.enabled\"", "precision" : "38", "scale" : "6", - "value" : "101234567890123456789012345678901.23456000000000000000000000000000000000" + "value" : "101234567890123456789012345678901.234560000000000000000000000000000000000" }, "queryContext" : [ { "objectType" : "", @@ -337,7 +337,7 @@ org.apache.spark.SparkArithmeticException "config" : "\"spark.sql.ansi.enabled\"", "precision" : "38", "scale" : "6", - "value" : "101234567890123456789012345678901.23456000000000000000000000000000000000" + "value" : "101234567890123456789012345678901.234560000000000000000000000000000000000" }, "queryContext" : [ { "objectType" : "", From eac87e3985fa614c790bef99cbf9b1d9f3a1f513 Mon Sep 17 00:00:00 2001 From: chenyu <119398199+chenyu-opensource@users.noreply.github.com> Date: Wed, 8 Nov 2023 19:16:48 +0800 Subject: [PATCH 094/521] [SPARK-45829][DOCS] Update the default value for spark.executor.logs.rolling.maxSize **What changes were proposed in this pull request?** The PR updates the default value of 'spark.executor.logs.rolling.maxSize' in configuration.html on the website **Why are the changes needed?** The default value of 'spark.executor.logs.rolling.maxSize' is 1024 * 1024, but the website is wrong. **Does this PR introduce any user-facing change?** No **How was this patch tested?** It doesn't need to. **Was this patch authored or co-authored using generative AI tooling?** No Closes #43712 from chenyu-opensource/branch-SPARK-45829. Authored-by: chenyu <119398199+chenyu-opensource@users.noreply.github.com> Signed-off-by: Kent Yao (cherry picked from commit a9127068194a48786df4f429ceb4f908c71f7138) Signed-off-by: Kent Yao --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 25080784f7374..4604360dda287 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -676,7 +676,7 @@ Apart from these, the following properties are also available, and may be useful spark.executor.logs.rolling.maxSize - (none) + 1024 * 1024 Set the max size of the file in bytes by which the executor logs will be rolled over. Rolling is disabled by default. See spark.executor.logs.rolling.maxRetainedFiles From 85fbb3aca54e93a3a66c4eb5743f70486a8383fc Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 8 Nov 2023 09:18:34 -0800 Subject: [PATCH 095/521] [SPARK-45509][SQL][3.5] Fix df column reference behavior for Spark Connect backport https://github.com/apache/spark/pull/43465 to 3.5 ### What changes were proposed in this pull request? This PR fixes a few problems of column resolution for Spark Connect, to make the behavior closer to classic Spark SQL (unfortunately we still have some behavior differences in corner cases). 1. resolve df column references in both `resolveExpressionByPlanChildren` and `resolveExpressionByPlanOutput`. Previously it's only in `resolveExpressionByPlanChildren`. 2. when the plan id has multiple matches, fail with `AMBIGUOUS_COLUMN_REFERENCE` ### Why are the changes needed? fix behavior differences between spark connect and classic spark sql ### Does this PR introduce _any_ user-facing change? Yes, for spark connect scala client ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #43699 from cloud-fan/backport. Authored-by: Wenchen Fan Signed-off-by: Ruifeng Zheng --- .../main/resources/error/error-classes.json | 9 ++ .../apache/spark/sql/ClientE2ETestSuite.scala | 58 +++++++++++ docs/sql-error-conditions.md | 9 ++ python/pyspark/pandas/indexes/multi.py | 2 +- python/pyspark/sql/connect/plan.py | 4 +- .../analysis/ColumnResolutionHelper.scala | 98 +++++++++++-------- 6 files changed, 138 insertions(+), 42 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 9bc65ae32a276..2d50fe1a1a1a8 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -28,6 +28,15 @@ ], "sqlState" : "42702" }, + "AMBIGUOUS_COLUMN_REFERENCE" : { + "message" : [ + "Column is ambiguous. It's because you joined several DataFrame together, and some of these DataFrames are the same.", + "This column points to one of the DataFrame but Spark is unable to figure out which one.", + "Please alias the DataFrames with different names via `DataFrame.alias` before joining them,", + "and specify the column using qualified name, e.g. `df.alias(\"a\").join(df.alias(\"b\"), col(\"a.id\") > col(\"b.id\"))`." + ], + "sqlState" : "42702" + }, "AMBIGUOUS_LATERAL_COLUMN_ALIAS" : { "message" : [ "Lateral column alias is ambiguous and has matches." diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala index df36b53791a81..feefd19000d1d 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala @@ -767,6 +767,64 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM assert(joined2.schema.catalogString === "struct") } + test("SPARK-45509: ambiguous column reference") { + val session = spark + import session.implicits._ + val df1 = Seq(1 -> "a").toDF("i", "j") + val df1_filter = df1.filter(df1("i") > 0) + val df2 = Seq(2 -> "b").toDF("i", "y") + + checkSameResult( + Seq(Row(1)), + // df1("i") is not ambiguous, and it's still valid in the filtered df. + df1_filter.select(df1("i"))) + + val e1 = intercept[AnalysisException] { + // df1("i") is not ambiguous, but it's not valid in the projected df. + df1.select((df1("i") + 1).as("plus")).select(df1("i")).collect() + } + assert(e1.getMessage.contains("MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT")) + + checkSameResult( + Seq(Row(1, "a")), + // All these column references are not ambiguous and are still valid after join. + df1.join(df2, df1("i") + 1 === df2("i")).sort(df1("i").desc).select(df1("i"), df1("j"))) + + val e2 = intercept[AnalysisException] { + // df1("i") is ambiguous as df1 appears in both join sides. + df1.join(df1, df1("i") === 1).collect() + } + assert(e2.getMessage.contains("AMBIGUOUS_COLUMN_REFERENCE")) + + val e3 = intercept[AnalysisException] { + // df1("i") is ambiguous as df1 appears in both join sides. + df1.join(df1).select(df1("i")).collect() + } + assert(e3.getMessage.contains("AMBIGUOUS_COLUMN_REFERENCE")) + + val e4 = intercept[AnalysisException] { + // df1("i") is ambiguous as df1 appears in both join sides (df1_filter contains df1). + df1.join(df1_filter, df1("i") === 1).collect() + } + assert(e4.getMessage.contains("AMBIGUOUS_COLUMN_REFERENCE")) + + checkSameResult( + Seq(Row("a")), + // df1_filter("i") is not ambiguous as df1_filter does not exist in the join left side. + df1.join(df1_filter, df1_filter("i") === 1).select(df1_filter("j"))) + + val e5 = intercept[AnalysisException] { + // df1("i") is ambiguous as df1 appears in both sides of the first join. + df1.join(df1_filter, df1_filter("i") === 1).join(df2, df1("i") === 1).collect() + } + assert(e5.getMessage.contains("AMBIGUOUS_COLUMN_REFERENCE")) + + checkSameResult( + Seq(Row("a")), + // df1_filter("i") is not ambiguous as df1_filter only appears once. + df1.join(df1_filter).join(df2, df1_filter("i") === 1).select(df1_filter("j"))) + } + test("broadcast join") { withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "-1") { val left = spark.range(100).select(col("id"), rand(10).as("a")) diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index 90d21f9758573..0cf05748f58f0 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -55,6 +55,15 @@ See '``/sql-migration-guide.html#query-engine'. Column or field `` is ambiguous and has `` matches. +### AMBIGUOUS_COLUMN_REFERENCE + +[SQLSTATE: 42702](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) + +Column `` is ambiguous. It's because you joined several DataFrame together, and some of these DataFrames are the same. +This column points to one of the DataFrame but Spark is unable to figure out which one. +Please alias the DataFrames with different names via `DataFrame.alias` before joining them, +and specify the column using qualified name, e.g. `df.alias("a").join(df.alias("b"), col("a.id") > col("b.id"))`. + ### AMBIGUOUS_LATERAL_COLUMN_ALIAS [SQLSTATE: 42702](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) diff --git a/python/pyspark/pandas/indexes/multi.py b/python/pyspark/pandas/indexes/multi.py index dd93e31d0235e..74e0b328e4dfb 100644 --- a/python/pyspark/pandas/indexes/multi.py +++ b/python/pyspark/pandas/indexes/multi.py @@ -815,7 +815,7 @@ def symmetric_difference( # type: ignore[override] sdf_symdiff = sdf_self.union(sdf_other).subtract(sdf_self.intersect(sdf_other)) if sort: - sdf_symdiff = sdf_symdiff.sort(*self._internal.index_spark_columns) + sdf_symdiff = sdf_symdiff.sort(*self._internal.index_spark_column_names) internal = InternalFrame( spark_frame=sdf_symdiff, diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 9af5823dd8b84..b49274e399c48 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -2123,7 +2123,9 @@ def __init__( self._input_grouping_cols = input_grouping_cols self._other_grouping_cols = other_grouping_cols self._other = cast(LogicalPlan, other) - self._func = function._build_common_inline_user_defined_function(*cols) + # The function takes entire DataFrame as inputs, no need to do + # column binding (no input columns). + self._func = function._build_common_inline_user_defined_function() def plan(self, session: "SparkConnectClient") -> proto.Relation: assert self._child is not None diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala index 98cbdea72d53b..c48006286be9a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala @@ -29,10 +29,10 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.CurrentOrigin.withOrigin import org.apache.spark.sql.catalyst.trees.TreePattern._ import org.apache.spark.sql.catalyst.util.toPrettySQL -import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryCompilationErrors} import org.apache.spark.sql.internal.SQLConf -trait ColumnResolutionHelper extends Logging { +trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { def conf: SQLConf @@ -337,7 +337,7 @@ trait ColumnResolutionHelper extends Logging { throws: Boolean = false, allowOuter: Boolean = false): Expression = { resolveExpression( - expr, + tryResolveColumnByPlanId(expr, plan), resolveColumnByName = nameParts => { plan.resolve(nameParts, conf.resolver) }, @@ -358,21 +358,8 @@ trait ColumnResolutionHelper extends Logging { e: Expression, q: LogicalPlan, allowOuter: Boolean = false): Expression = { - val newE = if (e.exists(_.getTagValue(LogicalPlan.PLAN_ID_TAG).nonEmpty)) { - // If the TreeNodeTag 'LogicalPlan.PLAN_ID_TAG' is attached, it means that the plan and - // expression are from Spark Connect, and need to be resolved in this way: - // 1, extract the attached plan id from the expression (UnresolvedAttribute only for now); - // 2, top-down traverse the query plan to find the plan node that matches the plan id; - // 3, if can not find the matching node, fail the analysis due to illegal references; - // 4, resolve the expression with the matching node, if any error occurs here, apply the - // old code path; - resolveExpressionByPlanId(e, q) - } else { - e - } - resolveExpression( - newE, + tryResolveColumnByPlanId(e, q), resolveColumnByName = nameParts => { q.resolveChildren(nameParts, conf.resolver) }, @@ -392,39 +379,46 @@ trait ColumnResolutionHelper extends Logging { } } - private def resolveExpressionByPlanId( + // If the TreeNodeTag 'LogicalPlan.PLAN_ID_TAG' is attached, it means that the plan and + // expression are from Spark Connect, and need to be resolved in this way: + // 1. extract the attached plan id from UnresolvedAttribute; + // 2. top-down traverse the query plan to find the plan node that matches the plan id; + // 3. if can not find the matching node, fail the analysis due to illegal references; + // 4. if more than one matching nodes are found, fail due to ambiguous column reference; + // 5. resolve the expression with the matching node, if any error occurs here, return the + // original expression as it is. + private def tryResolveColumnByPlanId( e: Expression, - q: LogicalPlan): Expression = { - if (!e.exists(_.getTagValue(LogicalPlan.PLAN_ID_TAG).nonEmpty)) { - return e - } - - e match { - case u: UnresolvedAttribute => - resolveUnresolvedAttributeByPlanId(u, q).getOrElse(u) - case _ => - e.mapChildren(c => resolveExpressionByPlanId(c, q)) - } + q: LogicalPlan, + idToPlan: mutable.HashMap[Long, LogicalPlan] = mutable.HashMap.empty): Expression = e match { + case u: UnresolvedAttribute => + resolveUnresolvedAttributeByPlanId( + u, q, idToPlan: mutable.HashMap[Long, LogicalPlan] + ).getOrElse(u) + case _ if e.containsPattern(UNRESOLVED_ATTRIBUTE) => + e.mapChildren(c => tryResolveColumnByPlanId(c, q, idToPlan)) + case _ => e } private def resolveUnresolvedAttributeByPlanId( u: UnresolvedAttribute, - q: LogicalPlan): Option[NamedExpression] = { + q: LogicalPlan, + idToPlan: mutable.HashMap[Long, LogicalPlan]): Option[NamedExpression] = { val planIdOpt = u.getTagValue(LogicalPlan.PLAN_ID_TAG) if (planIdOpt.isEmpty) return None val planId = planIdOpt.get logDebug(s"Extract plan_id $planId from $u") - val planOpt = q.find(_.getTagValue(LogicalPlan.PLAN_ID_TAG).contains(planId)) - if (planOpt.isEmpty) { - // For example: - // df1 = spark.createDataFrame([Row(a = 1, b = 2, c = 3)]]) - // df2 = spark.createDataFrame([Row(a = 1, b = 2)]]) - // df1.select(df2.a) <- illegal reference df2.a - throw new AnalysisException(s"When resolving $u, " + - s"fail to find subplan with plan_id=$planId in $q") - } - val plan = planOpt.get + val plan = idToPlan.getOrElseUpdate(planId, { + findPlanById(u, planId, q).getOrElse { + // For example: + // df1 = spark.createDataFrame([Row(a = 1, b = 2, c = 3)]]) + // df2 = spark.createDataFrame([Row(a = 1, b = 2)]]) + // df1.select(df2.a) <- illegal reference df2.a + throw new AnalysisException(s"When resolving $u, " + + s"fail to find subplan with plan_id=$planId in $q") + } + }) try { plan.resolve(u.nameParts, conf.resolver) @@ -434,4 +428,28 @@ trait ColumnResolutionHelper extends Logging { None } } + + private def findPlanById( + u: UnresolvedAttribute, + id: Long, + plan: LogicalPlan): Option[LogicalPlan] = { + if (plan.getTagValue(LogicalPlan.PLAN_ID_TAG).contains(id)) { + Some(plan) + } else if (plan.children.length == 1) { + findPlanById(u, id, plan.children.head) + } else if (plan.children.length > 1) { + val matched = plan.children.flatMap(findPlanById(u, id, _)) + if (matched.length > 1) { + throw new AnalysisException( + errorClass = "AMBIGUOUS_COLUMN_REFERENCE", + messageParameters = Map("name" -> toSQLId(u.nameParts)), + origin = u.origin + ) + } else { + matched.headOption + } + } else { + None + } + } } From 35d00618d92e855d7b0bd2551b48309d07f4d180 Mon Sep 17 00:00:00 2001 From: xieshuaihu Date: Thu, 9 Nov 2023 15:56:40 +0800 Subject: [PATCH 096/521] [SPARK-45814][CONNECT][SQL] Make ArrowConverters.createEmptyArrowBatch call close() to avoid memory leak ### What changes were proposed in this pull request? Make `ArrowBatchIterator` implement `AutoCloseable` and `ArrowConverters.createEmptyArrowBatch()` call close() to avoid memory leak. ### Why are the changes needed? `ArrowConverters.createEmptyArrowBatch` don't call `super.hasNext`, if `TaskContext.get` returns `None`, then memory allocated in `ArrowBatchIterator` is leaked. In spark connect, `createEmptyArrowBatch` is called in [SparkConnectPlanner](https://github.com/apache/spark/blob/master/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala#L2558) and [SparkConnectPlanExecution](https://github.com/apache/spark/blob/master/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala#L224), which cause a long running driver consume all off-heap memory specified by `-XX:MaxDirectMemorySize`. This is the exception stack: ``` org.apache.arrow.memory.OutOfMemoryException: Failure allocating buffer. at io.netty.buffer.PooledByteBufAllocatorL.allocate(PooledByteBufAllocatorL.java:67) at org.apache.arrow.memory.NettyAllocationManager.(NettyAllocationManager.java:77) at org.apache.arrow.memory.NettyAllocationManager.(NettyAllocationManager.java:84) at org.apache.arrow.memory.NettyAllocationManager$1.create(NettyAllocationManager.java:34) at org.apache.arrow.memory.BaseAllocator.newAllocationManager(BaseAllocator.java:354) at org.apache.arrow.memory.BaseAllocator.newAllocationManager(BaseAllocator.java:349) at org.apache.arrow.memory.BaseAllocator.bufferWithoutReservation(BaseAllocator.java:337) at org.apache.arrow.memory.BaseAllocator.buffer(BaseAllocator.java:315) at org.apache.arrow.memory.BaseAllocator.buffer(BaseAllocator.java:279) at org.apache.arrow.vector.BaseValueVector.allocFixedDataAndValidityBufs(BaseValueVector.java:192) at org.apache.arrow.vector.BaseFixedWidthVector.allocateBytes(BaseFixedWidthVector.java:338) at org.apache.arrow.vector.BaseFixedWidthVector.allocateNew(BaseFixedWidthVector.java:308) at org.apache.arrow.vector.BaseFixedWidthVector.allocateNew(BaseFixedWidthVector.java:273) at org.apache.spark.sql.execution.arrow.ArrowWriter$.$anonfun$create$1(ArrowWriter.scala:44) at scala.collection.StrictOptimizedIterableOps.map(StrictOptimizedIterableOps.scala:100) at scala.collection.StrictOptimizedIterableOps.map$(StrictOptimizedIterableOps.scala:87) at scala.collection.convert.JavaCollectionWrappers$JListWrapper.map(JavaCollectionWrappers.scala:103) at org.apache.spark.sql.execution.arrow.ArrowWriter$.create(ArrowWriter.scala:43) at org.apache.spark.sql.execution.arrow.ArrowConverters$ArrowBatchIterator.(ArrowConverters.scala:93) at org.apache.spark.sql.execution.arrow.ArrowConverters$ArrowBatchWithSchemaIterator.(ArrowConverters.scala:138) at org.apache.spark.sql.execution.arrow.ArrowConverters$$anon$1.(ArrowConverters.scala:231) at org.apache.spark.sql.execution.arrow.ArrowConverters$.createEmptyArrowBatch(ArrowConverters.scala:229) at org.apache.spark.sql.connect.planner.SparkConnectPlanner.handleSqlCommand(SparkConnectPlanner.scala:2481) at org.apache.spark.sql.connect.planner.SparkConnectPlanner.process(SparkConnectPlanner.scala:2426) at org.apache.spark.sql.connect.execution.ExecuteThreadRunner.handleCommand(ExecuteThreadRunner.scala:202) at org.apache.spark.sql.connect.execution.ExecuteThreadRunner.$anonfun$executeInternal$1(ExecuteThreadRunner.scala:158) at org.apache.spark.sql.connect.execution.ExecuteThreadRunner.$anonfun$executeInternal$1$adapted(ExecuteThreadRunner.scala:132) at org.apache.spark.sql.connect.service.SessionHolder.$anonfun$withSession$2(SessionHolder.scala:189) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900) at org.apache.spark.sql.connect.service.SessionHolder.$anonfun$withSession$1(SessionHolder.scala:189) at org.apache.spark.JobArtifactSet$.withActiveJobArtifactState(JobArtifactSet.scala:94) at org.apache.spark.sql.connect.service.SessionHolder.$anonfun$withContextClassLoader$1(SessionHolder.scala:176) at org.apache.spark.util.Utils$.withContextClassLoader(Utils.scala:178) at org.apache.spark.sql.connect.service.SessionHolder.withContextClassLoader(SessionHolder.scala:175) at org.apache.spark.sql.connect.service.SessionHolder.withSession(SessionHolder.scala:188) at org.apache.spark.sql.connect.execution.ExecuteThreadRunner.executeInternal(ExecuteThreadRunner.scala:132) at org.apache.spark.sql.connect.execution.ExecuteThreadRunner.org$apache$spark$sql$connect$execution$ExecuteThreadRunner$$execute(ExecuteThreadRunner.scala:84) at org.apache.spark.sql.connect.execution.ExecuteThreadRunner$ExecutionThread.run(ExecuteThreadRunner.scala:228) Caused by: io.netty.util.internal.OutOfDirectMemoryError: failed to allocate 4194304 byte(s) of direct memory (used: 1069547799, max: 1073741824) at io.netty.util.internal.PlatformDependent.incrementMemoryCounter(PlatformDependent.java:845) at io.netty.util.internal.PlatformDependent.allocateDirectNoCleaner(PlatformDependent.java:774) at io.netty.buffer.PoolArena$DirectArena.allocateDirect(PoolArena.java:721) at io.netty.buffer.PoolArena$DirectArena.newChunk(PoolArena.java:696) at io.netty.buffer.PoolArena.allocateNormal(PoolArena.java:215) at io.netty.buffer.PoolArena.tcacheAllocateSmall(PoolArena.java:180) at io.netty.buffer.PoolArena.allocate(PoolArena.java:137) at io.netty.buffer.PoolArena.allocate(PoolArena.java:129) at io.netty.buffer.PooledByteBufAllocatorL$InnerAllocator.newDirectBufferL(PooledByteBufAllocatorL.java:181) at io.netty.buffer.PooledByteBufAllocatorL$InnerAllocator.directBuffer(PooledByteBufAllocatorL.java:214) at io.netty.buffer.PooledByteBufAllocatorL.allocate(PooledByteBufAllocatorL.java:58) ... 37 more ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manually test ### Was this patch authored or co-authored using generative AI tooling? No Closes #43691 from xieshuaihu/spark-45814. Authored-by: xieshuaihu Signed-off-by: yangjie01 (cherry picked from commit c128f811820e5a31ddd5bd1c95ed8dd49017eaea) Signed-off-by: yangjie01 --- .../sql/execution/arrow/ArrowConverters.scala | 25 +++++++++++++------ 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala index 86dd7984b5859..a843582e9c2c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala @@ -80,7 +80,7 @@ private[sql] object ArrowConverters extends Logging { maxRecordsPerBatch: Long, timeZoneId: String, errorOnDuplicatedFieldNames: Boolean, - context: TaskContext) extends Iterator[Array[Byte]] { + context: TaskContext) extends Iterator[Array[Byte]] with AutoCloseable { protected val arrowSchema = ArrowUtils.toArrowSchema(schema, timeZoneId, errorOnDuplicatedFieldNames) @@ -93,13 +93,11 @@ private[sql] object ArrowConverters extends Logging { protected val arrowWriter = ArrowWriter.create(root) Option(context).foreach {_.addTaskCompletionListener[Unit] { _ => - root.close() - allocator.close() + close() }} override def hasNext: Boolean = rowIter.hasNext || { - root.close() - allocator.close() + close() false } @@ -124,6 +122,11 @@ private[sql] object ArrowConverters extends Logging { out.toByteArray } + + override def close(): Unit = { + root.close() + allocator.close() + } } private[sql] class ArrowBatchWithSchemaIterator( @@ -226,11 +229,19 @@ private[sql] object ArrowConverters extends Logging { schema: StructType, timeZoneId: String, errorOnDuplicatedFieldNames: Boolean): Array[Byte] = { - new ArrowBatchWithSchemaIterator( + val batches = new ArrowBatchWithSchemaIterator( Iterator.empty, schema, 0L, 0L, timeZoneId, errorOnDuplicatedFieldNames, TaskContext.get) { override def hasNext: Boolean = true - }.next() + } + Utils.tryWithSafeFinally { + batches.next() + } { + // If taskContext is null, `batches.close()` should be called to avoid memory leak. + if (TaskContext.get() == null) { + batches.close() + } + } } /** From fbc150fbbb702f18ca12c6e6dec3fe01dbe76612 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 9 Nov 2023 16:23:38 +0800 Subject: [PATCH 097/521] [SPARK-45847][SQL][TESTS] CliSuite flakiness due to non-sequential guarantee for stdout&stderr ### What changes were proposed in this pull request? In CliSuite, This PR adds a retry for tests that write errors to STDERR. ### Why are the changes needed? To fix flakiness tests as below https://github.com/chenhao-db/apache-spark/actions/runs/6791437199/job/18463313766 https://github.com/dongjoon-hyun/spark/actions/runs/6753670527/job/18361206900 ```sql [info] Spark master: local, Application Id: local-1699402393189 [info] spark-sql> /* SELECT /*+ HINT() 4; */; [info] [info] [PARSE_SYNTAX_ERROR] Syntax error at or near ';'. SQLSTATE: 42601 (line 1, pos 26) [info] [info] == SQL == [info] /* SELECT /*+ HINT() 4; */; [info] --------------------------^^^ [info] [info] spark-sql> /* SELECT /*+ HINT() 4; */ SELECT 1; [info] 1 [info] Time taken: 1.499 seconds, Fetched 1 row(s) [info] [info] [UNCLOSED_BRACKETED_COMMENT] Found an unclosed bracketed comment. Please, append */ at the end of the comment. SQLSTATE: 42601 [info] == SQL == [info] /* Here is a unclosed bracketed comment SELECT 1; [info] spark-sql> /* Here is a unclosed bracketed comment SELECT 1; [info] spark-sql> /* SELECT /*+ HINT() */ 4; */; [info] spark-sql> ``` As you can see the fragment above, the query on the 3rd line from the bottom, came from STDOUT, was printed later than its error output, came from STDERR. In this scenario, the error output would not match anything and would simply go unnoticed. Finally, timed out and failed. ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? existing tests and CI ### Was this patch authored or co-authored using generative AI tooling? no Closes #43725 from yaooqinn/SPARK-45847. Authored-by: Kent Yao Signed-off-by: Kent Yao (cherry picked from commit 06d8cbe073499ff16bca3165e2de1192daad3984) Signed-off-by: Kent Yao --- .../sql/hive/thriftserver/CliSuite.scala | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 8ba9ea28a5a96..343b32e6227c2 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -383,7 +383,7 @@ class CliSuite extends SparkFunSuite { ) } - test("SPARK-11188 Analysis error reporting") { + testRetry("SPARK-11188 Analysis error reporting") { runCliWithin(timeout = 2.minute, errorResponses = Seq("AnalysisException"))( "select * from nonexistent_table;" -> "nonexistent_table" @@ -551,7 +551,7 @@ class CliSuite extends SparkFunSuite { ) } - test("SparkException with root cause will be printStacktrace") { + testRetry("SparkException with root cause will be printStacktrace") { // If it is not in silent mode, will print the stacktrace runCliWithin( 1.minute, @@ -575,8 +575,8 @@ class CliSuite extends SparkFunSuite { runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15") } - test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") { - runCliWithin(4.minute)( + testRetry("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") { + runCliWithin(1.minute)( "/* SELECT 'test';*/ SELECT 'test';" -> "test", ";;/* SELECT 'test';*/ SELECT 'test';" -> "test", "/* SELECT 'test';*/;; SELECT 'test';" -> "test", @@ -623,8 +623,8 @@ class CliSuite extends SparkFunSuite { ) } - test("SPARK-37555: spark-sql should pass last unclosed comment to backend") { - runCliWithin(5.minute)( + testRetry("SPARK-37555: spark-sql should pass last unclosed comment to backend") { + runCliWithin(1.minute)( // Only unclosed comment. "/* SELECT /*+ HINT() 4; */;".stripMargin -> "Syntax error at or near ';'", // Unclosed nested bracketed comment. @@ -637,7 +637,7 @@ class CliSuite extends SparkFunSuite { ) } - test("SPARK-37694: delete [jar|file|archive] shall use spark sql processor") { + testRetry("SPARK-37694: delete [jar|file|archive] shall use spark sql processor") { runCliWithin(2.minute, errorResponses = Seq("ParseException"))( "delete jar dummy.jar;" -> "Syntax error at or near 'jar': missing 'FROM'.(line 1, pos 7)") } @@ -678,7 +678,7 @@ class CliSuite extends SparkFunSuite { SparkSQLEnv.stop() } - test("SPARK-39068: support in-memory catalog and running concurrently") { + testRetry("SPARK-39068: support in-memory catalog and running concurrently") { val extraConf = Seq("-c", s"${StaticSQLConf.CATALOG_IMPLEMENTATION.key}=in-memory") val cd = new CountDownLatch(2) def t: Thread = new Thread { @@ -698,7 +698,7 @@ class CliSuite extends SparkFunSuite { } // scalastyle:off line.size.limit - test("formats of error messages") { + testRetry("formats of error messages") { def check(format: ErrorMessageFormat.Value, errorMessage: String, silent: Boolean): Unit = { val expected = errorMessage.split(System.lineSeparator()).map("" -> _) runCliWithin( @@ -810,7 +810,6 @@ class CliSuite extends SparkFunSuite { s"spark.sql.catalog.$catalogName.url=jdbc:derby:memory:$catalogName;create=true" val catalogDriver = s"spark.sql.catalog.$catalogName.driver=org.apache.derby.jdbc.AutoloadedDriver" - val database = s"-database $catalogName.SYS" val catalogConfigs = Seq(catalogImpl, catalogDriver, catalogUrl, "spark.sql.catalogImplementation=in-memory") .flatMap(Seq("--conf", _)) From 0b68e1700f60ad1a32f066c10a0f76bea893b7ce Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 10 Nov 2023 21:09:43 +0800 Subject: [PATCH 098/521] [SPARK-45878][SQL][TESTS] Fix ConcurrentModificationException in CliSuite ### What changes were proposed in this pull request? This PR changes the ArrayBuffer for logs to immutable for reading to prevent ConcurrentModificationException which hides the actual cause of failure ### Why are the changes needed? ```scala [info] - SPARK-29022 Commands using SerDe provided in ADD JAR sql *** FAILED *** (11 seconds, 105 milliseconds) [info] java.util.ConcurrentModificationException: mutation occurred during iteration [info] at scala.collection.mutable.MutationTracker$.checkMutations(MutationTracker.scala:43) [info] at scala.collection.mutable.CheckedIndexedSeqView$CheckedIterator.hasNext(CheckedIndexedSeqView.scala:47) [info] at scala.collection.IterableOnceOps.addString(IterableOnce.scala:1247) [info] at scala.collection.IterableOnceOps.addString$(IterableOnce.scala:1241) [info] at scala.collection.AbstractIterable.addString(Iterable.scala:933) [info] at org.apache.spark.sql.hive.thriftserver.CliSuite.runCliWithin(CliSuite.scala:205) [info] at org.apache.spark.sql.hive.thriftserver.CliSuite.$anonfun$new$20(CliSuite.scala:501) ``` ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? existing tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #43749 from yaooqinn/SPARK-45878. Authored-by: Kent Yao Signed-off-by: Kent Yao (cherry picked from commit b347237735094e9092f4100583ed1d6f3eacf1f6) Signed-off-by: Kent Yao --- .../org/apache/spark/sql/hive/thriftserver/CliSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 343b32e6227c2..38dcd1d8b00af 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -193,7 +193,7 @@ class CliSuite extends SparkFunSuite { ThreadUtils.awaitResult(foundAllExpectedAnswers.future, timeoutForQuery) log.info("Found all expected output.") } catch { case cause: Throwable => - val message = + val message = lock.synchronized { s""" |======================= |CliSuite failure output @@ -207,6 +207,7 @@ class CliSuite extends SparkFunSuite { |End CliSuite failure output |=========================== """.stripMargin + } logError(message, cause) fail(message, cause) } finally { From 68b531dd2b485fa2203d6a2bd2de90afc97a13bb Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 10 Nov 2023 07:50:17 -0800 Subject: [PATCH 099/521] [SPARK-45883][BUILD] Upgrade ORC to 1.9.2 ### What changes were proposed in this pull request? This PR aims to upgrade ORC to 1.9.2 for Apache Spark 4.0.0 and 3.5.1. ### Why are the changes needed? To bring the latest bug fixes. - https://github.com/apache/orc/releases/tag/v1.9.2 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43754 from dongjoon-hyun/SPARK-45883. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 917947e62e1e67f49a83c1ffb0833b61f0c48eb6) Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 6 +++--- pom.xml | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 1d02f8dba567e..9ab51dfa011a2 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -212,9 +212,9 @@ opencsv/2.3//opencsv-2.3.jar opentracing-api/0.33.0//opentracing-api-0.33.0.jar opentracing-noop/0.33.0//opentracing-noop-0.33.0.jar opentracing-util/0.33.0//opentracing-util-0.33.0.jar -orc-core/1.9.1/shaded-protobuf/orc-core-1.9.1-shaded-protobuf.jar -orc-mapreduce/1.9.1/shaded-protobuf/orc-mapreduce-1.9.1-shaded-protobuf.jar -orc-shims/1.9.1//orc-shims-1.9.1.jar +orc-core/1.9.2/shaded-protobuf/orc-core-1.9.2-shaded-protobuf.jar +orc-mapreduce/1.9.2/shaded-protobuf/orc-mapreduce-1.9.2-shaded-protobuf.jar +orc-shims/1.9.2//orc-shims-1.9.2.jar oro/2.0.8//oro-2.0.8.jar osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8//paranamer-2.8.jar diff --git a/pom.xml b/pom.xml index be8400c33bf2b..14e0ab3e0f620 100644 --- a/pom.xml +++ b/pom.xml @@ -141,7 +141,7 @@ 10.14.2.0 1.13.1 - 1.9.1 + 1.9.2 shaded-protobuf 9.4.52.v20230823 4.0.3 From 19d225bf3f56d392ebb4e7727bd30109b1b75bf5 Mon Sep 17 00:00:00 2001 From: "longfei.jiang" Date: Sat, 11 Nov 2023 13:49:18 +0800 Subject: [PATCH 100/521] [MINOR][DOCS] Fix the example value in the docs ### What changes were proposed in this pull request? fix the example value ### Why are the changes needed? for doc ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? Just example value in the docs, no need to test. ### Was this patch authored or co-authored using generative AI tooling? No Closes #43750 from jlfsdtc/fix_typo_in_doc. Authored-by: longfei.jiang Signed-off-by: Kent Yao (cherry picked from commit b501a223bfcf4ddbcb0b2447aa06c549051630b0) Signed-off-by: Kent Yao --- docs/sql-ref-datetime-pattern.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-ref-datetime-pattern.md b/docs/sql-ref-datetime-pattern.md index 5e28a18acefa4..e5d5388f262e4 100644 --- a/docs/sql-ref-datetime-pattern.md +++ b/docs/sql-ref-datetime-pattern.md @@ -41,7 +41,7 @@ Spark uses pattern letters in the following table for date and timestamp parsing |**a**|am-pm-of-day|am-pm|PM| |**h**|clock-hour-of-am-pm (1-12)|number(2)|12| |**K**|hour-of-am-pm (0-11)|number(2)|0| -|**k**|clock-hour-of-day (1-24)|number(2)|0| +|**k**|clock-hour-of-day (1-24)|number(2)|1| |**H**|hour-of-day (0-23)|number(2)|0| |**m**|minute-of-hour|number(2)|30| |**s**|second-of-minute|number(2)|55| From 5c7a55f331b7a41f37b55aff3c5fb29af7916d06 Mon Sep 17 00:00:00 2001 From: Bruce Robbins Date: Sun, 12 Nov 2023 14:34:32 -0800 Subject: [PATCH 101/521] [SPARK-45896][SQL] Construct `ValidateExternalType` with the correct expected type ### What changes were proposed in this pull request? When creating a serializer for a `Map` or `Seq` with an element of type `Option`, pass an expected type of `Option` to `ValidateExternalType` rather than the `Option`'s type argument. ### Why are the changes needed? In 3.4.1, 3.5.0, and master, the following code gets an error: ``` scala> val df = Seq(Seq(Some(Seq(0)))).toDF("a") val df = Seq(Seq(Some(Seq(0)))).toDF("a") org.apache.spark.SparkRuntimeException: [EXPRESSION_ENCODING_FAILED] Failed to encode a value of the expressions: mapobjects(lambdavariable(MapObject, ObjectType(class java.lang.Object), true, -1), mapobjects(lambdavariable(MapObject, ObjectType(class java.lang.Object), true, -2), assertnotnull(validateexternaltype(lambdavariable(MapObject, ObjectType(class java.lang.Object), true, -2), IntegerType, IntegerType)), unwrapoption(ObjectType(interface scala.collection.immutable.Seq), validateexternaltype(lambdavariable(MapObject, ObjectType(class java.lang.Object), true, -1), ArrayType(IntegerType,false), ObjectType(class scala.Option))), None), input[0, scala.collection.immutable.Seq, true], None) AS value#0 to a row. SQLSTATE: 42846 ... Caused by: java.lang.RuntimeException: scala.Some is not a valid external type for schema of array at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.MapObjects_0$(Unknown Source) ... ``` However, this code works in 3.3.3. Similarly, this code gets an error: ``` scala> val df = Seq(Seq(Some(java.sql.Timestamp.valueOf("2023-01-01 00:00:00")))).toDF("a") val df = Seq(Seq(Some(java.sql.Timestamp.valueOf("2023-01-01 00:00:00")))).toDF("a") org.apache.spark.SparkRuntimeException: [EXPRESSION_ENCODING_FAILED] Failed to encode a value of the expressions: mapobjects(lambdavariable(MapObject, ObjectType(class java.lang.Object), true, -1), staticinvoke(class org.apache.spark.sql.catalyst.util.DateTimeUtils$, TimestampType, fromJavaTimestamp, unwrapoption(ObjectType(class java.sql.Timestamp), validateexternaltype(lambdavariable(MapObject, ObjectType(class java.lang.Object), true, -1), TimestampType, ObjectType(class scala.Option))), true, false, true), input[0, scala.collection.immutable.Seq, true], None) AS value#0 to a row. SQLSTATE: 42846 ... Caused by: java.lang.RuntimeException: scala.Some is not a valid external type for schema of timestamp ... ``` As with the first example, this code works in 3.3.3. `SerializerBuildHelper#validateAndSerializeElement` will construct `ValidateExternalType` with an expected type of the `Option`'s type parameter. Therefore, for element types `Option[Seq/Date/Timestamp/BigDecimal]`, `ValidateExternalType` will try to validate that the element is of the contained type (e.g., `BigDecimal`) rather than of type `Option`. Since the element type is of type `Option`, the validation fails. Validation currently works by accident for element types `Option[Map/ Signed-off-by: Dongjoon Hyun (cherry picked from commit e440f3245243a31e7bdfe945e1ce7194609b78fb) Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/SerializerBuildHelper.scala | 7 ++++++- .../catalyst/encoders/ExpressionEncoderSuite.scala | 12 ++++++++++++ .../scala/org/apache/spark/sql/DatasetSuite.scala | 9 +++++++++ 3 files changed, 27 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala index 27090ff6fa5d6..cd087514f4be3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala @@ -450,10 +450,15 @@ object SerializerBuildHelper { private def validateAndSerializeElement( enc: AgnosticEncoder[_], nullable: Boolean): Expression => Expression = { input => + val expected = enc match { + case OptionEncoder(_) => lenientExternalDataTypeFor(enc) + case _ => enc.dataType + } + expressionWithNullSafety( createSerializer( enc, - ValidateExternalType(input, enc.dataType, lenientExternalDataTypeFor(enc))), + ValidateExternalType(input, expected, lenientExternalDataTypeFor(enc))), nullable, WalkedTypePath()) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index 9d2051b01d62e..724a91806c7e0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -477,6 +477,18 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes encodeDecodeTest(Option.empty[Int], "empty option of int") encodeDecodeTest(Option("abc"), "option of string") encodeDecodeTest(Option.empty[String], "empty option of string") + encodeDecodeTest(Seq(Some(Seq(0))), "SPARK-45896: seq of option of seq") + encodeDecodeTest(Map(0 -> Some(Seq(0))), "SPARK-45896: map of option of seq") + encodeDecodeTest(Seq(Some(Timestamp.valueOf("2023-01-01 00:00:00"))), + "SPARK-45896: seq of option of timestamp") + encodeDecodeTest(Map(0 -> Some(Timestamp.valueOf("2023-01-01 00:00:00"))), + "SPARK-45896: map of option of timestamp") + encodeDecodeTest(Seq(Some(Date.valueOf("2023-01-01"))), + "SPARK-45896: seq of option of date") + encodeDecodeTest(Map(0 -> Some(Date.valueOf("2023-01-01"))), + "SPARK-45896: map of option of date") + encodeDecodeTest(Seq(Some(BigDecimal(200))), "SPARK-45896: seq of option of bigdecimal") + encodeDecodeTest(Map(0 -> Some(BigDecimal(200))), "SPARK-45896: map of option of bigdecimal") encodeDecodeTest(ScroogeLikeExample(1), "SPARK-40385 class with only a companion object constructor") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 207c66dc4d43b..0878ae134e9d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -270,6 +270,13 @@ class DatasetSuite extends QueryTest (ClassData("one", 2), 1L), (ClassData("two", 3), 1L)) } + test("SPARK-45896: seq of option of seq") { + val ds = Seq(DataSeqOptSeq(Seq(Some(Seq(0))))).toDS() + checkDataset( + ds, + DataSeqOptSeq(Seq(Some(List(0))))) + } + test("select") { val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkDataset( @@ -2629,6 +2636,8 @@ case class ClassNullableData(a: String, b: Integer) case class NestedStruct(f: ClassData) case class DeepNestedStruct(f: NestedStruct) +case class DataSeqOptSeq(a: Seq[Option[Seq[Int]]]) + /** * A class used to test serialization using encoders. This class throws exceptions when using * Java serialization -- so the only way it can be "serialized" is through our encoders. From 6313e6cc5090036eacb9c234584705d4b398c39e Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 13 Nov 2023 19:31:22 +0800 Subject: [PATCH 102/521] [SPARK-45906][YARN] Fix error message extraction from ResourceNotFoundException ### What changes were proposed in this pull request? This PR aims to fix the error message extraction from `ResourceNotFoundException`, the current wrong implementation also has a potential NPE issue. ### Why are the changes needed? This bug is introduced in SPARK-43202, previously, `e.getCause()` is used to unwrap `InvocationTargetException`, after replacing reflection invocation with direct API calling, we should not apply `getCause()`. ### Does this PR introduce _any_ user-facing change? Yes, bug fix. ### How was this patch tested? Review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43782 from pan3793/SPARK-45906. Authored-by: Cheng Pan Signed-off-by: Kent Yao (cherry picked from commit c29b127dcdd99b0038e96b90177b44b828b32c4b) Signed-off-by: Kent Yao --- .../org/apache/spark/deploy/yarn/ResourceRequestHelper.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala index 0dd4e0a6c8ad9..f9aa11c4d48d6 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala @@ -168,7 +168,7 @@ private object ResourceRequestHelper extends Logging { if (numResourceErrors < 2) { logWarning(s"YARN doesn't know about resource $name, your resource discovery " + s"has to handle properly discovering and isolating the resource! Error: " + - s"${e.getCause.getMessage}") + s"${e.getMessage}") numResourceErrors += 1 } } From d9f0c44e7f24cba95f7bf1737bb52ff73a7b9094 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 14 Nov 2023 12:09:37 +0800 Subject: [PATCH 103/521] [SPARK-45770][SQL][PYTHON][CONNECT][3.5] Introduce plan DataFrameDropColumns for Dataframe.drop ### What changes were proposed in this pull request? backport https://github.com/apache/spark/pull/43683 to 3.5 ### Why are the changes needed? to fix a connect bug ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #43776 from zhengruifeng/sql_drop_plan_35. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/tests/test_dataframe.py | 37 ++++++++++++++ .../sql/catalyst/analysis/Analyzer.scala | 1 + .../ResolveDataFrameDropColumns.scala | 49 +++++++++++++++++++ .../plans/logical/basicLogicalOperators.scala | 14 ++++++ .../sql/catalyst/trees/TreePatterns.scala | 1 + .../scala/org/apache/spark/sql/Dataset.scala | 15 +----- 6 files changed, 104 insertions(+), 13 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index 33049233dee98..5907c8c09fb46 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -106,6 +106,43 @@ def test_drop(self): self.assertEqual(df.drop(col("name"), col("age")).columns, ["active"]) self.assertEqual(df.drop(col("name"), col("age"), col("random")).columns, ["active"]) + def test_drop_join(self): + left_df = self.spark.createDataFrame( + [(1, "a"), (2, "b"), (3, "c")], + ["join_key", "value1"], + ) + right_df = self.spark.createDataFrame( + [(1, "aa"), (2, "bb"), (4, "dd")], + ["join_key", "value2"], + ) + joined_df = left_df.join( + right_df, + on=left_df["join_key"] == right_df["join_key"], + how="left", + ) + + dropped_1 = joined_df.drop(left_df["join_key"]) + self.assertEqual(dropped_1.columns, ["value1", "join_key", "value2"]) + self.assertEqual( + dropped_1.sort("value1").collect(), + [ + Row(value1="a", join_key=1, value2="aa"), + Row(value1="b", join_key=2, value2="bb"), + Row(value1="c", join_key=None, value2=None), + ], + ) + + dropped_2 = joined_df.drop(right_df["join_key"]) + self.assertEqual(dropped_2.columns, ["join_key", "value1", "value2"]) + self.assertEqual( + dropped_2.sort("value1").collect(), + [ + Row(join_key=1, value1="a", value2="aa"), + Row(join_key=2, value1="b", value2="bb"), + Row(join_key=3, value1="c", value2=None), + ], + ) + def test_with_columns_renamed(self): df = self.spark.createDataFrame([("Alice", 50), ("Alice", 60)], ["name", "age"]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 8e3c9b30c61bf..80cb5d8c60876 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -307,6 +307,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor ResolveWindowFrame :: ResolveNaturalAndUsingJoin :: ResolveOutputRelation :: + new ResolveDataFrameDropColumns(catalogManager) :: ExtractWindowExpressions :: GlobalAggregates :: ResolveAggregateFunctions :: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala new file mode 100644 index 0000000000000..2642b4a1c5daa --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.catalyst.plans.logical.{DataFrameDropColumns, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.DF_DROP_COLUMNS +import org.apache.spark.sql.connector.catalog.CatalogManager + +/** + * A rule that rewrites DataFrameDropColumns to Project. + * Note that DataFrameDropColumns allows and ignores non-existing columns. + */ +class ResolveDataFrameDropColumns(val catalogManager: CatalogManager) + extends Rule[LogicalPlan] with ColumnResolutionHelper { + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning( + _.containsPattern(DF_DROP_COLUMNS)) { + case d: DataFrameDropColumns if d.childrenResolved => + // expressions in dropList can be unresolved, e.g. + // df.drop(col("non-existing-column")) + val dropped = d.dropList.map { + case u: UnresolvedAttribute => + resolveExpressionByPlanChildren(u, d.child) + case e => e + } + val remaining = d.child.output.filterNot(attr => dropped.exists(_.semanticEquals(attr))) + if (remaining.size == d.child.output.size) { + d.child + } else { + Project(remaining, d.child) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 96b67fc52e0d7..0e460706fc5b8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -235,6 +235,20 @@ object Project { } } +case class DataFrameDropColumns(dropList: Seq[Expression], child: LogicalPlan) extends UnaryNode { + override def output: Seq[Attribute] = Nil + + override def maxRows: Option[Long] = child.maxRows + override def maxRowsPerPartition: Option[Long] = child.maxRowsPerPartition + + final override val nodePatterns: Seq[TreePattern] = Seq(DF_DROP_COLUMNS) + + override lazy val resolved: Boolean = false + + override protected def withNewChildInternal(newChild: LogicalPlan): DataFrameDropColumns = + copy(child = newChild) +} + /** * Applies a [[Generator]] to a stream of input rows, combining the * output of each into a new stream of rows. This operation is similar to a `flatMap` in functional diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index b806ebbed52d0..bf7b2db1719f5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -105,6 +105,7 @@ object TreePattern extends Enumeration { val AS_OF_JOIN: Value = Value val COMMAND: Value = Value val CTE: Value = Value + val DF_DROP_COLUMNS: Value = Value val DISTINCT_LIKE: Value = Value val EVAL_PYTHON_UDF: Value = Value val EVAL_PYTHON_UDTF: Value = Value diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index e047b927b9057..f53c6ddaa3880 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -3013,19 +3013,8 @@ class Dataset[T] private[sql]( * @since 3.4.0 */ @scala.annotation.varargs - def drop(col: Column, cols: Column*): DataFrame = { - val allColumns = col +: cols - val expressions = (for (col <- allColumns) yield col match { - case Column(u: UnresolvedAttribute) => - queryExecution.analyzed.resolveQuoted( - u.name, sparkSession.sessionState.analyzer.resolver).getOrElse(u) - case Column(expr: Expression) => expr - }) - val attrs = this.logicalPlan.output - val colsAfterDrop = attrs.filter { attr => - expressions.forall(expression => !attr.semanticEquals(expression)) - }.map(attr => Column(attr)) - select(colsAfterDrop : _*) + def drop(col: Column, cols: Column*): DataFrame = withPlan { + DataFrameDropColumns((col +: cols).map(_.expr), logicalPlan) } /** From 556caeace66bc12ae12ae304ade21fc24c437af9 Mon Sep 17 00:00:00 2001 From: ulysses-you Date: Tue, 14 Nov 2023 19:53:50 +0800 Subject: [PATCH 104/521] [SPARK-45882][SQL][3.5] BroadcastHashJoinExec propagate partitioning should respect CoalescedHashPartitioning This pr backport https://github.com/apache/spark/pull/43753 to branch-3.5 ### What changes were proposed in this pull request? Add HashPartitioningLike trait and make HashPartitioning and CoalescedHashPartitioning extend it. When we propagate output partiitoning, we should handle HashPartitioningLike instead of HashPartitioning. This pr also changes the BroadcastHashJoinExec to use HashPartitioningLike to avoid regression. ### Why are the changes needed? Avoid unnecessary shuffle exchange. ### Does this PR introduce _any_ user-facing change? yes, avoid regression ### How was this patch tested? add test ### Was this patch authored or co-authored using generative AI tooling? no Closes #43792 from ulysses-you/partitioning-3.5. Authored-by: ulysses-you Signed-off-by: youxiduo --- .../plans/physical/partitioning.scala | 46 +++++++++---------- .../joins/BroadcastHashJoinExec.scala | 11 +++-- .../org/apache/spark/sql/JoinSuite.scala | 28 ++++++++++- 3 files changed, 54 insertions(+), 31 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 1eefe65859bdd..211b5a05eb70c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -258,18 +258,8 @@ case object SinglePartition extends Partitioning { SinglePartitionShuffleSpec } -/** - * Represents a partitioning where rows are split up across partitions based on the hash - * of `expressions`. All rows where `expressions` evaluate to the same values are guaranteed to be - * in the same partition. - * - * Since [[StatefulOpClusteredDistribution]] relies on this partitioning and Spark requires - * stateful operators to retain the same physical partitioning during the lifetime of the query - * (including restart), the result of evaluation on `partitionIdExpression` must be unchanged - * across Spark versions. Violation of this requirement may bring silent correctness issue. - */ -case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) - extends Expression with Partitioning with Unevaluable { +trait HashPartitioningLike extends Expression with Partitioning with Unevaluable { + def expressions: Seq[Expression] override def children: Seq[Expression] = expressions override def nullable: Boolean = false @@ -294,6 +284,20 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) } } } +} + +/** + * Represents a partitioning where rows are split up across partitions based on the hash + * of `expressions`. All rows where `expressions` evaluate to the same values are guaranteed to be + * in the same partition. + * + * Since [[StatefulOpClusteredDistribution]] relies on this partitioning and Spark requires + * stateful operators to retain the same physical partitioning during the lifetime of the query + * (including restart), the result of evaluation on `partitionIdExpression` must be unchanged + * across Spark versions. Violation of this requirement may bring silent correctness issue. + */ +case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) + extends HashPartitioningLike { override def createShuffleSpec(distribution: ClusteredDistribution): ShuffleSpec = HashShuffleSpec(this, distribution) @@ -306,7 +310,6 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) override protected def withNewChildrenInternal( newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren) - } case class CoalescedBoundary(startReducerIndex: Int, endReducerIndex: Int) @@ -316,25 +319,18 @@ case class CoalescedBoundary(startReducerIndex: Int, endReducerIndex: Int) * fewer number of partitions. */ case class CoalescedHashPartitioning(from: HashPartitioning, partitions: Seq[CoalescedBoundary]) - extends Expression with Partitioning with Unevaluable { - - override def children: Seq[Expression] = from.expressions - override def nullable: Boolean = from.nullable - override def dataType: DataType = from.dataType + extends HashPartitioningLike { - override def satisfies0(required: Distribution): Boolean = from.satisfies0(required) + override def expressions: Seq[Expression] = from.expressions override def createShuffleSpec(distribution: ClusteredDistribution): ShuffleSpec = CoalescedHashShuffleSpec(from.createShuffleSpec(distribution), partitions) - override protected def withNewChildrenInternal( - newChildren: IndexedSeq[Expression]): CoalescedHashPartitioning = - copy(from = from.copy(expressions = newChildren)) - override val numPartitions: Int = partitions.length - override def toString: String = from.toString - override def sql: String = from.sql + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression]): CoalescedHashPartitioning = + copy(from = from.copy(expressions = newChildren)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala index 9f9f874314639..b82cee2c0fbe7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, HashPartitioning, Partitioning, PartitioningCollection, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, HashPartitioningLike, Partitioning, PartitioningCollection, UnspecifiedDistribution} import org.apache.spark.sql.execution.{CodegenSupport, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics @@ -73,7 +73,7 @@ case class BroadcastHashJoinExec( joinType match { case _: InnerLike if conf.broadcastHashJoinOutputPartitioningExpandLimit > 0 => streamedPlan.outputPartitioning match { - case h: HashPartitioning => expandOutputPartitioning(h) + case h: HashPartitioningLike => expandOutputPartitioning(h) case c: PartitioningCollection => expandOutputPartitioning(c) case other => other } @@ -99,7 +99,7 @@ case class BroadcastHashJoinExec( private def expandOutputPartitioning( partitioning: PartitioningCollection): PartitioningCollection = { PartitioningCollection(partitioning.partitionings.flatMap { - case h: HashPartitioning => expandOutputPartitioning(h).partitionings + case h: HashPartitioningLike => expandOutputPartitioning(h).partitionings case c: PartitioningCollection => Seq(expandOutputPartitioning(c)) case other => Seq(other) }) @@ -111,11 +111,12 @@ case class BroadcastHashJoinExec( // the expanded partitioning will have the following expressions: // Seq("a", "b", "c"), Seq("a", "b", "y"), Seq("a", "x", "c"), Seq("a", "x", "y"). // The expanded expressions are returned as PartitioningCollection. - private def expandOutputPartitioning(partitioning: HashPartitioning): PartitioningCollection = { + private def expandOutputPartitioning( + partitioning: HashPartitioningLike): PartitioningCollection = { PartitioningCollection(partitioning.multiTransformDown { case e: Expression if streamedKeyToBuildKeyMapping.contains(e.canonicalized) => e +: streamedKeyToBuildKeyMapping(e.canonicalized) - }.asInstanceOf[Stream[HashPartitioning]] + }.asInstanceOf[Stream[HashPartitioningLike]] .take(conf.broadcastHashJoinOutputPartitioningExpandLimit)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 14f1fb27906a1..9dcf7ec29048d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.catalyst.plans.logical.{Filter, HintInfo, Join, JoinHint, NO_BROADCAST_AND_REPLICATION} import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, ProjectExec, SortExec, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper -import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.execution.exchange.{ShuffleExchangeExec, ShuffleExchangeLike} import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.python.BatchEvalPythonExec import org.apache.spark.sql.internal.SQLConf @@ -1729,4 +1729,30 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan checkAnswer(joined, expected) } + + test("SPARK-45882: BroadcastHashJoinExec propagate partitioning should respect " + + "CoalescedHashPartitioning") { + val cached = spark.sql( + """ + |select /*+ broadcast(testData) */ key, value, a + |from testData join ( + | select a from testData2 group by a + |)tmp on key = a + |""".stripMargin).cache() + try { + val df = cached.groupBy("key").count() + val expected = Seq(Row(1, 1), Row(2, 1), Row(3, 1)) + assert(find(df.queryExecution.executedPlan) { + case _: ShuffleExchangeLike => true + case _ => false + }.size == 1, df.queryExecution) + checkAnswer(df, expected) + assert(find(df.queryExecution.executedPlan) { + case _: ShuffleExchangeLike => true + case _ => false + }.isEmpty, df.queryExecution) + } finally { + cached.unpersist() + } + } } From 4ca65c69a33da33f66969477bc8a6f88154ed305 Mon Sep 17 00:00:00 2001 From: Maryann Xue Date: Tue, 14 Nov 2023 08:51:26 -0800 Subject: [PATCH 105/521] [SPARK-45592][SPARK-45282][SQL] Correctness issue in AQE with InMemoryTableScanExec ### What changes were proposed in this pull request? This PR fixes an correctness issue while enabling AQE for SQL Cache. This issue was caused by AQE coalescing the top-level shuffle in the physical plan of InMemoryTableScan and wrongfully reported the output partitioning of that InMemoryTableScan as HashPartitioning as if it had not been coalesced. The caller query of that InMemoryTableScan in turn failed to align the partitions correctly and output incorrect join results. The fix addresses the issue by disabling coalescing in InMemoryTableScan for shuffles in the final stage. This fix also guarantees that AQE enabled for SQL cache vs. disabled would always be a performance win, since AQE optimizations are applied to all non-top-level stages and meanwhile no extra shuffle would be introduced between the parent query and the cached relation (if coalescing in top-level shuffles of InMemoryTableScan was not disabled, an extra shuffle would end up being added on top of the cached relation when the cache is used in a join query and the partition key matches the join key in order to avoid the correctness issue). ### Why are the changes needed? To fix correctness issue and to avoid potential AQE perf regressions in queries using SQL cache. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added UTs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43760 from maryannxue/spark-45592. Authored-by: Maryann Xue Signed-off-by: Dongjoon Hyun (cherry picked from commit 128f5523194d5241c7b0f08b5be183288128ba16) Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/internal/SQLConf.scala | 9 ++++ .../spark/sql/execution/CacheManager.scala | 5 +- .../adaptive/AdaptiveSparkPlanExec.scala | 8 ++- .../apache/spark/sql/CachedTableSuite.scala | 52 +++++++++++++------ .../org/apache/spark/sql/DatasetSuite.scala | 33 ++++++++---- 5 files changed, 79 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 4ea0cd5bcc126..70bd21ac1709d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -657,6 +657,15 @@ object SQLConf { .booleanConf .createWithDefault(false) + val ADAPTIVE_EXECUTION_APPLY_FINAL_STAGE_SHUFFLE_OPTIMIZATIONS = + buildConf("spark.sql.adaptive.applyFinalStageShuffleOptimizations") + .internal() + .doc("Configures whether adaptive query execution (if enabled) should apply shuffle " + + "coalescing and local shuffle read optimization for the final query stage.") + .version("3.4.2") + .booleanConf + .createWithDefault(true) + val ADAPTIVE_EXECUTION_LOG_LEVEL = buildConf("spark.sql.adaptive.logLevel") .internal() .doc("Configures the log level for adaptive execution logging of plan changes. The value " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index e906c74f8a5ee..9b79865149abd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -402,8 +402,9 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { if (session.conf.get(SQLConf.CAN_CHANGE_CACHED_PLAN_OUTPUT_PARTITIONING)) { // Bucketed scan only has one time overhead but can have multi-times benefits in cache, // so we always do bucketed scan in a cached plan. - SparkSession.getOrCloneSessionWithConfigsOff( - session, SQLConf.AUTO_BUCKETED_SCAN_ENABLED :: Nil) + SparkSession.getOrCloneSessionWithConfigsOff(session, + SQLConf.ADAPTIVE_EXECUTION_APPLY_FINAL_STAGE_SHUFFLE_OPTIMIZATIONS :: + SQLConf.AUTO_BUCKETED_SCAN_ENABLED :: Nil) } else { SparkSession.getOrCloneSessionWithConfigsOff(session, forceDisableConfigs) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 36895b17aa847..fa671c8faf8b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -159,7 +159,13 @@ case class AdaptiveSparkPlanExec( ) private def optimizeQueryStage(plan: SparkPlan, isFinalStage: Boolean): SparkPlan = { - val optimized = queryStageOptimizerRules.foldLeft(plan) { case (latestPlan, rule) => + val rules = if (isFinalStage && + !conf.getConf(SQLConf.ADAPTIVE_EXECUTION_APPLY_FINAL_STAGE_SHUFFLE_OPTIMIZATIONS)) { + queryStageOptimizerRules.filterNot(_.isInstanceOf[AQEShuffleReadRule]) + } else { + queryStageOptimizerRules + } + val optimized = rules.foldLeft(plan) { case (latestPlan, rule) => val applied = rule.apply(latestPlan) val result = rule match { case _: AQEShuffleReadRule if !applied.fastEquals(latestPlan) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 1e4a67347f5b1..8331a3c10fc97 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -29,7 +29,7 @@ import org.apache.commons.io.FileUtils import org.apache.spark.CleanerListener import org.apache.spark.executor.DataReadMethod._ import org.apache.spark.executor.DataReadMethod.DataReadMethod -import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} +import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent, SparkListenerJobStart} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.TempTableAlreadyExistsException import org.apache.spark.sql.catalyst.expressions.SubqueryExpression @@ -39,6 +39,7 @@ import org.apache.spark.sql.execution.{ColumnarToRowExec, ExecSubqueryExpression import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, AQEPropagateEmptyRelation} import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} @@ -1623,23 +1624,44 @@ class CachedTableSuite extends QueryTest with SQLTestUtils SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { - withTempView("t1", "t2", "t3") { - withSQLConf(SQLConf.CAN_CHANGE_CACHED_PLAN_OUTPUT_PARTITIONING.key -> "false") { - sql("CACHE TABLE t1 as SELECT /*+ REPARTITION */ * FROM values(1) as t(c)") - assert(spark.table("t1").rdd.partitions.length == 2) + var finalPlan = "" + val listener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case SparkListenerSQLAdaptiveExecutionUpdate(_, physicalPlanDesc, sparkPlanInfo) => + if (sparkPlanInfo.simpleString.startsWith( + "AdaptiveSparkPlan isFinalPlan=true")) { + finalPlan = physicalPlanDesc + } + case _ => // ignore other events + } } + } - withSQLConf(SQLConf.CAN_CHANGE_CACHED_PLAN_OUTPUT_PARTITIONING.key -> "true") { - assert(spark.table("t1").rdd.partitions.length == 2) - sql("CACHE TABLE t2 as SELECT /*+ REPARTITION */ * FROM values(2) as t(c)") - assert(spark.table("t2").rdd.partitions.length == 1) - } + withTempView("t0", "t1", "t2") { + try { + spark.range(10).write.saveAsTable("t0") + spark.sparkContext.listenerBus.waitUntilEmpty() + spark.sparkContext.addSparkListener(listener) - withSQLConf(SQLConf.CAN_CHANGE_CACHED_PLAN_OUTPUT_PARTITIONING.key -> "false") { - assert(spark.table("t1").rdd.partitions.length == 2) - assert(spark.table("t2").rdd.partitions.length == 1) - sql("CACHE TABLE t3 as SELECT /*+ REPARTITION */ * FROM values(3) as t(c)") - assert(spark.table("t3").rdd.partitions.length == 2) + withSQLConf(SQLConf.CAN_CHANGE_CACHED_PLAN_OUTPUT_PARTITIONING.key -> "false") { + sql("CACHE TABLE t1 as SELECT /*+ REPARTITION */ * FROM (" + + "SELECT distinct (id+1) FROM t0)") + assert(spark.table("t1").rdd.partitions.length == 2) + spark.sparkContext.listenerBus.waitUntilEmpty() + assert(finalPlan.nonEmpty && !finalPlan.contains("coalesced")) + } + + finalPlan = "" // reset finalPlan + withSQLConf(SQLConf.CAN_CHANGE_CACHED_PLAN_OUTPUT_PARTITIONING.key -> "true") { + sql("CACHE TABLE t2 as SELECT /*+ REPARTITION */ * FROM (" + + "SELECT distinct (id-1) FROM t0)") + assert(spark.table("t2").rdd.partitions.length == 2) + spark.sparkContext.listenerBus.waitUntilEmpty() + assert(finalPlan.nonEmpty && finalPlan.contains("coalesced")) + } + } finally { + spark.sparkContext.removeSparkListener(listener) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 0878ae134e9d4..c2fe31520acf6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -2550,16 +2550,29 @@ class DatasetSuite extends QueryTest } test("SPARK-45592: Coaleasced shuffle read is not compatible with hash partitioning") { - val ee = spark.range(0, 1000000, 1, 5).map(l => (l, l)).toDF() - .persist(org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK) - ee.count() - - val minNbrs1 = ee - .groupBy("_1").agg(min(col("_2")).as("min_number")) - .persist(org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK) - - val join = ee.join(minNbrs1, "_1") - assert(join.count() == 1000000) + withSQLConf(SQLConf.CAN_CHANGE_CACHED_PLAN_OUTPUT_PARTITIONING.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SHUFFLE_PARTITIONS.key -> "20", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "2000") { + val ee = spark.range(0, 1000, 1, 5).map(l => (l, l - 1)).toDF() + .persist(org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK) + ee.count() + + // `minNbrs1` will start with 20 partitions and without the fix would coalesce to ~10 + // partitions. + val minNbrs1 = ee + .groupBy("_2").agg(min(col("_1")).as("min_number")) + .select(col("_2") as "_1", col("min_number")) + .persist(org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK) + minNbrs1.count() + + // shuffle on `ee` will start with 2 partitions, smaller than `minNbrs1`'s partition num, + // and `EnsureRequirements` will change its partition num to `minNbrs1`'s partition num. + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "5") { + val join = ee.join(minNbrs1, "_1") + assert(join.count() == 999) + } + } } } From 41a7a4a3233772003aef380428acd9eaf39b9a93 Mon Sep 17 00:00:00 2001 From: Deepayan Patra Date: Wed, 15 Nov 2023 14:27:34 +0800 Subject: [PATCH 106/521] [SPARK-43393][SQL] Address sequence expression overflow bug Spark has a (long-standing) overflow bug in the `sequence` expression. Consider the following operations: ``` spark.sql("CREATE TABLE foo (l LONG);") spark.sql(s"INSERT INTO foo VALUES (${Long.MaxValue});") spark.sql("SELECT sequence(0, l) FROM foo;").collect() ``` The result of these operations will be: ``` Array[org.apache.spark.sql.Row] = Array([WrappedArray()]) ``` an unintended consequence of overflow. The sequence is applied to values `0` and `Long.MaxValue` with a step size of `1` which uses a length computation defined [here](https://github.com/apache/spark/blob/16411188c7ba6cb19c46a2bd512b2485a4c03e2c/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala#L3451). In this calculation, with `start = 0`, `stop = Long.MaxValue`, and `step = 1`, the calculated `len` overflows to `Long.MinValue`. The computation, in binary looks like: ``` 0111111111111111111111111111111111111111111111111111111111111111 - 0000000000000000000000000000000000000000000000000000000000000000 ------------------------------------------------------------------ 0111111111111111111111111111111111111111111111111111111111111111 / 0000000000000000000000000000000000000000000000000000000000000001 ------------------------------------------------------------------ 0111111111111111111111111111111111111111111111111111111111111111 + 0000000000000000000000000000000000000000000000000000000000000001 ------------------------------------------------------------------ 1000000000000000000000000000000000000000000000000000000000000000 ``` The following [check](https://github.com/apache/spark/blob/16411188c7ba6cb19c46a2bd512b2485a4c03e2c/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala#L3454) passes as the negative `Long.MinValue` is still `<= MAX_ROUNDED_ARRAY_LENGTH`. The following cast to `toInt` uses this representation and [truncates the upper bits](https://github.com/apache/spark/blob/16411188c7ba6cb19c46a2bd512b2485a4c03e2c/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala#L3457) resulting in an empty length of `0`. Other overflows are similarly problematic. This PR addresses the issue by checking numeric operations in the length computation for overflow. There is a correctness bug from overflow in the `sequence` expression. No. Tests added in `CollectionExpressionsSuite.scala`. Closes #41072 from thepinetree/spark-sequence-overflow. Authored-by: Deepayan Patra Signed-off-by: Wenchen Fan (cherry picked from commit afc4c49927cb7f0f2a7f24a42c4fe497796dd9e3) Signed-off-by: Wenchen Fan --- .../expressions/collectionOperations.scala | 48 +++++++++++----- .../CollectionExpressionsSuite.scala | 56 +++++++++++++++++-- 2 files changed, 84 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index ade4a6c5be722..c3c235fba677e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -22,6 +22,8 @@ import java.util.Comparator import scala.collection.mutable import scala.reflect.ClassTag +import org.apache.spark.QueryContext +import org.apache.spark.SparkException.internalError import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion, UnresolvedAttribute, UnresolvedSeed} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch @@ -40,7 +42,6 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SQLOpenHashSet import org.apache.spark.unsafe.UTF8StringBuilder import org.apache.spark.unsafe.array.ByteArrayMethods -import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH import org.apache.spark.unsafe.types.{ByteArray, CalendarInterval, UTF8String} /** @@ -3080,6 +3081,34 @@ case class Sequence( } object Sequence { + private def prettyName: String = "sequence" + + def sequenceLength(start: Long, stop: Long, step: Long): Int = { + try { + val delta = Math.subtractExact(stop, start) + if (delta == Long.MinValue && step == -1L) { + // We must special-case division of Long.MinValue by -1 to catch potential unchecked + // overflow in next operation. Division does not have a builtin overflow check. We + // previously special-case div-by-zero. + throw new ArithmeticException("Long overflow (Long.MinValue / -1)") + } + val len = if (stop == start) 1L else Math.addExact(1L, (delta / step)) + if (len > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { + throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(prettyName, len) + } + len.toInt + } catch { + // We handle overflows in the previous try block by raising an appropriate exception. + case _: ArithmeticException => + val safeLen = + BigInt(1) + (BigInt(stop) - BigInt(start)) / BigInt(step) + if (safeLen > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { + throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(prettyName, safeLen) + } + throw internalError("Unreachable code reached.") + case e: Exception => throw e + } + } private type LessThanOrEqualFn = (Any, Any) => Boolean @@ -3451,13 +3480,7 @@ object Sequence { || (estimatedStep == num.zero && start == stop), s"Illegal sequence boundaries: $start to $stop by $step") - val len = if (start == stop) 1L else 1L + (stop.toLong - start.toLong) / estimatedStep.toLong - - require( - len <= MAX_ROUNDED_ARRAY_LENGTH, - s"Too long sequence: $len. Should be <= $MAX_ROUNDED_ARRAY_LENGTH") - - len.toInt + sequenceLength(start.toLong, stop.toLong, estimatedStep.toLong) } private def genSequenceLengthCode( @@ -3467,7 +3490,7 @@ object Sequence { step: String, estimatedStep: String, len: String): String = { - val longLen = ctx.freshName("longLen") + val calcFn = classOf[Sequence].getName + ".sequenceLength" s""" |if (!(($estimatedStep > 0 && $start <= $stop) || | ($estimatedStep < 0 && $start >= $stop) || @@ -3475,12 +3498,7 @@ object Sequence { | throw new IllegalArgumentException( | "Illegal sequence boundaries: " + $start + " to " + $stop + " by " + $step); |} - |long $longLen = $stop == $start ? 1L : 1L + ((long) $stop - $start) / $estimatedStep; - |if ($longLen > $MAX_ROUNDED_ARRAY_LENGTH) { - | throw new IllegalArgumentException( - | "Too long sequence: " + $longLen + ". Should be <= $MAX_ROUNDED_ARRAY_LENGTH"); - |} - |int $len = (int) $longLen; + |int $len = $calcFn((long) $start, (long) $stop, (long) $estimatedStep); """.stripMargin } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index 1787f6ac72dd4..d001006c58cf1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{outstandingZoneIds, import org.apache.spark.sql.catalyst.util.IntervalUtils._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.types.UTF8String class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -769,10 +769,6 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper // test sequence boundaries checking - checkExceptionInExpression[IllegalArgumentException]( - new Sequence(Literal(Int.MinValue), Literal(Int.MaxValue), Literal(1)), - EmptyRow, s"Too long sequence: 4294967296. Should be <= $MAX_ROUNDED_ARRAY_LENGTH") - checkExceptionInExpression[IllegalArgumentException]( new Sequence(Literal(1), Literal(2), Literal(0)), EmptyRow, "boundaries: 1 to 2 by 0") checkExceptionInExpression[IllegalArgumentException]( @@ -782,6 +778,56 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkExceptionInExpression[IllegalArgumentException]( new Sequence(Literal(1), Literal(2), Literal(-1)), EmptyRow, "boundaries: 1 to 2 by -1") + // SPARK-43393: test Sequence overflow checking + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(Int.MinValue), Literal(Int.MaxValue), Literal(1)), + errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + parameters = Map( + "numberOfElements" -> (BigInt(Int.MaxValue) - BigInt { Int.MinValue } + 1).toString, + "functionName" -> toSQLId("sequence"), + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), + "parameter" -> toSQLId("count"))) + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(0L), Literal(Long.MaxValue), Literal(1L)), + errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + parameters = Map( + "numberOfElements" -> (BigInt(Long.MaxValue) + 1).toString, + "functionName" -> toSQLId("sequence"), + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), + "parameter" -> toSQLId("count"))) + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(0L), Literal(Long.MinValue), Literal(-1L)), + errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + parameters = Map( + "numberOfElements" -> ((0 - BigInt(Long.MinValue)) + 1).toString(), + "functionName" -> toSQLId("sequence"), + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), + "parameter" -> toSQLId("count"))) + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(Long.MinValue), Literal(Long.MaxValue), Literal(1L)), + errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + parameters = Map( + "numberOfElements" -> (BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1).toString, + "functionName" -> toSQLId("sequence"), + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), + "parameter" -> toSQLId("count"))) + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(Long.MaxValue), Literal(Long.MinValue), Literal(-1L)), + errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + parameters = Map( + "numberOfElements" -> (BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1).toString, + "functionName" -> toSQLId("sequence"), + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), + "parameter" -> toSQLId("count"))) + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(Long.MaxValue), Literal(-1L), Literal(-1L)), + errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", + parameters = Map( + "numberOfElements" -> (BigInt(Long.MaxValue) - BigInt { -1L } + 1).toString, + "functionName" -> toSQLId("sequence"), + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), + "parameter" -> toSQLId("count"))) + // test sequence with one element (zero step or equal start and stop) checkEvaluation(new Sequence(Literal(1), Literal(1), Literal(-1)), Seq(1)) From e38310c74e6cae8c8c8489ffcbceb80ed37a7cae Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 15 Nov 2023 09:12:42 -0800 Subject: [PATCH 107/521] Revert "[SPARK-43393][SQL] Address sequence expression overflow bug" This reverts commit 41a7a4a3233772003aef380428acd9eaf39b9a93. --- .../expressions/collectionOperations.scala | 48 +++++----------- .../CollectionExpressionsSuite.scala | 56 ++----------------- 2 files changed, 20 insertions(+), 84 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index c3c235fba677e..ade4a6c5be722 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -22,8 +22,6 @@ import java.util.Comparator import scala.collection.mutable import scala.reflect.ClassTag -import org.apache.spark.QueryContext -import org.apache.spark.SparkException.internalError import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion, UnresolvedAttribute, UnresolvedSeed} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch @@ -42,6 +40,7 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SQLOpenHashSet import org.apache.spark.unsafe.UTF8StringBuilder import org.apache.spark.unsafe.array.ByteArrayMethods +import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH import org.apache.spark.unsafe.types.{ByteArray, CalendarInterval, UTF8String} /** @@ -3081,34 +3080,6 @@ case class Sequence( } object Sequence { - private def prettyName: String = "sequence" - - def sequenceLength(start: Long, stop: Long, step: Long): Int = { - try { - val delta = Math.subtractExact(stop, start) - if (delta == Long.MinValue && step == -1L) { - // We must special-case division of Long.MinValue by -1 to catch potential unchecked - // overflow in next operation. Division does not have a builtin overflow check. We - // previously special-case div-by-zero. - throw new ArithmeticException("Long overflow (Long.MinValue / -1)") - } - val len = if (stop == start) 1L else Math.addExact(1L, (delta / step)) - if (len > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { - throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(prettyName, len) - } - len.toInt - } catch { - // We handle overflows in the previous try block by raising an appropriate exception. - case _: ArithmeticException => - val safeLen = - BigInt(1) + (BigInt(stop) - BigInt(start)) / BigInt(step) - if (safeLen > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { - throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(prettyName, safeLen) - } - throw internalError("Unreachable code reached.") - case e: Exception => throw e - } - } private type LessThanOrEqualFn = (Any, Any) => Boolean @@ -3480,7 +3451,13 @@ object Sequence { || (estimatedStep == num.zero && start == stop), s"Illegal sequence boundaries: $start to $stop by $step") - sequenceLength(start.toLong, stop.toLong, estimatedStep.toLong) + val len = if (start == stop) 1L else 1L + (stop.toLong - start.toLong) / estimatedStep.toLong + + require( + len <= MAX_ROUNDED_ARRAY_LENGTH, + s"Too long sequence: $len. Should be <= $MAX_ROUNDED_ARRAY_LENGTH") + + len.toInt } private def genSequenceLengthCode( @@ -3490,7 +3467,7 @@ object Sequence { step: String, estimatedStep: String, len: String): String = { - val calcFn = classOf[Sequence].getName + ".sequenceLength" + val longLen = ctx.freshName("longLen") s""" |if (!(($estimatedStep > 0 && $start <= $stop) || | ($estimatedStep < 0 && $start >= $stop) || @@ -3498,7 +3475,12 @@ object Sequence { | throw new IllegalArgumentException( | "Illegal sequence boundaries: " + $start + " to " + $stop + " by " + $step); |} - |int $len = $calcFn((long) $start, (long) $stop, (long) $estimatedStep); + |long $longLen = $stop == $start ? 1L : 1L + ((long) $stop - $start) / $estimatedStep; + |if ($longLen > $MAX_ROUNDED_ARRAY_LENGTH) { + | throw new IllegalArgumentException( + | "Too long sequence: " + $longLen + ". Should be <= $MAX_ROUNDED_ARRAY_LENGTH"); + |} + |int $len = (int) $longLen; """.stripMargin } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index d001006c58cf1..1787f6ac72dd4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{outstandingZoneIds, import org.apache.spark.sql.catalyst.util.IntervalUtils._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.array.ByteArrayMethods +import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH import org.apache.spark.unsafe.types.UTF8String class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -769,6 +769,10 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper // test sequence boundaries checking + checkExceptionInExpression[IllegalArgumentException]( + new Sequence(Literal(Int.MinValue), Literal(Int.MaxValue), Literal(1)), + EmptyRow, s"Too long sequence: 4294967296. Should be <= $MAX_ROUNDED_ARRAY_LENGTH") + checkExceptionInExpression[IllegalArgumentException]( new Sequence(Literal(1), Literal(2), Literal(0)), EmptyRow, "boundaries: 1 to 2 by 0") checkExceptionInExpression[IllegalArgumentException]( @@ -778,56 +782,6 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkExceptionInExpression[IllegalArgumentException]( new Sequence(Literal(1), Literal(2), Literal(-1)), EmptyRow, "boundaries: 1 to 2 by -1") - // SPARK-43393: test Sequence overflow checking - checkErrorInExpression[SparkRuntimeException]( - new Sequence(Literal(Int.MinValue), Literal(Int.MaxValue), Literal(1)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", - parameters = Map( - "numberOfElements" -> (BigInt(Int.MaxValue) - BigInt { Int.MinValue } + 1).toString, - "functionName" -> toSQLId("sequence"), - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), - "parameter" -> toSQLId("count"))) - checkErrorInExpression[SparkRuntimeException]( - new Sequence(Literal(0L), Literal(Long.MaxValue), Literal(1L)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", - parameters = Map( - "numberOfElements" -> (BigInt(Long.MaxValue) + 1).toString, - "functionName" -> toSQLId("sequence"), - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), - "parameter" -> toSQLId("count"))) - checkErrorInExpression[SparkRuntimeException]( - new Sequence(Literal(0L), Literal(Long.MinValue), Literal(-1L)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", - parameters = Map( - "numberOfElements" -> ((0 - BigInt(Long.MinValue)) + 1).toString(), - "functionName" -> toSQLId("sequence"), - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), - "parameter" -> toSQLId("count"))) - checkErrorInExpression[SparkRuntimeException]( - new Sequence(Literal(Long.MinValue), Literal(Long.MaxValue), Literal(1L)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", - parameters = Map( - "numberOfElements" -> (BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1).toString, - "functionName" -> toSQLId("sequence"), - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), - "parameter" -> toSQLId("count"))) - checkErrorInExpression[SparkRuntimeException]( - new Sequence(Literal(Long.MaxValue), Literal(Long.MinValue), Literal(-1L)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", - parameters = Map( - "numberOfElements" -> (BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1).toString, - "functionName" -> toSQLId("sequence"), - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), - "parameter" -> toSQLId("count"))) - checkErrorInExpression[SparkRuntimeException]( - new Sequence(Literal(Long.MaxValue), Literal(-1L), Literal(-1L)), - errorClass = "COLLECTION_SIZE_LIMIT_EXCEEDED.PARAMETER", - parameters = Map( - "numberOfElements" -> (BigInt(Long.MaxValue) - BigInt { -1L } + 1).toString, - "functionName" -> toSQLId("sequence"), - "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString(), - "parameter" -> toSQLId("count"))) - // test sequence with one element (zero step or equal start and stop) checkEvaluation(new Sequence(Literal(1), Literal(1), Literal(-1)), Seq(1)) From a9f95e8203bede86462e681bb7a3e6123b8c00a2 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 15 Nov 2023 14:12:36 -0800 Subject: [PATCH 108/521] [SPARK-45934][DOCS] Fix `Spark Standalone` documentation table layout MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR fixes `Spark Standalone` documentation table layout. **BEFORE** - https://spark.apache.org/docs/3.5.0/spark-standalone.html **AFTER** - Spark Standalone Screenshot 2023-11-15 at 2 40 59 AM No. Manual review. No. Closes #43814 from dongjoon-hyun/SPARK-45934. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit e8c2a590f99d8c87968c79960e6b69191f28b420) Signed-off-by: Dongjoon Hyun --- docs/spark-standalone.md | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 3e87edad0aadd..ebda8d897eae9 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -254,7 +254,7 @@ SPARK_MASTER_OPTS supports the following system properties: 0.6.2 - spark.worker.resource.{resourceName}.amount + spark.worker.resource.{name}.amount (none) Amount of a particular resource to use on the worker. @@ -262,7 +262,7 @@ SPARK_MASTER_OPTS supports the following system properties: 3.0.0 - spark.worker.resource.{resourceName}.discoveryScript + spark.worker.resource.{name}.discoveryScript (none) Path to resource discovery script, which is used to find a particular resource while worker starting up. @@ -275,8 +275,10 @@ SPARK_MASTER_OPTS supports the following system properties: (none) Path to resources file which is used to find various resources while worker starting up. - The content of resources file should be formatted like - [{"id":{"componentName": "spark.worker","resourceName":"gpu"},"addresses":["0","1","2"]}]. + The content of resources file should be formatted like + [{"id":{"componentName": + "spark.worker", "resourceName":"gpu"}, + "addresses":["0","1","2"]}]. If a particular resource is not found in the resources file, the discovery script would be used to find that resource. If the discovery script also does not find the resources, the worker will fail to start up. From 44bd909ef9e6f4d5419b5757a265fa9ead001cbb Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 16 Nov 2023 00:52:48 -0800 Subject: [PATCH 109/521] [SPARK-45764][PYTHON][DOCS][3.5] Make code block copyable ### What changes were proposed in this pull request? The pr aims to make code block `copyable `in pyspark docs. Backport above to `branch 3.5`. Master branch pr: https://github.com/apache/spark/pull/43799 ### Why are the changes needed? Improving the usability of PySpark documents. ### Does this PR introduce _any_ user-facing change? Yes, users will be able to easily copy code block in pyspark docs. ### How was this patch tested? - Manually test. - Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43827 from panbingkun/branch-3.5_SPARK-45764. Authored-by: panbingkun Signed-off-by: Dongjoon Hyun --- .github/workflows/build_and_test.yml | 2 +- LICENSE | 5 -- dev/create-release/spark-rm/Dockerfile | 2 +- dev/requirements.txt | 1 + licenses/LICENSE-copybutton.txt | 49 ----------------- python/docs/source/_static/copybutton.js | 67 ------------------------ python/docs/source/conf.py | 7 +-- 7 files changed, 7 insertions(+), 126 deletions(-) delete mode 100644 licenses/LICENSE-copybutton.txt delete mode 100644 python/docs/source/_static/copybutton.js diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 674e59508510c..f202a7d49c9a2 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -678,7 +678,7 @@ jobs: # See also https://issues.apache.org/jira/browse/SPARK-35375. # Pin the MarkupSafe to 2.0.1 to resolve the CI error. # See also https://issues.apache.org/jira/browse/SPARK-38279. - python3.9 -m pip install 'sphinx<3.1.0' mkdocs pydata_sphinx_theme nbsphinx numpydoc 'jinja2<3.0.0' 'markupsafe==2.0.1' 'pyzmq<24.0.0' + python3.9 -m pip install 'sphinx<3.1.0' mkdocs pydata_sphinx_theme 'sphinx-copybutton==0.5.2' nbsphinx numpydoc 'jinja2<3.0.0' 'markupsafe==2.0.1' 'pyzmq<24.0.0' python3.9 -m pip install ipython_genutils # See SPARK-38517 python3.9 -m pip install sphinx_plotly_directive 'numpy>=1.20.0' pyarrow pandas 'plotly>=4.8' python3.9 -m pip install 'docutils<0.18.0' # See SPARK-39421 diff --git a/LICENSE b/LICENSE index 1735d3208f2e2..74686d7ffa388 100644 --- a/LICENSE +++ b/LICENSE @@ -218,11 +218,6 @@ docs/js/vendor/bootstrap.js connector/spark-ganglia-lgpl/src/main/java/com/codahale/metrics/ganglia/GangliaReporter.java -Python Software Foundation License ----------------------------------- - -python/docs/source/_static/copybutton.js - BSD 3-Clause ------------ diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index 85155b67bd5a3..cd57226f5e017 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -42,7 +42,7 @@ ARG APT_INSTALL="apt-get install --no-install-recommends -y" # We should use the latest Sphinx version once this is fixed. # TODO(SPARK-35375): Jinja2 3.0.0+ causes error when building with Sphinx. # See also https://issues.apache.org/jira/browse/SPARK-35375. -ARG PIP_PKGS="sphinx==3.0.4 mkdocs==1.1.2 numpy==1.20.3 pydata_sphinx_theme==0.8.0 ipython==7.19.0 nbsphinx==0.8.0 numpydoc==1.1.0 jinja2==2.11.3 twine==3.4.1 sphinx-plotly-directive==0.1.3 pandas==1.5.3 pyarrow==3.0.0 plotly==5.4.0 markupsafe==2.0.1 docutils<0.17 grpcio==1.56.0 protobuf==4.21.6 grpcio-status==1.56.0 googleapis-common-protos==1.56.4" +ARG PIP_PKGS="sphinx==3.0.4 mkdocs==1.1.2 numpy==1.20.3 pydata_sphinx_theme==0.8.0 ipython==7.19.0 nbsphinx==0.8.0 numpydoc==1.1.0 jinja2==2.11.3 twine==3.4.1 sphinx-plotly-directive==0.1.3 sphinx-copybutton==0.5.2 pandas==1.5.3 pyarrow==3.0.0 plotly==5.4.0 markupsafe==2.0.1 docutils<0.17 grpcio==1.56.0 protobuf==4.21.6 grpcio-status==1.56.0 googleapis-common-protos==1.56.4" ARG GEM_PKGS="bundler:2.3.8" # Install extra needed repos and refresh. diff --git a/dev/requirements.txt b/dev/requirements.txt index 38a9b2447108c..597417aba1f3d 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -37,6 +37,7 @@ numpydoc jinja2<3.0.0 sphinx<3.1.0 sphinx-plotly-directive +sphinx-copybutton<0.5.3 docutils<0.18.0 # See SPARK-38279. markupsafe==2.0.1 diff --git a/licenses/LICENSE-copybutton.txt b/licenses/LICENSE-copybutton.txt deleted file mode 100644 index 45be6b83a53be..0000000000000 --- a/licenses/LICENSE-copybutton.txt +++ /dev/null @@ -1,49 +0,0 @@ -PYTHON SOFTWARE FOUNDATION LICENSE VERSION 2 --------------------------------------------- - -1. This LICENSE AGREEMENT is between the Python Software Foundation -("PSF"), and the Individual or Organization ("Licensee") accessing and -otherwise using this software ("Python") in source or binary form and -its associated documentation. - -2. Subject to the terms and conditions of this License Agreement, PSF hereby -grants Licensee a nonexclusive, royalty-free, world-wide license to reproduce, -analyze, test, perform and/or display publicly, prepare derivative works, -distribute, and otherwise use Python alone or in any derivative version, -provided, however, that PSF's License Agreement and PSF's notice of copyright, -i.e., "Copyright (c) 2001, 2002, 2003, 2004, 2005, 2006, 2007, 2008, 2009, 2010, -2011, 2012, 2013, 2014, 2015, 2016, 2017, 2018, 2019 Python Software Foundation; -All Rights Reserved" are retained in Python alone or in any derivative version -prepared by Licensee. - -3. In the event Licensee prepares a derivative work that is based on -or incorporates Python or any part thereof, and wants to make -the derivative work available to others as provided herein, then -Licensee hereby agrees to include in any such work a brief summary of -the changes made to Python. - -4. PSF is making Python available to Licensee on an "AS IS" -basis. PSF MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR -IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, PSF MAKES NO AND -DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS -FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON WILL NOT -INFRINGE ANY THIRD PARTY RIGHTS. - -5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON -FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS -A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON, -OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. - -6. This License Agreement will automatically terminate upon a material -breach of its terms and conditions. - -7. Nothing in this License Agreement shall be deemed to create any -relationship of agency, partnership, or joint venture between PSF and -Licensee. This License Agreement does not grant permission to use PSF -trademarks or trade name in a trademark sense to endorse or promote -products or services of Licensee, or any third party. - -8. By copying, installing or otherwise using Python, Licensee -agrees to be bound by the terms and conditions of this License -Agreement. - diff --git a/python/docs/source/_static/copybutton.js b/python/docs/source/_static/copybutton.js deleted file mode 100644 index 896faad3f9df1..0000000000000 --- a/python/docs/source/_static/copybutton.js +++ /dev/null @@ -1,67 +0,0 @@ -// Copyright 2014 PSF. Licensed under the PYTHON SOFTWARE FOUNDATION LICENSE VERSION 2 -// File originates from the cpython source found in Doc/tools/sphinxext/static/copybutton.js - -$(document).ready(function() { - /* Add a [>>>] button on the top-right corner of code samples to hide - * the >>> and ... prompts and the output and thus make the code - * copyable. */ - var div = $('.highlight-python .highlight,' + - '.highlight-default .highlight,' + - '.highlight-python3 .highlight') - var pre = div.find('pre'); - - // get the styles from the current theme - pre.parent().parent().css('position', 'relative'); - var hide_text = 'Hide the prompts and output'; - var show_text = 'Show the prompts and output'; - var border_width = pre.css('border-top-width'); - var border_style = pre.css('border-top-style'); - var border_color = pre.css('border-top-color'); - var button_styles = { - 'cursor':'pointer', 'position': 'absolute', 'top': '0', 'right': '0', - 'border-color': border_color, 'border-style': border_style, - 'border-width': border_width, 'color': border_color, 'text-size': '75%', - 'font-family': 'monospace', 'padding-left': '0.2em', 'padding-right': '0.2em', - 'border-radius': '0 3px 0 0', - 'user-select': 'none' - } - - // create and add the button to all the code blocks that contain >>> - div.each(function(index) { - var jthis = $(this); - if (jthis.find('.gp').length > 0) { - var button = $('>>>'); - button.css(button_styles) - button.attr('title', hide_text); - button.data('hidden', 'false'); - jthis.prepend(button); - } - // tracebacks (.gt) contain bare text elements that need to be - // wrapped in a span to work with .nextUntil() (see later) - jthis.find('pre:has(.gt)').contents().filter(function() { - return ((this.nodeType == 3) && (this.data.trim().length > 0)); - }).wrap(''); - }); - - // define the behavior of the button when it's clicked - $('.copybutton').click(function(e){ - e.preventDefault(); - var button = $(this); - if (button.data('hidden') === 'false') { - // hide the code output - button.parent().find('.go, .gp, .gt').hide(); - button.next('pre').find('.gt').nextUntil('.gp, .go').css('visibility', 'hidden'); - button.css('text-decoration', 'line-through'); - button.attr('title', show_text); - button.data('hidden', 'true'); - } else { - // show the code output - button.parent().find('.go, .gp, .gt').show(); - button.next('pre').find('.gt').nextUntil('.gp, .go').css('visibility', 'visible'); - button.css('text-decoration', 'none'); - button.attr('title', hide_text); - button.data('hidden', 'false'); - } - }); -}); - diff --git a/python/docs/source/conf.py b/python/docs/source/conf.py index 0f57cb37ceeb1..a0d087de176ff 100644 --- a/python/docs/source/conf.py +++ b/python/docs/source/conf.py @@ -63,6 +63,7 @@ 'sphinx.ext.viewcode', 'sphinx.ext.mathjax', 'sphinx.ext.autosummary', + 'sphinx_copybutton', 'nbsphinx', # Converts Jupyter Notebook to reStructuredText files for Sphinx. # For ipython directive in reStructuredText files. It is generated by the notebook. 'IPython.sphinxext.ipython_console_highlighting', @@ -70,6 +71,9 @@ 'sphinx_plotly_directive', # For visualize plot result ] +# sphinx copy button +copybutton_exclude = '.linenos, .gp, .go' + # plotly plot directive plotly_include_source = True plotly_html_show_formats = False @@ -416,9 +420,6 @@ # If false, no index is generated. #epub_use_index = True -def setup(app): - # The app.add_javascript() is deprecated. - getattr(app, "add_js_file", getattr(app, "add_javascript", None))('copybutton.js') # Skip sample endpoint link (not expected to resolve) linkcheck_ignore = [r'https://kinesis.us-east-1.amazonaws.com'] From b962cb26ed20d695e408958be452f0a947e7e989 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 16 Nov 2023 18:00:56 +0900 Subject: [PATCH 110/521] [SPARK-45935][PYTHON][DOCS] Fix RST files link substitutions error ### What changes were proposed in this pull request? The pr aims to fix RST files `link substitutions` error. Target branch: branch-3.3, branch-3.4, branch-3.5, master. ### Why are the changes needed? When I was reviewing Python documents, I found that `the actual address` of the link was incorrect, eg: https://spark.apache.org/docs/latest/api/python/getting_started/install.html#installing-from-source image `The ref link url` of `Building Spark`: from `https://spark.apache.org/docs/3.5.0/#downloading` to `https://spark.apache.org/docs/3.5.0/building-spark.html`. We should fix it. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43815 from panbingkun/SPARK-45935. Authored-by: panbingkun Signed-off-by: Hyukjin Kwon (cherry picked from commit 79ccdfa31e282ebe9a82c8f20c703b6ad2ea6bc1) Signed-off-by: Hyukjin Kwon --- python/docs/source/conf.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/docs/source/conf.py b/python/docs/source/conf.py index a0d087de176ff..08a25c5dd0712 100644 --- a/python/docs/source/conf.py +++ b/python/docs/source/conf.py @@ -98,9 +98,9 @@ .. |examples| replace:: Examples .. _examples: https://github.com/apache/spark/tree/{0}/examples/src/main/python .. |downloading| replace:: Downloading -.. _downloading: https://spark.apache.org/docs/{1}/building-spark.html +.. _downloading: https://spark.apache.org/docs/{1}/#downloading .. |building_spark| replace:: Building Spark -.. _building_spark: https://spark.apache.org/docs/{1}/#downloading +.. _building_spark: https://spark.apache.org/docs/{1}/building-spark.html """.format( os.environ.get("GIT_HASH", "master"), os.environ.get("RELEASE_VERSION", "latest"), From f0054c5a10bf388688e7b2914cb639c96ffdd8f3 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 16 Nov 2023 08:16:20 -0800 Subject: [PATCH 111/521] [SPARK-45920][SQL][3.5] group by ordinal should be idempotent backport https://github.com/apache/spark/pull/43797 ### What changes were proposed in this pull request? GROUP BY ordinal is not idempotent today. If the ordinal points to another integer literal and the plan get analyzed again, we will re-do the ordinal resolution which can lead to wrong result or index out-of-bound error. This PR fixes it by using a hack: if the ordinal points to another integer literal, don't replace the ordinal. ### Why are the changes needed? For advanced users or Spark plugins, they may manipulate the logical plans directly. We need to make the framework more reliable. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? new test ### Was this patch authored or co-authored using generative AI tooling? no Closes #43836 from cloud-fan/3.5-port. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/analysis/Analyzer.scala | 14 ++++++++++- .../SubstituteUnresolvedOrdinalsSuite.scala | 23 +++++++++++++++++-- .../analyzer-results/group-by-ordinal.sql.out | 2 +- 3 files changed, 35 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 80cb5d8c60876..02b9c2445433b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1970,7 +1970,19 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor throw QueryCompilationErrors.groupByPositionRefersToAggregateFunctionError( index, ordinalExpr) } else { - ordinalExpr + trimAliases(ordinalExpr) match { + // HACK ALERT: If the ordinal expression is also an integer literal, don't use it + // but still keep the ordinal literal. The reason is we may repeatedly + // analyze the plan. Using a different integer literal may lead to + // a repeat GROUP BY ordinal resolution which is wrong. GROUP BY + // constant is meaningless so whatever value does not matter here. + // TODO: (SPARK-45932) GROUP BY ordinal should pull out grouping expressions to + // a Project, then the resolved ordinal expression is always + // `AttributeReference`. + case Literal(_: Int, IntegerType) => + Literal(index) + case _ => ordinalExpr + } } } else { throw QueryCompilationErrors.groupByPositionRangeError(index, aggs.size) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinalsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinalsSuite.scala index b0d7ace646e2e..953b2c8bb1011 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinalsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinalsSuite.scala @@ -17,10 +17,11 @@ package org.apache.spark.sql.catalyst.analysis -import org.apache.spark.sql.catalyst.analysis.TestRelations.testRelation2 +import org.apache.spark.sql.catalyst.analysis.TestRelations.{testRelation, testRelation2} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, Literal} +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.internal.SQLConf class SubstituteUnresolvedOrdinalsSuite extends AnalysisTest { @@ -67,4 +68,22 @@ class SubstituteUnresolvedOrdinalsSuite extends AnalysisTest { testRelation2.groupBy(Literal(1), Literal(2))($"a", $"b")) } } + + test("SPARK-45920: group by ordinal repeated analysis") { + val plan = testRelation.groupBy(Literal(1))(Literal(100).as("a")).analyze + comparePlans( + plan, + testRelation.groupBy(Literal(1))(Literal(100).as("a")) + ) + + val testRelationWithData = testRelation.copy(data = Seq(new GenericInternalRow(Array(1: Any)))) + // Copy the plan to reset its `analyzed` flag, so that analyzer rules will re-apply. + val copiedPlan = plan.transform { + case _: LocalRelation => testRelationWithData + } + comparePlans( + copiedPlan.analyze, // repeated analysis + testRelationWithData.groupBy(Literal(1))(Literal(100).as("a")) + ) + } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-ordinal.sql.out index c8c34a856d492..1bcde5bd367f7 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-ordinal.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-ordinal.sql.out @@ -61,7 +61,7 @@ Aggregate [a#x, a#x], [a#x, 1 AS 1#x, sum(b#x) AS sum(b)#xL] -- !query select a, 1, sum(b) from data group by 1, 2 -- !query analysis -Aggregate [a#x, 1], [a#x, 1 AS 1#x, sum(b#x) AS sum(b)#xL] +Aggregate [a#x, 2], [a#x, 1 AS 1#x, sum(b#x) AS sum(b)#xL] +- SubqueryAlias data +- View (`data`, [a#x,b#x]) +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x] From e3549b253644749a373daf11108825ad38b3f055 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 16 Nov 2023 15:36:59 -0800 Subject: [PATCH 112/521] [SPARK-45961][DOCS][3.5] Document `spark.master.*` configurations MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR documents `spark.master.*` configurations. ### Why are the changes needed? Currently, `spark.master.*` configurations are undocumented. ``` $ git grep 'ConfigBuilder("spark.master' core/src/main/scala/org/apache/spark/internal/config/UI.scala: val MASTER_UI_DECOMMISSION_ALLOW_MODE = ConfigBuilder("spark.master.ui.decommission.allow.mode") core/src/main/scala/org/apache/spark/internal/config/package.scala: private[spark] val MASTER_REST_SERVER_ENABLED = ConfigBuilder("spark.master.rest.enabled") core/src/main/scala/org/apache/spark/internal/config/package.scala: private[spark] val MASTER_REST_SERVER_PORT = ConfigBuilder("spark.master.rest.port") core/src/main/scala/org/apache/spark/internal/config/package.scala: private[spark] val MASTER_UI_PORT = ConfigBuilder("spark.master.ui.port") ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual review. ![Screenshot 2023-11-16 at 2 55 09 PM](https://github.com/apache/spark/assets/9700541/da096ad6-0dec-4cda-90dd-ecf376988ac8) ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43849 from dongjoon-hyun/SPARK-45961-3.5. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- docs/spark-standalone.md | 35 +++++++++++++++++++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index ebda8d897eae9..9152547f1bec4 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -190,6 +190,41 @@ SPARK_MASTER_OPTS supports the following system properties: + + + + + + + + + + + + + + + + + + + + + + + + From 01eb6c83324be76dc30b0857aab9e126fe2ea25d Mon Sep 17 00:00:00 2001 From: Kazuyuki Tanimura Date: Fri, 17 Nov 2023 02:49:51 -0800 Subject: [PATCH 113/521] [SPARK-45786][SQL][FOLLOWUP][TEST] Fix Decimal random number tests with ANSI enabled ### What changes were proposed in this pull request? This follow-up PR fixes the test for SPARK-45786 that is failing in GHA with SPARK_ANSI_SQL_MODE=true ### Why are the changes needed? The issue discovered in https://github.com/apache/spark/pull/43678#discussion_r1395693417 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Test updated ### Was this patch authored or co-authored using generative AI tooling? No Closes #43853 from kazuyukitanimura/SPARK-45786-FollowUp. Authored-by: Kazuyuki Tanimura Signed-off-by: Dongjoon Hyun (cherry picked from commit 949de3416a8ef5b7faa22149f5e07d8235237f40) Signed-off-by: Dongjoon Hyun --- .../expressions/ArithmeticExpressionSuite.scala | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala index 568dcd10d1166..2dc7e82f77226 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala @@ -308,27 +308,35 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val mulResult = Decimal(mulExact.setScale(mulType.scale, RoundingMode.HALF_UP)) val mulExpected = if (mulResult.precision > DecimalType.MAX_PRECISION) null else mulResult - checkEvaluation(mulActual, mulExpected) + checkEvaluationOrException(mulActual, mulExpected) val divType = Divide(null, null).resultDecimalType(p1, s1, p2, s2) val divResult = Decimal(divExact.setScale(divType.scale, RoundingMode.HALF_UP)) val divExpected = if (divResult.precision > DecimalType.MAX_PRECISION) null else divResult - checkEvaluation(divActual, divExpected) + checkEvaluationOrException(divActual, divExpected) val remType = Remainder(null, null).resultDecimalType(p1, s1, p2, s2) val remResult = Decimal(remExact.setScale(remType.scale, RoundingMode.HALF_UP)) val remExpected = if (remResult.precision > DecimalType.MAX_PRECISION) null else remResult - checkEvaluation(remActual, remExpected) + checkEvaluationOrException(remActual, remExpected) val quotType = IntegralDivide(null, null).resultDecimalType(p1, s1, p2, s2) val quotResult = Decimal(quotExact.setScale(quotType.scale, RoundingMode.HALF_UP)) val quotExpected = if (quotResult.precision > DecimalType.MAX_PRECISION) null else quotResult - checkEvaluation(quotActual, quotExpected.toLong) + checkEvaluationOrException(quotActual, quotExpected.toLong) } } + + def checkEvaluationOrException(actual: BinaryArithmetic, expected: Any): Unit = + if (SQLConf.get.ansiEnabled && expected == null) { + checkExceptionInExpression[SparkArithmeticException](actual, + "NUMERIC_VALUE_OUT_OF_RANGE") + } else { + checkEvaluation(actual, expected) + } } } From 9e492b71c4aaa070bc36bfae120e1c6ca05e4a7a Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Fri, 17 Nov 2023 13:04:19 -0800 Subject: [PATCH 114/521] [SPARK-45963][SQL][DOCS][3.5] Restore documentation for DSv2 API This PR cherry-picks https://github.com/apache/spark/pull/43855 to branch-3.5 --- ### What changes were proposed in this pull request? This PR restores the DSv2 documentation. https://github.com/apache/spark/pull/38392 mistakenly added `org/apache/spark/sql/connect` as a private that includes `org/apache/spark/sql/connector`. ### Why are the changes needed? For end users to read DSv2 documentation. ### Does this PR introduce _any_ user-facing change? Yes, it restores the DSv2 API documentation that used to be there https://spark.apache.org/docs/3.3.0/api/scala/org/apache/spark/sql/connector/catalog/index.html ### How was this patch tested? Manually tested via: ``` SKIP_PYTHONDOC=1 SKIP_RDOC=1 SKIP_SQLDOC=1 bundle exec jekyll build ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43865 from HyukjinKwon/SPARK-45963-3.5. Authored-by: Hyukjin Kwon Signed-off-by: Dongjoon Hyun --- project/SparkBuild.scala | 2 +- .../spark/sql/connector/catalog/SupportsMetadataColumns.java | 4 ++-- .../spark/sql/connector/catalog/InMemoryBaseTable.scala | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 718f2bb28cec4..60d52368de458 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1401,7 +1401,7 @@ object Unidoc { .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/util/io"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/util/kvstore"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/catalyst"))) - .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/connect"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/connect/"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/execution"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/internal"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/hive"))) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsMetadataColumns.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsMetadataColumns.java index 894184dbcc82d..e42424268b44d 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsMetadataColumns.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsMetadataColumns.java @@ -58,8 +58,8 @@ public interface SupportsMetadataColumns extends Table { * Determines how this data source handles name conflicts between metadata and data columns. *

* If true, spark will automatically rename the metadata column to resolve the conflict. End users - * can reliably select metadata columns (renamed or not) with {@link Dataset.metadataColumn}, and - * internal code can use {@link MetadataAttributeWithLogicalName} to extract the logical name from + * can reliably select metadata columns (renamed or not) with {@code Dataset.metadataColumn}, and + * internal code can use {@code MetadataAttributeWithLogicalName} to extract the logical name from * a metadata attribute. *

* If false, the data column will hide the metadata column. It is recommended that Table diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala index a0a4d8bdee9f5..a309db341d8e6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala @@ -619,9 +619,9 @@ class BufferedRows(val key: Seq[Any] = Seq.empty) extends WriterCommitMessage } /** - * Theoretically, [[InternalRow]] returned by [[HasPartitionKey#partitionKey()]] + * Theoretically, `InternalRow` returned by `HasPartitionKey#partitionKey()` * does not need to implement equal and hashcode methods. - * But [[GenericInternalRow]] implements equals and hashcode methods already. Here we override it + * But `GenericInternalRow` implements equals and hashcode methods already. Here we override it * to simulate that it has not been implemented to verify codes correctness. */ case class PartitionInternalRow(keys: Array[Any]) From f3baf086acdf166445aef81181d13d4884d44e92 Mon Sep 17 00:00:00 2001 From: Deepayan Patra Date: Fri, 17 Nov 2023 13:17:43 -0800 Subject: [PATCH 115/521] [SPARK-43393][SQL][3.5] Address sequence expression overflow bug ### What changes were proposed in this pull request? Spark has a (long-standing) overflow bug in the `sequence` expression. Consider the following operations: ``` spark.sql("CREATE TABLE foo (l LONG);") spark.sql(s"INSERT INTO foo VALUES (${Long.MaxValue});") spark.sql("SELECT sequence(0, l) FROM foo;").collect() ``` The result of these operations will be: ``` Array[org.apache.spark.sql.Row] = Array([WrappedArray()]) ``` an unintended consequence of overflow. The sequence is applied to values `0` and `Long.MaxValue` with a step size of `1` which uses a length computation defined [here](https://github.com/apache/spark/blob/16411188c7ba6cb19c46a2bd512b2485a4c03e2c/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala#L3451). In this calculation, with `start = 0`, `stop = Long.MaxValue`, and `step = 1`, the calculated `len` overflows to `Long.MinValue`. The computation, in binary looks like: ``` 0111111111111111111111111111111111111111111111111111111111111111 - 0000000000000000000000000000000000000000000000000000000000000000 ------------------------------------------------------------------ 0111111111111111111111111111111111111111111111111111111111111111 / 0000000000000000000000000000000000000000000000000000000000000001 ------------------------------------------------------------------ 0111111111111111111111111111111111111111111111111111111111111111 + 0000000000000000000000000000000000000000000000000000000000000001 ------------------------------------------------------------------ 1000000000000000000000000000000000000000000000000000000000000000 ``` The following [check](https://github.com/apache/spark/blob/16411188c7ba6cb19c46a2bd512b2485a4c03e2c/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala#L3454) passes as the negative `Long.MinValue` is still `<= MAX_ROUNDED_ARRAY_LENGTH`. The following cast to `toInt` uses this representation and [truncates the upper bits](https://github.com/apache/spark/blob/16411188c7ba6cb19c46a2bd512b2485a4c03e2c/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala#L3457) resulting in an empty length of `0`. Other overflows are similarly problematic. This PR addresses the issue by checking numeric operations in the length computation for overflow. ### Why are the changes needed? There is a correctness bug from overflow in the `sequence` expression. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Tests added in `CollectionExpressionsSuite.scala`. Closes #43820 from thepinetree/spark-sequence-overflow-3.5. Authored-by: Deepayan Patra Signed-off-by: Dongjoon Hyun --- .../expressions/collectionOperations.scala | 47 +++++++++++++------ .../CollectionExpressionsSuite.scala | 44 +++++++++++++++-- 2 files changed, 71 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index ade4a6c5be722..3ddbe38fdedfb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -22,6 +22,7 @@ import java.util.Comparator import scala.collection.mutable import scala.reflect.ClassTag +import org.apache.spark.SparkException.internalError import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion, UnresolvedAttribute, UnresolvedSeed} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch @@ -40,7 +41,6 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SQLOpenHashSet import org.apache.spark.unsafe.UTF8StringBuilder import org.apache.spark.unsafe.array.ByteArrayMethods -import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH import org.apache.spark.unsafe.types.{ByteArray, CalendarInterval, UTF8String} /** @@ -3080,6 +3080,34 @@ case class Sequence( } object Sequence { + private def prettyName: String = "sequence" + + def sequenceLength(start: Long, stop: Long, step: Long): Int = { + try { + val delta = Math.subtractExact(stop, start) + if (delta == Long.MinValue && step == -1L) { + // We must special-case division of Long.MinValue by -1 to catch potential unchecked + // overflow in next operation. Division does not have a builtin overflow check. We + // previously special-case div-by-zero. + throw new ArithmeticException("Long overflow (Long.MinValue / -1)") + } + val len = if (stop == start) 1L else Math.addExact(1L, (delta / step)) + if (len > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { + throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(len) + } + len.toInt + } catch { + // We handle overflows in the previous try block by raising an appropriate exception. + case _: ArithmeticException => + val safeLen = + BigInt(1) + (BigInt(stop) - BigInt(start)) / BigInt(step) + if (safeLen > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { + throw QueryExecutionErrors.createArrayWithElementsExceedLimitError(safeLen) + } + throw internalError("Unreachable code reached.") + case e: Exception => throw e + } + } private type LessThanOrEqualFn = (Any, Any) => Boolean @@ -3451,13 +3479,7 @@ object Sequence { || (estimatedStep == num.zero && start == stop), s"Illegal sequence boundaries: $start to $stop by $step") - val len = if (start == stop) 1L else 1L + (stop.toLong - start.toLong) / estimatedStep.toLong - - require( - len <= MAX_ROUNDED_ARRAY_LENGTH, - s"Too long sequence: $len. Should be <= $MAX_ROUNDED_ARRAY_LENGTH") - - len.toInt + sequenceLength(start.toLong, stop.toLong, estimatedStep.toLong) } private def genSequenceLengthCode( @@ -3467,7 +3489,7 @@ object Sequence { step: String, estimatedStep: String, len: String): String = { - val longLen = ctx.freshName("longLen") + val calcFn = classOf[Sequence].getName + ".sequenceLength" s""" |if (!(($estimatedStep > 0 && $start <= $stop) || | ($estimatedStep < 0 && $start >= $stop) || @@ -3475,12 +3497,7 @@ object Sequence { | throw new IllegalArgumentException( | "Illegal sequence boundaries: " + $start + " to " + $stop + " by " + $step); |} - |long $longLen = $stop == $start ? 1L : 1L + ((long) $stop - $start) / $estimatedStep; - |if ($longLen > $MAX_ROUNDED_ARRAY_LENGTH) { - | throw new IllegalArgumentException( - | "Too long sequence: " + $longLen + ". Should be <= $MAX_ROUNDED_ARRAY_LENGTH"); - |} - |int $len = (int) $longLen; + |int $len = $calcFn((long) $start, (long) $stop, (long) $estimatedStep); """.stripMargin } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index 1787f6ac72dd4..99eece31a1efc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{outstandingZoneIds, import org.apache.spark.sql.catalyst.util.IntervalUtils._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.types.UTF8String class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -769,10 +769,6 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper // test sequence boundaries checking - checkExceptionInExpression[IllegalArgumentException]( - new Sequence(Literal(Int.MinValue), Literal(Int.MaxValue), Literal(1)), - EmptyRow, s"Too long sequence: 4294967296. Should be <= $MAX_ROUNDED_ARRAY_LENGTH") - checkExceptionInExpression[IllegalArgumentException]( new Sequence(Literal(1), Literal(2), Literal(0)), EmptyRow, "boundaries: 1 to 2 by 0") checkExceptionInExpression[IllegalArgumentException]( @@ -782,6 +778,44 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkExceptionInExpression[IllegalArgumentException]( new Sequence(Literal(1), Literal(2), Literal(-1)), EmptyRow, "boundaries: 1 to 2 by -1") + // SPARK-43393: test Sequence overflow checking + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(Int.MinValue), Literal(Int.MaxValue), Literal(1)), + errorClass = "_LEGACY_ERROR_TEMP_2161", + parameters = Map( + "count" -> (BigInt(Int.MaxValue) - BigInt { Int.MinValue } + 1).toString, + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(0L), Literal(Long.MaxValue), Literal(1L)), + errorClass = "_LEGACY_ERROR_TEMP_2161", + parameters = Map( + "count" -> (BigInt(Long.MaxValue) + 1).toString, + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(0L), Literal(Long.MinValue), Literal(-1L)), + errorClass = "_LEGACY_ERROR_TEMP_2161", + parameters = Map( + "count" -> ((0 - BigInt(Long.MinValue)) + 1).toString(), + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(Long.MinValue), Literal(Long.MaxValue), Literal(1L)), + errorClass = "_LEGACY_ERROR_TEMP_2161", + parameters = Map( + "count" -> (BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1).toString, + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(Long.MaxValue), Literal(Long.MinValue), Literal(-1L)), + errorClass = "_LEGACY_ERROR_TEMP_2161", + parameters = Map( + "count" -> (BigInt(Long.MaxValue) - BigInt { Long.MinValue } + 1).toString, + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) + checkErrorInExpression[SparkRuntimeException]( + new Sequence(Literal(Long.MaxValue), Literal(-1L), Literal(-1L)), + errorClass = "_LEGACY_ERROR_TEMP_2161", + parameters = Map( + "count" -> (BigInt(Long.MaxValue) - BigInt { -1L } + 1).toString, + "maxRoundedArrayLength" -> ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH.toString())) + // test sequence with one element (zero step or equal start and stop) checkEvaluation(new Sequence(Literal(1), Literal(1), Literal(-1)), Seq(1)) From 96bfd8370c27baf5283646f2f93cb66ab70de844 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 20 Nov 2023 17:50:04 -0800 Subject: [PATCH 116/521] [SPARK-46012][CORE] EventLogFileReader should not read rolling logs if app status file is missing ### What changes were proposed in this pull request? This PR aims to prevent `EventLogFileReader` from reading rolling event logs if `appStatus` is missing. ### Why are the changes needed? Since Apache Spark 3.0.0, `appstatus_` is supposed to exist. https://github.com/apache/spark/blob/839f0c98bd85a14eadad13f8aaac876275ded5a4/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala#L277-L283 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43914 from dongjoon-hyun/SPARK-46012. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 6ca1c67de082269b9337503bff5161f5a2d87225) Signed-off-by: Dongjoon Hyun --- .../deploy/history/EventLogFileReaders.scala | 3 +- .../history/EventLogFileReadersSuite.scala | 31 +++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala index b21c67a2823af..714987a8eb873 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala @@ -119,7 +119,8 @@ object EventLogFileReader extends Logging { if (isSingleEventLog(status)) { Some(new SingleFileEventLogFileReader(fs, status.getPath, Option(status))) } else if (isRollingEventLogs(status)) { - if (fs.listStatus(status.getPath).exists(RollingEventLogFilesWriter.isEventLogFile)) { + if (fs.listStatus(status.getPath).exists(RollingEventLogFilesWriter.isEventLogFile) && + fs.listStatus(status.getPath).exists(RollingEventLogFilesWriter.isAppStatusFile)) { Some(new RollingEventLogFilesFileReader(fs, status.getPath)) } else { logDebug(s"Rolling event log directory have no event log file at ${status.getPath}") diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala index efb8393403043..f34f792881f90 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala @@ -229,6 +229,37 @@ class SingleFileEventLogFileReaderSuite extends EventLogFileReadersSuite { } class RollingEventLogFilesReaderSuite extends EventLogFileReadersSuite { + test("SPARK-46012: appStatus file should exist") { + withTempDir { dir => + val appId = getUniqueApplicationId + val attemptId = None + + val conf = getLoggingConf(testDirPath) + conf.set(EVENT_LOG_ENABLE_ROLLING, true) + conf.set(EVENT_LOG_ROLLING_MAX_FILE_SIZE.key, "10m") + + val writer = createWriter(appId, attemptId, testDirPath.toUri, conf, + SparkHadoopUtil.get.newConfiguration(conf)) + + writer.start() + val dummyStr = "dummy" * 1024 + writeTestEvents(writer, dummyStr, 1024 * 1024 * 20) + writer.stop() + + // Verify a healthy rolling event log directory + val logPathCompleted = getCurrentLogPath(writer.logPath, isCompleted = true) + val readerOpt = EventLogFileReader(fileSystem, new Path(logPathCompleted)) + assert(readerOpt.get.isInstanceOf[RollingEventLogFilesFileReader]) + assert(readerOpt.get.listEventLogFiles.length === 3) + + // Make unhealthy rolling event directory by removing appStatus file. + val appStatusFile = fileSystem.listStatus(new Path(logPathCompleted)) + .find(RollingEventLogFilesWriter.isAppStatusFile).get.getPath + fileSystem.delete(appStatusFile, false) + assert(EventLogFileReader(fileSystem, new Path(logPathCompleted)).isEmpty) + } + } + allCodecs.foreach { codecShortName => test(s"rolling event log files - codec $codecShortName") { val appId = getUniqueApplicationId From 24079adc8257871e88879796333b8f44633995b8 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 20 Nov 2023 17:54:41 -0800 Subject: [PATCH 117/521] [SPARK-46014][SQL][TESTS] Run `RocksDBStateStoreStreamingAggregationSuite` on a dedicated JVM ### What changes were proposed in this pull request? This PR aims to run `RocksDBStateStoreStreamingAggregationSuite` on a dedicated JVM to reduce the flakiness. ### Why are the changes needed? `RocksDBStateStoreStreamingAggregationSuite` is flaky. - https://github.com/apache/spark/actions/runs/6936862847/job/18869845206 - https://github.com/apache/spark/actions/runs/6926542106/job/18838877151 - https://github.com/apache/spark/actions/runs/6924927427/job/18834849433 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43916 from dongjoon-hyun/SPARK-46014. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 9bb1fe2a8410e6a0dbf73a420d8e9b359363b932) Signed-off-by: Dongjoon Hyun --- project/SparkBuild.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 60d52368de458..75a0650b5fac2 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -580,6 +580,7 @@ object SparkParallelTestGrouping { "org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite", "org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite", "org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite", + "org.apache.spark.sql.streaming.RocksDBStateStoreStreamingAggregationSuite", "org.apache.spark.shuffle.KubernetesLocalDiskShuffleDataIOSuite", "org.apache.spark.sql.hive.HiveScalaReflectionSuite" ) From a436736c02a689e4536acea16af6caea7be67fa7 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Tue, 21 Nov 2023 14:38:24 +0900 Subject: [PATCH 118/521] [MINOR][DOCS] Correct Python Spark Connect documentation about pip installation ### What changes were proposed in this pull request? This PR fixes the Spark Connect documentation from `pyspark==3.5.0` to `pyspark[connect]==3.5.0`; otherwise it will fail to execute the example as is because of missing dependencies. This is sort of a followup of SPARK-44867. https://github.com/apache/spark/blob/d31c8596cd714766892d1395e30358bd1cd3cb84/python/setup.py#L325-L332 ### Why are the changes needed? To guide users about using Spark Connect ### Does this PR introduce _any_ user-facing change? Yes, this fixes the user-facing documentation for Python Spark Connect. ### How was this patch tested? Manually checked with Markdown editor. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43919 from HyukjinKwon/SPARK-44867-followup. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit df1280cb10ee71ea362a95705f355402e2bcaff2) Signed-off-by: Hyukjin Kwon --- docs/spark-connect-overview.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/spark-connect-overview.md b/docs/spark-connect-overview.md index 0673763f03bcc..9da559c37fdef 100644 --- a/docs/spark-connect-overview.md +++ b/docs/spark-connect-overview.md @@ -279,11 +279,11 @@ The connection may also be programmatically created using _SparkSession#builder_

-First, install PySpark with `pip install pyspark==3.5.0` or if building a packaged PySpark application/library, +First, install PySpark with `pip install pyspark[connect]==3.5.0` or if building a packaged PySpark application/library, add it your setup.py file as: {% highlight python %} install_requires=[ -'pyspark==3.5.0' +'pyspark[connect]==3.5.0' ] {% endhighlight %} From ece4ebe575798bf92d8e6c2c454c62e6cbfecf01 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 21 Nov 2023 10:17:30 -0800 Subject: [PATCH 119/521] [SPARK-46033][SQL][TESTS] Fix flaky ArithmeticExpressionSuite ### What changes were proposed in this pull request? The pr aims to fix flaky ArithmeticExpressionSuite. https://github.com/panbingkun/spark/actions/runs/6940660146/job/18879997046 image ### Why are the changes needed? Fix bug. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Manually test. - Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43935 from panbingkun/SPARK-46033. Authored-by: panbingkun Signed-off-by: Dongjoon Hyun (cherry picked from commit b7930e718f453f8a9d923ad57161a982f16ca8e8) Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/expressions/ArithmeticExpressionSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala index 2dc7e82f77226..7a80188d445de 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala @@ -326,7 +326,8 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val quotResult = Decimal(quotExact.setScale(quotType.scale, RoundingMode.HALF_UP)) val quotExpected = if (quotResult.precision > DecimalType.MAX_PRECISION) null else quotResult - checkEvaluationOrException(quotActual, quotExpected.toLong) + checkEvaluationOrException(quotActual, + if (quotExpected == null) null else quotExpected.toLong) } } From fcf55737490a5d6a1b491b4be4c9924492b8e740 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 21 Nov 2023 10:26:36 -0800 Subject: [PATCH 120/521] [SPARK-46019][SQL][TESTS] Fix `HiveThriftServer2ListenerSuite` and `ThriftServerPageSuite` to create `java.io.tmpdir` if it doesn't exist ### What changes were proposed in this pull request? The pr aims to fix `HiveThriftServer2ListenerSuite` and `ThriftServerPageSuite` failed when there are running on local. ``` [info] ThriftServerPageSuite: [info] - thriftserver page should load successfully *** FAILED *** (515 milliseconds) [info] java.lang.IllegalStateException: Could not initialize plugin: interface org.mockito.plugins.MockMaker (alternate: null) [info] at org.mockito.internal.configuration.plugins.PluginLoader$1.invoke(PluginLoader.java:84) [info] at jdk.proxy2/jdk.proxy2.$Proxy20.isTypeMockable(Unknown Source) [info] at org.mockito.internal.util.MockUtil.typeMockabilityOf(MockUtil.java:78) [info] at org.mockito.internal.util.MockCreationValidator.validateType(MockCreationValidator.java:22) [info] at org.mockito.internal.creation.MockSettingsImpl.validatedSettings(MockSettingsImpl.java:267) [info] at org.mockito.internal.creation.MockSettingsImpl.build(MockSettingsImpl.java:234) [info] at org.mockito.internal.MockitoCore.mock(MockitoCore.java:86) [info] at org.mockito.Mockito.mock(Mockito.java:2037) [info] at org.mockito.Mockito.mock(Mockito.java:2010) [info] at org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite.getStatusStore(ThriftServerPageSuite.scala:49) ``` It can be simply reproduced by running the following command: ``` build/sbt "hive-thriftserver/testOnly org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite" -Phive-thriftserver build/sbt "hive-thriftserver/testOnly org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite" -Phive-thriftserver ``` ### Why are the changes needed? Fix tests failed. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually test: ``` build/sbt "hive-thriftserver/testOnly org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite" -Phive-thriftserver build/sbt "hive-thriftserver/testOnly org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite" -Phive-thriftserver ``` After it: ``` [info] - listener events should store successfully (live = true) (1 second, 711 milliseconds) [info] - listener events should store successfully (live = false) (6 milliseconds) [info] - cleanup session if exceeds the threshold (live = true) (21 milliseconds) [info] - cleanup session if exceeds the threshold (live = false) (3 milliseconds) [info] - update execution info when jobstart event come after execution end event (9 milliseconds) [info] - SPARK-31387 - listener update methods should not throw exception with unknown input (8 milliseconds) [info] Run completed in 3 seconds, 734 milliseconds. [info] Total number of tests run: 6 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 6, failed 0, canceled 0, ignored 0, pending 0 [info] All tests passed. [success] Total time: 156 s (02:36), completed Nov 21, 2023, 1:57:21 PM ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43921 from panbingkun/SPARK-46019. Authored-by: panbingkun Signed-off-by: Dongjoon Hyun (cherry picked from commit fdcd20f4b51c3ddddaae12f7d3f429e7b77c9f5e) Signed-off-by: Dongjoon Hyun --- .../ui/HiveThriftServer2ListenerSuite.scala | 10 ++++++++++ .../hive/thriftserver/ui/ThriftServerPageSuite.scala | 10 ++++++++++ 2 files changed, 20 insertions(+) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala index f5167a4ea7377..62d97772bcbc1 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.hive.thriftserver.ui +import java.io.File import java.util.Properties import org.mockito.Mockito.{mock, RETURNS_SMART_NULLS} @@ -34,6 +35,15 @@ class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { private var kvstore: ElementTrackingStore = _ + protected override def beforeAll(): Unit = { + val tmpDirName = System.getProperty("java.io.tmpdir") + val tmpDir = new File(tmpDirName) + if (!tmpDir.exists()) { + tmpDir.mkdirs() + } + super.beforeAll() + } + after { if (kvstore != null) { kvstore.close() diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala index d7e1852199639..1245e6740ebbe 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.hive.thriftserver.ui +import java.io.File import java.util.{Calendar, Locale} import javax.servlet.http.HttpServletRequest @@ -34,6 +35,15 @@ class ThriftServerPageSuite extends SparkFunSuite with BeforeAndAfter { private var kvstore: ElementTrackingStore = _ + protected override def beforeAll(): Unit = { + val tmpDirName = System.getProperty("java.io.tmpdir") + val tmpDir = new File(tmpDirName) + if (!tmpDir.exists()) { + tmpDir.mkdirs() + } + super.beforeAll() + } + after { if (kvstore != null) { kvstore.close() From 8f52fd55d42045d4aadb2cb18c7c3f99ad75eb35 Mon Sep 17 00:00:00 2001 From: Mark Jarvin Date: Tue, 21 Nov 2023 11:38:31 -0800 Subject: [PATCH 121/521] [SPARK-44973][SQL] Fix `ArrayIndexOutOfBoundsException` in `conv()` ### What changes were proposed in this pull request? Increase the size of the buffer allocated for the result of base conversion in `NumberConverter` to prevent ArrayIndexOutOfBoundsException when evaluating `conv(s"${Long.MinValue}", 10, -2)`. ### Why are the changes needed? I don't think the ArrayIndexOutOfBoundsException is intended behaviour. ### Does this PR introduce _any_ user-facing change? Users will no longer experience an ArrayIndexOutOfBoundsException for this specific set of arguments and will instead receive the expected base conversion. ### How was this patch tested? New unit test cases ### Was this patch authored or co-authored using generative AI tooling? No Closes #43880 from markj-db/SPARK-44973. Authored-by: Mark Jarvin Signed-off-by: Dongjoon Hyun (cherry picked from commit 2ac8ff76a5169fe1f6cf130cc82738ba78bd8c65) Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/util/NumberConverter.scala | 9 ++++++++- .../sql/catalyst/util/NumberConverterSuite.scala | 6 ++++++ .../org/apache/spark/sql/MathFunctionsSuite.scala | 11 +++++++++++ 3 files changed, 25 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala index 59765cde1f926..06d3910311b1c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala @@ -23,6 +23,13 @@ import org.apache.spark.unsafe.types.UTF8String object NumberConverter { + /** + * The output string has a max length of one char per bit in the 64-bit `Long` intermediate + * representation plus one char for the '-' sign. This happens in practice when converting + * `Long.MinValue` with `toBase` equal to -2. + */ + private final val MAX_OUTPUT_LENGTH = java.lang.Long.SIZE + 1 + /** * Decode v into value[]. * @@ -148,7 +155,7 @@ object NumberConverter { var (negative, first) = if (n(0) == '-') (true, 1) else (false, 0) // Copy the digits in the right side of the array - val temp = new Array[Byte](Math.max(n.length, 64)) + val temp = new Array[Byte](Math.max(n.length, MAX_OUTPUT_LENGTH)) var v: Long = -1 System.arraycopy(n, first, temp, temp.length - n.length + first, n.length - first) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberConverterSuite.scala index c634c5b739b8f..3de331f90a6d3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberConverterSuite.scala @@ -55,6 +55,12 @@ class NumberConverterSuite extends SparkFunSuite { checkConv("-10", 11, 7, "45012021522523134134555") } + test("SPARK-44973: conv must allocate enough space for all digits plus negative sign") { + checkConv(s"${Long.MinValue}", 10, -2, BigInt(Long.MinValue).toString(2)) + checkConv((BigInt(Long.MaxValue) + 1).toString(16), 16, -2, BigInt(Long.MinValue).toString(2)) + checkConv(BigInt(Long.MinValue).toString(16), 16, -2, BigInt(Long.MinValue).toString(2)) + } + test("byte to binary") { checkToBinary(0.toByte) checkToBinary(1.toByte) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala index 0adb89c3a9eaf..ba04e3b691a1b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala @@ -262,6 +262,17 @@ class MathFunctionsSuite extends QueryTest with SharedSparkSession { } } + test("SPARK-44973: conv must allocate enough space for all digits plus negative sign") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> false.toString) { + val df = Seq( + ((BigInt(Long.MaxValue) + 1).toString(16)), + (BigInt(Long.MinValue).toString(16)) + ).toDF("num") + checkAnswer(df.select(conv($"num", 16, -2)), + Seq(Row(BigInt(Long.MinValue).toString(2)), Row(BigInt(Long.MinValue).toString(2)))) + } + } + test("floor") { testOneToOneMathFunction(floor, (d: Double) => math.floor(d).toLong) // testOneToOneMathFunction does not validate the resulting data type From 20657549acff80769af889eb3f0599df06956d3c Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 21 Nov 2023 17:51:10 -0800 Subject: [PATCH 122/521] [SPARK-46012][CORE][FOLLOWUP] Invoke `fs.listStatus` once and reuse the result ### What changes were proposed in this pull request? This PR is a follow-up of #43914 and aims to invoke `fs.listStatus` once and reuse the result. ### Why are the changes needed? This will prevent the increase of the number of `listStatus` invocation . ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs with the existing test case. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43944 from dongjoon-hyun/SPARK-46012-2. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 6be4a0358265fb81f68a27589f9940bd726c8ee7) Signed-off-by: Dongjoon Hyun --- .../apache/spark/deploy/history/EventLogFileReaders.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala index 714987a8eb873..8c3dda4727784 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala @@ -119,8 +119,9 @@ object EventLogFileReader extends Logging { if (isSingleEventLog(status)) { Some(new SingleFileEventLogFileReader(fs, status.getPath, Option(status))) } else if (isRollingEventLogs(status)) { - if (fs.listStatus(status.getPath).exists(RollingEventLogFilesWriter.isEventLogFile) && - fs.listStatus(status.getPath).exists(RollingEventLogFilesWriter.isAppStatusFile)) { + val files = fs.listStatus(status.getPath) + if (files.exists(RollingEventLogFilesWriter.isEventLogFile) && + files.exists(RollingEventLogFilesWriter.isAppStatusFile)) { Some(new RollingEventLogFilesFileReader(fs, status.getPath)) } else { logDebug(s"Rolling event log directory have no event log file at ${status.getPath}") From 1f81e26e03803238ee6292762bcbee49e1a7c066 Mon Sep 17 00:00:00 2001 From: Angerszhuuuu Date: Wed, 22 Nov 2023 16:50:21 +0800 Subject: [PATCH 123/521] [SPARK-46006][YARN] YarnAllocator miss clean targetNumExecutorsPerResourceProfileId after YarnSchedulerBackend call stop MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? We meet a case that user call sc.stop() after run all custom code, but stuck in some place. Cause below situation 1. User call sc.stop() 2. sc.stop() stuck in some process, but SchedulerBackend.stop was called 3. Since yarn ApplicationMaster didn't finish, still call YarnAllocator.allocateResources() 4. Since driver endpoint stop new allocated executor failed to register 5. untll trigger Max number of executor failures 6. Caused by Before call CoarseGrainedSchedulerBackend.stop() will call YarnSchedulerBackend.requestTotalExecutor() to clean request info ![image](https://github.com/apache/spark/assets/46485123/4a61fb40-5986-4ecc-9329-369187d5311d) When YarnAllocator handle then empty resource request, since resourceTotalExecutorsWithPreferedLocalities is empty, miss clean targetNumExecutorsPerResourceProfileId. ![image](https://github.com/apache/spark/assets/46485123/0133f606-e1d7-4db7-95fe-140c61379102) ### Why are the changes needed? Fix bug ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? No ### Was this patch authored or co-authored using generative AI tooling? No Closes #43906 from AngersZhuuuu/SPARK-46006. Authored-by: Angerszhuuuu Signed-off-by: Kent Yao (cherry picked from commit 06635e25f170e61f6cfe53232d001993ec7d376d) Signed-off-by: Kent Yao --- .../spark/deploy/yarn/YarnAllocator.scala | 28 +++++++++++-------- 1 file changed, 17 insertions(+), 11 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 19c06f957318b..f14fc9d5de461 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -384,19 +384,25 @@ private[yarn] class YarnAllocator( this.numLocalityAwareTasksPerResourceProfileId = numLocalityAwareTasksPerResourceProfileId this.hostToLocalTaskCountPerResourceProfileId = hostToLocalTaskCountPerResourceProfileId - val res = resourceProfileToTotalExecs.map { case (rp, numExecs) => - createYarnResourceForResourceProfile(rp) - if (numExecs != getOrUpdateTargetNumExecutorsForRPId(rp.id)) { - logInfo(s"Driver requested a total number of $numExecs executor(s) " + - s"for resource profile id: ${rp.id}.") - targetNumExecutorsPerResourceProfileId(rp.id) = numExecs - allocatorNodeHealthTracker.setSchedulerExcludedNodes(excludedNodes) - true - } else { - false + if (resourceProfileToTotalExecs.isEmpty) { + targetNumExecutorsPerResourceProfileId.clear() + allocatorNodeHealthTracker.setSchedulerExcludedNodes(excludedNodes) + true + } else { + val res = resourceProfileToTotalExecs.map { case (rp, numExecs) => + createYarnResourceForResourceProfile(rp) + if (numExecs != getOrUpdateTargetNumExecutorsForRPId(rp.id)) { + logInfo(s"Driver requested a total number of $numExecs executor(s) " + + s"for resource profile id: ${rp.id}.") + targetNumExecutorsPerResourceProfileId(rp.id) = numExecs + allocatorNodeHealthTracker.setSchedulerExcludedNodes(excludedNodes) + true + } else { + false + } } + res.exists(_ == true) } - res.exists(_ == true) } /** From 18bcd020118a8efb49c03546ec501be6f0fc0852 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 21 Nov 2023 16:07:50 +0900 Subject: [PATCH 124/521] [MINOR][BUILD] Rename gprcVersion to grpcVersion in SparkBuild ### What changes were proposed in this pull request? This PR aims to fix a typo. ``` - val gprcVersion = "1.56.0" + val grpcVersion = "1.56.0" ``` There are two occurrences. ``` $ git grep gprc project/SparkBuild.scala: val gprcVersion = "1.56.0" project/SparkBuild.scala: "io.grpc" % "protoc-gen-grpc-java" % BuildCommons.gprcVersion asProtocPlugin(), ``` ### Why are the changes needed? To fix a typo. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43923 from dongjoon-hyun/minor_grpc. Authored-by: Dongjoon Hyun Signed-off-by: Hyukjin Kwon --- project/SparkBuild.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 75a0650b5fac2..79b58deafde57 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -91,7 +91,7 @@ object BuildCommons { // SPARK-41247: needs to be consistent with `protobuf.version` in `pom.xml`. val protoVersion = "3.23.4" // GRPC version used for Spark Connect. - val gprcVersion = "1.56.0" + val grpcVersion = "1.56.0" } object SparkBuild extends PomBuild { @@ -694,7 +694,7 @@ object SparkConnectCommon { SbtPomKeys.effectivePom.value.getProperties.get( "guava.failureaccess.version").asInstanceOf[String] Seq( - "io.grpc" % "protoc-gen-grpc-java" % BuildCommons.gprcVersion asProtocPlugin(), + "io.grpc" % "protoc-gen-grpc-java" % BuildCommons.grpcVersion asProtocPlugin(), "com.google.guava" % "guava" % guavaVersion, "com.google.guava" % "failureaccess" % guavaFailureaccessVersion, "com.google.protobuf" % "protobuf-java" % protoVersion % "protobuf" From c3671942abbd5d96d7d2c7496a882be91533838b Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Thu, 23 Nov 2023 20:11:43 +0900 Subject: [PATCH 125/521] [SPARK-46064][SQL][SS] Move out EliminateEventTimeWatermark to the analyzer and change to only take effect on resolved child This PR proposes to move out EliminateEventTimeWatermark to the analyzer (one of the analysis rule), and also make a change to eliminate EventTimeWatermark node only when the child of EventTimeWatermark is "resolved". Currently, we apply EliminateEventTimeWatermark immediately when withWatermark is called, which means the rule is applied immediately against the child, regardless whether child is resolved or not. It is not an issue for the usage of DataFrame API initiated by read / readStream, because streaming sources have the flag isStreaming set to true even it is yet resolved. But mix-up of SQL and DataFrame API would expose the issue; we may not know the exact value of isStreaming flag on unresolved node and it is subject to change upon resolution. No. New UTs. No. Closes #43971 from HeartSaVioR/SPARK-46064. Authored-by: Jungtaek Lim Signed-off-by: Jungtaek Lim (cherry picked from commit a703dace0aa400fa24b2bded1500f44ae7ac8db0) Signed-off-by: Jungtaek Lim --- .../sql/catalyst/analysis/Analyzer.scala | 6 +++-- .../sql/catalyst/analysis/AnalysisSuite.scala | 23 +++++++++++++++++++ .../sql/catalyst/analysis/AnalysisTest.scala | 2 ++ .../sql/catalyst/analysis/TestRelations.scala | 14 +++++++++++ .../optimizer/FilterPushdownSuite.scala | 8 +++---- .../scala/org/apache/spark/sql/Dataset.scala | 3 +-- 6 files changed, 48 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 02b9c2445433b..8fe87a05d02d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -348,7 +348,9 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor Batch("Cleanup", fixedPoint, CleanupAliases), Batch("HandleSpecialCommand", Once, - HandleSpecialCommand) + HandleSpecialCommand), + Batch("Remove watermark for batch query", Once, + EliminateEventTimeWatermark) ) /** @@ -3844,7 +3846,7 @@ object CleanupAliases extends Rule[LogicalPlan] with AliasHelper { object EliminateEventTimeWatermark extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning( _.containsPattern(EVENT_TIME_WATERMARK)) { - case EventTimeWatermark(_, _, child) if !child.isStreaming => child + case EventTimeWatermark(_, _, child) if child.resolved && !child.isStreaming => child } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 802b6d471a65c..843d51034aa2b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -1668,4 +1668,27 @@ class AnalysisSuite extends AnalysisTest with Matchers { checkAnalysis(ident2.select($"a"), testRelation.select($"a").analyze) } } + + test("SPARK-46064 Basic functionality of elimination for watermark node in batch query") { + val dfWithEventTimeWatermark = EventTimeWatermark($"ts", + IntervalUtils.fromIntervalString("10 seconds"), batchRelationWithTs) + + val analyzed = getAnalyzer.executeAndCheck(dfWithEventTimeWatermark, new QueryPlanningTracker) + + // EventTimeWatermark node is eliminated via EliminateEventTimeWatermark. + assert(!analyzed.exists(_.isInstanceOf[EventTimeWatermark])) + } + + test("SPARK-46064 EliminateEventTimeWatermark properly handles the case where the child of " + + "EventTimeWatermark changes the isStreaming flag during resolution") { + // UnresolvedRelation which is batch initially and will be resolved as streaming + val dfWithTempView = UnresolvedRelation(TableIdentifier("streamingTable")) + val dfWithEventTimeWatermark = EventTimeWatermark($"ts", + IntervalUtils.fromIntervalString("10 seconds"), dfWithTempView) + + val analyzed = getAnalyzer.executeAndCheck(dfWithEventTimeWatermark, new QueryPlanningTracker) + + // EventTimeWatermark node is NOT eliminated. + assert(analyzed.exists(_.isInstanceOf[EventTimeWatermark])) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index 997308c6ef44f..5152666473286 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -84,6 +84,8 @@ trait AnalysisTest extends PlanTest { createTempView(catalog, "TaBlE3", TestRelations.testRelation3, overrideIfExists = true) createGlobalTempView(catalog, "TaBlE4", TestRelations.testRelation4, overrideIfExists = true) createGlobalTempView(catalog, "TaBlE5", TestRelations.testRelation5, overrideIfExists = true) + createTempView(catalog, "streamingTable", TestRelations.streamingRelation, + overrideIfExists = true) new Analyzer(catalog) { override val extendedResolutionRules = extendedAnalysisRules } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TestRelations.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TestRelations.scala index d54237fcc1407..01b1a627e2871 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TestRelations.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TestRelations.scala @@ -68,4 +68,18 @@ object TestRelations { val mapRelation = LocalRelation( AttributeReference("map", MapType(IntegerType, IntegerType))()) + + val streamingRelation = LocalRelation( + Seq( + AttributeReference("a", IntegerType)(), + AttributeReference("ts", TimestampType)() + ), + isStreaming = true) + + val batchRelationWithTs = LocalRelation( + Seq( + AttributeReference("a", IntegerType)(), + AttributeReference("ts", TimestampType)() + ), + isStreaming = false) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index ee56d1fa9acd3..2ebb43d4fba3e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -1190,7 +1190,7 @@ class FilterPushdownSuite extends PlanTest { test("watermark pushdown: no pushdown on watermark attribute #1") { val interval = new CalendarInterval(2, 2, 2000L) - val relation = LocalRelation(attrA, $"b".timestamp, attrC) + val relation = LocalRelation(Seq(attrA, $"b".timestamp, attrC), Nil, isStreaming = true) // Verify that all conditions except the watermark touching condition are pushed down // by the optimizer and others are not. @@ -1205,7 +1205,7 @@ class FilterPushdownSuite extends PlanTest { test("watermark pushdown: no pushdown for nondeterministic filter") { val interval = new CalendarInterval(2, 2, 2000L) - val relation = LocalRelation(attrA, attrB, $"c".timestamp) + val relation = LocalRelation(Seq(attrA, attrB, $"c".timestamp), Nil, isStreaming = true) // Verify that all conditions except the watermark touching condition are pushed down // by the optimizer and others are not. @@ -1221,7 +1221,7 @@ class FilterPushdownSuite extends PlanTest { test("watermark pushdown: full pushdown") { val interval = new CalendarInterval(2, 2, 2000L) - val relation = LocalRelation(attrA, attrB, $"c".timestamp) + val relation = LocalRelation(Seq(attrA, attrB, $"c".timestamp), Nil, isStreaming = true) // Verify that all conditions except the watermark touching condition are pushed down // by the optimizer and others are not. @@ -1236,7 +1236,7 @@ class FilterPushdownSuite extends PlanTest { test("watermark pushdown: no pushdown on watermark attribute #2") { val interval = new CalendarInterval(2, 2, 2000L) - val relation = LocalRelation($"a".timestamp, attrB, attrC) + val relation = LocalRelation(Seq($"a".timestamp, attrB, attrC), Nil, isStreaming = true) val originalQuery = EventTimeWatermark($"a", interval, relation) .where($"a" === new java.sql.Timestamp(0) && $"b" === 10) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index f53c6ddaa3880..c063af9381ff2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -774,8 +774,7 @@ class Dataset[T] private[sql]( val parsedDelay = IntervalUtils.fromIntervalString(delayThreshold) require(!IntervalUtils.isNegative(parsedDelay), s"delay threshold ($delayThreshold) should not be negative.") - EliminateEventTimeWatermark( - EventTimeWatermark(UnresolvedAttribute(eventTime), parsedDelay, logicalPlan)) + EventTimeWatermark(UnresolvedAttribute(eventTime), parsedDelay, logicalPlan) } /** From a8552627cdc0945c52d2ae5115b1218c1254264d Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Thu, 23 Nov 2023 22:32:16 +0900 Subject: [PATCH 126/521] [SPARK-46062][SQL] Sync the isStreaming flag between CTE definition and reference This PR proposes to sync the flag `isStreaming` from CTE definition to CTE reference. The essential issue is that CTE reference node cannot determine the flag `isStreaming` by itself, and never be able to have a proper value and always takes the default as it does not have a parameter in constructor. The other flag `resolved` is handled, and we need to do the same for `isStreaming`. Once we add the parameter to the constructor, we will also need to make sure the flag is in sync with CTE definition. We have a rule `ResolveWithCTE` doing the sync, hence we add the logic to sync the flag `isStreaming` as well. The bug may impact some rules which behaves differently depending on isStreaming flag. It would no longer be a problem once CTE reference is replaced with CTE definition at some point in "optimization phase", but all rules in analyzer and optimizer being triggered before the rule takes effect may misbehave based on incorrect isStreaming flag. No. New UT. No. Closes #43966 from HeartSaVioR/SPARK-46062. Authored-by: Jungtaek Lim Signed-off-by: Jungtaek Lim (cherry picked from commit 43046631a5d4ac7201361a00473cc87fa52ab5a7) Signed-off-by: Jungtaek Lim --- .../catalyst/analysis/CTESubstitution.scala | 2 +- .../catalyst/analysis/ResolveWithCTE.scala | 2 +- .../optimizer/MergeScalarSubqueries.scala | 3 +- ...wnPredicatesAndPruneColumnsForCTEDef.scala | 2 +- .../plans/logical/basicLogicalOperators.scala | 1 + .../sql/catalyst/analysis/AnalysisSuite.scala | 27 +++++-- .../MergeScalarSubqueriesSuite.scala | 3 +- .../double-quoted-identifiers-enabled.sql.out | 2 +- .../analyzer-results/cte-nested.sql.out | 38 +++++----- .../analyzer-results/cte-nonlegacy.sql.out | 64 ++++++++--------- .../sql-tests/analyzer-results/cte.sql.out | 70 +++++++++---------- .../analyzer-results/join-lateral.sql.out | 4 +- .../non-excludable-rule.sql.out | 2 +- .../postgreSQL/window_part3.sql.out | 8 +-- .../analyzer-results/postgreSQL/with.sql.out | 12 ++-- .../exists-subquery/exists-cte.sql.out | 16 ++--- .../in-subquery/in-multiple-columns.sql.out | 4 +- .../subquery/in-subquery/in-with-cte.sql.out | 64 ++++++++--------- .../scalar-subquery-select.sql.out | 20 +++--- .../analyzer-results/transform.sql.out | 4 +- .../analyzer-results/using-join.sql.out | 4 +- .../sql/streaming/StreamingQuerySuite.scala | 47 ++++++++++++- 22 files changed, 231 insertions(+), 168 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala index 954f5f19cd3ec..7321f5becdc48 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala @@ -263,7 +263,7 @@ object CTESubstitution extends Rule[LogicalPlan] { d.child } else { // Add a `SubqueryAlias` for hint-resolving rules to match relation names. - SubqueryAlias(table, CTERelationRef(d.id, d.resolved, d.output)) + SubqueryAlias(table, CTERelationRef(d.id, d.resolved, d.output, d.isStreaming)) } }.getOrElse(u) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala index 78b776f12f074..f1077378b2d9c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveWithCTE.scala @@ -51,7 +51,7 @@ object ResolveWithCTE extends Rule[LogicalPlan] { case ref: CTERelationRef if !ref.resolved => cteDefMap.get(ref.cteId).map { cteDef => - CTERelationRef(cteDef.id, cteDef.resolved, cteDef.output) + CTERelationRef(cteDef.id, cteDef.resolved, cteDef.output, cteDef.isStreaming) }.getOrElse { ref } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala index 6184160829ba6..ff0bc5e66d755 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueries.scala @@ -381,7 +381,8 @@ object MergeScalarSubqueries extends Rule[LogicalPlan] { val subqueryCTE = header.plan.asInstanceOf[CTERelationDef] GetStructField( ScalarSubquery( - CTERelationRef(subqueryCTE.id, _resolved = true, subqueryCTE.output), + CTERelationRef(subqueryCTE.id, _resolved = true, subqueryCTE.output, + subqueryCTE.isStreaming), exprId = ssr.exprId), ssr.headerIndex) } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala index e643a1af363a1..aa13e6a67c510 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala @@ -141,7 +141,7 @@ object PushdownPredicatesAndPruneColumnsForCTEDef extends Rule[LogicalPlan] { cteDef } - case cteRef @ CTERelationRef(cteId, _, output, _) => + case cteRef @ CTERelationRef(cteId, _, output, _, _) => val (cteDef, _, _, newAttrSet) = cteMap(cteId) if (needsPruning(cteDef.child, newAttrSet)) { val indices = newAttrSet.toSeq.map(cteDef.output.indexOf) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 0e460706fc5b8..b4d7716a566e4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -853,6 +853,7 @@ case class CTERelationRef( cteId: Long, _resolved: Boolean, override val output: Seq[Attribute], + override val isStreaming: Boolean, statsOpt: Option[Statistics] = None) extends LeafNode with MultiInstanceRelation { final override val nodePatterns: Seq[TreePattern] = Seq(CTE) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 843d51034aa2b..8e5329d986ef7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -1517,7 +1517,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { test("SPARK-43030: deduplicate relations in CTE relation definitions") { val join = testRelation.as("left").join(testRelation.as("right")) val cteDef = CTERelationDef(join) - val cteRef = CTERelationRef(cteDef.id, false, Nil) + val cteRef = CTERelationRef(cteDef.id, false, Nil, false) withClue("flat CTE") { val plan = WithCTE(cteRef.select($"left.a"), Seq(cteDef)).analyze @@ -1530,7 +1530,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { withClue("nested CTE") { val cteDef2 = CTERelationDef(WithCTE(cteRef.join(testRelation), Seq(cteDef))) - val cteRef2 = CTERelationRef(cteDef2.id, false, Nil) + val cteRef2 = CTERelationRef(cteDef2.id, false, Nil, false) val plan = WithCTE(cteRef2, Seq(cteDef2)).analyze val relations = plan.collect { case r: LocalRelation => r @@ -1542,7 +1542,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { test("SPARK-43030: deduplicate CTE relation references") { val cteDef = CTERelationDef(testRelation.select($"a")) - val cteRef = CTERelationRef(cteDef.id, false, Nil) + val cteRef = CTERelationRef(cteDef.id, false, Nil, false) withClue("single reference") { val plan = WithCTE(cteRef.where($"a" > 1), Seq(cteDef)).analyze @@ -1565,7 +1565,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { withClue("CTE relation has duplicated attributes") { val cteDef = CTERelationDef(testRelation.select($"a", $"a")) - val cteRef = CTERelationRef(cteDef.id, false, Nil) + val cteRef = CTERelationRef(cteDef.id, false, Nil, false) val plan = WithCTE(cteRef.join(cteRef.select($"a")), Seq(cteDef)).analyze val refs = plan.collect { case r: CTERelationRef => r @@ -1577,14 +1577,14 @@ class AnalysisSuite extends AnalysisTest with Matchers { withClue("CTE relation has duplicate aliases") { val alias = Alias($"a", "x")() val cteDef = CTERelationDef(testRelation.select(alias, alias).where($"x" === 1)) - val cteRef = CTERelationRef(cteDef.id, false, Nil) + val cteRef = CTERelationRef(cteDef.id, false, Nil, false) // Should not fail with the assertion failure: Found duplicate rewrite attributes. WithCTE(cteRef.join(cteRef), Seq(cteDef)).analyze } withClue("references in both CTE relation definition and main query") { val cteDef2 = CTERelationDef(cteRef.where($"a" > 2)) - val cteRef2 = CTERelationRef(cteDef2.id, false, Nil) + val cteRef2 = CTERelationRef(cteDef2.id, false, Nil, false) val plan = WithCTE(cteRef.union(cteRef2), Seq(cteDef, cteDef2)).analyze val refs = plan.collect { case r: CTERelationRef => r @@ -1691,4 +1691,19 @@ class AnalysisSuite extends AnalysisTest with Matchers { // EventTimeWatermark node is NOT eliminated. assert(analyzed.exists(_.isInstanceOf[EventTimeWatermark])) } + + test("SPARK-46062: isStreaming flag is synced from CTE definition to CTE reference") { + val cteDef = CTERelationDef(streamingRelation.select($"a", $"ts")) + // Intentionally marking the flag _resolved to false, so that analyzer has a chance to sync + // the flag isStreaming on syncing the flag _resolved. + val cteRef = CTERelationRef(cteDef.id, _resolved = false, Nil, isStreaming = false) + val plan = WithCTE(cteRef, Seq(cteDef)).analyze + + val refs = plan.collect { + case r: CTERelationRef => r + } + assert(refs.length == 1) + assert(refs.head.resolved) + assert(refs.head.isStreaming) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala index 8af0e02855b12..13e138414781f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala @@ -42,7 +42,8 @@ class MergeScalarSubqueriesSuite extends PlanTest { } private def extractorExpression(cteIndex: Int, output: Seq[Attribute], fieldIndex: Int) = { - GetStructField(ScalarSubquery(CTERelationRef(cteIndex, _resolved = true, output)), fieldIndex) + GetStructField(ScalarSubquery( + CTERelationRef(cteIndex, _resolved = true, output, isStreaming = false)), fieldIndex) .as("scalarsubquery()") } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out index 0a009a3a282f9..b45e461264e27 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out @@ -434,7 +434,7 @@ Project [a1#x AS a2#x] : +- OneRowRelation +- Project [a#x] +- SubqueryAlias v - +- CTERelationRef xxxx, true, [a#x] + +- CTERelationRef xxxx, true, [a#x], false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out index d96965edde136..de0e6dfae2ce3 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out @@ -15,10 +15,10 @@ WithCTE : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x] +: +- CTERelationRef xxxx, true, [1#x], false +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x] + +- CTERelationRef xxxx, true, [1#x], false -- !query @@ -37,7 +37,7 @@ Aggregate [max(c#x) AS max(c)#x] : +- OneRowRelation +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x] + +- CTERelationRef xxxx, true, [c#x], false -- !query @@ -54,7 +54,7 @@ Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [1#x] +: +- CTERelationRef xxxx, true, [1#x], false +- OneRowRelation @@ -136,11 +136,11 @@ WithCTE : : : +- OneRowRelation : : +- Project [c#x] : : +- SubqueryAlias t -: : +- CTERelationRef xxxx, true, [c#x] +: : +- CTERelationRef xxxx, true, [c#x], false : +- OneRowRelation +- Project [scalarsubquery()#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [scalarsubquery()#x] + +- CTERelationRef xxxx, true, [scalarsubquery()#x], false -- !query @@ -189,7 +189,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x] + +- CTERelationRef xxxx, true, [c#x], false -- !query @@ -218,7 +218,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x] + +- CTERelationRef xxxx, true, [c#x], false -- !query @@ -253,7 +253,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x] + +- CTERelationRef xxxx, true, [c#x], false -- !query @@ -352,14 +352,14 @@ WithCTE : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x] +: +- CTERelationRef xxxx, true, [1#x], false :- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- OneRowRelation +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x] + +- CTERelationRef xxxx, true, [1#x], false -- !query @@ -420,15 +420,15 @@ WithCTE : +- SubqueryAlias t3 : +- Project [1#x] : +- SubqueryAlias t1 -: +- CTERelationRef xxxx, true, [1#x] +: +- CTERelationRef xxxx, true, [1#x], false :- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [1#x] : +- SubqueryAlias t3 -: +- CTERelationRef xxxx, true, [1#x] +: +- CTERelationRef xxxx, true, [1#x], false +- Project [1#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [1#x] + +- CTERelationRef xxxx, true, [1#x], false -- !query @@ -451,12 +451,12 @@ WithCTE : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x] +: +- CTERelationRef xxxx, true, [1#x], false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x] + +- CTERelationRef xxxx, true, [1#x], false -- !query @@ -484,19 +484,19 @@ WithCTE : +- SubqueryAlias cte_inner_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x] +: +- CTERelationRef xxxx, true, [1#x], false :- CTERelationDef xxxx, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias __auto_generated_subquery_name : +- Project [1#x] : +- SubqueryAlias cte_inner_inner -: +- CTERelationRef xxxx, true, [1#x] +: +- CTERelationRef xxxx, true, [1#x], false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x] + +- CTERelationRef xxxx, true, [1#x], false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out index bd9b443d01d0a..f1a302b06f2a8 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out @@ -15,10 +15,10 @@ WithCTE : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x] +: +- CTERelationRef xxxx, true, [1#x], false +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x] + +- CTERelationRef xxxx, true, [1#x], false -- !query @@ -37,7 +37,7 @@ Aggregate [max(c#x) AS max(c)#x] : +- OneRowRelation +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x] + +- CTERelationRef xxxx, true, [c#x], false -- !query @@ -54,7 +54,7 @@ Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [1#x] +: +- CTERelationRef xxxx, true, [1#x], false +- OneRowRelation @@ -106,10 +106,10 @@ WithCTE : +- SubqueryAlias t2 : +- Project [2#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [2#x] +: +- CTERelationRef xxxx, true, [2#x], false +- Project [2#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [2#x] + +- CTERelationRef xxxx, true, [2#x], false -- !query @@ -144,11 +144,11 @@ WithCTE : : : +- OneRowRelation : : +- Project [c#x] : : +- SubqueryAlias t -: : +- CTERelationRef xxxx, true, [c#x] +: : +- CTERelationRef xxxx, true, [c#x], false : +- OneRowRelation +- Project [scalarsubquery()#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [scalarsubquery()#x] + +- CTERelationRef xxxx, true, [scalarsubquery()#x], false -- !query @@ -181,15 +181,15 @@ WithCTE : +- SubqueryAlias t2 : +- Project [3#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [3#x] +: +- CTERelationRef xxxx, true, [3#x], false :- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [3#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [3#x] +: +- CTERelationRef xxxx, true, [3#x], false +- Project [3#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [3#x] + +- CTERelationRef xxxx, true, [3#x], false -- !query @@ -214,7 +214,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x] + +- CTERelationRef xxxx, true, [c#x], false -- !query @@ -243,7 +243,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x] + +- CTERelationRef xxxx, true, [c#x], false -- !query @@ -278,7 +278,7 @@ WithCTE +- SubqueryAlias __auto_generated_subquery_name +- Project [c#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x] + +- CTERelationRef xxxx, true, [c#x], false -- !query @@ -301,7 +301,7 @@ WithCTE : : +- OneRowRelation : +- Project [2#x] : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [2#x] + : +- CTERelationRef xxxx, true, [2#x], false +- OneRowRelation @@ -328,7 +328,7 @@ WithCTE : : : +- OneRowRelation : : +- Project [2#x] : : +- SubqueryAlias t - : : +- CTERelationRef xxxx, true, [2#x] + : : +- CTERelationRef xxxx, true, [2#x], false : +- OneRowRelation +- OneRowRelation @@ -362,7 +362,7 @@ WithCTE : : : +- OneRowRelation : : +- Project [3#x] : : +- SubqueryAlias t - : : +- CTERelationRef xxxx, true, [3#x] + : : +- CTERelationRef xxxx, true, [3#x], false : +- OneRowRelation +- OneRowRelation @@ -391,9 +391,9 @@ WithCTE : : +- OneRowRelation : +- Project [c#x] : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [c#x] + : +- CTERelationRef xxxx, true, [c#x], false +- SubqueryAlias t - +- CTERelationRef xxxx, true, [c#x] + +- CTERelationRef xxxx, true, [c#x], false -- !query @@ -414,14 +414,14 @@ WithCTE : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 -: +- CTERelationRef xxxx, true, [1#x] +: +- CTERelationRef xxxx, true, [1#x], false :- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- OneRowRelation +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x] + +- CTERelationRef xxxx, true, [1#x], false -- !query @@ -446,10 +446,10 @@ WithCTE : +- SubqueryAlias t : +- Project [2#x] : +- SubqueryAlias aBC -: +- CTERelationRef xxxx, true, [2#x] +: +- CTERelationRef xxxx, true, [2#x], false +- Project [2#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [2#x] + +- CTERelationRef xxxx, true, [2#x], false -- !query @@ -472,7 +472,7 @@ WithCTE : : +- OneRowRelation : +- Project [2#x] : +- SubqueryAlias aBC - : +- CTERelationRef xxxx, true, [2#x] + : +- CTERelationRef xxxx, true, [2#x], false +- OneRowRelation @@ -496,15 +496,15 @@ WithCTE : +- SubqueryAlias t3 : +- Project [1#x] : +- SubqueryAlias t1 -: +- CTERelationRef xxxx, true, [1#x] +: +- CTERelationRef xxxx, true, [1#x], false :- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [1#x] : +- SubqueryAlias t3 -: +- CTERelationRef xxxx, true, [1#x] +: +- CTERelationRef xxxx, true, [1#x], false +- Project [1#x] +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [1#x] + +- CTERelationRef xxxx, true, [1#x], false -- !query @@ -527,12 +527,12 @@ WithCTE : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x] +: +- CTERelationRef xxxx, true, [1#x], false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x] + +- CTERelationRef xxxx, true, [1#x], false -- !query @@ -560,19 +560,19 @@ WithCTE : +- SubqueryAlias cte_inner_inner : +- Project [1#x] : +- SubqueryAlias cte_outer -: +- CTERelationRef xxxx, true, [1#x] +: +- CTERelationRef xxxx, true, [1#x], false :- CTERelationDef xxxx, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias __auto_generated_subquery_name : +- Project [1#x] : +- SubqueryAlias cte_inner_inner -: +- CTERelationRef xxxx, true, [1#x] +: +- CTERelationRef xxxx, true, [1#x], false +- Project [1#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [1#x] +- SubqueryAlias cte_inner - +- CTERelationRef xxxx, true, [1#x] + +- CTERelationRef xxxx, true, [1#x], false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out index b9a0f776528d8..e817aaf9e59ff 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out @@ -73,7 +73,7 @@ WithCTE : +- LocalRelation [id#x] +- Project [1#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [1#x] + +- CTERelationRef xxxx, true, [1#x], false -- !query @@ -113,13 +113,13 @@ WithCTE : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- SubqueryAlias t1 -: +- CTERelationRef xxxx, true, [id#x] +: +- CTERelationRef xxxx, true, [id#x], false +- Project [id#x, 2#x] +- Join Cross :- SubqueryAlias t1 - : +- CTERelationRef xxxx, true, [id#x] + : +- CTERelationRef xxxx, true, [id#x], false +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [2#x] + +- CTERelationRef xxxx, true, [2#x], false -- !query @@ -157,10 +157,10 @@ WithCTE +- Join Cross :- SubqueryAlias t1 : +- SubqueryAlias CTE1 - : +- CTERelationRef xxxx, true, [id#x] + : +- CTERelationRef xxxx, true, [id#x], false +- SubqueryAlias t2 +- SubqueryAlias CTE1 - +- CTERelationRef xxxx, true, [id#x] + +- CTERelationRef xxxx, true, [id#x], false -- !query @@ -176,7 +176,7 @@ WithCTE +- Project [x#x] +- Filter (x#x = 1) +- SubqueryAlias t - +- CTERelationRef xxxx, true, [x#x] + +- CTERelationRef xxxx, true, [x#x], false -- !query @@ -192,7 +192,7 @@ WithCTE +- Project [x#x, y#x] +- Filter ((x#x = 1) AND (y#x = 2)) +- SubqueryAlias t - +- CTERelationRef xxxx, true, [x#x, y#x] + +- CTERelationRef xxxx, true, [x#x, y#x], false -- !query @@ -207,7 +207,7 @@ WithCTE : +- OneRowRelation +- Project [x#x, x#x] +- SubqueryAlias t - +- CTERelationRef xxxx, true, [x#x, x#x] + +- CTERelationRef xxxx, true, [x#x, x#x], false -- !query @@ -309,46 +309,46 @@ WithCTE : +- Project [c8#x AS c7#x] : +- Project [c8#x] : +- SubqueryAlias w8 -: +- CTERelationRef xxxx, true, [c8#x] +: +- CTERelationRef xxxx, true, [c8#x], false :- CTERelationDef xxxx, false : +- SubqueryAlias w6 : +- Project [c7#x AS c6#x] : +- Project [c7#x] : +- SubqueryAlias w7 -: +- CTERelationRef xxxx, true, [c7#x] +: +- CTERelationRef xxxx, true, [c7#x], false :- CTERelationDef xxxx, false : +- SubqueryAlias w5 : +- Project [c6#x AS c5#x] : +- Project [c6#x] : +- SubqueryAlias w6 -: +- CTERelationRef xxxx, true, [c6#x] +: +- CTERelationRef xxxx, true, [c6#x], false :- CTERelationDef xxxx, false : +- SubqueryAlias w4 : +- Project [c5#x AS c4#x] : +- Project [c5#x] : +- SubqueryAlias w5 -: +- CTERelationRef xxxx, true, [c5#x] +: +- CTERelationRef xxxx, true, [c5#x], false :- CTERelationDef xxxx, false : +- SubqueryAlias w3 : +- Project [c4#x AS c3#x] : +- Project [c4#x] : +- SubqueryAlias w4 -: +- CTERelationRef xxxx, true, [c4#x] +: +- CTERelationRef xxxx, true, [c4#x], false :- CTERelationDef xxxx, false : +- SubqueryAlias w2 : +- Project [c3#x AS c2#x] : +- Project [c3#x] : +- SubqueryAlias w3 -: +- CTERelationRef xxxx, true, [c3#x] +: +- CTERelationRef xxxx, true, [c3#x], false :- CTERelationDef xxxx, false : +- SubqueryAlias w1 : +- Project [c2#x AS c1#x] : +- Project [c2#x] : +- SubqueryAlias w2 -: +- CTERelationRef xxxx, true, [c2#x] +: +- CTERelationRef xxxx, true, [c2#x], false +- Project [c1#x] +- SubqueryAlias w1 - +- CTERelationRef xxxx, true, [c1#x] + +- CTERelationRef xxxx, true, [c1#x], false -- !query @@ -384,7 +384,7 @@ WithCTE +- Project [42#x, 10#x] +- Join Inner :- SubqueryAlias same_name - : +- CTERelationRef xxxx, true, [42#x] + : +- CTERelationRef xxxx, true, [42#x], false +- SubqueryAlias same_name +- Project [10 AS 10#x] +- OneRowRelation @@ -423,7 +423,7 @@ WithCTE : +- OneRowRelation +- Project [x#x, typeof(x#x) AS typeof(x)#x] +- SubqueryAlias q - +- CTERelationRef xxxx, true, [x#x] + +- CTERelationRef xxxx, true, [x#x], false -- !query @@ -483,7 +483,7 @@ Project [y#x] : +- OneRowRelation +- Project [(x#x + 1) AS y#x] +- SubqueryAlias q - +- CTERelationRef xxxx, true, [x#x] + +- CTERelationRef xxxx, true, [x#x], false -- !query @@ -497,7 +497,7 @@ Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- OneRowRelation : +- Project [x#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [x#x] +: +- CTERelationRef xxxx, true, [x#x], false +- OneRowRelation @@ -512,7 +512,7 @@ Project [1 IN (list#x []) AS (1 IN (listquery()))#x] : : +- OneRowRelation : +- Project [1#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [1#x] +: +- CTERelationRef xxxx, true, [1#x], false +- OneRowRelation @@ -560,14 +560,14 @@ WithCTE :- Join Inner : :- SubqueryAlias x : : +- SubqueryAlias T1 - : : +- CTERelationRef xxxx, true, [a#x] + : : +- CTERelationRef xxxx, true, [a#x], false : +- SubqueryAlias y : +- Project [b#x] : +- SubqueryAlias T1 - : +- CTERelationRef xxxx, true, [b#x] + : +- CTERelationRef xxxx, true, [b#x], false +- SubqueryAlias z +- SubqueryAlias T1 - +- CTERelationRef xxxx, true, [a#x] + +- CTERelationRef xxxx, true, [a#x], false -- !query @@ -595,9 +595,9 @@ WithCTE +- Project [c#x, a#x] +- Join Inner :- SubqueryAlias ttTT - : +- CTERelationRef xxxx, true, [c#x] + : +- CTERelationRef xxxx, true, [c#x], false +- SubqueryAlias tttT_2 - +- CTERelationRef xxxx, true, [a#x] + +- CTERelationRef xxxx, true, [a#x], false -- !query @@ -613,7 +613,7 @@ Project [scalar-subquery#x [x#x] AS scalarsubquery(x)#x] : : +- OneRowRelation : +- Project [x#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [x#x] +: +- CTERelationRef xxxx, true, [x#x], false +- SubqueryAlias T +- Project [1 AS x#x, 2 AS y#x] +- OneRowRelation @@ -632,7 +632,7 @@ Project [scalar-subquery#x [x#x && y#x] AS scalarsubquery(x, y)#x] : : +- OneRowRelation : +- Project [((outer(x#x) + outer(y#x)) + z#x) AS ((outer(T.x) + outer(T.y)) + z)#x] : +- SubqueryAlias q -: +- CTERelationRef xxxx, true, [z#x] +: +- CTERelationRef xxxx, true, [z#x], false +- SubqueryAlias T +- Project [1 AS x#x, 2 AS y#x] +- OneRowRelation @@ -652,12 +652,12 @@ WithCTE : +- SubqueryAlias q2 : +- Project [x#x] : +- SubqueryAlias q1 -: +- CTERelationRef xxxx, true, [x#x] +: +- CTERelationRef xxxx, true, [x#x], false +- Project [x#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [x#x] +- SubqueryAlias q2 - +- CTERelationRef xxxx, true, [x#x] + +- CTERelationRef xxxx, true, [x#x], false -- !query @@ -674,12 +674,12 @@ WithCTE : +- SubqueryAlias q1 : +- Project [(x#x + 1) AS (x + 1)#x] : +- SubqueryAlias q1 -: +- CTERelationRef xxxx, true, [x#x] +: +- CTERelationRef xxxx, true, [x#x], false +- Project [(x + 1)#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [(x + 1)#x] +- SubqueryAlias q1 - +- CTERelationRef xxxx, true, [(x + 1)#x] + +- CTERelationRef xxxx, true, [(x + 1)#x], false -- !query @@ -720,9 +720,9 @@ WithCTE : +- Aggregate [max(j#x) AS max(j)#x] : +- SubqueryAlias cte2 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [j#x] + : +- CTERelationRef xxxx, true, [j#x], false +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [j#x] + +- CTERelationRef xxxx, true, [j#x], false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out index 4c032b7cbf9a2..2c7b31f62c6f4 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out @@ -1310,10 +1310,10 @@ WithCTE : : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] : : +- LocalRelation [col1#x, col2#x] : +- SubqueryAlias cte1 -: +- CTERelationRef xxxx, true, [c1#x] +: +- CTERelationRef xxxx, true, [c1#x], false +- Project [c1#x, c2#x] +- SubqueryAlias cte2 - +- CTERelationRef xxxx, true, [c1#x, c2#x] + +- CTERelationRef xxxx, true, [c1#x, c2#x], false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out index 305a59f01e443..b80bed6f7c2aa 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out @@ -47,7 +47,7 @@ WithCTE +- Filter (id#xL > scalar-subquery#x []) : +- Aggregate [max(id#xL) AS max(id)#xL] : +- SubqueryAlias tmp - : +- CTERelationRef xxxx, true, [id#xL] + : +- CTERelationRef xxxx, true, [id#xL], false +- Range (0, 3, step=1, splits=None) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out index 6b6a37b4e7fb4..6698d1fb083f0 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out @@ -98,7 +98,7 @@ WithCTE +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, 1)) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] +- Project [x#xL] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [x#xL] + +- CTERelationRef xxxx, true, [x#xL], false -- !query @@ -120,7 +120,7 @@ WithCTE +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as bigint), cast(1 as bigint))) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] +- Project [x#xL] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [x#xL] + +- CTERelationRef xxxx, true, [x#xL], false -- !query @@ -153,7 +153,7 @@ WithCTE +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, 1)) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] +- Project [x#xL] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [x#xL] + +- CTERelationRef xxxx, true, [x#xL], false -- !query @@ -186,7 +186,7 @@ WithCTE +- Window [sum(x#xL) windowspecdefinition(x#xL ASC NULLS FIRST, specifiedwindowframe(RangeFrame, cast(-1 as bigint), cast(1 as bigint))) AS sum(x) OVER (ORDER BY x ASC NULLS FIRST RANGE BETWEEN (- 1) FOLLOWING AND 1 FOLLOWING)#xL], [x#xL ASC NULLS FIRST] +- Project [x#xL] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [x#xL] + +- CTERelationRef xxxx, true, [x#xL], false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out index c978c583152c5..b3ce967f2a6b5 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out @@ -12,10 +12,10 @@ WithCTE +- Project [x#x, y#x, x#x, y#x] +- Join Inner :- SubqueryAlias q1 - : +- CTERelationRef xxxx, true, [x#x, y#x] + : +- CTERelationRef xxxx, true, [x#x, y#x], false +- SubqueryAlias q2 +- SubqueryAlias q1 - +- CTERelationRef xxxx, true, [x#x, y#x] + +- CTERelationRef xxxx, true, [x#x, y#x], false -- !query @@ -194,7 +194,7 @@ WithCTE +- SubqueryAlias q +- Project [foo#x] +- SubqueryAlias cte - +- CTERelationRef xxxx, true, [foo#x] + +- CTERelationRef xxxx, true, [foo#x], false -- !query @@ -222,13 +222,13 @@ WithCTE : +- Union false, false : :- Project [2#x] : : +- SubqueryAlias innermost -: : +- CTERelationRef xxxx, true, [2#x] +: : +- CTERelationRef xxxx, true, [2#x], false : +- Project [3 AS 3#x] : +- OneRowRelation +- Sort [x#x ASC NULLS FIRST], true +- Project [x#x] +- SubqueryAlias outermost - +- CTERelationRef xxxx, true, [x#x] + +- CTERelationRef xxxx, true, [x#x], false -- !query @@ -418,7 +418,7 @@ WithCTE : +- OneRowRelation +- Project [x#x] +- SubqueryAlias ordinality - +- CTERelationRef xxxx, true, [x#x] + +- CTERelationRef xxxx, true, [x#x], false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out index 2cd6ba5356371..cab83b2649974 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out @@ -133,7 +133,7 @@ WithCTE : +- Filter (outer(emp_name#x) = emp_name#x) : +- SubqueryAlias b : +- SubqueryAlias bonus_cte - : +- CTERelationRef xxxx, true, [emp_name#x, bonus_amt#x] + : +- CTERelationRef xxxx, true, [emp_name#x, bonus_amt#x], false +- SubqueryAlias a +- SubqueryAlias bonus +- View (`BONUS`, [emp_name#x,bonus_amt#x]) @@ -189,10 +189,10 @@ WithCTE : +- Join Inner, (dept_id#x = dept_id#x) : :- SubqueryAlias a : : +- SubqueryAlias emp_cte - : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false : +- SubqueryAlias b : +- SubqueryAlias dept_cte - : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x] + : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false +- SubqueryAlias bonus +- View (`BONUS`, [emp_name#x,bonus_amt#x]) +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] @@ -253,10 +253,10 @@ WithCTE : +- Join LeftOuter, (dept_id#x = dept_id#x) : :- SubqueryAlias a : : +- SubqueryAlias emp_cte - : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false : +- SubqueryAlias b : +- SubqueryAlias dept_cte - : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x] + : +- CTERelationRef xxxx, true, [dept_id#x, dept_name#x, state#x], false +- Join Inner :- Join Inner : :- SubqueryAlias b @@ -268,7 +268,7 @@ WithCTE : : +- LocalRelation [emp_name#x, bonus_amt#x] : +- SubqueryAlias e : +- SubqueryAlias emp_cte - : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] + : +- CTERelationRef xxxx, true, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x], false +- SubqueryAlias d +- SubqueryAlias dept +- View (`DEPT`, [dept_id#x,dept_name#x,state#x]) @@ -322,7 +322,7 @@ WithCTE : +- Filter (count(1)#xL > cast(1 as bigint)) : +- Aggregate [dept_id#x], [dept_id#x, max(salary#x) AS max(salary)#x, count(1) AS count(1)#xL] : +- SubqueryAlias empdept - : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x] + : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false +- SubqueryAlias bonus +- View (`BONUS`, [emp_name#x,bonus_amt#x]) +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] @@ -375,7 +375,7 @@ WithCTE : +- Filter (count(1)#xL < cast(1 as bigint)) : +- Aggregate [dept_id#x], [dept_id#x, max(salary#x) AS max(salary)#x, count(1) AS count(1)#xL] : +- SubqueryAlias empdept - : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x] + : +- CTERelationRef xxxx, true, [id#x, salary#x, emp_name#x, dept_id#x], false +- SubqueryAlias bonus +- View (`BONUS`, [emp_name#x,bonus_amt#x]) +- Project [cast(emp_name#x as string) AS emp_name#x, cast(bonus_amt#x as double) AS bonus_amt#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out index ab16f4b9d687c..1717e553f5c3c 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out @@ -330,7 +330,7 @@ WithCTE +- Project [t1a#x, t1b#x, t1a#x, t1b#x] +- Join Inner, (t1b#x = t1b#x) :- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out index 9d82c707177b7..6d0a944bfcfe2 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out @@ -138,7 +138,7 @@ WithCTE : +- Project [t1b#x] : +- Filter (cast(t1b#x as int) > 0) : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false +- SubqueryAlias t1 +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] @@ -197,21 +197,21 @@ WithCTE : : : :- Project [t1b#x] : : : : +- Filter (cast(t1b#x as int) > 0) : : : : +- SubqueryAlias cte1 - : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false : : : +- Project [t1b#x] : : : +- Filter (cast(t1b#x as int) > 5) : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false : : +- Intersect false : : :- Project [t1b#x] : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false : : +- Project [t1b#x] : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false : +- Project [t1b#x] : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false +- SubqueryAlias t1 +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] @@ -268,22 +268,22 @@ WithCTE : : : :- Join FullOuter, (t1c#x = t1c#x) : : : : :- Join Inner, (t1b#x > t1b#x) : : : : : :- SubqueryAlias cte1 - : : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x] + : : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false : : : : : +- SubqueryAlias cte2 : : : : : +- SubqueryAlias cte1 - : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x] + : : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false : : : : +- SubqueryAlias cte3 : : : : +- SubqueryAlias cte1 - : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x] + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false : : : +- SubqueryAlias cte4 : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x] + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false : : +- SubqueryAlias cte5 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x] + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false : +- SubqueryAlias cte6 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x] + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x], false +- SubqueryAlias t1 +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] @@ -354,16 +354,16 @@ WithCTE :- Join FullOuter, (t1a#x = t1a#x) : :- Join Inner, ((cast(t1b#x as int) > 5) AND (t1a#x = t1a#x)) : : :- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false : : +- SubqueryAlias cte2 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false : +- SubqueryAlias cte3 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false +- SubqueryAlias cte4 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false -- !query @@ -424,10 +424,10 @@ WithCTE +- Project [t1a#x, t1b#x] +- Join Inner, (t1h#x >= t1h#x) :- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x] + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x] + +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1h#x], false -- !query @@ -485,16 +485,16 @@ WithCTE :- Join RightOuter, (t1b#x = t1b#x) : :- Join Inner, (t1a#x = t1a#x) : : :- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x] + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false : : +- SubqueryAlias cte2 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x] + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false : +- SubqueryAlias cte3 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x] + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false +- SubqueryAlias cte4 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x] + +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x], false -- !query @@ -538,10 +538,10 @@ WithCTE +- Project [t1a#x, t1b#x] +- Join RightOuter, (t1a#x = t1a#x) :- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false -- !query @@ -599,15 +599,15 @@ WithCTE : : +- SubqueryAlias t1 : : +- LocalRelation [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x, t1f#x, t1g#x, t1h#x, t1i#x] : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false +- SubqueryAlias s +- Project [t1b#x] +- Join LeftOuter, (t1b#x = t1b#x) :- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false +- SubqueryAlias cte2 +- SubqueryAlias cte1 - +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false -- !query @@ -642,7 +642,7 @@ WithCTE : +- Project [t1b#x] : +- Filter (cast(t1b#x as int) < 0) : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x] + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x], false +- SubqueryAlias t1 +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] @@ -722,16 +722,16 @@ WithCTE : :- Join RightOuter, (t1b#x = t1b#x) : : :- Join Inner, (t1a#x = t1a#x) : : : :- SubqueryAlias cte1 - : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x] + : : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false : : : +- SubqueryAlias cte2 : : : +- SubqueryAlias cte1 - : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x] + : : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false : : +- SubqueryAlias cte3 : : +- SubqueryAlias cte1 - : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x] + : : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false : +- SubqueryAlias cte4 : +- SubqueryAlias cte1 - : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x] + : +- CTERelationRef xxxx, true, [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x], false +- SubqueryAlias t1 +- View (`t1`, [t1a#x,t1b#x,t1c#x,t1d#xL,t1e#x,t1f#x,t1g#x,t1h#x,t1i#x]) +- Project [cast(t1a#x as string) AS t1a#x, cast(t1b#x as smallint) AS t1b#x, cast(t1c#x as int) AS t1c#x, cast(t1d#xL as bigint) AS t1d#xL, cast(t1e#x as float) AS t1e#x, cast(t1f#x as double) AS t1f#x, cast(t1g#x as double) AS t1g#x, cast(t1h#x as timestamp) AS t1h#x, cast(t1i#x as date) AS t1i#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out index cb41f7cdc4557..c7271d8b85628 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -623,7 +623,7 @@ Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#x] : : +- OneRowRelation : +- Project [(a#x + outer(c1#x)) AS (a + outer(t1.c1))#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [a#x] +: +- CTERelationRef xxxx, true, [a#x], false +- SubqueryAlias t1 +- View (`t1`, [c1#x,c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -647,7 +647,7 @@ Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : : +- LocalRelation [c1#x, c2#x] : +- Aggregate [sum(c2#x) AS sum(c2)#xL] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [c1#x, c2#x] +: +- CTERelationRef xxxx, true, [c1#x, c2#x], false +- SubqueryAlias t1 +- View (`t1`, [c1#x,c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -677,10 +677,10 @@ Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : : +- Project [c1#x, c2#x] : : +- Filter (outer(c1#x) = c1#x) : : +- SubqueryAlias t3 -: : +- CTERelationRef xxxx, true, [c1#x, c2#x] +: : +- CTERelationRef xxxx, true, [c1#x, c2#x], false : +- Aggregate [sum(c2#x) AS sum(c2)#xL] : +- SubqueryAlias t4 -: +- CTERelationRef xxxx, true, [c1#x, c2#x] +: +- CTERelationRef xxxx, true, [c1#x, c2#x], false +- SubqueryAlias t1 +- View (`t1`, [c1#x,c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -713,10 +713,10 @@ Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : +- Union false, false : :- Project [c1#x, c2#x] : : +- SubqueryAlias t -: : +- CTERelationRef xxxx, true, [c1#x, c2#x] +: : +- CTERelationRef xxxx, true, [c1#x, c2#x], false : +- Project [c2#x, c1#x] : +- SubqueryAlias t -: +- CTERelationRef xxxx, true, [c1#x, c2#x] +: +- CTERelationRef xxxx, true, [c1#x, c2#x], false +- SubqueryAlias t1 +- View (`t1`, [c1#x,c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -756,9 +756,9 @@ WithCTE : : +- Aggregate [sum(c2#x) AS sum(c2)#xL] : : +- Filter (c1#x = outer(c1#x)) : : +- SubqueryAlias t - : : +- CTERelationRef xxxx, true, [c1#x, c2#x] + : : +- CTERelationRef xxxx, true, [c1#x, c2#x], false : +- SubqueryAlias v - : +- CTERelationRef xxxx, true, [c1#x, c2#x] + : +- CTERelationRef xxxx, true, [c1#x, c2#x], false +- SubqueryAlias t1 +- View (`t1`, [c1#x,c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -779,7 +779,7 @@ WithCTE : +- Project [a#x] : +- Filter (a#x = outer(c1#x)) : +- SubqueryAlias t - : +- CTERelationRef xxxx, true, [a#x] + : +- CTERelationRef xxxx, true, [a#x], false +- SubqueryAlias t1 +- View (`t1`, [c1#x,c2#x]) +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] @@ -1027,7 +1027,7 @@ WithCTE : +- Aggregate [sum(1) AS sum(1)#xL] : +- Filter ((a#x = cast(outer(col#x) as int)) OR (upper(cast(outer(col#x) as string)) = Y)) : +- SubqueryAlias T - : +- CTERelationRef xxxx, true, [a#x] + : +- CTERelationRef xxxx, true, [a#x], false +- SubqueryAlias foo +- Project [null AS col#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out index cda76f716a8a8..ceca433a1c915 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out @@ -888,10 +888,10 @@ WithCTE +- Join Inner, (b#x = b#x) :- SubqueryAlias t1 : +- SubqueryAlias temp - : +- CTERelationRef xxxx, true, [b#x] + : +- CTERelationRef xxxx, true, [b#x], false +- SubqueryAlias t2 +- SubqueryAlias temp - +- CTERelationRef xxxx, true, [b#x] + +- CTERelationRef xxxx, true, [b#x], false -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out index 0fe7254d7348c..97410d3cdd369 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out @@ -833,6 +833,6 @@ WithCTE +- Project [coalesce(key#x, key#x) AS key#x, key#x, key#x, key#x] +- Join FullOuter, (key#x = key#x) :- SubqueryAlias t1 - : +- CTERelationRef xxxx, true, [key#x] + : +- CTERelationRef xxxx, true, [key#x], false +- SubqueryAlias t2 - +- CTERelationRef xxxx, true, [key#x] + +- CTERelationRef xxxx, true, [key#x], false diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 4a6325eb06074..8565056cda6fa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -39,7 +39,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Dataset, Row, SaveMode} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Literal, Rand, Randn, Shuffle, Uuid} -import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, CTERelationRef, LocalRelation} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Complete import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.connector.read.InputPartition @@ -1318,6 +1318,51 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi } } + test("SPARK-46062: streaming query reading from CTE, which refers to temp view from " + + "streaming source") { + val inputStream = MemoryStream[Int] + inputStream.toDF().createOrReplaceTempView("tv") + + val df = spark.sql( + """ + |WITH w as ( + | SELECT * FROM tv + |) + |SELECT value from w + |""".stripMargin) + + testStream(df)( + AddData(inputStream, 1, 2, 3), + CheckAnswer(1, 2, 3), + Execute { q => + var isStreamingForCteDef: Option[Boolean] = None + var isStreamingForCteRef: Option[Boolean] = None + + q.analyzedPlan.foreach { + case d: CTERelationDef => + assert(d.resolved, "The definition node must be resolved after analysis.") + isStreamingForCteDef = Some(d.isStreaming) + + case d: CTERelationRef => + assert(d.resolved, "The reference node must be marked as resolved after analysis.") + isStreamingForCteRef = Some(d.isStreaming) + + case _ => + } + + assert(isStreamingForCteDef.isDefined && isStreamingForCteRef.isDefined, + "Both definition and reference for CTE should be available in analyzed plan.") + + assert(isStreamingForCteDef.get, "Expected isStreaming=true for CTE definition, but " + + "isStreaming is set to false.") + + assert(isStreamingForCteDef === isStreamingForCteRef, + "isStreaming flag should be carried over from definition to reference, " + + s"definition: ${isStreamingForCteDef.get}, reference: ${isStreamingForCteRef.get}.") + } + ) + } + private def checkExceptionMessage(df: DataFrame): Unit = { withTempDir { outputDir => withTempDir { checkpointDir => From 351a5f8c004a449013ab25acbcfdd85e9e7868b8 Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Fri, 24 Nov 2023 19:38:31 +0900 Subject: [PATCH 127/521] [SPARK-46016][DOCS][PS] Fix pandas API support list properly ### What changes were proposed in this pull request? This PR proposes to fix a critical issue in the [Supported pandas API documentation](https://spark.apache.org/docs/latest/api/python/user_guide/pandas_on_spark/supported_pandas_api.html) where many essential APIs such as `DataFrame.max`, `DataFrame.min`, `DataFrame.mean`, `and DataFrame.median`, etc. were incorrectly marked as not implemented - marked as "N" - as below: Screenshot 2023-11-24 at 12 37 49 PM The root cause of this issue was that the script used to generate the support list excluded functions inherited from parent classes. For instance, `CategoricalIndex.max` is actually supported by inheriting the `Index` class but was not directly implemented in `CategoricalIndex`, leading to it being marked as unsupported: Screenshot 2023-11-24 at 12 30 08 PM ### Why are the changes needed? The current documentation inaccurately represents the state of supported pandas API, which could significantly hinder user experience and adoption. By correcting these inaccuracies, we ensure that the documentation reflects the true capabilities of Pandas API on Spark, providing users with reliable and accurate information. ### Does this PR introduce _any_ user-facing change? No. This PR only updates the documentation to accurately reflect the current state of supported pandas API. ### How was this patch tested? Manually build documentation, and check if the supported pandas API list is correctly generated as below: Screenshot 2023-11-24 at 12 36 31 PM ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43996 from itholic/fix_supported_api_gen. Authored-by: Haejoon Lee Signed-off-by: Hyukjin Kwon (cherry picked from commit 132bb63a897f4f4049f34deefc065ed3eac6a90f) Signed-off-by: Hyukjin Kwon --- python/pyspark/pandas/supported_api_gen.py | 16 ++-------------- 1 file changed, 2 insertions(+), 14 deletions(-) diff --git a/python/pyspark/pandas/supported_api_gen.py b/python/pyspark/pandas/supported_api_gen.py index 06591c5b26ad6..8c3cdec3671c1 100644 --- a/python/pyspark/pandas/supported_api_gen.py +++ b/python/pyspark/pandas/supported_api_gen.py @@ -138,23 +138,11 @@ def _create_supported_by_module( # module not implemented return {} - pd_funcs = dict( - [ - m - for m in getmembers(pd_module, isfunction) - if not m[0].startswith("_") and m[0] in pd_module.__dict__ - ] - ) + pd_funcs = dict([m for m in getmembers(pd_module, isfunction) if not m[0].startswith("_")]) if not pd_funcs: return {} - ps_funcs = dict( - [ - m - for m in getmembers(ps_module, isfunction) - if not m[0].startswith("_") and m[0] in ps_module.__dict__ - ] - ) + ps_funcs = dict([m for m in getmembers(ps_module, isfunction) if not m[0].startswith("_")]) return _organize_by_implementation_status( module_name, pd_funcs, ps_funcs, pd_module_group, ps_module_group From 132c1a1f08d6555c950600c102db28b9d7581350 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 24 Nov 2023 17:31:22 -0800 Subject: [PATCH 128/521] [SPARK-46095][DOCS] Document `REST API` for Spark Standalone Cluster MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR aims to document `REST API` for Spark Standalone Cluster. To help the users to understand Apache Spark features. No. Manual review. `REST API` Section is added newly. **AFTER** Screenshot 2023-11-24 at 4 13 53 PM No. Closes #44007 from dongjoon-hyun/SPARK-46095. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- docs/spark-standalone.md | 68 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 68 insertions(+) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 9152547f1bec4..e7ea2669a1139 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -446,6 +446,8 @@ Spark applications supports the following configuration properties specific to s # Launching Spark Applications +## Spark Protocol + The [`spark-submit` script](submitting-applications.html) provides the most straightforward way to submit a compiled Spark application to the cluster. For standalone clusters, Spark currently supports two deploy modes. In `client` mode, the driver is launched in the same process as the @@ -468,6 +470,72 @@ failing repeatedly, you may do so through: You can find the driver ID through the standalone Master web UI at `http://:8080`. +## REST API + +If `spark.master.rest.enabled` is enabled, Spark master provides additional REST API +via http://[host:port]/[version]/submissions/[action] where +host is the master host, and +port is the port number specified by `spark.master.rest.port` (default: 6066), and +version is a protocol version, v1 as of today, and +action is one of the following supported actions. + +
Property NameDefaultMeaningSince Version
spark.master.ui.port8080 + Specifies the port number of the Master Web UI endpoint. + 1.1.0
spark.master.ui.decommission.allow.modeLOCAL + Specifies the behavior of the Master Web UI's /workers/kill endpoint. Possible choices + are: LOCAL means allow this endpoint from IP's that are local to the machine running + the Master, DENY means to completely disable this endpoint, ALLOW means to allow + calling this endpoint from any IP. + 3.1.0
spark.master.rest.enabledfalse + Whether to use the Master REST API endpoint or not. + 1.3.0
spark.master.rest.port6066 + Specifies the port number of the Master REST API endpoint. + 1.3.0
spark.deploy.retainedApplications 200
+ + + + + + + + + + + + + + + + + + + +
CommandDescriptionHTTP METHODSince Version
createCreate a Spark driver via cluster mode.POST1.3.0
killKill a single Spark driver.POST1.3.0
statusCheck the status of a Spark job.GET1.3.0
+ +The following is a curl CLI command example with the `pi.py` and REST API. + +```bash +$ curl -XPOST http://IP:PORT/v1/submissions/create \ +--header "Content-Type:application/json;charset=UTF-8" \ +--data '{ + "appResource": "", + "sparkProperties": { + "spark.master": "spark://master:7077", + "spark.app.name": "Spark Pi", + "spark.driver.memory": "1g", + "spark.driver.cores": "1", + "spark.jars": "" + }, + "clientSparkVersion": "", + "mainClass": "org.apache.spark.deploy.SparkSubmit", + "environmentVariables": { }, + "action": "CreateSubmissionRequest", + "appArgs": [ "/opt/spark/examples/src/main/python/pi.py", "10" ] +}' +``` + +The following is the response from the REST API for the above create request. + +```bash +{ + "action" : "CreateSubmissionResponse", + "message" : "Driver successfully submitted as driver-20231124153531-0000", + "serverSparkVersion" : "3.5.1", + "submissionId" : "driver-20231124153531-0000", + "success" : true +} +``` + + # Resource Scheduling The standalone cluster mode currently only supports a simple FIFO scheduler across applications. From e4731e9d3b4443f79a23e7d4bf5b749b54f2e1bb Mon Sep 17 00:00:00 2001 From: wforget <643348094@qq.com> Date: Sun, 26 Nov 2023 23:28:52 +0800 Subject: [PATCH 129/521] [SPARK-45974][SQL] Add scan.filterAttributes non-empty judgment for RowLevelOperationRuntimeGroupFiltering ### What changes were proposed in this pull request? Add scan.filterAttributes non-empty judgment for RowLevelOperationRuntimeGroupFiltering. ### Why are the changes needed? When scan.filterAttributes is empty, an invalid dynamic pruning condition will be generated in RowLevelOperationRuntimeGroupFiltering. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? added test case ### Was this patch authored or co-authored using generative AI tooling? No Closes #43869 from wForget/SPARK-45974. Authored-by: wforget <643348094@qq.com> Signed-off-by: Wenchen Fan (cherry picked from commit ade861d19910df724d9233df98c059ff9d57f795) Signed-off-by: Wenchen Fan --- ...wLevelOperationRuntimeGroupFiltering.scala | 4 ++- .../connector/MergeIntoTableSuiteBase.scala | 32 +++++++++++++++++++ 2 files changed, 35 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelOperationRuntimeGroupFiltering.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelOperationRuntimeGroupFiltering.scala index 7360349284ec1..479e9065c0712 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelOperationRuntimeGroupFiltering.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelOperationRuntimeGroupFiltering.scala @@ -50,7 +50,8 @@ class RowLevelOperationRuntimeGroupFiltering(optimizeSubqueries: Rule[LogicalPla // apply special dynamic filtering only for group-based row-level operations case GroupBasedRowLevelOperation(replaceData, _, Some(cond), DataSourceV2ScanRelation(_, scan: SupportsRuntimeV2Filtering, _, _, _)) - if conf.runtimeRowLevelOperationGroupFilterEnabled && cond != TrueLiteral => + if conf.runtimeRowLevelOperationGroupFilterEnabled && cond != TrueLiteral + && scan.filterAttributes().nonEmpty => // use reference equality on scan to find required scan relations val newQuery = replaceData.query transformUp { @@ -115,6 +116,7 @@ class RowLevelOperationRuntimeGroupFiltering(optimizeSubqueries: Rule[LogicalPla matchingRowsPlan: LogicalPlan, buildKeys: Seq[Attribute], pruningKeys: Seq[Attribute]): Expression = { + assert(buildKeys.nonEmpty && pruningKeys.nonEmpty) val buildQuery = Aggregate(buildKeys, buildKeys, matchingRowsPlan) DynamicPruningExpression( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala index e7555c23fa4fc..5668e5981910c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala @@ -32,6 +32,38 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase { import testImplicits._ + test("SPARK-45974: merge into non filter attributes table") { + val tableName: String = "cat.ns1.non_partitioned_table" + withTable(tableName) { + withTempView("source") { + val sourceRows = Seq( + (1, 100, "hr"), + (2, 200, "finance"), + (3, 300, "hr")) + sourceRows.toDF("pk", "salary", "dep").createOrReplaceTempView("source") + + sql(s"CREATE TABLE $tableName (pk INT NOT NULL, salary INT, dep STRING)".stripMargin) + + val df = sql( + s"""MERGE INTO $tableName t + |USING (select * from source) s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET t.salary = s.salary + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin) + + checkAnswer( + sql(s"SELECT * FROM $tableName"), + Seq( + Row(1, 100, "hr"), // insert + Row(2, 200, "finance"), // insert + Row(3, 300, "hr"))) // insert + } + } + } + test("merge into empty table with NOT MATCHED clause") { withTempView("source") { createTable("pk INT NOT NULL, salary INT, dep STRING") From 92b6619d3ffe3531ac7b11363bf68ad4a6cc8f1e Mon Sep 17 00:00:00 2001 From: Angerszhuuuu Date: Tue, 28 Nov 2023 11:04:14 +0800 Subject: [PATCH 130/521] [SPARK-46006][YARN][FOLLOWUP] YarnAllocator set target executor number to 0 to cancel pending allocate request when driver stop ### What changes were proposed in this pull request? YarnAllocator set target executor number to 0 to cancel pending allocate request when driver stop Now for this issue we do: 1. AllocationFailure should not be treated as exitCausedByApp when driver is shutting down https://github.com/apache/spark/pull/38622 2. Avoid new allocation requests when sc.stop stuck https://github.com/apache/spark/pull/43906 3. Cancel pending allocation request, this pr https://github.com/apache/spark/pull/44036 ### Why are the changes needed? Avoid unnecessary allocate request ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? MT ### Was this patch authored or co-authored using generative AI tooling? No Closes #44036 from AngersZhuuuu/SPARK-46006-FOLLOWUP. Authored-by: Angerszhuuuu Signed-off-by: Kent Yao (cherry picked from commit dbc8756bdac823be42ed10bc011415f405905497) Signed-off-by: Kent Yao --- .../scala/org/apache/spark/deploy/yarn/YarnAllocator.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index f14fc9d5de461..5fccc8c9ff47c 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -385,7 +385,10 @@ private[yarn] class YarnAllocator( this.hostToLocalTaskCountPerResourceProfileId = hostToLocalTaskCountPerResourceProfileId if (resourceProfileToTotalExecs.isEmpty) { - targetNumExecutorsPerResourceProfileId.clear() + // Set target executor number to 0 to cancel pending allocate request. + targetNumExecutorsPerResourceProfileId.keys.foreach { rp => + targetNumExecutorsPerResourceProfileId(rp) = 0 + } allocatorNodeHealthTracker.setSchedulerExcludedNodes(excludedNodes) true } else { From 35ecb32e479a33a1454709d133c48295d6774f3b Mon Sep 17 00:00:00 2001 From: Jiaan Geng Date: Wed, 29 Nov 2023 01:37:35 +0100 Subject: [PATCH 131/521] [SPARK-46029][SQL] Escape the single quote, `_` and `%` for DS V2 pushdown ### What changes were proposed in this pull request? Spark supports push down `startsWith`, `endWith` and `contains` to JDBC database with DS V2 pushdown. But the `V2ExpressionSQLBuilder` didn't escape the single quote, `_` and `%`, it can cause unexpected result. ### Why are the changes needed? Escape the single quote, `_` and `%` for DS V2 pushdown ### Does this PR introduce _any_ user-facing change? 'No'. ### How was this patch tested? Exists test cases. ### Was this patch authored or co-authored using generative AI tooling? 'No'. Closes #43801 from beliefer/SPARK-38432_followup3. Authored-by: Jiaan Geng Signed-off-by: Wenchen Fan (cherry picked from commit d2cd98bdd32446b4106e66eb099efd8fb47acf40) Signed-off-by: Wenchen Fan --- .../util/V2ExpressionSQLBuilder.java | 35 +++++- .../org/apache/spark/sql/jdbc/H2Dialect.scala | 8 ++ .../datasources/v2/V2PredicateSuite.scala | 6 +- .../apache/spark/sql/jdbc/JDBCV2Suite.scala | 113 +++++++++++++++++- 4 files changed, 151 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java index 9ca0fe4787f10..dcb3c706946c5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java @@ -48,6 +48,35 @@ */ public class V2ExpressionSQLBuilder { + /** + * Escape the special chars for like pattern. + * + * Note: This method adopts the escape representation within Spark and is not bound to any JDBC + * dialect. JDBC dialect should overwrite this API if the underlying database have more special + * chars other than _ and %. + */ + protected String escapeSpecialCharsForLikePattern(String str) { + StringBuilder builder = new StringBuilder(); + + for (char c : str.toCharArray()) { + switch (c) { + case '_': + builder.append("\\_"); + break; + case '%': + builder.append("\\%"); + break; + case '\'': + builder.append("\\\'"); + break; + default: + builder.append(c); + } + } + + return builder.toString(); + } + public String build(Expression expr) { if (expr instanceof Literal) { return visitLiteral((Literal) expr); @@ -247,21 +276,21 @@ protected String visitStartsWith(String l, String r) { // Remove quotes at the beginning and end. // e.g. converts "'str'" to "str". String value = r.substring(1, r.length() - 1); - return l + " LIKE '" + value + "%'"; + return l + " LIKE '" + escapeSpecialCharsForLikePattern(value) + "%' ESCAPE '\\'"; } protected String visitEndsWith(String l, String r) { // Remove quotes at the beginning and end. // e.g. converts "'str'" to "str". String value = r.substring(1, r.length() - 1); - return l + " LIKE '%" + value + "'"; + return l + " LIKE '%" + escapeSpecialCharsForLikePattern(value) + "' ESCAPE '\\'"; } protected String visitContains(String l, String r) { // Remove quotes at the beginning and end. // e.g. converts "'str'" to "str". String value = r.substring(1, r.length() - 1); - return l + " LIKE '%" + value + "%'"; + return l + " LIKE '%" + escapeSpecialCharsForLikePattern(value) + "%' ESCAPE '\\'"; } private String inputToSQL(Expression input) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala index c246b50f4e156..8471a49153ff4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala @@ -240,6 +240,14 @@ private[sql] object H2Dialect extends JdbcDialect { } class H2SQLBuilder extends JDBCSQLBuilder { + override def escapeSpecialCharsForLikePattern(str: String): String = { + str.map { + case '_' => "\\_" + case '%' => "\\%" + case c => c.toString + }.mkString + } + override def visitAggregateFunction( funcName: String, isDistinct: Boolean, inputs: Array[String]): String = if (isDistinct && distinctUnsupportedAggregateFunctions.contains(funcName)) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2PredicateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2PredicateSuite.scala index a5fee51dc916f..4a8a231cc54ca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2PredicateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2PredicateSuite.scala @@ -315,7 +315,7 @@ class V2PredicateSuite extends SparkFunSuite { Array[Expression](ref("a"), literal)) assert(predicate1.equals(predicate2)) assert(predicate1.references.map(_.describe()).toSeq == Seq("a")) - assert(predicate1.describe.equals("a LIKE 'str%'")) + assert(predicate1.describe.equals(raw"a LIKE 'str%' ESCAPE '\'")) val v1Filter = StringStartsWith("a", "str") assert(v1Filter.toV2.equals(predicate1)) @@ -332,7 +332,7 @@ class V2PredicateSuite extends SparkFunSuite { Array[Expression](ref("a"), literal)) assert(predicate1.equals(predicate2)) assert(predicate1.references.map(_.describe()).toSeq == Seq("a")) - assert(predicate1.describe.equals("a LIKE '%str'")) + assert(predicate1.describe.equals(raw"a LIKE '%str' ESCAPE '\'")) val v1Filter = StringEndsWith("a", "str") assert(v1Filter.toV2.equals(predicate1)) @@ -349,7 +349,7 @@ class V2PredicateSuite extends SparkFunSuite { Array[Expression](ref("a"), literal)) assert(predicate1.equals(predicate2)) assert(predicate1.references.map(_.describe()).toSeq == Seq("a")) - assert(predicate1.describe.equals("a LIKE '%str%'")) + assert(predicate1.describe.equals(raw"a LIKE '%str%' ESCAPE '\'")) val v1Filter = StringContains("a", "str") assert(v1Filter.toV2.equals(predicate1)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala index ae0cfe17b11f5..51a15881088b5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala @@ -185,6 +185,19 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel conn.prepareStatement("INSERT INTO \"test\".\"datetime\" VALUES " + "('alex', '2022-05-18', '2022-05-18 00:00:00')").executeUpdate() + conn.prepareStatement( + "CREATE TABLE \"test\".\"address\" (email TEXT(32) NOT NULL)").executeUpdate() + conn.prepareStatement("INSERT INTO \"test\".\"address\" VALUES " + + "('abc_def@gmail.com')").executeUpdate() + conn.prepareStatement("INSERT INTO \"test\".\"address\" VALUES " + + "('abc%def@gmail.com')").executeUpdate() + conn.prepareStatement("INSERT INTO \"test\".\"address\" VALUES " + + "('abc%_def@gmail.com')").executeUpdate() + conn.prepareStatement("INSERT INTO \"test\".\"address\" VALUES " + + "('abc_%def@gmail.com')").executeUpdate() + conn.prepareStatement("INSERT INTO \"test\".\"address\" VALUES " + + "('abc_''%def@gmail.com')").executeUpdate() + conn.prepareStatement("CREATE TABLE \"test\".\"binary1\" (name TEXT(32),b BINARY(20))") .executeUpdate() val stmt = conn.prepareStatement("INSERT INTO \"test\".\"binary1\" VALUES (?, ?)") @@ -1096,7 +1109,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel val df3 = spark.table("h2.test.employee").filter($"name".startsWith("a")) checkFiltersRemoved(df3) - checkPushedInfo(df3, "PushedFilters: [NAME IS NOT NULL, NAME LIKE 'a%']") + checkPushedInfo(df3, raw"PushedFilters: [NAME IS NOT NULL, NAME LIKE 'a%' ESCAPE '\']") checkAnswer(df3, Seq(Row(1, "amy", 10000, 1000, true), Row(2, "alex", 12000, 1200, false))) val df4 = spark.table("h2.test.employee").filter($"is_manager") @@ -1240,6 +1253,94 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkAnswer(df17, Seq(Row(6, "jen", 12000, 1200, true))) } + test("SPARK-38432: escape the single quote, _ and % for DS V2 pushdown") { + val df1 = spark.table("h2.test.address").filter($"email".startsWith("abc_")) + checkFiltersRemoved(df1) + checkPushedInfo(df1, raw"PushedFilters: [EMAIL IS NOT NULL, EMAIL LIKE 'abc\_%' ESCAPE '\']") + checkAnswer(df1, + Seq(Row("abc_%def@gmail.com"), Row("abc_'%def@gmail.com"), Row("abc_def@gmail.com"))) + + val df2 = spark.table("h2.test.address").filter($"email".startsWith("abc%")) + checkFiltersRemoved(df2) + checkPushedInfo(df2, raw"PushedFilters: [EMAIL IS NOT NULL, EMAIL LIKE 'abc\%%' ESCAPE '\']") + checkAnswer(df2, Seq(Row("abc%_def@gmail.com"), Row("abc%def@gmail.com"))) + + val df3 = spark.table("h2.test.address").filter($"email".startsWith("abc%_")) + checkFiltersRemoved(df3) + checkPushedInfo(df3, raw"PushedFilters: [EMAIL IS NOT NULL, EMAIL LIKE 'abc\%\_%' ESCAPE '\']") + checkAnswer(df3, Seq(Row("abc%_def@gmail.com"))) + + val df4 = spark.table("h2.test.address").filter($"email".startsWith("abc_%")) + checkFiltersRemoved(df4) + checkPushedInfo(df4, raw"PushedFilters: [EMAIL IS NOT NULL, EMAIL LIKE 'abc\_\%%' ESCAPE '\']") + checkAnswer(df4, Seq(Row("abc_%def@gmail.com"))) + + val df5 = spark.table("h2.test.address").filter($"email".startsWith("abc_'%")) + checkFiltersRemoved(df5) + checkPushedInfo(df5, + raw"PushedFilters: [EMAIL IS NOT NULL, EMAIL LIKE 'abc\_\'\%%' ESCAPE '\']") + checkAnswer(df5, Seq(Row("abc_'%def@gmail.com"))) + + val df6 = spark.table("h2.test.address").filter($"email".endsWith("_def@gmail.com")) + checkFiltersRemoved(df6) + checkPushedInfo(df6, + raw"PushedFilters: [EMAIL IS NOT NULL, EMAIL LIKE '%\_def@gmail.com' ESCAPE '\']") + checkAnswer(df6, Seq(Row("abc%_def@gmail.com"), Row("abc_def@gmail.com"))) + + val df7 = spark.table("h2.test.address").filter($"email".endsWith("%def@gmail.com")) + checkFiltersRemoved(df7) + checkPushedInfo(df7, + raw"PushedFilters: [EMAIL IS NOT NULL, EMAIL LIKE '%\%def@gmail.com' ESCAPE '\']") + checkAnswer(df7, + Seq(Row("abc%def@gmail.com"), Row("abc_%def@gmail.com"), Row("abc_'%def@gmail.com"))) + + val df8 = spark.table("h2.test.address").filter($"email".endsWith("%_def@gmail.com")) + checkFiltersRemoved(df8) + checkPushedInfo(df8, + raw"PushedFilters: [EMAIL IS NOT NULL, EMAIL LIKE '%\%\_def@gmail.com' ESCAPE '\']") + checkAnswer(df8, Seq(Row("abc%_def@gmail.com"))) + + val df9 = spark.table("h2.test.address").filter($"email".endsWith("_%def@gmail.com")) + checkFiltersRemoved(df9) + checkPushedInfo(df9, + raw"PushedFilters: [EMAIL IS NOT NULL, EMAIL LIKE '%\_\%def@gmail.com' ESCAPE '\']") + checkAnswer(df9, Seq(Row("abc_%def@gmail.com"))) + + val df10 = spark.table("h2.test.address").filter($"email".endsWith("_'%def@gmail.com")) + checkFiltersRemoved(df10) + checkPushedInfo(df10, + raw"PushedFilters: [EMAIL IS NOT NULL, EMAIL LIKE '%\_\'\%def@gmail.com' ESCAPE '\']") + checkAnswer(df10, Seq(Row("abc_'%def@gmail.com"))) + + val df11 = spark.table("h2.test.address").filter($"email".contains("c_d")) + checkFiltersRemoved(df11) + checkPushedInfo(df11, raw"PushedFilters: [EMAIL IS NOT NULL, EMAIL LIKE '%c\_d%' ESCAPE '\']") + checkAnswer(df11, Seq(Row("abc_def@gmail.com"))) + + val df12 = spark.table("h2.test.address").filter($"email".contains("c%d")) + checkFiltersRemoved(df12) + checkPushedInfo(df12, raw"PushedFilters: [EMAIL IS NOT NULL, EMAIL LIKE '%c\%d%' ESCAPE '\']") + checkAnswer(df12, Seq(Row("abc%def@gmail.com"))) + + val df13 = spark.table("h2.test.address").filter($"email".contains("c%_d")) + checkFiltersRemoved(df13) + checkPushedInfo(df13, + raw"PushedFilters: [EMAIL IS NOT NULL, EMAIL LIKE '%c\%\_d%' ESCAPE '\']") + checkAnswer(df13, Seq(Row("abc%_def@gmail.com"))) + + val df14 = spark.table("h2.test.address").filter($"email".contains("c_%d")) + checkFiltersRemoved(df14) + checkPushedInfo(df14, + raw"PushedFilters: [EMAIL IS NOT NULL, EMAIL LIKE '%c\_\%d%' ESCAPE '\']") + checkAnswer(df14, Seq(Row("abc_%def@gmail.com"))) + + val df15 = spark.table("h2.test.address").filter($"email".contains("c_'%d")) + checkFiltersRemoved(df15) + checkPushedInfo(df15, + raw"PushedFilters: [EMAIL IS NOT NULL, EMAIL LIKE '%c\_\'\%d%' ESCAPE '\']") + checkAnswer(df15, Seq(Row("abc_'%def@gmail.com"))) + } + test("scan with filter push-down with ansi mode") { Seq(false, true).foreach { ansiMode => withSQLConf(SQLConf.ANSI_ENABLED.key -> ansiMode.toString) { @@ -1325,10 +1426,11 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkFiltersRemoved(df6, ansiMode) val expectedPlanFragment6 = if (ansiMode) { "PushedFilters: [BONUS IS NOT NULL, DEPT IS NOT NULL, " + - "CAST(BONUS AS string) LIKE '%30%', CAST(DEPT AS byte) > 1, " + + raw"CAST(BONUS AS string) LIKE '%30%' ESCAPE '\', CAST(DEPT AS byte) > 1, " + "CAST(DEPT AS short) > 1, CAST(BONUS AS decimal(20,2)) > 1200.00]" } else { - "PushedFilters: [BONUS IS NOT NULL, DEPT IS NOT NULL, CAST(BONUS AS string) LIKE '%30%']" + "PushedFilters: [BONUS IS NOT NULL, " + + raw"DEPT IS NOT NULL, CAST(BONUS AS string) LIKE '%30%' ESCAPE '\']" } checkPushedInfo(df6, expectedPlanFragment6) checkAnswer(df6, Seq(Row(2, "david", 10000, 1300, true))) @@ -1538,8 +1640,9 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel test("show tables") { checkAnswer(sql("SHOW TABLES IN h2.test"), - Seq(Row("test", "people", false), Row("test", "empty_table", false), - Row("test", "employee", false), Row("test", "item", false), Row("test", "dept", false), + Seq(Row("test", "address", false), Row("test", "people", false), + Row("test", "empty_table", false), Row("test", "employee", false), + Row("test", "item", false), Row("test", "dept", false), Row("test", "person", false), Row("test", "view1", false), Row("test", "view2", false), Row("test", "datetime", false), Row("test", "binary1", false))) } From c2342ba0e4ded541916eeb3478a1db2c129d3130 Mon Sep 17 00:00:00 2001 From: wforget <643348094@qq.com> Date: Thu, 30 Nov 2023 11:07:47 +0900 Subject: [PATCH 132/521] [SPARK-45943][SQL] Move DetermineTableStats to resolution rules ### What changes were proposed in this pull request? Move DetermineTableStats to resolution rules. ### Why are the changes needed? `MergeIntoTable#sourceTable` is used for `ReplaceData#groupFilterCondition` in `RewriteMergeIntoTable`, SourceTable in `ReplaceData#groupFilterCondition` is resolved and will not be applied to `DetermineTableStats` through `ResolveSubquery#resolveSubQueries`. So, when there is a hive table without stats in `MergeIntoTable#sourceTable`, IllegalStateException will occur. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? added test case ### Was this patch authored or co-authored using generative AI tooling? No Closes #43867 from wForget/SPARK-45943. Authored-by: wforget <643348094@qq.com> Signed-off-by: Hyukjin Kwon (cherry picked from commit d1aea92daf254334bcbd6d96901a54a2502eda29) Signed-off-by: Hyukjin Kwon --- .../sql/hive/HiveSessionStateBuilder.scala | 2 +- .../HiveSourceRowLevelOperationSuite.scala | 72 +++++++++++++++++++ 2 files changed, 73 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/connector/HiveSourceRowLevelOperationSuite.scala diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index 2d0bcdff07151..08e02c90ebd63 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -92,11 +92,11 @@ class HiveSessionStateBuilder( new ResolveSessionCatalog(catalogManager) +: ResolveWriteToStream +: new EvalSubqueriesForTimeTravel +: + new DetermineTableStats(session) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = DetectAmbiguousSelfJoin +: - new DetermineTableStats(session) +: RelationConversions(catalog) +: QualifyLocationWithWarehouse(catalog) +: PreprocessTableCreation(catalog) +: diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/connector/HiveSourceRowLevelOperationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/connector/HiveSourceRowLevelOperationSuite.scala new file mode 100644 index 0000000000000..344fdc21fe2cf --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/connector/HiveSourceRowLevelOperationSuite.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.hive.connector + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.connector.catalog.InMemoryRowLevelOperationTableCatalog +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils + +class HiveSourceRowLevelOperationSuite extends QueryTest with TestHiveSingleton + with BeforeAndAfter with SQLTestUtils { + + before { + spark.conf.set("spark.sql.catalog.cat", classOf[InMemoryRowLevelOperationTableCatalog].getName) + } + + after { + spark.sessionState.catalogManager.reset() + spark.sessionState.conf.unsetConf("spark.sql.catalog.cat") + } + + test("SPARK-45943: merge into using hive table without stats") { + val inMemCatNs = "cat.ns1" + val inMemCatTable = "in_mem_cat_table" + withTable("hive_table", s"$inMemCatNs.$inMemCatTable") { + // create hive table without stats + sql("create table hive_table(pk int, salary int, dep string)") + + sql( + s""" + |create table $inMemCatNs.$inMemCatTable ( + | pk INT NOT NULL, + | salary INT, + | dep STRING) + |PARTITIONED BY (dep) + | """.stripMargin) + + try { + // three-part naming is not supported in + // org.apache.spark.sql.hive.test.TestHiveQueryExecution.analyzed.{referencedTables} + sql(s"use $inMemCatNs") + sql( + s"""MERGE INTO $inMemCatTable t + |USING (SELECT pk, salary, dep FROM spark_catalog.default.hive_table) s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET t.salary = s.salary + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin) + } finally { + sql("use spark_catalog.default") + } + } + } +} From 00bb4ad46e373311a6303952f3944680b08e03d7 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 30 Nov 2023 14:56:48 -0800 Subject: [PATCH 133/521] [SPARK-46188][DOC][3.5] Fix the CSS of Spark doc's generated tables ### What changes were proposed in this pull request? After https://github.com/apache/spark/pull/40269, there is no border in the generated tables of Spark doc(for example, https://spark.apache.org/docs/latest/sql-ref-ansi-compliance.html) . This PR is to fix it by restoring part of the table style in https://github.com/apache/spark/pull/40269/files#diff-309b964023ca899c9505205f36d3f4d5b36a6487e5c9b2e242204ee06bbc9ce9L26 This PR also unifies all the styles of tables by removing the `class="table table-striped"` in HTML style tables in markdown docs. ### Why are the changes needed? Fix a regression in the table CSS of Spark docs ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manually build docs and verify. Before changes: image After changes: image ### Was this patch authored or co-authored using generative AI tooling? Generated-by: ChatGPT 4 Closes #44097 from gengliangwang/fixTable3.5. Authored-by: Gengliang Wang Signed-off-by: Gengliang Wang --- docs/building-spark.md | 2 +- docs/cluster-overview.md | 2 +- docs/configuration.md | 40 +++++++++---------- docs/css/custom.css | 13 ++++++ docs/ml-classification-regression.md | 14 +++---- docs/ml-clustering.md | 8 ++-- docs/mllib-classification-regression.md | 2 +- docs/mllib-decision-tree.md | 2 +- docs/mllib-ensembles.md | 2 +- docs/mllib-evaluation-metrics.md | 10 ++--- docs/mllib-linear-methods.md | 4 +- docs/mllib-pmml-model-export.md | 2 +- docs/monitoring.md | 10 ++--- docs/rdd-programming-guide.md | 8 ++-- docs/running-on-kubernetes.md | 8 ++-- docs/running-on-mesos.md | 2 +- docs/running-on-yarn.md | 8 ++-- docs/security.md | 26 ++++++------ docs/spark-standalone.md | 12 +++--- docs/sparkr.md | 6 +-- docs/sql-data-sources-avro.md | 12 +++--- docs/sql-data-sources-csv.md | 2 +- docs/sql-data-sources-hive-tables.md | 4 +- docs/sql-data-sources-jdbc.md | 2 +- docs/sql-data-sources-json.md | 2 +- docs/sql-data-sources-load-save-functions.md | 2 +- docs/sql-data-sources-orc.md | 4 +- docs/sql-data-sources-parquet.md | 4 +- docs/sql-data-sources-text.md | 2 +- ...ql-distributed-sql-engine-spark-sql-cli.md | 4 +- docs/sql-error-conditions-sqlstates.md | 26 ++++++------ docs/sql-migration-guide.md | 4 +- docs/sql-performance-tuning.md | 16 ++++---- docs/storage-openstack-swift.md | 2 +- docs/streaming-custom-receivers.md | 2 +- docs/streaming-programming-guide.md | 10 ++--- .../structured-streaming-kafka-integration.md | 20 +++++----- .../structured-streaming-programming-guide.md | 12 +++--- docs/submitting-applications.md | 2 +- docs/web-ui.md | 2 +- 40 files changed, 164 insertions(+), 151 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index 4b8e70655d59c..33d253a49dbf3 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -286,7 +286,7 @@ If use an individual repository or a repository on GitHub Enterprise, export bel ### Related environment variables - +
diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index 7da06a852089e..34913bd97a418 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -91,7 +91,7 @@ The [job scheduling overview](job-scheduling.html) describes this in more detail The following table summarizes terms you'll see used to refer to cluster concepts: -
Variable NameDefaultMeaning
SPARK_PROJECT_URL
+
diff --git a/docs/configuration.md b/docs/configuration.md index 4604360dda287..248f9333c9a3b 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -135,7 +135,7 @@ of the most common options to set are: ### Application Properties -
TermMeaning
+
@@ -520,7 +520,7 @@ Apart from these, the following properties are also available, and may be useful ### Runtime Environment -
Property NameDefaultMeaningSince Version
spark.app.name
+
@@ -907,7 +907,7 @@ Apart from these, the following properties are also available, and may be useful ### Shuffle Behavior -
Property NameDefaultMeaningSince Version
spark.driver.extraClassPath
+
@@ -1282,7 +1282,7 @@ Apart from these, the following properties are also available, and may be useful ### Spark UI -
Property NameDefaultMeaningSince Version
spark.reducer.maxSizeInFlight
+
@@ -1674,7 +1674,7 @@ Apart from these, the following properties are also available, and may be useful ### Compression and Serialization -
Property NameDefaultMeaningSince Version
spark.eventLog.logBlockUpdates.enabled
+
@@ -1872,7 +1872,7 @@ Apart from these, the following properties are also available, and may be useful ### Memory Management -
Property NameDefaultMeaningSince Version
spark.broadcast.compress
+
@@ -1997,7 +1997,7 @@ Apart from these, the following properties are also available, and may be useful ### Execution Behavior -
Property NameDefaultMeaningSince Version
spark.memory.fraction
+
@@ -2247,7 +2247,7 @@ Apart from these, the following properties are also available, and may be useful ### Executor Metrics -
Property NameDefaultMeaningSince Version
spark.broadcast.blockSize
+
@@ -2315,7 +2315,7 @@ Apart from these, the following properties are also available, and may be useful ### Networking -
Property NameDefaultMeaningSince Version
spark.eventLog.logStageExecutorMetrics
+
@@ -2478,7 +2478,7 @@ Apart from these, the following properties are also available, and may be useful ### Scheduling -
Property NameDefaultMeaningSince Version
spark.rpc.message.maxSize
+
@@ -2962,7 +2962,7 @@ Apart from these, the following properties are also available, and may be useful ### Barrier Execution Mode -
Property NameDefaultMeaningSince Version
spark.cores.max
+
@@ -3009,7 +3009,7 @@ Apart from these, the following properties are also available, and may be useful ### Dynamic Allocation -
Property NameDefaultMeaningSince Version
spark.barrier.sync.timeout
+
@@ -3151,7 +3151,7 @@ finer granularity starting from driver and executor. Take RPC module as example like shuffle, just replace "rpc" with "shuffle" in the property names except spark.{driver|executor}.rpc.netty.dispatcher.numThreads, which is only for RPC module. -
Property NameDefaultMeaningSince Version
spark.dynamicAllocation.enabled
+
@@ -3294,7 +3294,7 @@ External users can query the static sql config values via `SparkSession.conf` or ### Spark Streaming -
Property NameDefaultMeaningSince Version
spark.{driver|executor}.rpc.io.serverThreads
+
@@ -3426,7 +3426,7 @@ External users can query the static sql config values via `SparkSession.conf` or ### SparkR -
Property NameDefaultMeaningSince Version
spark.streaming.backpressure.enabled
+
@@ -3482,7 +3482,7 @@ External users can query the static sql config values via `SparkSession.conf` or ### GraphX -
Property NameDefaultMeaningSince Version
spark.r.numRBackendThreads
+
@@ -3497,7 +3497,7 @@ External users can query the static sql config values via `SparkSession.conf` or ### Deploy -
Property NameDefaultMeaningSince Version
spark.graphx.pregel.checkpointInterval
+
@@ -3547,7 +3547,7 @@ copy `conf/spark-env.sh.template` to create it. Make sure you make the copy exec The following variables can be set in `spark-env.sh`: -
Property NameDefaultMeaningSince Version
spark.deploy.recoveryMode
+
@@ -3684,7 +3684,7 @@ Push-based shuffle helps improve the reliability and performance of spark shuffl ### External Shuffle service(server) side configuration options -
Environment VariableMeaning
JAVA_HOME
+
@@ -3718,7 +3718,7 @@ Push-based shuffle helps improve the reliability and performance of spark shuffl ### Client side configuration options -
Property NameDefaultMeaningSince Version
spark.shuffle.push.server.mergedShuffleFileManagerImpl
+
diff --git a/docs/css/custom.css b/docs/css/custom.css index 4576f45d1ab7d..e7416d9ded618 100644 --- a/docs/css/custom.css +++ b/docs/css/custom.css @@ -1110,5 +1110,18 @@ img { table { width: 100%; overflow-wrap: normal; + border-collapse: collapse; /* Ensures that the borders collapse into a single border */ } +table th, table td { + border: 1px solid #cccccc; /* Adds a border to each table header and data cell */ + padding: 6px 13px; /* Optional: Adds padding inside each cell for better readability */ +} + +table tr { + background-color: white; /* Sets a default background color for all rows */ +} + +table tr:nth-child(2n) { + background-color: #F1F4F5; /* Sets a different background color for even rows */ +} diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index d184f4fe0257c..604b3245272fc 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -703,7 +703,7 @@ others. ### Available families -
Property NameDefaultMeaningSince Version
spark.shuffle.push.enabled
+
@@ -1224,7 +1224,7 @@ All output columns are optional; to exclude an output column, set its correspond ### Input Columns -
Family
+
@@ -1251,7 +1251,7 @@ All output columns are optional; to exclude an output column, set its correspond ### Output Columns -
Param name
+
@@ -1326,7 +1326,7 @@ All output columns are optional; to exclude an output column, set its correspond #### Input Columns -
Param name
+
@@ -1353,7 +1353,7 @@ All output columns are optional; to exclude an output column, set its correspond #### Output Columns (Predictions) -
Param name
+
@@ -1407,7 +1407,7 @@ All output columns are optional; to exclude an output column, set its correspond #### Input Columns -
Param name
+
@@ -1436,7 +1436,7 @@ Note that `GBTClassifier` currently only supports binary labels. #### Output Columns (Predictions) -
Param name
+
diff --git a/docs/ml-clustering.md b/docs/ml-clustering.md index 00a156b6645ce..fdb8173ce3bbe 100644 --- a/docs/ml-clustering.md +++ b/docs/ml-clustering.md @@ -40,7 +40,7 @@ called [kmeans||](http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf). ### Input Columns -
Param name
+
@@ -61,7 +61,7 @@ called [kmeans||](http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf). ### Output Columns -
Param name
+
@@ -204,7 +204,7 @@ model. ### Input Columns -
Param name
+
@@ -225,7 +225,7 @@ model. ### Output Columns -
Param name
+
diff --git a/docs/mllib-classification-regression.md b/docs/mllib-classification-regression.md index 10cb85e392029..b3305314abc56 100644 --- a/docs/mllib-classification-regression.md +++ b/docs/mllib-classification-regression.md @@ -26,7 +26,7 @@ classification](http://en.wikipedia.org/wiki/Multiclass_classification), and [regression analysis](http://en.wikipedia.org/wiki/Regression_analysis). The table below outlines the supported algorithms for each type of problem. -
Param name
+
diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 174255c48b699..0d9886315e288 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -51,7 +51,7 @@ The *node impurity* is a measure of the homogeneity of the labels at the node. T implementation provides two impurity measures for classification (Gini impurity and entropy) and one impurity measure for regression (variance). -
Problem TypeSupported Methods
+
diff --git a/docs/mllib-ensembles.md b/docs/mllib-ensembles.md index b1006f2730db5..fdad7ae68dd49 100644 --- a/docs/mllib-ensembles.md +++ b/docs/mllib-ensembles.md @@ -191,7 +191,7 @@ Note that each loss is applicable to one of classification or regression, not bo Notation: $N$ = number of instances. $y_i$ = label of instance $i$. $x_i$ = features of instance $i$. $F(x_i)$ = model's predicted label for instance $i$. -
ImpurityTaskFormulaDescription
+
diff --git a/docs/mllib-evaluation-metrics.md b/docs/mllib-evaluation-metrics.md index f82f6a01136b9..30acc3dc634be 100644 --- a/docs/mllib-evaluation-metrics.md +++ b/docs/mllib-evaluation-metrics.md @@ -76,7 +76,7 @@ plots (recall, false positive rate) points. **Available metrics** -
LossTaskFormulaDescription
+
@@ -179,7 +179,7 @@ For this section, a modified delta function $\hat{\delta}(x)$ will prove useful $$\hat{\delta}(x) = \begin{cases}1 & \text{if $x = 0$}, \\ 0 & \text{otherwise}.\end{cases}$$ -
MetricDefinition
+
@@ -296,7 +296,7 @@ The following definition of indicator function $I_A(x)$ on a set $A$ will be nec $$I_A(x) = \begin{cases}1 & \text{if $x \in A$}, \\ 0 & \text{otherwise}.\end{cases}$$ -
MetricDefinition
+
@@ -447,7 +447,7 @@ documents, returns a relevance score for the recommended document. $$rel_D(r) = \begin{cases}1 & \text{if $r \in D$}, \\ 0 & \text{otherwise}.\end{cases}$$ -
MetricDefinition
+
@@ -553,7 +553,7 @@ variable from a number of independent variables. **Available metrics** -
MetricDefinitionNotes
+
diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index b535d2de307a9..448d881f794a5 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -72,7 +72,7 @@ training error) and minimizing model complexity (i.e., to avoid overfitting). The following table summarizes the loss functions and their gradients or sub-gradients for the methods `spark.mllib` supports: -
MetricDefinition
+
@@ -105,7 +105,7 @@ The purpose of the encourage simple models and avoid overfitting. We support the following regularizers in `spark.mllib`: -
loss function $L(\wv; \x, y)$gradient or sub-gradient
+
diff --git a/docs/mllib-pmml-model-export.md b/docs/mllib-pmml-model-export.md index e20d7c2fe4e17..02b5fda7a36df 100644 --- a/docs/mllib-pmml-model-export.md +++ b/docs/mllib-pmml-model-export.md @@ -28,7 +28,7 @@ license: | The table below outlines the `spark.mllib` models that can be exported to PMML and their equivalent PMML model. -
regularizer $R(\wv)$gradient or sub-gradient
+
diff --git a/docs/monitoring.md b/docs/monitoring.md index 91b158bf85d26..e90ef46bdffe0 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -69,7 +69,7 @@ The history server can be configured as follows: ### Environment Variables -
spark.mllib modelPMML model
+
@@ -145,7 +145,7 @@ Use it with caution. Security options for the Spark History Server are covered more detail in the [Security](security.html#web-ui) page. -
Environment VariableMeaning
SPARK_DAEMON_MEMORY
+
@@ -470,7 +470,7 @@ only for applications in cluster mode, not applications in client mode. Applicat can be identified by their `[attempt-id]`. In the API listed below, when running in YARN cluster mode, `[app-id]` will actually be `[base-app-id]/[attempt-id]`, where `[base-app-id]` is the YARN application ID. -
Property Name
+
@@ -669,7 +669,7 @@ The REST API exposes the values of the Task Metrics collected by Spark executors of task execution. The metrics can be used for performance troubleshooting and workload characterization. A list of the available metrics, with a short description: -
EndpointMeaning
/applications
+
@@ -827,7 +827,7 @@ In addition, aggregated per-stage peak values of the executor memory metrics are Executor memory metrics are also exposed via the Spark metrics system based on the [Dropwizard metrics library](https://metrics.dropwizard.io/4.2.0). A list of the available metrics, with a short description: -
Spark Executor Task Metric name
+
diff --git a/docs/rdd-programming-guide.md b/docs/rdd-programming-guide.md index aee22ad484e60..cc897aea06c93 100644 --- a/docs/rdd-programming-guide.md +++ b/docs/rdd-programming-guide.md @@ -378,7 +378,7 @@ resulting Java objects using [pickle](https://github.com/irmen/pickle/). When sa PySpark does the reverse. It unpickles Python objects into Java objects and then converts them to Writables. The following Writables are automatically converted: -
Executor Level Metric name Short description
+
@@ -954,7 +954,7 @@ and pair RDD functions doc [Java](api/java/index.html?org/apache/spark/api/java/JavaPairRDD.html)) for details. -
Writable TypePython Type
Textstr
IntWritableint
+
@@ -1069,7 +1069,7 @@ and pair RDD functions doc [Java](api/java/index.html?org/apache/spark/api/java/JavaPairRDD.html)) for details. -
TransformationMeaning
map(func)
+
@@ -1214,7 +1214,7 @@ to `persist()`. The `cache()` method is a shorthand for using the default storag which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The full set of storage levels is: -
ActionMeaning
reduce(func)
+
diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 38a745f1afca3..a684e7caa1a04 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -579,7 +579,7 @@ See the [configuration page](configuration.html) for information on Spark config #### Spark Properties -
Storage LevelMeaning
MEMORY_ONLY
+
@@ -1645,7 +1645,7 @@ See the below table for the full list of pod specifications that will be overwri ### Pod Metadata -
Property NameDefaultMeaningSince Version
spark.kubernetes.context
+
@@ -1681,7 +1681,7 @@ See the below table for the full list of pod specifications that will be overwri ### Pod Spec -
Pod metadata keyModified valueDescription
name
+
@@ -1734,7 +1734,7 @@ See the below table for the full list of pod specifications that will be overwri The following affect the driver and executor containers. All other containers in the pod spec will be unaffected. -
Pod spec keyModified valueDescription
imagePullSecrets
+
diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index b1a54a089a542..3d1c57030982d 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -374,7 +374,7 @@ See the [configuration page](configuration.html) for information on Spark config #### Spark Properties -
Container spec keyModified valueDescription
env
+
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 97cc9ac135af1..d577b70a68039 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -143,7 +143,7 @@ To use a custom metrics.properties for the application master and executors, upd #### Spark Properties -
Property NameDefaultMeaningSince Version
spark.mesos.coarse
+
@@ -696,7 +696,7 @@ To use a custom metrics.properties for the application master and executors, upd #### Available patterns for SHS custom executor log URL -
Property NameDefaultMeaningSince Version
spark.yarn.am.memory
+
@@ -783,7 +783,7 @@ staging directory of the Spark application. ## YARN-specific Kerberos Configuration -
PatternMeaning
{{HTTP_SCHEME}}
+
@@ -882,7 +882,7 @@ to avoid garbage collection issues during shuffle. The following extra configuration options are available when the shuffle service is running on YARN: -
Property NameDefaultMeaningSince Version
spark.kerberos.keytab
+
diff --git a/docs/security.md b/docs/security.md index 3c6fd507fec6d..c5d132f680a41 100644 --- a/docs/security.md +++ b/docs/security.md @@ -60,7 +60,7 @@ distributing the shared secret. Each application will use a unique shared secret the case of YARN, this feature relies on YARN RPC encryption being enabled for the distribution of secrets to be secure. -
Property NameDefaultMeaning
spark.yarn.shuffle.stopOnFailure
+
@@ -82,7 +82,7 @@ that any user that can list pods in the namespace where the Spark application is also see their authentication secret. Access control rules should be properly set up by the Kubernetes admin to ensure that Spark authentication is secure. -
Property NameDefaultMeaningSince Version
spark.yarn.shuffle.server.recovery.disabled
+
@@ -103,7 +103,7 @@ Kubernetes admin to ensure that Spark authentication is secure. Alternatively, one can mount authentication secrets using files and Kubernetes secrets that the user mounts into their pods. -
Property NameDefaultMeaningSince Version
spark.authenticate
+
@@ -159,7 +159,7 @@ is still required when talking to shuffle services from Spark versions older tha The following table describes the different options available for configuring this feature. -
Property NameDefaultMeaningSince Version
spark.authenticate.secret.file
+
@@ -219,7 +219,7 @@ encrypting output data generated by applications with APIs such as `saveAsHadoop The following settings cover enabling encryption for data written to disk: -
Property NameDefaultMeaningSince Version
spark.network.crypto.enabled
+
@@ -287,7 +287,7 @@ below. The following options control the authentication of Web UIs: -
Property NameDefaultMeaningSince Version
spark.io.encryption.enabled
+
@@ -391,7 +391,7 @@ servlet filters. To enable authorization in the SHS, a few extra options are used: -
Property NameDefaultMeaningSince Version
spark.ui.allowFramingFrom
+
@@ -440,7 +440,7 @@ protocol-specific settings. This way the user can easily provide the common sett protocols without disabling the ability to configure each one individually. The following table describes the SSL configuration namespaces: -
Property NameDefaultMeaningSince Version
spark.history.ui.acls.enable
+
@@ -471,7 +471,7 @@ describes the SSL configuration namespaces: The full breakdown of available SSL options can be found below. The `${ns}` placeholder should be replaced with one of the above namespaces. -
Config Namespace
+
@@ -641,7 +641,7 @@ Apache Spark can be configured to include HTTP headers to aid in preventing Cros (XSS), Cross-Frame Scripting (XFS), MIME-Sniffing, and also to enforce HTTP Strict Transport Security. -
Property NameDefaultMeaning
${ns}.enabled
+
@@ -697,7 +697,7 @@ configure those ports. ## Standalone mode only -
Property NameDefaultMeaningSince Version
spark.ui.xXssProtection
+
FromToDefault PortPurposeConfiguration @@ -748,7 +748,7 @@ configure those ports. ## All cluster managers - +
FromToDefault PortPurposeConfiguration @@ -824,7 +824,7 @@ deployment-specific page for more information. The following options provides finer-grained control for this feature: - +
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index e7ea2669a1139..5babac9e25295 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -53,7 +53,7 @@ You should see the new node listed there, along with its number of CPUs and memo Finally, the following configuration options can be passed to the master and worker: -
Property NameDefaultMeaningSince Version
spark.security.credentials.${service}.enabled
+
@@ -116,7 +116,7 @@ Note that these scripts must be executed on the machine you want to run the Spar You can optionally configure the cluster further by setting environment variables in `conf/spark-env.sh`. Create this file by starting with the `conf/spark-env.sh.template`, and _copy it to all your worker machines_ for the settings to take effect. The following settings are available: -
ArgumentMeaning
-h HOST, --host HOST
+
@@ -188,7 +188,7 @@ You can optionally configure the cluster further by setting environment variable SPARK_MASTER_OPTS supports the following system properties: -
Environment VariableMeaning
SPARK_MASTER_HOST
+
@@ -324,7 +324,7 @@ SPARK_MASTER_OPTS supports the following system properties: SPARK_WORKER_OPTS supports the following system properties: -
Property NameDefaultMeaningSince Version
spark.master.ui.port
+
@@ -429,7 +429,7 @@ You can also pass an option `--total-executor-cores ` to control the n Spark applications supports the following configuration properties specific to standalone mode: -
Property NameDefaultMeaningSince Version
spark.worker.cleanup.enabled
+
@@ -646,7 +646,7 @@ ZooKeeper is the best way to go for production-level high availability, but if y In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env using this configuration: -
Property NameDefault ValueMeaningSince Version
spark.standalone.submit.waitAppCompletion
+
diff --git a/docs/sparkr.md b/docs/sparkr.md index 8e6a98e40b680..a34a1200c4c00 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -77,7 +77,7 @@ sparkR.session(master = "local[*]", sparkConfig = list(spark.driver.memory = "2g The following Spark driver properties can be set in `sparkConfig` with `sparkR.session` from RStudio: -
System propertyMeaningSince Version
spark.deploy.recoveryMode
+
@@ -588,7 +588,7 @@ The following example shows how to save/load a MLlib model by SparkR. {% include_example read_write r/ml/ml.R %} # Data type mapping between R and Spark -
Property NameProperty groupspark-submit equivalent
spark.master
+
@@ -728,7 +728,7 @@ function is masking another function. The following functions are masked by the SparkR package: -
RSpark
byte
+
diff --git a/docs/sql-data-sources-avro.md b/docs/sql-data-sources-avro.md index b01174b918245..c846116ebf3e3 100644 --- a/docs/sql-data-sources-avro.md +++ b/docs/sql-data-sources-avro.md @@ -233,7 +233,7 @@ Data source options of Avro can be set via: * the `.option` method on `DataFrameReader` or `DataFrameWriter`. * the `options` parameter in function `from_avro`. -
Masked functionHow to Access
cov in package:stats
+
@@ -331,7 +331,7 @@ Data source options of Avro can be set via: ## Configuration Configuration of Avro can be done using the `setConf` method on SparkSession or by running `SET key=value` commands using SQL. -
Property NameDefaultMeaningScopeSince Version
avroSchema
+
@@ -418,7 +418,7 @@ Submission Guide for more details. ## Supported types for Avro -> Spark SQL conversion Currently Spark supports reading all [primitive types](https://avro.apache.org/docs/1.11.2/specification/#primitive-types) and [complex types](https://avro.apache.org/docs/1.11.2/specification/#complex-types) under records of Avro. -
Property NameDefaultMeaningSince Version
spark.sql.legacy.replaceDatabricksSparkAvro.enabled
+
@@ -483,7 +483,7 @@ All other union types are considered complex. They will be mapped to StructType It also supports reading the following Avro [logical types](https://avro.apache.org/docs/1.11.2/specification/#logical-types): -
Avro typeSpark SQL type
boolean
+
@@ -516,7 +516,7 @@ At the moment, it ignores docs, aliases and other properties present in the Avro ## Supported types for Spark SQL -> Avro conversion Spark supports writing of all Spark SQL types into Avro. For most types, the mapping from Spark types to Avro types is straightforward (e.g. IntegerType gets converted to int); however, there are a few special cases which are listed below: -
Avro logical typeAvro typeSpark SQL type
date
+
@@ -552,7 +552,7 @@ Spark supports writing of all Spark SQL types into Avro. For most types, the map You can also specify the whole output Avro schema with the option `avroSchema`, so that Spark SQL types can be converted into other Avro types. The following conversions are not applied by default and require user specified Avro schema: -
Spark SQL typeAvro typeAvro logical type
ByteType
+
diff --git a/docs/sql-data-sources-csv.md b/docs/sql-data-sources-csv.md index 31167f5514302..241aae3571221 100644 --- a/docs/sql-data-sources-csv.md +++ b/docs/sql-data-sources-csv.md @@ -52,7 +52,7 @@ Data source options of CSV can be set via: * `OPTIONS` clause at [CREATE TABLE USING DATA_SOURCE](sql-ref-syntax-ddl-create-table-datasource.html) -
Spark SQL typeAvro typeAvro logical type
BinaryType
+
diff --git a/docs/sql-data-sources-hive-tables.md b/docs/sql-data-sources-hive-tables.md index 0de573ec64b89..13cd8fc2cc056 100644 --- a/docs/sql-data-sources-hive-tables.md +++ b/docs/sql-data-sources-hive-tables.md @@ -75,7 +75,7 @@ format("serde", "input format", "output format"), e.g. `CREATE TABLE src(id int) By default, we will read the table files as plain text. Note that, Hive storage handler is not supported yet when creating table, you can create a table using storage handler at Hive side, and use Spark SQL to read it. -
Property NameDefaultMeaningScope
sep
+
@@ -123,7 +123,7 @@ will compile against built-in Hive and use those classes for internal execution The following options can be used to configure the version of Hive that is used to retrieve metadata: -
Property NameMeaning
fileFormat
+
diff --git a/docs/sql-data-sources-jdbc.md b/docs/sql-data-sources-jdbc.md index f96776514c672..edcdef4bf0084 100644 --- a/docs/sql-data-sources-jdbc.md +++ b/docs/sql-data-sources-jdbc.md @@ -51,7 +51,7 @@ For connection properties, users can specify the JDBC connection properties in t user and password are normally provided as connection properties for logging into the data sources. -
Property NameDefaultMeaningSince Version
spark.sql.hive.metastore.version
+
diff --git a/docs/sql-data-sources-json.md b/docs/sql-data-sources-json.md index 881a69cb1cea4..4ade5170a6d81 100644 --- a/docs/sql-data-sources-json.md +++ b/docs/sql-data-sources-json.md @@ -109,7 +109,7 @@ Data source options of JSON can be set via: * `schema_of_json` * `OPTIONS` clause at [CREATE TABLE USING DATA_SOURCE](sql-ref-syntax-ddl-create-table-datasource.html) -
Property NameDefaultMeaningScope
url
+
diff --git a/docs/sql-data-sources-load-save-functions.md b/docs/sql-data-sources-load-save-functions.md index 9d0a3f9c72b9a..31f6d944bc972 100644 --- a/docs/sql-data-sources-load-save-functions.md +++ b/docs/sql-data-sources-load-save-functions.md @@ -218,7 +218,7 @@ present. It is important to realize that these save modes do not utilize any loc atomic. Additionally, when performing an `Overwrite`, the data will be deleted before writing out the new data. -
Property NameDefaultMeaningScope
+
diff --git a/docs/sql-data-sources-orc.md b/docs/sql-data-sources-orc.md index 4e492598f595d..561f601aa4e56 100644 --- a/docs/sql-data-sources-orc.md +++ b/docs/sql-data-sources-orc.md @@ -129,7 +129,7 @@ When reading from Hive metastore ORC tables and inserting to Hive metastore ORC ### Configuration -
Scala/JavaAny LanguageMeaning
SaveMode.ErrorIfExists (default)
+
@@ -230,7 +230,7 @@ Data source options of ORC can be set via: * `DataStreamWriter` * `OPTIONS` clause at [CREATE TABLE USING DATA_SOURCE](sql-ref-syntax-ddl-create-table-datasource.html) -
Property NameDefaultMeaningSince Version
spark.sql.orc.impl
+
diff --git a/docs/sql-data-sources-parquet.md b/docs/sql-data-sources-parquet.md index 925e47504e5ef..f49bbd7a9d042 100644 --- a/docs/sql-data-sources-parquet.md +++ b/docs/sql-data-sources-parquet.md @@ -386,7 +386,7 @@ Data source options of Parquet can be set via: * `DataStreamWriter` * `OPTIONS` clause at [CREATE TABLE USING DATA_SOURCE](sql-ref-syntax-ddl-create-table-datasource.html) -
Property NameDefaultMeaningScope
mergeSchema
+
@@ -434,7 +434,7 @@ Other generic options can be found in +
Property NameDefaultMeaningScope
datetimeRebaseMode
diff --git a/docs/sql-data-sources-text.md b/docs/sql-data-sources-text.md index bb485d29c396a..aed8a2e9942fb 100644 --- a/docs/sql-data-sources-text.md +++ b/docs/sql-data-sources-text.md @@ -47,7 +47,7 @@ Data source options of text can be set via: * `DataStreamWriter` * `OPTIONS` clause at [CREATE TABLE USING DATA_SOURCE](sql-ref-syntax-ddl-create-table-datasource.html) -
Property NameDefaultMeaningSince Version
spark.sql.parquet.binaryAsString
+
diff --git a/docs/sql-distributed-sql-engine-spark-sql-cli.md b/docs/sql-distributed-sql-engine-spark-sql-cli.md index a67e009b9ae10..6d506cbb09c21 100644 --- a/docs/sql-distributed-sql-engine-spark-sql-cli.md +++ b/docs/sql-distributed-sql-engine-spark-sql-cli.md @@ -62,7 +62,7 @@ For example: `/path/to/spark-sql-cli.sql` equals to `file:///path/to/spark-sql-c ## Supported comment types -
Property NameDefaultMeaningScope
wholetext
+
@@ -115,7 +115,7 @@ Use `;` (semicolon) to terminate commands. Notice: ``` However, if ';' is the end of the line, it terminates the SQL statement. The example above will be terminated into `/* This is a comment contains ` and `*/ SELECT 1`, Spark will submit these two commands separated and throw parser error (`unclosed bracketed comment` and `Syntax error at or near '*/'`). -
CommentExample
simple comment
+
diff --git a/docs/sql-error-conditions-sqlstates.md b/docs/sql-error-conditions-sqlstates.md index 5529c961b3bfb..49cfb56b36626 100644 --- a/docs/sql-error-conditions-sqlstates.md +++ b/docs/sql-error-conditions-sqlstates.md @@ -33,7 +33,7 @@ Spark SQL uses the following `SQLSTATE` classes: ## Class `0A`: feature not supported -
CommandDescription
quit or exit
+
@@ -48,7 +48,7 @@ Spark SQL uses the following `SQLSTATE` classes:
SQLSTATEDescription and issuing error classes
0A000
## Class `21`: cardinality violation - +
@@ -63,7 +63,7 @@ Spark SQL uses the following `SQLSTATE` classes:
SQLSTATEDescription and issuing error classes
21000
## Class `22`: data exception - +
@@ -168,7 +168,7 @@ Spark SQL uses the following `SQLSTATE` classes:
SQLSTATEDescription and issuing error classes
22003
## Class `23`: integrity constraint violation - +
@@ -183,7 +183,7 @@ Spark SQL uses the following `SQLSTATE` classes:
SQLSTATEDescription and issuing error classes
23505
## Class `2B`: dependent privilege descriptors still exist - +
@@ -198,7 +198,7 @@ Spark SQL uses the following `SQLSTATE` classes:
SQLSTATEDescription and issuing error classes
2BP01
## Class `38`: external routine exception - +
@@ -213,7 +213,7 @@ Spark SQL uses the following `SQLSTATE` classes:
SQLSTATEDescription and issuing error classes
38000
## Class `39`: external routine invocation exception - +
@@ -228,7 +228,7 @@ Spark SQL uses the following `SQLSTATE` classes:
SQLSTATEDescription and issuing error classes
39000
## Class `42`: syntax error or access rule violation - +
@@ -648,7 +648,7 @@ Spark SQL uses the following `SQLSTATE` classes:
SQLSTATEDescription and issuing error classes
42000
## Class `46`: java ddl 1 - +
@@ -672,7 +672,7 @@ Spark SQL uses the following `SQLSTATE` classes:
SQLSTATEDescription and issuing error classes
46110
## Class `53`: insufficient resources - +
@@ -687,7 +687,7 @@ Spark SQL uses the following `SQLSTATE` classes:
SQLSTATEDescription and issuing error classes
53200
## Class `54`: program limit exceeded - +
@@ -702,7 +702,7 @@ Spark SQL uses the following `SQLSTATE` classes:
SQLSTATEDescription and issuing error classes
54000
## Class `HY`: CLI-specific condition - +
@@ -717,7 +717,7 @@ Spark SQL uses the following `SQLSTATE` classes:
SQLSTATEDescription and issuing error classes
HY008
## Class `XX`: internal error - +
diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 5cf0b28982c24..88635ee3d1f44 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -469,7 +469,7 @@ license: | ## Upgrading from Spark SQL 2.3 to 2.4 - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. -
SQLSTATEDescription and issuing error classes
XX000
+
@@ -583,7 +583,7 @@ license: | - Since Spark 2.3, the Join/Filter's deterministic predicates that are after the first non-deterministic predicates are also pushed down/through the child operators, if possible. In prior Spark versions, these filters are not eligible for predicate pushdown. - Partition column inference previously found incorrect common type for different inferred types, for example, previously it ended up with double type as the common type for double type and date type. Now it finds the correct common type for such conflicts. The conflict resolution follows the table below: - +
- - - - - - @@ -499,15 +491,6 @@ To use a custom metrics.properties for the application master and executors, upd - - - - - - From 9c83bf501ccefa7c6c0ba071f69e2528f3504854 Mon Sep 17 00:00:00 2001 From: Amy Tsai Date: Mon, 11 Dec 2023 18:35:31 +0300 Subject: [PATCH 145/521] [MINOR][DOCS] Fix documentation for `spark.sql.legacy.doLooseUpcast` in SQL migration guide ### What changes were proposed in this pull request? Fixes an error in the SQL migration guide documentation for `spark.sql.legacy.doLooseUpcast`. I corrected the config name and moved it to the section for migration from Spark 2.4 to 3.0 since it was not made available until Spark 3.0. ### Why are the changes needed? The config was documented as `spark.sql.legacy.looseUpcast` and is inaccurately included in the Spark 2.4 to Spark 2.4.1 section. I changed the docs to match what is implemented in https://github.com/apache/spark/blob/20df062d85e80422a55afae80ddbf2060f26516c/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L3873 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Docs only change ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44262 from amytsai-stripe/fix-migration-docs-loose-upcast. Authored-by: Amy Tsai Signed-off-by: Max Gekk (cherry picked from commit bab884082c0f82e3f9053adac6c7e8a3fcfab11c) Signed-off-by: Max Gekk --- docs/sql-migration-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 88635ee3d1f44..2eba9500e907e 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -251,6 +251,8 @@ license: | - In Spark 3.0, the column metadata will always be propagated in the API `Column.name` and `Column.as`. In Spark version 2.4 and earlier, the metadata of `NamedExpression` is set as the `explicitMetadata` for the new column at the time the API is called, it won't change even if the underlying `NamedExpression` changes metadata. To restore the behavior before Spark 3.0, you can use the API `as(alias: String, metadata: Metadata)` with explicit metadata. + - When turning a Dataset to another Dataset, Spark will up cast the fields in the original Dataset to the type of corresponding fields in the target DataSet. In version 2.4 and earlier, this up cast is not very strict, e.g. `Seq("str").toDS.as[Int]` fails, but `Seq("str").toDS.as[Boolean]` works and throw NPE during execution. In Spark 3.0, the up cast is stricter and turning String into something else is not allowed, i.e. `Seq("str").toDS.as[Boolean]` will fail during analysis. To restore the behavior before Spark 3.0, set `spark.sql.legacy.doLooseUpcast` to `true`. + ### DDL Statements - In Spark 3.0, when inserting a value into a table column with a different data type, the type coercion is performed as per ANSI SQL standard. Certain unreasonable type conversions such as converting `string` to `int` and `double` to `boolean` are disallowed. A runtime exception is thrown if the value is out-of-range for the data type of the column. In Spark version 2.4 and below, type conversions during table insertion are allowed as long as they are valid `Cast`. When inserting an out-of-range value to an integral field, the low-order bits of the value is inserted(the same as Java/Scala numeric type casting). For example, if 257 is inserted to a field of byte type, the result is 1. The behavior is controlled by the option `spark.sql.storeAssignmentPolicy`, with a default value as "ANSI". Setting the option as "Legacy" restores the previous behavior. @@ -464,8 +466,6 @@ license: | need to specify a value with units like "30s" now, to avoid being interpreted as milliseconds; otherwise, the extremely short interval that results will likely cause applications to fail. - - When turning a Dataset to another Dataset, Spark will up cast the fields in the original Dataset to the type of corresponding fields in the target DataSet. In version 2.4 and earlier, this up cast is not very strict, e.g. `Seq("str").toDS.as[Int]` fails, but `Seq("str").toDS.as[Boolean]` works and throw NPE during execution. In Spark 3.0, the up cast is stricter and turning String into something else is not allowed, i.e. `Seq("str").toDS.as[Boolean]` will fail during analysis. To restore the behavior before 2.4.1, set `spark.sql.legacy.looseUpcast` to `true`. - ## Upgrading from Spark SQL 2.3 to 2.4 - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. From ac031d68a01f14cc73f05e83a790a6787aa6453d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 11 Dec 2023 15:05:21 -0800 Subject: [PATCH 146/521] [SPARK-46369][CORE] Remove `kill` link from `RELAUNCHING` drivers in `MasterPage` MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR aims to remove `kill` hyperlink from `RELAUNCHING` drivers in `MasterPage`. ### Why are the changes needed? Since Apache Spark 1.4.0 (SPARK-5495), `RELAUNCHING` drivers have `kill` hyperlinks in the `Completed Drivers` table. ![Screenshot 2023-12-11 at 1 02 29 PM](https://github.com/apache/spark/assets/9700541/38f4bf08-efb9-47e5-8a7a-f7d127429012) However, this is a bug because the driver was already terminated by definition. Newly relaunched driver has an independent ID and there is no relationship with the previously terminated ID. https://github.com/apache/spark/blob/7db85642600b1e3b39ca11e41d4e3e0bf1c8962b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala#L27 If we clicked the `kill` link, `Master` always complains like the following. ``` 23/12/11 21:25:50 INFO Master: Asked to kill driver 202312112113-00000 23/12/11 21:25:50 WARN Master: Driver 202312112113-00000 has already finished or does not exist ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44301 from dongjoon-hyun/SPARK-46369. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit e434c9f0d5792b7af43c87dd6145fd8a6a04d8e2) Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/deploy/master/ui/MasterPage.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index a71eb33a2fe1d..e7e90aa0a37da 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -322,8 +322,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { private def driverRow(driver: DriverInfo, showDuration: Boolean): Seq[Node] = { val killLink = if (parent.killEnabled && (driver.state == DriverState.RUNNING || - driver.state == DriverState.SUBMITTED || - driver.state == DriverState.RELAUNCHING)) { + driver.state == DriverState.SUBMITTED)) { val confirm = s"if (window.confirm('Are you sure you want to kill driver ${driver.id} ?')) " + "{ this.parentNode.submit(); return true; } else { return false; }" From eb1e6ad13aab3960f1543b75bf3b75b3a7d62746 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 13 Dec 2023 18:04:38 +0800 Subject: [PATCH 147/521] [SPARK-46388][SQL] HiveAnalysis misses the pattern guard of `query.resolved` ### What changes were proposed in this pull request? This PR adds `query.resolved` as a pattern guard when HiveAnalysis converts InsertIntoStatement to InsertIntoHiveTable. ### Why are the changes needed? Due to https://github.com/apache/spark/pull/41262/files#diff-ed19f376a63eba52eea59ca71f3355d4495fad4fad4db9a3324aade0d4986a47R1080, the `table` field is resolved regardless of the query field. Before, it never got a chance to be resolved as `HiveTableRelation` and then match any rule of HiveAnalysis. But now, it gets the chance always and results in a spark-kernel bug - `Invalid call to toAttribute on unresolved object.` ``` insert into t2 select cast(a as short) from t where b=1; Invalid call to toAttribute on unresolved object ``` ### Does this PR introduce _any_ user-facing change? no, bugfix for 3.5 and later ### How was this patch tested? added new test ### Was this patch authored or co-authored using generative AI tooling? no Closes #44326 from yaooqinn/SPARK-46388. Authored-by: Kent Yao Signed-off-by: Kent Yao (cherry picked from commit ccc436d829cd0b07088e2864cb1ecc55ab97a491) Signed-off-by: Kent Yao --- .../spark/sql/hive/HiveStrategies.scala | 2 +- .../sql/hive/execution/SQLQuerySuite.scala | 26 +++++++++++++++++++ 2 files changed, 27 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 3da3d4a0eb5c8..c53a6c378d457 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -161,7 +161,7 @@ object HiveAnalysis extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case InsertIntoStatement( r: HiveTableRelation, partSpec, _, query, overwrite, ifPartitionNotExists, _) - if DDLUtils.isHiveTable(r.tableMeta) => + if DDLUtils.isHiveTable(r.tableMeta) && query.resolved => InsertIntoHiveTable(r.tableMeta, partSpec, query, overwrite, ifPartitionNotExists, query.output.map(_.name)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 9308d1eda146f..6160c3e5f6c65 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -2660,6 +2660,32 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi checkAnswer(df, Seq.empty[Row]) } } + + test("SPARK-46388: HiveAnalysis convert InsertIntoStatement to InsertIntoHiveTable " + + "iff child resolved") { + withTable("t") { + sql("CREATE TABLE t (a STRING)") + checkError( + exception = intercept[AnalysisException](sql("INSERT INTO t SELECT a*2 FROM t where b=1")), + errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + sqlState = None, + parameters = Map("objectName" -> "`b`", "proposal" -> "`a`"), + context = ExpectedContext( + fragment = "b", + start = 38, + stop = 38) ) + checkError( + exception = intercept[AnalysisException]( + sql("INSERT INTO t SELECT cast(a as short) FROM t where b=1")), + errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION", + sqlState = None, + parameters = Map("objectName" -> "`b`", "proposal" -> "`a`"), + context = ExpectedContext( + fragment = "b", + start = 51, + stop = 51)) + } + } } @SlowHiveTest From 908c472728f24034baf0b59f03b04ca148eabeca Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 14 Dec 2023 00:06:22 -0800 Subject: [PATCH 148/521] [SPARK-46396][SQL] Timestamp inference should not throw exception ### What changes were proposed in this pull request? When setting `spark.sql.legacy.timeParserPolicy=LEGACY`, Spark will use the LegacyFastTimestampFormatter to infer potential timestamp columns. The inference shouldn't throw exception. However, when the input is 23012150952, there is exception: ``` For input string: "23012150952" java.lang.NumberFormatException: For input string: "23012150952" at java.base/java.lang.NumberFormatException.forInputString(NumberFormatException.java:67) at java.base/java.lang.Integer.parseInt(Integer.java:668) at java.base/java.lang.Integer.parseInt(Integer.java:786) at org.apache.commons.lang3.time.FastDateParser$NumberStrategy.parse(FastDateParser.java:304) at org.apache.commons.lang3.time.FastDateParser.parse(FastDateParser.java:1045) at org.apache.commons.lang3.time.FastDateFormat.parse(FastDateFormat.java:651) at org.apache.spark.sql.catalyst.util.LegacyFastTimestampFormatter.parseOptional(TimestampFormatter.scala:418) ``` This PR is to fix the issue. ### Why are the changes needed? Bug fix, Timestamp inference should not throw exception ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? New test case + existing tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #44338 from gengliangwang/fixParseOptional. Authored-by: Gengliang Wang Signed-off-by: Gengliang Wang (cherry picked from commit 4a79ae9d821e9b04fbe949251050c3e4819dff92) Signed-off-by: Gengliang Wang --- .../spark/sql/catalyst/util/TimestampFormatter.scala | 12 ++++++++---- .../sql/catalyst/util/TimestampFormatterSuite.scala | 3 ++- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index 55eee41c14ca5..0866cee9334c5 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -414,10 +414,14 @@ class LegacyFastTimestampFormatter( override def parseOptional(s: String): Option[Long] = { cal.clear() // Clear the calendar because it can be re-used many times - if (fastDateFormat.parse(s, new ParsePosition(0), cal)) { - Some(extractMicros(cal)) - } else { - None + try { + if (fastDateFormat.parse(s, new ParsePosition(0), cal)) { + Some(extractMicros(cal)) + } else { + None + } + } catch { + case NonFatal(_) => None } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala index 2134a0d6ecd36..27d60815766dc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala @@ -502,10 +502,11 @@ class TimestampFormatterSuite extends DatetimeFormatterSuite { assert(fastFormatter.parseOptional("2023-12-31 23:59:59.9990").contains(1704067199999000L)) assert(fastFormatter.parseOptional("abc").isEmpty) + assert(fastFormatter.parseOptional("23012150952").isEmpty) assert(simpleFormatter.parseOptional("2023-12-31 23:59:59.9990").contains(1704067208990000L)) assert(simpleFormatter.parseOptional("abc").isEmpty) - + assert(simpleFormatter.parseOptional("23012150952").isEmpty) } test("SPARK-45424: do not return optional parse results when only prefix match") { From 8abf9583ac2303765255299af3e843d8248f313f Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 15 Dec 2023 18:55:10 +0800 Subject: [PATCH 149/521] [SPARK-46417][SQL] Do not fail when calling hive.getTable and throwException is false ### What changes were proposed in this pull request? Uses can set up their own HMS and let Spark connects to it. We have no control over it and somtimes it's not even Hive but just a HMS-API-compatible service. Spark should be more fault-tolerant when calling HMS APIs. This PR fixes an issue in `hive.getTable` with `throwException = false`, to make sure we don't throw error when can't fetch the table. ### Why are the changes needed? avoid query failure caused by HMS bugs. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? in our product environment ### Was this patch authored or co-authored using generative AI tooling? No Closes #44364 from cloud-fan/hive. Lead-authored-by: Wenchen Fan Co-authored-by: Wenchen Fan Signed-off-by: Kent Yao (cherry picked from commit 59488039f58b18617cd6dfd6dbe3bf014af222e7) Signed-off-by: Kent Yao --- .../scala/org/apache/spark/sql/hive/client/HiveShim.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 60ff9ec42f29d..7025e09ae9d9e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -620,7 +620,13 @@ private[client] class Shim_v0_12 extends Shim with Logging { tableName: String, throwException: Boolean): Table = { recordHiveCall() - val table = hive.getTable(dbName, tableName, throwException) + val table = try { + hive.getTable(dbName, tableName, throwException) + } catch { + // Hive may have bugs and still throw an exception even if `throwException` is false. + case e: HiveException if !throwException => + null + } if (table != null) { table.getTTable.setTableName(tableName) table.getTTable.setDbName(dbName) From cc4f5787414e4392499a349dec5b24c8e25e50f3 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 19 Dec 2023 12:21:20 +0300 Subject: [PATCH 150/521] [SPARK-46453][CONNECT] Throw exception from `internalError()` in `SessionHolder` ### What changes were proposed in this pull request? In the PR, I propose to throw `SparkException` returned by `internalError` in `SessionHolder`. ### Why are the changes needed? Without the bug fix user won't see the internal error. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? N/a ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44400 from MaxGekk/throw-internal-error. Authored-by: Max Gekk Signed-off-by: Max Gekk (cherry picked from commit dc0bfc4c700c347f2f58625facec8c5771bde59a) Signed-off-by: Max Gekk --- .../org/apache/spark/sql/connect/service/SessionHolder.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala index 1cef02d7e3466..218819d114c12 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala @@ -197,7 +197,7 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio */ private[connect] def cacheDataFrameById(dfId: String, df: DataFrame): Unit = { if (dataFrameCache.putIfAbsent(dfId, df) != null) { - SparkException.internalError(s"A dataframe is already associated with id $dfId") + throw SparkException.internalError(s"A dataframe is already associated with id $dfId") } } @@ -221,7 +221,7 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio */ private[connect] def cacheListenerById(id: String, listener: StreamingQueryListener): Unit = { if (listenerCache.putIfAbsent(id, listener) != null) { - SparkException.internalError(s"A listener is already associated with id $id") + throw SparkException.internalError(s"A listener is already associated with id $id") } } From 0c00c54583fe3e56f940425aac6e0e4f05c4b9db Mon Sep 17 00:00:00 2001 From: zhouyifan279 Date: Wed, 20 Dec 2023 16:50:38 +0800 Subject: [PATCH 151/521] [SPARK-46330] Loading of Spark UI blocks for a long time when HybridStore enabled ### What changes were proposed in this pull request? Move `LoadedAppUI` invalidate operation out of `FsHistoryProvider` synchronized block. ### Why are the changes needed? When closing a HybridStore of a `LoadedAppUI` with a lot of data waiting to be written to disk, loading of other Spark UIs will be blocked for a long time. See more details at https://issues.apache.org/jira/browse/SPARK-46330 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Passed existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44260 from zhouyifan279/SPARK-46330. Authored-by: zhouyifan279 Signed-off-by: Kent Yao (cherry picked from commit cf54e8f9a51bf54e8fa3e1011ac370e46134b134) Signed-off-by: Kent Yao --- .../spark/deploy/history/FsHistoryProvider.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 49b479f3124e9..387bc7d9e45b3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -925,11 +925,12 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) * UI lifecycle. */ private def invalidateUI(appId: String, attemptId: Option[String]): Unit = { - synchronized { - activeUIs.get((appId, attemptId)).foreach { ui => - ui.invalidate() - ui.ui.store.close() - } + val uiOption = synchronized { + activeUIs.get((appId, attemptId)) + } + uiOption.foreach { ui => + ui.invalidate() + ui.ui.store.close() } } From d7534a3ec1eab53bbd349f9ae31684337c734958 Mon Sep 17 00:00:00 2001 From: Aleksandar Tomic Date: Thu, 21 Dec 2023 15:58:15 +0800 Subject: [PATCH 152/521] [SPARK-46380][SQL] Replace current time/date prior to evaluating inline table expressions With this PR proposal is to do inline table resolution in two phases: 1) If there are no expressions that depend on current context (e.g. expressions that depend on CURRENT_DATABASE, CURRENT_USER, CURRENT_TIME etc.) they will be evaluated as part of ResolveInlineTable rule. 2) Expressions that do depend on CURRENT_* evaluation will be kept as expressions and they evaluation will be delayed to post analysis phase. This PR aims to solve two problems with inline tables. Example1: ```sql SELECT COUNT(DISTINCT ct) FROM VALUES (CURRENT_TIMESTAMP()), (CURRENT_TIMESTAMP()), (CURRENT_TIMESTAMP()) as data(ct) ``` Prior to this change this example would return 3 (i.e. all CURRENT_TIMESTAMP expressions would return different value since they would be evaluated individually as part of inline table evaluation). After this change result is 1. Example 2: ```sql CREATE VIEW V as (SELECT * FROM VALUES(CURRENT_TIMESTAMP()) ``` In this example VIEW would be saved with literal evaluated during VIEW creation. After this change CURRENT_TIMESTAMP() will eval during VIEW execution. See section above. New test that validates this behaviour is introduced. No. Closes #44316 from dbatomic/inline_tables_curr_time_fix. Lead-authored-by: Aleksandar Tomic Co-authored-by: Aleksandar Tomic <150942779+dbatomic@users.noreply.github.com> Signed-off-by: Wenchen Fan (cherry picked from commit 5fe963f8560ef05925d127e82ab7ef28d6a1d7bc) Signed-off-by: Wenchen Fan --- .../analysis/ResolveInlineTables.scala | 68 ++++++++++--------- .../sql/catalyst/analysis/unresolved.scala | 15 ++++ .../sql/catalyst/optimizer/Optimizer.scala | 4 +- .../catalyst/optimizer/finishAnalysis.scala | 33 +++++++++ .../sql/catalyst/rules/RuleIdCollection.scala | 1 + .../sql/catalyst/trees/TreePatterns.scala | 1 + .../analysis/ResolveInlineTablesSuite.scala | 31 +++++++-- .../analyzer-results/inline-table.sql.out | 16 ++++- .../postgreSQL/create_view.sql.out | 2 +- .../sql-tests/inputs/inline-table.sql | 6 ++ .../sql-tests/results/inline-table.sql.out | 16 +++++ .../spark/sql/execution/SQLViewSuite.scala | 14 ++++ 12 files changed, 165 insertions(+), 42 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala index 760ea466b8579..73600f5c70649 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala @@ -17,28 +17,29 @@ package org.apache.spark.sql.catalyst.analysis -import scala.util.control.NonFatal - -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{AliasHelper, EvalHelper} -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.expressions.{AliasHelper, EvalHelper, Expression} +import org.apache.spark.sql.catalyst.optimizer.EvalInlineTables +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.AlwaysProcess +import org.apache.spark.sql.catalyst.trees.TreePattern.CURRENT_LIKE import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.catalyst.util.TypeUtils.{toSQLExpr, toSQLId} import org.apache.spark.sql.types.{StructField, StructType} /** - * An analyzer rule that replaces [[UnresolvedInlineTable]] with [[LocalRelation]]. + * An analyzer rule that replaces [[UnresolvedInlineTable]] with [[ResolvedInlineTable]]. */ object ResolveInlineTables extends Rule[LogicalPlan] with CastSupport with AliasHelper with EvalHelper { - override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning( - AlwaysProcess.fn, ruleId) { - case table: UnresolvedInlineTable if table.expressionsResolved => - validateInputDimension(table) - validateInputEvaluable(table) - convert(table) + override def apply(plan: LogicalPlan): LogicalPlan = { + plan.resolveOperatorsWithPruning(AlwaysProcess.fn, ruleId) { + case table: UnresolvedInlineTable if table.expressionsResolved => + validateInputDimension(table) + validateInputEvaluable(table) + val resolvedTable = findCommonTypesAndCast(table) + earlyEvalIfPossible(resolvedTable) + } } /** @@ -74,7 +75,10 @@ object ResolveInlineTables extends Rule[LogicalPlan] table.rows.foreach { row => row.foreach { e => // Note that nondeterministic expressions are not supported since they are not foldable. - if (!e.resolved || !trimAliases(prepareForEval(e)).foldable) { + // Only exception are CURRENT_LIKE expressions, which are replaced by a literal + // In later stages. + if ((!e.resolved && !e.containsPattern(CURRENT_LIKE)) + || !trimAliases(prepareForEval(e)).foldable) { e.failAnalysis( errorClass = "INVALID_INLINE_TABLE.CANNOT_EVALUATE_EXPRESSION_IN_INLINE_TABLE", messageParameters = Map("expr" -> toSQLExpr(e))) @@ -84,14 +88,12 @@ object ResolveInlineTables extends Rule[LogicalPlan] } /** - * Convert a valid (with right shape and foldable inputs) [[UnresolvedInlineTable]] - * into a [[LocalRelation]]. - * * This function attempts to coerce inputs into consistent types. * * This is package visible for unit testing. */ - private[analysis] def convert(table: UnresolvedInlineTable): LocalRelation = { + private[analysis] def findCommonTypesAndCast(table: UnresolvedInlineTable): + ResolvedInlineTable = { // For each column, traverse all the values and find a common data type and nullability. val fields = table.rows.transpose.zip(table.names).map { case (column, name) => val inputTypes = column.map(_.dataType) @@ -105,26 +107,30 @@ object ResolveInlineTables extends Rule[LogicalPlan] val attributes = DataTypeUtils.toAttributes(StructType(fields)) assert(fields.size == table.names.size) - val newRows: Seq[InternalRow] = table.rows.map { row => - InternalRow.fromSeq(row.zipWithIndex.map { case (e, ci) => - val targetType = fields(ci).dataType - try { + val castedRows: Seq[Seq[Expression]] = table.rows.map { row => + row.zipWithIndex.map { + case (e, ci) => + val targetType = fields(ci).dataType val castedExpr = if (DataTypeUtils.sameType(e.dataType, targetType)) { e } else { cast(e, targetType) } - prepareForEval(castedExpr).eval() - } catch { - case NonFatal(ex) => - table.failAnalysis( - errorClass = "INVALID_INLINE_TABLE.FAILED_SQL_EXPRESSION_EVALUATION", - messageParameters = Map("sqlExpr" -> toSQLExpr(e)), - cause = ex) - } - }) + castedExpr + } } - LocalRelation(attributes, newRows) + ResolvedInlineTable(castedRows, attributes) + } + + /** + * This function attempts to early evaluate rows in inline table. + * If evaluation doesn't rely on non-deterministic expressions (e.g. current_like) + * expressions will be evaluated and inlined as [[LocalRelation]] + * This is package visible for unit testing. + */ + private[analysis] def earlyEvalIfPossible(table: ResolvedInlineTable): LogicalPlan = { + val earlyEvalPossible = table.rows.flatten.forall(!_.containsPattern(CURRENT_LIKE)) + if (earlyEvalPossible) EvalInlineTables(table) else table } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index b1dcb465b4778..07ad5e57306a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -126,6 +126,21 @@ case class UnresolvedInlineTable( lazy val expressionsResolved: Boolean = rows.forall(_.forall(_.resolved)) } +/** + * An resolved inline table that holds all the expressions that were checked for + * the right shape and common data types. + * This is a preparation step for [[org.apache.spark.sql.catalyst.optimizer.EvalInlineTables]] which + * will produce a [[org.apache.spark.sql.catalyst.plans.logical.LocalRelation]] + * for this inline table. + * + * @param output list of column attributes + * @param rows expressions for the data rows + */ +case class ResolvedInlineTable(rows: Seq[Seq[Expression]], output: Seq[Attribute]) + extends LeafNode { + final override val nodePatterns: Seq[TreePattern] = Seq(INLINE_TABLE_EVAL) +} + /** * A table-valued function, e.g. * {{{ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index bb2a86556c031..ec5f00d34cd8c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -287,7 +287,9 @@ abstract class Optimizer(catalogManager: CatalogManager) ComputeCurrentTime, ReplaceCurrentLike(catalogManager), SpecialDatetimeValues, - RewriteAsOfJoin) + RewriteAsOfJoin, + EvalInlineTables + ) override def apply(plan: LogicalPlan): LogicalPlan = { rules.foldLeft(plan) { case (sp, rule) => rule.apply(sp) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index 466781fa1def7..d7efc16a514bd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -19,7 +19,12 @@ package org.apache.spark.sql.catalyst.optimizer import java.time.{Instant, LocalDateTime} +import scala.util.control.NonFatal + import org.apache.spark.sql.catalyst.CurrentUserContext.CURRENT_USER +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.{CastSupport, ResolvedInlineTable} +import org.apache.spark.sql.catalyst.analysis.ResolveInlineTables.prepareForEval import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ @@ -27,6 +32,7 @@ import org.apache.spark.sql.catalyst.trees.TreePattern._ import org.apache.spark.sql.catalyst.trees.TreePatternBits import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, convertSpecialTimestamp, convertSpecialTimestampNTZ, instantToMicros, localDateTimeToMicros} +import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLExpr import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -71,6 +77,33 @@ object RewriteNonCorrelatedExists extends Rule[LogicalPlan] { } } +/** + * Computes expressions in inline tables. This rule is supposed to be called at the very end + * of the analysis phase, given that all the expressions need to be fully resolved/replaced + * at this point. + */ +object EvalInlineTables extends Rule[LogicalPlan] with CastSupport { + override def apply(plan: LogicalPlan): LogicalPlan = { + plan.transformDownWithSubqueriesAndPruning(_.containsPattern(INLINE_TABLE_EVAL)) { + case table: ResolvedInlineTable => + val newRows: Seq[InternalRow] = + table.rows.map { row => InternalRow.fromSeq(row.map { e => + try { + prepareForEval(e).eval() + } catch { + case NonFatal(ex) => + table.failAnalysis( + errorClass = "INVALID_INLINE_TABLE.FAILED_SQL_EXPRESSION_EVALUATION", + messageParameters = Map("sqlExpr" -> toSQLExpr(e)), + cause = ex) + }}) + } + + LocalRelation(table.output, newRows) + } + } +} + /** * Computes the current date and time to make sure we return the same result in a single query. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala index caf679f3e7a7a..96f78d251c39a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala @@ -166,6 +166,7 @@ object RuleIdCollection { "org.apache.spark.sql.catalyst.optimizer.SimplifyConditionals" :: "org.apache.spark.sql.catalyst.optimizer.SimplifyExtractValueOps" :: "org.apache.spark.sql.catalyst.optimizer.TransposeWindow" :: + "org.apache.spark.sql.catalyst.optimizer.EvalInlineTables" :: "org.apache.spark.sql.catalyst.optimizer.UnwrapCastInBinaryComparison" :: Nil } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index bf7b2db1719f5..ce8f5951839e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -53,6 +53,7 @@ object TreePattern extends Enumeration { val IF: Value = Value val IN: Value = Value val IN_SUBQUERY: Value = Value + val INLINE_TABLE_EVAL: Value = Value val INSET: Value = Value val INTERSECT: Value = Value val INVOKE: Value = Value diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTablesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTablesSuite.scala index 2e6c6e4eaf4c3..758b6b73e4eb1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTablesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTablesSuite.scala @@ -20,8 +20,9 @@ package org.apache.spark.sql.catalyst.analysis import org.scalatest.BeforeAndAfter import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, Literal, Rand} +import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, CurrentTimestamp, Literal, Rand} import org.apache.spark.sql.catalyst.expressions.aggregate.Count +import org.apache.spark.sql.catalyst.optimizer.{ComputeCurrentTime, EvalInlineTables} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.types.{LongType, NullType, TimestampType} @@ -83,9 +84,10 @@ class ResolveInlineTablesSuite extends AnalysisTest with BeforeAndAfter { assert(ResolveInlineTables(table) == table) } - test("convert") { + test("cast and execute") { val table = UnresolvedInlineTable(Seq("c1"), Seq(Seq(lit(1)), Seq(lit(2L)))) - val converted = ResolveInlineTables.convert(table) + val resolved = ResolveInlineTables.findCommonTypesAndCast(table) + val converted = ResolveInlineTables.earlyEvalIfPossible(resolved).asInstanceOf[LocalRelation] assert(converted.output.map(_.dataType) == Seq(LongType)) assert(converted.data.size == 2) @@ -93,11 +95,28 @@ class ResolveInlineTablesSuite extends AnalysisTest with BeforeAndAfter { assert(converted.data(1).getLong(0) == 2L) } + test("cast and execute CURRENT_LIKE expressions") { + val table = UnresolvedInlineTable(Seq("c1"), Seq( + Seq(CurrentTimestamp()), Seq(CurrentTimestamp()))) + val casted = ResolveInlineTables.findCommonTypesAndCast(table) + val earlyEval = ResolveInlineTables.earlyEvalIfPossible(casted) + // Early eval should keep it in expression form. + assert(earlyEval.isInstanceOf[ResolvedInlineTable]) + + EvalInlineTables(ComputeCurrentTime(earlyEval)) match { + case LocalRelation(output, data, _) => + assert(output.map(_.dataType) == Seq(TimestampType)) + assert(data.size == 2) + // Make sure that both CURRENT_TIMESTAMP expressions are evaluated to the same value. + assert(data(0).getLong(0) == data(1).getLong(0)) + } + } + test("convert TimeZoneAwareExpression") { val table = UnresolvedInlineTable(Seq("c1"), Seq(Seq(Cast(lit("1991-12-06 00:00:00.0"), TimestampType)))) val withTimeZone = ResolveTimeZone.apply(table) - val LocalRelation(output, data, _) = ResolveInlineTables.apply(withTimeZone) + val LocalRelation(output, data, _) = EvalInlineTables(ResolveInlineTables.apply(withTimeZone)) val correct = Cast(lit("1991-12-06 00:00:00.0"), TimestampType) .withTimeZone(conf.sessionLocalTimeZone).eval().asInstanceOf[Long] assert(output.map(_.dataType) == Seq(TimestampType)) @@ -107,11 +126,11 @@ class ResolveInlineTablesSuite extends AnalysisTest with BeforeAndAfter { test("nullability inference in convert") { val table1 = UnresolvedInlineTable(Seq("c1"), Seq(Seq(lit(1)), Seq(lit(2L)))) - val converted1 = ResolveInlineTables.convert(table1) + val converted1 = ResolveInlineTables.findCommonTypesAndCast(table1) assert(!converted1.schema.fields(0).nullable) val table2 = UnresolvedInlineTable(Seq("c1"), Seq(Seq(lit(1)), Seq(Literal(null, NullType)))) - val converted2 = ResolveInlineTables.convert(table2) + val converted2 = ResolveInlineTables.findCommonTypesAndCast(table2) assert(converted2.schema.fields(0).nullable) } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/inline-table.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/inline-table.sql.out index 2a17f092a06b7..adce16bf23578 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/inline-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/inline-table.sql.out @@ -73,9 +73,7 @@ Project [a#x, b#x] -- !query select a from values ("one", current_timestamp) as data(a, b) -- !query analysis -Project [a#x] -+- SubqueryAlias data - +- LocalRelation [a#x, b#x] +[Analyzer test output redacted due to nondeterminism] -- !query @@ -241,3 +239,15 @@ select * from values (10 + try_divide(5, 0)) -- !query analysis Project [col1#x] +- LocalRelation [col1#x] + + +-- !query +select count(distinct ct) from values now(), now(), now() as data(ct) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] + + +-- !query +select count(distinct ct) from values current_timestamp(), current_timestamp() as data(ct) +-- !query analysis +[Analyzer test output redacted due to nondeterminism] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out index b199cb55f2a44..7f477c80d46ca 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out @@ -1661,7 +1661,7 @@ select * from tt7a left join tt8a using (x), tt8a tt8ax, false, false, Persisted :- Project [a#x, b#x, c#x, d#x, e#x] : +- SubqueryAlias v : +- Project [col1#x AS a#x, col2#x AS b#x, col3#x AS c#x, col4#x AS d#x, col5#x AS e#x] - : +- LocalRelation [col1#x, col2#x, col3#x, col4#x, col5#x] + : +- ResolvedInlineTable [[now(), 2, 3, now(), 5]], [col1#x, col2#x, col3#x, col4#x, col5#x] +- Project [cast(x#x as timestamp) AS x#x, y#x, z#x, x#x, z#x] +- Project [x#x, y#x, z#x, x#x, z#x] +- Join Inner diff --git a/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql b/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql index 6867248f5765d..8f65dc77c960a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql @@ -60,3 +60,9 @@ select * from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991- select * from values (try_add(5, 0)); select * from values (try_divide(5, 0)); select * from values (10 + try_divide(5, 0)); + +-- now() should be kept as tempResolved inline expression. +select count(distinct ct) from values now(), now(), now() as data(ct); + +-- current_timestamp() should be kept as tempResolved inline expression. +select count(distinct ct) from values current_timestamp(), current_timestamp() as data(ct); diff --git a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out index 709d7ab73f6c4..b6c90b95c1d34 100644 --- a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out @@ -266,3 +266,19 @@ select * from values (10 + try_divide(5, 0)) struct -- !query output NULL + + +-- !query +select count(distinct ct) from values now(), now(), now() as data(ct) +-- !query schema +struct +-- !query output +1 + + +-- !query +select count(distinct ct) from values current_timestamp(), current_timestamp() as data(ct) +-- !query schema +struct +-- !query output +1 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala index e258d600a2aa8..a1147c16cc861 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -1216,4 +1216,18 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { } } } + + test("Inline table with current time expression") { + withView("v1") { + sql("CREATE VIEW v1 (t1, t2) AS SELECT * FROM VALUES (now(), now())") + val r1 = sql("select t1, t2 from v1").collect()(0) + val ts1 = (r1.getTimestamp(0), r1.getTimestamp(1)) + assert(ts1._1 == ts1._2) + Thread.sleep(1) + val r2 = sql("select t1, t2 from v1").collect()(0) + val ts2 = (r2.getTimestamp(0), r2.getTimestamp(1)) + assert(ts2._1 == ts2._2) + assert(ts1._1.getTime < ts2._1.getTime) + } + } } From 286c469ad1305f91ea796fd453ae896617fb3883 Mon Sep 17 00:00:00 2001 From: Jiaan Geng Date: Fri, 22 Dec 2023 09:55:00 +0800 Subject: [PATCH 153/521] [SPARK-46443][SQL] Decimal precision and scale should decided by H2 dialect ### What changes were proposed in this pull request? This PR fix a but by make JDBC dialect decide the decimal precision and scale. **How to reproduce the bug?** https://github.com/apache/spark/pull/44397 proposed DS V2 push down `PERCENTILE_CONT` and `PERCENTILE_DISC`. The bug fired when pushdown the below SQL to H2 JDBC. `SELECT "DEPT",PERCENTILE_CONT(0.5) WITHIN GROUP (ORDER BY "SALARY" ASC NULLS FIRST) FROM "test"."employee" WHERE 1=0 GROUP BY "DEPT"` **The root cause** `getQueryOutputSchema` used to get the output schema of query by call `JdbcUtils.getSchema`. The query for database H2 show below. `SELECT "DEPT",PERCENTILE_CONT(0.5) WITHIN GROUP (ORDER BY "SALARY" ASC NULLS FIRST) FROM "test"."employee" WHERE 1=0 GROUP BY "DEPT"` We can get the five variables from `ResultSetMetaData`, please refer: ``` columnName = "PERCENTILE_CONT(0.5) WITHIN GROUP (ORDER BY SALARY NULLS FIRST)" dataType = 2 typeName = "NUMERIC" fieldSize = 100000 fieldScale = 50000 ``` Then we get the catalyst schema with `JdbcUtils.getCatalystType`, it calls `DecimalType.bounded(precision, scale)` actually. The `DecimalType.bounded(100000, 50000)` returns `DecimalType(38, 38)`. At finally, `makeGetter` throws exception. ``` Caused by: org.apache.spark.SparkArithmeticException: [DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION] Decimal precision 42 exceeds max precision 38. SQLSTATE: 22003 at org.apache.spark.sql.errors.DataTypeErrors$.decimalPrecisionExceedsMaxPrecisionError(DataTypeErrors.scala:48) at org.apache.spark.sql.types.Decimal.set(Decimal.scala:124) at org.apache.spark.sql.types.Decimal$.apply(Decimal.scala:577) at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.$anonfun$makeGetter$4(JdbcUtils.scala:408) at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.nullSafeConvert(JdbcUtils.scala:552) at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.$anonfun$makeGetter$3(JdbcUtils.scala:408) at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.$anonfun$makeGetter$3$adapted(JdbcUtils.scala:406) at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anon$1.getNext(JdbcUtils.scala:358) at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anon$1.getNext(JdbcUtils.scala:339) ``` ### Why are the changes needed? This PR fix the bug that `JdbcUtils` can't get the correct decimal type. ### Does this PR introduce _any_ user-facing change? 'Yes'. Fix a bug. ### How was this patch tested? Manual tests in https://github.com/apache/spark/pull/44397 ### Was this patch authored or co-authored using generative AI tooling? 'No'. Closes #44398 from beliefer/SPARK-46443. Authored-by: Jiaan Geng Signed-off-by: Wenchen Fan (cherry picked from commit a921da8509a19b2d23c30ad657725f760932236c) Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/jdbc/H2Dialect.scala | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala index 8471a49153ff4..3f56eb035f5c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.connector.catalog.functions.UnboundFunction import org.apache.spark.sql.connector.catalog.index.TableIndex import org.apache.spark.sql.connector.expressions.{Expression, FieldReference, NamedReference} import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} -import org.apache.spark.sql.types.{BooleanType, ByteType, DataType, DecimalType, ShortType, StringType} +import org.apache.spark.sql.types.{BooleanType, ByteType, DataType, DecimalType, MetadataBuilder, ShortType, StringType} private[sql] object H2Dialect extends JdbcDialect { override def canHandle(url: String): Boolean = @@ -57,6 +57,20 @@ private[sql] object H2Dialect extends JdbcDialect { override def isSupportedFunction(funcName: String): Boolean = supportedFunctions.contains(funcName) + override def getCatalystType( + sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = { + sqlType match { + case Types.NUMERIC if size > 38 => + // H2 supports very large decimal precision like 100000. The max precision in Spark is only + // 38. Here we shrink both the precision and scale of H2 decimal to fit Spark, and still + // keep the ratio between them. + val scale = if (null != md) md.build().getLong("scale") else 0L + val selectedScale = (DecimalType.MAX_PRECISION * (scale.toDouble / size.toDouble)).toInt + Option(DecimalType(DecimalType.MAX_PRECISION, selectedScale)) + case _ => None + } + } + override def getJDBCType(dt: DataType): Option[JdbcType] = dt match { case StringType => Option(JdbcType("CLOB", Types.CLOB)) case BooleanType => Some(JdbcType("BOOLEAN", Types.BOOLEAN)) From 98042e34796ec8d83071256142f8e121f50ad1f4 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 22 Dec 2023 11:45:10 +0800 Subject: [PATCH 154/521] [SPARK-46464][DOC] Fix the scroll issue of tables when overflow ### What changes were proposed in this pull request? https://spark.apache.org/docs/3.4.1/running-on-kubernetes.html#spark-properties https://spark.apache.org/docs/latest/running-on-kubernetes.html#spark-properties As listed above, the doc content in 3.5.0 cannot scroll horizontally. Users can only see the rest of its content when a table overflows if they zoom out as much as possible, resulting in hard-to-read minor characters. This PR changes the HTML body overflow-x from hidden to auto to enable the underlying table to scroll horizontally. ### Why are the changes needed? Fix documentation ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? #### Before ![image](https://github.com/apache/spark/assets/8326978/437bee91-ab0d-4616-aaaf-f99171dcf9f9) #### After ![image](https://github.com/apache/spark/assets/8326978/327ed82b-3e14-4a27-be1a-835a7b21c000) ### Was this patch authored or co-authored using generative AI tooling? no Closes #44423 from yaooqinn/SPARK-46464. Authored-by: Kent Yao Signed-off-by: Kent Yao (cherry picked from commit fc7d7bce7732a2bccb3a7ccf3ed6bed4ac65f8fc) Signed-off-by: Kent Yao --- docs/css/custom.css | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/css/custom.css b/docs/css/custom.css index e7416d9ded618..1239c0ed440ef 100644 --- a/docs/css/custom.css +++ b/docs/css/custom.css @@ -7,7 +7,7 @@ body { font-style: normal; font-weight: 400; overflow-wrap: anywhere; - overflow-x: hidden; + overflow-x: auto; padding-top: 80px; } From a001482b43d24b4761049687b87bceba0e21c8fd Mon Sep 17 00:00:00 2001 From: ulysses-you Date: Fri, 22 Dec 2023 17:28:59 +0800 Subject: [PATCH 155/521] [SPARK-46480][CORE][SQL][3.5] Fix NPE when table cache task attempt This pr backports https://github.com/apache/spark/pull/44445 for branch-3.5 ### What changes were proposed in this pull request? This pr adds a check: we only mark the cached partition is materialized if the task is not failed and not interrupted. And adds a new method `isFailed` in `TaskContext`. ### Why are the changes needed? Before this pr, when do cache, task failure can cause NPE in other tasks ``` java.lang.NullPointerException at java.nio.ByteBuffer.wrap(ByteBuffer.java:396) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificColumnarIterator.accessors1$(Unknown Source) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificColumnarIterator.hasNext(Unknown Source) at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458) at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:155) at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52) at org.apache.spark.scheduler.Task.run(Task.scala:131) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:497) ``` ### Does this PR introduce _any_ user-facing change? yes, it's a bug fix ### How was this patch tested? add test ### Was this patch authored or co-authored using generative AI tooling? no Closes #44457 from ulysses-you/fix-cache-3.5. Authored-by: ulysses-you Signed-off-by: youxiduo --- .../scala/org/apache/spark/BarrierTaskContext.scala | 2 ++ core/src/main/scala/org/apache/spark/TaskContext.scala | 5 +++++ .../main/scala/org/apache/spark/TaskContextImpl.scala | 2 ++ .../org/apache/spark/scheduler/TaskContextSuite.scala | 10 ++++++++++ project/MimaExcludes.scala | 4 +++- .../sql/execution/columnar/InMemoryRelation.scala | 8 +++++--- 6 files changed, 27 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala index ecc0c891ea161..94ba3fe64a859 100644 --- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala +++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala @@ -193,6 +193,8 @@ class BarrierTaskContext private[spark] ( override def isCompleted(): Boolean = taskContext.isCompleted() + override def isFailed(): Boolean = taskContext.isFailed() + override def isInterrupted(): Boolean = taskContext.isInterrupted() override def addTaskCompletionListener(listener: TaskCompletionListener): this.type = { diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 450c00928c9e6..af7aa4979dc1c 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -94,6 +94,11 @@ abstract class TaskContext extends Serializable { */ def isCompleted(): Boolean + /** + * Returns true if the task has failed. + */ + def isFailed(): Boolean + /** * Returns true if the task has been killed. */ diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 526627c28607d..46273a1b6d687 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -275,6 +275,8 @@ private[spark] class TaskContextImpl( @GuardedBy("this") override def isCompleted(): Boolean = synchronized(completed) + override def isFailed(): Boolean = synchronized(failureCauseOpt.isDefined) + override def isInterrupted(): Boolean = reasonIfKilled.isDefined override def getLocalProperty(key: String): String = localProperties.getProperty(key) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index 54a42c1a66184..a5c2cbf52aafd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -669,6 +669,16 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark assert(invocationOrder === Seq("C", "B", "A", "D")) } + test("SPARK-46480: Add isFailed in TaskContext") { + val context = TaskContext.empty() + var isFailed = false + context.addTaskCompletionListener[Unit] { context => + isFailed = context.isFailed() + } + context.markTaskFailed(new RuntimeException()) + context.markTaskCompleted(None) + assert(isFailed) + } } private object TaskContextSuite { diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 9805ad7f09d6e..376ddfde1b937 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -72,7 +72,9 @@ object MimaExcludes { ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.api.java.function.FlatMapGroupsWithStateFunction"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.api.java.function.MapGroupsWithStateFunction"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SaveMode"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.GroupState") + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.GroupState"), + // [SPARK-46480][CORE][SQL] Fix NPE when table cache task attempt + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.isFailed") ) // Default exclude rules diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 45d006b58e879..65f7835b42cf8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -279,9 +279,11 @@ case class CachedRDDBuilder( cachedPlan.conf) } val cached = cb.mapPartitionsInternal { it => - TaskContext.get().addTaskCompletionListener[Unit](_ => { - materializedPartitions.add(1L) - }) + TaskContext.get().addTaskCompletionListener[Unit] { context => + if (!context.isFailed() && !context.isInterrupted()) { + materializedPartitions.add(1L) + } + } new Iterator[CachedBatch] { override def hasNext: Boolean = it.hasNext override def next(): CachedBatch = { From 0948e24c30f6f7a05110f6e45b6723897e095aeb Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 22 Dec 2023 23:25:12 +0800 Subject: [PATCH 156/521] [SPARK-46466][SQL][3.5] Vectorized parquet reader should never do rebase for timestamp ntz backport https://github.com/apache/spark/pull/44428 ### What changes were proposed in this pull request? This fixes a correctness bug. The TIMESTAMP_NTZ is a new data type in Spark and has no legacy files that need to do calendar rebase. However, the vectorized parquet reader treat it the same as LTZ and may do rebase if the parquet file was written with the legacy rebase mode. This PR fixes it to never do rebase for NTZ. ### Why are the changes needed? bug fix ### Does this PR introduce _any_ user-facing change? Yes, now we can correctly write and read back NTZ value even if the date is before 1582. ### How was this patch tested? new test ### Was this patch authored or co-authored using generative AI tooling? No Closes #44446 from cloud-fan/ntz2. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../parquet/ParquetVectorUpdaterFactory.java | 31 ++++++++++--------- .../parquet/ParquetQuerySuite.scala | 12 +++++++ 2 files changed, 29 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java index 42442cf8ea8a4..8c4fe20853879 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetVectorUpdaterFactory.java @@ -109,24 +109,32 @@ public ParquetVectorUpdater getUpdater(ColumnDescriptor descriptor, DataType spa // For unsigned int64, it stores as plain signed int64 in Parquet when dictionary // fallbacks. We read them as decimal values. return new UnsignedLongUpdater(); - } else if (isTimestamp(sparkType) && - isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MICROS)) { - validateTimestampType(sparkType); + } else if (sparkType == DataTypes.TimestampType && + isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MICROS)) { if ("CORRECTED".equals(datetimeRebaseMode)) { return new LongUpdater(); } else { boolean failIfRebase = "EXCEPTION".equals(datetimeRebaseMode); return new LongWithRebaseUpdater(failIfRebase, datetimeRebaseTz); } - } else if (isTimestamp(sparkType) && - isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MILLIS)) { - validateTimestampType(sparkType); + } else if (sparkType == DataTypes.TimestampType && + isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MILLIS)) { if ("CORRECTED".equals(datetimeRebaseMode)) { return new LongAsMicrosUpdater(); } else { final boolean failIfRebase = "EXCEPTION".equals(datetimeRebaseMode); return new LongAsMicrosRebaseUpdater(failIfRebase, datetimeRebaseTz); } + } else if (sparkType == DataTypes.TimestampNTZType && + isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MICROS)) { + validateTimestampNTZType(); + // TIMESTAMP_NTZ is a new data type and has no legacy files that need to do rebase. + return new LongUpdater(); + } else if (sparkType == DataTypes.TimestampNTZType && + isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit.MILLIS)) { + validateTimestampNTZType(); + // TIMESTAMP_NTZ is a new data type and has no legacy files that need to do rebase. + return new LongAsMicrosUpdater(); } else if (sparkType instanceof DayTimeIntervalType) { return new LongUpdater(); } @@ -196,12 +204,11 @@ boolean isTimestampTypeMatched(LogicalTypeAnnotation.TimeUnit unit) { ((TimestampLogicalTypeAnnotation) logicalTypeAnnotation).getUnit() == unit; } - void validateTimestampType(DataType sparkType) { + private void validateTimestampNTZType() { assert(logicalTypeAnnotation instanceof TimestampLogicalTypeAnnotation); - // Throw an exception if the Parquet type is TimestampLTZ and the Catalyst type is TimestampNTZ. + // Throw an exception if the Parquet type is TimestampLTZ as the Catalyst type is TimestampNTZ. // This is to avoid mistakes in reading the timestamp values. - if (((TimestampLogicalTypeAnnotation) logicalTypeAnnotation).isAdjustedToUTC() && - sparkType == DataTypes.TimestampNTZType) { + if (((TimestampLogicalTypeAnnotation) logicalTypeAnnotation).isAdjustedToUTC()) { convertErrorForTimestampNTZ("int64 time(" + logicalTypeAnnotation + ")"); } } @@ -1152,10 +1159,6 @@ private static boolean isLongDecimal(DataType dt) { return false; } - private static boolean isTimestamp(DataType dt) { - return dt == DataTypes.TimestampType || dt == DataTypes.TimestampNTZType; - } - private static boolean isDecimalTypeMatched(ColumnDescriptor descriptor, DataType dt) { DecimalType d = (DecimalType) dt; LogicalTypeAnnotation typeAnnotation = descriptor.getPrimitiveType().getLogicalTypeAnnotation(); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index ea5444a1791fd..828ec39c7d727 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -255,6 +255,18 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS } } + test("SPARK-46466: write and read TimestampNTZ with legacy rebase mode") { + withSQLConf(SQLConf.PARQUET_REBASE_MODE_IN_WRITE.key -> "LEGACY") { + withTable("ts") { + sql("create table ts (c1 timestamp_ntz) using parquet") + sql("insert into ts values (timestamp_ntz'0900-01-01 01:10:10')") + withAllParquetReaders { + checkAnswer(spark.table("ts"), sql("select timestamp_ntz'0900-01-01 01:10:10'")) + } + } + } + } + test("Enabling/disabling merging partfiles when merging parquet schema") { def testSchemaMerging(expectedColumnNumber: Int): Unit = { withTempDir { dir => From 432ab15013b7109d020fe66dee1c4287d9bc7cc3 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 27 Dec 2023 11:46:43 -0800 Subject: [PATCH 157/521] [SPARK-46478][SQL][3.5] Revert SPARK-43049 to use oracle varchar(255) for string ### What changes were proposed in this pull request? Revert SPARK-43049 to use Oracle Varchar (255) for string for performance consideration ### Why are the changes needed? for performance consideration ### Does this PR introduce _any_ user-facing change? yes, storing strings in Oracle table, which is defined by spark DDL with string columns. Users will get an error if string values exceed 255 ```java org.apache.spark.SparkRuntimeException: [EXCEED_LIMIT_LENGTH] Exceeds char/varchar type length limitation: 255. SQLSTATE: 54006 [info] at org.apache.spark.sql.errors.QueryExecutionErrors$.exceedMaxLimit(QueryExecutionErrors.scala:2512) ``` ### How was this patch tested? revised unit tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #44493 from yaooqinn/SPARK-46478-B. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- .../sql/jdbc/OracleIntegrationSuite.scala | 3 +-- .../sql/jdbc/v2/OracleIntegrationSuite.scala | 23 ++++++++++++++----- .../apache/spark/sql/jdbc/v2/V2JDBCTest.scala | 2 +- .../sql/catalyst/util/CharVarcharUtils.scala | 3 ++- .../apache/spark/sql/jdbc/OracleDialect.scala | 2 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 ++-- 6 files changed, 24 insertions(+), 13 deletions(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index 483f6087c81d2..70afad781ca25 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -173,8 +173,7 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSpark } - // SPARK-43049: Use CLOB instead of VARCHAR(255) for StringType for Oracle jdbc-am"" - test("SPARK-12941: String datatypes to be mapped to CLOB in Oracle") { + test("SPARK-12941: String datatypes to be mapped to VARCHAR(255) in Oracle") { // create a sample dataframe with string type val df1 = sparkContext.parallelize(Seq(("foo"))).toDF("x") // write the dataframe to the oracle table tbl diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/OracleIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/OracleIntegrationSuite.scala index 5124199328ce2..6b5dd043a617f 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/OracleIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/OracleIntegrationSuite.scala @@ -22,8 +22,9 @@ import java.util.Locale import org.scalatest.time.SpanSugar._ -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkRuntimeException} import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.util.CharVarcharUtils.CHAR_VARCHAR_TYPE_STRING_METADATA_KEY import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog import org.apache.spark.sql.jdbc.DatabaseOnDocker import org.apache.spark.sql.types._ @@ -86,6 +87,11 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTes s"jdbc:oracle:thin:system/$oracle_password@//$ip:$port/xe" } + override val defaultMetadata: Metadata = new MetadataBuilder() + .putLong("scale", 0) + .putString(CHAR_VARCHAR_TYPE_STRING_METADATA_KEY, "varchar(255)") + .build() + override def sparkConf: SparkConf = super.sparkConf .set("spark.sql.catalog.oracle", classOf[JDBCTableCatalog].getName) .set("spark.sql.catalog.oracle.url", db.getJdbcUrl(dockerIp, externalPort)) @@ -104,11 +110,11 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTes override def testUpdateColumnType(tbl: String): Unit = { sql(s"CREATE TABLE $tbl (ID INTEGER)") var t = spark.table(tbl) - var expectedSchema = new StructType().add("ID", DecimalType(10, 0), true, defaultMetadata) + var expectedSchema = new StructType().add("ID", DecimalType(10, 0), true, super.defaultMetadata) assert(t.schema === expectedSchema) sql(s"ALTER TABLE $tbl ALTER COLUMN id TYPE LONG") t = spark.table(tbl) - expectedSchema = new StructType().add("ID", DecimalType(19, 0), true, defaultMetadata) + expectedSchema = new StructType().add("ID", DecimalType(19, 0), true, super.defaultMetadata) assert(t.schema === expectedSchema) // Update column type from LONG to INTEGER val sql1 = s"ALTER TABLE $tbl ALTER COLUMN id TYPE INTEGER" @@ -129,12 +135,17 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTes override def caseConvert(tableName: String): String = tableName.toUpperCase(Locale.ROOT) - test("SPARK-43049: Use CLOB instead of VARCHAR(255) for StringType for Oracle JDBC") { + test("SPARK-46478: Revert SPARK-43049 to use varchar(255) for string") { val tableName = catalogName + ".t1" withTable(tableName) { sql(s"CREATE TABLE $tableName(c1 string)") - sql(s"INSERT INTO $tableName SELECT rpad('hi', 256, 'spark')") - assert(sql(s"SELECT char_length(c1) from $tableName").head().get(0) === 256) + checkError( + exception = intercept[SparkRuntimeException] { + sql(s"INSERT INTO $tableName SELECT rpad('hi', 256, 'spark')") + }, + errorClass = "EXCEED_LIMIT_LENGTH", + parameters = Map("limit" -> "255") + ) } } } diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala index b5f5b0e5f20bd..99f435611f2c4 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala @@ -49,7 +49,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu def notSupportsTableComment: Boolean = false - val defaultMetadata = new MetadataBuilder().putLong("scale", 0).build() + def defaultMetadata: Metadata = new MetadataBuilder().putLong("scale", 0).build() def testUpdateColumnNullability(tbl: String): Unit = { sql(s"CREATE TABLE $catalogName.alt_table (ID STRING NOT NULL)") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala index b9d83d444909d..f3c272785a7be 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharVarcharUtils.scala @@ -28,7 +28,8 @@ import org.apache.spark.sql.types._ object CharVarcharUtils extends Logging with SparkCharVarcharUtils { - private val CHAR_VARCHAR_TYPE_STRING_METADATA_KEY = "__CHAR_VARCHAR_TYPE_STRING" + // visible for testing + private[sql] val CHAR_VARCHAR_TYPE_STRING_METADATA_KEY = "__CHAR_VARCHAR_TYPE_STRING" /** * Replaces CharType/VarcharType with StringType recursively in the given struct type. If a diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala index 3a0333cca33fd..95774d38e50ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala @@ -118,7 +118,7 @@ private case object OracleDialect extends JdbcDialect { case DoubleType => Some(JdbcType("NUMBER(19, 4)", java.sql.Types.DOUBLE)) case ByteType => Some(JdbcType("NUMBER(3)", java.sql.Types.SMALLINT)) case ShortType => Some(JdbcType("NUMBER(5)", java.sql.Types.SMALLINT)) - case StringType => Some(JdbcType("CLOB", java.sql.Types.CLOB)) + case StringType => Some(JdbcType("VARCHAR2(255)", java.sql.Types.VARCHAR)) case _ => None } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 71c7245b06090..c4145f4cbf73b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -1274,7 +1274,7 @@ class JDBCSuite extends QueryTest with SharedSparkSession { test("SPARK 12941: The data type mapping for StringType to Oracle") { val oracleDialect = JdbcDialects.get("jdbc:oracle://127.0.0.1/db") assert(oracleDialect.getJDBCType(StringType). - map(_.databaseTypeDefinition).get == "CLOB") + map(_.databaseTypeDefinition).get == "VARCHAR2(255)") } test("SPARK-16625: General data types to be mapped to Oracle") { @@ -1292,7 +1292,7 @@ class JDBCSuite extends QueryTest with SharedSparkSession { assert(getJdbcType(oracleDialect, DoubleType) == "NUMBER(19, 4)") assert(getJdbcType(oracleDialect, ByteType) == "NUMBER(3)") assert(getJdbcType(oracleDialect, ShortType) == "NUMBER(5)") - assert(getJdbcType(oracleDialect, StringType) == "CLOB") + assert(getJdbcType(oracleDialect, StringType) == "VARCHAR2(255)") assert(getJdbcType(oracleDialect, BinaryType) == "BLOB") assert(getJdbcType(oracleDialect, DateType) == "DATE") assert(getJdbcType(oracleDialect, TimestampType) == "TIMESTAMP") From 5d4a913e3f3ba906c029e5e8a08194eac8ef250e Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 28 Dec 2023 10:53:02 +0800 Subject: [PATCH 158/521] [SPARK-46514][TESTS] Fix HiveMetastoreLazyInitializationSuite ### What changes were proposed in this pull request? This PR enabled the assertion in HiveMetastoreLazyInitializationSuite ### Why are the changes needed? fix test intenton ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? pass HiveMetastoreLazyInitializationSuite ### Was this patch authored or co-authored using generative AI tooling? no Closes #44500 from yaooqinn/SPARK-46514. Authored-by: Kent Yao Signed-off-by: Kent Yao (cherry picked from commit d0245d34c004935bb2c904bfd906836df3d574fa) Signed-off-by: Kent Yao --- .../spark/sql/hive/HiveMetastoreLazyInitializationSuite.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreLazyInitializationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreLazyInitializationSuite.scala index af11b817d65b0..b8739ce56e41a 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreLazyInitializationSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreLazyInitializationSuite.scala @@ -61,11 +61,10 @@ class HiveMetastoreLazyInitializationSuite extends SparkFunSuite { spark.sql("show tables") }) for (msg <- Seq( - "show tables", "Could not connect to meta store", "org.apache.thrift.transport.TTransportException", "Connection refused")) { - exceptionString.contains(msg) + assert(exceptionString.contains(msg)) } } finally { Thread.currentThread().setContextClassLoader(originalClassLoader) From 6838f0db692892fe5ffdd86e4a59a8e9733d5d1b Mon Sep 17 00:00:00 2001 From: zouxxyy Date: Thu, 28 Dec 2023 19:57:01 +0300 Subject: [PATCH 159/521] [SPARK-46535][SQL] Fix NPE when describe extended a column without col stats ### What changes were proposed in this pull request? ### Why are the changes needed? Currently executing DESCRIBE TABLE EXTENDED a column without col stats with v2 table will throw a null pointer exception. ```text Cannot invoke "org.apache.spark.sql.connector.read.colstats.ColumnStatistics.min()" because the return value of "scala.Option.get()" is null java.lang.NullPointerException: Cannot invoke "org.apache.spark.sql.connector.read.colstats.ColumnStatistics.min()" because the return value of "scala.Option.get()" is null at org.apache.spark.sql.execution.datasources.v2.DescribeColumnExec.run(DescribeColumnExec.scala:63) at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43) at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43) at org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49) at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:118) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$6(SQLExecution.scala:150) at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:241) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$1(SQLExecution.scala:116) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:918) ``` This RP will fix it ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? Add a new test `describe extended (formatted) a column without col stats` ### Was this patch authored or co-authored using generative AI tooling? Closes #44524 from Zouxxyy/dev/fix-stats. Lead-authored-by: zouxxyy Co-authored-by: Kent Yao Signed-off-by: Max Gekk (cherry picked from commit af8228ce9aee99eae9d08dbdefaaad32cf5438eb) Signed-off-by: Max Gekk --- .../datasources/v2/DescribeColumnExec.scala | 2 +- .../command/v2/DescribeTableSuite.scala | 21 +++++++++++++++++++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala index 61ccda3fc9543..2683d8d547f00 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala @@ -53,7 +53,7 @@ case class DescribeColumnExec( read.newScanBuilder(CaseInsensitiveStringMap.empty()).build() match { case s: SupportsReportStatistics => val stats = s.estimateStatistics() - Some(stats.columnStats().get(FieldReference.column(column.name))) + Option(stats.columnStats().get(FieldReference.column(column.name))) case _ => None } case _ => None diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala index e2f2aee56115f..a21baebe24d8f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala @@ -175,4 +175,25 @@ class DescribeTableSuite extends command.DescribeTableSuiteBase Row("max_col_len", "NULL"))) } } + + test("SPARK-46535: describe extended (formatted) a column without col stats") { + withNamespaceAndTable("ns", "tbl") { tbl => + sql( + s""" + |CREATE TABLE $tbl + |(key INT COMMENT 'column_comment', col STRING) + |$defaultUsing""".stripMargin) + + val descriptionDf = sql(s"DESCRIBE TABLE EXTENDED $tbl key") + assert(descriptionDf.schema.map(field => (field.name, field.dataType)) === Seq( + ("info_name", StringType), + ("info_value", StringType))) + QueryTest.checkAnswer( + descriptionDf, + Seq( + Row("col_name", "key"), + Row("data_type", "int"), + Row("comment", "column_comment"))) + } + } } From f0e5fc973c1c87dbe1b0574e5d14e97b55abfa03 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Tue, 2 Jan 2024 08:13:29 -0800 Subject: [PATCH 160/521] [SPARK-46562][SQL] Remove retrieval of `keytabFile` from `UserGroupInformation` in `HiveAuthFactory` ### What changes were proposed in this pull request? This pr removed the retrieval of `keytabFile` from `UserGroupInformation` in `HiveAuthFactory` because `keytabFile` no longer exists in `UserGroupInformation` after Hadoop 3.0.3. Therefore, in `HiveAuthFactory`, `keytabFile` will always be null and in `HiveAuthFactory`, `keytabFile` will only be used when it is not null. For the specific changes in Hadoop, please refer to https://issues.apache.org/jira/browse/HADOOP-9747 | https://github.com/apache/hadoop/commit/59cf7588779145ad5850ad63426743dfe03d8347. ### Why are the changes needed? Clean up the invalid code. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #44557 from LuciferYang/remove-keytabFile. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun (cherry picked from commit bc7e949cf99382ecf70d5b59fca9e7e415fbbb48) Signed-off-by: Dongjoon Hyun --- .../apache/hive/service/auth/HiveAuthFactory.java | 14 +------------- 1 file changed, 1 insertion(+), 13 deletions(-) diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java index 8d77b238ff41f..e3316cef241c3 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java @@ -17,7 +17,6 @@ package org.apache.hive.service.auth; import java.io.IOException; -import java.lang.reflect.Field; import java.lang.reflect.Method; import java.util.HashMap; import java.util.Map; @@ -85,18 +84,9 @@ public String getAuthName() { public static final String HS2_PROXY_USER = "hive.server2.proxy.user"; public static final String HS2_CLIENT_TOKEN = "hiveserver2ClientToken"; - private static Field keytabFile = null; private static Method getKeytab = null; static { Class clz = UserGroupInformation.class; - try { - keytabFile = clz.getDeclaredField("keytabFile"); - keytabFile.setAccessible(true); - } catch (NoSuchFieldException nfe) { - LOG.debug("Cannot find private field \"keytabFile\" in class: " + - UserGroupInformation.class.getCanonicalName(), nfe); - keytabFile = null; - } try { getKeytab = clz.getDeclaredMethod("getKeytab"); @@ -347,9 +337,7 @@ public static boolean needUgiLogin(UserGroupInformation ugi, String principal, S private static String getKeytabFromUgi() { synchronized (UserGroupInformation.class) { try { - if (keytabFile != null) { - return (String) keytabFile.get(null); - } else if (getKeytab != null) { + if (getKeytab != null) { return (String) getKeytab.invoke(UserGroupInformation.getCurrentUser()); } else { return null; From 6e8dbacf8a1402878a2a4be295bbe78e7c78327e Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Fri, 15 Dec 2023 17:52:46 -0800 Subject: [PATCH 161/521] [SPARK-46425][INFRA] Pin the bundler version in CI Currently documentation build is broken: https://github.com/apache/spark/actions/runs/7226413850/job/19691970695 ``` ... ERROR: Error installing bundler: The last version of bundler (>= 0) to support your Ruby & RubyGems was 2.4.22. Try installing it with `gem install bundler -v 2.4.22` bundler requires Ruby version >= 3.0.0. The current ruby version is 2.7.0.0. ``` This PR uses the suggestion. To recover the CI. No, dev-only. CI in this PR verify it. No. Closes #44376 from HyukjinKwon/SPARK-46425. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit d0da1172b7d87b68a8af8464c6486aa586324241) Signed-off-by: Dongjoon Hyun --- .github/workflows/build_and_test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index f202a7d49c9a2..32f6a44102bf9 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -687,7 +687,7 @@ jobs: Rscript -e "install.packages(c('devtools', 'testthat', 'knitr', 'rmarkdown', 'markdown', 'e1071', 'roxygen2', 'ggplot2', 'mvtnorm', 'statmod'), repos='https://cloud.r-project.org/')" Rscript -e "devtools::install_version('pkgdown', version='2.0.1', repos='https://cloud.r-project.org')" Rscript -e "devtools::install_version('preferably', version='0.4', repos='https://cloud.r-project.org')" - gem install bundler + gem install bundler -v 2.4.22 cd docs bundle install - name: R linter From 2891d92e9d8a5050f457bb116530d46de3babf97 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 3 Jan 2024 05:54:57 -0800 Subject: [PATCH 162/521] [SPARK-46577][SQL] HiveMetastoreLazyInitializationSuite leaks hive's SessionState MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? The upcoming tests with the new hive configurations will have no effect due to the leaked SessionState. ``` 06:21:12.848 pool-1-thread-1 INFO ThriftServerWithSparkContextInHttpSuite: Trying to start HiveThriftServer2: mode=http, attempt=0 .... 06:21:12.851 pool-1-thread-1 INFO AbstractService: Service:OperationManager is inited. 06:21:12.851 pool-1-thread-1 INFO AbstractService: Service:SessionManager is inited. 06:21:12.851 pool-1-thread-1 INFO AbstractService: Service: CLIService is inited. 06:21:12.851 pool-1-thread-1 INFO AbstractService: Service:ThriftBinaryCLIService is inited. 06:21:12.851 pool-1-thread-1 INFO AbstractService: Service: HiveServer2 is inited. 06:21:12.851 pool-1-thread-1 INFO AbstractService: Service:OperationManager is started. 06:21:12.851 pool-1-thread-1 INFO AbstractService: Service:SessionManager is started. 06:21:12.851 pool-1-thread-1 INFO AbstractService: Service: CLIService is started. 06:21:12.852 pool-1-thread-1 INFO AbstractService: Service:ThriftBinaryCLIService is started. 06:21:12.852 pool-1-thread-1 INFO ThriftCLIService: Starting ThriftBinaryCLIService on port 10000 with 5...500 worker threads 06:21:12.852 pool-1-thread-1 INFO AbstractService: Service:HiveServer2 is started. ``` As the logs above revealed, ThriftServerWithSparkContextInHttpSuite started the ThriftBinaryCLIService instead of the ThriftHttpCLIService. This is because in HiveClientImpl, the new configurations are only applied to hive conf during initializing but not for existing ones. This cause ThriftServerWithSparkContextInHttpSuite retrying or even aborting. ### Why are the changes needed? Fix flakiness in tests ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ran tests locally with the hive-thriftserver module locally, ### Was this patch authored or co-authored using generative AI tooling? no Closes #44578 from yaooqinn/SPARK-46577. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun (cherry picked from commit 605fecd22cc18fc9b93fb26d4aa6088f5a314f92) Signed-off-by: Dongjoon Hyun --- .../sql/hive/HiveMetastoreLazyInitializationSuite.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreLazyInitializationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreLazyInitializationSuite.scala index b8739ce56e41a..cb85993e5e099 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreLazyInitializationSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreLazyInitializationSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.hadoop.hive.ql.session.SessionState import org.apache.logging.log4j.LogManager import org.apache.logging.log4j.core.Logger @@ -69,6 +71,10 @@ class HiveMetastoreLazyInitializationSuite extends SparkFunSuite { } finally { Thread.currentThread().setContextClassLoader(originalClassLoader) spark.sparkContext.setLogLevel(originalLevel.toString) + SparkSession.clearActiveSession() + SparkSession.clearDefaultSession() + SessionState.detachSession() + Hive.closeCurrent() spark.stop() } } From fb90ade2c7390077d2755fc43b73e63f5cf44f21 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Wed, 3 Jan 2024 12:07:15 -0800 Subject: [PATCH 163/521] [SPARK-46546][DOCS] Fix the formatting of tables in `running-on-yarn` pages ### What changes were proposed in this pull request? The pr aims to fix the formatting of tables in `running-on-yarn` pages. ### Why are the changes needed? Make the tables on the page display normally. Before: image After: image ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually check. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44540 from panbingkun/SPARK-46546. Authored-by: panbingkun Signed-off-by: Dongjoon Hyun (cherry picked from commit 85b44ccef4c4aeec302c12e03833590c7d8d6b9e) Signed-off-by: Dongjoon Hyun --- docs/running-on-yarn.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 9b4e59a119eeb..ce7121b806cb0 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -866,7 +866,7 @@ to avoid garbage collection issues during shuffle. The following extra configuration options are available when the shuffle service is running on YARN:
diff --git a/docs/sql-performance-tuning.md b/docs/sql-performance-tuning.md index 1467409bb500d..2dec65cc553ed 100644 --- a/docs/sql-performance-tuning.md +++ b/docs/sql-performance-tuning.md @@ -34,7 +34,7 @@ memory usage and GC pressure. You can call `spark.catalog.uncacheTable("tableNam Configuration of in-memory caching can be done using the `setConf` method on `SparkSession` or by running `SET key=value` commands using SQL. - +
@@ -62,7 +62,7 @@ Configuration of in-memory caching can be done using the `setConf` method on `Sp The following options can also be used to tune the performance of query execution. It is possible that these options will be deprecated in future release as more optimizations are performed automatically. -
Property NameDefaultMeaningSince Version
spark.sql.inMemoryColumnarStorage.compressed
+
@@ -253,7 +253,7 @@ Adaptive Query Execution (AQE) is an optimization technique in Spark SQL that ma ### Coalescing Post Shuffle Partitions This feature coalesces the post shuffle partitions based on the map output statistics when both `spark.sql.adaptive.enabled` and `spark.sql.adaptive.coalescePartitions.enabled` configurations are true. This feature simplifies the tuning of shuffle partition number when running queries. You do not need to set a proper shuffle partition number to fit your dataset. Spark can pick the proper shuffle partition number at runtime once you set a large enough initial number of shuffle partitions via `spark.sql.adaptive.coalescePartitions.initialPartitionNum` configuration. -
Property NameDefaultMeaningSince Version
spark.sql.files.maxPartitionBytes
+
@@ -298,7 +298,7 @@ This feature coalesces the post shuffle partitions based on the map output stati
Property NameDefaultMeaningSince Version
spark.sql.adaptive.coalescePartitions.enabled
### Spliting skewed shuffle partitions - +
@@ -320,7 +320,7 @@ This feature coalesces the post shuffle partitions based on the map output stati ### Converting sort-merge join to broadcast join AQE converts sort-merge join to broadcast hash join when the runtime statistics of any join side is smaller than the adaptive broadcast hash join threshold. This is not as efficient as planning a broadcast hash join in the first place, but it's better than keep doing the sort-merge join, as we can save the sorting of both the join sides, and read shuffle files locally to save network traffic(if `spark.sql.adaptive.localShuffleReader.enabled` is true) -
Property NameDefaultMeaningSince Version
spark.sql.adaptive.optimizeSkewsInRebalancePartitions.enabled
+
@@ -342,7 +342,7 @@ AQE converts sort-merge join to broadcast hash join when the runtime statistics ### Converting sort-merge join to shuffled hash join AQE converts sort-merge join to shuffled hash join when all post shuffle partitions are smaller than a threshold, the max threshold can see the config `spark.sql.adaptive.maxShuffledHashJoinLocalMapThreshold`. -
Property NameDefaultMeaningSince Version
spark.sql.adaptive.autoBroadcastJoinThreshold
+
@@ -356,7 +356,7 @@ AQE converts sort-merge join to shuffled hash join when all post shuffle partiti ### Optimizing Skew Join Data skew can severely downgrade the performance of join queries. This feature dynamically handles skew in sort-merge join by splitting (and replicating if needed) skewed tasks into roughly evenly sized tasks. It takes effect when both `spark.sql.adaptive.enabled` and `spark.sql.adaptive.skewJoin.enabled` configurations are enabled. -
Property NameDefaultMeaningSince Version
spark.sql.adaptive.maxShuffledHashJoinLocalMapThreshold
+
@@ -393,7 +393,7 @@ Data skew can severely downgrade the performance of join queries. This feature d
Property NameDefaultMeaningSince Version
spark.sql.adaptive.skewJoin.enabled
### Misc - +
diff --git a/docs/storage-openstack-swift.md b/docs/storage-openstack-swift.md index 73b21a1f7c27b..5b30786bdd7f9 100644 --- a/docs/storage-openstack-swift.md +++ b/docs/storage-openstack-swift.md @@ -60,7 +60,7 @@ required by Keystone. The following table contains a list of Keystone mandatory parameters. PROVIDER can be any (alphanumeric) name. -
Property NameDefaultMeaningSince Version
spark.sql.adaptive.optimizer.excludedRules
+
diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index 591a4415bb1a5..11a52232510fd 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -243,7 +243,7 @@ interval in the [Spark Streaming Programming Guide](streaming-programming-guide. The following table summarizes the characteristics of both types of receivers -
Property NameMeaningRequired
fs.swift.service.PROVIDER.auth.url
+
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index f8f98ca54425d..4b93fb7c89ad1 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -433,7 +433,7 @@ Streaming core artifact `spark-streaming-xyz_{{site.SCALA_BINARY_VERSION}}` to the dependencies. For example, some of the common ones are as follows. -
Receiver Type
+
@@ -820,7 +820,7 @@ Similar to that of RDDs, transformations allow the data from the input DStream t DStreams support many of the transformations available on normal Spark RDD's. Some of the common ones are as follows. -
SourceArtifact
Kafka spark-streaming-kafka-0-10_{{site.SCALA_BINARY_VERSION}}
Kinesis
spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}} [Amazon Software License]
+
@@ -1109,7 +1109,7 @@ JavaPairDStream windowedWordCounts = pairs.reduceByKeyAndWindow Some of the common window operations are as follows. All of these operations take the said two parameters - windowLength and slideInterval. -
TransformationMeaning
map(func)
+
@@ -1280,7 +1280,7 @@ Since the output operations actually allow the transformed data to be consumed b they trigger the actual execution of all the DStream transformations (similar to actions for RDDs). Currently, the following output operations are defined: -
TransformationMeaning
window(windowLength, slideInterval)
+
@@ -2485,7 +2485,7 @@ enabled](#deploying-applications) and reliable receivers, there is zero data los The following table summarizes the semantics under failures: -
Output OperationMeaning
print()
+
diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 66e6efb1c8a9f..c5ffdf025b173 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -297,7 +297,7 @@ df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); Each row in the source has the following schema: -
Deployment Scenario
+
@@ -336,7 +336,7 @@ Each row in the source has the following schema: The following options must be set for the Kafka source for both batch and streaming queries. -
ColumnType
key
+
@@ -368,7 +368,7 @@ for both batch and streaming queries. The following configurations are optional: -
Optionvaluemeaning
assign
+
@@ -607,7 +607,7 @@ The caching key is built up from the following information: The following properties are available to configure the consumer pool: -
Optionvaluedefaultquery typemeaning
startingTimestamp
+
@@ -657,7 +657,7 @@ Note that it doesn't leverage Apache Commons Pool due to the difference of chara The following properties are available to configure the fetched data pool: -
Property NameDefaultMeaningSince Version
spark.kafka.consumer.cache.capacity
+
@@ -685,7 +685,7 @@ solution to remove duplicates when reading the written data could be to introduc that can be used to perform de-duplication when reading. The Dataframe being written to Kafka should have the following columns in schema: -
Property NameDefaultMeaningSince Version
spark.kafka.consumer.fetchedData.cache.timeout
+
@@ -725,7 +725,7 @@ will be used. The following options must be set for the Kafka sink for both batch and streaming queries. -
ColumnType
key (optional)
+
@@ -736,7 +736,7 @@ for both batch and streaming queries. The following configurations are optional: -
Optionvaluemeaning
kafka.bootstrap.servers
+
@@ -912,7 +912,7 @@ It will use different Kafka producer when delegation token is renewed; Kafka pro The following properties are available to configure the producer pool: -
Optionvaluedefaultquery typemeaning
topic
+
@@ -1039,7 +1039,7 @@ When none of the above applies then unsecure connection assumed. Delegation tokens can be obtained from multiple clusters and ${cluster} is an arbitrary unique identifier which helps to group different configurations. -
Property NameDefaultMeaningSince Version
spark.kafka.producer.cache.timeout
+
diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 3e87c45a34915..845f0617898b4 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -545,7 +545,7 @@ checkpointed offsets after a failure. See the earlier section on [fault-tolerance semantics](#fault-tolerance-semantics). Here are the details of all the sources in Spark. -
Property NameDefaultMeaningSince Version
spark.kafka.clusters.${cluster}.auth.bootstrap.servers
+
@@ -1819,7 +1819,7 @@ regarding watermark delays and whether data will be dropped or not. ##### Support matrix for joins in streaming queries -
Source
+
@@ -2307,7 +2307,7 @@ to `org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider`. Here are the configs regarding to RocksDB instance of the state store provider: -
Left Input
+
@@ -2474,7 +2474,7 @@ More information to be added in future releases. Different types of streaming queries support different output modes. Here is the compatibility matrix. -
Config Name
+
@@ -2613,7 +2613,7 @@ meant for debugging purposes only. See the earlier section on [fault-tolerance semantics](#fault-tolerance-semantics). Here are the details of all the sinks in Spark. -
Query Type
+
@@ -3201,7 +3201,7 @@ The trigger settings of a streaming query define the timing of streaming data pr the query is going to be executed as micro-batch query with a fixed batch interval or as a continuous processing query. Here are the different kinds of triggers that are supported. -
Sink
+
diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index becdfb4b18f5d..4821f883eef9d 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -159,7 +159,7 @@ export HADOOP_CONF_DIR=XXX The master URL passed to Spark can be in one of the following formats: -
Trigger Type
+
diff --git a/docs/web-ui.md b/docs/web-ui.md index 079bc6137f020..cdf62e0d8ec0b 100644 --- a/docs/web-ui.md +++ b/docs/web-ui.md @@ -380,7 +380,7 @@ operator shows the number of bytes written by a shuffle. Here is the list of SQL metrics: -
Master URLMeaning
local Run Spark locally with one worker thread (i.e. no parallelism at all).
local[K] Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine).
+
From 53e2e7bdd618e2a7dec5a84b9d5ae965fb136179 Mon Sep 17 00:00:00 2001 From: Bruce Robbins Date: Fri, 1 Dec 2023 10:28:33 +0800 Subject: [PATCH 134/521] [SPARK-46189][PS][SQL] Perform comparisons and arithmetic between same types in various Pandas aggregate functions to avoid interpreted mode errors ### What changes were proposed in this pull request? In various Pandas aggregate functions, remove each comparison or arithmetic operation between `DoubleType` and `IntergerType` in `evaluateExpression` and replace with a comparison or arithmetic operation between `DoubleType` and `DoubleType`. Affected functions are `PandasStddev`, `PandasVariance`, `PandasSkewness`, `PandasKurtosis`, and `PandasCovar`. ### Why are the changes needed? These functions fail in interpreted mode. For example, `evaluateExpression` in `PandasKurtosis` compares a double to an integer: ``` If(n < 4, Literal.create(null, DoubleType) ... ``` This results in a boxed double and a boxed integer getting passed to `SQLOrderingUtil.compareDoubles` which expects two doubles as arguments. The scala runtime tries to unbox the boxed integer as a double, resulting in an error. Reproduction example: ``` spark.sql("set spark.sql.codegen.wholeStage=false") spark.sql("set spark.sql.codegen.factoryMode=NO_CODEGEN") import numpy as np import pandas as pd import pyspark.pandas as ps pser = pd.Series([1, 2, 3, 7, 9, 8], index=np.random.rand(6), name="a") psser = ps.from_pandas(pser) psser.kurt() ``` See Jira (SPARK-46189) for the other reproduction cases. This works fine in codegen mode because the integer is already unboxed and the Java runtime will implictly cast it to a double. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44099 from bersprockets/unboxing_error. Authored-by: Bruce Robbins Signed-off-by: Ruifeng Zheng (cherry picked from commit 042d8546be5d160e203ad78a8aa2e12e74142338) Signed-off-by: Ruifeng Zheng --- .../aggregate/CentralMomentAgg.scala | 16 ++-- .../expressions/aggregate/Covariance.scala | 2 +- .../aggregate/CentralMomentAggSuite.scala | 77 +++++++++++++++++++ .../aggregate/CovarianceAggSuite.scala | 39 ++++++++++ .../DeclarativeAggregateEvaluator.scala | 10 +-- .../aggregate/TestWithAndWithoutCodegen.scala | 35 +++++++++ 6 files changed, 165 insertions(+), 14 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAggSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CovarianceAggSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/TestWithAndWithoutCodegen.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala index 133a39d987459..316cb9e0bbc34 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala @@ -353,7 +353,7 @@ case class PandasStddev( override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), - If(n === ddof, divideByZeroEvalResult, sqrt(m2 / (n - ddof)))) + If(n === ddof.toDouble, divideByZeroEvalResult, sqrt(m2 / (n - ddof.toDouble)))) } override def prettyName: String = "pandas_stddev" @@ -375,7 +375,7 @@ case class PandasVariance( override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), - If(n === ddof, divideByZeroEvalResult, m2 / (n - ddof))) + If(n === ddof.toDouble, divideByZeroEvalResult, m2 / (n - ddof.toDouble))) } override def prettyName: String = "pandas_variance" @@ -405,8 +405,8 @@ case class PandasSkewness(child: Expression) val _m2 = If(abs(m2) < 1e-14, Literal(0.0), m2) val _m3 = If(abs(m3) < 1e-14, Literal(0.0), m3) - If(n < 3, Literal.create(null, DoubleType), - If(_m2 === 0.0, Literal(0.0), sqrt(n - 1) * (n / (n - 2)) * _m3 / sqrt(_m2 * _m2 * _m2))) + If(n < 3.0, Literal.create(null, DoubleType), + If(_m2 === 0.0, Literal(0.0), sqrt(n - 1.0) * (n / (n - 2.0)) * _m3 / sqrt(_m2 * _m2 * _m2))) } override protected def withNewChildInternal(newChild: Expression): PandasSkewness = @@ -423,9 +423,9 @@ case class PandasKurtosis(child: Expression) override protected def momentOrder = 4 override val evaluateExpression: Expression = { - val adj = ((n - 1) / (n - 2)) * ((n - 1) / (n - 3)) * 3 - val numerator = n * (n + 1) * (n - 1) * m4 - val denominator = (n - 2) * (n - 3) * m2 * m2 + val adj = ((n - 1.0) / (n - 2.0)) * ((n - 1.0) / (n - 3.0)) * 3.0 + val numerator = n * (n + 1.0) * (n - 1.0) * m4 + val denominator = (n - 2.0) * (n - 3.0) * m2 * m2 // floating point error // @@ -436,7 +436,7 @@ case class PandasKurtosis(child: Expression) val _numerator = If(abs(numerator) < 1e-14, Literal(0.0), numerator) val _denominator = If(abs(denominator) < 1e-14, Literal(0.0), denominator) - If(n < 4, Literal.create(null, DoubleType), + If(n < 4.0, Literal.create(null, DoubleType), If(_denominator === 0.0, Literal(0.0), _numerator / _denominator - adj)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala index ff31fb1128b9b..b392b603ab8d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala @@ -156,7 +156,7 @@ case class PandasCovar( override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), - If(n === ddof, divideByZeroEvalResult, ck / (n - ddof))) + If(n === ddof.toDouble, divideByZeroEvalResult, ck / (n - ddof.toDouble))) } override def prettyName: String = "pandas_covar" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAggSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAggSuite.scala new file mode 100644 index 0000000000000..daf3ede0d0369 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAggSuite.scala @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions.aggregate + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.types.DoubleType + +class CentralMomentAggSuite extends TestWithAndWithoutCodegen { + val input = AttributeReference("input", DoubleType, nullable = true)() + + testBothCodegenAndInterpreted("SPARK-46189: pandas_kurtosis eval") { + val evaluator = DeclarativeAggregateEvaluator(PandasKurtosis(input), Seq(input)) + val buffer = evaluator.update( + InternalRow(1.0d), + InternalRow(2.0d), + InternalRow(3.0d), + InternalRow(7.0d), + InternalRow(9.0d), + InternalRow(8.0d)) + val result = evaluator.eval(buffer) + assert(result === InternalRow(-2.5772889417360285d)) + } + + testBothCodegenAndInterpreted("SPARK-46189: pandas_skew eval") { + val evaluator = DeclarativeAggregateEvaluator(PandasSkewness(input), Seq(input)) + val buffer = evaluator.update( + InternalRow(1.0d), + InternalRow(2.0d), + InternalRow(2.0d), + InternalRow(2.0d), + InternalRow(2.0d), + InternalRow(100.0d)) + val result = evaluator.eval(buffer) + assert(result === InternalRow(2.4489389171333733d)) + } + + testBothCodegenAndInterpreted("SPARK-46189: pandas_stddev eval") { + val evaluator = DeclarativeAggregateEvaluator(PandasStddev(input, 1), Seq(input)) + val buffer = evaluator.update( + InternalRow(1.0d), + InternalRow(2.0d), + InternalRow(3.0d), + InternalRow(7.0d), + InternalRow(9.0d), + InternalRow(8.0d)) + val result = evaluator.eval(buffer) + assert(result === InternalRow(3.40587727318528d)) + } + + testBothCodegenAndInterpreted("SPARK-46189: pandas_variance eval") { + val evaluator = DeclarativeAggregateEvaluator(PandasVariance(input, 1), Seq(input)) + val buffer = evaluator.update( + InternalRow(1.0d), + InternalRow(2.0d), + InternalRow(3.0d), + InternalRow(7.0d), + InternalRow(9.0d), + InternalRow(8.0d)) + val result = evaluator.eval(buffer) + assert(result === InternalRow(11.6d)) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CovarianceAggSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CovarianceAggSuite.scala new file mode 100644 index 0000000000000..2df053184c2b4 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CovarianceAggSuite.scala @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions.aggregate + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.types.DoubleType + +class CovarianceAggSuite extends TestWithAndWithoutCodegen { + val a = AttributeReference("a", DoubleType, nullable = true)() + val b = AttributeReference("b", DoubleType, nullable = true)() + + testBothCodegenAndInterpreted("SPARK-46189: pandas_covar eval") { + val evaluator = DeclarativeAggregateEvaluator(PandasCovar(a, b, 1), Seq(a, b)) + val buffer = evaluator.update( + InternalRow(1.0d, 1.0d), + InternalRow(2.0d, 2.0d), + InternalRow(3.0d, 3.0d), + InternalRow(7.0d, 7.0d), + InternalRow(9.0, 9.0), + InternalRow(8.0d, 6.0)) + val result = evaluator.eval(buffer) + assert(result === InternalRow(10.4d)) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala index b0f55b3b5c443..ac80e1419a99d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala @@ -17,24 +17,24 @@ package org.apache.spark.sql.catalyst.expressions.aggregate import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, JoinedRow, SafeProjection} +import org.apache.spark.sql.catalyst.expressions.{Attribute, JoinedRow, MutableProjection} /** * Evaluator for a [[DeclarativeAggregate]]. */ case class DeclarativeAggregateEvaluator(function: DeclarativeAggregate, input: Seq[Attribute]) { - lazy val initializer = SafeProjection.create(function.initialValues) + lazy val initializer = MutableProjection.create(function.initialValues) - lazy val updater = SafeProjection.create( + lazy val updater = MutableProjection.create( function.updateExpressions, function.aggBufferAttributes ++ input) - lazy val merger = SafeProjection.create( + lazy val merger = MutableProjection.create( function.mergeExpressions, function.aggBufferAttributes ++ function.inputAggBufferAttributes) - lazy val evaluator = SafeProjection.create( + lazy val evaluator = MutableProjection.create( function.evaluateExpression :: Nil, function.aggBufferAttributes) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/TestWithAndWithoutCodegen.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/TestWithAndWithoutCodegen.scala new file mode 100644 index 0000000000000..b43b160146eb4 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/TestWithAndWithoutCodegen.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions.aggregate + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.internal.SQLConf + +trait TestWithAndWithoutCodegen extends SparkFunSuite with SQLHelper { + def testBothCodegenAndInterpreted(name: String)(f: => Unit): Unit = { + val modes = Seq(CodegenObjectFactoryMode.CODEGEN_ONLY, CodegenObjectFactoryMode.NO_CODEGEN) + for (fallbackMode <- modes) { + test(s"$name with $fallbackMode") { + withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> fallbackMode.toString) { + f + } + } + } + } +} From fde0fe676358fb3e9142d6895e4c7fc2e6604d5e Mon Sep 17 00:00:00 2001 From: wforget <643348094@qq.com> Date: Sat, 2 Dec 2023 11:25:54 -0800 Subject: [PATCH 135/521] [SPARK-45975][SQL][TESTS][3.5] Reset storeAssignmentPolicy to original ### What changes were proposed in this pull request? Reset storeAssignmentPolicy to original in HiveCompatibilitySuite. ### Why are the changes needed? STORE_ASSIGNMENT_POLICY was not reset in HiveCompatibilitySuite, causing subsequent test cases to fail. Details: https://github.com/wForget/spark/actions/runs/6902668865/job/18779862759 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #44126 from LuciferYang/SPARK-45943-FOLLOWUP. Authored-by: wforget <643348094@qq.com> Signed-off-by: Dongjoon Hyun --- .../spark/sql/hive/execution/HiveCompatibilitySuite.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index bd323dc4b24e1..0467603c01cd0 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -41,6 +41,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled private val originalSessionLocalTimeZone = TestHive.conf.sessionLocalTimeZone private val originalAnsiMode = TestHive.conf.getConf(SQLConf.ANSI_ENABLED) + private val originalStoreAssignmentPolicy = + TestHive.conf.getConf(SQLConf.STORE_ASSIGNMENT_POLICY) private val originalCreateHiveTable = TestHive.conf.getConf(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT) @@ -76,6 +78,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, originalCrossJoinEnabled) TestHive.setConf(SQLConf.SESSION_LOCAL_TIMEZONE, originalSessionLocalTimeZone) TestHive.setConf(SQLConf.ANSI_ENABLED, originalAnsiMode) + TestHive.setConf(SQLConf.STORE_ASSIGNMENT_POLICY, originalStoreAssignmentPolicy) TestHive.setConf(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT, originalCreateHiveTable) // For debugging dump some statistics about how much time was spent in various optimizer rules From 273ef5708fc33872cfe3091627617bbac8fdd56f Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Sun, 3 Dec 2023 22:08:20 -0800 Subject: [PATCH 136/521] [SPARK-46182][CORE] Track `lastTaskFinishTime` using the exact task finished event ### What changes were proposed in this pull request? We found a race condition between lastTaskRunningTime and lastShuffleMigrationTime that could lead to a decommissioned executor exit before all the shuffle blocks have been discovered. The issue could lead to immediate task retry right after an executor exit, thus longer query execution time. To fix the issue, we choose to update the lastTaskRunningTime only when a task updates its status to finished through the StatusUpdate event. This is better than the current approach (which use a thread to check for number of running tasks every second), because in this way we clearly know whether the shuffle block refresh happened after all tasks finished running or not, thus resolved the race condition mentioned above. ### Why are the changes needed? To fix a race condition that could lead to shuffle data lost, thus longer query execution time. ### How was this patch tested? This is a very subtle race condition that is hard to write a unit test using current unit test framework. And we are confident the change is low risk. Thus only verify by passing all the existing tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #44090 from jiangxb1987/SPARK-46182. Authored-by: Xingbo Jiang Signed-off-by: Dongjoon Hyun (cherry picked from commit 6f112f7b1a50a2b8a59952c69f67dd5f80ab6633) Signed-off-by: Dongjoon Hyun --- .../executor/CoarseGrainedExecutorBackend.scala | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index c695a9ec2851b..537522326fc78 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -21,7 +21,7 @@ import java.net.URL import java.nio.ByteBuffer import java.util.Locale import java.util.concurrent.ConcurrentHashMap -import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} import scala.util.{Failure, Success} import scala.util.control.NonFatal @@ -80,6 +80,10 @@ private[spark] class CoarseGrainedExecutorBackend( private var decommissioned = false + // Track the last time in ns that at least one task is running. If no task is running and all + // shuffle/RDD data migration are done, the decommissioned executor should exit. + private var lastTaskFinishTime = new AtomicLong(System.nanoTime()) + override def onStart(): Unit = { if (env.conf.get(DECOMMISSION_ENABLED)) { val signal = env.conf.get(EXECUTOR_DECOMMISSION_SIGNAL) @@ -269,6 +273,7 @@ private[spark] class CoarseGrainedExecutorBackend( val msg = StatusUpdate(executorId, taskId, state, data, cpus, resources) if (TaskState.isFinished(state)) { taskResources.remove(taskId) + lastTaskFinishTime.set(System.nanoTime()) } driver match { case Some(driverRef) => driverRef.send(msg) @@ -341,7 +346,6 @@ private[spark] class CoarseGrainedExecutorBackend( val shutdownThread = new Thread("wait-for-blocks-to-migrate") { override def run(): Unit = { - var lastTaskRunningTime = System.nanoTime() val sleep_time = 1000 // 1s // This config is internal and only used by unit tests to force an executor // to hang around for longer when decommissioned. @@ -358,7 +362,7 @@ private[spark] class CoarseGrainedExecutorBackend( val (migrationTime, allBlocksMigrated) = env.blockManager.lastMigrationInfo() // We can only trust allBlocksMigrated boolean value if there were no tasks running // since the start of computing it. - if (allBlocksMigrated && (migrationTime > lastTaskRunningTime)) { + if (allBlocksMigrated && (migrationTime > lastTaskFinishTime.get())) { logInfo("No running tasks, all blocks migrated, stopping.") exitExecutor(0, ExecutorLossMessage.decommissionFinished, notifyDriver = true) } else { @@ -370,12 +374,6 @@ private[spark] class CoarseGrainedExecutorBackend( } } else { logInfo(s"Blocked from shutdown by ${executor.numRunningTasks} running tasks") - // If there is a running task it could store blocks, so make sure we wait for a - // migration loop to complete after the last task is done. - // Note: this is only advanced if there is a running task, if there - // is no running task but the blocks are not done migrating this does not - // move forward. - lastTaskRunningTime = System.nanoTime() } Thread.sleep(sleep_time) } From 97472c91ed5660c5af862e8da99d44a1c24f2815 Mon Sep 17 00:00:00 2001 From: Johan Lasperas Date: Mon, 4 Dec 2023 08:58:03 -0800 Subject: [PATCH 137/521] [SPARK-46092][SQL][3.5] Don't push down Parquet row group filters that overflow This is a cherry-pick from https://github.com/apache/spark/pull/44006 to spark 3.5 ### What changes were proposed in this pull request? This change adds a check for overflows when creating Parquet row group filters on an INT32 (byte/short/int) parquet type to avoid incorrectly skipping row groups if the predicate value doesn't fit in an INT. This can happen if the read schema is specified as LONG, e.g via `.schema("col LONG")` While the Parquet readers don't support reading INT32 into a LONG, the overflow can lead to row groups being incorrectly skipped, bypassing the reader altogether and producing incorrect results instead of failing. ### Why are the changes needed? Reading a parquet file containing INT32 values with a read schema specified as LONG can produce incorrect results today: ``` Seq(0).toDF("a").write.parquet(path) spark.read.schema("a LONG").parquet(path).where(s"a < ${Long.MaxValue}").collect() ``` will return an empty result. The correct result is either: - Failing the query if the parquet reader doesn't support upcasting integers to longs (all parquet readers in Spark today) - Return result `[0]` if the parquet reader supports that upcast (no readers in Spark as of now, but I'm looking into adding this capability). ### Does this PR introduce _any_ user-facing change? The following: ``` Seq(0).toDF("a").write.parquet(path) spark.read.schema("a LONG").parquet(path).where(s"a < ${Long.MaxValue}").collect() ``` produces an (incorrect) empty result before this change. After this change, the read will fail, raising an error about the unsupported conversion from INT to LONG in the parquet reader. ### How was this patch tested? - Added tests to `ParquetFilterSuite` to ensure that no row group filter is created when the predicate value overflows or when the value type isn't compatible with the parquet type - Added test to `ParquetQuerySuite` covering the correctness issue described above. ### Was this patch authored or co-authored using generative AI tooling? No Closes #44154 from johanl-db/SPARK-46092-row-group-skipping-overflow-3.5. Authored-by: Johan Lasperas Signed-off-by: Dongjoon Hyun --- .../datasources/parquet/ParquetFilters.scala | 10 ++- .../parquet/ParquetFilterSuite.scala | 71 +++++++++++++++++++ .../parquet/ParquetQuerySuite.scala | 20 ++++++ 3 files changed, 99 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 5899b6621ad8e..0983841dc8c2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.parquet -import java.lang.{Boolean => JBoolean, Double => JDouble, Float => JFloat, Long => JLong} +import java.lang.{Boolean => JBoolean, Byte => JByte, Double => JDouble, Float => JFloat, Long => JLong, Short => JShort} import java.math.{BigDecimal => JBigDecimal} import java.nio.charset.StandardCharsets.UTF_8 import java.sql.{Date, Timestamp} @@ -612,7 +612,13 @@ class ParquetFilters( value == null || (nameToParquetField(name).fieldType match { case ParquetBooleanType => value.isInstanceOf[JBoolean] case ParquetIntegerType if value.isInstanceOf[Period] => true - case ParquetByteType | ParquetShortType | ParquetIntegerType => value.isInstanceOf[Number] + case ParquetByteType | ParquetShortType | ParquetIntegerType => value match { + // Byte/Short/Int are all stored as INT32 in Parquet so filters are built using type Int. + // We don't create a filter if the value would overflow. + case _: JByte | _: JShort | _: Integer => true + case v: JLong => v.longValue() >= Int.MinValue && v.longValue() <= Int.MaxValue + case _ => false + } case ParquetLongType => value.isInstanceOf[JLong] || value.isInstanceOf[Duration] case ParquetFloatType => value.isInstanceOf[JFloat] case ParquetDoubleType => value.isInstanceOf[JDouble] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 269a3efb7360c..8e88049f51e10 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.datasources.parquet import java.io.File +import java.lang.{Double => JDouble, Float => JFloat, Long => JLong} import java.math.{BigDecimal => JBigDecimal} import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} @@ -901,6 +902,76 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } } + test("don't push down filters that would result in overflows") { + val schema = StructType(Seq( + StructField("cbyte", ByteType), + StructField("cshort", ShortType), + StructField("cint", IntegerType) + )) + + val parquetSchema = new SparkToParquetSchemaConverter(conf).convert(schema) + val parquetFilters = createParquetFilters(parquetSchema) + + for { + column <- Seq("cbyte", "cshort", "cint") + value <- Seq(JLong.MAX_VALUE, JLong.MIN_VALUE).map(JLong.valueOf) + } { + val filters = Seq( + sources.LessThan(column, value), + sources.LessThanOrEqual(column, value), + sources.GreaterThan(column, value), + sources.GreaterThanOrEqual(column, value), + sources.EqualTo(column, value), + sources.EqualNullSafe(column, value), + sources.Not(sources.EqualTo(column, value)), + sources.In(column, Array(value)) + ) + for (filter <- filters) { + assert(parquetFilters.createFilter(filter).isEmpty, + s"Row group filter $filter shouldn't be pushed down.") + } + } + } + + test("don't push down filters when value type doesn't match column type") { + val schema = StructType(Seq( + StructField("cbyte", ByteType), + StructField("cshort", ShortType), + StructField("cint", IntegerType), + StructField("clong", LongType), + StructField("cfloat", FloatType), + StructField("cdouble", DoubleType), + StructField("cboolean", BooleanType), + StructField("cstring", StringType), + StructField("cdate", DateType), + StructField("ctimestamp", TimestampType), + StructField("cbinary", BinaryType), + StructField("cdecimal", DecimalType(10, 0)) + )) + + val parquetSchema = new SparkToParquetSchemaConverter(conf).convert(schema) + val parquetFilters = createParquetFilters(parquetSchema) + + val filters = Seq( + sources.LessThan("cbyte", String.valueOf("1")), + sources.LessThan("cshort", JBigDecimal.valueOf(1)), + sources.LessThan("cint", JFloat.valueOf(JFloat.NaN)), + sources.LessThan("clong", String.valueOf("1")), + sources.LessThan("cfloat", JDouble.valueOf(1.0D)), + sources.LessThan("cdouble", JFloat.valueOf(1.0F)), + sources.LessThan("cboolean", String.valueOf("true")), + sources.LessThan("cstring", Integer.valueOf(1)), + sources.LessThan("cdate", Timestamp.valueOf("2018-01-01 00:00:00")), + sources.LessThan("ctimestamp", Date.valueOf("2018-01-01")), + sources.LessThan("cbinary", Integer.valueOf(1)), + sources.LessThan("cdecimal", Integer.valueOf(1234)) + ) + for (filter <- filters) { + assert(parquetFilters.createFilter(filter).isEmpty, + s"Row group filter $filter shouldn't be pushed down.") + } + } + test("SPARK-6554: don't push down predicates which reference partition columns") { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 2e7b26126d24f..ea5444a1791fd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -1095,6 +1095,26 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS } } + test("row group skipping doesn't overflow when reading into larger type") { + withTempPath { path => + Seq(0).toDF("a").write.parquet(path.toString) + // The vectorized and non-vectorized readers will produce different exceptions, we don't need + // to test both as this covers row group skipping. + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { + // Reading integer 'a' as a long isn't supported. Check that an exception is raised instead + // of incorrectly skipping the single row group and producing incorrect results. + val exception = intercept[SparkException] { + spark.read + .schema("a LONG") + .parquet(path.toString) + .where(s"a < ${Long.MaxValue}") + .collect() + } + assert(exception.getCause.getCause.isInstanceOf[SchemaColumnConvertNotSupportedException]) + } + } + } + test("SPARK-36825, SPARK-36852: create table with ANSI intervals") { withTable("tbl") { sql("create table tbl (c1 interval day, c2 interval year to month) using parquet") From 1321b4e64deaa1e58bf297c25b72319083056568 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 4 Dec 2023 14:41:27 -0800 Subject: [PATCH 138/521] [SPARK-46239][CORE] Hide `Jetty` info **What changes were proposed in this pull request?** The PR sets parameters to hide the version of jetty in spark. **Why are the changes needed?** It can avoid obtaining remote WWW service information through HTTP. **Does this PR introduce any user-facing change?** No **How was this patch tested?** Manual review **Was this patch authored or co-authored using generative AI tooling?** No Closes #44158 from chenyu-opensource/branch-SPARK-46239. Lead-authored-by: Dongjoon Hyun Co-authored-by: chenyu <119398199+chenyu-opensource@users.noreply.github.com> Signed-off-by: Dongjoon Hyun (cherry picked from commit ff4f59341215b7f3a87e6cd8798d49e25562fcd6) Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 9582bdbf52641..21753361e627a 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -312,6 +312,12 @@ private[spark] object JettyUtils extends Logging { logDebug(s"Using requestHeaderSize: $requestHeaderSize") httpConfig.setRequestHeaderSize(requestHeaderSize) + // Hide information. + logDebug("Using setSendServerVersion: false") + httpConfig.setSendServerVersion(false) + logDebug("Using setSendXPoweredBy: false") + httpConfig.setSendXPoweredBy(false) + // If SSL is configured, create the secure connector first. val securePort = sslOptions.createJettySslContextFactory().map { factory => val securePort = sslOptions.port.getOrElse(if (port > 0) Utils.userPort(port, 400) else 0) From b5cbe1fcdb464fc064ffb5fbef3edfa408d6638f Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 6 Dec 2023 10:46:31 -0800 Subject: [PATCH 139/521] [SPARK-46286][DOCS] Document `spark.io.compression.zstd.bufferPool.enabled` This PR adds spark.io.compression.zstd.bufferPool.enabled to documentation - Missing docs - https://github.com/apache/spark/pull/31502#issuecomment-774792276 potential regression no doc build no Closes #44207 from yaooqinn/SPARK-46286. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun (cherry picked from commit 6b6980de451e655ef4b9f63d502b73c09a513d4c) Signed-off-by: Dongjoon Hyun --- docs/configuration.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index 248f9333c9a3b..f79406c5b6d89 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1752,6 +1752,14 @@ Apart from these, the following properties are also available, and may be useful + + + + + + From a697725d99a0177a2b1fbb0607e859ac10af1c4e Mon Sep 17 00:00:00 2001 From: Nick Young Date: Wed, 6 Dec 2023 15:20:19 -0800 Subject: [PATCH 140/521] [SPARK-46274][SQL] Fix Range operator computeStats() to check long validity before converting ### What changes were proposed in this pull request? Range operator's `computeStats()` function unsafely casts from `BigInt` to `Long` and causes issues downstream with statistics estimation. Adds bounds checking to avoid crashing. ### Why are the changes needed? Downstream statistics estimation will crash and fail loudly; to avoid this and help maintain clean code we should fix this. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? UT ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44191 from n-young-db/range-compute-stats. Authored-by: Nick Young Signed-off-by: Wenchen Fan (cherry picked from commit 9fd575ae46f8a4dbd7da18887a44c693d8788332) Signed-off-by: Wenchen Fan --- .../plans/logical/basicLogicalOperators.scala | 12 +++++++----- .../BasicStatsEstimationSuite.scala | 16 ++++++++++++++++ 2 files changed, 23 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index b4d7716a566e4..58c03ee72d6dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -1063,10 +1063,12 @@ case class Range( if (numElements == 0) { Statistics(sizeInBytes = 0, rowCount = Some(0)) } else { - val (minVal, maxVal) = if (step > 0) { - (start, start + (numElements - 1) * step) + val (minVal, maxVal) = if (!numElements.isValidLong) { + (None, None) + } else if (step > 0) { + (Some(start), Some(start + (numElements.toLong - 1) * step)) } else { - (start + (numElements - 1) * step, start) + (Some(start + (numElements.toLong - 1) * step), Some(start)) } val histogram = if (conf.histogramEnabled) { @@ -1077,8 +1079,8 @@ case class Range( val colStat = ColumnStat( distinctCount = Some(numElements), - max = Some(maxVal), - min = Some(minVal), + max = maxVal, + min = minVal, nullCount = Some(0), avgLen = Some(LongType.defaultSize), maxLen = Some(LongType.defaultSize), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala index 33e521eb65a57..d1276615c5faa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala @@ -176,6 +176,22 @@ class BasicStatsEstimationSuite extends PlanTest with StatsEstimationTestBase { expectedStatsCboOff = rangeStats, extraConfig) } +test("range with invalid long value") { + val numElements = BigInt(Long.MaxValue) - BigInt(Long.MinValue) + val range = Range(Long.MinValue, Long.MaxValue, 1, None) + val rangeAttrs = AttributeMap(range.output.map(attr => + (attr, ColumnStat( + distinctCount = Some(numElements), + nullCount = Some(0), + maxLen = Some(LongType.defaultSize), + avgLen = Some(LongType.defaultSize))))) + val rangeStats = Statistics( + sizeInBytes = numElements * 8, + rowCount = Some(numElements), + attributeStats = rangeAttrs) + checkStats(range, rangeStats, rangeStats) +} + test("windows") { val windows = plan.window(Seq(min(attribute).as("sum_attr")), Seq(attribute), Nil) val windowsStats = Statistics(sizeInBytes = plan.size.get * (4 + 4 + 8) / (4 + 8)) From dbb61981b804dbc03cf140c7c76653348e2ac740 Mon Sep 17 00:00:00 2001 From: Bruce Robbins Date: Wed, 6 Dec 2023 15:24:48 -0800 Subject: [PATCH 141/521] [SPARK-45580][SQL][3.5] Handle case where a nested subquery becomes an existence join ### What changes were proposed in this pull request? This is a back-port of #44193. In `RewritePredicateSubquery`, prune existence flags from the final join when `rewriteExistentialExpr` returns an existence join. This change prunes the flags (attributes with the name "exists") by adding a `Project` node. For example: ``` Join LeftSemi, ((a#13 = c1#15) OR exists#19) :- Join ExistenceJoin(exists#19), (a#13 = col1#17) : :- LocalRelation [a#13] : +- LocalRelation [col1#17] +- LocalRelation [c1#15] ``` becomes ``` Project [a#13] +- Join LeftSemi, ((a#13 = c1#15) OR exists#19) :- Join ExistenceJoin(exists#19), (a#13 = col1#17) : :- LocalRelation [a#13] : +- LocalRelation [col1#17] +- LocalRelation [c1#15] ``` This change always adds the `Project` node, whether `rewriteExistentialExpr` returns an existence join or not. In the case when `rewriteExistentialExpr` does not return an existence join, `RemoveNoopOperators` will remove the unneeded `Project` node. ### Why are the changes needed? This query returns an extraneous boolean column when run in spark-sql: ``` create or replace temp view t1(a) as values (1), (2), (3), (7); create or replace temp view t2(c1) as values (1), (2), (3); create or replace temp view t3(col1) as values (3), (9); select * from t1 where exists ( select c1 from t2 where a = c1 or a in (select col1 from t3) ); 1 false 2 false 3 true ``` (Note: the above query will not have the extraneous boolean column when run from the Dataset API. That is because the Dataset API truncates the rows based on the schema of the analyzed plan. The bug occurs during optimization). This query fails when run in either spark-sql or using the Dataset API: ``` select ( select * from t1 where exists ( select c1 from t2 where a = c1 or a in (select col1 from t3) ) limit 1 ) from range(1); java.lang.AssertionError: assertion failed: Expects 1 field, but got 2; something went wrong in analysis ``` ### Does this PR introduce _any_ user-facing change? No, except for the removal of the extraneous boolean flag and the fix to the error condition. ### How was this patch tested? New unit test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44215 from bersprockets/schema_change_br35. Authored-by: Bruce Robbins Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/optimizer/subquery.scala | 9 ++-- .../org/apache/spark/sql/SubquerySuite.scala | 46 +++++++++++++++++++ 2 files changed, 52 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index 91cd838ad617a..ee20053157816 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -118,16 +118,19 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { withSubquery.foldLeft(newFilter) { case (p, Exists(sub, _, _, conditions, subHint)) => val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) - buildJoin(outerPlan, sub, LeftSemi, joinCond, subHint) + val join = buildJoin(outerPlan, sub, LeftSemi, joinCond, subHint) + Project(p.output, join) case (p, Not(Exists(sub, _, _, conditions, subHint))) => val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) - buildJoin(outerPlan, sub, LeftAnti, joinCond, subHint) + val join = buildJoin(outerPlan, sub, LeftAnti, joinCond, subHint) + Project(p.output, join) case (p, InSubquery(values, ListQuery(sub, _, _, _, conditions, subHint))) => // Deduplicate conflicting attributes if any. val newSub = dedupSubqueryOnSelfJoin(p, sub, Some(values)) val inConditions = values.zip(newSub.output).map(EqualTo.tupled) val (joinCond, outerPlan) = rewriteExistentialExpr(inConditions ++ conditions, p) - Join(outerPlan, newSub, LeftSemi, joinCond, JoinHint(None, subHint)) + val join = Join(outerPlan, newSub, LeftSemi, joinCond, JoinHint(None, subHint)) + Project(p.output, join) case (p, Not(InSubquery(values, ListQuery(sub, _, _, _, conditions, subHint)))) => // This is a NULL-aware (left) anti join (NAAJ) e.g. col NOT IN expr // Construct the condition. A NULL in one of the conditions is regarded as a positive diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index a7a0f6156cb1d..fbc256b33968a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -2736,4 +2736,50 @@ class SubquerySuite extends QueryTest Row(1, "a", 3) :: Row(2, "a", 3) :: Row(3, "a", 3) :: Nil) } } + + test("SPARK-45580: Handle case where a nested subquery becomes an existence join") { + withTempView("t1", "t2", "t3") { + Seq((1), (2), (3), (7)).toDF("a").persist().createOrReplaceTempView("t1") + Seq((1), (2), (3)).toDF("c1").persist().createOrReplaceTempView("t2") + Seq((3), (9)).toDF("col1").persist().createOrReplaceTempView("t3") + + val query1 = + """ + |SELECT * + |FROM t1 + |WHERE EXISTS ( + | SELECT c1 + | FROM t2 + | WHERE a = c1 + | OR a IN (SELECT col1 FROM t3) + |)""".stripMargin + val df1 = sql(query1) + checkAnswer(df1, Row(1) :: Row(2) :: Row(3) :: Nil) + + val query2 = + """ + |SELECT * + |FROM t1 + |WHERE a IN ( + | SELECT c1 + | FROM t2 + | where a IN (SELECT col1 FROM t3) + |)""".stripMargin + val df2 = sql(query2) + checkAnswer(df2, Row(3)) + + val query3 = + """ + |SELECT * + |FROM t1 + |WHERE NOT EXISTS ( + | SELECT c1 + | FROM t2 + | WHERE a = c1 + | OR a IN (SELECT col1 FROM t3) + |)""".stripMargin + val df3 = sql(query3) + checkAnswer(df3, Row(7)) + } + } } From ab14430523473528bafa41d8f10bc33efbb74493 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Fri, 8 Dec 2023 16:40:27 +0900 Subject: [PATCH 142/521] [SPARK-46275] Protobuf: Return null in permissive mode when deserialization fails ### What changes were proposed in this pull request? This updates the the behavior of `from_protobuf()` built function when underlying record fails to deserialize. * **Current behvior**: * By default, this would throw an error and the query fails. [This part is not changed in the PR] * When `mode` is set to 'PERMISSIVE' it returns a non-null struct with each of the inner fields set to null e.g. `{ "field_a": null, "field_b": null }` etc. * This is not very convenient to the users. They don't know if this was due to malformed record or if the input itself has null. It is very hard to check for each field for null in SQL query (imagine a sql query with a struct that has 10 fields). * **New behavior** * When `mode` is set to 'PERMISSIVE' it simply returns `null`. ### Why are the changes needed? This makes it easier for users to detect and handle malformed records. ### Does this PR introduce _any_ user-facing change? Yes, but this does not change the contract. In fact, it clarifies it. ### How was this patch tested? - Unit tests are updated. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44214 from rangadi/protobuf-null. Authored-by: Raghu Angadi Signed-off-by: Hyukjin Kwon (cherry picked from commit 309c796876f310f8604292d84acc12e711ba7031) Signed-off-by: Hyukjin Kwon --- .../sql/protobuf/ProtobufDataToCatalyst.scala | 31 +++---------------- .../ProtobufCatalystDataConversionSuite.scala | 13 +------- 2 files changed, 6 insertions(+), 38 deletions(-) diff --git a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDataToCatalyst.scala b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDataToCatalyst.scala index 5c4a5ff068968..d2417674837be 100644 --- a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDataToCatalyst.scala +++ b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDataToCatalyst.scala @@ -22,12 +22,12 @@ import scala.util.control.NonFatal import com.google.protobuf.DynamicMessage import com.google.protobuf.TypeRegistry -import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, SpecificInternalRow, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, UnaryExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode} import org.apache.spark.sql.catalyst.util.{FailFastMode, ParseMode, PermissiveMode} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.protobuf.utils.{ProtobufOptions, ProtobufUtils, SchemaConverters} -import org.apache.spark.sql.types.{AbstractDataType, BinaryType, DataType, StructType} +import org.apache.spark.sql.types.{AbstractDataType, BinaryType, DataType} private[sql] case class ProtobufDataToCatalyst( child: Expression, @@ -39,16 +39,8 @@ private[sql] case class ProtobufDataToCatalyst( override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType) - override lazy val dataType: DataType = { - val dt = SchemaConverters.toSqlType(messageDescriptor, protobufOptions).dataType - parseMode match { - // With PermissiveMode, the output Catalyst row might contain columns of null values for - // corrupt records, even if some of the columns are not nullable in the user-provided schema. - // Therefore we force the schema to be all nullable here. - case PermissiveMode => dt.asNullable - case _ => dt - } - } + override lazy val dataType: DataType = + SchemaConverters.toSqlType(messageDescriptor, protobufOptions).dataType override def nullable: Boolean = true @@ -87,22 +79,9 @@ private[sql] case class ProtobufDataToCatalyst( mode } - @transient private lazy val nullResultRow: Any = dataType match { - case st: StructType => - val resultRow = new SpecificInternalRow(st.map(_.dataType)) - for (i <- 0 until st.length) { - resultRow.setNullAt(i) - } - resultRow - - case _ => - null - } - private def handleException(e: Throwable): Any = { parseMode match { - case PermissiveMode => - nullResultRow + case PermissiveMode => null case FailFastMode => throw QueryExecutionErrors.malformedProtobufMessageDetectedInMessageParsingError(e) case _ => diff --git a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufCatalystDataConversionSuite.scala b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufCatalystDataConversionSuite.scala index b7f17fece5fa6..62d0efd7459b2 100644 --- a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufCatalystDataConversionSuite.scala +++ b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufCatalystDataConversionSuite.scala @@ -79,20 +79,9 @@ class ProtobufCatalystDataConversionSuite .eval() } - val expected = { - val expectedSchema = ProtobufUtils.buildDescriptor(descBytes, badSchema) - SchemaConverters.toSqlType(expectedSchema).dataType match { - case st: StructType => - Row.fromSeq((0 until st.length).map { _ => - null - }) - case _ => null - } - } - checkEvaluation( ProtobufDataToCatalyst(binary, badSchema, Some(descBytes), Map("mode" -> "PERMISSIVE")), - expected) + expected = null) } protected def prepareExpectedResult(expected: Any): Any = expected match { From 28a8b181e96d4ce71e2f9888910214d14a859b7d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 9 Dec 2023 15:20:55 -0800 Subject: [PATCH 143/521] [SPARK-46339][SS] Directory with batch number name should not be treated as metadata log ### What changes were proposed in this pull request? This patch updates the document of `CheckpointFileManager.list` method to reflect the fact it is used to return both files and directories to reduce confusion. For the usage like `HDFSMetadataLog` where it assumes returned file status by `list` are all files, we add a filter there to avoid confusing error. ### Why are the changes needed? `HDFSMetadataLog` takes a metadata path as parameter. When it goes to retrieves all batches metadata, it calls `CheckpointFileManager.list` to get all files under the metadata path. However, currently all implementations of `CheckpointFileManager.list` returns all files/directories under the given path. So if there is a dictionary with name of batch number (a long value), the directory will be returned too and cause trouble when `HDFSMetadataLog` goes to read it. Actually, `CheckpointFileManager.list` method clearly defines that it lists the "files" in a path. That's being said, current implementations don't follow the doc. We tried to make `list` method implementations only return files but some usage (state metadata) of `list` method already break the assumption and they use dictionaries returned by `list` method. So we simply update `list` method document to explicitly define it returns both files/dictionaries. We add a filter in `HDFSMetadataLog` on the file statuses returned by `list` method to avoid this issue. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added test ### Was this patch authored or co-authored using generative AI tooling? No Closes #44272 from viirya/fix_metadatalog. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun (cherry picked from commit 75805f07f5caeb01104a7352b02790d03a043ded) Signed-off-by: Dongjoon Hyun --- .../execution/streaming/CheckpointFileManager.scala | 4 ++-- .../sql/execution/streaming/HDFSMetadataLog.scala | 2 ++ .../execution/streaming/HDFSMetadataLogSuite.scala | 12 ++++++++++++ 3 files changed, 16 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala index ad3212871fc94..677e2fccb6b48 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala @@ -65,10 +65,10 @@ trait CheckpointFileManager { /** Open a file for reading, or throw exception if it does not exist. */ def open(path: Path): FSDataInputStream - /** List the files in a path that match a filter. */ + /** List the files/directories in a path that match a filter. */ def list(path: Path, filter: PathFilter): Array[FileStatus] - /** List all the files in a path. */ + /** List all the files/directories in a path. */ def list(path: Path): Array[FileStatus] = { list(path, (_: Path) => true) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index 2b0172bb9555c..9a811db679d01 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -325,6 +325,8 @@ class HDFSMetadataLog[T <: AnyRef : ClassTag](sparkSession: SparkSession, path: /** List the available batches on file system. */ protected def listBatches: Array[Long] = { val batchIds = fileManager.list(metadataPath, batchFilesFilter) + // Batches must be files + .filter(f => f.isFile) .map(f => pathToBatchId(f.getPath)) ++ // Iterate over keySet is not thread safe. We call `toArray` to make a copy in the lock to // elimiate the race condition. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala index 980d532dd4779..08f245135f589 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala @@ -33,6 +33,18 @@ class HDFSMetadataLogSuite extends SharedSparkSession { private implicit def toOption[A](a: A): Option[A] = Option(a) + test("SPARK-46339: Directory with number name should not be treated as metadata log") { + withTempDir { temp => + val dir = new File(temp, "dir") + val metadataLog = new HDFSMetadataLog[String](spark, dir.getAbsolutePath) + assert(metadataLog.metadataPath.toString.endsWith("/dir")) + + // Create a directory with batch id 0 + new File(dir, "0").mkdir() + assert(metadataLog.getLatest() === None) + } + } + test("HDFSMetadataLog: basic") { withTempDir { temp => val dir = new File(temp, "dir") // use non-existent directory to test whether log make the dir From cbaefe9cc6a22c940728b6717aeaa51c7d550ddc Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Sun, 10 Dec 2023 14:03:37 -0800 Subject: [PATCH 144/521] [SPARK-45969][DOCS] Document configuration change of executor failure tracker It's a follow-up of SPARK-41210 (use a new JIRA ticket because it was released in 3.5.0), this PR updates docs/migration guide about configuration change of executor failure tracker Docs update is missing in previous changes, also is requested https://github.com/apache/spark/commit/40872e9a094f8459b0b6f626937ced48a8d98efb#r132516892 by tgravescs Yes, docs changed Review No Closes #43863 from pan3793/SPARK-45969. Authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun (cherry picked from commit 7a43de193aa5a0856e098088728dccea37f169c5) Signed-off-by: Dongjoon Hyun --- .../spark/internal/config/package.scala | 4 ++-- docs/configuration.md | 21 +++++++++++++++++++ docs/core-migration-guide.md | 6 ++++++ docs/running-on-yarn.md | 17 --------------- 4 files changed, 29 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 600cbf151e17b..c5e23cae1f847 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -924,7 +924,7 @@ package object config { private[spark] val MAX_EXECUTOR_FAILURES = ConfigBuilder("spark.executor.maxNumFailures") - .doc("Spark exits if the number of failed executors exceeds this threshold. " + + .doc("The maximum number of executor failures before failing the application. " + "This configuration only takes effect on YARN, or Kubernetes when " + "`spark.kubernetes.allocation.pods.allocator` is set to 'direct'.") .version("3.5.0") @@ -933,7 +933,7 @@ package object config { private[spark] val EXECUTOR_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS = ConfigBuilder("spark.executor.failuresValidityInterval") - .doc("Interval after which Executor failures will be considered independent and not " + + .doc("Interval after which executor failures will be considered independent and not " + "accumulate towards the attempt count. This configuration only takes effect on YARN, " + "or Kubernetes when `spark.kubernetes.allocation.pods.allocator` is set to 'direct'.") .version("3.5.0") diff --git a/docs/configuration.md b/docs/configuration.md index f79406c5b6d89..645c3e8208abc 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -514,6 +514,27 @@ of the most common options to set are: + + + + + + + + + + + +
SQL metricsMeaningOperators
number of output rows the number of output rows of the operator Aggregate operators, Join operators, Sample, Range, Scan operators, Filter, etc.
data size the size of broadcast/shuffled/collected data of the operator BroadcastExchange, ShuffleExchange, Subquery
2.3.0
spark.io.compression.zstd.bufferPool.enabledtrue + If true, enable buffer pool of ZSTD JNI library. + 3.2.0
spark.kryo.classesToRegister (none)3.2.0
spark.executor.maxNumFailuresnumExecutors * 2, with minimum of 3 + The maximum number of executor failures before failing the application. + This configuration only takes effect on YARN, or Kubernetes when + `spark.kubernetes.allocation.pods.allocator` is set to 'direct'. + 3.5.0
spark.executor.failuresValidityInterval(none) + Interval after which executor failures will be considered independent and + not accumulate towards the attempt count. + This configuration only takes effect on YARN, or Kubernetes when + `spark.kubernetes.allocation.pods.allocator` is set to 'direct'. + 3.5.0
Apart from these, the following properties are also available, and may be useful in some situations: diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index 3f97a484e1a68..36465cc3f4e86 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -22,6 +22,12 @@ license: | * Table of contents {:toc} +## Upgrading from Core 3.4 to 3.5 + +- Since Spark 3.5, `spark.yarn.executor.failuresValidityInterval` is deprecated. Use `spark.executor.failuresValidityInterval` instead. + +- Since Spark 3.5, `spark.yarn.max.executor.failures` is deprecated. Use `spark.executor.maxNumFailures` instead. + ## Upgrading from Core 3.3 to 3.4 - Since Spark 3.4, Spark driver will own `PersistentVolumnClaim`s and try to reuse if they are not assigned to live executors. To restore the behavior before Spark 3.4, you can set `spark.kubernetes.driver.ownPersistentVolumeClaim` to `false` and `spark.kubernetes.driver.reusePersistentVolumeClaim` to `false`. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index d577b70a68039..9b4e59a119eeb 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -291,14 +291,6 @@ To use a custom metrics.properties for the application master and executors, upd
1.4.0
spark.yarn.max.executor.failuresnumExecutors * 2, with minimum of 3 - The maximum number of executor failures before failing the application. - 1.0.0
spark.yarn.historyServer.address (none) 3.3.0
spark.yarn.executor.failuresValidityInterval(none) - Defines the validity interval for executor failure tracking. - Executor failures which are older than the validity interval will be ignored. - 2.0.0
spark.yarn.submit.waitAppCompletion true
- + @@ -875,6 +875,7 @@ The following extra configuration options are available when the shuffle service initialization. This prevents application failures caused by running containers on NodeManagers where the Spark Shuffle Service is not running. + @@ -883,6 +884,7 @@ The following extra configuration options are available when the shuffle service The namespace to use when emitting shuffle service metrics into Hadoop metrics2 system of the NodeManager. + @@ -894,6 +896,7 @@ The following extra configuration options are available when the shuffle service may expect the logger name to look like a class name, it's generally recommended to provide a value which would be a valid Java package or class name and not include spaces. + From 1b7ee9e56b30d99ae948ceb8ca42f025e3d24c96 Mon Sep 17 00:00:00 2001 From: Xinyi Yu Date: Fri, 5 Jan 2024 22:57:28 +0800 Subject: [PATCH 164/521] [SPARK-46602][SQL] Propagate `allowExisting` in view creation when the view/table does not exists ### What changes were proposed in this pull request? This PR fixes the undesired behavior that concurrent `CREATE VIEW IF NOT EXISTS` queries could throw `TABLE_OR_VIEW_ALREADY_EXISTS` exceptions. It's because the current implementation did not propagate the 'IF NOT EXISTS' when the detecting view/table does not exists. ### Why are the changes needed? Fix the above issue. ### Does this PR introduce _any_ user-facing change? Yes in the sense that if fixes an issue in concurrent case. ### How was this patch tested? Without the fix the following test failed while with this PR if passed. But following the [comment](https://github.com/apache/spark/pull/44603#discussion_r1442515458), I removed the test from this PR. ```scala test("CREATE VIEW IF NOT EXISTS never throws TABLE_OR_VIEW_ALREADY_EXISTS") { // Concurrently create a view with the same name, so that some of the queries may all // get that the view does not exist and try to create it. But with IF NOT EXISTS, the // queries should not fail. import ExecutionContext.Implicits.global val concurrency = 10 val tableName = "table_name" val viewName = "view_name" withTable(tableName) { sql(s"CREATE TABLE $tableName (id int) USING parquet") withView("view_name") { val futures = (0 to concurrency).map { _ => Future { Try { sql(s"CREATE VIEW IF NOT EXISTS $viewName AS SELECT * FROM $tableName") } } } futures.map { future => val res = ThreadUtils.awaitResult(future, 5.seconds) assert( res.isSuccess, s"Failed to create view: ${if (res.isFailure) res.failed.get.getMessage}" ) } } } } ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44603 from anchovYu/create-view-if-not-exist-fix. Authored-by: Xinyi Yu Signed-off-by: Wenchen Fan (cherry picked from commit 9b3c70f6094c97ed61018d9fca8a50320574ab49) Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/execution/command/views.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 3718794ea5909..b6159f92f9cef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -167,7 +167,7 @@ case class CreateViewCommand( } } else { // Create the view if it doesn't exist. - catalog.createTable(prepareTable(sparkSession, analyzedPlan), ignoreIfExists = false) + catalog.createTable(prepareTable(sparkSession, analyzedPlan), ignoreIfExists = allowExisting) } Seq.empty[Row] } From 9f095b71ca2fab7211f84fbf3a16d2f9ffb3d957 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 5 Jan 2024 11:23:23 -0800 Subject: [PATCH 165/521] [SPARK-46609][SQL] Avoid exponential explosion in PartitioningPreservingUnaryExecNode ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/37525 . When expanding the output partitioning/ordering with aliases, we have a threshold to avoid exponential explosion. However, we missed to apply this threshold in one place. This PR fixes it. ### Why are the changes needed? to avoid OOM ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? new test ### Was this patch authored or co-authored using generative AI tooling? no Closes #44614 from cloud-fan/oom. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun (cherry picked from commit f8115da1a2bb33e6344dd69cc38ca7a68c3654b1) Signed-off-by: Dongjoon Hyun --- .../AliasAwareOutputExpression.scala | 4 +-- ...rojectedOrderingAndPartitioningSuite.scala | 30 +++++++++++++++++-- 2 files changed, 30 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala index e1dcab80af307..428fe65501fb4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputExpression.scala @@ -30,7 +30,7 @@ trait PartitioningPreservingUnaryExecNode extends UnaryExecNode with AliasAwareOutputExpression { final override def outputPartitioning: Partitioning = { val partitionings: Seq[Partitioning] = if (hasAlias) { - flattenPartitioning(child.outputPartitioning).flatMap { + flattenPartitioning(child.outputPartitioning).iterator.flatMap { case e: Expression => // We need unique partitionings but if the input partitioning is // `HashPartitioning(Seq(id + id))` and we have `id -> a` and `id -> b` aliases then after @@ -44,7 +44,7 @@ trait PartitioningPreservingUnaryExecNode extends UnaryExecNode .take(aliasCandidateLimit) .asInstanceOf[Stream[Partitioning]] case o => Seq(o) - } + }.take(aliasCandidateLimit).toSeq } else { // Filter valid partitiongs (only reference output attributes of the current plan node) val outputSet = AttributeSet(outputExpressions.map(_.toAttribute)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala index f5839e9975602..ec13d48d45f84 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ProjectedOrderingAndPartitioningSuite.scala @@ -17,11 +17,14 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, PartitioningCollection, UnknownPartitioning} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection, UnknownPartitioning} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.StringType class ProjectedOrderingAndPartitioningSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { @@ -101,6 +104,22 @@ class ProjectedOrderingAndPartitioningSuite } } + test("SPARK-46609: Avoid exponential explosion in PartitioningPreservingUnaryExecNode") { + withSQLConf(SQLConf.EXPRESSION_PROJECTION_CANDIDATE_LIMIT.key -> "2") { + val output = Seq(AttributeReference("a", StringType)(), AttributeReference("b", StringType)()) + val plan = ProjectExec( + Seq( + Alias(output(0), "a1")(), + Alias(output(0), "a2")(), + Alias(output(1), "b1")(), + Alias(output(1), "b2")() + ), + DummyLeafPlanExec(output) + ) + assert(plan.outputPartitioning.asInstanceOf[PartitioningCollection].partitionings.length == 2) + } + } + test("SPARK-42049: Improve AliasAwareOutputExpression - multi-references to complex " + "expressions") { val df2 = spark.range(2).repartition($"id" + $"id").selectExpr("id + id as a", "id + id as b") @@ -192,3 +211,10 @@ class ProjectedOrderingAndPartitioningSuite assert(outputOrdering.head.sameOrderExpressions.size == 0) } } + +private case class DummyLeafPlanExec(output: Seq[Attribute]) extends LeafExecNode { + override protected def doExecute(): RDD[InternalRow] = null + override def outputPartitioning: Partitioning = { + PartitioningCollection(output.map(attr => HashPartitioning(Seq(attr), 4))) + } +} From fe22ec74db7895c6ea1f39236162ae39027111f4 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 6 Jan 2024 12:38:35 -0800 Subject: [PATCH 166/521] [SPARK-46598][SQL] OrcColumnarBatchReader should respect the memory mode when creating column vectors for the missing column This PR fixes a long-standing bug that `OrcColumnarBatchReader` does not respect the memory mode when creating column vectors for missing columbs. This PR fixes it. To not violate the memory mode requirement No new test no Closes #44598 from cloud-fan/orc. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun (cherry picked from commit 0c1c5e93e376b97a6d2dae99e973b9385155727a) Signed-off-by: Dongjoon Hyun --- .../orc/OrcColumnarBatchReader.java | 17 ++++++++++++----- .../datasources/orc/OrcFileFormat.scala | 9 ++++++++- .../v2/orc/OrcPartitionReaderFactory.scala | 6 ++++-- .../execution/datasources/v2/orc/OrcScan.scala | 8 +++++++- .../orc/OrcColumnarBatchReaderSuite.scala | 18 +++++++++++++++--- 5 files changed, 46 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java index b6184baa2e0ed..5bfe22450f36b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java @@ -31,12 +31,11 @@ import org.apache.orc.TypeDescription; import org.apache.orc.mapred.OrcInputFormat; +import org.apache.spark.memory.MemoryMode; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns; import org.apache.spark.sql.execution.datasources.orc.OrcShimUtils.VectorizedRowBatchWrap; -import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils; -import org.apache.spark.sql.execution.vectorized.ConstantColumnVector; -import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector; +import org.apache.spark.sql.execution.vectorized.*; import org.apache.spark.sql.types.*; import org.apache.spark.sql.vectorized.ColumnarBatch; @@ -73,11 +72,14 @@ public class OrcColumnarBatchReader extends RecordReader { @VisibleForTesting public ColumnarBatch columnarBatch; + private final MemoryMode memoryMode; + // The wrapped ORC column vectors. private org.apache.spark.sql.vectorized.ColumnVector[] orcVectorWrappers; - public OrcColumnarBatchReader(int capacity) { + public OrcColumnarBatchReader(int capacity, MemoryMode memoryMode) { this.capacity = capacity; + this.memoryMode = memoryMode; } @@ -177,7 +179,12 @@ public void initBatch( int colId = requestedDataColIds[i]; // Initialize the missing columns once. if (colId == -1) { - OnHeapColumnVector missingCol = new OnHeapColumnVector(capacity, dt); + final WritableColumnVector missingCol; + if (memoryMode == MemoryMode.OFF_HEAP) { + missingCol = new OffHeapColumnVector(capacity, dt); + } else { + missingCol = new OnHeapColumnVector(capacity, dt); + } // Check if the missing column has an associated default value in the schema metadata. // If so, fill the corresponding column vector with the value. Object defaultValue = ResolveDefaultColumns.existenceDefaultValues(requiredSchema)[i]; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index b7e6f11f67d69..53d2b08431f85 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -31,6 +31,7 @@ import org.apache.orc.mapred.OrcStruct import org.apache.orc.mapreduce._ import org.apache.spark.TaskContext +import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ @@ -152,6 +153,12 @@ class OrcFileFormat assert(supportBatch(sparkSession, resultSchema)) } + val memoryMode = if (sqlConf.offHeapColumnVectorEnabled) { + MemoryMode.OFF_HEAP + } else { + MemoryMode.ON_HEAP + } + OrcConf.IS_SCHEMA_EVOLUTION_CASE_SENSITIVE.setBoolean(hadoopConf, sqlConf.caseSensitiveAnalysis) val broadcastedConf = @@ -196,7 +203,7 @@ class OrcFileFormat val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) if (enableVectorizedReader) { - val batchReader = new OrcColumnarBatchReader(capacity) + val batchReader = new OrcColumnarBatchReader(capacity, memoryMode) // SPARK-23399 Register a task completion listener first to call `close()` in all cases. // There is a possibility that `initialize` and `initBatch` hit some errors (like OOM) // after opening a file. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala index 2b7bdae6b31b4..b23071e50cbed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -26,6 +26,7 @@ import org.apache.orc.mapred.OrcStruct import org.apache.orc.mapreduce.OrcInputFormat import org.apache.spark.broadcast.Broadcast +import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.connector.expressions.aggregate.Aggregation import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader} @@ -57,7 +58,8 @@ case class OrcPartitionReaderFactory( partitionSchema: StructType, filters: Array[Filter], aggregation: Option[Aggregation], - options: OrcOptions) extends FilePartitionReaderFactory { + options: OrcOptions, + memoryMode: MemoryMode) extends FilePartitionReaderFactory { private val resultSchema = StructType(readDataSchema.fields ++ partitionSchema.fields) private val isCaseSensitive = sqlConf.caseSensitiveAnalysis private val capacity = sqlConf.orcVectorizedReaderBatchSize @@ -146,7 +148,7 @@ case class OrcPartitionReaderFactory( val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) - val batchReader = new OrcColumnarBatchReader(capacity) + val batchReader = new OrcColumnarBatchReader(capacity, memoryMode) batchReader.initialize(fileSplit, taskAttemptContext) val requestedPartitionColIds = Array.fill(readDataSchema.length)(-1) ++ Range(0, partitionSchema.length) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala index 072ab26774e52..ca37d22eeb1e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala @@ -21,6 +21,7 @@ import scala.collection.JavaConverters.mapAsScalaMapConverter import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.connector.expressions.aggregate.Aggregation @@ -64,11 +65,16 @@ case class OrcScan( override def createReaderFactory(): PartitionReaderFactory = { val broadcastedConf = sparkSession.sparkContext.broadcast( new SerializableConfiguration(hadoopConf)) + val memoryMode = if (sparkSession.sessionState.conf.offHeapColumnVectorEnabled) { + MemoryMode.OFF_HEAP + } else { + MemoryMode.ON_HEAP + } // The partition values are already truncated in `FileScan.partitions`. // We should use `readPartitionSchema` as the partition schema here. OrcPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, dataSchema, readDataSchema, readPartitionSchema, pushedFilters, pushedAggregate, - new OrcOptions(options.asScala.toMap, sparkSession.sessionState.conf)) + new OrcOptions(options.asScala.toMap, sparkSession.sessionState.conf), memoryMode) } override def equals(obj: Any): Boolean = obj match { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala index a9389c1c21b40..06ea12f83ce75 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala @@ -26,11 +26,12 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.orc.TypeDescription import org.apache.spark.TestUtils +import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.vectorized.ConstantColumnVector +import org.apache.spark.sql.execution.vectorized.{ConstantColumnVector, OffHeapColumnVector} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -53,7 +54,7 @@ class OrcColumnarBatchReaderSuite extends QueryTest with SharedSparkSession { requestedDataColIds: Array[Int], requestedPartitionColIds: Array[Int], resultFields: Array[StructField]): OrcColumnarBatchReader = { - val reader = new OrcColumnarBatchReader(4096) + val reader = new OrcColumnarBatchReader(4096, MemoryMode.ON_HEAP) reader.initBatch( orcFileSchema, resultFields, @@ -117,7 +118,7 @@ class OrcColumnarBatchReaderSuite extends QueryTest with SharedSparkSession { val fileSplit = new FileSplit(new Path(file.getCanonicalPath), 0L, file.length, Array.empty) val taskConf = sqlContext.sessionState.newHadoopConf() val orcFileSchema = TypeDescription.fromString(schema.simpleString) - val vectorizedReader = new OrcColumnarBatchReader(4096) + val vectorizedReader = new OrcColumnarBatchReader(4096, MemoryMode.ON_HEAP) val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) @@ -148,4 +149,15 @@ class OrcColumnarBatchReaderSuite extends QueryTest with SharedSparkSession { } } } + + test("SPARK-46598: off-heap mode") { + val reader = new OrcColumnarBatchReader(4096, MemoryMode.OFF_HEAP) + reader.initBatch( + TypeDescription.fromString("struct"), + StructType.fromDDL("col1 int, col2 int, col3 int").fields, + Array(0, 1, -1), + Array(-1, -1, -1), + InternalRow.empty) + assert(reader.columnarBatch.column(2).isInstanceOf[OffHeapColumnVector]) + } } From 75b567daa705016f2ddb74dc8404ae66bc33bbcd Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 8 Jan 2024 16:24:06 -0800 Subject: [PATCH 167/521] [SPARK-46628][INFRA] Use SPDX short identifier in `license` name ### What changes were proposed in this pull request? This PR aims to use SPDX short identifier as `license`'s `name` field. - https://spdx.org/licenses/Apache-2.0.html ### Why are the changes needed? SPDX short identifier is recommended as `name` field by `Apache Maven`. - https://maven.apache.org/pom.html#Licenses ASF pom file has been using it. This PR aims to match with ASF pom file. - https://github.com/apache/maven-apache-parent/pull/118 - https://github.com/apache/maven-apache-parent/blob/7888bdb8ee653ecc03b5fee136540a607193c240/pom.xml#L46 ``` Apache-2.0 ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44631 from dongjoon-hyun/SPARK-46628. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit d008f81a9d8d4b5e8e434469755405f6ae747e75) Signed-off-by: Dongjoon Hyun --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 14e0ab3e0f620..9e945f8d959a4 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ https://spark.apache.org/ - Apache 2.0 License + Apache-2.0 http://www.apache.org/licenses/LICENSE-2.0.html repo From 2b0c3e12636b93758879ee3ff23626ea1f218264 Mon Sep 17 00:00:00 2001 From: Rui Wang Date: Tue, 9 Jan 2024 09:07:34 +0800 Subject: [PATCH 168/521] [SPARK-46610][SQL] Create table should throw exception when no value for a key in options ### What changes were proposed in this pull request? Before SPARK-43529, there was a check from `visitPropertyKeyValues` that throws for null values for option keys. After SPARK-43529, a new function is used to support expressions in options but the new function lose the check. This PR adds the check back. ### Why are the changes needed? Throw exception when a option value is null. ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? UT ### Was this patch authored or co-authored using generative AI tooling? NO Closes #44615 from amaliujia/fix_create_table_options. Lead-authored-by: Rui Wang Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit e7536f2484afce412256bf711452acde8df5a287) Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/AstBuilder.scala | 4 +++- .../spark/sql/catalyst/parser/DDLParserSuite.scala | 12 ++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index b80ea8fddcfe2..90fbdd94dc386 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3274,7 +3274,9 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { ctx: ExpressionPropertyListContext): OptionList = { val options = ctx.expressionProperty.asScala.map { property => val key: String = visitPropertyKey(property.key) - val value: Expression = Option(property.value).map(expression).orNull + val value: Expression = Option(property.value).map(expression).getOrElse { + operationNotAllowed(s"A value must be specified for the key: $key.", ctx) + } key -> value }.toSeq OptionList(options) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 31fd232181a4f..6f36a8c9719cb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -2356,6 +2356,18 @@ class DDLParserSuite extends AnalysisTest { stop = 42)) } + test("SPARK-46610: throw exception when no value for a key in create table options") { + val createTableSql = "create table test_table using my_data_source options (password)" + checkError( + exception = parseException(createTableSql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "A value must be specified for the key: password."), + context = ExpectedContext( + fragment = createTableSql, + start = 0, + stop = 62)) + } + test("UNCACHE TABLE") { comparePlans( parsePlan("UNCACHE TABLE a.b.c"), From a753239ab1afaeddf4c991d42b93e4845f12e576 Mon Sep 17 00:00:00 2001 From: Rui Wang Date: Mon, 8 Jan 2024 22:22:06 -0400 Subject: [PATCH 169/521] [SPARK-46600][SQL] Move shared code between SqlConf and SqlApiConf to SqlApiConfHelper ### What changes were proposed in this pull request? This code proposes to introduce a new object named `SqlApiConfHelper` to contain shared code between `SqlApiConf` and `SqlConf`. ### Why are the changes needed? As of now, SqlConf will access some of the variables of SqlApiConf while SqlApiConf also try to initialize SqlConf upon initialization. This PR is to avoid potential circular dependency between SqlConf and SqlApiConf. The shared variables or access to the shared variables are moved to the new `SqlApiConfHelper`. So either SqlApiConf and SqlConf wants to initialize the other side, they will only initialize the same third object. ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? Existing UT ### Was this patch authored or co-authored using generative AI tooling? NO Closes #44602 from amaliujia/refactor_sql_api. Authored-by: Rui Wang Signed-off-by: Herman van Hovell (cherry picked from commit 03fc5e26b866491b52f89f4d24beade7d1669a37) Signed-off-by: Herman van Hovell --- .../spark/sql/internal/SqlApiConf.scala | 26 +++------- .../spark/sql/internal/SqlApiConfHelper.scala | 48 +++++++++++++++++++ .../apache/spark/sql/internal/SQLConf.scala | 12 ++--- 3 files changed, 61 insertions(+), 25 deletions(-) create mode 100644 sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala index d746e9037ec48..5ec72b83837ee 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.internal import java.util.TimeZone -import java.util.concurrent.atomic.AtomicReference import scala.util.Try @@ -48,25 +47,14 @@ private[sql] trait SqlApiConf { private[sql] object SqlApiConf { // Shared keys. - val ANSI_ENABLED_KEY: String = "spark.sql.ansi.enabled" - val LEGACY_TIME_PARSER_POLICY_KEY: String = "spark.sql.legacy.timeParserPolicy" - val CASE_SENSITIVE_KEY: String = "spark.sql.caseSensitive" - val SESSION_LOCAL_TIMEZONE_KEY: String = "spark.sql.session.timeZone" - val LOCAL_RELATION_CACHE_THRESHOLD_KEY: String = "spark.sql.session.localRelationCacheThreshold" + val ANSI_ENABLED_KEY: String = SqlApiConfHelper.ANSI_ENABLED_KEY + val LEGACY_TIME_PARSER_POLICY_KEY: String = SqlApiConfHelper.LEGACY_TIME_PARSER_POLICY_KEY + val CASE_SENSITIVE_KEY: String = SqlApiConfHelper.CASE_SENSITIVE_KEY + val SESSION_LOCAL_TIMEZONE_KEY: String = SqlApiConfHelper.SESSION_LOCAL_TIMEZONE_KEY + val LOCAL_RELATION_CACHE_THRESHOLD_KEY: String = + SqlApiConfHelper.LOCAL_RELATION_CACHE_THRESHOLD_KEY - /** - * Defines a getter that returns the [[SqlApiConf]] within scope. - */ - private val confGetter = new AtomicReference[() => SqlApiConf](() => DefaultSqlApiConf) - - /** - * Sets the active config getter. - */ - private[sql] def setConfGetter(getter: () => SqlApiConf): Unit = { - confGetter.set(getter) - } - - def get: SqlApiConf = confGetter.get()() + def get: SqlApiConf = SqlApiConfHelper.getConfGetter.get()() // Force load SQLConf. This will trigger the installation of a confGetter that points to SQLConf. Try(SparkClassUtils.classForName("org.apache.spark.sql.internal.SQLConf$")) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala new file mode 100644 index 0000000000000..79b6cb9231c51 --- /dev/null +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.internal + +import java.util.concurrent.atomic.AtomicReference + +/** + * SqlApiConfHelper is created to avoid a deadlock during a concurrent access to SQLConf and + * SqlApiConf, which is because SQLConf and SqlApiConf tries to load each other upon + * initializations. SqlApiConfHelper is private to sql package and is not supposed to be + * accessed by end users. Variables and methods within SqlApiConfHelper are defined to + * be used by SQLConf and SqlApiConf only. + */ +private[sql] object SqlApiConfHelper { + // Shared keys. + val ANSI_ENABLED_KEY: String = "spark.sql.ansi.enabled" + val LEGACY_TIME_PARSER_POLICY_KEY: String = "spark.sql.legacy.timeParserPolicy" + val CASE_SENSITIVE_KEY: String = "spark.sql.caseSensitive" + val SESSION_LOCAL_TIMEZONE_KEY: String = "spark.sql.session.timeZone" + val LOCAL_RELATION_CACHE_THRESHOLD_KEY: String = "spark.sql.session.localRelationCacheThreshold" + + val confGetter: AtomicReference[() => SqlApiConf] = { + new AtomicReference[() => SqlApiConf](() => DefaultSqlApiConf) + } + + def getConfGetter: AtomicReference[() => SqlApiConf] = confGetter + + /** + * Sets the active config getter. + */ + def setConfGetter(getter: () => SqlApiConf): Unit = { + confGetter.set(getter) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 70bd21ac1709d..de4a89667aff6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -181,7 +181,7 @@ object SQLConf { // Make sure SqlApiConf is always in sync with SQLConf. SqlApiConf will always try to // load SqlConf to make sure both classes are in sync from the get go. - SqlApiConf.setConfGetter(() => SQLConf.get) + SqlApiConfHelper.setConfGetter(() => SQLConf.get) /** * Returns the active config object within the current scope. If there is an active SparkSession, @@ -894,7 +894,7 @@ object SQLConf { .booleanConf .createWithDefault(false) - val CASE_SENSITIVE = buildConf(SqlApiConf.CASE_SENSITIVE_KEY) + val CASE_SENSITIVE = buildConf(SqlApiConfHelper.CASE_SENSITIVE_KEY) .internal() .doc("Whether the query analyzer should be case sensitive or not. " + "Default to case insensitive. It is highly discouraged to turn on case sensitive mode.") @@ -2676,7 +2676,7 @@ object SQLConf { Try { DateTimeUtils.getZoneId(zone) }.isSuccess } - val SESSION_LOCAL_TIMEZONE = buildConf(SqlApiConf.SESSION_LOCAL_TIMEZONE_KEY) + val SESSION_LOCAL_TIMEZONE = buildConf(SqlApiConfHelper.SESSION_LOCAL_TIMEZONE_KEY) .doc("The ID of session local timezone in the format of either region-based zone IDs or " + "zone offsets. Region IDs must have the form 'area/city', such as 'America/Los_Angeles'. " + "Zone offsets must be in the format '(+|-)HH', '(+|-)HH:mm' or '(+|-)HH:mm:ss', e.g '-08', " + @@ -3180,7 +3180,7 @@ object SQLConf { .checkValues(StoreAssignmentPolicy.values.map(_.toString)) .createWithDefault(StoreAssignmentPolicy.ANSI.toString) - val ANSI_ENABLED = buildConf(SqlApiConf.ANSI_ENABLED_KEY) + val ANSI_ENABLED = buildConf(SqlApiConfHelper.ANSI_ENABLED_KEY) .doc("When true, Spark SQL uses an ANSI compliant dialect instead of being Hive compliant. " + "For example, Spark will throw an exception at runtime instead of returning null results " + "when the inputs to a SQL operator/function are invalid." + @@ -3779,7 +3779,7 @@ object SQLConf { .checkValues(LegacyBehaviorPolicy.values.map(_.toString)) .createWithDefault(LegacyBehaviorPolicy.EXCEPTION.toString) - val LEGACY_TIME_PARSER_POLICY = buildConf(SqlApiConf.LEGACY_TIME_PARSER_POLICY_KEY) + val LEGACY_TIME_PARSER_POLICY = buildConf(SqlApiConfHelper.LEGACY_TIME_PARSER_POLICY_KEY) .internal() .doc("When LEGACY, java.text.SimpleDateFormat is used for formatting and parsing " + "dates/timestamps in a locale-sensitive manner, which is the approach before Spark 3.0. " + @@ -4344,7 +4344,7 @@ object SQLConf { .createWithDefault(false) val LOCAL_RELATION_CACHE_THRESHOLD = - buildConf(SqlApiConf.LOCAL_RELATION_CACHE_THRESHOLD_KEY) + buildConf(SqlApiConfHelper.LOCAL_RELATION_CACHE_THRESHOLD_KEY) .doc("The threshold for the size in bytes of local relations to be cached at " + "the driver side after serialization.") .version("3.5.0") From d3e3084808453769ba0cd4278ee8650e40c185ea Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 10 Jan 2024 09:32:30 +0900 Subject: [PATCH 170/521] [SPARK-46637][DOCS] Enhancing the Visual Appeal of Spark doc website ### What changes were proposed in this pull request? Enhance the Visual Appeal of Spark doc website after https://github.com/apache/spark/pull/40269: #### 1. There is a weird indent on the top right side of the first paragraph of the Spark 3.5.0 doc overview page Before this PR image After this PR: image #### 2. All the titles are too big and therefore less readable. In the website https://spark.apache.org/downloads.html, titles are h2 while in doc site https://spark.apache.org/docs/latest/ titles are h1. So we should make the font size of titles smaller. Before this PR: image After this PR: image #### 3. The banner image can't be displayed correct. Even when it shows up, it will be hover by the text. To make it simple, let's not show the banner image as we did in https://spark.apache.org/docs/3.4.2/ image image ### Why are the changes needed? Improve the Visual Appeal of Spark doc website ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manually build doc and verify on local setup. ### Was this patch authored or co-authored using generative AI tooling? No Closes #44642 from gengliangwang/enhance_doc. Authored-by: Gengliang Wang Signed-off-by: Hyukjin Kwon --- docs/_layouts/global.html | 26 +++++++++------------ docs/css/custom.css | 35 ++++++++++++----------------- docs/img/spark-hero-thin-light.jpg | Bin 278664 -> 0 bytes 3 files changed, 25 insertions(+), 36 deletions(-) delete mode 100644 docs/img/spark-hero-thin-light.jpg diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 8c4435fdf31d9..5116472eaa769 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -138,25 +138,21 @@ {% if page.url == "/" %}
-
-

Apache Spark - A Unified engine for large-scale data analytics

-
-
- Apache Spark is a unified analytics engine for large-scale data processing. - It provides high-level APIs in Java, Scala, Python and R, - and an optimized engine that supports general execution graphs. - It also supports a rich set of higher-level tools including - Spark SQL for SQL and structured data processing, - pandas API on Spark for pandas workloads, - MLlib for machine learning, - GraphX for graph processing, - and Structured Streaming - for incremental computation and stream processing. -
+
+ Apache Spark is a unified analytics engine for large-scale data processing. + It provides high-level APIs in Java, Scala, Python and R, + and an optimized engine that supports general execution graphs. + It also supports a rich set of higher-level tools including + Spark SQL for SQL and structured data processing, + pandas API on Spark for pandas workloads, + MLlib for machine learning, + GraphX for graph processing, + and Structured Streaming + for incremental computation and stream processing.
diff --git a/docs/css/custom.css b/docs/css/custom.css index 1239c0ed440ef..8158938866c48 100644 --- a/docs/css/custom.css +++ b/docs/css/custom.css @@ -95,18 +95,7 @@ section { border-color: transparent; } -.hero-banner .bg { - background: url(/img/spark-hero-thin-light.jpg) no-repeat; - transform: translate(36%, 0%); - height: 475px; - top: 0; - position: absolute; - right: 0; - width: 100%; - opacity: 50%; -} - -.hero-banner h1 { +.hero-banner .container .row h1 { color: #0B9ACE; font-style: normal; font-weight: normal; @@ -115,13 +104,6 @@ section { letter-spacing: -0.045em; } -.hero-banner h2 { - font-style: normal; - font-weight: bold; - font-size: 32px; - line-height: 42px; -} - .what-is-spark { font-style: normal; font-weight: normal; @@ -822,18 +804,29 @@ ul { margin-bottom: 10px; } -.global h2, .global .h2 { +.global h1, .global .h1 { font-size: 30px; } -.global h3 { +#content h1.title { + font-size: 40px; +} + +.global h2 { font-size: 24px !important; } +.global h3 { + font-size: 20px !important; +} + .global h4 { font-size: 18px !important; } +.global h5 { + font-size: 16px !important; +} .global h1:first-letter, .global h2:first-letter, .global h3:first-letter, .global h4:first-letter, .global h5:first-letter, .global h6:first-letter, .global .h1:first-letter, .global .h2:first-letter, .global .h3:first-letter, .global .h4:first-letter, .global .h5:first-letter, .global .h6:first-letter { text-transform: uppercase; diff --git a/docs/img/spark-hero-thin-light.jpg b/docs/img/spark-hero-thin-light.jpg deleted file mode 100644 index 4d9ed926b361fc5b9182db20bc04b3ac766ac38c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 278664 zcmdSAcTiKo+cq3T1-qySQbeVLROtxV00Ytj1W70=Achcnh;+Mx6s0Ovfq)P~1X3g+ zprRlkAPE6UNFqYO(4v&k;p6wbZRY#yoA3SW*)y|y&z#-+>~&^m_b%6N{oMKmIBaJF zv;l~S001wxAHdcKAl3RV>^1=4-~iA70082ET_Q&SqT7ok^SE? zNJQ;__SpddJcR*v{?9&;?e;%qTl0TL|Ib$RjmZDe_-4od>b*1Ujp+Z9MJE3Gsm zDY|Rde?B9Uu>BseQ)1VV(|Q+oA9cAYb|(Co{{5G4#T75reUb$A{Z%@5E8@YPy~m}b zPsk{%sH&-J7#JEEo0ytizG7`-dlhKs>gEpixb6w@^AETkcn1~~8HJ2S-HnNT_~`MI zr_Yj8GPAOCa`Rrj&Mz%1M^{w7tE#SVz!Qk14~^u{tzW*jwSVjAq|y5s1A{+@m?OVO z$9UrtfB2LC=H?d`g-gpTtN+2Z{XhQ)tnK=LAp3ve+WrY5J9h3A-6{4TTp~MA{{b$s zbJuCT-A69Eh}{f7dPe`g__0eb-`0KFqj(PVSMpXw-`?X&1|!P<{sZm5ko|uH_Tc{) zvi}Y2|KMT)E^jZ)|GToat>phs!}jPc_V%(&-(mvxi;8S7CQ%816<~8y4e(~iy}f|{ zX3ff|@Z|>DQem_Eq*&h9L|w8}EJ08JAF&c2vR4ptQ;;{x0?^r7-=p8>dKwr?-A~asRisphU;2)iK+>$jjPk1qlqr0zbH>j^Aw@Kt1jF zI}74fB^L&07|8WYoei$^Y2svu_16|bYL}ub9qek#_9PjU9BbByx85jQPm26^^tWT# zIQytF?&-ol*E4^7j?nD0cITBr^)%*hJZb7z+5!NnB~kOiyG}Vx7E6wt)doIk{R&D| z$uqdI5;%=kj_I$mbf2kn1-om!dq=nbNwX_V*aGl2bw_1Zw&$mTR5o-$L#WikI=drH z>aLG@)4EZ`_&@4)u316?TD>D;h-Pr*N~-whwxgmUmt*dYHDu~X-iwG)ym#uu^6v{Shf~)5Kf7}%0ej3MKxUY#gfl0 z4=*2HE@^X?GwO(%Ipftv4`zN_NA0*$qTuOS@FKR>TW%Am%pN?jbM7;!XW-)gNmDkglz?x5rulK2sm z(w!sndhc)OejHU){ws1|&QPfgm7o*j)?A-MdY&l!$1m+u!bhy$`8;E7Tkir?mY#uW z+6KosRDNDe!x_#LYb5dCjiLmIDGo*AGg@ji!JHn9~7ei%^X9BOQc&vSwgbSK z<2~6RPv>6U0w|pwel^0Tj05gBX}tX0{#Y7k*Q{FnBPFb&_N_E77UaeXxNWnlmFAlM_*G z-)XCTuH83BGiQBtdPc*K~ReeD8r>o zvJqvn?+^a|h!cH5)r}1_Cr^a!6mWByQ`A~qQmb1_o^>48-cT)~q-c>bRSks&+#c-m zUeXT<73hON%T4cJCqLP-LV;rtE}D7ns4z*nX<)wdPEOdC;O}kh_hWj z#YNaVe@~wN=U$PwFYl#W=B0=84W?_)F!Op5U#8u`w?6IyeSH}1{8qg=uzq0535nCO zw*VK_q{_{xidQ*0cn|TDEbvr9D24WYLE%Z#4uX?R7yNfd^5J3F>TbVAdXvg@AA7QD z=`DHCj*LH^Y8gOx6BH$l@=C=zLyLMo@@Jm5z&91TcmDvg1bqy)d+tE%0GlvH1;aIpwWD8hqtoj<{0JAnt9oJ-sH zJGU9tgkQNY=`MM}P2_^yy+&NGs;OgH;qZjI(HKv&qkDVT>hcMUYo~S!1v3`N&4a{z zHVdYD(i_|}5G!C~)@r#&J%%*lKeBawYrLkD$~+GO-7zn>1vvKl z^v%?yV_A|HOhx95hQk;dUm4MrHbxUwE+4)c(<~<=a)_zNN|3U3PDdRJob8!|usKjg ze}&K^y8cmYqa8D7g{z^Tx+L{=lg1A_qtq1`s`nJw(0aqDvPvPJHwf4$KK=g1ZUs$J z$aRHay5xNrloAW+B zjbHgy_V2iRy8E3F#d|kHYj3R?WduiD^|o@_Y1seJ>TITHuI!VmB4GtCab~8 z7xUI6ZZ*k!3vlodw9Q=WY|StkHis{743vSoS1ANOs@MYXzbc@Z9h!3;0xb4+_q(i+ zu8wlG7*Xebdcv2-I|)`(q0cIZ6Uw23mmf*?`{EpSe{s`(hBvJKz(1N_!{tT zXI#n--oT?fdBihh$8sFJ{m|vRIB4ZO9{HAi@^%LDP5kbDL(%ccqa~}Thhm*#Cf3eB zrKd8ba>3E5g+YeZZ!fdU_RaJf|1%U8G1%;dB}x^?swpK3-|*#@f<+95xiBI;BZw+( z$_HwD$K6f6@qGsq^UGrM4F}QnZ$J%b=@X}}m?~enqcO<7fZc)*r@rg+dXce=u5C`r zoj{l+_AccLgL(@jo-JV8`Hk<_a$lTq0{82rO97B6`XZ+!=4O71 z-TIj2We5SS7(C@A1bRA6_U_Nfo%;^Y&MjI#63J6deenfww_T-C!>~Gg zg=}qGL7pm|Ba&u`W_)XbW8Rs#&RFB|W_61=DiWePJIp#^k_mp@xWNp(eL>;l>yDjK zei^k9Lq5I%YC7XE<0T@Nz!8#tgsFU&^!dUH7U>Kmo)+pnpv9w#_Dz-RAIa%edxb^X5S^Tj;R8*~q)CWX9vOL9xU&#GQ8-_%e$+igbCEHg`1L z(b8pM)={s=F7ST4%KI#jzNB9es+CVZhs~LAgAr@T87!LiU`scLjptfpv~B9t$3&Mi zfxRa>c(?S~ zsIm&m5ZU~GaIfUaDtDcC z&G^fzK&#POT~XJA&-!W^6i(WqVUFcXQEA7{sD6`ldnzgoc(?P2NRMOKv2JrE6P zn+0}saP7H{Kb!*40@dgO`gROPM%L;`Ed2#Y{O;T=?Ptb8~dum5vGO!3`{+f5rr z+wXP-T|nW<6X{YRPKi^DDGXy{9XrFi9WoP8o6-S`Y9TjkQbPkcgVBzrW&>2eH%wW| z&=I{%#e4er2H@bW3Xx2Zu45AN$Vg%71I@~Pk1(n4s%U9jfc)sC?U{2aon+2nsKd$3 zi7&tZ+*I#G=0hXCdUk?ZtT9G@+&YKaVs*uY8a`Z1wW9={HFnwz`#n16Ucg zXLm2&+b`m^+sz0t;Efcn@ngWxU~)>4b!?^p3uHmns5c!r>txd4-FNF?bNw%VQzmU* z_|ai-;3S;;hA@Qk(XA`Le+xI1W=sZ(b%X0$`7tB2JCbl9Y>ig9;9Pg z3D_z#jQuB2O5;bLf*2E@&N4wjbXNakkb|nS{PCs^Of#-5)|(U;yhJ2k#I$7gC0dq5 z;Rh2s+8EAnw2vXq+VF;xo=s$inaC!B_E_wJIj|Qg1GC+U`t|cmXzg4(S_K`1&s@U7 zz0DDUNhyLWSw6W}K-YHvstj6+yw^Rr_v#5zrGSVJqJY8%wYSkY44QTA1o0GGd z0-PJ8X|U-Q3)~Q-PTkZpZpID`S;$hcvs2w^sMvQWUe(6#|8*lNF4M+T;S2D5_~TfH zjUdNJ$OtN!%tQH8_zY&B?*T&Ys^*Zn)_M&MqFhni!Sq4Y+|B$A{ScLKk~!hdSlnaT z9Z>nRvM3T2)*kZ3op~k5N#KvDYd^_X&NSF*9D#N$GWGc4_E^JtX~MKC=aMF-$=3l_ z9bI1l)_tPu9&VL_G)LOs%mQ%E5Z3$TkRO&RFTWAUO-uN2e27F(@Z;(j-)*3ovW z(9WOlp09Gd(UnkD^)0Ru@MO0+qU@WF@1df9XX`X7q(pk{#i>IBD0fXP7FnZi7}7N7 zIEN1@TZ*wd1ZuT0tc$MlEyy=$R=aFpi}bH{uqX=@w`jzDPq?J_QSnrlsRdgpgVMk`uVmhDUMgQSocI!YfYwliRaRH zU$D|op$Y|w*aW5_!#8AU3F5&vjnT@XXlimPnS)e%cFIKsBclmG;wuxT~Nha9PqY&>^2WCVwmw?JhmPc8L)*AXN1Dk3S_qyzXVFzbS zCnzvNOB9kDSQQNMVqrK!m1I0`>NmQ5)qQIbS+?4r(0xV$TG4AmBl z*a`pRyQ(6x8oPbq9v)}wuBg6$;J5Z6Q^y$EB1T15*#0aHU0H1me)nl^w4-+Qx`rD) zn{77tV5wq!^kJDY4eLGSkkm^?Iq;+M3_fW5o^k$5oP26hlQ7N=xGfNOS5 zSQYa1CrrbQnVRs~_HK*E*iUC=O|@`Sh%V#r*Iry`|DX`3G1)azrHV*kF2c&Op~xFy zDU+J^HW3+f9N|^|^P0~1gQH3L99EgxMqcZ{ z^({cL-i?kV-A$oFT-41x!&XpyW_&f;EXOz{sKS!u8;dZ!bO1|^2D_NQqfew{Q*-Rx z&;cq%=XYr)DVa86dbIv5da-@pG4GRR8ZVu24#xyt^m}0FySf|p=7#8tV>7+x&A0b2 zel^Gvg1o9)mV%B%RB3LT`8JBtgYI`{{HJ?X9Xc7OtL?auFPKYGq&1@a8kr9>kMZT) zrq=i?5Cuc`>?dO@l(HYTX=x=}r$$?!226?uj`OON*EM}s+Ydcmy3sNzt<$rsY{EhKTb1C z%~3?jdn%W+SI-(AoLb@eZvnni`dU!7Cs)sA z;~ce{7N|kE+-$8Djltfl?G7x2QBc(pT~8nTp}QMj05#&a)wpj`N(w~<@)WHIdv+Qv zO^u4SO80=Usu9s1NwD@Alr1N|J`1ac8)?64vjir!5=xrJHjtoxYb3zSF85{ei_f;N zAKsHfIVZflJABaQs*lZYmNTe&@T(~|cXcw4T)Y8oahr)jeg)cc3t?X??_5R&G{AkI ztg1~l7s!D7UVf!E56=aD80mEk1Ny!ECUS3-`MyeX3WVAmJDXc{7JFw6t;qHh2V3Hn#t-+Z!qA__jr^VaF& z;#PbSW&F2vNli_r4m^~FK|~4WVy_b9f-1+O6`M!b2X~@YV;^Q+H#Ij`!u7%D_$#ZB zwtr9Z5M2{1hya#-C}M;4V_I56MD>Hh_pymN-!J|H`k{dvG1wL@Gv*)8UX3pM8Gqb3 z79o|id^>5Y@4urbaOg!6+?Of%lvKY3cto>*{QOIj#NzVpj8BPT#3Jyd@CN!I4pX0- zVBRX!*Doj_IqCB{nM36bmahEe&?j}KLX~})Io&t5*WKn(Qe1M?or^YmkyS~?KHC36 z(>D$7a^05G{^>v(2b)|l_C#QP*Ubvgs%~Q1ThHTn@NAXuiL;t_26ehR=daB!Zn(Cd zKlixf?}DFW;c2jL_L;Ydb_QdMkF@DoSp$R^WpL95pL72G(|Iw;@POb>EF>JOesjVQ z6!|U_ll%^!2I2{_ z(l@2y<5tunk8st_v|K|J?pgP_*jMcuZeCR*u9P+XYC0;yW7%-Y&!3p(FlS@saIb6h ziPxf{%&TO$G;=aDPPy`2iHAxqSyLNkKRokBUqP4j{kl|1?f1D-i#Jk{ow^#jvU?iL z=@XN^xqSpi%^N%dyHeoT0h#hu;lYc$O^jny8wO%2zU9#oaTw}${9tUfasMUVW9s4u zUCuo$5p>*7efQ1Xr|kayC$;L$al~2n?94nqgi*UlSt}~}s*Fp+%t@PH1K>N?X@vyc+h zp^3)@b|DsDb}W~*$s|3&4)O9$L-z;YxM%q_Eqt@y)9mYErP&<%_ZOL`FP&>>5zTXN z>zUL(mgO}MBb5zl*1|}_BQExy#&;XT{N^?8s!`tc5U9-XIF;#wSct{gG^eQ=XsD1! zkoWNVw=tb}gzW$?+pRWu<2)jw;OVMS$({o|Xs%e?6Xl+3=mH*&SoX9vpr2tD?nuZY zruRZS%gMcPZ)@dpmw!mOM|*#cU451vKU9I&1heT@!%EH;YzHPTBM>k&%NhpX8Qz9n zqeS_>?nxT>{&Sf3ZR~^60^sJr`)nTAfhd5-0bwDvdJtv@AJYd5sH*XkfnBue4cLW= zND6XVW0Dd{DKLy(Skr5j)~=Uxe-)pFc&8)^f%1wcrhU&ERSyWC2dNWFn7K=FUok|gcQ3qT+}f(A0? z_}qYi1VmIz!3}C=1Fs%IoaF}45y5Oh{bXmauHpl*lJaNf?{ysHXEk@!Q`(#!7`f0!6l z4e8ut0rl2iWhJh`Xahr~UbsH?ES`|LVm?EmuCZQyqV|_PF^+zrHs)V!f8(!ydwOHA z&?*izfRu8^tx$vk$7@j2i_%q_i0=hTo=QXx9@L16-` z7ixiEboh+6j%bcfwmsGeaYRI#ux<1v1^#97(=2`|LRE)gc|88qk3>0E0d8gg-P$=x zZ`UuVtHptekwg5x7fB5ZV-ex|uNW+Qe7GGEy$LUzxLZRjdsI^Q`I0ZW^%E&hGqOLA zl?8^e)ezDYqr~SqJn491(P+@CWKB1m*6)_weMtG>;(EA)c7MXX!{v|7DX~W^GP)m# zz39A**_Wz9Vs{oq>WOl=;aL^D~DCzmPmX+iAa z$C=)52c04+&*HHtt^7X7WN)0|;A>>42U1|^dX>}!ZSd%eU~4f3x;h|hh6*ulZaWWr zx0iUl5~i2>B394YRyJ{|(?YkASl|L~SSm|cWE=VyIC#SlOln`bD=!2B4;;*884hkI zVHOb(lcoIi@RgU>#b^B=YFhNyD$;8jJ{e0XAlb3vr{|SIp5AENtiG;pZr6~l2qn zlUbO5rl<5Xom#|_SErNpkEr1H&oeJHllOAlzW+3jn~_$hyV&XZY{*2|?IR89#YVQQ zMmj`9fg2}!5X|xk?Lf3yL&XFL!QcG)n8IV(F%(ZADe*l16`TEBgcIdHtyT0uW`}DrbL?b*aoM)da8PuA0z;Q$O)fF5B2BBvLtu_aS zc=N4EYVREElH_HOrgiQ|8C@ryqug#~wJ7-{GmRUJumeU;4c2l}rMtT>ioin;{d_MU zGJ8GQP~9!H4$(PrUfVKJCf%#2yb_e5j}O#Ik_B5-fjrVr>XbV2QA>`@@aH}~dXEDR zcxxb9MXVQ*8Wej8bCm+vd^AJx_RxVose}BOF-cZTbScZW!Tr~BS>K$%o9o&hVgRXq zDPtIAyYnx7+KUB4pgjE7flzD^`GYU&7!^dbjaO9nBw`|PzZn@SM{);*1|g04eP1c6 z{sJ_b5R5$+GcYB5l3~g~%Gm}U-U3wG`{5C7b~En1ff^3sZs93oRkP|^y8TwkTG3U8 z-PpyY!j_rA@`Sy*k$65XX!PH$I;k{i0;SB>o*(CFGX(>h=L;HL%sHT1(S&$J?Yuug5CmHR(bXp7dS#O6^n&UE66*u<+a0QCvIsV zaZo$w`O1CeGaEkzSavqL^5js-L=4jN#{xhrRfpbxMfd8bjZva*nB(MlR#*zprPao6 zDRI$Li3l+@nyGiOcaPC5BOY*s%-^bx^rR}xAp$&ob_!NJl)Chj&^SlB4*Dq7#*j~oLZ zMFuJ)v`K6MKAtIA0(m5?mw1ODTZb+(C6NBb{KzirI8$#m?k0x>4G#K3I+B%n$~FC1 zOlFu>Q`nnA5$@kp9@}ea;n*B;ONIDwL{hh*s4^KV7Xw5j9Y3jrbHMi_{P0R>g z-}~EkN5mIO@hpq=^M-$|MNF?_Oih3~e&!CghWe8=(3MDCfcW%HK2mS9e?4zMvcxLH z)eewb9H`oN+OE%h&Y8f3XT%@0{m}bs7%`CvWOuD@0gfVI+!L{n}hq`(8;gg_-y-i~M_W14n{7`vxg9eNjU(3Cto1r<#27pUf6+V94q! zD}ol>bJwqMKd(X1qsJQTA4{zTA=tzj>Oclgx+3o zgt$zks((}ztDLu3-Cyh!={2*sQgpa^AQ2k9=uLq!Fp68W|9(z~=*Sj;r6BD6i<|Bk z;q@&L@c(ST3Jx>3b5Y{$kB*1kqW?w)oILaIoDFMwAr94^b*L+{%HbJeuybDZ`P4G> zak!^Yr4a2{`e`0tn9%V%U|-~=LN>$9nY`MjTx$oMvn|G#iSm}8C+n^l?yWXd-?**! zybvvD-FIUBNa0aOY;0~G<9%KHqL)|Y^q)4@y!o=4sYLzdlHW7F+erwmps6p-8iTVt zSJSJa*@xvW->igAbkP?mhN0S>ZEUc#qN9lN>y|2$q^G)#;FCFKQNOC+U6T9pJXM{( zoFgLvGXYU~MqCR|&UPxHEO2f+{~K-*+(x)k-JhlaGbNa+_hVg0gv-geXzcB5zq_?| z$7bwO6~*W5NqtesRLx^;j`~J37Rg>uL_-q&KJE%L8*iqtpHk#aLCv^9B*}PW$Y*YO zZ%%-VX-6dnn4}Rsc$?ER*#iy|`W~uWj%*(>l~pxOOsB_C?4R1vA~naFe)X@NuBg-_L)*L? zm>PBl5k-8r)-uxD>Y(`1$j;A=I#`)($*v-2yvTU{2CseS-5cLe%)YF_)&9m?J;TPAO zM#-|Z3fNsxLura9*XQ<3$Y3xlUH88Ggx+w%7kU^Y5#!@oL33+k z+wmQ2Ssc_}=;RpkvlgmEfc(KplelhU(~HfuOHHnK3X9&P*M!-2cd0AHyh^Pet#G_v zDAHjly8vFGE%j-qtwQrc@?H!l_^b+JeS}~~Ho=bhCnhf~@$}E-=o7z5`NousMEJv} zDL;!}p87VgVl~$@B$||XOCLmSBB^0g!;9{GpmB74rYlxn9}}y z=j}*vjdj4HzVI!>Su9`ez}XC zG0A@C@>%1aOf!Le{c4|oSrRGp_?gEQA7I)KXq(8X zVhlS+{jpM&iV2)KR(0ZU>iW!G;tc9`@i3p=q9*hRpt*C5g?IN{yP?N!& zKd|Qba9AZ8x1BWdti%cutCyA{f;e`Aah9tp zWBB^eJM7TtOjE4u^(TxIY=)vUP+B&5Y}W6GNMR)1bQGpVH3L-{!Z9prCUejy7Iv~I z%GS^>?GKJQh%je)BN|YssX&3JdZgv?zqpiU?Vx(-oi%*R{-N%%$zqk9Jx_FHmvZw# zx|#`zZU}QAiEzGWnoBNIcs-&Iv0M^_BNgvDr!^K4BT*Y{1I8>9wd~qr5LZ8plX%5o z<>r&{)q@ntxOeDEgPA0xuha7@FPBgc&L0@ziohiPa85QHC;nO?c^uiCckU0ly1^TN z8#R;}Q@GyM_}%4Uz0wsI-vYmX%Qc_3sTDzy9ei=jnuhb5!Kjo!1-R~nNluyhHP;OXhF2q9P1n`+Mw)dTm-J|yo0z}13Mh735w%O=q6-RFYcC^5)d^@HE z414H`7aYpr`zAJnAj$!Vtmm<{xf(t}M)|Q~#m*eNG$p&Zo>q7Ur{2w74(BX+&?mkX zjdpS-Jv@4qj)Xih+%-l=Ok8hzcdrMDbvTpp2=bFMaFpfAp)L%;pS$828ZZAG~f_N zkX!w$egEUPhixvghV4bQi}q~+P?u?X<3UmuU)eO>+vcCllY?SSzpBr77=t9Lx_vw# z_EOK9z$f>p6q$tx={eFB`Kbb_ovcb1I{eUaJn_)JxPhv`bznvL;smhlqHUHsc+CxE ztV#NomOJ{zquOc$Kcsg@h%G$k{n+CnX>VE&D<2qV>iVR)4;OeJoiz6JYf5!)Q#YX- zF*_EV*H94QeiK}dd<4O~`h8}$ps;4-QCR?94l-Y#9Klh2QJ{CEUbGRPV00<>S)wlo z4k?QtJXEq+HyclK7~0)9*{N>U)cg8Ro%>?-7J%}obxK;(!f3^3ZnDx5UD2 zQ@!_c2?G5~Vy7dlw^=~&lU@dLp)nlyYSn@6NZYZ}q8V(Sar}|O%&c6EQq)*Q&Ur%% zGNx_HTe;U5-VFUYV0}ctJyOxDtMK_gW%W>LIklXI2W6*1aF*Fv&F^| zC$eGrFFQvxl*yNoe6S{g=x_{)p$Wm&Qx7@U?J?DkIG1KjXCeP`_g1J{`j^Q|# z$*jEUEPC!^&8dIpZSq{|KL-k_G)cI`WI(+{lxNyJ*6zJ3TXzZTPJgX0x4QqO*KAp6 zZkk10OQH`c+MWlG;l7woGo0-tT)_$@M-G@pEV7`a7T<=lo%d>4=Fw^)Bs(zaII}Sj&D=OI44!n!96(>8H`qz=>=SD}5{VGXRJHac zXaCC*iWM^ykCWxizilFa1kz2E!%K{rnND#b!s+6v?iKc(C_2yXUu9w;cYi2Co(b z0vIBRik-(C1*GgeQTviaEIj(TvAMQT9~1h;bTID<*RE|{E9RIv3Slq zvv1z}nbC#oAE%A=q%C3tSMPJvB^!BxoUS+>2A@mfmXqC}^t@cQxo_tN0)@+Acm2~m zh;}ps=xro{4vnTUUgD#`4k=$W7G zJRa$&*&i-bs#G<^o;_>m4p*8mO`JN2Ha4{#T`ow`U9sNV&u6$`#hF7Pm{{~++!eGPHm0q}#1e%F=1EUt&2kBNI-J4}aTRbymycTiwR}aC zxpuKu0q#2q(x_RANQdThJW)11-1hXwiJN2(k>;cF8Q3&Tulw9*_3EfwEN5sC7G}&~ z(Q_gr6Ktf!oc0|;au7afPxDStEHr9#o71(v!T0QHNgtN~;B!dUMkI^>dAZsADgW8Udhg>en~DShSd#r~Qxv^|HN>y>7Q~UNVNp}Hn2|d)>tDN+5G$Vd z0H=C(yg4DN+<59&VOFjVCEv-0NeR8esi(~@HFrR|VSlKz-?S3A4P}`_+I=2zBt9j? zsptq$qwR6Bo10i<%kZ<*qVF+hlb)#uwC#!Pn$3|??F#(9`8^ahM(ndB&VheaD$8Oe zi|fgFcQTPiwi7n2E8Q(gPgHJ&$^PUYx!SE6v+pdxf-&80MXvYKW4$EIesJqiTzM$7 zdaXe@g5YHb(GGC*a2rt=ufkH|uo%j`XGDqK>p&SZliN%9>UIxRW=Z~(IY#SNEh{hU z`ky7^ygk+Mw*h33e*^s@7jdg{zfxgga%GQa{-v-lH}jhxpzmDL&e`{lf5I)+E@U7` zY8jkM8}tqEE>_k+EmeDWaJ;J67ej_iJUtgd&BTFg{Cy2|>1X{P_!m9xOfmUfY;;dt zZg;PUGcUIhlH%1{7M(^&Z23N&XKxYe|FW;PT6;!dVX)xzI>us{#E*zd_RwqdT+->8 zrg%P6RGQmcc>~k+ekYG;VQaWCPHTuJR>cJ6Rx_!Ybo;q(*>MDzp4 zZHALq!=tMYn(e=F)vlpTMRsiGrJHu#AKnp8&ilfienXgD^=|PswIFjdAu6TCb2)Su zzGTv4Oefz$1~~O+!d>BytSl#w@1C3GHSDBq^Y_;+-r$ka_9I3%^ zM!CG8#s5WQOrwLh(IA=phe0P<-@zJ&UV4<76WuYda+*YuczpiwVTsi6dqcO|_nHm# zrG(e_A^eNDDAg^%w;>B6&`z6@a- z6UrZQc7IHI9vr4-`msZ{L@v|mnt$3S&FUxe{j+&RCFf%8rb_!f)vb&gp{CW~ESRk) zl|kn@+{q-ScM}B_o5Gjr?X7}GOzszc>EODfj)Cv4!0$w2*QcuGm!k0>q&ZH;UIb54c1q~5gK$4&tQbu6F==B&?r z!+4xJT=fEQB?7Sa@zBk}Rn&CjH>Lc~4!+%(PPThaV_tVKX}Tta*xcp8otPi=Z<@iV zu;+LAya|;4B&zsec>iIOiI=@y59=ZI;|VmndlCUox=kF=>FijWL3-n3gG!tn$>wbM zgyi`h-;HKgUNziwj&L&l=wtSeTe0VQ$PBahA0;7nYQ=K!8t$<-vuDa%3&A2G7M9}{lEO?9Qfhvuo zmkuq_((pOJc!SgMwMCgctrgeCcBot<1g$n?*V81>aL(D338!68=1#jVg_dO_b+2ak zgbOR1ay-ksDuvx&yrqY`yKzsa?`q#ltr_#XXTzLv)Y!8*vdYzohEBkYv8<}FguTR}s zkTpHmn|Fd0j`I~=52;r)oq*kds&~&1#o2;fc)Sd1YIRG#L)+losU zMmkS_?D(`ROE%K_;ybq#FIt(S*DE_sYV0fx=i)N}N)pcWJ3i~i08Kx;%edgcaz8t} z61|M%wY|>Eta*{Z&GjKJxdzu|r<=WAP(OCigA0czmkJu@C-a|m+&|&edvf`6+DBtf z&=Y|ItouX+!lzz%Hz@IXteJX;#nFZqPtTtpY=1I2V=M(#NDe+4)99(}eZ!TIM3 zv~}9%H6;Ty{rTfLRw|Fn^$y#7P>rsP-y5P@7K%Jj_^NZgyTe%Lcchw@Y2DpN*=Sn7 z+S*!ZCES8({#LfrpxWADazJeF%r=AIS#2#kq>8!3_eVEKAG9WXl)82=xG(CxBL4ne zmayu+Gb?<4kN7SV^@m;&Ia!!p)3sb99SN)_DQTn?oYnHviQ_8=wGeMF$Src>7?gEA zXpO_4%57=Nv|D8rbV*HsD-u>U5z%jABT{NLUA2T3BU;M6Nj++mFEJW&5BX zo*PHl`rUrG4<7v6@chP*Hef&75v1&#I14qcm2!zpoYcpHedep&Bkc!tV|9#;jaqU- zt?Z6*HH28_vVb7B;^(E~RkI}j=Fbm8rhEx?w|ge*V8l|B@Xcm-J!`IZ-DuD2#>tJy z!ll}vrWAn$XuckQW(EC++q#Rvf`-3wb z^O>`eo!0oe9+RPv56&WD;{o4BO-@L*@xUHJT{k{fZf&~6fL*@2V%h6QxKyYhs5WZ= zerWAQoZd~N&{n5fwCF~+;Efmd$VIz|EWOgamY6(m^HA)5?!NS^3g;s1w&OHBh(il9 zuFhAu)VAt9|o4mdh8`d<$;>5Qb-x41>s4<1l!FcWEt^g<<_Qd|LeR zgn!C_UZ2au4!ViHG2qw9V}32&-4T}dOg&^k0<`HC+1pwZ-y}@xj%#mJXXS&od9{T# zCG#-yROprF?r%Fizq;>svcU+w}p+nW$ZRhPt_FM{y$th8U zeLh#Pd3%;H%}k0yB$yBXaMJzNrSatuKvh-#iBshp(Z4kcIztb$9A~BwX0wdd;C#0A zy6P=g@OYK94@)^AUPzuBPe88^mb#7ZQF=-tU5e}vX^WFfFVc@4I2zK+aq-UGrY??i z_(9ErhQ3`VX+@k`sLw_SgYsvcQ1_ZU7gXS@09yfRfFnYGTSVUAlseR|s_Ve*At&do7m>A#H4daCf zj=F_b9G6VOIV2{=F^zxtrN8FWGukcM<0DxLUkjq4tlp$n+(OSYo7EXO)WOPd<=j^P ze={%0>YP=G0;}C&`7|Q#FJ;7mk>2<(f0f(WLjLT{_dSTvF1wNcq$=$%8_v6$<=D3; zRAWRPqgn#I91l=Dyb8lFAPE| z{#|$fdN;n}F~kiX(VsR7rdk3W&)Es~JpwhUJ`dv1B0%Ve&q0`2TBS|WeW5Ux^bDDH z`HPQYzYd{02};cZF&43)5rGcDyn`Dxh|1Fs32yMWvAjNdp$^(sS9)d(;Nst?t#G#0 z=9pi%%R7x6Q^gbZxkhQD3embLOR&DDT4pZwkD?;1UPW%6#=Qn-MTEVTGTtkZ6`vA8 zX-Hgz&_h5Na5)cDUOTycmASp&fNiGa{^h2d_Ae8M_BC4Mz1KZ`ORw*JTS>+tRQJI# z^%2prLg=3jzkpI|Mn;tio2N8LfzQsaZRVy|8Dzkg!~b@H|81;3yOImh!Y6OMt?D?a z_<3e%zj&f&xuRm1*R#Rbb}EwV>#cfa_sV&ib~RZ-w=$g=t`E#-ASG#}xpVshGDJG| zRX%PA7p-xAQ3q|#N1UvmBNgHTx>k3MWbC=Kd6ce5TY$O|iD{RCDr~w}=Osb@ZwGXF z=H&+mc|TJXGj818_xDv(KyZKu2+u;q)%Dp(tm&mjXB1Ec0z=KX7KbOF>8+n!D{8PY zz_lI_+(Z2iD{FxN2OHuO8d;N~L%wUFTbb@;6TR|b80aOZ?zR4|ijofEClC=_LP^VZ!PuWAdhHHdO@<7+MK&9yl_CP41k#r118 z;r+x50P=^@B8{BDyE1FNVsd6oqLd@?xUlKKP3J>pP3eX z0sRVE+i!v^vwoqmzrtJ5HpNE&$eH@oSKm8ImRI}7q}MEn0Ka`lXdJF9+V z5NijGnT45|>3O{JqW;S9T3jMD-eqNfBHZ-|TLxI|WbI&XmxwgNBXBy#O@v}>+xgnO zvW;BZW@=8Tp@%{f1!?I{&a?`@l?q7FHR-Gs%4J%*`aFgV;uWQ2G=}RyOXZB|ahd~# zjo``o-4+e=;<4YpE^nkw=B^ZHT5#` z*f*rA98Dkvu(~F`3%D@&abAF|ni5X=pX27_Ce&L?cE}snee0XoftRPOZRE5MnjdLZ zF!!IxdZaW8>mu}3EO!S&5hWzmsl7< z10hbgR3}*yH4sf2I+Uua;SuN%H84J*2gk8x^|j8Q52$(mNa&ZT6!)x*KX9V}d2pxr zw5Gvdywb9IuK7GI`Twf&-fSGA(xzi!bFTT=o zn6*E%O-;pT94qMwbKItpZokD{1@&+ip(jZIAq@BoepPmQMA=x3p)w&6@V`4g!wx6@vAK`$GzEu2tv@ZD^%baeEKT$T^JhcmToc2`NV8R6$?i?|o zshT%FE~3)A)f`Mx9IHDn>`-8n(o&F+SDN-7WgFFxkLJ~u%WgNnyKQl^esbf% z5t+S5b}3!n5*v1|LTPX10$9>WLrNwQA`L>`a$l%GSm1~8JYOq~h*^%~uVqy&Ev~s} zZo5PR7jFztWtzXA*=gss{}x4^NLj{&QW5cdNbfaD9S&eeM{_3Fj!7X^*=r;?PG%)C z^Zb3K;f?Pzx%-b^?uo5d!~?v@!irgmP>0I*SZ}CwFys^Oa=i1ZIS3IOdLeJK9A3KP zDBa#*MD+K#@VFHCOvyzK5bDzN6S1J#GQ(s|3TcX(_gO93EsEHMi7EZwH+8) z^DFNweew>vX>@CuTj$2F)`kY|E|tE6e&U3mpPWjluQ>L`&A$qO6rRBF7NG# z2L0>_op-!aa%&?Sj)U3{p7dgl)4kCO^+NCM+Sks;_3`bVn|lZ zs|M45gxbhby;C`~#3rJ0NdFLKuepUjzzcf99JHsqEq8h|^_r%k){mbqpm(S1zgB%& zNf-z_HWu2`U9b1a*{e2TCK;6-iA)V_GCEs_ZGnJC=p_@oC@;0_>-7_)sbtIRRsX&% zd8Ijm-DSb$_gXrnF~fxDc!`!zm{>pq|S z+7y!xwx2hCu2GsB?GRb!QdKj^ z|5|c>SUA>wJG-U3*s{e|KKNTGN*}aypcQ<-)Ax@ z0yEP1xdI0)W~?Wiv3!x@EHTiRuy%P zp@Wccz)BwRKpDRM+EXtWN+gM*A6D<5ueq4>8b76!Zd`-5nuY4RC!>(g8EgM?T9SzK ze&u=O06K+>u&y*|a&{!_#aaM5PcZ^`clgg=;yYMEXLD5R?9f0_+JB>W3^aP`Z|^Z5 z3Bi7BmoYwY2OH7s`l^KN`c?!q=L3O#59_jRJ?|D_PCZpNY6v4YoPnFM)3EbDawf|p zbX*#|`xpsuoz=<>T)UPTg05G%rX;qEt@5VXHTcnJff6^%38qUSOy8h3KS7(*l z==c^5L8qvH>CB14c;>|Os%2mgD#0mvqLe9&Bfoz~M9ofmcV$P{ov%@Mxu{0P+BMgr7=>cemWyMG`8Wc$@5RvDy9w{2ER2em4uW1oCa*a(l(=w!-z=$BB zS-w*-AY|SkFMD@+eSArb-h^s_x}na|DTN)f4|kfq*Ga$F*4eG0n)BEnXdcP9vm6f< zaO3kum^>{Tre|iJN7Uo^Z*^vzj+YX$-YB zy7l~8{Fa!TL9-bJr?0LaM*C~|#+W9ev^xZ`I@922@Ob1>A2gm* z?68{|x)R*We%}B1gD~I3L`A`}cu;o8(zU5s0+*TxICXpx%i{2$;5Q;-S{{*~EcV3{ zi4olWQ8$WDJ_I`^>~7Og9Q)@tPpNVboZbX3C_;@i|OZPE9OxunAKV z+=}DIrF@9!XTe%o|10yoadTFwRlj$Nz;Q#SNfy_P8=L+P{7+r74z>I8ayIvFIrO<# z==&&f>#}+L)gS*R0Wy-;1K|_i(<6ucU!vM!x3A-*yS$dJs<`@RIiW=hhAc*3x}fRb z(Vkt+la%S(;ja+(5d)>R#n23uLHY3WtOeZCyHzl$#mzRoR{Bk)m^2;2`Bk_SxrxkN z%E6FwPmulBPxi*09ZRGXDgfYT>s(b(vwn>og~_2;=7F3W=*<4*eXsRJ%ln1`X-o#A z0o#oeZ%&T3Uy@w>0=m9vVMs&|td7r@f;%GooB={x^79W4^d(x8ZJE zZMl_OMY1a?mOu8cq@lGlH-=rFXou5E!N;-VPZ_+3+zGAzoQPb4scC*eDQtl^D%^JK zl%N&rgY`jE5B(DEAdc|WBlWzLgCbb`GI^K&0s(wU_h#5R*T~SQGB8_ zQ#mO9ch&fN(5C}eTk(fy%#yqPzLyPO1j^}hu}^mt*7|4NQ&HV09KEly>u!SoSkSZo zCduz^geSY5zX0w-aF0=SD-c0|x2L1qE6w$S^oEU&-YakQs+Plszv400KJ6W(d3Gc$ zKCH=T9M52o9b~~@s{l*<3oIQS^T3z;&_H3p^|E2_SHH@|o+2&IVHlDw`KX9HYT z2C^c{coEq6!K65S5M5Omu#|?kVmlS>z2WrO^)@ZE6g5A;#Nc}M3}TwOt_s^i*X(Gi zL4bzqy%AlH%5S?q%u3_uF!NSN5TTST;xW2S1U=8l^DdbtWgB8GLMtcJD92R*cYs(>C@hUHN6DvpGR;VBIM}Y;@@4ZGN5k ze#X;R#lN7B#5rS%pUMF57Ez;>@yJ8luI>PF9gL=kb6LPTPo75vo5>9!yVE9Cbiqb- zW<`$IZJ0N0Hs`ZzM;v2eI_3nv=NvMJz?}}O&xq_a-$i4*mhgS-AMR5;h$*D7TxmKS|uDfD=R$b;cbbLvxPr^@=o z`*3HAZ%c=*L=oTiGMM%LgQs9818hhB zbw$%{@|0JQG+%)`g~$$ACBWAP!F*(7LjzEZf?0YWzraC}ItgUzMS(l*LN)27{JS-k zy;okO9^*ZzyV&tW<`xqrbMS%>T}h!dHc^F&RQ^KV%*O~eE!~0 zHd@WmCWpA!3M#?VR<#bn#Gr1S4l8t4z)~2{Fhor{xx(<1CyE6#1)xfg{>;$AF#E{0 zOc6FcIYu?@v&Lzoux;n(7~29y-*v}{lMJf(;@RmYI%SjnliV*+io|jUYnW1r9ItxS zM+<7ZG?9N!hWiM8R~9=QkpITtA~rXDC;H!NVZ52|e4l=-WIDP~7~LsNn^SUFkl5J@ zBsWA<>&~5)4bWcfu>5@yxy5DMKNI(rVpHn`uY0Y|!sC_Jv^%;bX*i*d#ZmY8&`8@A z2oV6l{3J7$y^o_M@ztJxWtww7x0M*&KCh9qlj@+>24I6P|kD+<7>9!8{CpQCm zAukw9=Pp(2(X#quHmi$Y5n0hQ_U(StO(IJKQSnr?7lL0FvDgqn6PntPnI{a`5hSM& zY4)Bgq+i^2h?$_88Sx(@rW$Ad@LAq!PuZ3f2g*MSgl`vm$#W*{@`J;H3PJ_Gb{u)Q zyoBag1}!+^yt;9l9Jp^Ln=;WRF0CSH=?qThx`BND$_U*ku3EfJFQ?jYwlf!DLwz~E z`<%_IPwbNq((;x2P`T+%3+I{p9alfBAEua;2X5dz4b2!P7= zh5$kCV38Vdtz;$41mq*8%nmvWX4)}za^?oOA6nNkHy06>g0bki&t%#rsea$RVLb=( zg|iQG_wU!WbS*NkzPa|k+4Wh2@@e{`&uTI4zGJz{tK3a-yQg!>E0VeC*HY(Yjgp$y zAH0xk7^eTbW+_Xoo(8%0Va=iob?<;qIrWzS*SXPbuH=+|@$6cC zR=aMLSRFaFm>D5HjAGzbKt8ZgrR2#0?Pz~{zBxvBr!p~<$Lj}k5TL?Fz-3k z8#Mf@_x1lxm*778@g^LL1K|cDm< zhYKs|(7NagrsQQ2C`n5{6_5qaO_s>=$lUqpqqt)o_YkZt>B*0jlyHoj`lwgLwdo1w z)psJ=fnyOw$OW;}O;0UL7(+vSwV^3SNS1RC5OwQ)YfUpH!Hu6c$7f5~M}pM@wUQd4 zDC9LU=e1mCqS^Fd<87|SsmSVc+q!l+A2hONK0w_!4);6~N4`DV4;HY($c@{U`TDyO~;P;-i!&Vr!Z`GA z+Ty0f!EleoR8skIw7Lka$3dafOo#~`XdYA#;rGyb80@`}@qTD)?0xX`*7;F!2T5`< z^Y}}??+_cP^;hNuk;dc_>Ax-0*V-t)u=cu4iWzhSsxH)^Lg6fN94%awc6`I_v0X*A zYJJ?vWwqy)V@(<-Ykq@pjzUN_l+x({o%hsR94hUYe?=QD#(yiM>nBt)s7x9o6b5Ha zKGuG8@T?kbqj32-Apd7$y4hiw;fs>%TN@^MjWHD75X(~!7Yd=li3Pv!Mz?sV7bWBl zU__-C9f(o(dhPU5!1V_S_AyVvGN=C45e+Cljl6&G`cij9r-Sa>zcScP>HA1NpT>{% z5!zK3G#0cuAxUfXI#Irdf;iBqCPu98gC}8IHWggLw=n}$e$9^5S>9Evq%aefeH5sU#+k znJwDjStdLWofJ=oIK=wQbX(A%ggGDk8;A0*@Q~B;=DGJPdw&-3)EfxL)uNT-l{l1< z>{zR>KU2~uILS;V6{J=%9Zf|NzD+)kIAJt%*(buW{1PU@>e7i;*O^3J9F1Y&O`-p$ zFrl6IdbHHK3J4=32~88@u&b*7o;x^}k-)4U^(O^RHB_Ur`Ov9&T3nrUfN569X8T%M zSpn8MyXS&!uPB>}cz50|#jf7d{BY)hNXL+P0iPhi1|DC&;we>^$ z4~nY!X6QMSletnRq{X0`1#Oz`-slh!K9aGBApjJk#urja!wI^Z!+&Ld-FOX9AGF+) z(0igSr}9xa;m_0$B>qC9r_$3S-od91H#p2jRvnqVS~h!5^bFMgwADV;O>k)Np2q!I z{|a{Nq}+;Y;2!I&>sq!?I2ZDQLHzRPl)Ot9C9p)bPU-hDkA|#v-tHz%o=V(0`@K zqBmS&W#N%>e_bXgg*P&faVx_-)3k%JCa_3%N~x1aMZ9iNMHXm| z*~<-S&u4DiVumYs1D->8nw2`m2s`&}6pM7G;cHEy-)?>k+zuJ+Q$2k3gpFKS>-*vm zx|@I6@xAo$ryu^xcu2mLon@*M?VP>XlUQtHZr=EykXn*g#h#GjNAb$qA?JZQSvG1) z=+MHN+P=~vj(gy}?3>#Z6h5{kf@MZik6<&8r`CMu!z4BXLu3fjC0c>imeekL&IMxZWkCQQYtR|<@*{>MSiC-{@^ zSKZH^|1dESj&)dj1xZIBS1V!(S_UgoeZljuKc;uVzN;WKJVr8=GzoBEA(gJIPmlF5 zqF9DYFZsU5eeA8cf$43S;?;s9uQ@Np-r8sPe%@OCO#-!@J6hG_#TopRwO`qA_eax8 z6K8wMrZ!7LFf|tI6P6MB=!rnjI=$i<*Bo+IuAn=(L8~+XCq~h^9iBNx;=0xN8YuMR z$s5fL*IUu&zjSsC4bNRytv%Wm^MwoY*8)aNVIxAJVs91s0Adi_8^N9%c7+Y8De2eb zZOkvibV?+B{7~wdyBZ7EUt2iI+)BECzW%7A_Pu}ipZoabY!M_ce-&Pdna0p7Emye` z&w37Em|(8u%TF(l@m+${u*5Xo9fExqcU+b``gxm7esZI1ddlKzK0(_@sD=_PB6L>u z7Af?b^({kuO_EU^Q$z<{tFEf5T9!L~ar^p<(NRP*_I>;4@{RLPLvv(q1wQpKhI1FO zn7P9B!TF7xlq`BkFc-w9xx@5=y3M6QdJ%(96l3Ir>fMxNKKL9x|8=KIY6R$Dscy40 zMrR(K9LM*{UtC(*$eJF~%j_xU#+%!oXKhj)UrT_Sh?%Z7EYj2Z@%^2Gj*Bnp=2R;L zW9PYswa!v+*%47J45OEVp%bIdA;pu7`Fvb#DUprv*~sVJY-#F0s%Ggg^ts~m?Spx- z>S$1jpm+);NF$m#V{|m9=Iv69 z7wG`aZI2*~L<rIF7&J=TSRi>CCtlXT_ukw#CWK30JvHeSi)6S&*{F>|890`@j?*?eHMk!OsJ__p=m>?Er%C-iY6WWYze4gclzmcXEN-gS|cyT zUToh&W^mNMRQhD?t`5@BvWmzBXj7CRWEEs6nvo5ytC~1vwz)ZVL%?yALsgz}37V3B zPu19R+^ON2YzQ^xqxXdmkAGW5Yx!j_O}jif<$I@OhFm~)a;_Mt=I^$T_&&|s^c{4E zyH>L<(I}=N)!lO3nQBFN$cDcw_lvBN{qc%ad91hVgE~OVOBFbgUCDTlHVLTq5kZ z2@iDjNM|3&&2iha8gM(tL!E@0=r%L3mb=IO#%)nQI(dC$?ZjY2jJMHOTmvUsd1zyO zGwRSv^cJnC6GDO?{ zugsoLxfJ(_k4~{O*&#P8%1NRSQVhH^JoL5E%%^C1;)4Iez79_MvGh82CWoE%{>xLl zV>(ExnqzhpX3ySlys4M!l!|eK#UPyZczS|MUWai9gKGZlB6YB$b-B21;*+2FAqb|X zR84PY>Dbx3xWG64G}|kjkTcFLw`CSP9H3NV2+1b9T>_ObMpn=d zK9`BzL#oCG7b8Nw4Id*u)%XC_D5tqp?Rx|IH`)l?-@gG*-u<|9dG{fOThQp{WjC4^e%nWbLeb7N<;Mi!%(i}(TFVb2D@EPv8=tDEW*ZQ z(dk$MoUmXOE|?bsEe@y3-zg-dG`Fm=P&Wu{0eZX;zewn5q9Kf-0yQp>9j&=C8wxntx?*$qKt(<|rQ0l3BjV!}){^PC6cO>Ms)) z*;nf2lAty*R{)!8a8ThRiSIz!+6f4q1nI-l{@^c0ga#eAiVjr(cAtQ}wCxIf;CmzY zBtzKSjyQeWDQ7~viT_}0W>~=d@N%GzC*npL@r3KZHNBr~RHM)lKy#>>6tqx$De`0QAXn#_-7iQBR-yox;qvj!ejl+%5hk z883G$@N`FCls>AT2y^`B*0FJz>UdEHb`yXmdHX3bSP&Le^mX=8L|jdtFmd>+y{-0| z=}&s_fUp{8@OH%tm5TeYFz%_J7vYM|_HWdhQOZALWxTV2Kxz`X!Z&@~B(Q4cTof}0 zR?Ri&8#n-v8*`wB2JPs!%4KCbOebPkT>QX~`=UGP!|9;&viwuSwUZ`EQ+a=uS{-hW znKEnXeek`|CCDTm4lk;!`*$uSRgZUrT{)E_rSd6Gvo*_%FG|g3USR9)sjkK?3UFR} zVvQ+)-UU|vfsrHBnRxPIx% zrSaXmHxIrQsX;VUQm^jjp3jX|m@hIs?UL#eS%HU?Ye?l3Fr&XRy@cYT!qy(}0)g36 ziCj(8mO%1xD;hd|-XW2rj1Ot%s;~#?D?hD{URU1GQDnc}Rjgk!@4oJ9?j9SA%RAaH zFs$t1RQw>Yw4h)dqY>Y@-h?v?yQ<!5HuHM}N&~R+jt6JD{Gm z4qwm*y&i4g{OedVW%J8>?r3F8@p})zG9dPBtKonI+Ngg+7UNZ>m`*rQYkn9n1aoEk;98$2tWm+f3$&;08RxV+!Bw z0uICe%e$eP`y(o1=}l*-INt zuN$_kUUijgNuWk4ZG$o?x)Gr;rm)pE)ayz0Bx?>IIzi2&uoG|t4Ba>)F!NOob|gAp zPatwk7QveU8zZ%wN8%1bZK(@~QUN&V+oY(C6K*cb1-76(n+p(omw(!VBcFYY?=mXw zoNO2GlLm|r95D%A19*cQlz5|& zaW)DIfsWLPrFLc#B8s(IZz$JmxbQ3k5q0kl^w$_1uGBefNM*drf~I zYqzoK*`pVL2gs449pEBl?#5((xy8wZ$js=Ra;Lgd4I@GVtTK3VK)|!@qq@FipRKa- zPjb8dsnE7U{(S2ZcZ+i|r=$E!6gWEc0fD%&x?qXgB*Aj-927@H=d~?_%J{i_bD~pI#K6KDs$F%jgS@zJ8I9SRp|69Y z)v_4{pD}!<_$N7GR{coo9#ww|O+%M;D>dQd(VGzwGTgm5(`;J$#)&?=Cae?$wNM>? zS?meQMoa44$pyV`)#=?DH2R;XnKvhIyxFqYY;tYepWL(}lMk!7ZOH4T6k=^`r_f53 z7)=qOf3vPe40V@#3g#E2iO4KNG=LzyB_AdpxXGFjjGF*YV%aC8B5XpfbY8qfXu3hSwaQW|d+v;a;_(o#dwR9$A@ zuij?{p?ju{R*yfstMZOLy4HTW*P^R9PU4O}kQCIAmA-Q4KYZjT#I}j`wc|;oRFbZ( zc2T)A2qjdgY>LQ@25RZp>NrGWzpYgP_TCH$$v>6-SS8dH?C7r5X*-S+j-U#&s-wQ7 zD73E($Gu6HY5!UDcFfcikh(slHk(FOUxq{JE1~g1#H#N^N3`0V4OtdXR5-s}H)62C zNsZ|KdVq(P*)5gcx1O`SonW)PU8WN%LH9{T3~w>?ScUm_$jXP@%;pq)%zn}LBbTO&8G zXi0?w)638zL5n;`PNCWDxQkl&T+E8g=g)#=`4wRfpd(IAwfkD&=xurC)NYl~%=l6v z)-g;4>!*51n6tbwkSe|Oxff6WdA;fS0(U!&o1d(u*(xdBeWk;Ab~!{}PzfL<5$_Q_ zDX-Gix+a|3f18xdbVSR~{gwHtNAn`T5tbZ3xUy{jJMMh8;o|t?(*_{zzcPioMju1R z!DzjgCGVuc#Jf9{To}Hlt$~WC%|AQSXhHmYew=X{qprYSD5o#aNzVo!dc4u@f6OSkz+ep$y}eDU;U-cEITHDJP3|r^^W5h%oX=dBA13{y zl`|ZvM0PoTKKaTha{~30&%aJLs9N}zktDep;wfl1Jbp*~C)$wOSZ>k^VZ6H+pSgM0 z4I#_mSggoS_f`=K2NyoQt1h0^{m-}Nr&(N3yYFR0pW9UH*2dZ&r<%_EQ}{cwA@Y*h zgy-77h#0-ZJlkQvgqKfd09dq(QJ8yb_=VK*<);T9#wBy(_Oi<6tEQa$`E{^}&(;Bn z8eIeP{;Vn)w+u%L{x0j+-WH3lMDz30;~tR(IbDHgZKfWjVAGH0zL(GAk(@7m&4{r+ zDiy>HMnmWkcB6PulfIF5_*=U`+gzfldR39>M{giA5jC~h!Xj7cm1&(j?l+8Y!bX~@lHf|# zkz&YW|3`1KhG%TF$jgm>n5zyk6&QXz&lK2=eR{zt?QX=7@rl7=57!5LA1{0L4;LwM zOys2KHdjbfUA*1nmQkQUE^bA*X@Bh|WmPU*@)&RxSU_29J zquExf{q zGJgNCn8`CLnV25|d0N~R|BL_cdl><&&y^n6dcQipE>7kmCxYNy5T;~V^N=hY|0lg; zx4}BXrsI7!3L@XV?@QL~f*SONPoDa(GrWgvzzQAM7!;9wa+b~go)+A* zE2S%ui2;pub9mG|%mz+Xsy?{u)^P=MOpu1fJs$hia<5q<7h^af^<4V^`O&}4zS|FJ zIC(?A*CymCOW=-rX?E5+uA?hm!|A)GN75tn|H{n&Qp#(eF{>-@eIX z6DhvQa}w~TZK~6jR{PeAyz0b%)8FWq=>6rD+@mBt1CkZ~N%FK; zMY*{tT0DXQvw+L6zM?&@l@OrZ^d~*%+~T=R5Q&J-9(V~Mm>EoYR*iwPOd9%g^NJLR zyx5I)@1=(Ig#_+j8EQ3YK``a~$K@AtbQqbeQs?Ddmg zM<(L%W?#medqjW#l>-@K$t!`$Z1RLVu3471MI7&El-SyXE8Uf<_A{>9> zQ)=L|BGpWK+MZ|#t3(f=FMxI1?dCbLZvNL$&wloid+3?}lfyAr+_qLc@qR16%bzUs z=uv9i@D}R*Cg$in*LVHX>{9T#CnlAe(ob*}*&f!L4FI>K@TBn$iM# zv0fF==i=~L)meE(98B9ll=rdu)k%&68=ygNY>Y~mPDU3ynsax!xlrUzE*ib@%Dmix zaxGV|jm$}vb3t@KlD(0lz-rpc$ZK-Z?}*%rZW|4Cevv{oyt5Fk=MtYq&5K#i6?j=dypDe$jPhk^kNaYm20lE#GehcXo=+zU7NPbBTy z>v3PZYuD9>rz5Ey#}RSe9*kHjp^@CDmm~_v$_9aI`H#)!XsXL2j0rRbIkz0LPQp!j zX>P}z+>*Mbbsu@bzR?k@O-`e4q6aWQBA9!{Sb}ynepmz4iST( z^t;)-Li!7!)5A!t^qAylCePO4c(0-H{FgrxF8)b+g;CZj0mXQ=OkFGXodG<0UNyGX z$IHLee%;vx7P)u+z|!jkx6Tj>D}vG=z$O4!wOld41g~!2Yk7rn0xP2_hOYUmTN@lF zm|5O{A>tq50&;+yHZ(ct8NhL|!&K}Z$5~}_KVFAJPa0iIz1}SG-%jrRH2%1CDnzW| zq59p%pf1lYT5Babg6X37LrTc7>#SpR#3@fAPbf zQMmMO`zhOj@$g)#%vhxMVoOLa)&VsD5TsO5J@wp}xg}}8F)tAkEQB28o+zKSN<~SF zrbM_~<%hRz|8jjBRf&ehnIAeWG)j<|xF5&xq*3fnZyFS&bCac|xmXP^4OpUx2(+>l z^%Y7JjiXh&!{b|HPLj<2?V|@{r*@gh=zC4%9wq+F9#TW5G1jXoC??av8!weE=YYm@ z`qXF$V^T9#1aFydNkF$ojud?G4{igmYkuGrWIKBsKzRH#$A zK>Xxq#;P`Ctx?hFAj$yEzETl?eD8zwt)*V4?5zZmt-uFqMGx{L9~&0m;?*E1oI;^M zf|iioISzyz1d(gcpW_(KYzmp6?7=#bG-KC$1Umh-Q?;f-`w<<3*rrnX{eh_;#ot-E zp_MA#n6EhCYFUqGSou03t#U{)5#XioTNlq34a^ z2L_%!{Z>Uwd{z+8EFzY5+F6!Zdgw;gZC4QP$_kHoc0r-&dt)y9($nKP2kz$vdSfU$ zm1a?dTGyozg10d})n~oC=x4<6YfhcY@vd_FNb4L2aCHtfxZY)1LZ|>6fqi{FxmVz! zp|GIsMmR`cmC@kf*EV@Zqoz!{rk{01@Z23ucaSqx|O8w0o>SA1%+sJPU1* zXV7F-t7BBvU85r?1Z_hdb8BsALp$9zdlS~lxA);OVxhtyXNtB?cBwB)TdN}y9}?~> z*Mw_V&9C!4q!eGy_Z2}O;s#&V+v_&6-kq7LtmJ>5R7x7&;GcF}8Ty62u-@$>o%Q`R zY>&+{MOG}Xbz2Bd<>KiI%gqCm_Lx3FTaIVrOoLg*khJrwyn(X2sdoAGM`>>?%@^`k zWF%C_23NhrMHGW%pFjV`bR)8$N9a>XM59H?i$>cbVXeBoUQ3Ga1t*94c2i%WYpFJ# zQdnK#51(O7g*e5_S6Qd#RyA3tn*!{)`ThJXYR(jF%I6xy)-HjR^ptaWm6_8GgtbZ@EUPKMef(3)f*W;ZIrERDa z;=~`n8*PaD2~{R%Ha~{G)BbVppo~`oHZ|~GMYzPz)0JBUUez28BjMcC1^HYwT2jph zdTQ2<=LW^-g}m~U=0!K=kY}JnZ9#M8jI+W!uP+5$FT8Pg@+o@@GSIR1Pb5$~Jk9c( zQM8WqkJv!UGrxMX2{+qeXj(;UwDIG_n2cs`4-71%%moIQM;O$SU!xF_J~?TI$QaCj zx)u^~6lY;ZEiv)VY8%7@vd9s_b#b4($_JKrj8ym>y_aJ5?w|O4l@F;E>F3-IZYu$( zZ(x%P^qSm?bZ(RqR^CUGr(ldkC4P}j)PPUWZc0#cr4lFS=evIS2Dv)tS?Q4G!je(T z8JI%eiBDJ8){=@3RQm2;@3!d5X}mw~|0VSj-);N7V769&W5aDG1)}%$vD>?TPmFbL zf)Tl~ci%pv7IoLo6(4V^r$vrQ;#Cl z6Vs-ug<|4fxWfsrcL5j9)W7&9ZA~px*@1PU-KvB9n{eCNaru1nqXO-b$N()E_*(yX z@!Qh~wZ}y|WnYV%Fzz6|d%V{j-fG6gVORgo_Ok`56&kiyUd=Xocdt!^7zCoD2p!Vs z&J#6b!tfwA>r0_W>F}Z@;~r6x*dHtoI56aQKlR2UsP>C-KuiE$szSFuW`ApXRup$H zyL~!9gfT2h*PNUP+m&2~Sjc>4gO-aoqRdW_7q;*(n%t~hzH-$JT#gIQ(8{Bf2ztKiRM zKdjHasbvw!!PSs}dw`VSh!LE&1M4%%=6V5+BG9 zR?70AFZ1hSb(N6wa?2iio|@mrP-9r{mmU53E=SyKX1n(~yJYhwrhQrcON3*CsSbzDtVo4UO=Zyp2C2KcX%#?vH-`_}r4>ik!^o zkq>HLW*;Rlge&~1EbmYCUb@j((wJW@aBe(J+!(6hv)s|n zRkz3zf4#M}_cLpzb_*dE)y!$MizdTQ*>LukrmPa(e*pM2CGX{Yn@Py{R!Gf0Yl_YB3^A20SDMtj!K|8`XZ$nxS>vaM zgWj~7wvS=km^+69>%>jd(_9iTNHFzXCI$kKHMQH#MI48cgp!Y$~`Pt7e){(o6;b~gLRzG(eReV0=@FR@fZc9jQd?;2Z;x?90{ z`z$3dw!iF?Sx0n;b?$9?gF`VhRMbE)a|>y=&N9mi5T}^{BNCFes2ic=hR&uFe;{A7 za-i?^E-qd<)8H@dj_x@r`)J1MAikY*>-|tUiB$Fek@OaQBWv z-4&l(H_ZRxoTXbU-@)}7&{~rtjc^y4dCb{Tz1YO${NovxPW{OAuHPk(aV7k z%B^IF@O)VkazxG#aehbZ$hH1rZ3ST_^d@+MN4b9J%nnIh>3wTz*tUD@g-YG}XpO=&Agpd8Tb}U;wlz;FDHre^frE3F(u3XAv z|6miY&+@XTPKjG+inAyz|FnslbBV*1D-oYxY~>k!J+!^fRM-m7^tlL`D*u1XorgQy z>*M~N)1g{zkGA&cvLbfv`ROo1tr#Iz?U6?86{A(v+ATsWL0ck-P-GxRQPe(S)rcK? z)ZWf7zyIR$sn)|XchtspHsCJ{i7wS5|%1SZ{Cd=*vLw5ywr0aDr2&g`>^&pX{o)9-5|r#f!{P$MiC_00|l&2Gu|+Y zxn)AebhZh5M%&w3m%a_m_n=Yh7ykq=Y*Q%diHw8Np+assP`-anVufpyZj^6C45$8& zCXw_`exO;E!f)<($)U%KSQ}?=pVg}4@RKrEo$H6im}~uQCS)*nA;c~Rr64T z@3tUkPAzvrfeX`xmy@T1v)ag)>O$}zLYLN4t;u>D>6@?0d5@Bcif{sRA;!k?7m$a> zS~KjfH2{xK`WB9NfG5v2zGBY~pgAhKU63%(oW1%~`9}7|PeDBz4a9t!MU6x;3=w3W zo`D)7CO{h9<3iExQ8)gC#3`oo9%lW_*7`V;{C68%4>DX<{)NlR>O&z@rNTKLKw)E- zdzT$%fB!9?p6Qfr=I~dH1(lWBm65z{G%s(K! zjDK^U2TA<4ZV7^-^g+~7UD*r^tg0bB$r%e^=7Le?M?cRhwBZP@s6oE^zL+|p>m^t{BB&(EY6DEpTp(|v>IkMDnSnjUz4vFY#IRiRq0 zqm0+k!=#o+#mU))TV_1_v%#1R!DVh_4;uy}f@xJ&{Zipx#6YV5 zUOpFd+j^|SG3(oU$;u{jgPtzNLjK=F7RLJ-Gd;m5)8L?si*j2e!<<8PcvNjj_`yNi z!sPH_Bia}W-8XV{h(~x-4&a(g%ZF)i%mzyp@Q~U%jLt0qE+5x4U4F9e=kByB|NE?| zbv9NHUyAp zDAoV>(Lrm2QvLA8DKON&-m z=}jZJPuLDMr}ZmGGLU=0m0A$~)Z&D!sswLzqm;sWH|dN4fe|>z;003>1}nPPsdVpm$qnPF&*(I@$c2X=zPz~M|0Q|i zlpHZaMFtY#ga&@yl^#+su^-oC29D@h%E-zs-&{y<_-JBWY=0}Mefz;1QR{!K|M|RO zvSpyV@yTL7nnzMO1fE|>PG0U;EC>##6mADAF;biJ`Ol6UDn~(^f!Hpg-lb39lcqU@ ztA#7(rGvM7e(+vgEDUY)=fnb!bkCt1L@1@Kq^D>f_2Gz;WI4GV<>+l_bUS1f-w#}o z`{Z#yQCWl2mq}tn8R7f&a&$qY#jMJ#Rs0i)@b zNPdjpX5L(_>zdnxjm3uC!XyFy!>x}E2uJyB(xI)JVYq^lwNgwzzs*WnqzIotnyxO} zZj&aPOPfls?4$KmG^1eGcAP{e??Z-8H^>O9J{)MOuw&_16CE7fuh@l9$VUXbt(06w z`fTm3F{^VM&M(6s8in`%#+t~$z*$EG|x)mGH zSBF*A@-oP$6)c=#GL%U+lV&a8u|dF&RW{8W;+d=R(^<$>9)+kB{-+d@%)Rv=i@S|} zM2^I-(`F-sQ_b$gnh{U+68=WzmX4m25!D*-0>Y%!62we~Jcor+H)eO?>rxLn*`A#C z8bvXlaFxuahkP}BsR)d)TxqpPv+EoIC%Bs2R#9qnjdT3WkRFpFwFi4WT0Bg*=vDbe zxfRsLm2G?r+Bar!wNnDf3D%su?iABPJR5P-f-L&23h%xXcA)qlV^yahfblVx3Tczt z1n=Hs?uVBF82~D`^3%U{beMHuGgRAeXWn;D2Yh9@T>`pE0}v31KYMGDgs-GKhKZH=30Rw-o@^#qU2;w7l=aud{k<(D$;G$Go0@zZ)7< z^{Fh&^QWBrYWByAbc4%$&!qqI(V(zYF#trAamdA?hAK;Kgt|IXL+Z}iFZD6`%7JxY z_7lF^h?0DVov%jQ|Kfu8TCLy#eULz^jaZmGqYgZ2kj#oI?qU5UM<@BbtV|C}NcOShxvX=N> zR@PPbeKtQkIW7|%ZMkmTTX3QZ-0`O5cyJPER*aOvBfIQxxuUfPZtsf;Y9UM5yFum0 zAtsitn2jT>)?t42V)uyR+qOG!u1rb4cf53jYeK*~kuqLC3`^2SFBKbGwzrc;_w$_rMHC&q>yJ;Z6B=8KvZ;(JlXLnz8IveTfGAwG)meDU5?$Ne z??_F|oalhNuVKw7T!hYzSwnFqlN!XW{K!Ra32OEqi=GZ}`;7du=;P%JO8Y6CkADOK zh0VCTSS934Q|f+q>zozr5$iEX`gl|bvuz5?t2>gD-%pO<5m5S&%tV2ZB4Ryv(uVE6 zClg8{=F!NgnDyKcuDo1@1M zEOPdiN^?7aHo9tJ(e$2xZR|@?emd@AZUWX;#}+>}yxLvh*}TH+Iy_LN3H-u*7_P4sQMc)PFj9G7$|q^}InN$F z6DOS$4MlaCqxXdU`P{lh*l0D~!1}l^xlZ#-NozEc^XI7o)GY}y&)W36*yAr{Wsh7T z_Z0ak=U1}^V{wiNt;jz(37EyIW!@&#ad+G5tq5G|M#yTZGv1xYn=5HWw={FG7;kF3 z^rikdS+QK*tf8!D(#~R1bxZsGr>27O>k=cv!rP?*4#F>^j`iBmh2imHkAA7#RTT^I z?(((P>9IVqA%*D#h|^&n(jE>CC(_n?qoV#A?c9lVo-Ly~Hll;BGYo#_fhnvz?iei` z+JG-jUaM*NiuQ|u7Fc3Vv<#C zr_F4`%o0J$lcnWDJKia4qL)eTI59OEY;GYKW+WIM55iI{zIK)}Wh|)`OHtQMSyFD_ zm&&qr^?V_GkUf*Lh)H7UQRwYfmH2%r3dx!64+CJ~HH;o}o%|%pIew9Smpio%2U_`; zPywUjs^hxFW!8mz2PMiq0O`6OCh7(-?1u;13MyLIPqg@OU$h-#q2<}3_~U!E8J}5V zZgey1_^LaH#`i67=Gn}7{bFhB%~85z=lY(3mF97%1P=@e{ZOz=Qgh@tTBBndrmm7G z5Fh*NUoTuRrZE@e26{Fu%MQJvo%3uRJcD!95xKN+`Af;XTIP>P|)*Fu0AJPjCM;I)7FJc8JIt{cp z4g>#VX;h~$|Fk+XHC2siwJhY*Q&$MAmvA13^&+EB0MV@10fi%tif*f6od`om z!ba(EiF(EOt!)Vd<)TJ^uBWrbp%Y7&f^)CLW*RHgqZmSBiB;{)QYo4iSB_}~NCg>9j zS?Dy!Y};TaR~uxWB$)wF^@4Lz#?GY5-Pgx@_~har^Sog?kGx`B&RLZ6TDCTH0t}{l zr|i*ZLDAZI<|P`-qQ&F7ISkHo#4|#qnmWnV*5pxY4HJ`@IBDdd;L5Z<&6D9+!W6{A zbIKUKz8HyIdRc>}b#0AP%KQ2!TQeZj=_xtn#U1wWTz}S=9(Sev$>U2o2Pb=0Cwaey z0JhO*ls}BDvj13=j7+UfS^sZoG5`7Sm+&iA;q+Im#hQ(MSW%i0ISoptr`eA$p9S+7 zQH95mB6wA}VOy@?02>HFn|;By{j#YxO1rLqL2)rDb9$e*JhMk-;GKV&=Wu&-y(hkw zM`6oSE2?vM4iBm^(W@`*L$J1dZcqf6H+Q0x*4O2(o2Jy)|8#?gmC-$IInArl?mk-dzPs1fScQmrY9bP8x_y!9$Y8?vB>;8vn`N# zx#sKZw>C|!zQ1pn zLpvt>sAN#=Jp-Ru{{H7AZWHJ%(Ob7{%(vZZ=>Z1WkFF5%gBz*q=EZzhP4kVDMZa#C zV~rK^!H+791uBvdaG&y#VbK1i=awvcz*r4Tj5F2H+0&MVn(n}KEJ(N+c#@$UAjwq7} zP2(5f0HhdtoFBLDqrF&8u-V!Qa5~BIy_Ko4-sP$=WMYuLS~X}JZ)AQv$@L9Vg;_@G zbk2Xt7yZf{r#gB*3=B6gE$2VgV1l~kMoZZSIFTtJd;?34jfdT?bdwbGoWDrq-Wu;> zJcy_>ejxspQa!Q0pcKq?F6b1S5B9CssI4m=G(L=I%j6>#o4;Wkz?w6@Ug^q)Wk=Vi z`AD5qJP3_%HK4Wjn*C}`3Ooy@ZE%2eSO2VkOO79g=*XMM#k|0u^q3o^O8#^b0EC$eIKBE`=no0Ko&kGH zlv>otZTp#3@%DqyWe}YmqcUrgkDbKZl)8Lg&~lgXW)Di4$JqEzaS8D#+waHh;*JFo~7U?+<%g&L6gxHKQ z>Q~WMWfwaSd4=q9by2rnWB!*Kk#)0Xy9pBZg7beY&V^*&mWiJd2^zGsiSzP|fsiV{ zWa+S3M%bi*+)$YXD*e?A1P!$#Z1&xRrDl_}7X99Gitu|6T0|}ekNKAjSHx=YeCuU- zrHSF~J0;fD-T$qu{`mp;a9Y5<#v?ZA-iTP`x5lNrd6%9-m`BI2n$ZW`%K!jCdWK`$ zbS7F*Hn=!hWz8%ELIY*HQ0mZ5Fwu5Ms`uKga!SPAKh$Qv^5i-=z9s(v{VFLJMqno=RqzyVlcH=JUb@++G}8ZV8CXbGRrl%>>8!!zh?j*$ zYd8|8)^BfJ+t1wMNeKv;>5CgP>d$IEzn%h#cl8{N4Id^kM1zc+U61Bc9`9>PS?gUT z7L9P8ss5<@OYa@vDW}zt+}_ts7ZHO%(X$PN+p8AjUYJXM@{x!bCzzOR_u<@sE=9BW zvoZFnOj@kn9G}HPi7wjD>jc^fQcG6u11r~N6!#~+7|%;tQp%o8yPy*i!ee(+TF>U= zle0NyIqwnT0!sQu8^7NNW@6Wvppq`ZNa2?22DD|H2TE93*ZLWT&E+6`mL~z#{pLw& zk+yb?eMh<6GVQ&_4MMT=;}Ki$Dl_OC^@#~F=~Rbp<6i0YvG#xO-sEo0l^lFnL8&CB zwcX9$H2o|@Q}@)BXAHEmdm`r9iI`H zF+JB~kaIjp zTk!v}TnaTYPBbD{PzUDIooWZ&;(go)7fh8Wx)dDMY}$+mFS(?CC@F{-Qm8}xs`9R| z!LkQi(Q9NbB{j8ElZr$XO`^Q_iUtzN=zQGy{e71=iJ>5;tw`6QKYNRaw+4#06VhzW zaH^E(FoH=qZBO&T5!evOGzob!KM*RU#P7{Kx#J&pwHQnP?(uLg1fi=sRiy6$7Ulj) z@}-A+zHXhMgVj{sl1W-+Vn2DN%dNkvtJTf=^kRlz=yM%5J>5kE<7}^$nzdn+WoqM%(K9ic~wDM)G$(Zbt*%EY?zFdkMBP zzCjfM2iotrH+%3Ue&PP~Q)mQ-@wz>QANhOM&$Q*q_AnfoF_>XT*w5Q;KVs%OWtPG9 zQ@NyMq~(^>>f1cO_!!O3VkNbg5gRcb^3M)zL)}G=_!%zLuJY2B8Lt=1Sog9V)H=A7 zc(6iM=9#j*NIL;U<}>iKK<5Pp5JXTuqbr_%Pl@je>Y>7{NFPN|NnCOtYSe#sC{!w) zBxb30b=~K6zj$|}1pcCgjBwQPHMPqYJCiN2fA?}!Mi9yEu;0?R1m zL@U%Oqpypobv$yvvr`P^^-15GI-6eJSX?4Z4i<5WUY1U%7;R(UmU`#Kc zE)7dXYSq?D6Rwx$ji%};VF5Plf+EsfisKv?7)|K{|NNeT6pN{>8UFp0JptN*$H5~< z8cVas@wPUe<)!bCVw*CmHl(u7YnFvnO~<-*A23&j#EjpoM|~Re?9`BeA1qFmpGp>e z$JV0`f=5WC1LpD*;dF24sxcs_DjbX741*pFskcf=)9MofCSrT#(lIw@d6%r`ogyvL zKPF*Vr%LZJ3hO7*rW&$sh!n`kw*#AZ^5KpUENUGN_lU|NIZ%1HAzwq(Yhxxb`%O(O zZvHrx)#}{KN~iR|WvpvDQL1ui8E0Eapze{$5BB>hq}{#x%uOQ_X@fr5&MCEkpA|II zP@aMDDjGSgigi7$oE*M>!nb`solG9~W6uoked4USF%AqziO&E!8$@xHSRsoXbOsi0 znqcLRhQmj!h6wyv-7<*+FVnf%VSt^%6BDTJq}x~WZhv01tl0KVUUs@ueid};1(&Eyo?y5k zVOKnd=8v%ZHo@k4tem15Th|SzleC*^vu<(s{Hizn zH}tyrUT3B0I=!`E-w0#I<=1sNB@8n;{Id{2P&Myk9ccnFfB2xzJ;-a|O~(A!OV~+5 z2O)nL9NaI~BSrMMTeNz6BHpEC3@WYhOJnnamEOEoaqz1|o`PTFPCmj7m>06V;|^~H zH15sG%v!*)S`MM2WrQc_b7o*U+dJq~+fdiw7GY5$#F0n?0=H z)C0xY+828i_&;hZkmqrx3;4xWrCoEk+u*0@e6f~~fi57F!PVFDP`^cB0O5koK!*D~ z-Fh%PoWQLlUx_qNa(N=SR_4Y78r2kV3A%$)eIht$KYTSK-LW2>`=G3x?oA8H5pcfr z#)mc!)%ZuFB0j8h2^#QGWW7T;Ygi`Z0V7-ELOn#dU}0e#E@EbESDxi2Y47^2s72?c zrbtK?%u6dUgHwU%Z;J7v2Z$k^d^acYX8)h5GYs9vc^Q94>}0x4J`Kytth9HZP|_Yb zOtj>IeSUnAnNdRX08w*${19b{#Kb8&PVriwk$|r?5^H>2<{%;Q^AWkUkxuGg!w0BpDMtJLRvft&;fjPIvdG6GFh$``)F8LAZAc zt!)_makEgr4;64-I&N3LKvp) zVIcd~wUoX*?O8auMk(11(yj+Uxx-bVH0NRM?~`Y^K_3*{YrL)GKeZjn=nFMDAtr6v_NzYz(q3& z$dYTsZ)I#GlPr3Z8G?>8a=dM8L|pKqL}CiZMjR8|oDc%R#r$5T`hOW|b-~*xQ8>KO zgN=VKUqrmD867q)iHL`wQFL<4QtHPU|b==LHf9c zUMyhpUh6@qON+riL>Uwr!gF6k`&+Zv**?LUQcbN61G99?qeD^}yxz8yyEFLGjHfJ`YmmajodmduG9uBfu>~m< zEt62^Hz)sq?0Be0d?}l9&FXjCz2J|FPL{S}!Yx97)}lfw5t9i`FXVR_+WVzXgM%cb zz3kzI^@8;WoaA+XjC{LGMS=#?S0<>5;TECNv)+H0FDG2pmm{oHU{cA(!A9wRg4W2Tv}O)-jMN=;ke^l3eK-aHA3j8qrsxYGjDAVnlt9Y5GizFZ2Lp6*SGJ_(ljj9nJ8paQ#WU#<;Oi@-5e z*muD`=QP+TZnW5l0{o99$cr}Yr<_l&e<7UdN%*jn!SaF`T-qv&i9Ph!?K+GB##wsF z1hWmof{^sGx?+V|b=5E!P8oaEIx_<+-k*%F3#1^Tu({+$>J7;7NQK+IDuFth-FHoH z$$Zsb!7lN;8)d9cy$~_A9`z>*RwHn+k+rt%dg^#>4biLL@y?iT6(KS)~s|>8z{==a~?X@%H)OxQqvvfxF>7 zo8@W2J1~36ah+XT=9Xlp&SUNE{(bR9o-t~`;WTDv*;*N-DYh9MtkKuWT%a@TI9iSl zDUql;5x}3&7?81buPL7q=-7N`8OvqQf5LUqDO#mG9s?R7O-nTt?r4AG|5$o}QVr2K zH&V|-E=rD~&quo{WVCWgWX^%)EBQwc8wCPQ{l=5d-=yGFkE+67rma$8#;YOQ$o-m9 zOFD)jgKFGg4vI4DLGWLF9eyjaqc^&5V4$26osRsE#WChTmNyJypuxDq_>@TtI#O>kE&}h85_5Yg6_*H+eZS>5u$==w}5cnLPe+7rOIEU{130%VKJq*;nkG zF?fNX{l_eM#oz>}J2GIOG9l!c)NYu5?Oid%0+U8!md6tcYFzGQmOE0{!xr#cTwth_ zP`hTsP{?nW65GPd$W$c@m0r`knT_h*r@F;JTE}-~NM-AsJ@MSrDIca-YYA(r$8gfL zS}rv0(vqLIb1tsQj;)?n;YA8-we-8}v0s~}R`?o9M+Zr4j3*P#1V(d>%6(@Y>qB9V zY8Bp<=y%fr?CmN&^OG*0oPp+zkMem>)Hv!0U3?Zt_g&XE#fK_a)*_9$*|OZ0C53Jz ztJ)e=rfS+Kl%X*-^JSUE2K+LjU|WS$4YO?t69YE{gBFSEo9@_;w(s+;XQ0cEx<3bf2pR<93CJ z?X+F|5%Dh z$#lF=Kp5(%#}5U7PE>k;Hl2rfgTp=*vZUDdzLN?&qq&TGS7>fv8D3+V1j zqd0d^fS+6NAI>p+csE1Y4UI`fa->aD;0e|y*PQZ!Z41HKNQzp__IXSD?)849dSjH(r zH&-~9X1zP4xF2CrfZ|BJ|4Jr#Oxr%@52Q__75B8|98g#|1#EQ-Dl#86)F1BVB$78x zG}&31)SUYv+naMPw=`j+8W-|ANHse>Bp$Bt{DJl%39EZMN^IG_K1%<<37KWDP*pXQ zZKd~+1R4dc$wxCw=_Oyq>gP$$GKhV6KI$85(&AJ z%a!N@Xn>tOn_mz4^5J|`y9x<>d6Je`V@9bz)zr6jZ%W0-b1@sk{hn{!?pVNMcNd8lItA*(m5_#(G>ynu}l=Ppo zD+;s$ddo^?*V@kcqq7=qcaw{UQ3sCH!$hp3$;DfmDX15`ijmMi_qR(Who3dp*f9L! zC)G91rNb*quvt`S*$c%e$Knw>?71i)#E7?XZvTjcBB3mEThg!}e zKdVqjuRa(B{M9Q<^+z2WX}Ci!&zQ*nXvP}qjh0$?T6*oF6rrIBA5bF-%!T29V$&-P zmQHWWcdEo>%vX#G`MvYYdUV`o8=p*w#4Levg0~qc4Nafrj;iqpwm&{Qt($^tuh&jB z;rDN?2pOn7e|&f8IP7orr|JSSm^`>RfADa|#g{wmT`Ly13uoSAe2R1;9k5ryNQ;c= z;meo-V!nuETiyIxKCjB7AK2Hq4@&d3!ht3*fCO-$H)8L|vg$oPT5U>D$`+th=&aZ|fU&V)*}V)NHpTBoI_*zpNTl zB*(HeCT7n^M0D64B+y>lfR;;dhj84mJfpYGPm&(;$&9w;to!JAg`g7y0U)4brgcJ= zmlqO!J-_vqPx6?u8%LHi&n;!8mRGaSS(L`0`Co%udcP=5aWU`LKmlz&7}wcvIDjI! z!^UX94wW9>!ulqr@EbvG*}z8&qsmRK0fQ8!p1Kx}-z82HvgF@=o$zgmaKejt-KZc;Z1;#?* z!~0dJGlKr$eyAr?AabWKy?47bjiM1%{4Quh1ORE##v{ecYWvXMmF>(@!T&EqKc(D-pckL&Gh+ z7I_U@su2mD-WtYy*-Q@T4=K!#3v87txmwdTO=5H%={o&5%-+edyP-O`5I)e>I)FDn zDIv-j0kz*{awJUQHJ|DYD*z84rW#SnZaO+C|K6?YeZ{^W%I`A&H+IPE6Bqvh3v9O= zomd_H9-Ik5JNa1(Gy0yGTWon-EqGgd*O9`$k6L-l;A|sW zaKARpcz28mr-S{{$yrs(m24tEe(6q^cYbm4e!CbgP;y~A=4(}4`Iit|N2~0rA>6{c zD}&qLhtobAKR+nx_4~59WKIlt zci4AK0;Fz5+JysQVPT_V6a8?9WfBk56oX#wfdLfjoQbv#h_H?j#HA;dItQ5=g0vT} z_;e?OUyn0b;Jm!Tw2;9B2aBRW9cyw&zfSkwOa;F@I zr4B>YZ5(U@CNJ@`nrE(ls99n49~BHp@}>F$LkB{rqewj5d<54_s-lK(ZfV)g?-SE# z<1lXc?#kX!y_)rpy2cci?aq_SVS*j1ydC?!x-3l>rxk;mua5!jf!@dVxbjHofDwU4 z%M#9C*f;T@BX%I+d*KrcyPEtz^KL6D(@qu76AQa#DSDbOe z%_Z!&rY@y+0~a`R3Jbj&F2$`S7+mHp_c(jI~|K0f*VK4H>LMw>x&VY6_2lC zyGxGfqB$_tYi<|VKP%Q7Wrwl$@hlM!z*IZDK?*pk!$AEBN#DOV-9+c@w;C=QV)o<3 zZ!@sMq3N&JEW91>EcK&F;))FtL@XFm$F$RjIWuzqm$nWBhaDZ!0}qT%KF{-W(j|^V_mBJ{@%l3%a!twn+9D5rj%M5a?Jea z>o|t3VJr!;*HbHV!%I&iRzMKEZmbo6k{N)>LgPVGllvd`6Q^m`6HhiZ?^fWD%QPiu z!C;x3ky1ToNSz`U^tuHR<#>E0n|wgTB5j`N)5|OOzLFTY#i_~%FMZYO z;-lZ(bx$UV^Yst@?-RC0$C;!Cb zsdTeMMtzz$K;#BKmj-_3_U=rY7b@R=kiAi)(N+BER}RvDW#Gd{0>wB78p{jC$UBW_`K!V%TSYB1Hk7v#v?0vUmy zj0N~dNSOK$%ct#s>g-tW5VTa$O3RFoj7>M2IW^hhZi%{e%ht|bZjfK{So)(Zb+}A* zW-rp*H|5u@zg#EZs;l_boJop8652?LH@y7}GaDXgeDZWWfBk1nEpu6{VWL20I)292T^@I;?Yr1nH8A!J zhU#TOe!&Mtiv~2V^|x}Z|L84M(yll8D{DGW^Vybl5uSP7D{Z*?hiR>V`@rqXzHIh} zD*9DVz?TOcEnBEq<7~g$NY^!-RXFqn0aNxo7JJihqcZFfU8kyBLZ@fmZ4s5?4==T- zdc^Q$!vB3*FX5jYJG(<;*bMGWJ%ZE$6pESG@O_;$5{l|Ss*YTpN-fTu5AeB_VW-OU z2U}G}YUT)I&VqUhE9iyKZRsBsYc5txXMKT!Q-$xbmeF zpMQUCysm7VwqndySABF_Oa-B2m)j^S@s9_C{axISUl5KI@!YF@jaG<8Mhgp;R}Qb+ zhSV%O`EE6yDDe(?kKoPq$6e_l`|^6==*C*mH7D65t*=p;e|Ao&@b7wU^4GiD#aPo^ zYW7rF6H1Hi;J*t5m`g~k#xhBFt>&Jn+v?URYyQY;=kV9{z z@~Usb?MRYeiT~p_`JOq)ipB{&e-<59zWJB?V%oQe$PoI zMfVx57P9O%iL(h-^OOO4lU~-iDI<=M!sKwK7hrt^1eyv#ib)^)mTS6zb5yY9-`|7> zhb9n!tH}1kx{?mvX9-Pr53+;;f1ds7n(9SPlOsr~y#~9RR23WlQd;2h_LGsiBi}7U zBQTs2sQ`Dh5B}&E#|x9l9mekCdgAd>J-Cj$>gv+HBbh|AU;mT!Du10f<=9#AdXpie zU9z`Wh2~EW?x0-K!aMY0j0vW9+_4C$MQGjS;NRw#fz|( zc$xe0Vco$M7m@<`YsmrYxpmVCJ==(@w{!7Rc(IWXH`iKeGBPNhut4A)!7=k+_Y9wX zQdzAdKFWAqZsmJ=+D`Xx7GgQ^Zj`b>(Ectxoot)Ed_{EMFuMa7v~p5NSVXp^74G>I zjGc!mj7Me}hL}WJmf9wpADT+5xe6Q$c6A5Cc(p$oVv}S zW#>+b#p1m-7D1}VsirRsU>`DKbP}CsuQjJHLSBu8L?Y`+>kU8izZ^&BIW8Uza*-YK zMxy&RxnBSO*m&^&#Ok#i$te0c?ao=5uUiuk!EsaFul<*bsoy~kO3yqOKj&U^4oPpV zbH9(Y4PYL;0~BNNSHqp%gAxNd+f`z{zzxgf?z<39-%bH{c}o`SrU2`X15b* z(w1GAU#qDaUA6xH#|Oa?sVOZd-77q^_5b4Q~zel=^)Xm@E z3&NbPqmhp(6l(XkbjG4xZDE4leqF&AvOnB2bE5+YwSqTzbRMC`Jg@#bX>d(VlDnWf z!7}-~_x=m2NE1TT2IfQjQ?%2^037dHDS1N9+qf-V0}zzoj^IS~8jnzjqWH zm=@6}8e6x0nZD6Ql3z46t>3J*JsTZE93$4GdXGrDD4=OqlT@)%a1kQ|tz*R8mqBkp z1l!c))+T>MCNdg{UgW&&zyOEi?QWhfM1=PMg+xZ*X!!Uw^1eGvZct$3St* z1b5z??@pc5r=jM)3z^O8if8#L`=LQ5R*a_{{HYXpqZm-U2QTLC{^! zvcntTCYUb&2*1Kc?pUk{=%~+0sD$N?!sxmU9+f5KTC&{4SXQ`scVxh1sq{Qc8ub&u@!?mfz z@^_uG+gY9`Ji~1Y<@*)f;a>dY&?w5u177Ywrx7-z8oB5%aMS zCE$ZkxiW?EYcqsscqBC{RIx`tvhs5FFuo4$J5Ugm)WSzk5EL5fa4WnXu~BziF?pub zk}+6)g_-v~URFJ;H9#aSSF2!_vGlE++#!Sr3o2c`NUkCrrb1t)Q!PeF=(UQa4ay_= z%qZ?xKR3YtYju)Pz>2AgsCG`!vKj#hYMni*JOr0kvH!`NHyE1XKT##pMXXF=1$6n6 z6GrHPUZNR6%Ns2AVfkyO;YCxz#>J~3@wWbi zlsYoh`;->GM1q*;uDkY#eR5HKxjK68rmJ624iAIlyL3 za=j=t2-uDs38~9mp30!<-*J)qVtPZk^ZJ)R?VfiZ9CUuSTq{paS6^O51k>vdsJo4N z0m;MgWk8Kmx$%Hk(Wx3ez<@5Gnwz^rz5BM?iBG;P%3bPJtLoM0i2u3YyJcw!IvKr8 z(B$_vgINlVG>~C4p!z+gq^J`C1a&H^<`cog61G&Nc!Omy^Wl?Zl{_ktm!#p5AF9=D zXa9aa%{Cnh7wOi5#JLk^1Eu}%SdAbKVKYi1=;~8ZH6h- z7K?A}YOt?rw9Frd85I1-5^V-e;kC$9v`Qo9puMvhM66;@GL%WwC87G3p2hqA3D3ELPW;B&l9s4yWa!njFTR2jBQuKm&+B(YS1Cn z#xw(f|2cn--({!I&K}T0QS7EC(^v3MqG8%UNp7rHbP<+b0V)lyopOgry$H!nrP$oY zT9a8pKMDojYCQGnb+!C)@~CYV!?j_*u^u^U)3C1<6?*C%lJcO+FdtKzvq4&@&)w)K z7?F;*UfD&&g=_xiRwi@k_}Bv2qh8*&+Zj{<{}cbZ>u#D!L_+=-$IqPcV6-m5R) zzvA=5>u}EN9Dciba^Kf|U5{%bSG?I5zouf$y%}n36(M~icAtGR9_4g{{K6%9QRl5v ziD~%~_T=wyA-Zdj2Ek%6TCRqqom>1}M(5h{M1R4UG~q)#NeIFlrYGzj!tZ!k{MbL$ z4+L<>sii7sFBB)?m~Ai}<#1#`Y91Yyl;u%QCa3)%M7w%qg&zJ|&P zXZ~H_&1Duh?Qr?rObSaNzhUU@5@GKg;{kNAMbRt)6rwG9f8G3h&WL7R#TiEOSGY^! zz)=Fryt0!RZyQxfoD>q~>M6cQkZSBx*-hWa7OtGq)+b&~i*Y0fRbB;aBx?_e0TN&j zbLUtt6O^Co0l4VqGXjIU&q_)xbFjSqg+g;yBSxcsIga5-dpjc0ln_ zz3n8Zb^7aeZE$p1V#0nFXEb|VH{83KYg#i)9Uba50{>cvOppy7tJ&a07^)m~sgsP4 z^ZHceS5s#GF)@Wz76ff5X9_qE84mNWWV!hsxYecl^Nq`Yz0{S0s!n+?tJGx=qCHBi zQm>imN=6x*0dIsQ2BZl$^GUELJFmD9&<~$|$2u^8=Vn=31J}c@P3K>!x9F?WQOdk6 zACU6m{7sE(${9Y!SZ*sv638U4AlI##=f`5+yAuS&|61;gb>nN|N;MmO=MB5pKncm7 z&%fCSvD>w&#)(_k(xWT5xh|Psb!1dHPVD}Ep{Je>RVJcxw8&x|UB>>Xcc1g6d90B0 zI*ORM_9^cM>vF9Znu`Q(0}fp@uc-U#rX>)#Hf_34YBpN7Sd{)L5UuI|_oH9x|V6b16IBAjK zEK4Zq-m?V&DQY_5?aCEJqpORLJ_Q1;%!VpA3=$MyT&#!mg#A7U$@<6CoGh|ZDkczWMA z`Wv;X5x*JLzGcYvmF$5idrm5(!bs+*W0l@h&KX{SV_{8Hr+;x2kW}#%*(8fL+ItkJ z1^bheAyKqn*|DaqLyNn(s{z@)lGO3yHQ&@DjW2fh+5qjqnzh89acsC6=#3zV4F>2U z7fx{rbDNbH=EU&d{}a{mdg0sfTWut?M1R;{9O`~?j=45pdPWB9eE<@lcofYD=MDI3 zg05e`<(~Q&Ch}i!`jF6dX6Ty=b^i~LopKQc=8w-(Mb9L)&WM>j8Ev(U_tF?E%vJMg zD77_>XjB|XZX3kLXB0ao{{?7)$;qYNyU<$nno@Kg1Jo3W=1)p~_x-tC%AXRcYpM^E zFI>vm)=Vb|5x!ie7`l5#dhY@ny9cbY+=n6~$E9h+&?-9(#^Y7%Bi`8fH;^x{^g^<{ zkyLu2H-CuTnfq$|IVa2^U80C{7aN_~JbA`w%k#L2o=_s9yat*cT%1|ER^Sk6=inZ> zSa#Z+%n(`I)20=R#24efrSGH#eXgNPVZ89#$uZ)2}Z#n{rLW7Go^7=ZUyTEQfaYU`@1a zDvI@OM1?g>RT`S6#N`Rr6n_nA2X$$m~F9)IWy8`Y$>W`UyK?m zuoI&li;H1v8cdH^IA?(DXnhIxzbI_B-6~FW#7KFMv@*m1Ie8%?6Hsr3c*D^WoZITBz+!gOOdD1~AgbWQjFf1rtq4<$lRq+Vdtj2o0o1>0}!N@PY=(AKWzLI7I&sSBIy2yVo!5e zuo{+=c=!zN)athOzN94`V4gNKRAzIB}DtnVJv{e594<#Be`+qHX~Ki!w0l3hDdRRMb$2rQ4$ zkP$hH;qB#nHiQf?P&c9?XqY;&3C+rp%`^=d`X;?E5!Yu0Y7)F9;9+q2p7n0Ma0&sz zG0rUPnnH0;{st%+SC&8QC@|v^>`gfvN51ONO=nP0jUD?ovEdMMJekL4F&-rjpcM;C zmf8pVgBdqC`9k(HeUX5vzuo;!L%!KuHYLStLb%nYhwwUZui3q?kV>T?Ex6h3G_1Gv z4R6qYe=0tKv%`KvrE5w`c2nxawnWQ{dsr>}{(glkT1R+~1TU$GlFUAGj~KpV0p9$vG+Je>@+!#7AfGUw80dg|A|6z`WvqfY|NW0t!) z^-?w%k$U(EqX5M5*dF^9t+UWJ>b;+svUVQ1LaK2O$f!}u5vV{3Uq11!iuJsvXR8_k z_L|!W*zx_8>6(ysc>QJlxI;~(O=`QxQPuuyi`gE*;7IqaGiuK&5{*2?$I9{sC55>X zN?DZ=t6$;Ty_V(0!BxOc_NJ((crySHV-J+d}D)psjrj* zWIC^G_4n?OVig;=;C$kZmuY!U%BzkgWhOltdnj>@sS@L3vd6*l=w?d;HDd9$twfjd zN_ul*p)1dcN7IeDVk5h&?;QAn>xq$~B1?@#!PpQ3N|uByqayUD+B+lsnHMjfboMSi zOua|sn_uXydvQ%hxLaDc@%1ji`Tj@^#^O}kxLcEuG2imtWPDB3qwf}chR)ya;v1}g zr?qG`lR%R8sxLxIdSiRxXg?nrF5g!pvO_XRrB9OR-*=k$c6}S+-=JG>W8oh^|0p^z z$-MTv1w#sXQ>ie@ZUc#R<(FQov0!-FDrVIj1~M^WiKD(tHMA1b#XhS@ZHG14Ig(5d z(x9(3zee6W&N^8y7AK)gSl?nz84~(h9%yy?cW+4%pj33{4_@ zf$~EZd*tq}&9q5GzKDpQFU=%_`5nQgFW`uAiCwjdOoHFA5C7c|nz*8?Tk$2V=oa5_ zva{l{(p{Wza!Q$bSO6!pz93g2n6gbmC`m`BsENBsga$9sfzro7BU}3B(m{4!BC3=Z z-Q&HZ)vG01P|KHSG@RF^+MGEblF$>D)`8@XNB9Ioc2jrtqcR- zLPXnECC^0yy%pL9JdeuMJCdPg@ERbaM_V?*TU&L5ko>0hkn`G9G#mJ~c*V`BLb7jF z%-jTFu1lzjAh^P(sGu&tqVj&0IeLjVhV_!-hJ30#Q7P=9o`m9FbqcSxt-oC)u>X+O z0TAnK!=|*mSfCy@Npo)u!}DuOqotOP$f6>9@jTrF#_plw!MCsPtJ7!49jIKMS-p0- zy7#t#otL8XV9j7K&nneFY?L6FdGgEEC#P;@mp$N(gkB+5BFLct8!a(?t2~2HZ}j7q z0NPzJ>ZXwuHaqziTWTkAW%O$5_4oD8()VJEI%C#b8&UOqJFjdN>}NUamLb+ zi~{@$&@Q*dPj2dgM6ybuqw8n15C&+ylyFq{LLGkpYh+{}L~92uv}50eSBsRZ_}gKI zzWlrtzvfcOrX**115as&k7QNP;dxOI zRmB@`+mb=va5aBR)Vnx>498W-p%UvD##c>i6&}3`{(MqM{b5$>=sVrvGJfDrS`Ua~ z(8fPwBRdY;ZwWVNv39pmvZ|K|;Xl7t>u+Aq_|?vFB{|+mO@YD4s@*rPX@Eix=CoaA^d(zcMULULD_|Wt}OaIx4!R zVL;%KC}g)+P5k{fzZPr4tiLyx^o;IaFrPu#EOl6eR5%%2)s4&jx*t)uM2Cornp^DA z^n^&hNFY_)Dnnhdw(0o=`pQj-YsV2|m)B%E!oI`lH3Zk>TNt^ux;#)X!dB&^oez5NGCNt0XP{?;%nqA8 zj}qlERwMJ!X-v;RUTk9#Y%rjcY!L3F$o+@8Vh&H5(b*&&>eV+0p1e=G4Bz}Gm~}lT;jYsmj!& zMx}@OM1`QvsGL~h(D~;?yn|;Q&Z|AhEJ?i`qGFhbxmR%V+{Fq)lBB1$K5l+12F>G1- zMbMOASdP1i=R33n?8perBeRt;C>*&KGg`TzUYxYHX85S6w3_hWYw`KHQH=l|f3|?j z;&m^>)BQ7#N0nv-VswzHHr#Qo8to%lM`XPTc_Rsy9;A#w-$Z|5h<4|h)_r>gOSu#m zZ;X<$&iABAivtJ!S{N*G5YN5dlXm~?gd2%FaNv`i%t|NBudO<_vqIj)iU;@Mupw|4 zQ7n;>VYYx%QLFdf_Wjgtgs)gV?4sZKvRi8p|MP_hD`>0??55B;u|vKhx$k65#4g3= zHcz$6kPOTsd@iY44ZJ9C?813+xltA}Klpx9q(dBjv>+4;6z=k0cCM{CX0W!V@ zIYy!4p}zjj53_V)?57^6nZDk19p*@r4FS28eW}9Mm#KzZMs!%30mnz<=$o63r7c$xrAIIH=Ehl~h*%83a8l0$vV&Rh5 z)|dqm9gp?KATyepk6uT~GgOD`kVx$tQ&Bx~1!yji0>;QWpj_$XcXzFO;;ywiSTu;n zNU&30C%bn?UE5$iWl_*f^LJlQnmW42%wsU^d&_Y4Chxa;{Q4?#voDj$&2f^#+;RTb zrAfl}li`mW z@p%nh51lL{Ypor$2dzsG^J?EWUDCgIs!F-)G6}4&7|Ydd=#~b}71%8o%bLtHwvl=v zy?L_iLg+3&)9O>{CmmF8xVso}Zx#?JIccsdEzP!KM{}(J7!`X;Zyr?1m+j2T`VZ$; ziYt}D?V3wPkj?!^I%^DhEFGY_ZVJLG0zVvD)W1 zzX@;s{H{~?C`0f}&Y5Aqhqm^Iiy{+rNs-J_;(WzOZAeU}+5NDimKg_{WTSrf2v`;> zDSVz`mQy4T$%rHcHt4C{21=!B_jmeCHFmR9`t-y$xbg_GCsz*Ky0Zlhm#l4AF6sjZ zO>SxN_DJ=od_I+BcsO;M0UBOKR0Iszr%|d%iVpt_)n4gU7XEhSTe!!)`g8k$O@L(D zwCqn!*9?GhzI46>Y{%zh&(wh)E3K@YD$7=Sv{^s_&0DHrqsxh9fNh{UI5zUhP4^)g zP*)&Zmpe3x{ft~38_U%TE2n5^I- zW|2_A#`yuA8=XUtEKSQ-*z;Qg)N9S+$_4TCf`CYs)%NOzu-A!xHhQAKT%e^t%+wDZ zEf!eewj>)onb~I^iH-hE>zQrrX{37`Iy^G3L(Z?x>w3T}pFixPq)Rm4O}Q-bwcw5A zYB<*+8#lAbH6z`ed+>XRn*FFzr~X5&fwpR?A&xYh88M;)4@~>iZ*Fd+owte4Ot2kH zOp`J*{MhKrGf~Qa_P}%##=x<&6p^?-!XoG$D=+WxIxYuv+Kth$FyrojOfk86d7)YF zWWbd5K;lXb;zYUt7Ae&1-#EXGWo`AinYwgfInShd-{*>}=p}{PS(gV#rCF2~Wy6=4 zxoQ8X9#}>cmSPSlT*>=j0*bV$$=I%*-~JjNWNs5)9Vd=)?RI{d}DgmD0)f2PvNc7yhc>*U)=m!qa7r8RG)sw-`tqqQw7DyqzE^? z*tG@wCWr@gkXjy_1+<-2rqA$KO-i0i&ZUkPf0oR-scdb?$ zd~d1MFQ^CIn={=EZ(Kg1de4qG(5nZ9`a{;9g?}taxaY{y!N$A(%6l< z-7nKCU=BE?no@_bSiOd>%=Wv8`Wx*Zc@W#M>#rDfIF{1Gd2!XoKu7uv_GC=81qm_| zh92ZUrjgPxhVs2)HU{hxUn5j4fi(;b+LfiL@4IMO3dxp%MpW-H8rcR|#j=HEm=8I) zUpd=9r?Rni@zqpFz=MNX-2eq*?Gyo(h|%@O=$9{&U(csJQQ=ER+G>Z@0TlC6S5443 z!SDU@d7RS{fnVC}W#RJ@<@WC4&ZYV&W4cYBMN4dOhjH>PD#vhiwL7?3FLqhGTOC}@hMqLxJg^0<_|yB^iNWQDo>L%rlt;IA57 zS{`Tf7+a%jbiu^+8~S3x3D#%p#0Cq03aV0iI96ESJInk49tvFI(?^%id`;w6vAe46 zhP1DtqAX2|UzY%gmj)-GqYt9bPic znML8&T$Z|UU{izftVnQqSfFFFVWD>|_gL2^eNOLF;LkfAh)m0vP^Q4wpL}@K*a>9k zElmu!y8-XRKD0OVNFr+v`cR@@oei>xYzg=(kD* zU>H)n{gL1^^4i?TrcRNzXit$|xp0rWB@i(&xI)y8<#$r>$g(S+&JWx2Bd z)Pv({>T7gLE2+cW@Ci9C!w45SN%HfzGWubNK$}eXJgpAgNL+rBYzI-+-M{soU>k9H zHJA3|^UY6^et?eetdd-3GL?lPyD@wl?-W|=2B(voc_Vq zOIIiCw=w4MFT57B;PyMLNIz{pR6w`?jW3pj7czGnJ_OYM!g4O@9!dh0?qw(2XBhly zVL81$wojYdkWE^3K~BgMZfajg^n%0!czuGGLnzc#u9+eI**oI(u1y9`rBj*)Q~m!H zOW4aIohyU68y&qYr9TO(!XHSNr>0p~p|AQ(WR7SX!{WuQd3U#&jX&Y`ldYYy7PgDp)v~V!yu^nmzLJyR&;H7aE|V{6 zJ9iS@JMeMkk2#FgJ?1-QBYzW*dWU$N9R;Vs%M>RSLig@+tyyLRgu81R8DnB>q-_Yb zTDV{}-klTf>V+eyMJYP-YjoAlk=T|*xkE$!gXR1D=EJl0J_vc=n4l=+Pg3-h1>edI z;&b&Lr`2F{tFM<&VAcKJ*gJda9aZ^k=FB#?1N1~iAuSlw8rR6-GMrHOP|lJ?a+zUX zNI4UE{@)h*v1c95XIvZQ=R|n+3G94f%O7X7C zzSmg93EhQ9j#qU;3?1m&N<4VDc!suUEI`>k3LDjr+wA~jw=d8EbJ&>ibdQ+(!dk6fRx5wBV@ zc_}_ZpVeM3R;+V6O<^(CYYiuzeLMf}2AYs1k`58M5)qsyG>X4E*&ZyPd88Vna(%-2 ziprCCgr!pd_)vUA!m=%P<_*d{Qdhbn(uO((vYG{thnm{svaelQA4IRYU;^!C5dWA$ z(7}-eNy$m4;P3D6xIKx}M}_VxBfDQ&Y|TTE`OG}4%k#ckTmtXgW$aFOy1Izk)ixRq zUa0slYib%xM4ARJylsAHEK0NC#5OrIk-p_p$tToMH|aPqE~nIT9jq>$Mpa5W(Pri0 zx!jA!ZmYWA3E|hU-U!7_2C<}nG05=w^@>p&_X}edRWoJTAJN@q!fmQjUaWj@zh=97 zFN1(et}3;adyOzK8EYSNqJ9641qsDyTIx?W)W4X63I@$hQAXSTF|mhbf}v5jAX>>{ z$|loV#;O-l0-{7jP0NPNI<4`nX0=+w@EE_2Tar}i>a%XIU!V>pk2L{dalj8~>_8{K zWxLVN2xrDoK(%J~Sc+}MZ@Vp_8BI$^%PvOhoK1GmI?`)Ywx+bFSkf1%e*4D$qtKG4 z!P4f@SG2!^~*(KYy>=trY%rDz_youjlwAIf9A}w4>Nhg%zn<(Sg ziL(xKmZHG8+A0QP8K4tUQ9g75VTBV`ldSJvB?dDXa~QmFGOX0rh$w8SC(T#5QV^|D zxG(!?y~*Ff+e{eUnJJi3fsnx`#XH{6xTU3VD@QNp3F`9u&+*p_;;(M(R7)AF8)x#d z70QOLP|sKX3Z_L%$qWHuure z2oHn>ty@eqqJ2VgE7tg%tLn!zElM9Y0C}UY9h}(O?=O{7_4>G1hL$0l*n;vIfo1Mt znOrnU>a-uWx`n6tMiCUm`xjD#v;*TVKJb;$F1Zc|Qd%mD;Qp4;sQCK(VRrgCd}F}T zHx83+^|Pym^bpsFBbYU3uxg-PHh)S$APip*%LvRw7LL2FZcYI7ItxiPc92i423!jX zc;Oeswpmtw@U^~HZK%t$@zWI}QC^MAM}&d4=>_B5oKpASAux+Y#$m@FNINOa&CODg ziU_J-oVpKRvM|!Gbx-)ml%G-Kw@s%kn#Cg<07iHsJYNN+hYFV{b{K{!)Nqlu`<8mK zuPPY_2{iD&+}35m6~I&Ix@e+V!}n(ai-ND_OvB%b z!WeHkuJvX&50A$t2}8BkT(7Fx`Gb3KLoX_#^7j$o+C#m#SJ}%0GQ#Kf^JU+^hHAPF zd|+c0d@1>}Z3NwaN^lHAu6c8Y7p_6MP-{vt%_~tzs3GNt^D+<__EcK)Hm`#ryHh68 z79!)>!`dPX(QrH*$ifgX53|pM&byXe!#=>!2aQMKj8}wmQZKQr&aD z73ruKEq?`h4#aRuMgi@e$0j3vN3?!RalJReaCJv<=p0yD#_iaJtH3 z1XlcJ&JbkBZ4QcQYSCE4J@T~dp=9s#X}jBi>WmKXS0*VJ9BpysfLcW zV(5jKWx@$jc9}>V9fort^58P z%Q%n8oh=k1nZbcf4$!TuKV6f93@)^B1TG^>}tJJ%0LBh{?DY2?l zc*1w$nWGiFk^1OOR)52%3R&TBvU8$lpWS--$HCqpk{@DB4EsE^c3G-L-#cCnfUqhVRYCspdblZ*D3%*aK?boc?l0 zNY#{NhiLZc&R=O;eaS1x?0C_&jn@&>ce!h3V{r1CE)aMlmKCFXAg()L?AiOb-)|Pa z7?58b7W(H-g0|3+riTOkACt@aF`7%}zk%qYb25Gn^9<-&V7u#*q)OKt6lgNpjq>Io zt@m1DOYLY!OF~~@(Z2VbHqX9eRKZx7k4JVJ$n^f@NuLeDLMSsL0M=e)*Z%!mLx2jQ z)wOzMQf@xwsll)xX>{o$GY~N7p4h`pB-)N6IThZsWHn4m|IOh zB0lZURk!f_7^a)Y7q=T0CBg?6cmAutYw_1vOl%O%zr3<&_U6_azcSoWW?Rx~a|UUo zKgD*H_(o)Hp7t&C@!88iHBz6Jrwq@;e;H};Pji}WJobT^0lc&E4N>~BpgzcVV+-fk zg`*pl2(7m{geE~rGGbKE2q4Wtnx&U_r~6tf((jGO_8959)#b<-TR&vg^C@(5<;bMh z8bflV1+%lPrbIIZ>T#X zT&8#$!hN_em#>-gs~e+n$G^*9AVa5+`x!^dRc3;tci-u_{qhMtYX}Gy#+Kzhw31bv z59tTE{E|$beS1F9_Y>5pkm4WU_jbS%dW^@n@T$!POMu_^l~4=))^Uyiy9s-EsebOZ zeW{iAL}AN1eA1vB_wL0*#{MSPJgW3{Ys|c4jDKF?W?1Df7>L1u`S_d6=_B=y1RQH`$p94v7&&lK-=BJk{plaXhoBNNhv}fkq=N z;G|mUv}@`Wc5zMN`=K0$pGqdUUG)4`3a zeQgBWcp{eGOR5zHq&)s4s{Jp!NZ&$f7~Z(v={ z;lake_%roV-73#l)movFx|%~^AzUcch%hn9vP=z(Q*&FgFm0XjREM=lkKfkuw@{$P0x76jUs3j?QkyXu+h}PY z@Uh(@zbqaSuAzKD?3Do}{SZ~rRF)tB^o#Em&t0%pw9BX>>(!2A8L0*Xz#iY;%Y6Qm zUi)aSUi4^;Kd|E$&oxBiM!SDQ>fx4?p-wosnqWDxHyX+n8#?@^Qt?mj3oYx30i;e7 zkpM348!Nf^7r!<;*n*PrM*>pMI+^K zk&u^(C>x&NfqeQqOdON+&W@oY9mp9GN~N}uKj~R)Q*%dgvW<5IdCsY_r@0HKJtFX6 ziuHq)ld8yF#2cG~%VQW(hbHO-&hB%-}JfK-U5T$NK_B0=MEW zTCfWk4wvO0c+CdBPCnDcbB4%co}%Pd6LHXm#n&brOzaT{w{|eYZH5zI>eRcjB&v_6 zubmfS^>p}`n3?VqfRKV5B~)39|J=(5XXMX3HxMutIb4Qv22)9WHjV?Bh{g%h!Ysfp ztE=xXeMEfHiWCAh7_`~4=j{~ZNguvdFS@+fe@XwGkM`bPwecR5YJu~~ihMC@8F`=$ zRm)4K7{w;gu;lS9C0z84h>Fx!L<2o!;*!BnCbqUs&?Q{=Mryb!m))*mY$zxubIbwS zOyv8W0BJC@s)l$@MhYRyu`(gl5W5>^Ugx;-+;)o+c(wQS4O3GP|6BfoP4Cc#YG05i z?T~6&RN)95t$}TH*BFV40YHpNtq6DQVTMdCcl&2*n@niny&9AK*@d0>* z{p;NCu>N+o%L59W1BeS#(QBB|3PN)x4ee9w!BTtgHIc8nSctTMHnW&tS%dQVIs`*# zR%VF8=L1ua`%x##^J`D)^~YgUV_QXyJ*E%Zx)p0P0MoQ17C8`0Ms4%4;|3Tw>R?X5 z$r`S%1(eNK9tlGAX-pJp7nbl2e)L7-nWv)AT>`m?yK82KCvl1pX>dvI-?>7D`-H-r!5U#C_dcQ z`2KEGpV;tjB~WxW%H&U0ya2?kymSusGY~jFNmsqxd4-6gnEVzuMW*Zoiw*5U^d!u( zXh$>=RjS^WX?#rig*(YqOuf^h@w!RD0}6I)me>}Apn1(xfQ}@J$guDTv#}#d$Jy&taJ{*?1v(hKte+`t!~Ui5x&svk;GfCp zK?+=u5&F+l8zN?5H1?}=3Nlh`cjkQ_Kew?df(7oa&XXAIvw~7w95#chtm6Ab)nUnC zT}Cm)JI`b8wk8$zk1_XtZq(U^!o8zCuPnspxNaz!O29?AA{`p_3}Eu`oVnZ6ql|n1 zpzpT|^Ec#&o_ufdt6ZwmZHZpFZB)tgV>bH)a5v%Ml{CqQ_>=s%42G0_(%9g@6x+S; z!mN?D&5UB?q!dOQP4XMJN?C+>E2~v9L{+sLZQUP|HWNO|?EdMUVpYANWW_jf$V|#+`+o{ZbKazf2*iO%R zJL3D%_A%(>EG;o^axhA%^H`y6Bk|h|S<2h7-zG%M;8HT0(HtvddE1jE!SKniZ@}_0 zr#*x#SnSmN)A|S$&WPwK7C+nqxZEFD=5)(wESrvo65AuE2vAD{{Kd|$4>R_znd$snzFfS8 z_~{{q>j}B-$h_K8*ysvXtD5?h6J=Klw%~~CHU9$`Mx{XF(hDPjwG>ALy72qwk zfX#Ze4%qZaaocd23qgxWpua0T5L~A@$WNC71f6$r+FrXIf+^gf`uErB-J$t{kA~ho zy1R)M3vXrOX9m>u>_;Wq_jR>k%wxAVeJ9&N5zQ8}nVCj>Xpj80`UOq|JceK4$E;VT@nuDY+o779O7i*D zc!05AfR&|ig+$pRALrx|QD+X30daV7;B!9^$dcoM3SAk-#-nBDai)4b9F{uag$L$V z^G@t2rf^Z;udhlcJ6Hf1d#I6uiRA@b$;WB47quV6{gV5~ls9yYweBi;7-{wQSIoBG zdLzzek_M;FN30E)LniJ+3w?QxMvI8VutllIBnz>M;Tistl1AGcwZhZk?Ye!MAQDt9 z2pZ0k^e=bha!~I{zW*{U{c*Tpcdgp&r(&03^v!96-PB*-D@u|LWd5-}uW@uxeG&am z7M*LQB-XU?yVuOuP5 z>Qu$WluvPh&1dLnbON?m5rHSs|JJ`KP!Z-gQn?))y3FM|@E(K4zD0FYk~66R_z)UZ zvM-biX0KSK6n9?U0Y~|AI+Uq&E{f7$DHJyed|BF+B65QcE#u7*wtv3RwLV-8yx|YI zdT}?E|4K$~IL1az#zAHc@?o}MfxlmXGMtFmP`c4VOlTwY=wA(b9f+SUz`3>uPbL`# zO#VK-T$KdMlEH-ThQE9J$A53W27l7GkRW;{$$ zistVI(N;zvt9c)yelmWD?n&o&n&@421@r1CHQo48g1HgL9z;KT`MK+xyi*I$Faq|Y z^Zo0)s$u%si4AXVAQNp~*0}1+rg44<_%-lEL=5|er99%hS ze#DVI+TI=H*EO60HC8M-aNG;avCj#yRn_a#?~+zii@~tMo!9;1U)WC>=|#dYoD1Sk zFJ#!%WqeX&wk9W{`n94Ce=M%^(w|0LsG#A>y!k~(R@vW8)V^M&h%<4f+;1(;g2mT4 z#|IaC6FMyC8@}jneE&}Ot|nR9j?k%)Ek%mPLh`QMJeunz3hJ*k?wMTHAQ43G($j*Q z`-~V#it~gFIXd#nC}PY#uS_aV{5Q&*x9a2P$b;^!)BGUto%i>5S0Bl;LaF*+BE^?v7*&&n}!)}oa6{CoP{7dtlA3i0!8(^bVoUyEYL*T+OFmSW)gPLFEuAe}l z#(aliZpc-M?>fJxwcey(x*pHK(Aq@%^Ks1s)&Zx zMa}iq>m8r=T;fEi!4e@)9Ig*Y-i|S3b+-~a-Hy8urhwBU=c;oqdswH;lZax?TV}8~ zH=-dU8Ib{~Mex*eM~=6P#INqJ*?OWqw*G@+ByYCzS%%;1>O)Zy%BVWjr22^kBRGHW zj>7!u@MwXnCM{67BK57|C+nRXjz8-!Nhxzw#}UEC%iIZ2z0D@&ivV4T!;T6iv@x@t z_27JSEqF5fSF7_Bz+frN*lNeRy!#r(=xAa`;JmO|4Xv4X+R3nHrmv@PoswX>qd`pp zu3vY31$QvUcHX48S4L`>=xqsmKHOt}J7r%M94;&%&Es5Bdr&6wcPJ)UPRW&Cb}(i` zv`aZz?J`44q=)qyJ+<_h=>p(EPk?8LxHE4$n z-qVn1I$8lOhhP5eqU~mBy#E95uV$L)OZn zfs^ukrUL(coza91s#(PC2K#gR-Cus)^Mx)aKdPtpT^lr5Z)kKIi2|CUl*>r1pkB{A z2dRA`f1O<^7K>eYu^r(3h@cQ@!^nlJS(K(dZ`A2S%K2EA%4cjLT9ZFlKj+*$FV5RP zeq7g)_A~2G7~H!!ClAYf zNw}F+Lr5sGpavm_wf$Y0+OMT!^z$E6Ci_1oFK_97c~b7A-0TO3w-INMqf5b(GTs9m z2>^_^;cw@m+HRKq?rw_6ZfJEt=kl;uWL=o^Iu^Uwx~!TNIk6)&LA=v66K$5VV*SSZ z5asjEq^bTq?vTM(?qt(?tg824$fZ}Q#dDGzhk1P3yY~<(|CqGb2aO$5ZHW=fp|y#d z%Ebac(i5r?JHGC8|3J9&a#tunW*6V+*!j?Rst;&+cF)wBqetexIDq?c*9UVikXL(UN(+dVlNZ6geL z_}~H)L+L)l_=x>C`oC1Wuh1S@)(n{kx6b|S*8K=}`rG6jNd6sv`}pEkpcM5Ig0#>! zFbO-M@yK}N@4zGUkNOP};nmfbUbVS;_5ih)_Lq8qehxl|jZm&|?M&i;`Y;|sv#h+k ziG3w3{VVrf(U54kbzXdJ$tl;ji!JiKTTwCo${5C?;fVTZYiwV&vZUcyjt2$UCoFQ} zz|n76&%FJn&(M@39SYI5V5jJ4Mnn&3$3PE&$c>I_cDkt-BiH5PS{~(@2ymnx%z=8I zB}f0p+TzI+FgO}ZBxl)4F1q3gqP-LSBQ2l>6y1YN~ip%*6gaFG9%DaijE$EW`HOk{gk`ulCK?HH?oyDaU%} zZOaFEfYaBhWMuhQ&ZU5vz;f5Hry1wCF=08; z7NzPrrTDj~o#dx-I&DZ${`1#r`lav%Bwh z&TH~o971?`Gh>8txj1DkdGuKyA~WBx({Fg&BDoGt+w*s0ijpuP;KYX9+vf6K*f8o z8ms2j(uu>xAld|$wzL*W7&y6R9G>c_WN_vU=6$@Xguh>6@w+d^%E@;Jw1J$Z-^Dr< zr-ubO3*Z*vu356lwR1rZTF_sqvUtCaskiFii*+4!_2tjQ#C#ebxmlUv%+NIerNW{H z2*f;bK6KL2u?s`F^=qmbVkqCY~i$bGpf=STdF6n1#K|fX`J7y zI!+`StW@QkFQA&5$hP)yyw9CHZt55@WiYeMN?Dh=Zc@38$eISzOqKiQt+%B|KSME` z6yxMWk@uGGt{^)KGDkLb4Mf5`?iy%pZ4v~kk$fF+$TT0ww3X!l4zdLq8CW4;LW!^W zUv^c!oy%Xy(Lq?IDU@D`abq+=Y{YCkI>aGt2TN}e5x~&V$Ec8?kx8}~A)5UE5%*S2 zZM_fJE_H#HLUFC&K?=q7UqKQ|a7&Qlgb*aSw@{=&palXg+F}7h@Zf>s#fw953GVK{ zH+yE!?3w)q-lKJtlN_vPJ-1wUZ_s}f_o=_=d^kg49g-CJM=flaHO0m>!oTFH@`5;elTBd3^LfgdU+EceH-Iq1dc z{$Xa92JyeG^h>riE)q+f8G_#|yOdu_Pes2vi1G!k#c@*j3sn5fZcyYMS(-fY=x1NI z90{i~#o?%)+?9S-h(z2sxx?HX%bPEr=HJiG9TZ8cS5qsojicROz*iP^b|@WH#q7}y zziJ%+>F-mNZrEIX&rGgJ*_`Gfn$2;CQRlJAy3lcgtG&`)#P*xY-LO!fSI^(}2jJEh zlskGF?E);9`MLaEww##Fxe(G1gDmIRf_x0>s~RiX4VE<;@jKZGIt@5sln}GWYiC*k z?ZC;1auW%uVhKSW<0NUh_Cb%VB<*~rwrS~tP7p)mNv-sy)I7fdFaA{3#`_z<4Zb^i zy)TElVqUp!t`*Rt{%+j{dX-H&C7a8yoDA8;|Qhu|=Vor2_y6m*?s$JE5sGi)k8-o^T*PU}(8h6?C zXKZX&-Szo?`r)q9BuXr> zt+T4zvtk*D?}eG1*8fM*4(7u4+TCCh z;*)j}MEjt}PHdeP3nEX}w)<1pt|e;Fx5^oFY>kFER zp(_kpyA1Y8xGg!4i}WI;gt3JfZmd^C(-L#E*+J`)w6eh0_L(CSxnCpbk&#sB#LS9Y z=Bbo^T|&qFY-9s(sP8T8_!>*v7m$xOfOr+QZ9<`_pXBGxkx(%2T5az()Uyt}`Z1QJ zQiQE4*Go@>I?TFuYQFj%lWGw&#}wU6BSA{J{M5oz{@ND&mvdazJ|X-p;g<{~W97McDtCZHjD?4KUYx zpWAdNXL9q$V$%xe0V3x|90bRl3H9)5bGpGU`L|tb!;Rvv{FT47FOBG>ripF5tSOc6@;q`cx_hm1~ar>3(3qQb*C6r#wj?>cLY0aKryzf$Pne{$= zSQeeVqX|WJswW>$6v35!2)@wGO9rd2-avIqv#^)crgGAj`5@7-#%=?es06dJ3L<#2 z3TDH+jQ*OcMrP^C0&2_x-%lU#?9^4WZS%7Fj{O>p)HWE(orHRLW(1f;>BH5|PVNso z`JXq~l6v{75Dh~>GEV9$`TgEz^B1|Ba~4dwfmY4fPkK*0OU_Oxd@S9xy>LfWQtCMv z!+>*a1>O+fIuG#c+$U=A#3MDFro`du*hy32niSzjf1Fq!Zc)(IQJax}>lR1dQ{YQ8 zUdUZQ*ct5C@O1VZf|w^BZ(*Nz$vM=Rw`Pc2tyjQSkCz*jb3$ofEp30G?{w1pNJ@~N z{Qig4geFEsI(L-7K6H3dC%KKYViB;7fdg(4}c#M~^tDRMA zVJO%>r_QC`WXxy!--A_E-=89+kxN6(ZEa0-9bt)h6o2aC>J=bisHs}H59=rZ_cm%c zY2fvo?gazS)R-;)G>84UnD)w|`DU0q8gD9LFw|Y{D_2auX?&J*vs|7u^en?K6?+qo zVAWfm2@vxc-$w^(fK;~B-G}iZP0uciT6iBrUAFqghYfny;AUf<&bGyV;&B5L4wHk+ zw0^`j?tug0DpTwDYrPt1qr{-eU}4JtxPnz>Tw{NHsy-m~OhTMNnbWphMM~ffZH^(cZd@vD%ut3{WUyhaUk4O2 zmf*|FuBkjwfeql7d`}5Jdu=8?)Bh<@pCWKwVf^fWltlf`$1{>V1kg!B@{$25v=Bd0 z=bYu`)azIb=NO@SBq;whyJo#hGIc4A%zWjU)J)o(%&c`RzsWV7F~EeCb$zS>3KcYX zA+L&V%4=HSAw_1)hRS@WtX;Tqp}-@tLR0-OUvFW=6m3x${evC^k%YEyrjum})f8pk(|Vu2U(??s&59_aY(|mMf2SJ+=}h{9&C6l!S~Dt&H-r<0INIR%-g%EHJ1Ga_JX3@q11Q3_B_rlT^xE(>T0;I(If&OipQ0(- zBlyO_QZIgNjq&!znT)z1x{$wjI+bRu;8mLoCxN}7^P;%=-0Pl~?l4~KEK@#!h)r+{ zwog51raH$DXX2EP(@U+e1P_9A2A9QdP#!x|twFsUub<{?atz0a?5Oxx>pi~p#30q@ zMe5x4(eaP<3x4}86(K2}b9G~ad5zag*)p^$i-(mBru>fuu4hOSvTvXrS*~NS1XXmd z80E0F`uVI5`cd=PqoM&El3TE}GrS{OxvbpZ`(&m@=e>YrqVP#nnv~9LV_r?E$E*Wg z(bS8rJz#E1)$3ubjG53|2Qmo34kP@5{ZY{J3tVPFCwCweig{7yhAIj<^Ah56Lp+1d zn*5oa*hRCx_kA4m3+TruP3iXH&f0-kxUa)wso0mHNBJOnr2r9mGwx|!&9fJq=&Rw) zE)T^?2ZY%xtd5XQR72E*uv(<2*_K9nQXH9Vlo6@OyE?;}?gxG9TId~SqAEnl-s{f8 zmOg(ua^$)US#>FW7G%5D?V4B1j-?P}50ICHLTY2B1YLGVWZDYF%}}-Ogw6$<90Jbn zK2Ncac%Pt(WE#5(X^U$Mp1)uhKxZS-%kt3tU^`BCLeaEeJnG#ZrWKnvW+X;=f9YWH zpP<#gc+?B?D+F1(OiWW@o85>rbI^=CCI&RDHlHK0pS$LklhG=L*pbnS!on;dci+PC zvAZ5j2%|kweYST#s%X|nRw5#rSLudStSRjx!)m8H4%Aa@89V%Wo z($N0x5ss_MCTobfLo-hEu-Dh7|F4rI(T)^(M@7#(p<@M)9d<9-!}50>qqut`42Ndo z0{b5@963m37)#jo9{v!FL`w@)=j-YXnO(sxqH{fN&nL{jq_hKd;43b{tQr~k8~QB7bLXrcpRm^ zT}G=EZD%eeMs7?akIi_}?#~3HnA)TZoy0`-IiVIMeFM4p6KJ$k`ge48jE4l})mzn? zLBzDz(-rTe*4t+>ztY;fCiaB;-k@BaDcL_cA)>E}&@7^Qdrv2g=3c@0ri;&f_}Z}B zIeXpMG7UBkA|3|HWf5DBKRIB7J81+B<#kPA%{}?~CN&^YiRrQP$AsF0OhpJtP-@pB@Z~4w(&7RSSN|KOqAvd>Y$q(zCh~&!1-2B`jSQj?H zQ$uaF`D2b*W*UpWD1Q4tib+w$ZH^T9Ca6;#jbPCpD;!zMnt|AR^1p(1B(?cQ%e^z6 zEIObZR5`pAJ5hK;Nb$eE+qCx z@A?_HA$5`|f_cHl(M%NJFa8{i#=%9xCvsGB?(Yv`)cLY22oQHynZljp-ytF1f{#vC zTJ^Ee!p!9q3Bli$G4mPV^4f-BD2G#)D}TeVn!@skuqxV)c{@@4VaFxaI8RC8x-8HV zuKSxj`|04m_>L+)<1s@90iq$tYdY*)08N=6kVu@c zapuD8y3v8Hg{d~b+HUz6CY5Odjs_2(9DtWRpk%jyC&;I&6#Q^gH+gx4jBc~Kuj&X^ zxR6%ui{T`9o>inKrgQj?g!P3#fCW5wd3Hj&pXTd0T^P0!HI$Z5a=L3tU~N=IbzCo% zE33$(KdSt;Q9-!BkCA0loN=_%KrxqI^z5Y(C>37Cf8p33z;3YmG0UzVHX(y^KQT)N z^_r}Q0=2;6S7G}1va`a~pT?$|8;?3Uq#ztp=j{rR=8sK?qiBp$3Y%*{-eJRtwQSh0 z&1ndeky!h5a|expi+sJ6*9RBx??x+w^ffJAp9l{#lK5^^wlMnN+tAeg4BP0BNO0>t>2}3c1tj#@SUgk#b9a=I zlIc+2l_6hApFJswkXhNwd#Pt@gYVuW&vU*IiDp4oMInhw1w3vAVEs7o+!t7>-oy zhDng_5&5%q7To~twyf@(`*rgZ8NWjD|2>rc`}coOiocaK;{X1eQ*ei4`P_EWjxr5< zIE)i>i)40DUfG~2y`W}MvpW$a|S(r~le^VE%b(-IcTYZ2mxhq*zA3RSylpbN9;h z=1Ue2Omo#Dl_AqFGZm9*P)uYxe}M~XP)kEdeq{UHtxhVyt7I_s31jr}qIl+b&P#hSKQS-_~^HOa;+-wn~XGoQm)w=?5Ukj2mk!CAWWQCKiaKy*1@HePS- zT@n;XAlX&;qzoV@j1J5Ev!oW-Ivylg|MZj+?eV`o`@W8fVUYt%IPmNH`AHLB$DZ_m znSIa3Va_HTw9sp`AS#K0gx|F}jT*=~&Md#|`;w9xZ-N)wY(Q#4JWBO)dT$*(H6Pnr zXe5l)+Z4?>Tt6S-l44Bq4IMtOija^=-R?mF2ra0#FO~+8M0(4T_6T~w{DxtUicY>k zc7Z-t7}kaK5FmT@Gdf0RuET8x`3##S6{cw`0F9zrpai=Ilz@Xv1}HLy?Vn4~xjxSIib0}EhF?kqV-&<) zQ8~tvRaglQ6sSv8F8h(Hd9~ZT!ae;x{h`+nJbp?+)bdTK%*ubY;W8%?TXbsS9gjD=pzv|Aj>7epIAS*{6u^Oj%M$C0Ht z)-{L`F4r+Pn*vIJBXX= zVd%k~oJ_UoUXmu?1yNAZ=Ib@L{l~e%t7-aveGFk~5_DZQcIK}5n!cl2R%38JhBZ;L zAWp02G(9^`10k=d9g=qjLF`{y(MVV46(sy3+50kn@qanb3LmN3rEcl;*DQKj5Y>&~ z)Z06*Ezg_p-5fnB87~*%&Wveui$mTJ!V6;VXJ_#Zw~eW%%q$lhqU+oQ^oEx*)3;w~ zo5LjB{bQfFQqvf;$WB4q4);G4X(u!kn~VL~P1hGdj-f5`N1KP+xDK`R1@#)bwUxF-i}F} zism+--G9x*U@>p^uDz8zhYQd<4%!KcRbk=t%vC6khc2z_D)R5-PZ+^E(QCVX`i9IG zZI@^1BJ0&bff~xa2KkHxQlQmjz0!XaxBsIUEgIn7t7>a3!%ABr=6dfkEmG|6vKvCB zY^9d+L@e$f8Z0_vG{Z|dmP(m=6qqIsWVP}k!X~3Y%iOi*=!+JbaEI&lWy9sm4#{%} z>4%T3zv%oy(Xe}Q>(4@x=M90XEQ)ry@*$r+#M(U5hGO-J;c-~wzpygI?;5Teojv&3!@iHk% z>5rxGJ28o3uly8T znf#SaxX{GWY+6?@xPJF8b9i(*VC7+EoA>9|k>`pwvPOb~M!uTnJ^?zZELJxB2}@NEsNdf?PEr59F-7oe|qMre|K_!@=nU4sPVB+w66E&fef~nu0WkWw$FYV zcwkB%WNC5>k~KK-H5!j>@^2I#(m+`m3TjkFR&E+yW~&KtKm6abP%dhnLe*}Urmh?j zyLZAm+1XQRY(EDQa)Qo$OHOm{*=BGjFdAeJoA|H)$d@@7YuD)0#$#=D3*)AdT?9ps zpo$%*E-C+snU^CK8FyESNfG|Q0Owz;EXvtc#Kh8ku45JtboPCSac;8TU-a7>pFVs2 z9an3qdolX2YY&|5a#$jiNuRs?K>y>IDiYvBQ-3QD}2HPzb!TJshe9bh5 z2{~VXo#~%AiRXBd$2OKGhl<4;ACN&1jgwKvrz2i|bLL^lUUCU8baxxyFv>EcAZ51K z;~IzJ9!T8VKJ2Fk?vuNkvuluO?|7~KOe@D(f{R`Fe-wK8;k~aROb$O*|9x4Lt^UBn ztQ{Phu_J19K5|v>leEJNG;JRqw&Eh4o-n#jJqI^X-+&Ngg^|%yfKlw0>Fjt;80OwR z_yjP|Y(GiNHTEw=nNX{esST61VB(WhC%i3fQAJp*P2K16Y5D%ia~^vfEzGPP&98e* z8~yw~pT(YpZDHn8eAul!%x#@OV8@1}TU6ktq0FhGMaDvbUh1MbJ!onWe2~4!%TrktJAU0yGnni0UbS6>CbkHK2wK}B=o!G*%RN#&+xUc7AhYa8Y-97eQ1C5 zv$GSCH{*m5Ym zj+MmxHUoi3x#{Y17ZRSH#CP0Z3Xv|=f~d1}t8YuyU%^GZfKV2^vu|k&IJ$TTB7T+Q zD~eA>0=|qL39^c%K)cS*vYxKAyr$ZUQNK_?j(* zF76bive@7a&c9t2%A|WNXJoU+AjPsTYC@gdwm&bR)6P@zB_EV#M}F{&z#wj@Nw;%? zEHct9%1|q5^C#nwYD;RDsEKjY+4B2jhaCULr@?(CQSR%Y^yeRKRLB&NJ?Y5KwZD_` z^0(Tmlo-U-Y)kfHThgp_2ii|_O1~EFH z!KA{AAacy$ZIFo(mqyH!EjfS2@AFUNoAq%a$r>_R`Fw9A2LCyvRQB?99wLNnE)1T2 z_0G*@py#(ceJtEF(mp>jyyOC9>1aH>PmXf6z567Gpo5 z0(keP-mY+U^&aY)!#heP>eKm3Sg`lx|C}@KzLAV5cktYO{CEQ;IrV~lU**%2(yL+t zTI~)CQ6^*a)6$UQ@LB`s^yGNZ`2qOh=$Ja&-h5&CEA**VG}6CIGbIVoms)Kn4NsbP zKairLHzw{m#7JUZmqn-T2$N57-tc-LD{#L52`ssxS$k_s#0nk}W26=jQPWvq{uT{n z8{I1t`|{k^5c7^Pv`ys@Fj!bz_?xc|KO=SGY?DKR*B#3Xj&b#gIeZUuO;{{D1M)r? zZP7)E0KZeS*d^fG3~Y!r+^?|QPOX|zxAAcP;@iogPT#>s=czh(^RcX5 z>Bk;q#}Gon6LcYjE}TvwLnRDnrX35dB={ywCr(T{t2{;yq%is%2!s$@=z-YOD@z|@ z3oXs~HuGB--+49WXThE$T)0epvFEW_;>>ky){wF=ux^bzXOrlpy#x1GHyt5*Rf*x- z3B)`bMECd4)r<1S>zuqyLNScNNM#d?K&Znxtja5Zeaor#*qqL1U4o7XpGFY*E9?BQ zdRqasXO3FbnpyTA_YL?r!L#joo@{s>&KVO}P@giz0~)l_0ZTHU(MHWB0*OCH!SC4jR9G zAq*vO3X}_`4E+6M5!+?AR7BCa|4E79^s5xWa^(3pKa{svZa{9|jOZ1A@o-0N`xqpb ziiaEX>3GF7TmUx*NnbqU*W8C+);X+x)%@}1b{(1LLHU2*ozYbhm;gp4T0y+tcQd>{U!Rwc0gO{}sl)`XYlKA>>;Mu#7LP*DN=el&+8Css`&C{o zk%Bz81jL8xbR+%8taPPj1&nBxD|%YU6ac!OMv3lFRs-TJe_c3rupz%+Wf{zwo-TgJ zp1hzQU<$+wn+8ns{QwEG4H6Z1BuHEQ$~oC~+7kT$e5VTSF$~sM$v5lm3>?ns?X6){ zG-Fs+s3d+@;e2*rX??*(!Pi_TSMpb&1383(@l`RPGrR<}lML}2!(L>xogZYXBP*;| z^wg->${;JGaofhB)YZ^wnAEHJ062~9h=VzIjfsAD{(XvRDXKDK7_qj`wBV04MipvH zna+h?2xIstoZzc(MJdF8eNo?}&=Rxv^TG zi9TQxo1Kc0)62iRCZnXV7ifAzYLKSSew`uOdNOd3p67RMc82)Mj6Emw5gZU&wI`X% zu+>$V{ybf>VwjGjgeBCz!LDa2C3Y@V_s~4YjDBzGQOXbw*d914y>?}p!GqJm2nh_K(y~~F(M)TB z_HFzDhI2tNi8xm7MBLpR`>wBE1sidKiqDnh&pZK08o}CST<-Kei7gn}h}I$lLO!oH&?wVS3Jy>oKaW>B9dmrR>;h@ubSB z(vp(kdx;ZiuhE0<_Kk!1uLmc^PKk22mE_}J@B=2oo5|vhpMVElg+A8`r!!!4!!U4} zg#muzg3RYSjHwjOH%KFckTeq|mm&>XbgM4Uv-}o#;>+{p-tN^p?UVK%y=$G3*gN`H zF}O$=5bv&Uxd@L7da~|P)ujXNRtO{BHE|lM&=eR^?RLvcYl9OpUz+B!wnQI4bvdbl zXtfN*jp-}e$?(kX2wd-Bs|OA1{W}*n7VJ91b?b{6C3Xzrw&+^_11o`}5?F>~1!|-Jns{X$@d-zpJ2i)lf1tpVg(hR@lu<7}hGP85gtf8A5V$ z3s6X8HSX5W0TsL|ulRVVz}EO8U;hHqiNx9YMvBj6Twb^O(`>J4FZ)(^85$~#j$Lpn z}ZCT5B116^~2h%38D*}aJ0gZ)%;p}ztPA|Wo zBf?w&FL)L0T>r9rba*;mF($snB0p~)Hdy%kKMGO$`~vBRVx};iz~dJV=U|PCQUB36 z*OnyX+Tm>$gea?V^@r^-yz4RIMbEAOC|+&2h1z_O9Q{hedJXw-i{SC@5hzp$J7UBR)A% zt-`~D1eoOLFl;~HS+0_F$mCd-g3#0L*%JkJp}Bqsw%Yb7IAFN-D@f+L*G7EhTSko`ptN7`Z!Ty01yhVXJ74pY(IP8U znslDRSsJKuv)ar0On=V^r9gBT{1O-`t;`5!S|3Wy;eg24-b@363O4fwV#h=Ut~XQ8 z6Q35YD|sTm9k^>gIdV{&>Lwzoc49qiG{MU+R+Q(M|e=wJhniXgh zj{pzv`Y!x>RoC=dd-k)UvyVd+h$i;3r91Ju@jdq(O<#saa z(3R!8KX2t!nQvxFX91RqmaHsGtR(ibG-+vE#I=Kjr07WM_tQ`EAtq%hgh~?e7u{~Xu-AZ@z7E}RC`@E5J!My1L>)AJWx$iM$%*hvkq-LTXQ|H z926@16hFE!e*-16BE|S9I-TYd{fK-++MTTfDrH$1e9INBv*xwAhVMCFdkEWd^22aA z_@Es>q_YaP=oQYSci&QO6ki{Sa)mk@WNl`#=B34;cX!%kb35^bx9pFnYvAfHgpckT ztk9ZBZ9nI7@h|C7rGkbPv(mR3JFzA+3DCCp+82zP^*?TGp={3{^y8QFh(cvZHHOSP zj?u#w_n!P2fa>k^%J&-bS{Y*P#sUMufPg&iuCLC1yK{xwNeb!>SFwxl(Y zZa?qWh!p+DF>>HNwIkYLMy~C-*J}U1C+BJSJk_IaNdLZso!-Ei_K2v9};lGMpA^Jgd8eThTCtse{q z7*YZn(xl)4`nbU>_sYaN+Vn~d&qhO!@&V`Es(_I~|NM)$fuoI&u5Us*bEo)<4}ZGk51sc{Ewqn>8bCfv89^Swg8VBDN>jIz#)$n^!fL1*&+kVI z)(F^o_nFec$aBY1j1jFmAv#KruBH9`_h#EQfd0yRhOOVB_M_$EwVYoBA!_ncSsxRl zjn+cklXJ5f?PYQ6n*N}~@J;*Phu6U5B>gpYUpSSVp_qmCbOGUO<{QlHbIcdZuzGjgm zQJ_x4*PegCQM}ll|2$}OIjY51?NOaU!{pU(Su#WrUv<`FY!(~lyz}ZA%PwG29Q);knbm9KR|)Jl)Rrj!2(dpJET}Biu0T zrO+IhGqz%-eWcxDR{{fs?BPA(Z_Q{PfKbaZH1z zLH*6~XZLq5>(I#a4^mWfYpTh0`KTmuD-Pcb!_3K~?fK2seHh>5gr0{rd9!F*!+PM7 z)W!)?qIGM8t8y2An(Mjq{yz%0Zx&X9?jLbpf@6$Zdqu7=mEzK4##jO~Wb=Z$Me%0k zz~;ixCI5Vs0wXwt|x%K0mdaK0vCL(=?_q8D8eQ4VXvs*LK?6 zUt29+?Jk40z#b$izk;v2WZu%4xq}{wuXDr3ZCPN?>~V?%N?zHvxKjj~U^Vlon+U@3 zG%)olPpY?X^Bh&ROv_4&xCZ#SK9{kJZbK{!i!F-jwX$Hxpm>%symB7(k2p2kO3Xt!nqwU0Vmyh_R#_{ z?Gb>S(RfHx1KTGH_oZ)ERORaOdE?ubKK`lc7gkyUfQgn}yGQd`-E5WgPYn{_`o#vc zYhECafJi6e{!sa5v5(J@ca$`?K}_3gP;VZby0}!JBAP#>-gOO!!CErykN=}s7q7cd z$ij!J4zY?(4(<2)x{JxDPY_Tq@F zFl=y-nO(3(C&X-=KW*o$z$4k85Zh0{0)n3T& z(3q}n7{L3oXS@2Gosx@xE(b9xiug;PGR{rE>rkS>*9qNHE;lgJD(KF4IH?1MfCj{e zr`-x1yoA04k4`fM&sgK=1hxsMf|cI2U13ijG3%uO^K>KC{bZyu)sf63X)y{T?5N05@f*0d-KV_*oRBJjaTgVcc~u}7Ih>ykRPgV{nNMRjI|P?%pvxUKQ% z3j$E$D~`Ts#EuSq?l-F5*VV1AX{K3r&2Zh%kd!&u${|0v09D}^N(!GLDk<*HN&Ar~ z#9bSyggSQEf^dAlBKONbNNVV2P*ar)wd#3!2xwBO=&cix1M^kpPFM_==Ov{f)AwOBi)^&31* zU?MLWLftn+J-GX6>&_8lS;k0 zvhyIMaz>1zIH`Ev{BdvZ{D!UfJzlRPM;(OZ_F2J5j{G~A>9pmGNr$b8KPUXIVKA_$ z{`1gQV}KvTz|R=j%vlG8LUhc}wR_-Z^x^M57p0iRO1sh*9{+%4_j&S8*cdyp)yQ3T z&>%GinB?kZPUWIodxyHp^T0wNlj=?Cvgn%G*T*BfIv7aV0A{P8Pr|T{56COyRV3Fk zpNT;=bqOJ}cTXdR6C-Eqa?FIEAn>~?yYQc#_E`sG5nJ`9`CS%JzT9#u2tY`7Te?%f z^Fa886@uAvkpv4I-$;eDyF9WHd2n!dT3F-E zm2c&s$-V=*6xhMU*3|sEb*#Bfa$c22B|C29;;5A7_=qh%gSsRbpP88?^S9P~Tr4WW z`FC{$Ri`VQFPK71Ek={?8)ipp!s^rCdp~Tro|;RWcJ8skmQM6Sc%yqPJ|JBDq+|Af1kZhF^y zQ=bncYbKh-tfJVq<~Lt*xU`-2K3QJfA&wS43iRJ z)pqIQPu|i+p0}#xQMu+W3GI<9#ndGdZ0jFg$EO;Apy9nIMv+pU6;Nz9M3^W7H@pWv z=L4^+l3j|sSK^oIodIc&dHz_Nz{8?Q&h+mtl*E>_Ny|o?rGq-IhA?oLu$zBhVS;HE z$v3=dCuZiJkFdMVkiQ3}{df9K^VokB6S7$Sk`^%zE3AdIEo%z0gy@!ZIpzg;`)%L2$a_RgY1 zgR0=hgDV_b#nYsKIo%MXPxc2L*iNM{Bnorta~5SuDMKkh}&J(g-(@!=X%FyUboYjRal=9;b%HF}`eTL0Qp z8}Hx_Xz%1#zJwTf1_}(l@_)4VaI%$rsHHQU#?rZ(g$iWK`Ek;Xx5i9TII{kAw*LO( z7Ky^R)Z1k2!Hy1&&4!yG4~~Z^{)p6el7;l;$jo95K9HwQ=BNB|tyA2(+^xVI`HK*T zK5rid5|r_~B}V6glILEF-fs{C!cUba%kG-NrV=d%@qfEJgP)^prdQtDRMp-cX`xPF zMficAP81?iLtF-=r}5X8I+f&i1+8nlPJT#=^DLIbftG;RU%-H*2D`F<3lBh2b?3Uq z&ylUWFSyhyP$f4L1h-pci-wFeSY_Vb4TGzhUjsQ|YNGiX1OewLz1^F&{Bg7c3S<-bP+mCtQ&b!iCpYS-a!oV@(E$ ztjuApA4^3~cr~A@xaVfq@^va#)*>;C0SITmUxVU4MW#+3&kWcnv&!NNBU}$Y0J}ba zzNp8hrcJRKFhxTmkKH{yNp9x0(}}$}4|w7g$;we!cMF1x&`=qR7Cu|r@~%xBQQ1<2 z27PVs6FcN1Rfb77z2KfVxXsM!`p2l+!|ar~W11BSsZYp}%9$QoPMHi~P3+CxDlnX= zvi8+~mWKtJ8uJNR@rxJ!W;K=*+DzPPFg4o2%VWcMJC_92Cd?F1e!Nk%auG{?pLJy& z&h0=*4o$9-yzGUXY+Y84VDr_>1~@c);|ewxyV(&MT$ANQuzUdq0N#s2(fvp9mA6!0 zqwe;%Tke0R8(}Z)wXaKfweJ3X`KbN!N!Jt(%gZ(7JE7cA?8R_Ceb8OkNa&C@I~kDZ zY8W{$(#OcDt6G-~Odm*^qT=s7Wo7h?oaq5!A`x{>DTGVT3@bYlR>YzcxcfP3rxCMK-|jCdaD=Eg$rBc&Cbb4%h9`2 zjdo*$|3?A-H^+gTxD(H?IhK}+V8vc^*1FWj`)(L$YGh6iCCYy{nIzBV%1m5g7$KR? zaSAWrrYx^+Aum5X02jJOG{w5nv){?7bGvO##T<37kII`(&v>E^CK6@(GytyYfKu?0 zfXskZ-R!p@=YaIxF$s&eYrm^23?7nFzQZ?VS&c5Z3Fs$ zbp5({-|D-OUmqAa0pK;z#@FQyjb)XQ$m!WTDX-w)CuVfw4S%b2_D-7_p@xYXatBBN zQL0K1$4Am3eTmj%+M>I4n;Xa}{u@-Oa@QPwqd809yVjPS08CjV5_`IwzNSnN$~rNq zA{?nDe;?fzz>!%=Fv-mLDghiLoOn^0Ia45t5M#y4!TnF{e7^K;mqQ=Z6BL!uJ9f5Q&_ve1}CaGUuUzmtkJv z(gHe3sXlo`Ss1*`crloexHAcy0c62&vb@fI!-=k8?L+4c0l_&xb>p;-kN9;_YPjSa)%XcZ zj-BK}nY5Gyl9LY%W=`vGH}^9H{>UA%Kv}09H`niCNpY*(G@AL)$oR$?8~@VcZGxXS z+t|}^oJzwz8l6O*FEu4YIs9+5@-c)rDrO_YL%_GLsL6Vp?RN9{J+IGflx-!gqn!dB zLamlBb@W%cA9}3M9HaadpI@1eV<#v6CCTfb`S@h@ZYCRU*?8o&JdU-3KYdww7weLD zd>sCH8Tz*MU6IM1ds4)f)?1;XlT)I^*w! zWP6S2_W$2};RYZ^B;1XSLGEGVus~kD{AN7nD<;RmBOadq@UfGo@n%0M+VFwh4-?0d zvr0<(km8^)JhK2=mKUf#9gXl}LEF(at`IMjv|${~yW|MaBHWKvdHCp#-oaw9Nef34 zS=>ih=S}4Wk-Gk61Jd$oOi?EKd*k(v2{}4OzOws$*-Acu*{iXKMe);qkgxs4Vkb3z zc^_SVur`xV_D&HPH~|L|zCIasH5kE~>CG+PKGgOFx$hi}Hlh*FP#zZ%>VlpIKPz_I zlT|qzzFEp)w8w-5LlaQg8XC#pA;;=zopXz1yQGkiYhiAC=Crgny`Hr_S-&N00$*e| zmNht^3;~Jrb5k`$Lfwq>^jyp80tb()Y70rANf2SINXtYpovS|LGSRaMcPW$mO;P*h z=OvKVUXl+<`kDrnqF1bbN29uWF2BwUXUM#W^=X&V0j_1bKsSVZ$}r(LR6adlKfCy0 z7MMJDA~gu!7C{rvvM;!M?a$f6!=A$|qjbf*gRkXf?PWm`hbbCa0#4RbrT2%@24~c; z=)YR(hF_tg30D07T>lIv)d|{i&(a%Q6u6$`^c+^$w;ylUn2;BQZFr@JigIF$;e~gp zuVzq#hFEQ68{fo?hYiz=@Z|VTd-{_%cT3%>`ffO0iq0?nTtfTaPuYwVDORK4XLvg^ z6wE~!Z`x?~4&Ujg)F5UMpkh62FS1571j>%-456cFQ0-c{j!EhH{dp@+J56ROUTx(3 zetoS3HyM=ty;k(RojZF>YGg!!;k1-iGg;w!j@R?~d^{ih=ugkiCZ6QKazZJu1%m~}v8|5n0Md)jGW7Sr4%YqgRBAqVw8S9kz-|9kP+2sosfhIImlCONur|(|%SWI3dL^_jy3} ztXZHjnmFWT*Hb%MLAW}&#$4J>zIGWC6UpIYK~sGLkjWiH`!+U)5$rszyQ8;;x$|6r zpW;S+h|-S-t-~-1$>Sy>ruF?`W1(dRvDJD_JCK&X?yvc*8v%O1h=NYu<9HsA>^Nr4 z_>Oe(r=qlf<1wG#--T`{XXHFxK>@d(0d0TtBp5VTZ2kTY?6r>cQPcKHvP9Ibo0+Sl z8hPrBP`ax%YRqsI2iSTqZ&u~Kbgy*38k?Kh&SwEOW#0Z$(1uidP1RvoIiALMBPaQ} z#_MBIyTU3{VWOjSUGQt^kW%DxfKjzIRK)S@s_i$&j(KX{MjiGJ>16k^dMPnyqTm4= z?kf^g6PK$&_zoJatW+k5&M7H5=*YDXclND8=rodJ=uhjUdkTo~?33mA%D&CkvHc54 z$ly;S6PfObj_DV)j1>a(F2DQQO**H`X0i{Qz88OOvnWugjvxx@`t@Z`%`TMf!U5lp z^-HJ^zTj!C)Pk;V*Y|#JUjqBgw|Bo(kjZbT|!h}b| z-nS^@J#+Cy(OgK=c9`?I*WAG)G}POAN)kmj0&p>b02*u-2>1j!HVnT$G$nUaN;ql~Qp0^5)|o%DQ3ijz;woaut_T4~6=bG{UuEHizW&OPS0K7> zte+_TNJYJ*jwp>>*Ob*9!Y$S|HZ`Ry8YwE;HT&4sR3~M{uq^2Eq@_ALD^Lx=F&vL* zex#}zXl1I2VBmP0{Fc;bo>Nl|V;!T!^Qy0-pIR&>g>AU5ph?Ju((2mM%#dc10JxFw z0X$1PL{JJL7>|WQAc0oR=#vz^n6hTNcaQ#nzLh%c%`@akN1_+*a`z&GE}i z7afFYss)BE?38h?bAX)=UC2j&T&HT`;+1Q5<@a#G3`5SSlx^FOD5Y)rPSuO=pRw*I zS<%JRd|~K??Qo?a2M3n`>5$g-Wa88yqyRaQ*fXk)U##!)@Ck6Xsu|vzh8ivMm6zZ^Ta$B>p8miNtVVg_QX8MM~gz zZ?@jrf?vh(Sv{U-H^?y$xpV(MVX4C)^}|VChu%s>skA11aK}% z!WqpsMPz=X;NPlt&*d|_I%o-Qf_8^O)OYlXx40W7R6JPjz#5|Ny&^N>`1N|o5&hrQ z(-|tjTe7;V>NdIIsqk7Chi7C*74m0M(nbxjZ?CZFqbu9D1Uu3*qYU|;x$T{nFML1d zG0;_z9O!YavMOP-#MR^etp@0iV! zKn^SCOfUXM-ubt}V3BOR#)IH)h8fN;xgN?N+)H(!#CDmv!0>a$V--@+wbgu#zsgP^ znl#DXQnxySxHBZnRbNN65q0x=GOy-SWJCFvi`z&aCCSpl@)^o{8d~T*6TO@HyH*hd zW?0~vrkef=mXFgaeo*{UX@Ty6?$@dMUN1no7HCvQo_U!h214*)BUc0)H$CGS4ChNPdSYpZmKFZigiBaF?pb)i zCTH*J(zGqO+xS+!&d+46D#ECtk0a1Ll<>C}znpbmy3TieN$s1_>(u^_D$0yO1yRh= z6r;>z?;!p)C$e>hqYRO?a}-k11u7ZZS7bWTO4L4_1_hj=@_D)Ie!M9$YQMPK%JC?) zJFhuUU?h$b`O-*+qN+e!N~Lkg{zUf)vk(_EHr=+Gj)xUbX8}_C(Ur>xA%acfMtb_r z_MKzs9<;7va33kZji*KF#?ZQpj4R!={P;{&sUDZ0I(I02>c|!%EG;d-!_p!;mIRzOL5kHY@}BIsaN&ePNM8d-I%|BTrsLMk}1b)$|ce9_M&BeLW)R2 zD;ro7RA`*h&L#^M-xTPwo1Ezy4@kj3TUA7A{_GMGmmf21stvUGp5Lsu`O&|%W8<2c ziu}w#M9gT_TxznX6Eb(JD0TADw_Iem-vKJw>=!({xN;96gei3s4)XkXs- z;2M1LieG*SCGo*=;(2b)wyfF}D=)fO#OJYi@)lWs3r%c?>zMY%)HWCwO~`H>F3h}M zywxENG+m#b0c;$6D|rKAj1^*e%o=`U(Q^1nGr~A{GUd8&>U&`zPSob0!c)6SzW|>; zqCa>9|2!fS(|b1LOED7@FD4JY8WvoFg>9~WCafR2h0aziMYWM2gyxyI*Hb;7mbvKZ zCJd$;Ra|8I{3PNC|7|lD<;!UxS2MHqc@lK901E^~#fhV3I2N#Vy6Q&1*E+#-#+2SN zBY*F;$%|-v@=68tg@w(B_>C7W!PG)~hOsxD`>cKDU8AIgq)w0|h7tqH{lVc&A}c@k z98g^qWQXxnC$`#-=R7=SSu2ZzYr-m@MvU~M7!x%b9K3UgxO*mj@V(!ND$k63O(*&V zLS=($y!RR~`G(<>S66NfN`FIjeNXkDz45~v__E_f*$EyQLR4Tm)o#ILQ3J!Ai7x@dK%Mg43Fi&(}8NX3Dq*%w{R4tDS3Q|&;z z&^iB?$e-TH6PoO8$s#E@#8we+$;ke(=x{O3?{*iC!a}*5ZzNcD&x#o$j;BoZL)*>h zevtAkM79;=3WU<{nsmER)J_preBQR5Fz!JSi9N_y$eGrTD4)}%f}+=2iQjFJ@U z9Bm$6p*$&0trg(kEUB!WChWVD;V$ zx;zjLd(O3{Sq>~P8k#vFf3PK~&&XsrUH@xwq8?&6gy)j@*##sH{Dad;XK=(Zt3L!u zWJu*kpnnJH?}!eWwF$?r9fM^EyE96JCj_uT>Kn@h7e={uzdre#s|j}VNdD}p(lHKA zpyoL>@a>KR$xwpD}(T=+7BtcPIFc>oXISW)klx0dmAB589KA02RZyHhJHS6>-feU); zrM}`a*5z@8kDEQ1+dfF_V%NbbB>q(_(j6>aZQ4olf3ef^!C&Ji-23QNY$)05V`33+ z$kjzczPDZJuzA9JG*hWb$aD=R7USAcDS5ZMbPNG{Bt-f(TNm&MbhkU(GRt^)33_;< z5!0yGJl@ujac}wH7%7)h_Fr{=2Z~cq)K=q4nyUJ(uXUcmSbe$);AHZ1l`wy6QY^EY zqmp9Lq&+E*qbfv!AkN|?R|Rh3!)Jzn9+Bps=daPms-ybu-PKC) zNX2-tdTRCTh|vqt8{D8p_YG9p3B^{3EzpL9NXB)t;H*+n$&MG>UV&9Z;|E^I3E7F2 zzTpsg3RSfas>wrepFYyoKa4h-ixuHiR7Q@CM0otzwrKDyLvP8`KI5I0-+9ZlBL)uE zIhT&gLar{SC2r#9Q;Z|eh9dpOxB6u;g6I?!7B{_`Tc*cbxTbh6g}~q5+-k2ts%5V@ z?sjKLELo<#FwPL$qP3->QAcx*yuU51@z7uoX=|0@>051j)~U8esK@lOuWn_hmLd}y zJ@_l}n(e@s4f46RH1~bsG(@)X*RJPa@Aq17u$MLQAH*x=cb_`n`I4?QUKQi|`I`{m zqoL)5lqPPIHH~gJv#YSpT-D z8p}Y~#_cq}iUq6fM~^BS<}tm3so#P8AnLRF-j+1k09_7dX`l9e|B1DVKOK!D=4D=Q zX{SsquMysFv6p<0Az8EDS`HT5u7@vKWz+t^C5pBR}D&N~RY4IAL!IZg_PFJt z8Yktc&R3%e#7#Srj5h7B>rns8X-X{|4+otyNEHAJZMBE9~34v)GjV_07Hnl{kn zm7X;Z7oG*r(rE-d{zPUb^{iFO>JLubyvq&eh!am^SPI2H;3ZkSzwNGudQf*oCA8r~ ztW1xDae(kBPv>ZNIgNyIb*+cIxpl?!&&#c&At%LPyg;!J9-(aKKg;UvL8=NL1C6~4 zGzn|5AU}vRFxVYvSI96YNMz>^rPf}O2h$R_g^J7yl&H@%?PkQPVl~Jg6@1J@7uWtC`%$Ca3f!!CXWj3+&-shTzT( zCuFkfwM=j8(}MB*+sbd7ypGx1=`y!jt>W7(s^tmKf2$|slsP{~-)Uhe_Te$Xzh;RW zO&pF|Du_raJq!wy0to3;l&5WP;x#m~^@q+?NQO6v73Au>Y||!_ZjY&!&BRGj?@{6J z_4fgsJk85v2FvoCqO?)Ji4n71eu^-Og-7b;qt0Y(_L>`X?6j2eZ(FYbd9 z_1f#NX=yaA ze^A0`D-QLz)O}{DUf-!Do)c(`Gb9AO+ z;2+;6TD~FXPGU7tb#oB4kpupTR$vLchzA88C{^83lPg?d*d!N98IwXa4G9lp6ADyK znryq1vY3S5p(-h1hZbc=WsOJ&5%YrMRw!q3XSh+1#*N2In;T;%Pm;x$?aSRNN~!g* zd_Vl2BlOx|l3e`(X%zm>5aa;>z?P`rIEbpUeaA(t$unQLNZORje|<}($o2=W2Bjj2 zR%p%r z&47MWc>CeQNbozgXGiXjGsas;a}}4{WuNP?x4?1kTQ~953HyKNqpx zYW<^LXE_%P;(CaSh>Y@a@ZcYSEydvWMi!%M@!RPzS*hmt`&gJr&>cH-Tc4UbzeI7N zBzY6=Rnl;F$`N8O|4nJOx6Zs-Oyxx+1VU~1(CVT|FGd4l9iEZ+v-G+>y6}at@;DEs zbZvm~J@XwKm)0b}1wJr`^zq!Wy8dYPVKTYM>4(5zYoEL)y}oHDwFgCZvakMXS}_UH&1G@Q z8oprWofAe2E#kgL?HOBrxTusF#84y7IRr1-DL@;Nypj6Fw&fcGqBf&8!wuaRjJo$e z={Zw9`pCjLUi{6Ws|Hv(S?4fad0KhkaFXG~l80fTysTNW<4odUK>tS4aOnT%eDun{ zq5tBOe{Pv( zkkEwLmE_|PG-*kYRg)$OW1WTLntuOtRv%`1q4$r<)SUbDIlGc*$3H5@11vzs#H>?U zx@5;DB)EgcR#Q;+g&LHZVDHu6clC{N>0_Vw-pNX`Vn;-H$z8Z7H(J`L#(uRn8=mu7O z4W)^vgq}V~Jrl@2grbG{+w1HzJuAu*x4HUD{a=3hlA5QT&TFKkCgN5X0FP~gd#)A8 z<(pIx8@6wbDk}y4X%v|Gwm#wXd1H}_yh)mtVaZj!mDy0uEUS9>38fLzwWub(d$uo! z6!H58z-qWuo6I zYuRf=<-x)3;JV5Kf9pI@kH$7}d*N%Jt82-$QQi@|w@}w(Qh>o+iPWz|B>+oyg*FC~ zWRz{3$-6Q6cMFGa*kVqZ=g}S~=%ysB^&4#$!r{{s+g+wBk^yrLS!@|!rLA^PMY86F z@;q&~ftB++QJG!Wi+h__s0Q;n?%rV4b4S?Mq!zr2Cf4TiSDoNEBMtS011(LeNq@nO zMcpXY`2c8qv_8Yc>aW>ifsX#4c1$r=pXYfR5~q;0vSIBjgo3QXmIU2LY(Gs>Ku^`cmbWhbY~ZLU8;!QbdzWS-p)e75U8)yif%7i#)@ z_hJ$^9?ch*eZ@>Ma;NiIhnh~ypPX&4O^F`kUxUQiG2L|DG)k8-Y}4V{el^8men%11 zD9IS+H4@_%9P(C3(P&*PlnFOEAhTw--96jDRT{|lkE+=?kbP~)Xmyq`@X;~$cU9CY zobv>JN9aZ(UEiRv%yGGxhM^JI3X9c@XSON@tPT=EA)?Q&V49n zzpSBZq&O>_Uhe0pC48}m9UA<_sBmH zo`*3U`)S4aw*s%tWSOS{hDUN73xFP|_d658IE0GLWQb}&lb5og;*q||^L67xwq3~E z|10n8|4u*qpZ#>lQ<2+`fJ!{xtSFB=3TeVZddN=TFf38Q z)Ff9qwP5wRHIEHu)m%&hYo8cEk$@_o8ICanLA-|$nCphAx~-?Jg@$C4aHo>1@0AY8 z?_<}Zira@1K|~%{VJ;MUlneE(k;TuTH0&k@K;7b=HCw7`sk#<%8gHBJ;?^BfkaZ0+ zI4rkRRMjn(ThCb5zjv)}-J~F3?5;(mt0Mt*n~wrK@%XwX^=`evDzBN|Rb1oIGQJGl z#i>Y+TySdb|CG)i-`1!+64z@Zp`P{kaLyPx^Q8VOEiUG34%X*LA~o>PliJq)W0#xM z+alu9^2vzFcT%FiH4O8d|4;Brn}xkUNGiW5(hCZfBC*$W@z1Da9=nlZS&lJjh4Ej? zD7aD079W(QVXZtE5}KM|DkRy_r`-32sVI%7_iM4xs}Mffrp@&N6$vNXX8fMh${@ScYrL_&CNO>0VPFhtLn>Ad4|z+XGE*4bOj6p=2%w3pd2s zI}g>c<_^f(xQD!=iLxE2B50Ync1cr~5tNF@uow9R70%oacy#R4s-S(@+WI=&y92`r zkzTkHz|xUUmlk*!qe=6f(y5~dxA@=}Mb%`>i1o{5OF2%+x7Rlkf8 z1Eoj-Qjb*myCLSVoOw#KS<*tlgi9hGgEpUi4oY-#G_z$L3D8SXFY}kWNW&L1t;68& zPF4R=#hhyj&>I~KwQ3P~(Yep!%N!hcSmy0?u}sN_NPyC&wUhkE$-nO<1jh=#g@D}J zhz@e^cgP7p>uPv#Z*vQC`Ph)A4oN3XY~GzH=S#z{Jp#)Hj*MLbGmO<}1uTnYX&lh) zHK0`m0cCtN3Q&?321|%&BZf5d1Ka#LE1c z4d&9_J9ZzBgXRcbBURGlScxmqj$-|*g<$LOghquE8xt5b_p^h%>uAS?MS*APlba{c zPHpEXk9Z>RbN2Y(My5aR{deq`?7mML8W^hPjz8s}ObEUKbTpun6vjl-0~5}Pr!kNg zmgEG|9TBEo6~%vLcpJ55lVc)3M0fFFLEAsnC%*(pl0V$6o%d#X>Cx$3UFPu@OpzI|(D zk6oohLNo9adZ)-RSj*$dLH9uUN)aUvD3qBTH&HGy*Mzv4%{>naAVgR?ruLO$(i^K$rIG`uc zkZ1QS_E8m|kIdwBgVCOo=S9$eaEt$Sz3NlxQX!mQs(r-&qsnelS>JwT<;&HT*F?Gq zf?eFQoem)1|L=2fw{qbv&nM3IsGL@&Mkc_;eedSA;Giuu33P@><{V^a?Rg&sAFz-c}@1s zHkcXly{5h;Pe^oN!J5J-)VA^MOLkem>>BG6l;lH&G6Ww~)6I3}ymniAEe?O+i!akJ zJ!RZEBBg(x(|F>9d3~6sX06vWBJvDYUHv_9C-#>8V++rhEW1OWpVdIw+u9cjEC9WB zP<TFyY`BE<>D%I7e)vX3*W-0l@P+(i z+aAwF$Hyz&n=f~Ng)Y~B)pc{&Xyoh)@)R;rJn*Zt?_0Xvj3~x!O*aYvXGcvcVYP31 zbk%lypSIHr*nUfk__*6<`;&I~dz`#5R;y?t%3_IkBe7^w(}*-My@m#(12;m6sTdxX zgMeae<%G1;R}-8GW#tT4arAN7AJ6I}v&rWt>jO5K zT&``h4pN*(jWAiz=Ie3bN)~7&uEZ}tVP<9F6TT&QlPw2Yp)WM5ySzVXB!Zi+LK4cR zlhpv*?2490@B*QV2m8B(VwoXcXTUj-{$QqFJRMT+;HepT6s6X6pt_Ha6fmi+Z_>eTnO}B!+Z_n@iwG1)SVXK4E79R(Q&b3Oa3|$ zZD1+#4?QS#UUw0va-Vi%Sqes$U#?3u_+BcH^;yov!ju#wbmyGE6d6fC%^k_zCbN9HyENJH_+<&}Qmt!CQ@VF-*l|&a?ZH&}0}X0mO-tpyF)5s01V#8Tk?E9uy$NeNzB!ovPtuOYEu`- zFsy0NnyAHyq&f(gWq>?Ee{u>`b>oPmZgDw5_RgY14};|jq$D!vk9~~+5m8I^$Xi7Hgbz*j`eAtpS@;Y`r|zTekDIp-QF0`;?Y+ z!0fF%vw}PsUvUerEa7qgs5IF>OIGS(_j+vfG$yNubIn9LU#-U(9~-|B-nL0k&EG2U zQ|GgqpRNfi@kTyxBX%=c|Mu5Nbg`uw1wyXD8$_OxPR=wffpBQMb@f+OX%_W0$MmX5 zP$$sRkNmJzoV73)e_TcS(Z8KtW>C?@wmLEwxGDMpZdFAzWb^_u#RBScuP}9(^dJ+7 zJz|~);Q@~5aoj8M(5C`+OM;?D9stj2o1aSoPxrNwkHL5Q1?A=uBNR7=LfX*j66nVa zYdg^w&KjZs#eR&+9;BF+X-b@qfBcx7*{7p9f3Z^>F`*G4tL3K&A3O1a^JYFo)Dly3 zmmd&vrcQLm^17DoetE&}$y#fAXt!y#zwAzY+c(R>Zm&DB3Zt*U03MdXW@lm|RGz0h zhuwERD$9>-hCKMv_OhS$$DFaa!8<+namd~9Un^lD0(K3#Qp=TK(bof62n&Z;)|zjO z9>IWF+pt04l44Jk+qd1D*Z)x!gj+d4r=@wW4@D*{3q$^3F=S4jGNReVFWdM3C1=dKKLb>b!xyGFEbuM}+sug*J32@S`JaMS3+(3oNO4cPL z+f%x);sX0*C&yc1lSF`m=+`Rxx7iP>PZkjdu4d+q9_uy0Zpuh*utRZ$Rk0*i`(kA$pMeDLqjLUAsoueL6EeMxPWlV~2so}&ssuPUKM;DH}b zxWZ=4A8#~a0S|e8v3%l0%vcq|ZT#*a?bG78o zJ(|qFZ-qXYdU3CVpz(*OGeDCyd!649%*_i@tU258uo2_y?LBc%YOZ0dwo75RA)ZpL zzym%toJ}5CZd{V_OQNRXu22^QO@^X&DR+ZFE%33|k@&@+glBxjEb2ZSuiYUr;u)=g zM5+ydDEfKektWCGpdNkXa&}JgQp!rZ9%N%LuX-1c!2rx^mvLJg6RX~;XIe5l)xWDg zQj6ODx3+s@D(TkGu@j{@ARR&fq2~p9n>&LnD5M=`^02}Lm(|pj9TQ=|2SHCz`jyi# zY!B+O{pT9>>6;)8NW{o1Q*WVnwO3uJ2eN7RG~ZKtJS_Xz)Lsmfi#IE~mIl6kT>8LO z*h5uFhD`4O*J&yV`&H`(oJDdMHZ0{Ic;8uIMh%|Uuf1ZCe=e4YuF8i>5V766ttK8{ z=81#aaW2c#J7JNR0_RVv_v&S_#xJvt@9$0;@l7}#aQ7|mRmi_iY4}Ig>E^VU1KtLd z!5ZVAEC??=ZVy0m_mHK8TI}`s&oq_*t3ibWc{3Dxz|?75mU}J}S`^(*8Tz=)n^Kd! z!vtCu$C3wJG9wb@gY5e>4V5@$A`B`fj*P9YS=~OUql>VJP9G9D9E>CKe zg2MDl4Lwv2Ck{mGQws)6_X5}LE=9=?|3qXP)P8&{Oap zSwfTY$Hxnl@X)GO03A-M!3f-lm-wqM21IxUBOL0VyGmH`Nbl1=W#0ngz{w>+V)cAUpxIK<4 z$^Xb6yM>-6U%SZmQ17V9`}s;UFFX^sc$C7o@YX;P4{B;TZ>hXo_OhG9BG~<7V=VL6i+u|9O>no}a{gC-tF@qk zVn-UwxT&;@&SxOfW`1dprEuZaXHZ0{tpM;93rCr20#cihAI%S^YC7sRE*K?4&U7-)AjJ^jshh?c|3;)vW<-CoCSAYBX@=RiL+|*Mx+}n#M z#of5PzD2e^6S|M+3P`d@HD=H)_BI#5ZmbEN%@kb=@y<}zHVAKh+`VMGv7}jW@NDUe zj!wKw)wVd^+K)2XR(NytkLs5H(;c(Va7C=qwE`qf0hiGTt5b30m9DB8tdmvJl4V6O zk7?4j*^LykjZ3PVPQKY%wc4bC($hmJXiz^=x9}fT1I%g2GcZ2R*;F;obr=(RIwQ`V zF3WCzaMom#teYoPmp2#%$T@tse|nO&{{O5f_#X`E=>84=zo{A52>xzl{Z~7m;=z@- zjL4j`@oJ^cnl+B}n&6d{nC|;vL%LIqyo)K#Z^$(XWu{J>bNpIRsrAt)D_r#GE0Ujt zI4i2(#1Qj4eIRPFvGXDl1Xqy{(kh4NgELznVKr4+A^6`-9+K?iN@Kv$zOrk?69GiuZ>TrlvP7gWY9v!ZP}y z9^!fcI!Dqb8NWD<<1|I6%A{q!4Tn^vzsZ*v(~FQMQF7QHt2;$HH4al<`j)q zK;xoT7`E0dczCA^(Y5{#WTV4^oug3rLaWMc++(s#4cC?}S(q&O*YlkOYsK+ww*y#} zJF(j3ZaUjw)-o&7<3ZKliV8vgQ4wvN|Xp6r;eTZ)TJ<4`9MDj4TE2^FJ$1_c4x$=U$Eu zrd^%$=#}Fr&u7eV4Wx*Wmtm<=ua{5Ggylz?Q?mSFE%PlVS0%ien z2*Eg~w*~gD@t#2AHG)KHzbuxG)syaFWZq0cqP+CI!TjguAhzjlM{x}=b{vwbVEb&w(2Jh?cb8++TfFMxM={J6j0wsPt9H$Lus^n)NaOJ868ubj#~{`ZKpK>mKQYf#&Rl-tWC|T zBkk9Zvmx2jO@{%eNPnYYXriy{p**Wt< zIQ1j+T;es72#hd~2~=+}Qohj!t!_Oalq&+id}8y>xF&!@<|va=46YGrhXA_=3_iD}N`#%QkwFLHxf3ySu}4E9~e|Ow0HqMKw7ZYE{Hpo+75G z7$u7|Jq4rh^lMC&m>k;{9@PBWqo1t=`XCw$FZ5Ki;|Hvh6bT zF}0Q!30=@r$d8q6ZhP$ap47N{4>@%{U(zas7P_W| zqQ@Bv*A$-*+up```4=6oQu@s)t8+jP6cZh3y6=*}KMF&>Z9;&+{Tq9bH(NtPD52t; z-id!wOlfLdHw8Yk)vS}BFNfwgSc<_^^!r?y1gy0l-=(kcwUJnm&U_h-flLBn0Y6lm%xsp*7? z?tx_7qMSzh1Sh&>m(Z(8>I~MbJr7&wpU}jN=0aV-+Z*|AhJ>@|4zB^#-LB3Jy1mN) zp{3Yg|AS5Ge`4SI-+pqXa4o{{KfOo5Il7heCZ*v%X9}PqPV0-_l7J$2Q&F0y$t{)| zX_lPHanbej2H1*9K)&Q>4(@EL`^Et$&qKwuZ}&fozaXXRcP$$>fVy={wsi9-`?+AE z66*Y0U`u&#&-I1;u-jFwWyJL$J%?KEgtS<*1gp6#agdTVSBNRK(haIV z3m(O_MYDqN#*y*;Y3#b)>^$Gf-Z=MxSGi}|-^#i=+cqIxoiiS^O__&w>+TpZ-Z8cA zPR_}YVW;*h^4BGiHAQ{pJox*5A%H_+jX@CaNBbYSS5GU&0Mr=79aN{(GP3a_`FchG1z6(=(^ijfm1l^i^j z!p@jmcO>JIZ4#|w0EIX}-*6bX>}=yJ&&Cn+jpd+bH~&-22UUMn-`x(&wYp&}jM7je zAf)rf865 zvsMQMWWM!)|KpCM@99yBGzx~*+si4LD%&?8`lKf7@_FpzBazctclG2B<0I>iEhcLY z<@?zdm4v`Prs`p>HZ_SilzpWHCxr+X$wzB8a{o=PGYVjOVPg4qc~!7_26*f`^Ty`c z4P^3G2EuEembN%#=0<|Ld{x||85Q!Y#CwiB%;B5aof2~wSRJ5963{^SJx#G$jx$BH z9!Y(oQ0jf>nY~x4eCdxLyE&NKy}2x1G7Xp@gFzQ-?O z6(B4`!x^P^QWFrZf_3eDB&Te>;S7y=1T@XzElgby9SaI+x}n6kv#lEYa)&E)FN!b^Jw z=Oeeeb-C%_jb{i-Q*xU(yYCWXD?Aw~SqOHDDL3#}kqBkJrdhG@`bKaLiuszFOqo`g ztiI>q^I6EwMM&t#m9Uf|}p^024|Ubd2Vm-CFIMo;_JZ2>ALdYmj7-GfE{t?*+AS3h#7= z04Umf+vVc{|L0Z_CPjE0xL-5OZte99@PxYI0J{HIbZkOLDdGBOYx2*J5y# zKR%xyXBshDH#iM368KND{_FtSF4-AZpriG!5tW8{afB zMAA%AQf7Ce_dZ;|M&Nw#M^wc9$nwu!Cv8m?pw$72J^zn;*W=CeSvfNL<8HdQFmajj zXU5hJgRP-FfP=y)8PZ(0St9V28)Xz)H&T(eo~Y$`aHNFr&(2c3iEbaHIgBkSQB<4v zOHrMBv#Q`6&ZtwoWv?{dM9_R-VVk!;$uaTQi?%{9PI2;eG&)oNGvA?E*qomY_s&xX z_)}_{p6qQ(`xGSGLB&^|?;bIK>nzdb0KYs5H?ZXgniUjdHioL|4#!lk9Y3Rr@2_6r zN+L#pb>-Xya>H8XRZ2=v@CGAGz}YnGy+(2*eyQ8U|MkT$kCOS7PG9xV?_C$8A@$F4 znoHfjVcq6gm0l{pAZ45J#~39()^D>scOlbqES4nfo>!)L+NU$ht0D^W;S@*oKdPoB z9~0NJF~|EmLBdOi6am^+#|WZ4&ta`Dd)^lcx$`S6Wkf7F+0U=i(SMp4)&|fHns+0+ z%`=X-ExHI3U+`{mTmHVZfp`DcQG!Lo~MiHCHawfeOsj5 zpU8&UCO=og9$4!1rDchf(EGB)xQ_D3TpebPcy5=F`O)%U={L0|R=af<;|68GyXRdc z$Ihlc+qOjM_Ud%~YSgmXeWa`L6htcB>Zl4Wv8IMh`O0k*hm52gM-V32aq6CmO+RRUp-~nGNyZv z6%$43iNMPSArwxO?}?1~PsOCMdDT_uI1)VE`)leY`P%z$85*r#G9Q1+pIjYT$A35g zD83namTG^JAX6`P2&cf@xpg@Pqc={=#iPwmS|fls=mYH*<)E%8u&r{pN~kxI&-JH?vt zkBUR;1^p3lN9F56`0Y#NBmLGsM|ryr>^Kf9cqxA1l(ET|&(6gPhkb+@gBI zUG=wyYmhz6jkP6ImcPeYT4ia|ULq^9b+AdpO|4;TvK)ik%EqgA+00yj6)j*P;VA!K zX({6D4zE*Khc!!gY!&{Ci?;BSv?_z|oxw0@6CzkQT>)F&ajPl0RBvAfsc%=mrU|{qHCD{op>1>$;xr z*pAir`+2|5^L(B39K9Uhcahmq!v+(sIes67gQNq+Z%ndekiVL=iM~g0M=ltG%oWD3 z;EravcCUR99SZ-odzad}5QzTX`uXX({epX!<3B;+#2CinliZPKz*HtUK z=phW*45yCo4x%`G=C&?qGXwlSV}E05tB5*!00^Zh!dTHy3gI3A388PSsJH8DV1AUMPgt9t*efxpZ`#;?aX7)>HD%@+94K$xRzW*=Lznm4mz%I_ zo(K~pwJJ|y(3SY^X^?#WkT<9~S|obr%*4zBW*+m9DCY^nbsvWq!%m7B?)>@r>hiY? z@k;0G1hKnnOLm#HtTYnI#?uYC+xP=YVf+Zr9eif^>0>t!@ zBTsUgn@t`cOg!?ylPKEqGk$Q{FNWQpnRImAN z^M#xfAywZwtWFm`O!Y>5?kRwEX{kW_0~HI6Qai*krar(CKO|G9B)@SsT{Fb%3ET%& zZ=sq{(}C-Sl_U0;Blo2`S`bfC_!Q!e7cetS^0BC(2RQ5J)k$dl7U@U)@ZLzbd{cA4 zgs~db*A(aPKaqQ>F$*aU8i48%(AE00&vibOuiWpv!IIR#!p$uo?orpXP>=1Xs?t9_ zT&;0g*D%i%3GC(bI%P@^;N8hPt~@ElooQdp^8TzV23mZj19TCsr7DWgna! zccG@8G5~E~U#XMj9!{!I+8;&^$Bug2LqJ>oWlA=QZKC#sF-D!3bYUo)6vWP}b(g6v`T?#nf8 zD~;4`sq2{r5tjoU-QA5L=JTJ@;InDPfY*LT&;C)ZTnN-gPfk~k{#9uB>?v3;YGLS? zCa)67m)0Vk%;rU2!=NE)eZPw}0sbys*93L;rsCpH{#!4(zM&7I zR+vV8_vec5BXqe#US#{}W!p^=-0tUR4o68U3p{MK_abZ$9)b`3MO+gfl+!clO|-g2 zohe+d2rshV5cu6LYsm|IgDAp$Zta=oF!st3h!0M?mrti4c$$pa&I(>=x#?%odBkkY zm2fVf?~HOQjEyxjXRK)+>2rAANX&K5tp~nruVM>Hi7l9-u~XHS3{g&7`t}~0bzAh8 zKxH^gr$Z}~9S+DnY#YI9SUboB4o;3?HrDFEPDri=^}~VNyEHMi+|TnfcM|VWf*(@y zUwZ;c_&cmG;zVJ-iG!;L73h4J z8!cAlnDJ~vGM=T6%wmN@EVk+CL(*jJAk55O)SwbbWt_zFpgr9zcqt2 z)(+ZR_3EAF^wXyzzUT{O40`r4vuI;TNWnSpC9WYP$F)&tw@Yv=?3OLn_3F&jv@(y* zdCdE_=E8^}>qVB%B5&isM|@{ms62qbOR$Z5^dATBmI`VuiL_Y~N&a zSFLx3^~CB&#|klm8nO_UPsfn-@c%ptmV>mLvB${#2im?po5R$k?B5*>dPj75UG5CZ-p%I51+2@RauI?Fx0U zyV-lpth0*wGd>1>tV*)!xZf;teOCjNuU&KAucM*l;)Lf&>bFRtDy{jof26eLE<4{Kj=svAt#ugl&1 zvDy10pTk0+%0z!DtRlGqr&cWSU(HkAU!U)csGU7cR1)4iyPopyRuBJFc%jO7oI$GW zXekg>3upp#$bv(%?bEiVN+{BZFT+SYyUQAfIuAWm)>3EYn%`S>^;l}mBWO@bS4`k6-N+OGO3 zcpSaYz+bC1-mU2%#dZ8PXGH=sFJ0k-smtC&m0xuY_OPY*UXh|HiQGI?sXXW9aSVop zc4++C9=56~Ledc!s&Irnnl>rQuJ2N%0eC`iUe=}k=7n*%QH{-;O>8DSTVq5c@%*Z) zgsF$nBlyp3{DALF-_5ffER|ZWrBzwro#LwTOnqEd;_+k%nN2%yt16>uQhq%zd%?8F zJd<6*@fY|$B|0)*Y_33X%5A2UhsnH{va?4R3=b_6YCin%|F749!N&XWmqTp^LY&=g zvG41DbG(H+V(eu^Lbr~u%iY7g>>bJqCZc&Xf2R{rRcciHo=`M2L~B~E$NgT24N)^z z4mlKP@i)0UE}oIQ-AfBT_R%qD^!||`Z&q}r26k76Y}j}K_K^oYBsvOL`_o)d&bLZB zHXO_dh@MMkA%E@)nBmCJ80K9ZeQf4DIiGv#1H$#D`?WIivhpgtFcR_cD|D*1Gg5@R zeN)}i(m9iKr-fD*o9`vX`g=ToYH0LJ?7b_VoV3CxEiJTXR4V_P;hyPYBvrlnxo_O4 zv9^bK?=8qtJsJc8H)hsUO~M*-|CB7Oumzn2jErGq)Yn<@j-`0>hDNv1>2xRId+KT! z+k`hp!&Q%}uPi>FDF-mDF;w|({9J^0=0-1@Ngc_C@h?$s7r#Kd2mHZSCBb$qlWBQR zrSh5JYgQKo4I`5*w1(@pL|s)*`$=3LmjPN&53MtVO?lP$7T9~4z2NVlJ)o{q2EKID z=n3RxXgukdJGLlu^2N=^pQd%vh&m)&PV$e{Oe&;Uv~U1J0L#6 zsx37Ocq@%U7<`ZkQ@^64KPI`>4~|3gdR=6)G=PYu#Ke^LW@fjPSU=`Z3ksDqZM!%w z4;{>wF0v-yn0{TxI39F-lxzYm`hj@w6Qnnwxy zi3jT@WiAKp-6TP6xYo6}yUs^~^us2S{Y|Lb3E06~AUvC2V_tfsk#N><^MeeGutP&8 zd26zwK`B)xanfVKUE_`P=stkpdDE1GDT_qmH zFFcYtyh9JIBnV=zAQpNSWJj($EdTN4_7NZ#-F>C?hUvmWsK26b@eZB2K|-4iWsP0< zEugFb`4^;I_u6bxB2DRTUqw+bfBPU!nq|2RyN1+g!2$)KSVgWw=$N~^$fXfhGp_uf zu&=oO5&nPWPF9}i2MP$eE_SCuo`jxgHvXxBt#*pDSY34PE`*BL)Ou;uIs@|z&C(&Y zG+6->GI4#2Dqf8Ms8B*&$u^860Em(y!@(PP5QOT@nK4Dj-RzG3H|@*}C%*Gjaj2Xu zij}0JFg^UaAmY_`;8pgpn{W-3+J46magnqZjBO+L8)^t^;=NpQAfDe&VGIW2Rq)1x z@`@uNV*jYJDO5Ue- zW#Z|^XWe-hr^A)*=VK?`yOg&`>Ch~(9XeQ)jyS(^y#rnq2=W;$5RjelEYf`VVAV9g;_SUi* zym&#c4>4`lqc1ZA6-s%JT=LIkUISWFpm;ra{}D7e*?&azcahSmWzhd@$6z zwANZ8h8K_t@cy5^bg&l|iZmKcQNH;~nlBj6IJxG0)=a;sM7%rD36jW;tp4hp7L&x7 z#oJQjnL8;~Ee(?rekz^Ij;wEd+Nyj79rhj{;-9d{ZtraFA4+SL^<<(m&%ykGVStbR zGWwRx8A#azRcxe^H6T#vB6i`xx3h99MJK_o)3A0P286Kc-RfHR7S7E1jAatB@utjS z{Pjq@6Eyc?_54B-RlX3f>@C7bPD8ceOr`&;=zTHtvX4Fqey^wFrp!Mo)vCyw0bHpO ztgbX8-eXGg?PLA!TLX}y$^09C-`FwzfJlr!*;-?D4D=fS8k$Ov%7oIq0sMSr6e!av z$>&l5pMk;}z(s|G<7)8gapIjIx|1j@vADk_V)kvBrNw6bV#oK1Z$JP0f?wys_{?pK zK7UeKyU28;mBm2M1Wrdo?x|1Z{vhD%NeSb=L%A1T9_PnQ@z~D=BpvTvNuK8E<(U?{ z8lGXeQ3vsOLJD0e{ZZle{N~z1OI$PUh|8-<-pCIr=Dbbum52n1eV)e&uUxJyvMlKi zO6J_=U;nSPhYbC6R@1#b8sJ5$PE>a8dGevJT=<}jB~|&NZlnO)Bz05~FQERzAgTIz zh+9s5@I!Uaf6cNd;l=N6+#G3qEV!AvCw()oc1o&TLG71MIO4|LjWM{60VzkXQ`7Ht z(Ab1h$a7y~dEQCGsmU`2_Tqsu33Eiqfhi?YM@d%V20Ag;`Up4#f6T zLzz=xMY=qb^^mnD^uwZUm$X0sWlb?0f|ElpC@RV93fE+#DO7vR$SDF?8Zd{YA9;kxF=0x-D?xRdEVdjaFg{x zbEL<61?aVbV%xA57E{*zW8e#aGq497hABy#SbeKh_0_0&<~C{I-54|KxUi6hjxqqCjGkzB zYYV&u@hx44mwa9@850(Zsh&AZ3)b;%nrx9wsz%j&bj-t)iMb9MGu?WhzGLzd_G4mQ z9eOa;h@m7Czv`pOW3h6Sm{g;*=xITZ=kc&|Xro*Pu-eU5ccFCMw1xd)=H@!S$+frv z2uvLW#vSHM_aRGfY;$7wEu6P^O}wXyJ@ZdQ+J$)oVG2svq~uYrl)dp#V7;@?#bl2A zJ~i#l|5)cl663qY;yPVgcyMZw%8uYmO zuYF|?k9Sz(m5QxM91@9=`4IQSHW^8t)Ax&LQArMlc4J0og)pW2YxlsFzY5prx(ojF zGDN9IR!5I55d9wArLZ;<8Dm%t<<6U5ILVHgfVKLb4=ubawn%G!m|>HulXu`*{HYPP zpsRP9ALT{T@A_;$4Otuaxu_!j9;>Q1M}#u&pR9ca&b3ke;EU&XdWqIh*N6*_8X z|EP{AWU{ctbNv%tlrdM_mT-vZx!gyah~mRXKNX*xxKI;g>g{X1R_pujz0H8tpGZTP zj{}O{Ca26*x?Jm)LBzicrCWfFv;&l!BTx8l1s52!^&&8``K5w6;2)56B91E2a9vK<|3g-By~lbMkAS<{ZmQTT}_HAvBx1vcHLucfdwawH%A{|D-1oocW17Jt(+7luk+)Ug%T1%aFe1Rk#V+#ZmrxLuR z^0hP7*Vl?ogwK)A$w@{hj;xD_v*LseXz;%h@$c~n02)nK5AVUz0a`e7jGP2u6suE2 zkkegzttX&|mE~pgTKgpn_DpGD;qTE2;1&6B9(+*}I+rsxrsd3e^@O45%~g}=jYjod z%R)&^*6$(HwaZ)kuVS{geQZMSDL{?iw&$gC^!KEGKM#+oJuSZA8R(N&fAH__6rU>F~@s9k!C}c zVyC*fCrkz=bJMoS_6>FLS#jC>H4A>85E5(O6y_XE4nW!}rpX^46>Xo}pC>yug#Q1s5v_2;tLpbKX>0p^5 zQRBNqRTyJ;_1lqYp?jO^)A_uCfyO0JpNlCyV#=x6Lf0_xbL|*QT>ce+Tj|ZZ+Ae`# zV@Z9UGRj1;0nyPZ0qcxrk;30cG6JpvI=-pLQknC;f$!CQl>1KalS^FZI)EqBSuKeW2c- zzm*UFabdo(Mny#>cjaS(3}+iv$hE!Fd=L*pU;Qnr+s6dHcFZaI21@X1de{S3H5`5K zRjfFnK4n2R@Hvuldwd<0woEp(jDHD5Y>A%Oxwl)poq%;pV@g%;sNCxDb|yryxr5B= zAh~Ep1Y!E`q-QDhhc0`=G|HDw6VtT_1gRu2(?sE7+6hT`q#WZDxk`9d)1QWEF7cD> zqtYD>oT-r?4S1}X$LT%;OWp7jIoC+0|9;xp!O^& zw-D7$fF*fsP}Hqncjx{;DhHk89~tFPysR=rDy0INg)Qio2$?|m(X
    I@y<&TGP- zC&gr!^*IIm$|t)t{aO0Ps@pzKmh?h@Q>Ss*-J2Lt_X3K>ZohtK z@p1410-FzzpgN<8J)QjiA>h=7(v@FX3~ry8LO1HqjpCx31VbMh=!zW-Hz3(lM2e%@ zI@v6eI1>6Kkf*3}j_;TKa@9DmwC4O=3#YW$=RRZS&RA2jT!QJ<4&E`|`uY#-fC*>n zM%5n$f;kFtN|DX{?!n42eFgUS;wD2T-|F>=q(i?mMGKAM%uc>T11K}uHkH*63gC9=PSP3 z2mh$NR+tX`EbhoK+)SyQ^ncauZF)r9KK>Yb^n{hh1k&3zK{eR=Uxt`lMEd8F=s|0lQU8mdD|uDkLGZ)jkk8JDS*)-E45AU zSyrBRuPshmmuH{KgWQ`Z+w}WTLpc_ks!bWEZXu*!Gpuu1a2Aj?W$psADD#F~8)B@f zf(f;=6OPd)NcwCclmH7QFEUwlbKWpp=<&-229!XJXMnTjn>EXtnxvCNRHOJD5*6qy z|3Z4RCWWrN&6Tqv zG=pPr0?aG%mpEm&%^aId)@|3*(u6pTkg9tIQ=grgc~uvkjm%qYtmLgNjGOG{W!nV? z?nzdEwhRk&hFnS9w6SDn0G!5IetWXLyg5}KcR6sJqfQ;>x~ZG>FkF9Z%~N~Ataw~n zT`H)d!+Kp_^}ab$5snOq@eF~1yA}URqmzue$=}YozrXudJLHw8O#3u;L(AXs1{Iac z*DV3jFqIpM+P@@s4QJTr8$A*2wycUz22)3q_coLyCTGr*7wv$~_SyO)C&3C*c4=A_ zm~V@KJ=|;2c7l71iSfSg*tVWaG_M_t!-@A?Z-wqj2<5OI(!XM?HbM4P=BFxWor|R6 zR1voT#O+kJof{`%+hk~kYJ9_w{>Yn;!+vK3uX}GD`Mx{m-wl1BU1gQFAYj(PYl6Td zhJxID^&1FHazPM9qlMDzC!Afu&1uIgdXGOcN^6@MPi`$9cls?^8#IFW6)fiy4p0BK zsB+%vo2^0{$o9-#ayi6CJ{2i#i)|y|_^S>Y7iYFMHd10)m#_X_`nl)?KW7~x&HDew z^yDrx`t^giMaAJ-r+s=S*&Y8y9W=7@V&v{+IGlVHHf>?j^k>r4s(Qz@%f_U zj%wmw3rrtnbh!HJ+(=&-1uzQiK4T3w$JfF8S#;6^*Q=T*fxGI0IuI1hgj!o-d`!=Y zKK-(}D@~Ya3~@KmpTKhgl;|ZxX{Jsu*+zpj=Ww-4yv*QH$vXgbVa@HTsglem!eAt# zI~m*6g9$s2XYWezQ+lwd%8+4u2zFr^R*A>;vQ&a*@3RIu8`U3Rk+ z_ZuA0fgK~R2lM$%|ERd>p3#fqczd1O*h=H_y&bq5K1Ts#^));>I#YHj2b%~wcQX8Y z(903+aUjmPIjT7M)z3uTVckFPR#PeAsdz4)#2GWd>G~mE)Y`2MR?m3Bb(+Y2tPr0L zM*3671aR$hPXCQ*%9WJOAVATKL;<`;X0bSQkG2Yqf7r!0h#rjcsCj43MbyrJNF*H%h+hUXxScP`?1;oV z0u1IiAO;=Z)Ucp8g6&F6#jPme0g#%nt!xSAN?5v zPx1!H<`L3BHEE*mjCF>x2T7C7#>-hc#8Cjt!ihS(6(3rQCf=b(mlU63Aiu^~Agp>wmyP5T1 zAr2&Bcx`JE()nU~;lt!jJsrqaxv-c=i0h1^d+io6_x_+hhNgjVfG2FB^D?^Owwq1W z<0*<>C4VW)rGHe4iK#7bQDZWhPzwQ@$Z0#2Kgn0nunz0+G zmn_%a!YX>AkI~aK0x@-WV79#^>Z@Di$KaY2_1i4c`bBkxA_03R&?NjOaNXNmE~R;N zO&!F+e2k7`8{?2jk2IX?Fg|jgBnhY?{&2cfZeqGAf%Wf6UzrkUwfaqn*Q7d`; znW4$@Iqd8EXGO(^}M-0e1K2r5o)j%{w_cQ&U71(yTv{Iv5?-7Vf6x_(l2e? z;B7Yy^mJ5JAr(*Okps^u9f2tx$;L<{3nvLiK*tuznN3@j8)6y#46)*<9|_=2SbMr2 z+xu#a|ceaq{s_EOvw(p%obwm^TgeDw4mRayCMjws}m$9p}sp{~9Be&90gU8J5+ zTKg}s>1<=X5wFa}vQg)sVBT+h;0hm*I;>RRI^82R5eU!_g3BoX*$ZX#gR1P> z#mzk0t%;l`c%4;Qj+yg}46hrty!*re)0QhrvvvMrQy-@GbKqJzugUKF{YWc-y2zl| z_@>)t4-1_)z)GL_2QI2Znhh_YymrfKmnk?eBdOTo<%L7|$+pt``!@P^c8=_2Sz%e3 zGlfb14v0+srqkL$99N|-N7IS5qkt1&zm-j2QEE4IqowG?e8J3SE<1Xv<633^TqzS7 z0mnZM?P|_m#3#IQ7q1maQ$)K4r#zjSX~*0T73udAVTyDzGJ{z=HgsFMpO?2!mX11? z7(?@!D5Gc7r>W6W9VT{I*I8RzOP;+^s@2+3)^l6N!dBB{yN)2~sPJ`Hc((0;R^-z{ zea|d)1A3Wrmx$01hdy)uU8xAq;!w*Vt)0u#x$VzKpo(#5RZXJuu5vlo^Dj?Ni9V%y z;gSAUX8}q*?;OcFA;KjPlDTdnl+xPS1FKUha)>Im`w=t(vW8Kh?-uZ=Wr#2ujy}5` zd6Iv}MBs9KXC>OBL~K50Q||OFF4U2v1=T-trI%*6pkG*O`lHH9rzqCHx7TTaoV??_ zpqThz%4t3*bgunLWZnGgN{aT#@5H8J7Mi$vWU(cioXl^K zg!+Ou*If@1javc8WWVu~*4fFbgWXw93Z*{yMt`J+iQ2r`#*6$dYj||kw_HTbvoY?y zIrv+*wPik7+Y9rfl(}D{l%yd=;;rFDv~<)fdWz1o_eShKXj?Iq8(q)aK+5h-l=?ui zF2CUHs+$AuhGd0-uQ+0rBY~| zk3B_}i)YtQh7k%SN=QN>lLSnQxRX>f)-gcWZ72`<7Oms15$sa$)~3@`l@DI4MqWO0 z-reW<;rfOhmpZbtr!;0U{gv_g$1tDPmG@xjkKH!4HZ)Hj#pom?FHQM3voX2?1~ePT z#WN2H{1^gG%4;OIkWUAhi?~=8VeW;*E6xmN2Ff(Cg(1nk5U_NN`%Kh-j$;nC?Ojj* zUT}%4h0Ki{Y~&v2?vonBWaxWrUZOqS2QxtoT7ibX9x_d4-~x zI{RZ;In%(oAcWDcTVLFf0ZD>S`rf%@U__=nHj?>R3MM3-wc~=wW~PmbnrZeABb3=I zuj~d=cB)MJX$3tx8I#RPr9y;J27UQr@iwA;Qe<^_P#e`u-$VcG%-l#zI;EN04@W0| zbL5Uk#PqovHWcF762YR!mi3#VAxD)IhS1iblLhoc&vRCi$SJz|Hsbi){2g@~R*n6-yU4PAtyM9kU)s-nL+V#X(6e@wgVkdO9CZ-!}gv zd!|uoYu9rlF=My2bbCs4(Y~rI$Rx~5K+&*hkeWt-A++B(tRdpSMRzi+>7edBwo6I1 zagZOY-$Jj43@}JWf9wPc=+7Oz>Bp&T^p^ z_Zr`p)g?S=roT_)0BMqa)gm*^E7OdKg?|cPU{*C=Vo&s9953^QG+chzx zL7zu%)1UcjKE-FMJ)3**=jY>VnXOe?7o&w{)4Fj&@E5BC1})yb_`iqZo*`or8%1ja z@L7{itr-1{(e$QzFJXQAkN@wh>NPi4j=+Uox3?@SE2<*4gk4JP)EttksUDR?*vFqZlp|OkpTzKtDGoqQ`qkumAe>Y zf*lJq|K>UXqT>YMoj!H;Tq=F&H)(W&^L>_>bCLgUN5Zw*NX=3B98xg5?((V~RWHso zO^pQ#kr&i6RC--zaU@wz`bftv*2E*fxMl4|?%Xv}e`#dekX}dbra&xM1nh(GOk6nK zzHd`o(pbjoTGeK&outQCEH{xYLQD@TLEUPyf}9hOnHE0a%m`ty(;Ud$Yay6I)9*b*7^<-By*JnkslRE#qXQ?Lf1@rp};_ZU^Fo4~u1n(xlTVo^i7qqQqEiH~6eruGD<6JNr>|=W^=NQ)oWx0{bIMR?S7m*d$~tBNLmK z;^1{DvC%Jmvf?~nYgc)qJjcP;`h2n1_2Eg9ebd6k30I3)pH3S@Xx}wbqvcY}3XFsN1EfXo)z~5%hMX$4z$`6W^iNW#H>X`8*{yQrb3UO^Hsk zzcAE8plMnEgWg@=h3N^4P0dFqM`YERYW<_MgnUd6-!)2t4W4gO@Qc&aRnP2?P<(F( zl=er)WsBStwhL7(6KsTU49?}C-hQzyhid+!IsHOy;rWT?G_-do)hES6kI_BeC3lVR zx8F!UR7ehT*5pbV)j=2ELZnaC6?1lGg!mv-4@XPzv9N_|uJ&VnqOEU=+jSaG*LUCP zY$w79ZP3q-cG|I>mH?GIGkQBNh6TTb7_ch$^3CN!MI`ixDc3i5$pTVyB{#;+!B7M|O+f#+uBY=oEfbmYu?pGjD zuAg&0`FR%2=(rDatXQAN^3MgBLDz)_v-H_HZYwJ7j9M%R$ExzM%!9_o$naygQvbVRAb-|3>|!GwBf*D ztnu;<>>2a9ewn^AJn)xss?O^!0WzCv>K}iPsnpS}$BXwlJ=|WlZmYjQ%V4~HOOl<; zOlFQVr6rkzr>kLSH7eq5BVj9E?r!7AnWkGE4I45LpYOBayOJxeUGo2bQ+Up zGaK#^O}4-b^3*CI<9~7Id72IVrO;(E*z{wtoGRy|ncEXGJx7}kPhI0;T!53UM;>jX zde*2?!`FNGs-WK{W*e>Qz(~zBl}w1=x9H@>JdWruq$n_ z9Tw8Ul+={eBBjpUD-wPW6^P_ldtO_bj!WjI3bb z)s}!VBygink5F7#>Er8)+BQprhtRwiEgJ#>j9>$GlT_blJeqAbWe)TYTCgp%=IMSd(KVrw#0<%|MaVG(NRAckbeFbNrhW zXIp0idwXOMm2TLCy(QWr=P^i#3{ww?eJ+*)taRJi1rZLq_tUkH@}UG0XD-oa>NF>qJguj!x6Tm?!;o_o0nj1qpb0Zg2>{=Rpmjq!7#4 zk4a4zr7e-h%>hAEUymya_c4-j^Xzv1J$FXicebSRpUEYG_YK(+v=f9URnpY&4TB1+ zAr^HllrY#iW6R53n>F(r7dhh^JF2n5Zy%-^ByL^_|Ct3qUaC!Med$&RVQk4fn&09} zQmOO3db`2b$|*Zx*C5KT57WzvY$>(mg|+IYVYk%v1Uw*McLMLW5}>D>t8izK)6&!R z$kcA<8!?0g_O@Od5H|)(T5c>Xj4UJKbk4NGAgb%^;%__|p_9GrI~r{2bieDn8Ozmj z77>7Z6`)60x`eo%-mqKO+0OerI%>Nmd5AEa#8|u}T0=43VtDm}T4LDU2}I7#95G`H zhR$BHqrS9$_`k#g(2kf2*cs3*j_%|qG>HXQ9&%&+{ib?Y?D%!?;7Q*IslWZj)0eH| z8m7Lesx?{iR?=j{!DWm1166;JHAsf+e=d6d_^NwiaZoWBDayo71`X)6NK6I@4~-4Q z5PLNwTUrCOwBvsuK+dkfZOmb4#AaS&*@x#ohu2C?9XMWy1txr7DZ`Qod$^6MEL3&2 z;$QNa5D7mrX6CJKP&2pHh^!f^jNcuMKj+<9X$#1XJ$EiK^z&^CP*C*ADI|-3{-M#% z#L zWNyG{5{Nf2Te(#rf*&y2ZtVcX; zXF{a3?X8P)!MMFYeEZwXyM%W{v9@1>w~!!3(?>At^5NL}EI-JG9-(c~>5+t_ zD6u6JHzNHHscRnv<~^sjvKXRejwf1mI(rl+pU#8&pRJJ$G$=`{fRQZg(M8sMiYDF) zfOhom*R(ZL?)cZtlXGXdZkCMTk=0EycuoiBnWF5c)zQzATgt@~0Q0l-+y+9uRunBfrm&+hxh{D= z65F#a*n6LbFDjqezliBpAk6r%P`gE;#gdZ3R{8_pHX4yQoA!Lyc;df9RtejD-?@73 zxv0@YPZVCtdg6^SHtnhku3mj~s+}pg`Mx!G^KU5Kj^w+yFhculzjut7hse^Hju~l4 z<^}0t6G1;$wiF}hiV{yb9bb$Z{t;;w-)U9R=kh2{rj<62W5K1+t?Q1&=0@4?+=~RI z`>!FImuhb%yV{%+ z7n`wn@QOMUfn8&{$txH;U#Q1&)w7g=ziBQK9Vy7-f}Tzc;D! zG)yv{<|Vqc?gi*rR;fGW9~FI6PDDPNVSp9}BCkR_zVGUC*RD`n1S`{FrJCiTLE$GDjUawtj!Ns3tqj=fh zZF%Dzs7V=bcMog31Loi2Blha|HAvz-W#5dAer)XisH~*W%NW?oK zA$Hw0y90Nq`GNVfxW$L=50$8ce0Rl;IGnqTBP$oyP|B!B*^GVFM5eEI8l{l)Cr`uQ zed7Ky)O{zpa^DpKNaylPG~Ng*3V-?5v>3sw`KoourMF z6Y~J^83UBn*LKaHxs_5|71}id!+TFcvMZAXe*d=yZg!47em|xpjngTSLT9+-)7BOY zQ%XXl!K%ZYY?NPF6W}Ra-=XCz)jui~yhZs$iTs1Qte2WPoR0$-=a??!pX@M1?U>>{ z`|N@SsuTQzgm(OZN6E-tJ>;+fFzDLCMgr}QYgL;W&~ko=>i?RKp&kvA`G zg6(jV(@*QlOD^t^(a>tm*3yZd``qisNz(UboILoE3Q9k2 zT<5t*Hrw2X?6ZZGy17SVjFDNnVEl za;xg=o5xR`?d&fNQucj!vlm@E5{Xfra7vgxI-oJ$&Q)S|C;kN733{?K7*T}j5voqI zUW%&UD--cfZxgy$Pi4hK-Z0U>9k<4W7#IR|!|b|*#lO;7 zc0bu*b}t(!l~`g+f4`P$mL@0CIMAEw00yj9I1j6p`eTl2@cQnRX?PHOz=| zLM9`@t~Ljn^EIs+^uHeFEJVBFQ?0MA=w;qdcRw3?Df@6Ie3+%%m;JvDcXJ;R&K1Y5 z=`bI63x;#=5tEvptnO_Qrq|x2e(2?NjPIvMA@0Yd74+JV^3C5r%QvWQ@jUU@{q@3) z-d68&i>4${I$kcpVC_-Uqo$0S)l#72*6|izo|~HEX%fXG8k%Avyw=iKE>ULO=}?D~ zdHRVN0b&t&d@WB?NAzQ-=1yc!;R;KS$I>TL%(G-TF2#4;xBhI?IYrpm-2<*7+TekT z_HN!I;75YJ0=Ro6|535^2bm8li?}-W>8KykCB4hF{AO^ygHWL1FENE1aEUyF!g=lJ+SFripj$c9$l zWv;SXvjk+5G!Vd^fdH9)G>?+pI%88eCWqH#_0uM*38f^0;!wShnSn;P zp4wTQm>kE4r@p-Y+zM_q3btRn3*ZzfO=_pK8GGRHeNStTb50wuuIQh4!XkG^bTH9q za3foQ1X4|2dQ2U80J_MEk%{>OQYjX|TbZEI0H_n_I1Y_J3O!+T-S)4>?@*3N-@bMf zrJ458mi}RX8m6gQof7owO{k|$&GD$^__F^>SjL@LlTz!AOEzPViP45^X>a zQwo5H#?1^RdpQs+W!67Gza#hOvyBZ}ch`L-P2Fx2wl_Lbm03l2w&d3mt$2Ng$;R@% zq*?+=jU%MnGfcIIDK+Qo=fR#5OgZ8g$oL$p^jCi6LEcXC`qn;K*AD=44oH)n?ds~-D3OB7vMz_t(GA*EW1ju7=s#eT#lP0P<2=VdVL{rr{|cOAarwE=1_?R+@H@A!jgEW*E2 z)m%&ioa;1?G=V)!U-`9Q&^YQKLjVtc#Z6t<(03X6-k%$zmuVxh8=&?If(^tl>Kq$3 zI18^CWv z;orZMOMNj6^J#bhJW^!-`)1h?)^?};)K-+TfsGP0e*nB4vpui~2-a`Jn_L+68Wf+} z?Vh$oC{VWxc>4uPcI+f5!fGw|nV?et(xeNaCg0Xb{O9Y;d`iCTm8_k;v!4C@?)$oQW^v^ZWSb1`;riiOq4NW& zQF+rZv5v^(l=YeRnLKNeRnd&wM52QAX_JytLHYZ-I97}XF&EdTzWr^QPjUrBX*E04 zXz)?ctFe8*=%Vw*Xd7U~^^?}K*dc&z?NIo!%-{y|BA>P!Ct#+6sr-~Ft{d&L-VI~U^TS*xaeB87 zsptNc9)ceB)LP<@!B?pq9@)=IderZEr6s=5?i7xGQ*Xerun3CQc&U-Y)6@@mmUuI9 z4!P2WoX=xV*3b9z>R?Y~yi0i}A!*rZsZN9Ykg%f!-?-0X5_I%_rMiY6zP~rncI1nF z7N<&2fmX}wEqmkSMfS8;{`2(P|7zSaT}8)!?@tJE(;Wa?C3u}24itgn7rq<4qo6AZ zJ09izy`p-5LS-su;XvNCqG&=Gk5zM1w0C;aKQq9q^mam`tBE%e?y+AqX{UCtGHkTS zR7mwJkAt%yok!pSUq9KSj`#(AYE3H>9QW$#g(afWq8Q(>wxyYU@)9qmA5hx z4d(;s1h=UM@Jf^=_&U_%n$3$>hAN8J%A8!`;>~5Fj=MaS2Ej54G9DGKx%!zx3R%B8 zELtkquiWvJ*v9JGWWFL(MVTY?-hHq2w?z=+W1&!0HCWDCkku2gQER$sYF(9teP?TE z0m3#|u3y)uD9g!Is@K4RR@U;oy6B2?)FVz~I&tj5GuPsn-5QaFEQ+jK`?c=VV`TQ4 z&l+DMg3Ch1n_)K5uBdWb^?1;dkTux<^yAgHk6Hn<$9w|5WBnIq8flP7vo_BY`(a&| z!YU(G$5zZ=lHX^f-z^>kGSIL}r9VGB*k7anWxrM2S5w4_vOeZ9e$Fr>Y+0(hus1?c ze4M)8JH4Mz9-QK|kMY7>HVtCMacP|Na%I4hWhXO(J#PCwPMWhe$YfJ7^w)wt@Kr9( zA(wc!tYDl35>h2eMK(W5%jh%U&bE|amY(Bn2mi8>O3PQi_QW zfNyCvzq)e%atd6=##rz!KcUDWPdGgmJe+Bnuo=kc%yzUSAM5Ha7UJqvf1cNNRMFok zrn`n=h+GMIvuDp3Sb{-iP;1A*g%Dj{sFJPm0lhOHU{!^6A&S0d^a-7_6IKQc=`>N( z4Z$a&&jJr`>xx*Ea2+vD0q|cc0+p@LTuz(0xpp*KG#m5Wr~qvOMHW2A`gBd16p7b6 zeKTazap|J(26#*P4dj}Z@M-ojh$zD~gKFa{Z++XL3v0vlAWMNVXZfLoPXEJ$guuLr z8bTP)au`?Ns)4Qj>F-u*A;yyM>t8trW#&+{;(e`j1cv4`7=dG(B-nv9_>v2oZug89b2J0?{ zO4ieAKie#8eb}f9wX{xEwB<=b^(iKizpr?O_7T4VNKI2Vy;=C^FNtas`Bp@ixmI+~ z;dxxMfFkqnGHEotPlk&M5z*SoxmZOEo5Q(kSZieSXOn zhb-UZO4cD*;JNzWXF;Dj9dDro7CdA6S~y2j)+;LW1g;-&X9z8rETHHF@W>D7d_cPJ zaV|Bjmc}0hSbr^8$-kJQN-#5C8}ivdDV;xgC!~=r8^qT=fht=HQC>x78-Vv+OoKYasrw<==%HV%`M4O<Y&D1B(O`mBMUk*u$>Q`fxC0s+oat-AFFVgIuxbosE&r@?LQA`Ng%FKjjb*Qm8A zylyMwV#QhdWu*qxflrvB`f=ksuh!3n_3co0fNu6Ul)5xN2f6OFAYNH@Fm>zUpEa|b z@qh~41|zGEMC@$?$51>?2lV=J9bCm{@W{yK4pL6{AYXER-G{Jw`UHMDqm0$3U}15$ zC)&T5YKNu$WhiL=aY^jgS&ugjM-Ayan{HVUhgU0$Ety7OXex4Y4hWr;DV6o(y3lji z_mXa5bm=^^)B*3iS(%;pvk1S?`)P^nqx@1rJsQI-BXxqiPZH1fT-j|TFSN@o zcsQB_y7$x~xBG2Waa!kIt$UwT@iIKS^^4s;rJ1I`pIR z@zcu?Y^iI5Ge{UBCl7Rs(z#C0bSKrv>fP=F$GeejHfEpSrf#oRWD4Z%=QT4nJXm~5 zOY@f`G!7ZEE+yP&ldYF2z;{*eWGOz0oL1#k@Z0eA@bHtO-mI2kCFe7rC=Gt6oXpp2 zP`^D`Zk1A>uHcvUst}#+t_LTx?crVd^ck!`!Ib-PpR#k^H5G9*{k^E(Y2oVBqNV)8 zM0rQmxAG(NST4jA`>{Mka7q*I0)KJ_B+MoC)79_Z`7Z_@xvRR1tL`iH5>3}7#k*>C2( zKR&Kl7zK5qTB}+&N-2YmJTP?tx>mj%RFo-WK!b;I^gkxM;9jlZjh4X zM*S;efqfu7a7R2ktM~xk8Ye02p;)Pj5!>kpq#i-%3bzj~|N4#A0 zF3r4Dp6l(qX3NNyJr3rG=%HNri~km)WNU#{aZ+O%M;f=_DVO&8qo!QhvIv-DneInf zgOxqr)*W}C<(#j^9Cx9DR9BcRzA8e9y=6>7u6RcMJM-h`j;|*);jQs5t3P{!UfQ4N zd^TGc^l3wL_6|HtTPa_1###zzXRyMMtEHs-9YbQ^za+y|t$1e5?s>7^z@R_B52XW- zEtLKAe3!XlTQ<1%4%$Tzm*M-Z?ptggAqqDo;ucQqb$I<(Lz^~I6Z|UyjT9uLdl``z zXCAEwcA|HG_A3*U4N@zttW|YAUR(^z&cQX8w~SP!F}bKLm_*QPxgDNXfz$TUbwR;H zB~cX1JFtolt|Y{(3+MQ7%By3>RrzLw(`I4*gKd+BpQ5;epURw^6Zq#Uv0S=(U@@m)5Dv>Va= z%RAQPT}WAj(jh1sJO}a%&1wlgt@kj}8Tx;5+)n>*q=CXnulb5CPF~XIQ)ySlmYv)) zR^5dsV9|*D?^%N>uV2(MNUoOBPJ-oUT-9&n(ixLGYIEcQt^(RIN3#QuP#NvSgt-#! z+aL>0C52EdRilMPg`dHY`~UguJ6bvu{Jz;t?OJ-RpZmM}9|^ke?-v=n?JdQEHti*&JA=V4> z)dv)1539PB$TcgRE8>~o&&@bZ?G~IHcJO_3`(Y_+l4Ku#YKZ3>WWT4)=h`SRfTvIt z|Ije!?8m1mr4c1z4@e>o7#pctvz^q&T~Q*xYxh}x%BGv%no0+n81_zp-p<+IT$u_d z9$>5>mc%|E>u0*sucx;@BP z`zD`xU!63XmCV9`|V{yBdi2}1dn>{;5)(EB)pRiv+O|5?Gqng0*X$u<1 z@1K$$|8nWYCGmE9-q3T?#r@{%s^Izunpt1U)fsP4a=StKfLNV11Z6Dj>ce@u6%QUA z`djw}7XO%H66hIHvi$9=dL^svVXBWry1q%D=k5AXv4MKn%*F@~QO8ReHsiw+PXqbI zH@|#BGW|K+`sh|86R$D0Yl9V-ABuUpbrxLd=viy=yW(mNC+Zr{ko7+}TNW)BogKAE zG_e{*%NRKI%$Jwl+GWN83&9IZzW89xa(s8bUU`MXywM^t4O4sJ&RC&7DhgME@nP zJf|RHA~nqdY$3BIqfBYIieAes3X!-;@8Asq1|nThN(43E>~)ik=@5gB!P14I7y+LL zX}dy|rtkJm4vHI1WS8#o(_y+P|pk)REB85j2nm9Iy>0H29(IIInRRIZAPq^U7r277l0$-s!xR6 z&z;uz4+eb*Cu9!$O5d#TzF!Cp=^O6!g9ihUdE(IAA(2`*Q)lfD$XtpXvv+AiZtmQy zUTRzE`}bQ|Gb=Z<@6)9YKMKsWLw-dQZ*# ztcdImNk#RKukSPX`yzsBdncg?*>r$JnBnS513q!>_hi=1e0D^8pC1*xsMa2plLF=BWtGSMR?oa?7lc zW184cHd?0dcHr;3$bmau5KS{P6)a9FW)*H%=S5B#AS5#l3;`8V9S}H!DXM&JjZ2Zz zBp1hG1|E}8LEcW@#r&M6C`xVgKRcdN@qfDpo1>{f91SI>{v~0_^K@Wm(A#Nt>9?0j zT~8gbAHHH3ng%gNWkCUY-th@};tm;QALp+qSXX{4T4}!biKBmf#pNGa&5$WI<E1O>7Pd!HjlS~Gr$tJdJh{Wjc`wyOGkkayo2&PEs1rbnRkOEYk&R--8?z=-8G_`PtI)>p1FRcFH?9jP-fI{@r>MJJq#I^%?)iPb;KJEQcW0P|oih|_ z4;MfnN{aW>Z>MRU#j?OfPAfk!mM6_3yhs;7{dlHg9+`2xP@GOPJ~Jtd_F9$TZAZ*N zxdZDXZIqTd4hgau8BbI5_i1U&jUOll?}@0=+vv+JyKI&?(3BqxVQ)u8GXX|Z*}!6m z+mS}F1!e_IeQL} zkYQ}cxl7tm9yl%}R?q47ZaweyC6xKu1pd#bxHW__lj&9Q)YiiO`L*HV@))QkS`fws z<+RJK@m|;TZ)nimJV9oGLAFwKXF6@5+Z-G3RMX*O9`hrk7xj+0BI%=~&R_X*^~=T+ zQgwQD(=ST(r?$#XF*^>ExrtcG%I}+HO|I_>b={5My|Zf;p{Y-wR$CE=`9n6QfG(jr zmgD=<&+?J6g8Wvmc9c6WUcJl91u-<#)Mc;B@F3i{|M@`2IrsB2?NG3AFf^-YxY1YKkGOv^WzQd_I)Y;lu38$mkJTO~sV$00x0r=YYTGd)QhNKyvvSzs|GJ%ft zs)vApO}NskWy4HBV?8Jm$U>GeBtuIfYw7y;M_9&%bwp2&BfZezlgCTW8Da| zCG+7@{K86|Pti)RlDgvmhkE?ZBN8ew$+p%7+vpLDS53qlq7!FCWOKF6jJwUc4~d!A zT(C0-xGsHFkbVDbTQsz}ik#tFGn`CNf2*Cv%+!1FEuR$4nFr86n1yD`jq706%yDDk zcQLVm?eht;6+!gB?gk&9j?yjyd30mvbfx9>m!oX5&o&P;dc+LZdpcWPL#- zo?@d6+}ET!xeBit9;8`Mzf~^)UJYJD>Ow8&is~g3(@)of6@o7%igQ@2+LOr$$5+bN z(l@!++ycpyD zF-#+Tk>SJW2KFs!Zq(u%EY#tQ>Cx3qH+tBD?wj^p?ItK)i3CFW!nn{ycyOlIN;u2* zIRB;9XMT|&dx!i8f%SH8WwVDB{b<>@V@}WcTgffom00x+9mzY`$3~Vb2ezpKxj~c< zn_=UptSQa{hzTe3f>EtwDNZO!M(}kCZu#p<$>9Lm+M#dFyiR-v8}*kqHvoeH%^J}p zCRe8Wx%8Xu^Riqr{_7iQ6{qsJ!2<)e=R|;`!Vgi@L4>F zif9I!Tlp}t|57ypZ|g#q;eSIky2&g0ZtIoD!L{h(qia4-lSXPS`MC4ySETA{ma09$ zKw0LKk6ii=iio85KYgp_obry8hkcH^xk3yw(vH`v&li=Zwv`J5-_fd}FVo3Vp^d%M zc`ad)TbBy$DaU5i=1-LZ)rY=qzOwvhy@AUlPm0 zvyZ_eM`4*dw<>+>v|du5Z?JreDGGKA`)(d#7@_uNL%dYHywnm_(9NZfaU?H)%ad7Q zA>XsQ@K@NbFxR_ADavuV;19)FY8Seqz{<7(G1L)M;GMf;LeG+CefowKwW0WrHVDLg zJd~Yg_0m^X^j(AuPTo(xksaj~lq*|V)3eMoR8e@p-E`a#uAHFTwlf9geH}dR7NW#W zC_MXu_lTQ4If#;F+MJ?HQxw5#LYQV(q*>h~mZJentfq3qPJc-9qq6~uYuD^wt{tns z_;ejQSOF?6al^*|gO{2^XW@EXil*s#M-$~A6ehUo7*OZGle=!&e(8HbgTGgx=(HC8 zk{EgOzcYNEHo3^?hSlq_uPMP|0)K%8sZfWFyEj#V+1gfAgdt*IR%g=vU#F1Q zxS)k|!w++$L+D*E8b-Hj%nvh6pDpQ7g{>fH`nP=t1KxuH_uQ#>=C^W`(da33a(K$? zBPxwZL#36Qt{T9z`~j8Zn^(b_DM;@USqG4 zDFc0anC0_zlPIKj?ObnLlN0##6EaQ3FAM;@XOSKSZbQSM!?vA7vH5M?L|6Ul87B*V zLE51lMWb==bw7aIhxww)dZ!!${gNjs3B}~O<6smC@xlGB#kho{zy2HDk>AQ?WeIg+ z%=ZnL>!_ocp?5U7&NQ)ents7TMuh7Km)2PUvyXg2Rw_qKXWc|jRn}(M9f%RZ@k)Ns zdWDUxAMJG6m=wo(9vQ;!8H36d;3*Xj%(H0p5EWYk-Xz1#)o;@|SAV%??*@iP)_g~p zdY6N>i&j&U#xn^S2%-J*&ULRk^^x`Y*HNg_!K=4efrDanVG3k(ZBpm^LFQw?!|I~ zq?txmqI4k);t(!tPxrbvZLFpMj%GC|npRQJl?UM-gu)|C``2_)Y#zzKG+oMs7 zcl?y7Li2Q{t@G6XfLUz(L>3o+Y_cnhIo-W;i<~SX+Nz(DyjwH?q*bHRy+?|C*Afxo z2Op{&>-8%d0tl2Sy!4iSTDl}$Zs=4h(r;>!P8{LhV0d0-S9m@U9l{6Olzi%RshZ|5 zX;?yQ`{llbKhw=(B4|sybpzto zILs?`panXsH_TXc%V?C79@D;`{Gp59{?}CxJ;?_1k)2}mT?T>m!DhJz?CI2{zh!8% zN7%lh?6}5|y340e( zt7(f*A`tQE%dEz#M?)3$pBok(jnL^nrf1;`{T*Dn48h~>npf3H`&UxPjEr@E3ut!D zYF0loyi&$xR=)iG>$rZZZs{BH@KN=xOrMnGwl+lYOo)QKs79L1Mwy>|tH!jMoO&&2 zXnes+$%@*H>c9Y#elw))4>A+*oiy{$-fnjPuA@huaX}*g;;Qf8wdAdoX@5^TdIYCL zy$AP{mV1T;Ydhn$>$kM1b z!S5A}Z)bd;SS?4Sws)+bgu2GCh_{>`%=$KD3wVSwwyK!^OAkosK>u7ySPB{NU|>Kc ze{ZiPW(EBh!xfOVH@4`JgY?~#b2z%zV|;20+p%L)j;c;Al7|D#g~`6N%gK`b`-@Fv z^h&1B?cuQ#C+Z#=$k%`w-#Y22+Q6HRjp*qROw-U|XJsNr$jsd7X$zJW$6=Ff`IHPc zF)`o4Rpwl}YSw}On){rl2e&)~JK=?BTN$~^J-MB+PBQ@^;?KRClH(OnfA3wUy=y}d zN7Afby`z#8=^thrWTbB^GIjh$){01WvAn{5NC-Md?%(o#SXktfYwcKghdV{Jn!E@U z#5Fdl8Dg0?YkO(DjSEW|e&SSLCcB^1n1oS^7Rn3CJ&NgHc4)R&r&Z4RsN0d@S(0Vc zI%$GHPb?%l6M)^7tOyNSY7J2sud?$^P$5sN>nXa$3^k~?BmnRIe2xi02Wq`>($K63 z()6Oe9`c-O5zM;C*jE{9t8V(t%8>AbrmC;0uFIXGudHq`Up`M1YA@@{LkubkL}D=+ z>`x3ej6b>?n%6tOZY+!V2s&X-PV}jejT>mf9hPYb`nc1>$GX#viA5^Y%0LuiwJ})= zw>W#o!lvQI)zjmjDf{LfTC&hK8@2O2^MD9QGs?0!#>gn!UG#WYe+-+%qz`rM;Za=1=&sm=Ofg3pC{1`OgsZq zL(Y?6+7cZ1E-W_Q@3!ZZ zEy|>{?WyAZw0tdECfw$f#<@Wl(#Rq-&6Ymzb=Ki%46`RpPi%;of^5UpZroy*vwS!8 zN`pIHk_Q3cT$6aXke^anxP7+>u+$*Ctpj?hafi2ol{)Y{U$7B{y9)l6`w$dgD=Tr#pS*qTn1Z&%1m(VFOs|o47ETr~TW63k zU`COZ_Em-$Xh;Z)ETnDs1JuWc!u~vqkEPn7VlP9EwcC(TnY~dSE|;GB`Na{r0Fh% z)aN@)UZfQ|V@T6R*x#fj2{D2`of)V5L{w~-`~WpR4zE2KV+JOs{<_mW3^Y;O>nCCn z>jk)nTYLV5j40f+JFK1qcZQZ(9ZtX)O`qVh#gk_iX4#sdiJU{ogfhOFPHKK z=+J6iB7X`z@v#(83md()fTN-c{X7;lb}*n8(?%v4jV%qdi`fW8faSR5?JLBY{>q0d zX^wjT#Lo1&EP*|7x%X2Gcv{ato=U=0(~x^zm7VKNaO9C{ zfHpZlhxl)CRkZExH@Y~cY9^I4SR*&_yp0QFh0Eh+rlEv z?ZdSYWZu%Zq{9qJoIsReWU=Olqf|tS8ORSF2xi+ufDP@MJ5x@SVx{~+2vPkYo4@u@SnsZ2iNV7@X_oLf#$$O?(D$KZ=W`@l!CAX&NiS=)S6`FY2H+GOVPw@x zeCL3GHNP*aazxn_m#!1aT?4euMdwT3IyN^R`|U(>J75|6r}37yv9hmkDtdiEdC=w{;JkY{))>=IM7)pYz!AXJo9qjPZY&0q6j3H ziN6hc1UC>dI$oqFgvY`do090N9#203YRrU;6y%?RL08|yxQ6Hl`#9;*RrigErg><= z3iW9v^YCV~a&7>)m<$27@Z_CF#Cn4k`npE~0|(`r`?|B->JR+Ah)=k*hX**ff{)UDzY=VDnHVV^v+Uc=_08IOUEId5HOn)?+UnMA!G; z()V~O_~|}#K97UBLRO#9@`f*<)SA;M{a9rd4NK^0E*)E|NE?j}vA>ePwjs#3q-8y} zUGeA%ew%Ay`ws2KE_)usPqsD?FaG9NS-3$tHRT*XSR;;ejTZ?TC6|9APBXk&pdVHCPoF|>ikXO_E^-igVSL&br zT)rdZ>8C}$LgqFz=`(etQhvc$$6n-inY)GxPDa#)Y$t}ufu8W{!#ECNJmze&f-x}I z$E4dK5<^%jQ zAl>ih#O+_QEc)G>zs$=}C)%O#OE?X6GGAVm6FI`g-n{}?joyDfQscP}YHj+m%FD#* z``)Mau7`W`;KBop2(72IY}iPhan>J#XT81w0DNw$_<6gBu z1FNTL#komfsK6jtptBmXHm;j(Ub{y_+V!M~c3?;b^Kfu=IVDL({w~_bZaO*Gqy084 zySTo6uS!!YIoLZsWMOB0`unj9x-K55@Vs#`l{a6$fiy|?)b0!Z$FC{0T9>E+Jt>xU_Oq#`6 z@pV>Mk1~%e{9>cPnT{1roLs+}tmyHlu3y%ZKzjL7zOGY)94st&5hz&dw~e-}U0)b( zD9rwdDYCc(A*O~E^enP6+cz9ge}243aP?3dG-S_Tz4IqvC7u3=? zbiPxD&Q2}5u(;)!+r8H5=qR_cY`~0-N3*kGm+(iBbeXN1nj`8>{VF#&|KubS(8I4$ zqGGDyJQLHY6@|Fy%p@K>*VROMk8+WjD>&}_!@wh1hYTALRN3iVp*F)iF zY2N%jUyx?cpE+xPm8KGAwU|)x;fJfks%aRp_PG#CQ ziSp?yr)f3-(_HAN!$|l>y`z|^h=NM#zk$EF$*%$^1Ir`&yseKhO1c%8xV~}S3`Xyd z!T2eCWbLH)PK#1r6TF)l4S;U1iROwXu1-B~)(XS14AHGc-UVv$Jj0xCP7ttq@1;FQSU0IK(Gmu?;@THT}X2MjkXhD+Hcc+tnvYtVX0XI1CNUAXJ!oz$XZWD==vO~tK7G5B9w{D>wDOJ+03WGy%5vSdZpJ_UQ~d#nu)%(=?S`pr zWtm5Qp+T%+kb`gC{*suWYJW?SEoZyaLvP!dT?`V^N|)FlYn-33v^fi^F-Tm=c&;LcxlWD3sBKE;h1a1uVCGc+|;_2f%f@pCWNY8d&)9 zLG=FVuqlqnDb$NRvf*!>lCHH~xN`9cS?g824IuCBR_ z$pPSClhkiWj3RW1;yfY9c3WAlB(8yqsa8;1&hpw@}ECX~AIxydj z0^pG{W?*Y4K1T=`#d?I}2{?>Re|(f`!2X6!?A>n?yo$T_5R9c%BG;?kqEz%bF=~VJ(9zJE}PalaiSJ!2_{~2$y6D7?2-eLa<*65JH z;h%b()6=Uzt1=!cCsI-rm@^wn?;U(7?J&M-5@f~k4?R1hS1MUvb1@tk9E(eYtVgi| zjnWXczP$Ri1gIU1?JfHD&ZFPnftme`h%<__`oq&UQ`*MibI4vDuyi|{LME{Cg^A)( zk;hTvsEzbOk$z|0QJciQo0*Be5##1^zcU?u9xFF)vPz`|VBxm#Et?J`vW^)|8>cg+ z?^zQ~1@Z9PV=Dg=*HY_Qdhc{QX?tL4vB!+1LY~(;9B^2-<89$W&FoFiqU`eJoB}ML z;hM`59^Z}VGr4j3OVXY5QRJhBwB98d)n#Ck*VY`NumgF#F=&$U?J zF|H_6^&>hfn9ouW6y`QEPGq98R+}4dno4|oT~F>N*nR&OegCUyaZ8E?ixJuAQPk=v zQzBx2vJdBwr|~VTPE#r%ct~@xyxuTtPL<7_o|%?BSpNC*?5UAlR$}_Bb{qgeX__~* zu48yhoMa77)6VEF?GxAYd=qK=63z}=fJ}Z5V+JoW?_{Dhm!mha7A{@Ck1&DP#7oK9 zg*FmF%1AIpj6hee$2t{9^_$ZQZZ^%96cQSRtI}753A^`$-iypQ@h;&RLx?*4!B$^>gA`x!>t` zh=3HNN$G>v{C4x)Yn6@Ef#DV_-u$Cty>{A=9CHPEi-1IwHJ!Vq@o<~N6F;|LXP~J$ z9$XY|m(Ti?7(quh#RG;snN6gR{`4AjTW_s3Pa4pO(Xi8zw0~OX;wOqO~w5vXK(K>+|uC(V(Sb01(1PmA)2<;8>{{$#IUps$juVihb*wWVs>HhO2L&X#YsTiQ_FDLdKFK36=&@i8WYz{Tb*@P~K2W)sj z`Ff50B*>)A#~%e(ky}BYry{cp4PLAEIp_0R)kSY<83@UTx`0}so0!QuQ)enZjx6pk z0$k&#KvL|7JzQJt6I^W`KgiXOHuF5&Ch@`L`N|{d0qTenxGD(SyX?p~VT5V7f262} zbf-#?F?!_<8D|sH!)E8l!vKEB4mMh_zDsLxXEP+Z15+CxQ@`Qt#-qy1CK>|1Nxi1T zZ8T4|J@J)iz!}c3^WbJ-mh;c~sa!ZxY>ZL{EzTD7_l|<%W{;+)x*cZX42{157pAII zu_apvz+eP|8nT7R0w0ZNckY+!8*FWNAijXbk~*INa<2{d)C3j{h?#pW;_#h?K-!=8 z1o^Uvu=A#=hD);ngRDNT-}&>%;HXlvtWNmeH>^z>?Ms!t`9+=Rf$=J>yex@)0fV;! zd0_jIc{Aj^eR)f*W1hUYJ#U$y{szr_^=bQHpUWXH*9NVNPv3~(Wr^8W9P86>t!A}+ zoodwgTUek)0uXFG)zc_1DfjRuSM2dZrY>kPX<;hr?MRe;(bndCSUhQB z&vc>&`OF+O?J2aAHfRYCbcNCVTGpjB_1n^rj?-L#V$QYf33B+V0i}o6W^OBa(X8xR zWPz{*;qB)Y4A(3xpf}#bMT7`ost%3lJD*h*zH(K*ixx2KUR27hha^h$`Zi|Lr_7hv zhT|rKQaYc_KC;g!BW(+3v%2x;It3o#`Z4Gkt}at=q#~*lzxR<*D2u zzjI2YKD<~t+Q3v-eSyCwCIg(e$#MSI>DbB(-NglrfNFbZRr=)ZUy`;`zUTDZ(Bw2< zHr&FsZ9|~M63$D4Pbm~@c?hHXP?@=2=N;zeQBYe`Z<9&H=l*ORzx{YYGb%W{0d@P? zuP_ph9~&KpG~P2}GQ{$NKEMh(i_5mL?kAn%0g|&E^;(9Px50bk)8kh!Q}13-48Uuy zlcdk+hgFOq-czO=xkIDq!rCki${sgpMk5r!K z$BuHVWYc9$B((2###E!d0)MO2MTPi(lP62F0D>z@b;_?X1T;>l?*IeAQIwaclwk?U z<}E&KU*z3NE;3B%VqD1&l7Zu|5-p!dcHeW$khTAkdbHu{!G4gcsn+>jO#WDOR0LUe zWn$xHuvAtZmr+Pq`_sv>&+#)7FYS-vO|9%(Tesc$(4Z9iv9{} zm>Q-DVaq!a?vTdPxv{1i9()lo5MGo-2l!F$#TnS8A=pSs2v0XeYE(`YCJdJ$uUee1 z(E~QTK1~Z5cONegN{8XE>2C7cXb$Ky=(`J(ZkK z?!!WG>_*Cq&ks{TKykFjuaYl6>IT_)v>(YomfF&eX1e=&Zg%wl zm_4Smji#|%DyM2f(xKSgC=UBplQK&DtbZ!C@86`AUOly5nkzXor`6N;%m&>Bd5z-s=!g?CFVhI`*!VQF#F79@hd%cRHE z(^NLm6nt-MaulC}Ri~LtXU>JDAMB-v*i7<^3&L5Mdcq%s7_DFp*=PwX1|3tU)DYI1 zN^3V)ojD=On3E4va%S}E4;4qL#Pjv`==6x{si>caj&y`i@zGAb^q9Jyg_9jyc7QS9 z(d?W6Dt`!8oPv-za2ho&hIVQ}10i1ziysuLbk97NG>Jgx2@UP<;Gk5`@Nh!7Eq8tJlcab3DLU<4JdV ztA2oY*=XJ4lWNsAe9QqPw)2iJw)jN?05cYw#xTc2MEyM;9r(yc$Z4p%djzOXV$v}Z3(NG&WH96$o6kV zF-k~|-+aF{^0XukVSf)dm1AGkani|$l(@OFRH7#c{iJJ&nrz4^+!|kWrrLXpiy|TW zZzUw=&ig1?`NVpjF7B^<=kl$9?j@d*ZsE_7ei>6c#i?fPR8zn?knuH}JWiu&aKvxN z^hqmUyV)EPQ#&TuE@XQ`z8wCOMeg4#Nd!(mUVrLe6YF{mXEl0;DnKo{aZfbA>*`Hh z3F5a7AaM_EL{mt|Xv%|dtUIwh&B$SNtK6%@@fjgv{lVJB3#gT>X%mxj4n$zNQ+azu zsEIwCLxf#6@?X&}KiF*RuJ%Si)drS@o_7P1jh_9+#7~xk)Y7loF`8RL*&ah%>dwY~ z_nPa0mLe7`=kdYqvc(z6R>@_4vs_NPsIJ{#go~kF zTu0?ueaGAO*hjryKH)gFheDvAH&W_T{Z~_kDdz84nd(MbGlrCjfUr^j7hUiD&SoF~ zf9s{aI#sKswzjBE?afs!Drurt5UZ$>*dzAnW!J0_TC+x^M5IORMD3EIgs9ki?_J~i z=KkEr{XIUv-2Xz(9OwCdzh2Mh{CA%Z>SW?1F<=P_7VJ!H>3^b~z zjR_rF3Ee<6>JfbRX@5|OQP@By_%1MUp_w?6!`M0ZPLo^y3(a8h#8=NPtd>emB7vlW zj2Z@3f<+{h^DSVN%M)oUE@;us)pvirv^@J9es`^lJ9qp9_x)n0CM#7~i`!X$aWgyxci5jMlNyeAA|9&nN&)s?GFb+EJ7UaVT7#nUbIwmJKd~ z50CBsRO^HET{_>=D~%=Hw&aC5*_)h~xNY5D z+evfVyAcVW*nfY1?tELr>e;qU576Sw(7P&VR3ERa+z;9k8Eu}S4XiHG6&6E=Q`!L& zW4fE2*^4y;F`4IUVF&4v&VSoXlx>7&*L?WA@Fe|b0YCU^CL-jE=290(8S-{1Ja*C# zez?_;BHHNk33J0reAOhWg1ZuRk`+Jjg)S@^cY=*-YgR$ZVAlIEUuFF}0XN%7LX%SB zVZFXTC<)oD78XB!PmM3a?8RFAhvm_N?|KPk?;@HtX7RUNlyxSr(rU5q<@_Y zI-iG2dwIM&(@$0TJ;{qZCqxz3C`ZA?MU+AZpu(s_U=Z8Oi zlEoACo&gCuIh308`BD==F9vaMM^a+@d3;?svyDRcs%-!z*7GaE#Zmc|P)G@CIL{wN z&gP29k%Ae=x7!9*M+XZEpL0?+r7b_b8Vn@|pC&-0h564R{}wR~T0U**XwNofxBJO( z^A8d%(w2tQykFW1C0Bnli=_u^!sh*}pw>%swV={^Us4*gcjWcSxn;iqkPImm-kq8_;;XU< z|0{`tm0(>m%1|Ez&-cN(d#Ca@DjfRd#`MQbj!;SjkBMhC1X6!iK;x`V!c4nG@OzDs zpeEY#;ay=ftAF3xI-X{***$weE6AA3csi;vYcbe1U1scoQogj#y()I%leltGf8W+4vp#+)mjOzL&AMDZZK(J{_fcE(JXp7!Z@p64{2gwGNu+I z(v{$vSi z-Ir*&AOw@|)d*!7(N?S|>&D5|_W?e!ys3Yu4_e2F$+7tz|IWbJ}FHV7E*Z?9IJIORi zW#(`xp`VqPAQT3bUI~P{x#lL8Zy*%{D$?;{v`e||8r(esl@&{hn>h=YXaZNtv`o0A;Cllfh_{%5y3hBPN^FBHJkLy;=)9;~$ zW0$4KcHf4-CDj913-klDPsJN=PyNnWN5|z`ztpY|?=P#ZKd5`fbW6arDsj1|gfwVi z;v=rrgh1=na;r%NoI;TcKJV6>r?9y*m6d zz0`Y4%@&G!rAxeZn+bUD)9mcU1Rw!fSiH5>pM5hn<*ca#%v9*o&$PnXLFml)v3K*~ zP3Leg_mu$fXo3ceC2cNP{$q$MU>VUj`Qehp{EyyZnxJEb-RAJFn)ZhUZe5*ppAb>3 z%Hf}6r-U}Jc{Mszux<2JI1R4aNa=9Am(2U8Ohj8GR6#$X(?J%;%|suWF88y3c4E|SkvZ9TmZgAvCqtCGCR&MzK8yI#ATMwNdy3p+0o`5Ey<9Kq{T6M5$8nDc@h zyL%9`@gCo8Fa!%OpIy54|FAAc6ZFo}2r)csfY?joHeUjlXZ9@FqRZZ`v@iPZ1LN(5 zCO<#Ua)UBmZe9Iku2bg(0qDAqtm=hyyE5BzRHr^iLd-pRC{U98cu8I(%<0x73RXJ& zWqz>Kl(WdyIZMPrh1mv;hxwO`wJ>B=Jd3{)E^P4nCdLa8< zo`XISpAI89TvdZSxWX}*^9f^pjBUBY1JojAcjaAwPBM3!+mk+v`*F_25%VJ;tL*z?9Uw%c zok~zaIIZvIw1)Ct3g=QbvpcQ&JEu`ib>?z0oZCw7vF<*N*c zq*}(@3H#8B#2Y`%awHMX8YU1XlakGo0^HUa8IncF+f-G;nV)3d^(=AuFjA_|6^nHy z!UYB99ymQPP=&e3D+O02fO-2AoQj?y2wej999PxggPMy6@$M(|%Ird-*1Ua&x-fN-|ff(!5_1Im;8-@8x;6ZApRZmj(2*)nXX-uSCBx0!>(}HcntBd&f{K4j5du zvY^m4rADPS-;qWsIg&6C6pSe^%+e{o*>bzd_x2@`sYeZ!Z{~ff8B@EJD&mSaoO({3 z^FtLEDO=E$`^A9=f@NM|hjRFv_`u72)ell&;SAm{ChiJl*)w=~BXp zv&tG=T?1!Ypr=gHI6+5GW2FB`hADB7w0W^h@%@j%g80U{iy>8KJkR@{fC}%`&7HKN zx6?^oDfyR2tQ07UH8=tx{E>`^LG9ec?qZ;vP!8R_6yh-jpCQ}n3_^PnY3Em91)wG~d zb=Op4$@m0pP@)MndvSyyXKVp zh&&8i^~l-19E%H; zQI55649(*Q%reeuJfD~1!L!o_H>x#Rj>o>syf3KBnr*R=0Xj9fK&+LCl#jXDUuCva zBhwml3H!?K_IaKM+NM|DmvWjm%{=JWayi|HTGkF*Dbcn}-}YzqdV|?#8{Bi@a?7){ zqepQBl^wsJnQz|=mVhUs?&Vn!cN@^L>YQ~nUiUtk)-F88 z_`90>^v1dWqa7~B)BF+* z$#vGjiFBfsv7MgCbZBT+^Fc|*w-W-uM7Ym*wbgiuZB7^@FF;+vI5y)pyrGr<+gty? zEcTlJ4g9~17O7c&U*aedPm?yb$v;=r>Dt_{3>fA81Su7!_h+$gHhb!$|{*&tnz5?E?p)b$^O zL-Tuu3you^A{wzJ)9{zvgaY&A2cd>v6>fW{Dbu^)M0N9hjWw`uNOKWG;OY>3^EZNt!;`aumI zGdcw8{mdykIC$!g<%`qL)Pv)ovcE28xu-tfsIuA5CIru>Lc_2b(@)36o^|FF#P_dt zP}=PZU<2Q^0E4YFaMx8i8a_@hSQqZQp$$NUbM)@HnzEZ(_Ea#(EE^{-+b#tm=$E9- z`uGmr*(8DURd$#ZB5W7E`lXXgr2>yW_ITMiMBtbi+|QMnz~s&9Bb98mufj`PkXr4! zEK+@&tC2o;pw70~IWM{8fy5p5tOI3xo&{y1a`vf+z=DXPbMa*9%YZ~MA7e}|pfZ_{ zDK;{Ftnzh)sX-NZFKtj%W6w<>+xd)i+_>}Hd zP4z}~bDlLxO{mLT6Q}OaQVo7=!xc=V>tY|o#ogd@%cc_h8 zhmUqo-i=I|)&ai;OWIJ@8l-Syba%3+Tf-xt^e|vd``CV_hAOYQpRhuEDd#3=WuI+C zG=VVsOYxLnY0n%H2-|%6C{C>}zHUd6#a8APXS)f(QrmcC`fI9x>HBi(dogP{yA4qS zC;Uz$s8PqNNvQSrrsz)+HkQaO&&n~kgG{N9{!Am*5aHPdXuf3k-SkHsug(^S5xGvC zDhpfh&vd;Gv8PLWe{W@6d!(OnGdQ5Zf@UGKIUVPIVoE=Usn#xXNLt19dJ%liR>P1Zq^I1&5YN!^*W^jDfuY>4ID>Cdf>w)y{;%W2crNiHgkP-@lg|K#6y|+#KG% zXKD)Mx_9I?C&m9ZbYr*s`-dN|Ky5+8^M$C5!XxNB&_WlDwCr7TC@(A@Z~D}8ClPCs zuMS897ccl57Ei`+n!4LKs;(6di+UY2GleMZWYXR~Z zOZpVhA-O0(^pN#emuS<(Qbt5Y)4ouRRr#nryPHr|U+3Q~=hmB%5}>2yk~m zM|yxvYSC2nR_fC?SXN&5YB}x7{W8ZiJ2|-?eslBeT>PjK3I)y_4SJZX`i`LSNBdTY z5#MpGG5yP)!Apxm71D{k&65%oe{qpNTa?W8sU^1{5BDT1AK)^*wnIJg8=zM&DI+do z%6GOdUik4Zt1!rDj{ftK%mN?s+pVlQTtB6+WCf<4bEvwK4v{eR6NHug2nx2NF!Zr^ zE79}%H2(7sk(NH)em;gb3R@`JR9?1xl?U!WhK*4R2(8LHX2X#}!OJ!@1DC0#s@Cf1 zZ?j&w`ptVcB~9ok&jn~h7>DyFIs+$vmqGk!6Zfi`EuKtdL(N4~ccJ4kv_=c7DN~FE z$rx8VRwOVs82q*NLrZ;;$)!pwllvW#I_&o@-C(cKS&Yl%U=fzE20LX2BSfpSP14;| zOlzyHHjJ#pl5{Mqz+-ZK>ul0Sa9Nkk?1>!va7QW?#a6A~^q89D8eo&CcGrJ~{OQbM z`wt7XVvvqvZmGdPzOE~!Q8aV>?W6mg+u%6t3#+5(E^YQ7+x#81xtN_m!JtngOau7> zPAc^opljyVEW}sfOCnMZamiJ=*J$!Q3zsqen#&qobGNSEw-w&58*fmeX_6)ME>ZEuPcCiHc1q*HtDw8byo-+E54stIy;Yms=XRfk#Y7R8QRLvr zm)u501h;zlKihQqg?K(_=SfP*nx8lWa>}G8{!p%O8q*&@e)h1xr^ebcUZ15@0Po2B zAmRIv$=84^y87nGLCd>%!Nu*0;LprG>SA)HC+ftWoY3}W?;0RkmFYPvaO3%1Fe{P* z9P;u*+U#z|M^byS7%9e`tKY8>%q2DWTq;+z^$^?r@1aX(zbtDttkA3Eb3)+*E*9ZO z`3CRr>VCSUWfIoL*ooLrc6nxJUpw-&PtFUV2^)%x61Fq8kwpRGQS;}c3jObY20XZ_ z5%ftr&{oLh+~{t<%}r&XNzd$g4M#IVD_2p~yK^VrdvRNn6P9Lf4xC9z2xE?5QaI)^ zTN?Ta#m7j^Kh(Jg+_^8^#Y*_zQ*L0bivcHIeyzn7=uu7f`l$AnNCG8!e<|vP#PXx9 zbdJKNuB)EDIPy*`$?p;o(~>?`>%a2Jp^U!v=R5?!<-9zzas?&xy6&+);1F~cR2m!^ zkN5J)Au%0ePX%pO7T4OtuQ_BIR<@BfBP%dE!#K*NkIqLdVOx-hrva58(UVl|sH&^l ze_ly?uqsj7X-OcKvdmMtj0Qa-OvEo!UenK}DZVQ69$Iv0>WqX5#^R@1EXq=LBG{0k4rL&Y}6a__jVNy#X+d^91yR+h%pb@!R;>O?I#uG%z9fFP@s+zIQD)iiSC%IlDi@ z*eb*>PtSfu@ew-T(8sSY);QJuijhl#SIIA_rEoCbEp2G-#{jS{6+txVOsSrj$jIdQ|N^j z*eIq5={Choq8A}S2L2f8X+4l|MCRvrR8KddX(pmg^o4jv-Y7}9p-K>tN;ARK*xlQ zV9s{~_xmY47OGaong0ELr!Kg49D3u({AGBmRZn~Q&z=I~Mpd5pvdz5Z-}Qm@!=+g=UKzEaI8sz z)Tq>G)uBMBhyK836R@CrIwgj>oCedlTgB`!%Y3h(jVsCgYbK{-zHfN2m)a(+rjSd9 z>#b1u1~$ziDmuPIU=nmdA7M2@C@RaH4)zv*2z3@Dy>}Mxi zB>r@MTVJe{pCktG-Mjs;z0R&O2ML@wR1UF`=XRG#%2hF5X*<=tN_H-*wBB038l%Qn z7+?YTHYk^?Kz>D$`)xR;PZ58UOs8AgGs3l5L`z-II)2N_B7a)FmpGSA_uOO_rytsi z02j{u|dtn#Igi+jM|^T#($IutHA>^PgArrSUXI7)`B8~;jx zZd=G+d}(>BHwHO>C(XpCsYfP;Hzc1`oD@&6?7BzGytqK6)lOCTeVeDW$S19cj?A=#Ru!(EcazP z#JYr9=#S+7!}$}ZPjkuz;>L3v@}mCpC}NNIalxseT7P7SOiK^9I;gw{Ed;K;GdBk& z1tv7PDLL%hZ#qEyQu3w_Yw_Wtl_RwoRpUV%9QbkXk?M%KAItN*9 zhBn&3xb=kXf=*n%dyE|0x4Gu?M~l|=_)ipdWbUGO$?p% z=uk>X@o`;6*y`XcB{_?W7l(CrOe!WOpd(;KkkB>Nz@CevvUdy&w=R8W{LCAP(Z?*< z6bm5r!~_e0ZyP0Q_D@qH|L?2n*#8iB-gt38>?b5`8TnGpV+Rmy^dx(liE~w2A{&6A z6FNnQw0hb8#i%lox!+4-{5~fDg)aXfQB9F{dm?Q}7P=D~CU8>HP$#C&*5@XC=;QZ( z$=*kl{bhH(c8`i`v)}EJ>U{HXFAb;0be5y=T$_ad7&u507nD4D6g{eIHm_Om=N&cr z&(o27s?QYQIGn6qkyMg_(Bo8-=tFxu4tel-N}7LSnb4v8MD>v&$Rdp^=C25-wjkf< z8&whdsJLo+cV>xW-b~BWN%=BI4z7Xw;^44VYN@golIPPlv;v0@)^ ztzvc;22PWe8Q7>|8EA8YieYrgpQm)_MsR_pns`X*n# zFmr_%RI*v&yKh>54>4xATk1SHN}os%W5=b1PRWE4i1S;8m1^>ws8ydKcOU8mKdam4 ziphjBh}FdC>bAvmctO5P_xQF}-rtmeKbhH|n_V(3u#a%*E;U40Z|H1T;H@B}Re!=! zumz-V*(gcX2R;_mtj$+!KFWPrs#&F$J`>W%% z{nb>{*f!(x4Y10S?DXnHKQMI?G6J)4xodOx#$BSW-xm2w`NHA|c*;RcFjR_9l&bdp zi_434i}ehEF|I+;_WNG;g3J;ZX@;@z(L=VQr^f(xkD&1b&WR4F&CIa(iQ7^zhMA4@ zq~h_4B-Fq8F^O28Z-uC#4s|Erxg_xco02Vly&v>%=DRaqzbgBy$Sb~p;@p$ql`2#B z-tG#MLVYx9bHjyvg_Y|>%+GYiBf+?Vwo0?!l>Vq3j%00(7I4~}2~f6gy|7fH8%);7c#o$g_oSE=4H>%+`5-k*L~u@ z-*@RXv(Yb+Yh%7+RzZy(4ivm;=zgWQPg>Y2NX%Su7;BN;oY8#DRAHLhx&h%3Oxb)V z(RZoab$*Vr$w&!ffEu{<=({*?c!1;pTfnulVck!{i!{Sp>q#ql>R~YUq_JewEaQ!0 zo{AeIkGj6jO5~W@+qs)`uSqlNPYA(pfj~VnTzR|1ws+(aX4Cf{gF0_EZByasbKc71 zTe8F<`;?g_22D`A)1`l|HmjMm-Ug|g(1KP9Ev78CNSQ<&Uhc8cLQ+Kza#R6nGg-5t z>Y88natZH2efky&LRbD3g8qHG{mm8YKx=Phjy}^;oj<}DvfBmM1#Z;EO@6XdbjxpZ zf;fUBRGU;=sPCVYc#bp>7rzC`<=&L-E0Y&1Blb2CgDVD*BAorATE} zakrwBE`n-y=s+XuT7be+jO2&wiZazOsk17J4-l2<_F1%vl6J6)ssG4iHqdcIRv+}H zJ}+-Cl4`ZT>@7j>0Tr)j2CMqW?Sn19trm&B*cbQ*$hKCmC|7qawv0ouls39}S6p9G z`p`;6{JL0wdJSQHq@jqIw_)Ch_6SCODcLZ@4lU0^s)ogUgo;gaXNzf7E@0y=sfFTf zOR846ZT>^{A4omHaN+Mu460S1qRSH*cIPuUpxC85Eey(LZao0e6*pGapNkna@em)N z#g|i+8yEXajy;trauvh*ve}jMeuOPacXRbLB+B&Ap09F0?&LGGH1AW#u_(QwaSfNK zWbXl3LDx@&gpMA5T>9zv>z6FE5)lhQV_3tj5S~AbCz;d_gN)~rTqvDvKJx!a0nzVBd&I0ywWTk#Yiu&Xw@m2o~GmcL6f~ue_Yw{ zhG7UjCVI1f0q?cSR$}aJ5qSNYxqPXb>GW@|ANAY* zi3qNkk&S6BMQDJ3N1~)ub-(!EHw&scqp>nwS7~y~nPUClCndc6nIPy4N8>Wb`cMvU z@|6S=`vV|M)84S$`f*fxmT9LB`hKQ_%Ik;)jidK69?WX=&M?>`)>5_|RV2hK@22dT%u7N%>* zgl7BB6F;)#CpZ%5r2f&Bc|%=h@z7iuCiU=91btEB>Ce7h%W9PP_2Y6EpYWauS=uL2 z+N-7luu*QzI+4rk40G4sDmuwb3{jqSXO8c*^Bk%Yk3nk#ueWo&KqvITtpEd9`rY8W z-IFZuf|MlD?)CZgP>JdO2rsBW&{D#tc9W6j$5-ovmXm<9%74l@gG+Oe9JZIjY$p5K zwn8CNit+9-w|QyRCO9sg`v;T7xdmVm-wfh?d;XcyX(xy@Eoj1SAX>! zv4i3_u#)eiwYEZ+3u$?xh0EiCT%%%^n`R$hpLgC!pLv24(HZP8i5~3VU(%tgMv~=(@Nc$J%{^XX?ME&m$fPuC?cC z3#}Jj8=0@B@-YkPU%%oM^s)T87s?lPf)=3^*C5L1%5jt=7+RISJh?w-bB|TcLC=by zdovo~S-5a0(#a}X^B=>+jELnVfs#jCTbH)5F`u$58z+5ox)04ymcO=>Pq0b&l&`-{ z%$DLe-np_~)y1v>LN}%Z_TaO0{&cNR5hPk@Jwvp5DW+SvXC+uv5+xt^bM48-fh7qU zHPr=y>5*w|4tvq2&)V)jziS|hzp8bA$NV5*6AMX#=0kdfyTgB4J3&oYnyb({VClzo zLvmwWV#EI5td!?9X>3qyyqCvkpv=9xfUd6c<}qGrB9Fj;n)0Na`fs|IyP3v#Io>c~zBz&xyc%*oB z^WxP@TedS$l^*%EbARqFe|8-`-jnI6(J9@^#*}uxI|m{mfwNj<5#ZEQXEtx_N!&xq zg1-wj`rAVP7rA%#N3XRo4k2t_UB^B9lUE-VaHmPddx2A=M&R05MU@>r14GcHoT2;0 zxt_ZIpo}KdQ+6o!=`8n)sAj_-%exmtah%F&ve){K8%+seZ49|w1a+}cwCHTWt` zNwl^s4mn6XQrnY^{fbFRbO?uioDl>*P2^rM%uv?Jy%xgvWrcdA@nlF22kAVW{*R#p zr(w)llUX{dUXxpy8f!6F!PR@?g=tk>@xK9I4{S~I0?hkZ-(=xq z@E{5fY_OQ}M2K0Qqhx_YxECLsnZ+u|!T+b~Y9baW&}#pcy<8|RkI!y}e5%RwZ+qGC z&nsdVhj~}IPs$8(#V(G-e+qYK_ehp-rsaqN>jyL*uTJuj2-`;~)r%C5C+8EWUupdm zgo#ldy~bW}*+24T)9;^Ot=sFRT)la#!}_qWu*E7H=F!~R1vdfO1tfi#uxP!rdy;?k zjjXFg$XwL71g@Bp`fp}o`VPt8^_FKoP6uLCnD)HIO+-vuhg<;UY4y2R!w-q?Krof@ z7drltKl)+(ebfG)n{8 z#dDm=RKoCnM+1U8U4=Cb-{%t%S3k%my0yH~x%&KQJLqu;u2;#bS+YCqHZ@r63`m^F z2?^R)bg2sSBkseyHym-~P;N9oSiPcI=!#eLbVabJ6T|5(r~Guqsd7OCJ=5<$y4rxF zQc2k^!M|s+Jl%Sy!h@|HLdVmZQRnJwHt9K!^Tx~)H*9xLBQunj@?Khg2oLCDeL%4? zBGJ12swVKnQCLxvx3`NaOt-;(<$!?nC*;@j2^6sYbcP}eo-=;%ey$+9i5t2*5UHTT zSj~U}h^*oo2F3O)sng$5PMIBE8%Ep_> zAGw8|SPbCsg?glns`U}WeNP$Nw0Fuarxnb~4{t?;lcYD9w-2?g6_7-lDFNqHatNUw zQb4T9%2dOK47}sTutNEI_8{=IiE5#M6L5YJziu2-{CtRx|GJT!uy5`MGfa_C8NupFnrwybicCz(xDNuz6K>Wti>E$jTy=DsaA_%F)Pn zIqQ#LbnD#XvYEvKsgn~B%pWfS2%)K05XaWj8of{YH!RzvGKZ!2GsKvA)bsewg|VZ* za9ChD)LmWfTVr7WzGL$5)LZ3SwvCI(E5W(yBM%*e?N7;$wn(V@XI@XQn1kJ)OUgf- zf=u2nPi~531q4KqNTWqWKYz&!4_?D zkdIX;0CK&Z#Ln>_%Va?+I@dfzeZ;fj#)uL?7uJN3tg0JrgNrvb@T0hHB|&q@4kL3W|F?J7ePmm!?4t^(4L7Y&DWe4|^Y zl$Uq>HN3s>k1Fm}Yz#|+v26wa1IYgNN9X?-LF>sV2{|$ci83>FiT2Z7 zZsUnIUwtzb)2fX-j*|3}jx&ReI#(b6&TP!wcNOHm{VXxxr z)p)h%1p&-$wAl9{&!tA!agRpt-${4L9xG)J&Fk#ti*3l=zPhc$vG{>~Agf`oY3nL- z8gBoMz=8&v>B8&c@qmCJY2uPfa<_ylamDIMit6JZ-_D+YwB4l_OL77DBb3dyXSIF# znkkY{aK(Sp@Q8Im_{Z7L9jExFXFI@-N*9$VBaqXR?^lp3+Ej=#bRGJj&6?HN%4C|43-8y4mE@EiY);ehAo`r&EfXl~%1nR>} zR&2d5Zg&@@J5-3`rK&q-mz;ZWPF%wskhFzLXql(`5Nx+YS+zb4dC|vLc?)W^av^t& zapLJLi4g1MZwqmx)Mis7mxAol&|}Qe~{$+a3G0DNtz(s?4%Fz(beG^G#hknrS*|Gcp_Qf@=eH> zfR$egx^>e&b(c(7D^kuD797e>vsarNC65|LtnQsiSon>4Q1J4DiMMq0Rr`&8{kzrO zrQ9MQeDwQ@b3Vc?$femvBY$xWu3U?qJo5KGQy~geub?Y%-U_xU*0BpSI^e5HMtPps z=lL?}7Air!d+$N$#+G@|lr91hsKZTivzQt1oA92paHD6dP^ZW^^1uC*2!ID+qjIn1!lvd3Cg4w zD^s|7JZq>RS9mZ+WL87t7=|0Qng}?ZIm|OPaN9dBYV|}}c-+y!e%jn$t_^(c63@s{ za3dA|#$8ZKL9RL0ONyDy9G`z@yUoVDG0zF5vmK+XT@h6MW<*^2E3`+|j>SJ6-YID_ zzEGxFlFn5_+B9U{qic-aT4=3O4oSv0?rLs!fnAs9v(8K8hF^-blzq(!B$f-5 ziKd<^q`TI;QP5ks6a7{mT`)L@f>u%$2C2AbgZ8Xhz=J8-Be?7rdGOkYa}oFPs>W%= zz5ae7et#kEzZ4cT#h}hxF4zzd-6K)N`*qmHw5T3XXd#gX9N4p(-0)GyZ>fezxSBLZ zP+k-A$gk-}qpG^ZHXoS(WmARByOpR>iKf<-2oGZ5G9MWAFZz#X$6CF|#9mZ=?LV&qRcC8+>VF=pGm_HToHb=Qn=gNliT=$r*{ zL7i&L_j|xw7d`VK`CKro9?iRFR>#USTXD5ME4(uFNCWfE6ES~G9lmEt>N7Yr za+x5eWqzk(5?f}&@X2U31qm@c9KS)W@yz>%W6R2{PC)P@dL3ygi$^IW?trctzb_5p z@5L1@06%>v0KHA;QU?EC{|lsXkQwvOgZ|38mzcQifp?da_LV}cM8Aghh4ftPJJ!RE zJkZs(7pW3+!mEk^R@7$c&;k{P;mioCA2YWw=|g@YjYD6~%>|%A%hRdDv9}Xduo!Yo zpYFt3#MA2%#FGW|My(7;KK$#SWkOKL*{7%i74HQk;Lyc*PIyfBK_QZmmw_aGYwpmY z8|5Q*WqDeey3UQUr}g(EkA`E`@4OV0c^b%z?T>5u)nR0et#G#qq7$GB1j|rQh|5Z& za&s;O1QAV%_$jR^+QAiVk)doF!^ke3jy58f4uZ_Ed z=8LynH$87S8NM(QcyQzP?*j`@iq2C>ySgY>o2>TN?bD*hui)L~SMhy~Vo&Z;J2wx&$yw>3Q7jQtpkk*Y@$;ItKzb!C&ID&(XKgs>pR<}71}luowTlD3zLXkCE8NM!Zvmq)p3M8p?!boR1&im{c) z`j-z`D&ljMDi3gy=H?FcCot8vnAcq;mga;sZQfG3U(R2nb+<#hwpdo2RJH}*1`V^( z-M}wQF+RVoI;QOh^in05&SocQ zWHmw*67Sp?wKWWse;{qI`(fb^6Mi*7^Za2LN4s@-nU)UM?*2i9v#HEs?F(aCLGT$z z8~!PHW%l>(>G@pa>2Tx@S)=EfU+A|X&7Hl#hI-#5o4(o@^laWN)H{g3K1thd4u3H; z#(Kvq>0m~w(}{mX#=Ut=CC_P93V5B*d$bxLP0Dik$zk*rA_6N*9I%NI)w=h(-!H4O zVO^hJ_U;@0G8?=K=nfbQ^@Y@L7i}H_W(0^2|MPON$bA+dAkeqdI3D<6B989@(qmuj zZTCFZgd&eIMrBk{1B2f@z^i8E#^lFLDV?)u%)^t8EF#6|_pj8WL-p*Cj&iVM^Y#V! z+lZlQz3c}jbF()9obKskD5ao^E8l_#xfl5>36#a)UqG zC!4@yMT5+J33S@~IPIhIZMBcT8m)Kmhw0bUzDlpQ)+f`nQ&TL`jh-fhxe=l&i_>gY zYCkOnP6s4ma+fogef%pjw`(M++{WrdiqeRb12yEBG$ z{|#5TZklG6H+V^*YSyf&T3Drfu-#G2R3YKQhk3i4J%?(M9#49c03~-v9aRmy{-RL8 z+oG;iON9p{5vZX~WZwvhwoy8axqS+6pHpn^zuBq&V|)3!OX+CXESD!+Xk?E9T8j7? zt%|W{W{kF$5VKa7B#T9#T#@yK6GrMYy5~@UJ-?qAb63CMt+q2)wrWmc$zHR@ob|;; z`K~#)F+^yib6qd*6s4ZiUrteNfA$jHgI^1sGO)nX@E$F09(~SsP3GVTme8edC)U4S z3j8@a1*}msw*jWLpB-$~rK06$PrI zg><3AO?|`n2GIi1cT+saq6*idCGWOrgke&f`@~tOu8yGJP2A8fi{Z#YQRR4XS>MTqkA!M=O#YR8qP-poH2Fwdklp z^>_(TP~{9f+Ui)=lYTCVQ*)xL$A;Ktt-6Jw)sV$1ZC&XdL$;LMl4oKpt%H)5%e?-W zm!0OXXtB})oZ2-DSEa-Qx-Alc+3v%I(&aGawa^V|*nC}mvde>nze-kmzg~WSAU)|C zo{=oK@k5)FP43eT+3T#pn~zX65K>~|L6nEsxVf(>)@z|5*II}NY_(%mW`k4P0u?nD zeEyaGD;*GIRPSi0{{OJ8FC@*&-Srl@Gom7QH?+`>9I5y5*CC1O6Hdg3U5?9hgRyL- zLvyh4MxUQD^9WaVFj$IUQkG;EhBxSwWJU%_!3MrpwMk9F)jvQRnJ47|#P#7~bp$z8 zYJ!aOw^Qje>>s9|rC@x5;8S|UiR_2{GgVX79xHlvg!jlQm*mvv|3ZaK3my=!@F;}F zn=@c09O*r#|4s(YMs+sxz3mx&v-;4)X${AXcXFoogX)U%hhSi5i9CN%z6-{_hpn_k*N8$(D7Z&VVh(SEAv{k=y*|MrjM@hj%YPPWM-zjD`S=GsJ>pNHKKnC z8=nqYdN+xJ5=y!22b>;~+CqBOp*NA_l!ZYX5aibex@Cz3tZGG8rDW8^T%VU!z$PuH z)f{qrAlUPA>iQ%iCSo`~AHbcUs!?OxJ%=IHWc^m`!u`v%AEC+Mf4!nInJ-?UF%FEy z#u%27iYX7bSG#j2QNpW4ls~2g6$l9+&i3U`;ll_sj^cHVZSq7Ym1<$suZrvG`%^z8O!+d?b-Z`|CX%}sQi zgX{6)L0h&>2vko)s&-<-P?FnoY_^eeW^-n`zy{My2{s;YgI9m&ilZtDP4fE}uLuWU z%);}fwDc?+^Rci!ziTYjEWd*zDKk^ShW7chx= zLE-2@sag%4E!Dc#6s&H`apAJ7(v`+f@ka%xXE7{E;1iWCHP1$0%9O$8apb||rmsLm zHZqK+)jSD0yiw$mYMYGUX}{X6&-cuxxX@d?-Qad=E>^!7buba3<9N1IlR)b;0p=Gq zt^(Eu(Y39sN44Xp3=)#q1NA;G;#@Y128Gq(VA)$xf$#0X;G^U`N{GE9))9{)o!v#Y z2LG4KFz43#CTN{uLe^r!e8$Kj0+rgGK`FJWxtICB2+=N*HE3_$W~N#Tb#R&9{^-sxP?)Owl*5o) zCKl&lX<^fh8nL%oDyP6Bw47`62Ii+}#Yx!KUY(h&ym~Ths@lkHD5=p!L+G`v-rJu{ zA2)}teq-_fzI(O!<5rkdu<_=~n&;V+T5H|0J<^h5ioDei9Gt9;i8rw~{cPj7j{0CX zL~;%ee6c>b2{JXyIGcW!d!teeAw_@{_rknjT$wRBSt`3VRlxv9N1QX%cWPv_ts{Tv zsL?oLnr7*~Q0V7Bv8;Xd`%iwakf@KzRz%)4%sjrHu;bXmnbg*ekyR`WCbcx}!}Wiq zTesxwk@~YU$5h@+FQDv1{;^1(dgJ|?=h&|G7xRmOsnnQwEoen9ZyoXv6Wc`8dPN1k z%GzX|a)c)CI#Vf{byFeXGB(4o4Pl&s1&~}j%xmoVvWT>A?nfzFInM}cYyAd95sZ3;1vNMU4TYk*?%4{ z$Nl57_XE`TFkQ;Cq4gQdv=f-2lsTTtk_or?!&J{`0!2&?BT;1J-SB>spb1(T3}1zU z?ZFWOVJ&Fkto}P2Pk9nw3b=Y&-w|d6V#DQBkjDUneD@rN`uIe2=p> ze7%J0A}b~UK4sQsXg()}5PkiX=!D7r#3ibBFGcR8#5r8 z;lK5$8T&vbNskR>_1rs6VNMGcnE{U!%BPYaOwjN5s_YF5n- z_eff|?F$XlazrT*-OxUWFh0`CNQnU2cbSt4^ zRvHnGgVmUaNc4Jd9=CE>e~uiE31I~oP@Dk)Ycm*E&Vs(MtjJ%E*@&qqG@l(xZ)>$F z=OVli{N>w;^koc_t9+4b>7$)Dcph|&dn0dW_kpE&9r|tF!E*W{JWT8EOk+9ip3Wbp zFv$wFu~BRY?`>GE$DsN?&_Xd{3hJ%60p}?9A$6g(TXeM}iJ>^ON5<1EBavDilfrsI zsW-kUMUWx$6A-Lcf_iZ8yg!%0L90J)EF4s;SIt()T-e0QI!+KaMa%J59YN)_K+3p( z;`(t#C_W)tre|r`Z7vjqi;>$b8by0TUB*Bj&N7=Lf9+hl4s&hm@M6BX`m`UFd=L6a zV$UEqs6NNuz)_=kSt~|n>Hz|5K6XMeHo=JPHMMzM1?%Xn6^AW4SJP&@3EAV?&qQ81 z#?D%M0oH-KTag?UnB6lCKqi3+_&-pfs?am8M3fuTJ zI*>zp>Y{b4mc%DLIKRQuWqxa3uE?qZ2?J2EbAMspb0zEq$}iDZk92x*e#!ntbPua} zECW3x(8=*?NP^tGcpR$%k=D_^7YbxWb(rb{Ij-VR@q0?`G5osnbjjVDTJs{UgdM4l zp63cslqKhGqystVykJMZ+7acrly&#zk_r6qp2vf0Z{faIoLA{WE0?8<*{i5hZxk9N zVUKvy{V|evlZh9Y;aaB`m}2?Dv|OT9ZeHzY@L z04MyqPI&WVQgE%+b$j4s;dc32r|$iq&7=yG?nzZS=^pd5wXdTN?|o{lzjXJjn9S`o zfZzG}mByA@D3*?oMw>Rr2MQhEUw1 zy2A9&%nHfcrBwi63{j?Q2+BRH(5*cm^eNl!ES}wl5CaqGrFEHu;&UHYh-Q|KEXInl zhFpd+g=fE&;nx;;op;a6=KyQ{);4~c*P$vYxGg7u5OSUSxk{AvxINC$%yY4Q&h!T5 zGu7$-uaq0synal*I)fd|!9TH%D&}u2-Ah%>TiLFgE0?%r!hJr}dDt~6yEm0(;1gnQ zh?CNjP0jPJWbyxtmh`Dw>|9U?@|oybxVN!h^}IOQ?QK`kc&0r4-|U)Fr)KtLsbOvK zlj>@fnUsCMtTtPGxoo8S7hUzS)5j;ntdW}J+&gWir07Yk_LJawg!FV7hr+wK8tnIg zv;Mpr`r3JYasU^3flaqA+t{Wq1{L-J%H({b!uLqhmdB*gs?(KS=B?%09M)?DgPuzr zK=jeQSe0?ovX{q+O(FjNWjX4WJEEWF?Gqx!ezLO0beKqigquh;%K1|rb!0%={wxPn zEucGU9{OLjSJSw!YyG>pUqOu>GU0n)U8*SlgG$pOhv76DCXr~I_hiUF@%)y}o6nf| zzLmB(+8_6S{11?R5$`%tQm|T<-aUGA1z}m^d+R zRe9iI01JG4(pgq<27Nf=peFO98qncc-^~#1&<t(<50W!x~;Rib^UJQ5zI($r+W{!jOp#i2IPSYkY?pCca|?9skK#lKs~RW5Su^s7ij2X?_ePw8l1l`yqaT$0`%c)b};c znyHle)m81=hL;3%!bzEK!VAJ^9#J%pRB`?a!^s0yo}emEgEQ?-w2Jdk&@!my#JeS@ z`*gTl^kB&6DoA#`^dZf5W#VShL|HzoMo%hj#L$Ai2_3pM@teUyYSde5}c~$!r8-4z^ANfH`=> zBcU_8I`o+n9Xj7UE;F2!F-~9iJRcO{6@0r+Qx&*B;@I)ppFsh>O_3aWr}ThlJ*gM! z_qK8dtSGnHmLHlFrx1F>wbP4R^4L>16Wp;-Qah-QE$P3W=rHP^$##^_TUBae7XiLw zy?CCq-P64({!{zOo-gw-F1BkZXOOXLR!QpJ6YtnC8{AEPZ0Vtl5dq#U6P0;^C4gRum6ZNELJ1Nyn24!bY2R0Im2;#uxv|8EuqGCvbfgXc3c}8F5Xyq zmrber^W)o|$4{O}xwI|1a1W)!43#=NoZ7+uDe4Q}ftv5ZIDm%D>wF$@5JGY!;EmAP z!dG3%ZNo(MW|=41e1bQdpO_n`n!fGs)36S4+}(RDJb>v15xM|_F&<04NV!Rd3hDzI zaMG&wS+%r~(mgL2uUXTovYjK@7gSIqO%)-@eEW7yI%8uT=fb!W<`uRnyl`o1(=sm| z^*81y+$gCH9W~8&Xj1;UepUku2^vi|0oxdz2`Mrh7kKT$gO zWKf^R%091Z0obD~7o;C#6RcBtB#aIVu0}p0X#?PF`kMOkq7oeS#C|`_(H9`!$Mo?z zk#Fe~?~#$0R}?!D|Nh$Rp^(zCCe$H0_a-&pcSrW&`*4G_LG}1IO8m&81XOrlUoo9mBGZQ3t8k>T~o8n${77i4N;SVVAgnxUmPl0lnH6@tTpaGv^eJk8{In0ei4*X(CqSyJ64P3PZa=BI&={C%=xVCN zox9Hf*bgu}zPLSIkyk{C30NWP57exVh95vty87UN*mVjVXH!7Y5* zhnPAqys*z#m;`7DKGod4qsUVxRcL~s9juW~W?`z472XT)N_YhO7^%Hs4zXTIbB#%M?U- zlbY5x8yNi+lD+jhv(xF_t4sxt%I*0xUgw8%3XiX{dQi`tae0hqf`hx`0taz?lemif zHjgQT8hg3wD9#C|o%+8@r>3L)0yO6u_;0woeZPC*SkvVmytZb_tA!y|2qtH@oK1;_ zWqF8H9m7Av=fMYrroF20Mrc=TU_lPIM$f><->>`@-~#~$O2}@+V_$op{HBOm74+du zB#EjEGeFaPvq?^=gWbg&q!aPY6x+|IZ&!${4uSbsHyajj{ord_>0aU5OJUeBe1=0f z#v*NM8?#bbGOE^o?>C(Qu_2}=dHah^U!*pE-Yxv3E|hvz{hNl^4c&&P(5h?QvK7OI zc!D=UH_Z?hu`;@ZaIl~b16Mq^aAdHv=~C59-PziR!8iGz9D+p4_@|oL7lju-c`S=b zzvij<_E4=V35=kT1}CvPXHz;g5Oh&k8A#5yuE2J7CTrgTzg~Y@wc)-vGuWaVLE!oK z(1N>{_|Ng@cc$A-QKZI=_Iq!M(6*azak1Gt8peuMZI)X#o~=9~Hr4A{>tUfymi|wJ z{MpYKsW`NePEOK{itlgg(2>q#gPqHDA33RuA}#$lTP?D7z1E*RImt}dGzimq9Ys62 z(WZn>FwWEGwuE{Wd7g-FjUT zTm0v8Ai~fTRReN`ea7Mon1^oz>ISTRAyCXWxL$uBgCyr5!~t`wP{z&wmZO8mJ?lf0 zB4pdGU$nwJn`A3~cTUwe&+%rI7L`9tpw;4{dh!P~vWdwIA5lPfJ2&Cx=2m4ki=d1@ zMt!B{w5G;won@K+kYknBxCBe@H7LyZgh1dNw2`wc&878i%8pxdFY`ejogZYQcq}_= z4|4PnXdo94{OpGf(1aXJHKraBymd844&E=^So6~QDM=`O_4p6d6E7Z%^M&>356r@H zD|XK8Kb5X_TW0zlh%oxTc@p)f)uf4E#hXo2fxGJyvDN%_=Ke`t=83+bJL#pD7!m;q z;juOpiy6aIdiC$y51Bvlzuwej)b@v3vy@4Y;`~9M0@Uo@$uJ6n@nl57_Fc)h|6dW_ z|GIB5{~`ac4A@^U^lt0@{OH6paeghD%17bD_&8gCAx6#-0M*e7P-J{d!xgzecnE1^I&OlcN#WTMZ-zH z2lSiHf4BBG_do6Lk}j(gx=~5jA57^Vt;~GTDoOVaDDS+HO+giZN+?bzEaJve$@d+=8ul0JpPVxcY+aasWi0<5iE5r2Nz*!I`*0V~dpRnnH7x2NOcRgZ+E!RptI> zXp{#7<)QsJO?q|CCqi=#@BhE+h_m zldm(}2`6pHGh2#F=}chy*$acSzYeB}e`PuDc|F&}cKGgFiE%)FL#aZ)KC zc50?njpz9cesI5EVZLlXOT(+yg?lzXXq&F7p3sgzrk4%|B9<^$8>lPA<@%pO<%~ys zW#RWI;hgPW%7Ikn5d=>)F2}OcU zXRqVw;c+G)?pzb>l9jH6Na)j~+LfoieZTM|sN{;-tJH6&S&y zc;rnoR>G>6sy8YZ{)U}sMh|Gpbzr5OIq8isgdYF%8&dj`tGyxdQTyNQXKO) ziK4Gxk5z{|>I(e`SD#ag7KJb`^Ziw?i^dwWGct<2fW0GNI^< zY8*~v|2(_7LL33tM|7PLE@uC%a*+1ZK(GY;-Ixb*B*zFUuhXo{^J{%f8UZI|CStvi zLa~=M7t?zhd(QTfpCP#Shy0%wA?si`l0w}O1Rti*Qqg+VxvDdv9r*c{G`@8GBEn43}5TZ&n>O8EnFrMXgttqCdiFjmEi(z zya)1sp~KXE@U5rPQ3!4`Is5N#NBHuTvyw!__>>bi1lGdlC_CX8W}|$I&i{%Tvm?+` z@z0Qs2LG;t~Fq0j(jC8|6d zNHCxXk2Qv0cfLL>I???8pZE+0=Db0hBkYzXzh5x(0{<%|2aZ4^{eY498t8_pk<{}A z_KewKVC|OFUU+6eczWd6>aPA;uc=*|-3(DD*nE#rod$#!Lh4?GS&vljob>*K1ZmzI z&Q*I}VsU`itWaVDpB}hpoG2U#;ek?hEzYx_*37=ndE0d)Yh<+?SA?xpFJ-Nt$?Or7 zUtD~_bjTkgw|6`tF)qX8K?PT3H`X%;D=9xN`iStjSH$FnA50xg+zI5rl!KJJfisi! z8AXnzPvmQ%tz_1-HP6pJOg(CLu;LvWF+Df6rz%#&ixuw(OYBd6s08P7Uc0 z!Hjuj3*EN45(SR>#E<#{?uT!AT;rXMCg^pBrDa!({R8fTPcsriJ!_3g!}Kr%q!)Li z>o#<{@B=xmo3rK>vB$}-MM_4LTvK=we_@Aufk89z;u~G=^$;vQoK7jW4KXc zyiZg5;)yC5lBq`=HX>06 zhI>S@_%E^+@s*`-u$&8W9ugM*>AtH$v&umc-97#G7y-|dcFo*>YEuT(O4@=o;wLdj zXFp~5*tWi~8Vp9TD-9^u;EmCo`vI7fZ(0Zb(ZB1f;?l#ZSa z0>c1-k3ef10l3H(ssY1TET-VYNusnG&j&q=XIgMy1*?h)ztZRoOf`uJ@-gc&%waX> zun0Mu0s)T6!6-Oh6=mma8OpIT-U;YoGY$;{k7ET9I6O;&y3TI^V;6?h&|{}tbmps& z?TvP1vtea+3U@n~Xui)kFZ4OCOuT#ENZz`gx8n!~vL`;u_v0jN+fIAh+u5{F?}JP5H#~J-oYrA?e)5ifsua+x*=>c;@-S zj#oGQ6>n-<(JE(nY;xP3bAh2`U+sQ)O=!VCDF$v0Epbd$KJTXP4WNifp)p(?)~ zDtVrjz9Y*eC~i^y>1EH1IMt!S;YohFdxPGx)M`JBl1Anu%f%j-1$rG7a7u{rUsYv7lHnQw*`tbKv)L({bI)32Mpu`)0% zDKT$dtbNVKPKYUWM%H~x;KN8w*qp96z80el_vp7)00gWQZFufam;*TM7-TA;n5>st z-)`JeN)K~xy7Ha%QK)EL(Aj4*r&HdV&m@S`Z-CRFP6Pyvdfp8RqH&@n16wAf%V%nr z(qwWqb?%YJoQ6b=?Dju!;E?uMjxZ6U1;6_62qWek(Hx;oeY*?Y<2j18S-(Ww^<(m) zL)bDSSiPLDZCHBu1)^_gk7--qGN*<`0VMRv`K_}T9GgY&OQ{nbFr>L^qlL_pg{Vv^ z%UE>~E<6CqCjTZVR(mKHCIQR_xNfJ9fdJoriya1^p`(SOhnz&6@0@Ck;P8n{d|9M8=v&)jbf*Cn5E=W@HpTu^+PAL=UgFO^&vE z1TO1)TBG!&awbB%UHn&QX-FYTh7S~cgX{y{8#f#VE5p>!F zo6ibl2e!kCdGn>1!FfK22PnVv^k|gNAbgZck}ga3C$=1wYe8+`Be zX`A$TKz?=|W1XNSNB2E9zsjniqg2p&93$7)$VY&;4evW=Q~yG2`mdZDE@u=Rg;}@P zG&ydC%=W|kowDgKScP^3hj+GmOour#%WGG+j_r4MkBR38kLR5yOxJ#UF?5$T%o}2i z+(E#0GwKghnhHWQc0eiS8QlVuey!PXUh0k0gD*8A8>7n&$fvK`R zeVLxy{g2RPg#{$ zR0?i`D&>+}Md>dE*P8+~E-&n5%Gaj(|6%%f`}{o^^1_htpZidMth$u^kUvNr9DuiL zID^Yq;6TY9O#w8R6a*BjN3}e1D909A@$JD+$OG4P|>*j05I|nEOOH?775C z#fRTv=f&Vh$y59-Sz@6s^AyX%lb72zhY{N$?K8@Z;1s3cE#BoJj*c};R2{1o>!@zO zMpPb@5e#{Up{zT&6hk~@H_l)k{1>wQ&aMu2Qs#XOkKUucyy~(vuW#fNI|%2jvivx` z?CX;PGS?fByW8wIuXo44o@i;yUvSghd9v}=Ewq02&(9Tw)}MXp(?Nx&PaLBmPOI6{ z>_Rt(`6Hc4k#fo?)_5GF%wqY`@a%(!BsvZ`c;r5g&#Yq1pn_HUOVn={y8KIJ@r=4l zUqi5|RKIxnf>t8UF8Z#&Z6wEd#(=_|r8og8!*y5mTNEKhhflkeb9+yI7}BXSxW*{u zEh*0vY z&+W}OxN#lLZy#p+@AU{|PgPwuc+WEf~b9G_g=Riy5TBXj%Aj%#Cvw6*Af(t2Nuf)Nu7aeytgyPoDHo*TI>*n zBMXX@9qM3!yG1VeGCBDP0iO)@dNC@*+klFE7u@XLh_RA^ zo9$w8l+^#?R)7n7a`UpP82zDq-ssue6^)-qT>!0_q&C?!gqVgD)!!yTuaot~@7aVd zHD@nP)i=RPK3x846xnP+CA+Ub!mcqD=$ibOdev*G&A#*5mmj=lvFc^k)@I2Tr7~mF zIX;7gJ(b$@M8mlc6L&|uMspA%&UeBh%A5%p>_|&6)PpcKWmFJNXPqm4J$GGwLs>>A za)sehr>dZ?i~Sx5-Np2`r)P)?Fe4Go_vO`oO9Y0?r;Z0SUHl+=Wlg zuS$$%l&Tdy+fUZL_jir)#8lbX3^u) z3mYY&{F4Y%RA`^tk&S)t{u*8_NSv5o?VpQ-Ig<7=tYO%e2WS|cGm*Fc?RwCc=rM(` z#7Lyy#>tbUJ2S*-wF+}}D`bGuvOeX7^l3UftZuv)+mgREL+QAhy!T?Li#c!X#pR`r zi$_nNp$5lz%_DyVChARV^l-IhsvQMk_ zAC8;sNDcmYF_1rbpy?ND0X#;=l4Fv-GZQndhxnB0eK?uU%k!@~Rky;8>g|8Cvm zIG8X|4sOkp&d;srFWtrcN4WWs5X3SoXMU|$yPB<+{|b|m$^6;M3AQL=$`N+2NyMx6 zcSPOiu}u5u!wP$b$)F|T$fPBN0Hl7nzY7@N@?;hhKK5^AQXg6ndO18Gf6J)1Uovos zvXiC{L)eg^Qkc_0+*ECn(<(g-9R+XAfq8Bc`de^~5(tM)zRC8f*YDn)Z%$uf{?el^ z?8c#B9uZNhwc6Srg>1R3DDYHbK{{y{gB@F=+`e^GYb`D z=D)bH{jpMrZV(M~&9ouF!ux?o^)0x469Syn``&R7BRp+CspU+7J$65tg4z{5mfGq_ z|5xZE-eI;Fp|J(W2Q{~%KC=KeYBYy(y` zn+;u=ifA-PTA6goKUEA)%l00=u=ih~#`7Rw!vW(>%C%akPB;NNn@yOMZP7qBS4#X! zCsfr-0!^atl^2~%+B+R09lJNL04VYR4apGx1@B&|0*eny-{r2*j-3EMZMBm(Z1jx-^k{%_dGmK zh~>>VfyBYd-ky^(o5+?gm?{kVY!Xn6ks5r>_y z02x(660is(r8vYyVJM_*@3~N7^AS8$i~wso_N|V;imd^yOcL|!GpQw6{wa%ijckn= zerB`fsj}X{Ulh+Ye(M*D4vvajDgp-^SSWQ|k7YIo3ATvo2REaw&fXJO{;jfsTh8D( z$2PD@P&;Rwq@;Y)5yrE>|5~aMQvZW5uYBF1YCoS($}PWt&4?xy@!p+VRd@V4KZwZc zn~=?`6JLeo`J_5^1J-j3wiJ}Wlr`iLEfJu29Q7-3PXq03hUt|Y)JX>(g?jrOfDWw# zOnwy5_%>`GjXd6K&GNXP1gcZ-je=V|(;bn}?iH{V$hUky=vlF0ra?5-cPd-LS)jj+ zW*psZ$wWn$Z5ZP)LN$$uAV`0Drreg=TkPR))s{8FNmIlC@@G?z@G$$ga?UXd^&z@z z8kPniS%&@2>hQX*{PFQuztsyYqt7YH?lns#nPeOc??hONIg#V!W|8=IdC#SxDQNq;Jh5=DX$d z?`hf`OUo?e^!f?XAOKNluOeK$s_(&ZP) z$2NOo)!^t8$SfIRCz|IoxZ80^cSXzQv<6n94jIoR^*&SmYZIHlsCF_%@%jeZf0OoW z1)<|xTGB^C{nBcGl|fPLzrMqO`BpT9$`JZ3bl$f%gv0cv$sZ4yjDIHw#jz)TH!f! z0`}o3FdL;yR_F=|@)j+OFP=%yq+PzJbA%P0h`vC#ad@>P!%383vv8e#ZldV;-J!34 zPc0GQnCJw!dbUxKFR=K!_9J4(c}Vv`=)=xauMH(>;kk4#3sBHb=JtqP@B^O{_nH}X z2|&$x8^f8p6c~~za$>{mpAYE_Zf^?cq3tk$0cV8Y>zw(UQfk@S=|JaF^Fth_thzb+ z@s3_Qobyz5B8TX7XOy<>MDTbZM8`jofNyV}eA8;1-j?>=*gSauL&+}2 zv(MV7EvvHT^xy3ejpEg)nClyhg|S`FXxdQ@v#dN4%4H25S=!2^=56}K>o@lYK4~;) z6io0d^+c)=kC_hz?>wt&m62EU;SRhj@I~m zf0|n=`0!Nith}7zn*(n5^w?iw7}oFt%HL1D0-sIf-`;tGMv4n|P1(7rcwZ&32uEbfk<@nor<=g<8A8d}GgDRUD1~j$Lwl?I%5u zVH|P|UJ!_jq(bMfFf*-=%r$FQa+gMADU}m-Nk!>@m_GX4jbR}y9+oNp#I665l6A)Q zHY*J%33@muR+@Fp2|-BT6iXu{pSG^CIpwFCkBlCgXW0RCBcI^K1#P?q_`XNVVDyXV zA%Mz}nqP*!P!~A=9psj9EW0wP-UelzZX=u>0Nou3c1amn>*q-8w(g$D+A?nen4n|g z)L;fuMXrS$Rf&2o{Z#{P5}l8)A%FgQ_T@VTDUb^({=mz*6%&msVlM60!sSi&Tg``Rsd4CR6HMgmCFtGnF;tTR)Y8Wt! z1HNdbA3!R-w_u)=ywC*Q2%1xPUw+ejUOsEpWwi%F&Z=l40_Sp_Z3m6z%6{ya-i=gl zseZx_m*!u{?vGF<9opvocJHsD$<1yM$=?{}zS9eba7ihb4$iH5= zOq^;dDVlNk4upaLk6;FdOM=9LA6Vdoiq77xTb8h+>8ntZx9nAA=n*oS+&otFL z%+G@HWOO#=t_$0#+>^Wr)h0DlDn3O18z47VyIbnAZ1yu|lE%|NMQcb5jN`F1-c%@J zB#5&o%;{*5Y*I!BQeZdrf!?$LZeuTQBPMzGYy z&*y{|xZ41f84ui(9uLg9zBp?)1eZqYC)!DS$$7R$-di2Cu;_3lw>%f|;F1t741bp# z6OvFwc4$bvn^n*#T4oUBa`rlNemivf6*8h+NabS_H%6DmZ-*qvpUd)E8jwKkt9bm#|mPah9V#QJH}@fO26>9Gl) zSR?KjJbl{z(>|O$AfJ>oSugszk{xZqYDOU^Q{L#l7&$PKdyoo+q|Zw;8ETQq0kNZ(?h4)^;G4#v=wyxmX|by0$GM9?G5TC_ebolHtSb&)<>-xqb=A> z>W(YQk8Hxb5hLo8SVzZhoiGdUmA@%Qf6oy^ndxcf-1$0u8ke^1zHs;`KuzWk#$Z!3 zCNnMI{g$l8Vcx2A(D}ao_?kY0aDb+L9)Vmg9}Hb!N`7@KFxM~e`_E^ox42JR_GqvE z+HZ$M@ls|YA`iea5rI1VjA~DH%K>->9kUd37v_y}GKNJo)txdG!FmP7nJ!L0`@?jV zDcLRcr}%MBv4RP8dc3tBzBmz0I1YK)ZdJeKsD=+d@xjl6a4me{h2cMzFOt7lyju_6 zdp*IwP%=!dXafAoJ8T-8Swj%4$x#BUeQGpq)apbqWF;zS1xJ|)H>?KErFNMuyo+6F zKd%1$A3)>c{WNpk=T!&f*xE5f_#Y-&oDU9%Kg7lS8@87oGC~k5Bku&aR1hH0No5~{ z!i#^u`uLKpo%-TP+pBKZ($Hi^&*e@_T7mq;$SqdBw_fvHja`eztm?zSi^dgXl==E@TSV*3`WDpj^mFrE&kDL%cCk#D+LkpG^WLjQJ2V0!cG%=gdxAhnb&LDkLl37~x;1n2>c zXaM2%bAszqBB~RQ5Y^^ONv)-gk&Mxox3c}1`}Pzovmn(eCmv))NM1;9xm`Sl;d0-A z?E@hI9DXW$YmA>*5jT&PEK_8Jen{ZJ#OxoK)+qW#`_mqt^TY*t5GE< z-V`9bXKn6Hw@8jhK!08Qx7VL7HS7jPqpwNre!So0+JAa4qlvL#tChV+W~CrVF}Z01 zAu=+vifY;==!)GjuH(f^FKAl1Zsy@cV7bCMP2pOY=xnh1Xw1#ejSVD>ss%#+Y)u0c z!0}nD?&9Qke*W7H9IUd58Kto$+XsS^p#9=~`o}*^Mk|wcfkB-bxgSdwR?00o6R}7B z$|nt@O-tb`0~x=jTY2CR0l5in-4$4=(A!WeDj3Q&e7c*RU9al`wF}}MWy8}=ps4Npmql|FPTgnuvWWQX@rrusqbarW2$e>%O-<;h|Eu=R{Acuk=b?0? z&M>++y=U*8-=^ox9adO#jy}7)!kh4v0;TIf|DOs+5%O<|DGtoYwK5rGM@ zH!KUm+WS)EEXnOH6U@K@rN|-xBU0|gAse5OYnJy!Thbh2{mEQKqEV_R2Ws+srQc{Y ziT-gq`1z?6{rZmJJahXj7qs>E7)N=t%N*nJq;ZGZn593&(BrO)?YfMPXr7_DpgmDc z(n9GHfKV~;`HQ|CZYPm;=>k<$SpJnaVzE8#`c}lB7)1Pl{3h5GrHV=U)FSykTXj3L zEsy^CTGCu(_^@JlaPJ;3IpsO~DBARL^6Ly-_B^}Wrg$GHRC7Eji$_Gj9Va*@c;){bBNJh&g+^6f#rJ>l@~K9bh&46+_}kCVvLpRfz{g}yB=hUQ=Z}-R#h;ak z=H~e%T9uR9WHdvTO(0u+a#&cA*ZZAT)3`TCj?n>0illBw4hFCmx;I{`E^aKl0lJ+H z6&_*yc2Xi?^IHkwZ?o-kryzb>I{8W7cG5C_#&OHXs>?ezKyF|I45nF2(MTta-AokvSU3b3c_r1^WKIhJ# zIr$@h$}*5~7z&V0_}kgsI7 z%g1dW^uOcU7rs9^N`J6kNA&)O+D!QzbH#-1YOJGXkglIH>3RmGB&BC8nO=s!ymfNP z5dq4R2?A7yYA`M}5yqePz#Wgzr#YTq$+F7}*ENZ>0O#87XMk$FeM#>(^lcJ+vE%@oX+Y+%OkN>8d5cyLB>KmN;^_3zwiE8^$`xAO9 zQOw>)p^ews2b5dYV9IGdlEUNbQ=-KwjdC4nz`%I2od25H~NgBZt~*+Tt$ntPsbTUo#0@pr)$V z$WR=W8ZXVcl?qeH^~mmTZ;GxX_LO^JBtJYL{J0YJhfmarjr?Wl;p6kWVcl6qKj%GP zf6!y~~ z_D)ZTdh`{TSkqh6zHrZR)@?oAgpEwklHDs8h$L3>yCH>>R)>{e8$D{TQYL3IhHZvM z#!glu*uoBohgzPdE#!=&uuzN>8s~f}I$T+@>^Ef(>9-)IXAGU5u%$MZm-N#XFLLxe z(r4|rc|d0tBaPB`IZuN&8CsIp)5o;t9NO+=jaIs)du9lbBX^;FVv~sjO&goSXD!K? zJZyGI;GZI$b4~5vJp>Y(w{nIJ8Nv!R!HBulz^8Ru*a}yULZ^Nm3I0K9D8s)NHe#C# zLR3fxC;ZLjM6Lh*o4BE}fqv1wI?3(GQFygdZS9_4ENy0wd)2iCi={o0R)fVZyTXkf zQv_=ZHo!J(!lE~{*L~>u9c!tCZ{XhX?hNgb|7h&ZaE-*YF>R;9@4L8ewGTmKW4`C0 zW2b6-_p_3M^5xg8xSE|LMuFc(n?GqkzJOM9#sBlX?7EJ{lHucg@!n%}j6h21-egR7 zY=E9J0V1rC>BKI)t2d#|*k1%t3eyqP*8!{0#RR`bg=kK2-388f9Z6 zxzTbZDh(*p-pPdl!=wss*Jm4SuvXg5P+Bp^7K{|(mSmZ2CvkG;1{UI?OS zwk0^VWD{@AYSun<_ME1S5aVW08Pokb8y(X_S>g$03iz5Uuwjeum@S+HbagYs-3ueq zJ^ZduGFp;A=|2^3J<6253wy+I{Y}6c{_6I_go~-vjGIzrwi>~<%60^~XC_4pXf(6*U%_-WZR3Km2jFOW;7 z+V zy0=BY18vkL=_{ula}6x$$|zDi;aWU*&O|hVIw?khHJ*c5eqwIRJT}p+HcDZr-Y`Xc z!oK=|j{S>(5ap6vl@B@yP&A@CAeQEhjax?=wHFw=dB>M+$ZlD!EO>!oWuWblY|D&A z?fo%{mmKl+BmbmdO9L5XU%eJPxU_JlT=k2CYyXrT#hz_EkJ6y#!Spg8Gc^OA;=FF1u-pK+0G^~a5`_i}g_dy1o?T1T=xavwC zF!ij#;hAbdo2{tYm;cy=6Dj5{_Bcb|CE>>?wf6oD>rupyLdT^a%8WKVK@=fJk9m9Y zg|^w^YqvO$**X<_Vw5c?v(`fvRT%{)Rd448s|lE%&2uA4VulT1@pe&to(GBT{%*9n z(Y#br)}7CG0l{FTYKQs%uy$dP_? z6{b~E`YyRGe;Fb>0<4>$(l)%P6b`T9h^0l^jj;b{uJdm*{Jd>t(;jul(|?FPTphbt zz7^n{A*TF@BHZ$s7n;rSeC@PW>}PD6ZT~g*f2M8J8yMR-<_cJS*JJa-pBgC2Plk)@ z6$3n!@7~q*hdYH^1uti?%i|Lp%S=?Qy*8r|xIehWaQ(`x9a_%UJ=;q+iT>Z&!yBa+ zbUxOrbPak_s3 z3c4BkO1te9`+XK*=hIFHUN@=5H(IZ)Nb6d&L{#bZ)~qoR5FRx zwKcf2ySQ0-?I{mMxjUjrOmDyCTGqEp@vBm#^72_ql=+Mey~LNXs!n@|{&jd${cedq zb3zSn6Cs^zOI{Gsbz;Z5=uhlv`CQ)KiCek#{NZmY+Ka;a{?r{Y9oCPX#NhIpr)5x0 z2RF?#WzA4w^0y3%VTauYyXpMAG4d2sJ8Pz@DtXDEn`LHXLH`R`@!{pMN+s)EHa%a`eDO`Q_4k z%CWs~c(DgQUlrIP+tbxIU}wj5p_N38$B8&1#AB0M@cZRC(UM}U2Hi^|9ZMktYLVqq+}PmumQQm`bSgO%DbVH?BfIsY|Mww2AhRz~xcs^|O*ZR3v?pL_jJy8cy%jA$$dX~>XXtOY-MbV#P^Q=Tf=Eg+{0skKjq)WkV)!t1N(v#N;@l!BOZ<9)% zmiy3o`Eb?;>qhM+lSYqKu$Z{7nF;J}K?ZJg!wE@tK>p1#a|9WK!K_ns!rnN%XYU@R zdOp71Rit;!pWh8z!bEn(b=Ymg?czENN;HyXT#b;dH?`**TS!43t0b>%n@l3cjM2~p zJ(>qj4}vdF7F~K#fl5Gkn5h6J>3>Ptca6I96}F=~3hu7*i0q7g+)t%D_S1h8`SDmd`09|9-Q(oHvsi>i#pB4qWCi2QO_HYQ!qbbtT>=99IHUT9lhYSY+no--fSzdZ;G(qW+&&BuJX@wPK@h z0oy31i65BG!RDw5bT8KDI=~+;fZ!F`W`{LaNKxLFsA?+v)-=Mr#8ut?#_PK6dtbHK zIDX6kDz1vDzAlyhItQTb@36REIO!15>xII%Y+g*|kv6lBsu0m$;QV#ZyiD9ii5SkR zXk>M&cpeuBc50GR4y<~3Wghl#{mi{oLHAGN?LBlgu)pqece=lsHWT;KCjctq&~BBn zJza5O&%g$w!aUe}i#?sN%9!51)2)7%36joPYY_T)1VW~az+ReNk0gt{$>DlCP%fGg zV^;a)kz`c|P4pG(Jnn1u2*3bv>T*Dhx}}`X-x!xJ%q?te8B(l}^n(~28G;F66{#@A z-P2kmRJx+v7kG7WQDaMF1onGygYUj_I-i2WUee1az99<^bACaMy<*lDTC&4WF!Qil zf2y1fLy#T*xn8_A@cR^G*jzNfACxhf9tzz=Mu-=RvVX;PynI1jP_l<#ae0>D6hI!-fYF00XG{O9B{iNomKM1hJqcJpKo3HY-XdlKomc^%_d0CR4vE* zYG1ct%~@^I%%=`Mx1Ct}n@8J^OO9|a= z)k4~=j1YC6$eak$6|bfZ(>X^+`>5s*b%YW0mS&X<;M4;K?CSkTb86anpsQwK z8I-NJp@a-7?yod)DXdE!)G>cyBfwXwAQF1_O3$lLw0b%Y52GGH*-Oy^7(%1fM!Cmv z6J**<@R+pPV6_nzQnz0-ZZSpJJa969YVt#bmsbA3^UD+FT!xn3hurtZNhYjf6$7=E zO$n=;cGjlcYWI=8 z17#V^^@86sfa5j>Tg{D5mz+^LII;6qXNS4oRH`il7#sQ#Cq(6T6R;#~0bmh_a9u-c zs%wsV8fHwC@EEV>)Ic>^+Oj>gPeSrbYK|>_OsK`&>q8%s4PV??dR_n8#mRkhH__uh zv6@F#E}f-gQ8;obAjCv?w1T_^U9JVK67d!-0S1QIt1CqVB-2Z)jC4=Z*+9z#!B0O^ zaY6_Q38F(DiR1zR%^#5_OplH@SC$xiSZrsyarQEE7i>Cd?>Cpyv3ZlZdGN;GgF~O> z$vm9@g0Rg|M%EALP1(Gu%$jlEu7lhhQ=5Yw{t*_d9QbPaZ;h|^{cZF)@UOJYqAi2h zN^{w_WU_F$xL9nJf`K^+0^G%jG!#U=>jUgGPGztP4Q-f!si>&RK){~c77FNEgpdnD zj+B{Ys@Tyw^(($8RuJP)W4guh;wa05PhH9+lDw&dI(VcnfcJXT#ex(8AA;Vs# zpc%QDr)up=-h{O%D+ev6`(lXtI$c^yb$Hz~f zlcgHe54XN)B02!kEtAFEplo7fJ17WGXvCNLE_d>QGmoC;o z(_NM2Lc-MJA?nuPgt-!9HMoSxvt~T)0!r;MDM6!cGH;C+5n&4f|Ix5P6ggG85uto( zgsMUtwb#wB*^X5Gow#4zxKf9QP4xcTIc9s+O4=JM;7VV)`!4m@YUP=}-Db(|F=s4iz3@+$4dLY+Phldd9FU|7q?JhF* z1r|(co;7KDPp_8mhgUN!_k2SEmWt>kkA7x@8%PKKl8dNs0+Q6<9tCHyi%{+4yVnSVM!eQ&bX`9#Y~_w1pSoA^NJ z7cYg0+kx}2v^8&Q5ubkKabPEy)ZHYOIYMD}1MCJ&-?Mum_^(oV?QXE@o2hSsC%Yvm zOhVa^KsomuHgBTCPR3*U(wYGFLN_Y%8Jgyai1NBz=biHHeY!EmOjU1=?->fo=86McizmD2s$K-`S=h3RvlfkPU)KC^xRe~aRVi`vXu z2vDEH2(xCgINxEkA`F_uSurU+`{MNKg^5}3`d6y@7y$@Y^t#L)j%w~21@i|49+)<& zptZJJDsn`Oo&-s{N%rhoTdT7UeRN84xKs}{@cO=N>XK_^RjaNg5eLBRu{XB|AOsbj zk81%z8A5&ebYepnORQX@-6ndIqh&&JBRjeUCei(mK(BKh3}ujP{nq7tMqGkN?G&y< zd62#_ddwSiSZkqYHBgbQqvQ7J*KUR++1%X9EAP{M^W~Er`VE?dyHR1h1rWDsMPX|p z6EDsRel3YIg;I4HIukd1twQ;UUDk|k(Wh86HoR73v}IkDwLvcrU|42SqF76% z6PHp@0e;He<)tWN{4rukBQaIk0kG07F`S}gQkwz+ckRz#3^XTYndURnt1V1*-*){r zrtA@n85)=Fw@_iyy0b)9JU{3a8V#@%LhW>_MS3S9Eo5t`#qEqGL;dTp;qfp_D{%c^ zG{E1%g(LJH;`v`Kq&a2PE~5;eo8hLwt3>^1u&YtD#L0z4<;=|Ta6i%+WiQ|&mA<2v zupfM$EE()6lLwlQ$Zqe`RU9%J`zyI|CKsnH0r?-d`^}pthEV?&B2g_HLiwzOFKhqn zzI8RReew9!_3->=iyq33uNMH#7B>3(`{#U1YcCV)%X4Z5Sp}lmlPR%Kz&s?}O+h(f zzfVu>XP3OiXygek9tS~voj5#M^{=EE)VeR|u_uL>t$_U(2BbQgfoIo`+h!y3Houj9 z4CkkKyi`8Ayq@iPpw9wLd|WYC>UPuyNmlOq1%*=Kkw!&N$KPsQYVIg?Smr#+ulthD0gR(h=>-fYMsBj*D>H=qn=EbXoEmi~pCwAvm&|tUu_~21kcZRT#7DG(nl)^=dW4h*X#JApA{kT8B*i4Uo zW^c#S#+U!Y6ZP8gFz)E__T0~{dy*)(Oa`%;DQbC5Q8^4o zfsB32H?D)tiSR!~IV>20#6ns*dU%0*AS}JvPhLDjbOE<2@44%U)2K3US>vkgb%(KV zar6u6EtcTGl}3`=sWMD@_@9vXyUadzfuCU$tFC=bQ3l&f_Z@SoHOt%*HoA0!B($B2 zpf+pV;(s&+a5yr!u0Zsqf)Q0}LqMFQcQVk+<2u|UL%gW8I4z^^H7vk)EgAD9_unbdqE|%_0#$O|CPhk2-g$Msaik1$7AEMNe8IbF&!+VCT(z>OO(l*7yOyx2yV#V&V=eK&*ZK#W;oLU z*v~6){Nb_=v4+<8TVECk3RNMkbO_+CNsRV=1phh;$+s&tyR?3^!!;9-{A+Z5^HTDh zRg&ZG_v>*vbK*Bi=ayo z;q|pGr=STPR`zbjY7gJB!t9pGZ@OBBS*MJYt;9*L*;v4bpZ4CDT*q1cebuY=Z7v<* znx+QnW%MNvK1L2kqs0Tw^?X!Aa|iS2O@#(o_!h0X+eW3KFO}6PDyBB<$Eq)Y*XnjJ zDmxWl(Fd_KB|Oxsao`Nde|Y*f*-x~E$Z1^K)TLFjB?J;`F9&D9QZ_D1o5*@%k~J;m z@KoI3v(K-F7|n!+wpot-KCTS-1Am=?NKhu&nc^vnBlJ9Hlc0LcVxRY+zDr z+IDKPf91Uib!j#O=Et7qj6V}JZkp0{wFQn}r}xxqr~AbQHxm&C>fO)f!Ra8(#=Sg9 zra@;5$36GF2h{{BP4mF4Owk(vzeJOVu;!3Lc*vX!hmiUjZ=`ISa*JEL2if44(sdOcMwI0LJ2j+$OLK zk$ppFgJQB%Lf8kv8F30v=v~exFQ(-Dm)Xx(qdwM9*M;qJP%W$nWFXRdt#E@N3Ii&H z9B>A;rW`izk3%9;deKPKg5^l&R~k{8Au2=g1DKNW4$)qwBdq+_>ZH>{GK81|0;UYUad6jxQaUL<+gR8na?m} z4TE%vCD(19f{7j_&{k?w)FP}dZOJ6CrNE*EZbfSJLRL)YfEA<{9=sg&NlQzb7k}s- za;+kAD^g;pkH&N($qR$UZf;<*r!K^Kt=uO~ORi087B^0TZLni9S$$i`X~~pwz)KpQ z;J)vDY5D)EiT@7f^IpHzbET=QXg%Gmg}mj`QeIn2b$!bP=J&7y%|R>U`Oy1MX_U73kTImqx?$ggGN)2j)~Nisl9jL8APJVcNzOP<{aOXC&zEiw;- zEX2OU*70BnDf=SqK0-tO^8-Yxj5b+0Vw-0E@Y6q19oK0i{Izavmg6BS{`05SN2WG+ zwn&qN{#&&N#Tx>ZHPpH22zMj(GN~_2_TBh9;8W5#;2}Q6=X!CIbT(ayc)}8aL2boMH9C zegvO6u7$*9&-I}$MBA2OA%(W^;zzP>HC6SqS!x zsfzPv656K>TEOCvuJP`p>by!XYBDWs;z-dW9w7O$bh%^E?DbC7#;1;}3f*@%mAr*X z?+rnc32-ZXQkEoiDNGnboC;Dl6Vd>^PESm1wzNpf8t}qXpaV*L(#u-+r67Je&U~UC zkHjI5RQ`RQnfmvw>Q)(`zBPnM!fSAb28USLU_|K?oXid^6X7N&IH4aH9%Anu-- z&@9TigAY4H1zAnC1U2LKhgjW0Es%9zQ$EU~;^SfD9`*T2+uDe2uF#a|$C%J#Uw%#THLLt&wAQ2DRuQr-)Tp$W>i+SJAZZu19ip+R{Eb}1TMxs_-1!P zoP}=*`Dy1%M6s~s;f@Q}XV1to<4tExCM!JN{}+VB`yb(dGe5%QA7UOQZ>iUnm*Y{~ zP7!J!?^2aT5K!b`5g>{RjVrNgZo|Azc$_>^QEyHGgC3~%DZwXAr`(8ur3XSP^=441=fvQnRqzEiSqsu>MQ z7Tv7-!sz5B@_m<5?^_Xn8f(+N?=dmVm(`qk5a)9Fa4$H9N zarKvkR~Vx7@6VeWk_@>xUHSxLTH%KRyuttYSfBkaJCmPg25Ow@tg$%V$B|n^1L5h( zO_1R8us(E|por`vylu1ho;|8=p|7qaM8SOCtd^*1wqRlWAMHVi!`Q?B)wf;h0Lurq=78 z?#0Nk6~S2+5B=wghAMlNWrw89yDP+a$K~0?Tr5~=58^vY2-}mBo4QaA!hCU4KP9!( zAUJBaXG3VVwDLeU`+@RH)A8{qaljFe2V=ev;%_eB4SCqc2k8ZM=TE#skwLU+F3phm zSV+EUuw`pb3%elD5J_A4l7^K375b!mO=Os0bpCcYkZeT5fFDB4Y_nS~N|zopXM z#&#Wc?|!omZ+aPI`Ago|s>^I+=kt0_?6&l>wS2gwerAj+_;a(Rgs0}?YB4I)GNja%l8&tyMTU2u zthX)+J1igXM>x;2-ngeOemCpNWSwkhuBYLc6b=1u+OFAr$QJXQU&%V=z@46Va~r-Y zH!uVEH%HW)YiQ>S}lcps=Bg6{TOOz2&3Hbz_b_Ioy? z9Z4_GINhElpyek#+8k&X*H?AUzK2<>?qiP(eF4fzzXveygGpSdH{ov2FEw^MZ?9o$8 zQSbTfdxbmj_Po8sNP&!^cxKLUZ@7^uAw)YO?^J2>u&Tk|uHUfHs0@)LSE5_dG+{Kc zJ3<`R3GokD%K!R*AEVc4?5!qbYGk_VpWat0iO~vl+0tQO4LQlv){ql7vQhfHun9-T zcW57Z?qWc47bG&FI7@4DiKl?6u`v~)(TGB5)f|mr0jVepqa2b2SaNtj=vY59Qab!5 z&$EUo>UuZ(0oyEVA>Y8gG!OqFwl1&q2gSv`^%iav+N_ebe?8Tj>e?hNu1iGfcTKX< zZ~R9Cbz*gRKLpX*Db0@I_P2B4eDg&!4C{{toHdo2EWW3PWP+iyIg|-Tik=wKOmgP@ z1(F)%vf`zGUn+f~M=s6u1T6 z_AcWRu%MrA;!Flx#_yUYm)yXxsbO~b+AUS(ZfUDhPWL71Upd{JtzPe^759klwsR^E zL8)8&SL8;_e`g0J7^8p6_eq=$>5Z-jK;^~v&pvK?h~1O+?H=dYOi?UK(Bohi7ItL( zRk@t|ROwFgHy!5KQq*vHOJCZ!uItKBIY0oBXfUStU3<4eiZ;G~N9Nh*KB;fkNewQy zFyTe{BT1Rx;g;#(lCsFAy9TM8pk(fk)mk%XzEUUK&{(5S>PNfdHBQ{ z{a$orl#gVR3)`)3s*4uM!7tdWg4bIq8LwU9UlP@)@pfr3GnsQnKejsJ+C{aoT*V~_-=RXDo7Tts?Q`8MwFjJ zxl$Q|y9MECi*<6OX^)P658;T$zbEEwQY4(&?fr}QH%AJ`RlWm;n(_V6JqYMQ7R4hL= zt#OelI^3n+9G9bEH4`t@0(`E$y^%a8%KRbi`=#dkVutXU%+MXqB=_vi6><~$BC$(u zdty_dQm=ydSuv7BiFcoU=GsG5=YMYD78?5#r7Z`V}JnD7k1FZBkfMD}R21SLm#?-+rYjx3N zUt&WwljNj&<5aM!qrZ})>B9%Vf}ijp_1!6PkL;1GPm7A?Oc}^<6Xe~UMts^qVp{~i zv;Rm}nBZXUhG~q2P&>FgGI71vY64i1Ro6^#xFgI?%`ACIcyj*gJ0#-dB0HrAD1TvxG^dd7!;Y554Lx{ymH}w5O~|ILUxlr zao+F5m)%EcPjooLz5!UFW}Wy$x@SKZ_o=9Ku>?rZox~4tQ>VA!iP*=~noAP}^%JRO zQ_frMb@cFwIeg}-HGR0*uh<+SYCT#7x{r9z+CmqWec z!$r$irXK#0Nqpl~+u{Ihial&UCHgaPS6OOoCf$^GQgTeLhCOafv&s5DQfMm4TE;m$ z5P}JGBV8;l$Co|9>#7)m|D&-PbMXDt{{NRm(;!YS>n@u%ujwDP7F@QiZJkm=CWz#M z;_0ZRBl49c8>UvFWys;d=hUUnG%_eqv91@jOtJi zmXyJcM_QcC%1cW9k|O-C0wD<{vESG>%cNI{Ic=fewRfcSkh@EI6E{%DcUDus!4l1DW|i{Y&vel>+q{RL|D$;ycW!j? zRVJ)0disjA74-QZ@pBTyW54&jjTzeLpYRx3$qIeV`~CR&bETp^cg$9J@1L8Ct!d5u z0+(;*hS+w-akc%vsDNkkLO9yEZwAVql1#~%6wGaH1|9zuLwMl?2x&I7zI0i8%)hTZ zqjs};UF{=}$YO{yB!|MPIJ)Mryf^>FR8n-*SdzOMv%~#7TkFqZaKc|#JVGrp^MPQ` zy^z#)hH+oKCec67q3h#+G)$`eRv`2@SH%c~)w1G)xwqjM29T@S#?g1VDdNGC7jHvq zERb-|#WublPV(o|Y+9@keb)HW*{Z+vDsqBGkUdSq65d&m5L8Kv5j$1`TADnLpR3@L z?3bc%dI%4edcSPsB|Y#x_#**OXsZAm4OK?(X^NH3UngP(bM7oWyBx%R3Qrqct{yuO|K}K} zPq(~iseaS6v#TAW%yd*W$`7sd9EEif6VGlR-3x51%{5XR@M60BWb%D3+z;#tf#46a!FxT>H#PiwdkD+SOK1Gl}QSQg_v;UB^ zs{KvsUQ2K{BHIz#C1Ak?me{;e4_uwQr_D&srD)%=Gp|9r;pOnaBr`J0Zqn#|s7A43 zwP6ZVY-)y1b~)Swl(Zhn)iKHY+2uc)?Qcowi=1>+GBtWB;TwQ#E}JRs0|obOER!i9 z(J72E$>q~e$~5!wky)bAj%bIQ8Kc}6NO4c=0|qTq<=eq80Zy7UD&+Q-d9oa!uIs$l zR%%n?H)K{ysM+d`jH3KYzvm+T_J`id$h-cv8KJFsMguDW7P9>12MhV$U?Z`zcfCR) z4-|T`7?TS`_AL8!E~AW(Gh`29In$AGMO(oQVj*Emam~CXuC6c^oy4xIM_n{8s}$AQ z9!aF{ErqQ0kSoWi;hZ1yt1`}r_-78==Svat zF2t~-snWS<5(RBBH&?KUwAkGr^Q)*VQ~qD@)s>1BCS{mpkq=s_sml1pYaOMUdnnWu zk>47n^Pg@&ib%kxq(Gn>DeOsOUFQU_ZUktBD+GFymSJjAQIM@;AzZuu`|9aq9ZI`J zxaD!W=Pr1_32Ad;eeaLhpAF%P;DjlqcL6@kwyjx(Jt2PQ%tJ1pES4oVdAjBVt}6yG z=QX$*C1gNsydYr!38+P{VzLL7bgF79EQ8Qug7e>O7n)`?BY(w&ov9$rOYXP)J>w@9 z{yl$ujM?#!F z?vjKNd()-jp$i7&28`s%j%KEqfETl>&t)&NH!JMlpLIE|$i6YteLOrm{Yrjza}JK3 zr2YNT+Lz&}h}p`q`%|u~H}Ns@S`ySfjrYjypb(&N9}r{HT)C?=e>*p-;}#x>NVwZMs;_Q+uU+PmMdHgTyWyW)&5 zDO9C%Y`}K_U?ht2{v3AX)pB%(**b4ecGuNb7z}uN;8!|{@f_HUiPXa?;Aw8swkp|L$d5-JJ|Kn#Jfh=%R6Jw z#d9%=`Sapo5hW2GeS|Ui!tV{#YD(zwa#ub6V*MnjvDPdXo>5uoof5y|5;&yT+|)F> z$UHTv+@-xa$7MP}UvS{vkt*^JO~<`KZ^@qPXhMvlzftr<#P5rx=7sSF@Tt}MMnnXu zJ|BuPM7L;%+2|r3$m$?*e1{bkLe0iGT&y#_ zxgXA);eDc<{wZja!yMbP(llMTSvwxo7LnOUlsznPE@-;#NzOYx8*AJPBd_%bc9*`q z^x5mpceJJZ)?;jG%j$XM=T$z1e*p)9y1QNb#0{Gu=Zb9n0zy~nmUO|MPGp#e6AIEK zRvGp5pnh6Pfc>mz%W;!KB`f!QT!702$U*+0QJFKj(Y-_;l7%$y5eM$!!eWZEA^@KlN;O;9N0vF`k&QK zP+w!M6QT}M>DY}vSPg7^4f+HdbaK?>xfqUPdNz0S^Aq2oQzoDEUYd=LyP-*&5iEut zO?_r3{*b`d)5?~K;qFua*JcZ5q38SQO)DNYFXm5t#l#I+0{5=DepqyVXv=**@HEmo z_?JSrWKUiTo$`RnwEqTmOhm%#X5@GFd9=W$^tBF$#n8GKm*v*!FVHFZ1ky9}8=2

    I*VN?+nx(f z_BXXb)J#&8%9FXx2$S8Ds*c8eL5=5rwNfT~=PHxcCXyKto&Jh>ez|7%X7_MeFUJH+ z?SkHFtK^=YEuh)Kl0Kc`_|UHJ($?O**HQe!P!3v@4aObVgL;$}YGJgCXt=g^$&O@0 zV!~GSxVH}AFN*pSNNILDP}Q_V z>khlmTLD(XD?Ymq(*G0KgDidP<+gZIZEc$?(;eZmX#G%Xq*fB1Q~_kSqHgJ#2BZJW z--1d++*T@6EHiE+l~bG@GDYjDu|uE^3me;+4r-+bxEAy$Y2IX+FoiXQ1uOgvEzrQW zDZ4o{E}J@7GR1n;hUQf7C$AURjxtdcJ?{SQ64YlN1bIyunH@=O!itU(0{8lgjXI=u zkB3M|h#{2WhqSzN3N! zR>R%0Nk%VjOWbXdd)^#SeH!Q>T_)m-j<>5Wv#QjtS&p|`GVI)#^R|&$aG~~vyAtYSnd5_zC_?Ohsl!^Lo zAAErA>y{jY99H(iaoM|yY2UQotJy%r=RyE-Ix=g(?tUbU3i@=WV2rbu-DLd+xy&Vy z%!Mlp&$AV@1_I*7B|Q^P^2~*y%Bw{ex$a0x3BIw`dG(mKui#O*s{}s6jQT&^@6G46 zVX*o|rM%qSD&Req|GC@sLkF=%ZIj#E8l^8k6^&BMcMhBxvr5QJ8WbTngb3!z!hx|60NCZk8_~Q z)f13gb7mACktELla~&{YJL1y`<(8Udu1tLFZ8>3A9QM^b%zq>Fa7vk&Fkj3m*p)Ln zr*q*E3c=};9qSao5^apeS<~m|B3%hl=+;MH`My0zNy|w-l1lFNswFIgHp(>8n48%L zPnK#{!fcw`6LD>U!e(&eKy3}Z>UV23mv4z@!MMf^t%0JKguoMNbDbwY`97yhe2%|! zHGt-m&qF(7kCC;hGX7F@u`hl-g<6klgZ?sf9)_EPS%@Tj@Tdz3S7$b{I@4;$Q#u>< zkA4=EHKh0Hi)D>28vW6RsHyR&Q9Qf+vl(<~=$`>?%GzCH{}5DIHu9Hi3b5s?7jO~e zyYcMI8H3CTw#jr>M$z*WHA39p_gy>aC;hYCCpPZxk)LCzZ7q*HL>5?4Hi%ZPcr9Kr z*OHo6oR&qAKLIvD%Frf$w`Dlw@As#1XHj6i_Kk*baglZ-FM;=NWlDxvBdFcl&2Nib znt_>Io&4n{0H@^CwT=aeclfo!VFRECccQ<3V_#Oum)cv1w)9i9no@Yr`HE*$q#iJa zeSH7{kjzkTAVqF_L`A1uqveF<)V0vmZ5%2GXM+DiD>*8%??}taUKbdiWg^9~FoZ9$ zle8=!|A-73wLYrDz8QwVQrz*R$<}B4p(e(YlQ=8cjC}GlGTuL=R%NG~;WuYu*iQGW z=BK?XUHz;Atg{6dDYRSY=1b1{|n3otu}v?7!BC zLbi$!MP^*6E}-`DO_|@l%d@)A+PuSu&`Cc4j$tTHB&0LRhz z39jG@%lvCj&ECg~eCI@n$PHOoiX|qbf*&GRlrn`1OL7D1imZA7q7Ijx`!$Y&YLtCC z@}MoCzQO>XIeX%+-m4xT_5eym%bT_)f7=1gaw~~nk)Kn+S5jUwO!0W@d71V0$|qb~ zDor5Ux5iWgf#;6u(PB{Bg&_P|;dMGgn@!qLAdM+n1pl#bafrG6ULT z;)@s;5_%$UMwQUG4N8d%uJo3D%&&zBT+*`r=B8~I$g4dOxHo%~x1-(Rw%7G|CBI}v zx}ob?g)^qqOd!PGk$-rCbWni^6HY|vF`aybmSV$myUD_3}4ZTvaiG?Ab z89`J1^s?V?2LykKnG$_UbLXqU`gKhAUemO!Og2=mMxtBRS zjRQFZ-g=^Q!rrs8_CTIeUU`VTR^u)D{0Rq7@1hcdpdSZKy!%Y`eFNtifuqw<< zaH3+=7;LOITXz$tux=R-O24{(IW0-f59zHJeS%F}^k|tZS=x>Bt0vChG0Ag5CXu)})3ezT}An!hs z{#ZVqS=0FbR>pIPzkGbo=R~&KX`E^6l?gha)%A~T%ju}FkqBKoav>4Sb-fAwuKJMn ze!a#GWrWe=2F%aPhp^T6v4^+9SJLqRLj3XTNlkA1UaVE%W}_0Aa6&l4XHyaJ%b5q* z1QvuMRG^47T?UI8ST~J#9``DH`In}rTf=0{rl@Ofd(_NrgV|7L+myYZtE1%j<7B56 zA0u@F&!hq$23&zu>$Bg#x|aRq-7IE~_sf=q+VEq^IX=a^`vJL~e#HtxFFM64u!ZXO zpG)VF0r8G#;XNB?=shP=#em$~(-1jxz}L^&xxynO4{P*H`zG6CKkTS&dhUihRP+A9 zg5&`~dq?l5flA#EB0a$(hxP&N#+R;H2L0qo?JVh5ph2vK1v=@_C85p{?c_ z8awAXZKL&;U2W_Uz*RK6Q6Uld!K=f8y2Qy zBY$F&8jd_+`~jUN8^W>~dzhbL8bM!tDhwL?h>(4viH#6K{T~@4y#Ge-2o85i4`7x=^Z?0b{ z)7e{Ffl9%6w8Q#_9(3S&zgVs+50s$Dg`2xKl|ya^6z>|pW9#R4px#Z@Uj}ZT%cVTv zsIKtelPj*#xh2A@D~4`}^m$nRo(hP|4M-N;|8VSadndU*<|tq6WvP!I2q*F)X9gt-^xQ{4Y&|tzAL)lCWomN0692 zYp7}IT9AK&{f)emKX`EC<6pIesXv4s7q{EOQS=;=l|}J_tJ1-#!(ks{yXU2#WnX&C z;9qJCn*j*t6u6OF~WyWzj$9R_8nqiV6VFyy~F)(9B&jg8iJa8yt%`F zUOsV{7PC%Mg}nt_mDc~$l9FJliuCR@T}I0_gbkB&8k6iZK8L- z7TlY$Qr-R%#_XK&Fue154V@cyJ&dJk(TQ#Rv$(j}6=X|R>zb;g!#I&S1=VfwI1;BGQH6NLTewDQ7L3NZaj(mcrE#ZBnMdjm0YJ` zBD=ss?p>vDLKaZe2=?!&pz{Ai*L#Pv*~jnSdUp@iL2cUFyVRa{2a2Lr5UaLG6ML^N z+M2bZ2=3MhB1w%HiJ-M=mxxqIV(-{{^~rNQfBe4B_jsN^ufMM2xaQ~cKF{-Y8Y%a< zFa%vcMvt+X(tb=yh=`l6#k>>bZ)DxTKS?)R@9W)F@6Q3S-3ryQwP9$QPUaFnj~wQG z_|pELC*Vl7*o(NhSE|_)_Q%bd>8roC0X1z8Rh{yuELtTBN$XuH`c|nLB7#t~@Ge#+ znqdsL5n$0Azm(0aPud(Crg3V;ns-7T=Znd7Ago2Eq_I6WQ|Ck7%TaII>x;@uv5a-A zll+IQDm}mqpW76v#eu&61+l;he;oW3zc7I;eBIq0{`UH#i^PMVGWb37TL|-;aM1?$ z%k72=ra|KPlE$2)9Z7`@VZ>$CZrP0tJ9Fb8V_U@55ju>mjqQCC-$qG7OrdwUaY=O{ zJ!Knp=isd9tYnwc`(tBpz|}Lt3cAw%ZA&Qj@_$(m25$Ju3UU}0eOH`T_n*u7{xGsJ z>70?s-aW zO92EMuj$a~o47@u{?mx(hFRvIc(CikbA=2o_=w7dLS8#gRuSyz#Kxy@@r?C^n= z{_7hmLetqq#HakLajM-_Nvi(X>pW8+6ZM~PPZcBCPOHr3(Q<4oM(r<4b$8-aSx+u= zZuAk@AqB=ZCYTSR|G}cM&=)OM{kOCpxoAKTU1y*#qO$v+9I$4%pTAO>bl}NdG;O8D zOlLKJ(m*588!?`veN`W)yscXs71z}=FN!!xqqrr@%}70$C9^War2q?E-Bpg80nhe~ z#Vq0(x985RH_rH-5P$Hw*7vXWvcoEh<1%=?cSEovizvw89(`nfwju69poomxH_SJa zL#~FW7nVFZY|J_jbB^7HhnvDXZneqf|dq+FCCMX!- z)4#W6*xeNb+&0bW;vMO|#C^V!-fTY)_q^-Imc<|8gI+Q}bz?+;H;49ih{is3ZDCgF zBNP=xI#SbL9U;$1;x(7+=)TH!{rsg5>vXW_AiElLATNW+r>jF2cE zi|UNVrT+f@#qEO-5DJg9^O%G_Tl%RG?kT)M)_QYRjAdJgrg{5bw>3m{7r(iW1Jds# z)E@Wq1T_0@EosR>YHRD3J+ZhZW*V7)w)jR1zrkvF>e%oqf6BE8=u#%t^Z>g!(tYBz zS3``3*8yH+@lhaNsId)PaB#Dc6!gFWK8w6@j)g9-rKRsm_Ic8O{nFy*yGnc+wi+2@Y_>e3SeK3CXIFouPMOXvVjYKiQNtej{Y<$O!)2D|^57d5~CBN3A z)ytql1T`FNWt5z5dGAKn4QhX!%_WjOAxBQQa1>|MYES9wd7Z~aXOm;D?QFl=fcrT- zi1#M%ENm|#0r<@jdg?2`epBY$#>c7eINhOcglu4wXI6Mrppbk;fYw7S31Ft>uQggf zze_@4rbxTBdk#v`S$O%3;U?t_%!W(>dT-G)>dv0w?7|bzKsV&|hriUeHTlxX13UYU4?ow@VR*uU-np`<=r`f2z9r;sI8$Z0?-;Y*wMupE#`V(@zq_fU&5Zm<|h9~ z{9CV0h2YlVwacf1{U<*3wNXCVWfUq(aGQaxJzCL}g7F$~QHo9psu#zhOd1SeeLu?S zn2p%8-uV5r8TFTi*~$q+K0|iNGMyMZ&ACirC9+sUKiDudv>|*lEXRlx8bImW* zCdj83Y!v`T4KSw7jycicC%bAnF}Fxg**bkeG0HWy{fs1*o8&P`?O<1U;>{? z&dkz-k()k^fe|8`{?$x#i5u9+0R>oeV)79NHKewEdOu}hYQsI>!9%#Y79PzU-j4Go z>IY*A=ax4Qs9+jJw2)PX8GZdQx?>!zBdVt-pGrJ9oJpq9AHi444^CrdPou3#dVETY zi{(zDId)VPaL^uhF7avq4=bZIe@^bM?RnLuFin5cv+-zme)}P#7@fT0C`~lZ0TUD< z3(*!E6h}rfd`?8d=wVQxtw{44*IB7+e1VU~#mM_9+*+!W^-Cj}rop=(pp2lH^<4+q zV6(40z%?;)T^fZR+yu-*-uC_fcPub_4lW_y@_wOkPaC-U#p!cjJh-9na$Vmn5nxqn z-e!?0IXrECi>6?GfT|$t<(*tb0E)TGGZ5k?vP)*c;LYYjBH1;O@LNurkLtBq-JyMg zgg5;lS*(7#@Zg_Hh3maG?oB%UpIv-WW*WZLrHAsC}FxXaxfdn(^Hl&z7BJNyCMKnN5=qS$wNVjFE?Dsd4;zlNU4a-0I+>op7$ zvofRGab@BFg!oeV<6;4W$9`$%iIXw{45y6!`Kgo@nf1Dk5S1Keo)Oozb`@+sPrU)9rnL7)Y3WDiDQi z%i(CRmulE4xY(V(1awY^W!(vRp?}4vz|6j!+!EumF>u}_$R7@ncjj!CaA!p`E(S<7)O`pxPy=MO%F!1*==!F7UXPRPgU<>LlvJ_~8|0WHh3T>mZHlv| z=abEXE!Psh0s@%VoDbC>zSdf>FTR@@2h_%nYW z%tqjXhdoQ_KNzCBfV{^Sp8%zv(h;-$Wp_p=p75Llm}C)&$u74jme5cxWb-}VON@^D zvZA%2sgZ5Ht0X7M9w4rVk{paT(7(#GzjK+$NFe}9=EnFf6`>W(rD#&j z$l^LIfL^qZ$Ml;&;kt1?6nCz!B8=Dv#12}Q7GoX6uT@*d)dYi3bTlHvPvDJG8{x_u zKJS%ebpGA{1Uhr1T8KlKqSxZ zyE@8Vh2et}r6_BYWqr3dIj8o19Le#}9YUR^F2HG5_P-YXyFR-7^=YM(u3*=V1|J!n z+CT}r`%>?9UR50w%H1xbpRj(wyy^9=*%)PJHKZtCT}f;?Uk#RiuG!PmYXWo}<({BEE@hIo(S@ ztTMvGV(!QNx@^y=SD|)P$%Rs*k6iv6FAaGAsVaZK8~F88{>_f@>Y|yo^D>#I>ZaNL zSd5Y=U2c?GV5)ATIhTMmD0Y$RNy2C7@(Yk0~-a__^fVOv`%|4t|?0zD`6#SbC(WWPN+LZxyH02zhlGSfl61y zDu6u@fq*j^jf48HFJ^SUz1(qg?<>oS`vNV>QR3Hk(2;#d28Hx9OUpo})(j91a_@s2 z+N>4-939c3#o0L+8g?51WyU~3KnJb+hbyUB6{myrYpAI-_8f@A#$OhJ@_VdvFO$AK z#OVr8MnCBvcYb}LDTaH>#Q1~pi$X^{z6R0t;awG=DmQ~N>hS1fv2bw{b5XTb&^xVI z|E#Ie2pl16gLh<<{-EmzqZVT#x`Isk_m?|l`__*pPx{h+oq1>76hB83A2Hntu%nL~ zc_$_eS3I8g>89CEHR-Y5kLW!5 zv95A;z0LKs%CAC9oKBey1=dvFEK@zUT%IUvy)|4ym&`PEW6n#yReN5nI2PJ(om$kT zr$pbSJ=ugj+m^~L-MJb%<1hRsh#}x0yP&sF7+@%OCGo=+2Y70$|5}HirKIfrUU63; zRaH%G(f9HeeuJe(w@nm<#1{jXE!sBmin$GYa1|zc{kktJ&`S082e7lBrAP6#4vwbM zLi3vd)U`YJZ4i3iMo!ERZh$|^iXo@F?;!Z`^pf{ya6VV%(6MXTm_jZnS{Q^2X)F8s z^Aa4C6pf2U7GEX>2m`J=nj2M)2RME9Xx5Lu5G3-cICG>p!6%1f+FUKs+1hf+qOk8V zK>%DcWMTSrQA1B`b@0%Ce-S>rEZiIUv{xiGqGoB9eaZ3{h*X3aFi94>t*2T)O~_rDI?!|^D7tb=|b&<17UQi~|`rXW6WJ(k}BE@r~=cr;)<505ykaQ^Vyd9_23Iu>n{7=dknYX~iPKV~8Sn)4;y%QBZW}9S9_5<7zAY zPIIlrt88+knmcAeIc;79RdK8oY{B*pBf+|F!AGt9kROWwG4=!4JM1au=`F-82cy^HOrF z#q0)Ns=;fCXuOxT!@C=z_BuudD=YnJvzSjE?)DZr)ak|Ue1WAud?G-@cp&8= z*4C=caDXkLYp~Z!Unu7E= zfaH~K&?6NRbvHJ3+O<9zUNp;)1f&ONP$h}+Fb4IpF{7xKs!~21cXWD1=7?j#Ord=P z^Q77B*MmhK{ZDf9uVh2Lw*7cl7m5s9;AfIqqiQ+hDphUt@}#wRZl;;$Ta_8~6wgZf zXP}v6#&0AsjNAaA?;gO!m|Q{h9xOBy8R?;j>x~r=y_d+r4KuWR%sW@dq-v@g;@k(? z1aFOtM%EA`IU+oLWOeXi<$a$2%p|g$BpcX!;74>FWp^sM!p%TS#pX!d!Y~MRsDY;K zCjOYt9fI*R&IT3bNg586`{xpHNIBy$(|k>)YJ2|MZ(-|%NqS4lZSOjk%w#9_{d8mE zr0Xy1J2Zg=#|A05piH9wgLBafGja~s1(OW?^-AP8#nz*YF3lBf?>T9ne5-Bjdpt-R zs*@jZQb6iy?%ZL!cz&bNL*!9AI4lA=T9cVas0gGng@*T`38z;qj(m@wSl6wKz1CEk z(6IGSrGaR-hj;8+QU;R?r*a#lIY=gVp4IHX?w`1N#>wLK#5M8_a^){SW34E(tL;!C z({;~RZ`6sLN63!XMz3{w*vGWmT&-|op9m|A_Ez($LMdl$`AfT{FEn^wq)J6JtM8W& z?Be&^=1k+6y)MVkM=UO|6;t=imw4};gDVRD5#UpgdcXeJa%uf5;7qjeO^>aZQgKNFgNj(hie zn*j!o#Jgm2!ZVMRj%4-?M>BV1jLgY%7w6);Raa2lEk_9NJ~i{v&i+KT8Gq}QDyHv! zKjuu2`g&P>IsG9_w8%Ber^1yB0a=SM+zXE3oIk)M@|k`Ku=ruo>OM@zbLc4`EnvHj z+jL29$?--4w3q|x^YvJ1J*yQc2*X)_sDNCVes~G6w3Qf?W-4EIW^YgPIq%&)464xd z{G0eVPPWJKfI4BJ>x(K+U}g!@jQ3V+(FT-X0XO>hi*HQit$^})!)Vv1JE***K-7Qv)P21R&!_CjrV z$0@{7bFjFlmzrjLG?a-Z)r8=DP{R?$o@LJ1B%WVy&RNjAKg7A;S$})|LFC5VwfVc3 zgsgvUj?c>o{A}FwdkY=eFWDwBJKlXZx+!H!Un94_vaig+PSUZNtGRTDET%Ct6*%1% zp4l?$Q`P-&6Y|EvJ}vx0YT{A<8ZI(x+W#Sj!-&mbO(r}7S9$rVC~8NJnkVA zcVD#S9VfNCyq$NHJos+!v)JQ7H`J9b2Rs(CXS`fHXuh3QlW;NzAD{PAT@w1@t{Qsfx=FN;cqUH^Fhp$2dWt@jPy zWW5(yeMEP9SMsT3&amBNlF!TcDu36OD)d5!`6FT2mFrAdX8X@IQIM9LRG3%oQh00Ap2V+QRSV2brD*DcLkF19T5@}(W&?o}<@yn1skZLSL6 z|0Hh#%lt>jXe9a(asXXxyp7`+$+AC0xm3V%p}SbrLN4SUEgbUmL25k<%cS*ESeJ01 z3E{%@mp{JNjp#Zbj*=u5AuaY@O#5eyP z`+ba0NW0vBQK;l!jo|lp)2pHm@U*=_$_O58%u{Wk)RRzV4%E&O#yO4jF#>l0z~aEd z)DZO8n8szq7Po)NB}2_vyz~b@o2~_`v0knY?qpt@!A3hHC@N$rb2VqpBkxqqC2F@~ zB&DPdO2h32gQH=peHQJU)ek#;go8g<9t>W7s4(+V^N;j(`OTlD`#)GXEF5kWUMiXm zk=^xi@bTJPr9|yGE}9ug;g?2g!H^+`z%>G9d|GzLi_!Y5zYq4yCu`o76v9{OoZnVI zfA0-S%_6c4qk)%m%!*Dy=nXc4wxaTQh`9_~l)1k_Bx6}hiC>HD2kqX5ZSw4Nl#qV# z86RkdH2F;P2h+Nj7zyV+lb@HSyEBrOW9@a-O|!C1z(M>#=AwB#!FXocCvAR&?=x}A z#{^<&fgkMe>sxIAm1(RD0kq~@X*UNO<<83HLFP_-W}yjuSpGC3YCKJ8S$U=oBX4X_~Cw5AbH6E8WtnD2M-D+r>?EhqWdL~ z4#PpBxMFYzB(P|Wga27r zPr2Xg>!v^*|gv1ds-R6e|9#-ky351u|3ITE>PeHxxr|CnN{tE1~?9r5vH(%0C3 zI`54yvQapo^|K|J1H0&w_nsp8-7P5@lfJ#v7e$N3Dy-78B>if0gb7KU#k*zZlu^e) zUNaN+A+z9S5ysrC*xE1_&FHyET6X;~(g-O5RlH21oNfvoCGcyuOMmuEl!IyiKp-wg z6j>O4)m#2=EFswZXSYv|DQwq2h@G&e5;0sh+o#$YAtj!z4jaV;?}qD2#R%1WD=r_A z^j|{t$@<(MY4j=gZm*d1d{&h1Ox%Wf$lX&FgbpV>?2}&*iY@xGSvoQ86Dh8jJ4gM? zqDVyvt}bP8;GaaJ?gN&Yea|MRJLG4_0U@J(eFQ6XVIOhdXHpiAx`fu$m-14NaEV!2 zaL(M3n0C&aYr{SN@AbnFsVe#;+}+Uu`t3hgfd)XNn~DODM*BJM9PO6R-x8G#b|coO~-pP<6wn8&V0^7 zk^biUC~T=13jN{AFYU?f?^naDmMkmCBTHLiVUORtP<-@5on6MuLo6_>1O_`L#aOAQ zZ^y>JlxkR=W=_^ad;-Jx)xJL7Tq@bq;?!N&+C>RcFy~yl*m7n1{<7pD7Vgf^g);0> zrx>xjpH}BK5gy~E2a-*WVoJr2aDl5!Sg)b4-t~wQ_>jR+E7FI@&p)}}tjd3_bv6^+ zKAf2;I*86? zg;{)ESuy}))%SCtCc!it>2qq{B%rN{VV(@e+?e8k%K!40#Y_5p3*LqjoZZWrGydR# zaMwMT9G%#e(MKLRk8a;xjF8a7+^?0CWXrLniiw=RR5tX(1Z*Vb#Q9~!WaPaiCOA)= zHqZ1#_YF6?0yqvFoIoNbwaz95%%RAiYbEatO{~Pay0OdxZN<;F6gz4BL_B3W6c)NF zYu>SVFpym)S<*0Nn<7GN-LL^wSQg=`C$eUsP4|&+twdDqV7Un6kz)Ru*5+O5+AF(F z+CdwtM zW;_T5O;^8i;A=iFH$}n#ivagf_L44=2Cmf-wH4p@gPvR|kdwJAoY6SIcu-b{QkTA$;hQnkvssT^{?v99X)UIrEfQd}GjKJ!?{NO3gOX`HW#;oE zjmz%qAL3T$1foL(<=O;9Qe#j5`_q!fso+|Z==KiVflUC-Z{XmHJ* zE_?iY>&MIcgiQ9_$SMrOeHYH_{orUN{bvP0!LMTjsvdo0k#F@HCDwoBx+QkBv^mRI<53B1mLYjSVnF()b#Nl!litll>!&+@C+V_13daamrp0|?eP3Ee z#LKL8C`lDW+FX|KUXxI!?m*|LetAFshe?3te^HDtaDLQODcsNhv3=q)DC_*vP=}g- zQRts%A}lpf^^Hu`R-r2!Om=X0X!?20>)_a=aAO@$GoaG?R?OyRtX+Ub&dw}sC^3C6 z>N!HK*Z;YtUs&*cOK;JJNn|llY6XA@;%+}!sxubMkKt_gbh@#x<5bI(hnH=JHC3jf zgqaf#r$X`DTmv%2`#Hbv)eZ;ty#A)fKu>`|kG6#5pMUH+Zf?`$Y-{56kndNJ*1KP8 z$Ak85|7~tj*h>uDxal~=-mp?A2jYxHA?iyCdrqPz*Am0@Z_IM+Yb5%whz6v2T$c7> zoAW1V2Yu_D`BE}%cQ&~wHrglCxImzF1?YD#W8e@Y5Il^tfB@|l>@83(CZryvRG_D_EOBlXPh;f^6V)tJajn%yFwm5sx8X13U z4)KIjF&(R-GchH?+p{#3K3_w(Q(3~^p5xMU zr1gVcp10A&%a@JPlKu43S}jek-sV%X6&G|TFU@a-{_oqRT=va&DgJrF?D+Y6vNu@& z=(OI|k^UIY;XCeo z4ZPvR!QRua6HYu{Q@-`DQWEpUuQn}Yh(Z<3U+h-a@#y_tBd_>iKYPme%ebhzq;1H}m!jpf z2Z?G?e_TAzWnQztZHDYE10ZDNEz|K@xsS*%3GoWtc782C?yj0}`~5KBXgZetIy`gZ znVhw>rZwRX+$V}Z53_e5h23n%A2dNShg&Uk{QSU7{jbh_9gh+*E^6>@rH0@@8b-}n z-xiL$@A}nL)dtiAB{V-@hCZGsXlmn>sy03hY+44OiIj1gJ+5$OS+`_bq?`Fy&+0!a zp{#j=>vL=8wO_;Mf6pWe90;1FUDWotgn67$R~~*)x3@rcw5|v;#}3@DswGsAuww(` zwR@)wMyOZcmw#XRn>Gc&j}c;thyKy8ZUCRZay$&DZ`Gz8n_`qxeNas!-!+$*P&#sP zTGSob(3+85g#NVB{Zqcc>HIG(9TBei4+6F#o5ckjLM-gSnDLsbK@kk(BuuVLRx%~# zD8dL#>x&>)8DpC5BAzX_HFTOZw>?nh-{%r)IPm-ZjXgMbKz^8Y++HPSDRUp!T zC2g?;wj7KvCJ>DvKDfE67sU53w_6s8cViaR?p~XC`KF6<;gC_nASr?UQA$y;6FZV# zk5llfKEFOEsc5a-j;>D{C8$&p$ffMg`bvGfLC6_AaN4l8(_$=+ zc5<6$1qH5^^wGw&fMB9yUA4@&{ABe#eyu2L!N>ll&jawGdOZ$@(ZE-vwd$rZnSM~s zPQso!$(Cw|i`X0b%W`O#TqN7oGJDY_^qjx^{;6Op8t0!85|V(0^~vSHnn|PWfGbYa ztROPE>2NU$5@zea@H2#eth(btcyd_F;_J{~YyB6_Ko?Z&7f+&&7DomduCXO`Obz;& zv3Ig3D@HqvIc&x>FK5#3RD*08^uv?q^L4$jieG<;|3{K%`R~f+6Vhk@@Z{`0zM$Rd zKWX068eY3Qe_0q#bNIjp_}3`wzL4YS6aK@sRnnEPUqbxRFEm1qsPgc@!x_niJkrhq zGwi@@$)T2|$aT>Mp4P6QWC=6^G>y|kD+SaS=(i9Po7_{o+q{E1c~wV${E+L% z#9`3Rj?g*APLG?fUp$14IA-g_%pW5SWOYws^6??KX6HM1QopnBid27AWu`_i1bn_Q zy&TUQZ;f{}#W2m!dw?-;LgsL|l-yjD@^*7yIB-O+75X_@4Z0Fu;M97owajhYwOr@3 ze7HQmkSN4ow+4y9EPoR#m1c_F|0Asb|9R9kT;rf)X>Z?p)S_)Wbu!+w?R$Mfi{B>V zI(BY)6Mn~4kWQQ}^9d$sJg$ggENi)Q*pIn_D>LbDhL4*iK6Y(so`aX_A3eE_ac2v;c^;Ks zGB!_i@;|8|q=e_|NOxlBa-}Zn%?WHOF>_`Kh-L?A9Jm(;`1b7_2pYR!ozX!XaBcsB>%%dyddMjD68bS*Unge> z@j_4ANk#sbzw{Wb)BVqLUCRR5dyg#IestN^TNn438DQ#yNvIO$1nS0yXB+Jd$QQ%y z*#O{B-Ewbs$d{8uUg|O(CHGWDk6i|&r!|%xiT_}T6J|GcJ~dC|HnWP|GMZrf*nIQ& zrteJvW#sg9pab|^y5Sy0%Y~u^g`5aR<2NVR$A0-FckFv+0E{OpLmP#20ao_%HNzGp z5x`AMFbp-NXljXBi+SW$@O)MI+oWkisKz_hZ|Uhl$jJv^@_BkC%K{UtmUa|yP(jZu z$(qbzRKr}#5m)qZ(~ZommIA>@nX<)&0$d%538E{s*C;Vanm+QVE~#*PHS5CF6W5)| zF=^U5&P+|sY#vXus_JRhEHNLfzCgwahEr&-;b_dY|cT4ueY_;`6RXu#mBqFGGJ>NI{>)IoB?HnBj zlA1n@!d-RH`ybNDWxT^*WqIMHzcaAAzDVgk{(*DoOkKS!jiVb$Dcbw#>jAr@{dXYN zrsnm+A8Rol-;8za1~nl(_O{P2+?>vtTB7}%I+G&Z_WQHpvvmDUmGSiu_WkT1^^}0G z@5?^6m>9Tx_v<7EO6oR;5$BOtjWQ!8}ITu$i66OkGa*v9z9gM9`p0!Mo0_Kjl`mqk|mj` zTW)ocg0j;l&58Z@aBtYVL~1;qrmECD4n6xV#%U%D7S_%kCEX*l8(6H`PX6igP+Y>S zz@|^mYZzXUk>2R;t`;%gc|gpY&F5J+3HsRMXImQp%#=)zV01+*!@?zl32xm3bD z!&<@IG^dXUhM9e6VLEVGqJI~#x$)$+)8hs{$bLbUU_B#(kapj0eZ5DzbJc5TXu8w_ z7XSFnEh%~3UFS?cb!f5SurGSIv^XITti@~PdgTRnd$x>QiIadHyfNi;wKDAxbxqsv ziFhf=3QHzi3<I!%+mPP^%7E!16QZ((Vu>IZSiOzSjm!gc!H$<`GF^cU`$S*e@ zhzCmp>0_3R#DR@3VF(?Z3HLh()`2P!%&I7dG=Qe^Uzxv|irA;rbP z;~#zyw{#Qq1r=`5Eq80^`@-XCjkoW%!r3=hkQYmM&&Ivab=-WLXRDZxwy+YOt%iV4 z`DeHGkC`6O>djrM0iu6#iw=k7Y_qlrTkj=W`$v4x_p670T zjCSaLU^2J+;8+Ow&6GdUl4u?How-UP)TA4Bjc)!lSWT;(bWzQRi>$1F(N}};d})?Z zBZ=GTLfG@R$@;6iccO2u_hoOn5K(skEns9XZ&IglGsOw~Rd`j~_6&RMx%t^cX#wMm zS-<(h4~5+N)*mZKhUgJ8Wf<=WctKu4+H7p7p*@poBwdh(%2i)={U1eOOg&lMi(x1_ zDyOkVp#MIK6+`jl^HbLid3jj}uTVU$TSN;@R-Ik!zb$3b1ko*f!e+V*p+;j%f(+#6e=P$8H_zVZf4`q2ViExyP1&+aKUwpG$UB(T&hNz#WrT5n=nxSO?_ zMiQV2ehkq3$E;;;G*3d0VS`;k#&CqAl=x`2@U(w5Cl383|FR65@L*4o zX4JYE#+4BYGr*_39i>$DJ8P<$rwgdsD56px+#}vcFAsTV5Z(;Hgf82hGWSzgrmp>E zS^v0PSH{3|$SP zF1q(@Vfzj9g*T{7QQ{}Q6;U4e;EN%v7Y5V;FUrp`SUIB2J3Ftxo1BCH8Y#FQ`S5Jw z6nMGs#xvjWYU9#zfiIduFE9T)_6Cnje$rYWGvRI{8C(VBj+{F^AV%RDWH+r}ZB|Cw zw(ny_K22%%7defd{Dh)?=I zH@Z}GU+C_|*_;0sh%@%Yv0;uoVb*O_B`)Mx1Ue5QacHM!T~o|oMx}zy8XSV6?jJW; zZ4-}huD~xWwYAI$yLp;$%9}sAfj3oKbfiAI3&gkN4E&K=Ruj!%&J@Mhu3|7ji{B%x zwbY;(EYnO{XZfm@JVgSZ%xM=QpT|1xbv?fU81B3!A@-U_FcYu$i@DMNT#yC(rB`Nd z?zkMjFdt@sQIgU^ul2~*64p5SM0P70f((SQ!Qn?#rE0&CE+s!i%pIqG84eXp*g6gox+NE_>u^fg_VYx`SQA`9*luWMRA4R257L-x8YBlYBP*ID3jOdNCvtnT@Uk`7a-jEl0@7ygJ-6?9236L3P*F9@>s& z1=4f;9n`EEF0;gG7hkjjr=v7_;$jD_X zgGuwP2?p93HN`V%3aqoV*Fw5~2U-o^$)?muVuoH;T!{lkGGX zcF3^3VBC=53lIowxQHK>0uvp`VRo>+{r8s@;ZiqD_$tEgy53bWSLO4O>sS@De^uTr zH8N%k4nEGDsynp`aU;-*+Lyrr4o;b2UUU@)8Pp-QTcf+%u2b-Egg<)e^N{m*8#PLK z_Qm?HnUycB-Gst4Z^=WDSYSFZk~&3)uB!jES}7zgM!8@kDB}d#U6YfQT@mZf8+wIa zT`wQK)KK2^mH#CiHKhN*lS9FJpR$DVfMM#|nJvz^?!_eK8iZg@HAsxOf(!}?j&cs^ z9a4|;k>CGik-FqB-?>+te<6h3omE7MB2hfFTuX+~OH%A8lsXrn$*O5C;q(X_NRFzp zlK@NH^XpoY@34_fJelIXcHRGseBs%ylg%3(;@FZ;VUFey8KIsi4XM34-1cQdUaic$ zTH_EC3f=h5$!tvUd-8hQ*CI)dfVVUHZ0MEB@lmev75l)Y3R^`p{8DcCPK*YSLhDk6 z4q-peL5{U#)2gtCpcsvF?^dRTS?>kmfo5%gO zmgsF}?}VYrGMc`-{R+LtfdOk3VI-5(4CMzY{e5F4If1|)*rCIyn1K3`Lv!P`s^?Y_ z72JR`hc#)?8EDJh>a%dhxvJWY_s)%0M7vQ?Z31n~NC&q&moWPlLSxuP;Oi=&jKT<8 za8%UrNmW#z*#=NIx(%$x0shZ<<=ohyP%ldU?6jNW10JVNIs(`Asj+G z$;zx5bV3(L$(fQhO@nr$?vrxv+%+nFj^c;EbQKG=eu$fA<&zLKETwa6EJ<1>iS((U zYLXZsTU$8b-Vh)XSE_3?1qBQ#<9D#L?t}xl1118odZhCqnr|V;s6DLg+=vp3HK$5T zjOP!r7cvn4bsvjf4-W#IFZEogc$52cV_=Z;)MBuPhN|9Jw?R&WXqpif z-&jOy92%5jfb}4Yp~&iCb5SbKufE3fF#2Y$l9-vd!wyT_6&EC;1hZcF>sN7;Lfx&N)m4Sy~8rVVgfeMp}~kQ z2Qkoa)o3{=LQ><{!gzJ3@;TEYZQ{21Fcan_YOq@Umql|G+pq@I|&0+7`#;UGWfa*%(p`Cxqyoy3qFvj4u&Ai-E)$jLxvJAVlD zdW>WpI;V2;)0-a`Agj2f-3ux`Axg3T39A14ZmtHN1Kl$giM9s&fOR}g)}DU7l1^jP zt{sU_0Hs)BhDk~TgrtYw?0d6linAXZ~l8^qTT9J zbTSt!0{3SfS<4Q0j!QQfOEo+k$k`D{VkSLB+%g-+U8`qZ=4(-7UL7Nb>Vbo+W6k$7 zZ~?0+_esY-rQSnI@y2(zE?+HKKzH_L_W#bprE_~UWccA!BgryE?)&(!kJ$-F4m%kP zuhFPHA+D04QP3>&g@>oj)wuSpB0?m@)|^I5ilqp}TDvpP7aBFz3);e`E-Ni&wT3&7 z34u2B@n#K)n>;2H;OK_*nL_R$f_{ja(~7xy zQ&&g7n6b2i|H%AOT51+HKVvF<;BqP&6x5wPu!IeyT(yZ!b8yE-T4#e zH`zW|0DojpLt)Zh4!!Rw&SE_KfMDWm^No*j>cLU&YMerQMK!kLlck^$QA35a{X*-9eq|P%W@zwaRj5%W+Q4;s)ld^P zpWs^XBBeE-hbVh&RPpd4_IC|RgCSsF^p~Y`$m?d879pQ^9^%>_vbV>wnwft`8cV;1 zQ}i+w%$_Q9-v|4X+Nb=k-1hy;QsCcdD4%B}7v!wats3q#VV3r6%QM%iZOR!ovoF~I z6E&YJeBU}5Hi5vSOyx{07mcz5PC`^L@hHwCvLvMc=8n^~(*bR{4c8>2sW^h{CwJeo z&2f<`p`N6=1{H?s zxfno=qiwT24q$#CL*WD5PTaNx5ny53+f$(LzC^olS|YnSFi^*>yUHFZ)G94%@uqdc zOJ-UxC$Ze9Vre*}3;>}vSZ3rx@JoTDd4d1_mz?xJD2$D9J5>EZ~ zb#SoW-Tl?uqM&mDYwfx~%VlC7&+sH5^I5$m!d%i9?wG9bz9tPE_?P8Zy(QD%UaZy) zpA-={x#gGPhW)=ukH&!@YS{Sx8WW1w`d{dnaYOwOGg_;7@5pfUF@yEAAb>ZllD1&& zm!eIZ;{F$=m~5G*^1?YF*77uby5O>iNzlWl+sqPoN0fanYlGg{Vz+{ z1&p5IcfGAgdCpS~oMF%B^I6 z*&$zPy>s{T#)ltvN2X^Y!|Ms;$QxDYg8w3IU^H7L<7=FojBUDLJI`DM*zcoto7vN} zmNzD)=`KdN);EndsCP)ydsAey2py_<=deZ4!ls6*3$y__geY>biwKvH^XX5A#~4hf zT&NFeYm^m=Kb9zr?cUF~q{%5wTnPfTp3SE4?NpA7memee6s>`Cm(O$S>i+X4Q*VrtE=swaLBzpGu>es?FL)AUvEOnPQj2)J2hf>M(pmR*+W@P#vXB8Wg@^QkNpxe(v7_iAR`}C|6fnZ8NT1`e!RMhU}@-sUlSZU z`Cz% z=Eu3OWVO9)SYr$Gx19nBH*pcPD|L}|hT!Z#JYL6U!jwG#dIxE7jVkSEos=jI!Z$Z+IqEeMv5q<~YLu6SJD~6S8%y z4?WL(aT_;=4G5+JknukTQa|(ba{vZ3Yow398I_m!1aOncH?e(;bx_bA8sk!E~!KzvM!v9^pA7Eg@93Xnry$bFl?>-k-v`Qzz* zgMMFwP63Y)gqsi<;=?S5r%|*Z(OI-1CNkT=aW90l$c!uNt&Du4lOnH|z3(G}R^9Xx?BhxYG6u0iomvqQVf?lF(IL2y@$Up^bLVJTE+f8(y4 zr7KvmKo75@K?R45by4BO^pT~d-Eg2&FhQ{veTWn-vezBxW@5_utCQcAgu80t8Am;X z?6gJfq0UL}H>DWukez;v%@PZT>kCz{D43f3=_K==YP!~f(N{4$^D#_;7XKs%dWt`F zA|F3gbm^z*@iuA2bS@sA%q_jcZ4(bq3#Pp``=kz+q4bEDq2#>cKG7EtLbAiYU1f{hZ8 zAT>ZJB2q#Oy;l`1bOi#^RR{q>5-A~&5D*X$A|NG15<>3~LhtZw-Y+}*zPmd+yR-S0 zOxjH5f9Jl>`JL+$jWXJLt|c%wtNZGx@i#>(rR*Sj-(dG}YG0+dk;sk_oVx{7N=x}6 zTr%5_(beu_r(QOw|8l^R8qsLhxyug!a)|A+p9UUT`^a&MU z$L@1`E7SQ`PEMCw!1poaxT}Xl+^9tNSG@Y6vpXTHHk?8#;H&SER_J0%nE_(b%&&`` zR?>v>20E|SmWz0EIU=-JHyQ(%HWTU{$%VT%t6WJJJh zjH9e0?-$0(PWDDU(vw^X*iQw>s`&Mr1gCrzSg1OZJ*OfZg4Aab&1O;pTQ2P) zQg|<+*3Ik>ph4|klWyqguA}}JzHLhODk0Irz~zkFZ`u2~=lP-(T&KU&fA}0cN= z@+k#E+5OjqMY=;?jpo-RvRL#eP{H?8Yq%7n@#_6DC`=$OGP~`(NtCF zi=2FOO5yaecRs_ZKin#!mwD1)LnSC#IhMN^I&(e1sEg*HfQA3d@#lpgIuN48m?y>u zfi`h8Ydy&&l@!voJnf{l85Pj|2Pa-!KR$C}?ab^bPwd)3kFCr}&y--Eb8$KaSM6BE zIZ#+{zwz}<(MfGrbv-o*{wxkNn8X|fx<_8wH^|%B9(Rc_=$-D}?72`~W^oN~exq-q z+c___RUt#;Slkw2>F+{|kJPA2xPWf*#8M2u>GjNKtC#GgHavlf$PY0=>Fz!1MY{p! z)9}o8UU5iByb_ZMv}hRn+^d=g01d#wyR+#P(vRU%=KzTE=zYgjPv8=CWJC zURCURqW5KIXFE%X85XFDGN3=0C;>?0*^xwen55G-q^>-y6r$WZ7q239xX1=O?qoucFj& zoeVrF6t1~>6`fgj$s$F!B84Syf?9W8(CAc(p|p|v zTr!?ng$7Q(&Rr8;T-mX2JXbgW=j*A|!^Nt3VyXEW6%+_1v0T;<9PDAXe%&>?!gR!Q-&Skka2ZrMPYyR?Q~U{XFzmOt|aA#=$GvkbJFDMyn# zgxm|wA}}Vh@|}fVP$F7{K2Mk~#}rzD+$?_9-F?{gXZgb$7ml^1oCLv=FA}Z(#%ImH znWHrZO9@*^4UVV`sXq-gu>eG0hrp(79MpIY zkafL?oMxPl3(DZVS$ym+k5-ZB1c&OYvnn_lZql+xiGhAPI#RxD2@BOjjW)c`lS+ft zsWBv9PT*6+7!Kj3L%XCTn|Rj!j`MI~IUQ?`3-8GnPhI78hm)tKI3s76$hU9BLi~-q zSlJMj$PIu*>!=>omemElM4yJN`yrvs$h=WwEiqU2Y?-}Y;T<00ufQt+Ayf>e!nIEEubje5Y2gEhoa z>untC8r+*M)rp~Fc^-jG^R&v zD#Y(4qa3lmOWIl(eQucqyu0hF;l5rZ#eHn;wMxTS5$CVY=(ko6ljv-Rs@6uY10J<) zpOL3bWZ{2n~HP z**i39HF{vYl0i_-Y)#C8egmZv!yITp_8J-J9&Wn{s5au^b|KS_afGJc)jxA%+N&`2 z7MbH=2gUb@a&fuykZw(pi&G~yY#F;F(5!2q;Aoyy5W9KL2sAD33N-xQC~^Aom!=~D zagNQ4c-7f^-8)g+9n}Y9wFR8FUR6V$`BX-Zt^XeBJsB#Cqnb7^FP2e_v3j?f3N4*f z&L@9qc=+O=QHrQ}N%MPrNWqB{^t@`c>fs;;jz;V=NGc38)c{p{*y>eD%7#<6`+D^A zFiXqKnXxC89|2Zw{{3%wD-x4~;J?P0Q26<_^NKXT;?1y#y)b-lLqg})mDFHk$D@>G z9M%mhy=n9-98g_Qi|Sr}R9160*xBW==-OG;zGFDQ)+Dy5kJz7Ch&K*g3j;t|9x*ru zvp*8HHb_Gg)LuwL{=|s-!(}x1|K%XoOS#9qK^#!X&+H?<45;9;!>hM&j2s8dAwxEB zi-*n!XBCdRN1C&1=3szrsgpaYBh9lruW5AjBpSugEZ#L36c71wX{EfE4 zP^!#GwZwe1hST8IG;J{WX~w6SV3;j{4hF-w9oX*JDx+-_HF~3Vo_Z>&G zYp7i6V~K8{TH2r;Lq#~vajVBf)B6Up>q=hrQ?p1WI&%cGy%V?=WH#k>U>oUUI`rJ| zvZ>PW0juZq3l~pG&PPd2Dcb(E*vB4Ca$Sd?iMHslx^Oh45Qhui*Qh3Ru^P718nSJ% zI@z0E?TWtP%0Ef1KdDz<6y!fSx!rcItq|M!d0B}<_z&dTjiphCuz=AnMguJ+gNeqI zG6>#L+l%FayxeJ~v6V7{@_KjEUQF7SG|ZyHDj(O4i%e&!OI?lA4x z)l(Vnuu;RvChbKmr@EeZrv%m=b$lDTM?M_d^q8ANv5oLR)TM4{bdQy#1?rD~;I?xrE=&mZ@V zT|tzM#l&3}xeoGF*Kt9|p3Yde@wewuL^CXm6z#$R>NVKF!t~s$cF5I^fcQETApE;I zIIJ4iD<9SaVi`$8=Y~z6RrNoAqg;i5ecSM#lFG_AypN6Q4<_y^FDe)~;mARgrN_@f z-&s()I+-Z{PN3QbVJnTc6$_Ee99-kRYX6<#v3naStL8@?YWtUCG(86zSYPcDnD}e% zM`xC;P@m0gz|g^UP0Fup8LwWPif_Al|ChDPYxBW`sv&^UIjZA^hn`W<-q3)rX&E?4 zOO?e?%}&FIQsG}OfLctgu&pl;wiqk!rGPlM=gF`UkBo>FYm1#?7b!>;!ijxb?Q5Kb35a~O0aH=d| z`5XS-{TbzWf%;EyLs%P7jnw%0l7Rc&gxi$Vv!gU&HzKu4Ol+}OVmvQMQ$BipQs;jmDnuRzi9>o6tnx^1!3BH5fhwfyys$zS^ zh>2H>sFk#CW~{TcW0*L4Djg9SXt@i|%U@4N9`UQlRki%farVFf^EJmQTbFCzlb6=a z0Cf-LKN&~Z+z3(`O1pl_Je+QvS` zm=)3PC+c6b)(Bf&RnMkOlO0WP?n{ygUt7vCcgu;Tl`Uv2koBc6UX}SaV!BpvQDAT-5(0c zPtvi%m$P~c#ei4*P-3peT=Gy5Ebm3#Euj^}h3dKm=gKn5c#nEXueDdP$aa8L-UQ{^ z?i8V&4rDI63@EebH6W_zy*2f=hR!%uEoA@n7OV+7R_O6vKW$i_YdGYlA@G6P6Yltx zvPmc<@aD`s?O^Anw0Y*??+Juw0AfTbw;1j1&R_7rSgvJ!KnkMFB*(Qq^vuWipN@Hx zWH;}Th6G;9JU9%BV7v@-Xl-piWs4xjX>VS?q!$`_BHVwLHh&qv<&|(zDWMAp*}5e< z?9&=S_*@n+Dc*IXFKr8Ekz4vSx8Q(?PRNuKxvusvhmH^X%JjQa*tOW(WL;{m80#)o zrb978t5aK>r9G<^6dSG|H<8x$PDtOiAeMbAO8C>tCa#6`eZ707(o@(=c7|smHRPS4 zJG4WAm#?LHOHKbSr;F9o%Zu6mD66u|iK%Z0UrOk3U7SFJQanQNs#TTdhJ}ggLRG{c zt@%GK&iy|JN_ma!j4v&&OL})*&obhw6{CxN((>u_)lE8g6!N;P6(#`7zR}7qFcLMl^-re!eR)3OA=(y<}`2@L<1s0VP>r z8S>~y!<7}GT;;~IbJlz+*i$ahGy5jP1+Disr*f9D;(lrX*`I4wI^e{aM!7Ek@_wzK z+CWGWHlcLO3UIWXI@nuiq>3;)tEI1y__BIA<*#4Fdx>8@ix<+z6n3Pdy)3_w&=5!{ zwVY*LJ#{#5R5Uf56Q&xX(c{XH&$otzc_mc~ovUfVQUQ>p3sb`iii6~tcU zr|FT_>NoAWvdt5E+W5~Lb_ear+A8Fb;=CtrR}_A@OUbs?UwJF@WDvC|5Z9^Vg%)^x z{cdAJakZ}ASixogsLQR+yVBIsm_qBTvCigZKb`I1yDJn8cq)ZpUN2T1szYt!6T+^7 zwjNTE75xG1Lelk!+fZZyXyxQj;-);OXkTU><=}kCL8w4=VIWZTmpy2V@BwKK}K z>V0L$JBuvZ;{ifvJPd-L>oIIIwc9bm7;z)++xZ0 zLELVu%vstUW>+Hz;bA6qeJG*t*`{R#k0Zs2Wv&UK)7=%er>{(J`W>{NW09pTTP8g4 zo;LLt?_5VDHi3-u-`z<>0)0~{$^+qHn_OA{^|za>2aiP_Z_|Vg=9ELWs2Nm4YiRg~ zmL};Cd4JJZ?CMol>Mg>Y$R7TakeKb(Gg| z<(^DT+`5-w+xTi-kX!>_jZnluS#M5~5#+!OxRoaa~)^%rGTg+;gKy}=c|u)nJ=1W8s&y&zBR3{ssta?vKu?efRg{h1S!Uj`K}!H#`>|StgilGg1wRqIJ(}QvPkL^YmG@ zl~Ef6R<(PTQj8&hvBW_CqF0G4e(E<^ZCb1C7N3%TFy|er5$xSI;lQ-RiBXfZfOai> z+H2fej;k5$8yqdlM>9xr+3P0NGth^cE=VWfp@zuKz#DU6vE9;y93ut+=0SI|&IAtb zd^?iYDgKvZmWtSLh~{pfHhb!Ia0D-3RAS|ZZ~SOt(?q)%rii|V5$_JL-e_NUZ=-T; z8M~~F+nlGb4h@YAX5>1?Br6s;W^D_NB??TvKN9MUygGNZzlXJ^X|h)L+G5O0pmGj@}c_TZe$OZTuLiEk?lZ;TB`jjp*TP8MHn$Iz8=1!Bx#;u;x?HzgtUw zhDT1B<2WB^h^)%;0h4W^;9B(0HpT7BVT0E;haxuNzo>%A_q3S8Dad{uq^|sv+e;aZ zfy`0ttoQ=fCEuJi)JTjBJV<2<_PVJIKe`mQef-93ow{=y7ymnh-+wXb<>uyWWp~6o%$m|Vgb7iARSPidW79M3Jd*gY^%En({9ApfZVImo1Rh07sw1aoFb!c~rvk_9_nYjm^YOFp$i2Pw!_%94s}^K2sLA|6 zhtb|<)Z*I09)o%F;xZkuAtH>H9}O-(GEbQdjdF?}Qa)nW(47Xqd8HkNrtGurq2?lv@ ztfpS`uhsJK>fJlmstNmz=Y8`cXxhn`t0%KP7RRRwD2k;{*?u!YNHdaE$R-@6uWH3L ztD`U$mL-FgQdzAt&Xl{(YYczx0|a(nop#(%C0!eK?L34JlX9cA8kC}bXXr@aystv~ z^_j{-u*L*sb#6NbX80|&dMECMRiI)=fNHgP^yig|A*V0qXwFXaLPy+_J32a(2(5>E zL;6P(Xd$plxjSAY6;i-u!1MWqb`=7cuO*RVQ7EieIa0~aWg}cQP3G0vthw1_ z;yCuof!2WDyQ&w9yw739!)_I%4+2FsYL8*NuTT7-J1D)=<4}SC^CB9)CHAM=H@-Kw zUx@xaHvr#t0s?Gh!|nG@tu5dFDmCEgXEnWzJk!Qgs){DZeZi$N&9scDK&P{R=E62f zKeR;G{=(ei+El;q|11&R=pKB~q@jMCXTwHn1KO%-)Q3Z!_UJsjnI5)~+Gl;VS*`O( zq0?9WEYA80TrJw&T4%!@1u}2QUKpcA5;TG{s85G6Vp>2R%QdCms@h{s9Et|~$$xYr zKxjV2nfknGia{ifGJhwhrk9TCT@^31CxQJS%)TQG%N@Oemn|&HXc3hg9PK{Fy9-o5 zamhjH)fxVR4Cxxe^FnSkqS`LJ`|vOfhj)B`HDQpsh6YCkS2}LP!1dJ(FAtijujlTY z(AzJWqDqqIKN>P}6&O^F5#RMoK@_^)Z_E-cjJ8KV*xq(p_x#NaM|>mg1{rr#lWQP% z`e;29+u>TeJPwaLguY(SJ*7e8p}(#saIZ~C)kN{J=vvO*M|gqQgjf1! zWo^|vpze{%y=YZWayX-=KT=jc&{*;z2*Bnc(>m$`!oWpfmMGobfG@8Q`b)$+%ksq9 z)aj(w+b-H!^L06*HFtd;>1S)nL5FaW{7Z~)SXAW*!|@F+EqROX;0jB(&P($~=22CO zoFw51?fkcfiQ|Sh^1k?PPoEZAR-Ke0+6>@4Xp+_a1PUU$PA#~wrUAqU8tk|iX1h(o zkyynEeIVDr8CFkAZ+nNDj)o&;kiMcie+%cd{Fg#_)TmHBSs1yf8n;Z3e&4!wI)q_s zOS<9>B6>frEAZ|dE}E;-`pz-^%VBih5Z&ZdkNEv6Eia+E@xFf0A0|(DaRy0slx4IP zl)5n3oa!Z8GMZJ)S{~URv93l}zA-jt*1wQAZkW8VC`8ff(4OMGNd9o<{2J40B*+iI zab;#JBe%ygBhzlVAJj`bC|mAwWE!8m*1v!a4vumxMTrOW&xcDhpdP=7Ajm_W_ah#v zS2U)Uac~Mx3Zm-b>CReZXfUycur4<04$B}CJxPn-iqoA;tWv9B^acz$GVNcEE8oBV zx$=;`^$@+L;q|2q`*&Jn>bkQyXNdZ4Mz-`^URotHeAAKLyLC_<%FSbQN{=Lw^8mfag^p=>U@#E_7 zExJBwX0xWIacy*|_LhdsSgKB7((y?~rdMyl_&Pe%cQ@hHF;KW3g3a@EJJ^nnqFVQl z^coZrz>PR$M#A8lV<{41lg!gSp3!El_MClo4pi-U$C3M7Nhxf((h zX}@Lww;9j;y*v!jSF~enQ3KazKqebe_>5L1%P18o^Z0n9-a&?3%@Se$08brp;W{V%cf%RV{l#5cxNEuD!N8u!M?9F0Vmt*Az$9ejB~MU& zew;VGN2=pIJtE}nRrcWHY^Zso^mCJ81Enbud&~SzfM@w&HUuilqFNPaBmKPbw#r2V zK@CJQAIpO*dH;{B}14}ET&`)+bX-`n>x<%{cHrLA^4e&w2I?}{S3Ydlc`;Vo5yHe5xDv=g;tYMW!*os zOo$F*p>rZCmU4!OFY_=ttSIQnHYpcZ^Hv%}Q7OHUpX4e=Z!_u;y%q79J5b5tm-)yg z&Goy6`ax|&{OM&6^AK8e9D{BL;R(kTn&z+*^2sZOFLIGL&{t=?)JXZ6fNf$8`{rIa zH+n9MBCl-`Iir^}(@nE*ch8-UB_#e5BQ)LI4YIC$Aq*caTQlXCuI-CAAMD!io8P)k zOL1_Qpre;9j2$i0t#kXfHNwI%0Q@KrO4mE8&yaHmflE$htIOPyhmAsO8U4-;2A zP)^GNWlaJ?9yb$nVe7NjfiHG*a;xjjlk`b18B*Q{Kv*#7iKjrrR{VDTFybPNxNcP% zh?lQw*y@#pC>ijTKd^--PrYLgkX3)oeO8?1)whuSUgL0jprUXE>}FVE)-Y&|8-|$! zz-WlBs|obm8<<^(mJK#YXOYT;UOzD!?Q)$dE_bjFU=^1mRpwqOR`H5FDyZCr2$s8u zPW=TFzz=#Jl`x1O6cZ{6aQzSM=a(}PLn6(u3o@tyQFf_kc4Y!~jSeZbj_0Ak__T;m zeoowZh#+R6ZzFs56F)168Al}M?$Q?EGysHdQd*4N5|Llk-_FeUEVvx>Uen?G-#P!% z1;2afdKB@^$IR5d4fj{6z}wFMa>#&|v6c$>oxJjmc);SeL_3~%e$yg&Iu#TqIxAi{ z4eLTHXM()hmEoC*!kKl?y`DIXHGcq!I&H0E?+CW@*xLV~k=(6n^;a`ZyJDlmkvaGk znC|o!*wwM9Lu(AgQ}x*Jwh^prXJg2&2%>P+1XHSo<1R4N-&>3fC2>98d4%%3+!8*) z04UQrI(vt}>2Gdw>Db$vC;Iu;oQG!dCmIetueb6)2iVuvcJTbvQ0nBw0_!1|JrX6@ z$A$Hhy79e*XCXc201f3WQjWJJsZ|`?+#k^}InFvWM>}EG^bGXVz2mJ$BS;u_HVJ?l0X)>f8{iEju zNsL`>f*OUEENs{`3N7s%tanH;?vKzYhQPRM22I6=9xE&mro8=2I_3|viXK;V_og1R zcCCn6&VQPLy;!5uAZDagM#=@7e~rzBjI*)L&Nu2XdSXPgt@UtOkPPWS$Kes@xXQH! za%M)uU8T~vG|vv_6*Hs47-Iu@RMWcCV|u35#R6^W-HzM8wx8aF)K9(HTJF6+?53!g z@MvWK{6NUO*r0xF zd5ob3auG|@Oo-@0XA#8uXKzB4zYqL7?+d|Pt0$Jbldhpg)-!)>m@hvvI;4t%Z}QZu zS$wJMd7Aw@gITu7Yf&K55*Bgo$MGA3rx=TE3!tCIf15AVYH1q5Rp*N~W}4cfEt@?X z<33M_n|W_a%6YVt%$moYnTuA|DB+-bl%*s4+}V+m>!`$ExO2B$Oe(y|CH-rU(373I zP3a*vIBk&o9AOEbdxD2*4Kchn}t zw9_sa|6Kl5t@!pF8uATn_p|WBvgJSgQ%bs*)|O6)y&bP_P?^eM%OrkFN<1|=B=APK z%7r?OY<(uArQ_jh%LT#dJD@B#o@?^)>l%GAW^60c5}_LLAr-W($eUnhIq8;J*<^vd zF(hG?l(i`zdvJN&^f+YFc%}F)a;_$qH>!ifcDab9!lbk6>a%iW`i4zJrt*PNPQR6& zOe%F(a9z!sl%7~%3PW}Ma>E`biR}}f8j&;3X*m>Nr+I}eK8|NxmH6SjVi}xSQJlU& zE?=~Ypm=tX?8_{3h@_F;1=R0*&;E}%h5zvYtl`8)`aE7u&L89>Ro0zr9+~ISYpqgf zb8(@ZogRzUN_$T>o_D80}@JRB&A0z_tis-D<1LA5y*j7aY|ta6RXC%CV&n!~HB z>xj>2-S4FgkGNM6MUR!ho~jsZa$7kiQO*`m~eN zl$|5BKx4Nm0?53(r@kUO4`>V4zn=R9{l1;$D0;Z~ww5S6ktpgpUR>&O>kRqon_R5A zgZzkj6szXTXp6bz&_0~oEjwj>Hcsc-Y=fXnY5n8UnOxn^M&=#rX~(av@FvN}SroKx zYDe5Uh`7N1NJwWvb z?g;x91A;L-=OiMshGr?+Qaj1bhjAn85aio_l`DStA8Z-F*U)+VJ}}BZ$A|Rtd}-Vd zaqE~&<+-^!uT5XO5*>mayEn4uryD#je|3J+<$Y%@24M;@Qe^9*3Rx2?ap4t+B+_(J zQ>(2*W4Za8O1O>qadNeZyK|t6Pch}UnvX+U?*gBFXHX8zh|v(7WO`oxwAXPjky-4i zb1HJFzS9wv=GN(=*nr8Pfkqv?iAH3%d&auxM5*!OS*sRGONV9CWGTs7uCgI!}b#Vv%k`>Q}3P@#6JsvoqRB1@yK1<#(d(gCuXDH>7AypfBRFdXr~9nnEufV+o*>0Fzt^8~AL*&tu#Z&NV?&w_Y&QylAbg9Z7uBbPv~I-}Q-z|rVj z$GFkF28)caduZ4-vL0g|#_FAq4z~2rKW)|@!%B|zo%2!J0lnm5P-m%ypoB2TPT2-( z(k{>_6$;E|Hby#m5j@e=)w$mKB(CFvhwT`BF75TsjomQnW2=m028``r>FmKZ2zA&@ zrWY(Wnv%i6!T46`b)$ugS-i(nF!#7_a&YwvWpxEK$zCNjB2oQ4PK14dKKe8smfeUU z)z^4g|KPY7y3sED{`8uTN&`bQ)q70 z;@>Qvg<+hN!$Uu|%iJvqnR-OA1>a1W^3gwY3In%G_k4(!WIHMR%wxxB=`U&^#RZcX z+Jd_^TxtWUk5jion9K1eOOwY0JRiuovvF ztMVmtvp&LX9s8K+-}&nV1^dpmttF#G)hg}{GVZDg~hZF-cQ-BAnUzQZ+HE%bJoa!e_JeMN#WLRcc zW%VA3>Atx@Ub2n_o0*4OA(;y6$=G0tBgcqmhgtRRD`d+Iu!-spm<5p?_Y?b6XjY(P zG?1pc*ZSSjT7G!so8L-zRM@tJeuk#5U8nCkk~*<7(X}I?MU^s8D6OGSF9k#ONVZgC zwY)o(M>kszt&3PZc4o6bCRhR4>0rzJM(zOW_?v;%d-6Z2s-{y;A{P=}KGkTuI-C)5 z_{mweTFW-<5-Y#l#c+6AT*&Iq*|JFqlOzt9fl>}5VyeFl;(Z%iDj`)l3gZib{b=@% z-pt%mOadoSeYl{K<_j6=P0U9z)mFozGSQC$;(JXRZ}|-gynaOw5vlkwdtv?L4f6g5 zs@cap!92l{C4)sG*h%szszK{>1*;NEpIp~dIoqN*A_Q z!#*uJX;dr3YMjm9R=WLN8tgz{b=+(KmF0REm9E#Z1n~%4_jp-N)N9d%pBZS zvA#WZGP30($H}^*1(d6QZx&v$L+)qlW%I@2MyeML)2RCa2(qtf9CxQUnZOX3WjI^M zk*j|hG)-%&s5|!HqmRouW7xUBTAlAdy}$fze8^#BnH4TZTpFkbR1=udal`LBsj_&} zoZxFLSn5KJJ#7nA7uC0RMAvn%%N85C?Pe-1&G@q{&w7`aZSS3Fzf^XkjN! z*JH`w6GT)%Cb}fN0O;b2$9aA$+ADbLD=?(>&&&aV;W#}b711w~(z4Sm6=YzSVDw2? zAWgYU?+)VhU{{^=!7he?u4W2q;8H#)2O3U=iX}EI{eWT2l_PwtJ+P`7{P=%*kL6AY zJ)GrJk)u!wtCtZP`rE+?b`aN)kwc+njULeNG$Ui1nmVt%G!qRC6%@#Lddn`5$@A{Q zlJ3ov;G8@9Tf+|b>+n{?jgk`MVvAjba^yRzwi$=py z_X>biuTIG!t##>Y<;6WI4Bx6zjllsfS+!%#>z|p^J+)RG(-ohLhb*h=1LxT)E?aS7 zFuu5x`6-CzOLPd$%iO`bqrxM!z{YZ|DO+B?Vr9GH5rlYn`cN@GxeuS(tho#LgseHjI?lrQP|I-hn-_35rZ$Dx4_re-seeAg^m+D7)% z^B2}2Py+7D=;*&3uP@L&z*)>dDA<9Y#;}@>^xDvewc$$aXEfy9Tu->XcypskWckZ` zzp}j!&Tpz6oH0SuzKApX-PeM`jA)BgTP?a?R~JkcR;0lwS@(p~84lt>8FAvMN_ILO zot9w%xXBsyrwsgN`HYz9h16gnj^+|K1FK=p*LE1|&3ExkTTFo*dXu77W!F*L)&oZ( zNzldM061o9Bxkug$eR({Q3rH!KfydbLj#z?(@Qv{4#)nQb6u$Qw=5i#rAJZZzg8@*Z8}XD%lympcQ$n(Pl#?bZxz8&g4}^_gWfEeWq_ ziB>SGipdzQtVxLU4h^%8TS@Nsv|q#r^L=+nhQSn(bF27~^QvD%p4+;M%ARj|khRwsv;hUL`$nDnso&K!OZZtm_M_t2@tP$K%e{CQ% zSAqeuxJW0%&8(YQ+F^=atv zSFxkE)W^e>aPqKHz2CDHKwa%R?vz0O>8%LCogXUL%3euZ>3KoX>flBhdr9Uu0!pQ7 zX1F;}W||x*4s8t2aY)OlXYBdyE5n(v%dMVKs(muQKO%l+zwKb}1_2M`W?8eJr`BHc z-Wpk0UP=6?ms0}s`A-^N#@yOCyU^QIocztGL!w#G>S*}wbjnpXkWfs{tuysw+Su}FAZ9!!8}K$ z@OlWk1JCdw&ypC>qS}Sm62d0BbBcHyLw;w}1UZPfoCo$rm`s4{=ffO+_y7JWXl!}c z-oEAtQ$fh$3H$!j45PT;+IDT$I`0P3{3*NA)KOIJ%-V z@id82Ue;9>fa}Hm#ww@L6vKy9UQGAi=mxAJ@7K7*-vrC=tG)Y_Ma$^~Vr-x1y^}Thp$(y@JLgiJ zNQ`9R9JNVa6PeM18ghNPJ}qyUw#K)%(n z*yQcPxTaska81F4EVM27d?DM1(z+c#gZtQq|I_4t`n_@~o}(W}zn_1t&qhhG>xA9N z_hY-5yn%bc&lPGpvvVu#AAIin!QPzr21=3n44B_J^R%)YX5|PXWS;0xs-RK9M+IRAAN83;Q?!(+IA5~sEIO#bsSF<_& zb8_YgA6M1emPwf>SgfxDrOu`LxnO#>m(->>Y*tP3UQXZI2u7s-|NemZ4`TCeJ`ySmKdcW{ttz#> z4Sytgx^~Vgv#j!CXXwc=s+`ivx4#uAKA&mat2e@Wr}L!Z=XWs`sUvq2%uOwOd*A-Y zo_~JA2j{L&QVX$+sd6-Z(LdBWw>RoCp_@=pwYim5)-*Z#e(&wf`w7`wv`8g~{s!>k zwOLSYHGS7S1cIHrLNz1Job~GvS?OI+A zP}p_AIJGD`*pY9Yya#PzU03+Kbkgza#ITUCO2_({GMiy6KZc(j19&OQ;&t{-&s#8u zYv>oHRoku4>8{M#{2@|Vunjv+FvRHe>QpGis&X_cdK}%POePl-n zeEdYEfGkjUj7F(yIFFWxm?4<{a)N=fFXdoBz8*S!ST z!nVsXzF*_)hU6KPMY8-m8d^Rezgs%Xcthyw|9TdJCXwpGn3eQcZ~J35!y!NwG3%XT#97G!>>0a%Nn^9kL#tO4sYfC15EbLKNp%V&d?6!Sp zd=+8G3amS0zglmq*Y7>(3L4EF5B_Mu>AwanSnIwY(sbhLucnt2DWI%6ing6VvGsPJ z+ps3!;N=lsT4&|~ZFEQbOcPsdayrgD#!35Ir0 zFBY-9k`Bvcix1-D4d5iu?NxtLKZ+whRv%Xvps`^~1)E`9H}9e|+=9)<;jz zEINF*uJj~F#i`)7GxmqH0G`8WabZnpaAVXSY};v{APX8&zQr!g2*=&?&wWiCm<>9{ zUsXedME3ND(fzqRFoCPNDUOQF4c6uYjt-QMI*e8$ z?%+)%*g~5fmv%FBIWyXntT(sm0N+iz@k4zqrfF8&WuBUR55(Ji&_sBK9UO7O(!xd& z=ss$72okXw1RCuDA0l_e8d4m=6(!q~{m4f3N$P7>H9MX5LoNo)*TeB0ntv_O}fk31IGe3ToKEj)$Vxm>I>Ju)c>!wCwA%Tcyq&@gG+zv zU!Fo2d;d8w^Arsiv@rFBX|f{d>Xr8#2S)tO28adJh3en|TI&sZcG)JB zZ|Kk?r39qYNzE+xOJk|_N9=5?hQg*mQD0TWC z^Tiyfkgo9+_e=J%5V!SqZDS=P;{+!)tm5P+YGc{V{h%=gC;zjp%F|-TT3^Xl7{kex zUbA#G%+sPW$+DadC}pTYH1u-GQmxOvElkzEcj6>1WVtDJ8l7GA^pCDDl=jH3cT6!> zjj*{Xyfg0(npY@OHme9~7L9qFUjs{-HAU85I1;Kf{n zXo;aOYJB}n+VYboZ#EJJxL^I%@gYC<(Zur?n&q|g*9~*6p4;*n>e9do=*=arakSbpHHOeD?bk&j=#)|8GmGbG!%A(HU|^;GkLipj z%a7iiej56t=c#{E8Ip(DpANNcEo}#gU%!?*v{$`3vIl1L76^Z69%$)kMrb^Q+UlD@ z7!6$>BLp2+ig78`*0b}6v|lXj*1PkZ5oJm59gs+yZInOpPH7u;@F-6+80PseN9T5Z zUGBX>w0rdX0Bq#OS_2-3eLhfvYd1D!0JB3x9Y)oxxI)wIqDOfx@xr?}l=c1LL|sa<1= zSXSx9iHgZVc;}qvkSIow&@YH+9lN9Jv|-wQ9g4ZC-WUwG#Z3pKR_6raQ-M_Bt*2H- z-xa^x%c$OjckFkH3}xK%(+xo_nQG%DVzKi|FcIK%yi6@N#>>sJrN{iXLiyCM+cT~yc={3K(p zGpwnND|;X1P)v+CiwMVUZ{IN~`6`mG_W6v>x7vKA>&~A?M(&?awp4x#=H=zjNHd3H z$}@V1Yj%!D1Q=o{J5wsSrqQiqaTXPJ6dWA%h+=(BPWx|;L}A3cn>R|k#gZjCoSSwF z#TRRGU^G?Wg3)5645q*4s5GeITQVv;YB8DN6v=atI=_H+es_0NtE{YI7}2Q(C|K5P zisP*`T#r6!Ka=bVy)zoFC0YQAiYLQ}D4NN-TBy?+47OIe$X0DVO*C@VAG;gnBu;L- zo;H2f`p(VOI?ydp-flCFdo##uZ>R==om$%P0_#wPl4NBXt*Sh!RbiWr;Gv51|Bbfy z3~G7-*L|@pv7p$HCZN)LkzSXw00|PLLqZX$A%q@!vCyO{kf5OnAp{5^3P~uTNEZ;0 z&_nON_qtB@{%~i`K6lQYIrr?3`Iedd|C#rF-sgFK1zio4%2Cq}ihU9yS?ZVK{8Q0< z#&J~+uHG^OhqQkXCkl8Vz)5pD<*Jgr*H z+J=5z8HD0xDTZCjDWmO=%AG2+fR`WWs`X(7HN^ZD*&MvX!*Ijj(PFE~^I%EYs=(nO zg&Jbbs_VD@b$n6Sj;_nG{78_k7B{)9CzCsKGU?88Ji+r(PcEP1-Zk^Jcr0Sr&VE0x zv+;fI{QQ^!`{VK}^I1>sz_gzhVykEJe8k1Opqq=)!XX>6Xm;=6yLzzW>b`(A(qn@=nWVd z^VU8OS=#%P1_h)1NXdtCN5t`cSLs2RdUxQp#6LODZWc1QslSqWE5*W_b*Lxu-~_i1 z{k!$;g(JXr8_;H@IxRI#qI{r_8b{v9LIVc&+xzqJbXYk$CV*_3TIw2KuqX zY|nke8VrJQ!$oy~4l`G6!JvaQ$%4aP-Lg+dwD-E+%Fmpujt}M4m%l^Pk_6dIQ2j3$ z*I&b^vJ%C)$KuxCsNrL}8tRS$VIa$`1Xz%{s*6i8Ahw7Wf6KJ`v!T)_0%BlJfe?=}ztQx|?I=vx$S`glpR3r`{Q; zs`z_B`yS>s8=(ZaK(=thIJ6wq82D`OCOq!;_ZA!^|BBz^x3Q8E*E~IycJka4-z9=A zo{OxDwzbLc8P=B8AokW%hFw$PuTp)}qqx^Bbt_GJ+|Zou8z!|Z5+%Z*Ptm3cYGh;OP(Coz`_MThSfoH}28`t%+3_~$01zI5k z0|V!KMScee1%dUN{KqGXSce*#xp%l+ST#X&;8blB*9ekM^e1PV)u{N5PRJt7*1^ed z;y>=`kr*Gb+(r`w+kX+%OYA`YhMu{kI&mDZ<0$V4VhmF=9RaHmL{sl0@)wMu&ukI| zVMmaLJrOf?iZD#wXQo+CH%b zwD%TpQ4l;3YomwG&)$&hFR^HbqW!BwtL1ef-`D7fndEFJ`({A?=(`u0ROR9Q;Yw|I zPCfft;t!X#wAIh`_Sl>|N#AA7{4_sFOO$(md|0e$mqW#4(iN}nJmRkg48@#iXm1CUrFbBX;uRgS96|xNh7R zpWsbm&P^6NA>2qdvzA4vV(yEjn)r1C_L$z3A%Mr&zL3LdHE~&i-J~oa$dmBg^lniE4HkY=Gm~LW^TBGlsS)8@_yP+MsHruSBA5F4>m}x7JvQg_`luPt zU&;ypGQ>$zfnn_>&8$}H%rdS0aXH$=Iev`x%CiFEUSwds6U(FCe!BukSu5H71Z|nm zbf~hDBkT2k7olDXf-U&^=Xu~LNys>R-xzMRLQmOjd zluE+Wm!DM&PMxRQBz!3r_%<5w%faoEN}pRTx5&rb>EDJ}RP6mVYk#y%D>_H#2syJS zsM_$$DLGpD+Sz!cC}(HNaBhyLDOGUkp=w88s?QwE2bMM&Q1@|JyF?FZ`qR!WhiOJ8 z^QCfvie<5yYu|aLh+kQPYWcNyjTP;GGSGRU|}ks00yPInBAF6*}&g~87LWpLh8Rp(@mKpQYe=IGsw#;|bsx)?`_m`syjVK1Xu zX4A&4iuK>W{Xe^@Uv@M%GrNFzXT~Ixcjx_8)!RC+!tnyBDe=jv4yaU66=bm&xRg(X z;6qx{^<`<0e3TyqZ>;Z%3YI(Okv@HMcmW;I-nQXOae99|cHV!HrUD)ZS!<6wn4_$J z2RlW%U@mGU@I5cS5ZPBEy0tE`xd!m*q)VrXPsIChAM)1xYUae0sr$)wr3Y3#HCp36 z8nDfk=Hia1EyKncT(BFz>8i-t8xb<+ooLitVRWk@DpjPnkWr}JD84Uct`Mk8_MBh} zQ-;>w=!1pvoVtElh1_kWr#0uFv5>|m#_F+$Io}JRp`RWnnbN+G>PJv4Fi&(scw6`G z&!lYe$cgmPuzFmklqIah({zMfu+X%zeh5opi*A>#Vaxb~IV-&gLUS#L(E<9}8=u76 z`Dd6AwX@@j-w8HGgh<`0l%Ljp80PR>3;NzVYybFuHl1Fa7>Y})WI3<3sdEiQ`Iq5D-2WYJIgwi2g7X^jB+i7Y4o6iBAgy?$@{lJrE z=W{&-F}0xsg4u;e7L$xTNK2(v&5qNTlMe1Zo>sB8c?52&xHM;3J7X`>B;SAd+w9Vv zw}ovss=KpmdUtL<7I#lgKIhW&V8x8}A~tx2wogYyr{g8%uZ7xjUS9av?#Q>?i`*fk z^`#_{au0kD64r!pK3wu?A^flKEj=mhDp{)AdE((h=%(=skBKA-tTgs2aa&*Tks z!U`z|pch-xL(cl-s&d!37oDq9ic72WWLInM)DuESXHAghS}%AkXEFZtBQ+*J zeI!)Y!yvtOpsXfQB+0ygZ^!jqujbhHgo3(rVX(Jun|))cmv9hd{*U^>QCO`tqPj~7 zbZl$w@4vA!45#O7((Jj>Sexqj-P11V)Wce2S$vN42buh6j{%0uzlI(BxdH=Jy5isL#F>6F%-?>Azu((%y z4f%pcL)KxEfI-d=triiXuU7yMl2Z%GV@eg2{Atu2s+?kUi-mXJfutHg!Q0_ebt_7P znFA$vy^!B5TLyv=RjEHK@|njb99{dl3-lN^n^sIO}jENbwZ`lQbk0rk`AiSW^vJ~F!-Ei zvT4u+RUzCA7qrpYzXV> z5nm;SH(Inj*5_THB?}{`gIJ1!ABHNu{vpv0hZqxGb>%CI?B-kwZ4P|1YPZee9cFMp zoh(EE0saOG9jld(K^R_FcZs14)j2!^&Cq@pkF__u~%R+kLnr=*<(H`3mgmVbJO zbNMK~_uh~TX(Vi#BL_;Ml{~OI!MF#7l;k3+6uW0@ zX1WnCu~N0%ltiApy8&+P>J1@n1i!?`JBFESjzXp%{PYm*i}SMCTvqiz+~uv-Mz2lG zwtqXDY|Nj^c(^9szCMyryUwa;Dw|Q*2W&SZx>{?@LA3Njxk$sh=!LRdEVCiICW~1A z`_s$MY7qH|wP4vkN9r1Adlx3zVs?+)K}vc#8KK#I5-#%TiqDRSeVN9uaia8n5n?J8 zW+P&=GFUZCCQgJc`RL6<=f1SxnB4ya=z0azYm+VMKN&vpe&Q4um6DiP zfB_7Jdm$ctw`IIXNY3Q_9x%)XS+z|)u8Y>`mW+cN;Qf0Lyg!(~({W&l&wvk+xjw!*-bTW~ ztmW~ic-k7FEH};Ncd>hWokd%{=Y^G%A+v{2BIUX0qYV!lz z>YQ02(0<(NkvknvH=mUT!awxjtWyK=SMhtlFjf+L;bW9+F~`u6WtRuz|6!@)Gn z$S_4MrQV><((vPT|{sfDeLYseZ0Iq^h>X)hY=nB-D>DINhZ^;a4Px%C(goxBali_?YKI=bS{wvcrLc`H1*|yGuj2#TJ+-Jj0%p|x zSLaxT;KK6?T0P&_{x0>_3AD&`e$McEmx8>05w*gC>8F`Nh-(X> z!N*Jf?U-xsduAjex#)zvXSR$VAN*khGrHP$FS6zD_EC|Acb}NH9$z~9sixBJF=#Ch zsRI*7g7=P8J{qU=3k($PgLI*^W7~g@{B2ym$(dKGxWU#XFpGBw256Q(ttu&NGR!G+ zN+Ua;m|qsJ&IMX3K+E$7$tNqbh?4XZxmDi-MET%Ek9zx-flZt*X|C#Hjq_B}gXQ>= zf$(9^a&`9{FGcSNpY#A;GB5k(z~UmG`ufK{$R`=bM6?37DsyeTTW^2ZdU*q#@+!Z$ z=0MyUA;(Rzfo+hHyRM5XbkB6uu~DG6tN);Ke0KBwD_qJCldr0LJMXD=WTgClyYtrD zZ%Rzx+Q={#99-SqHA;=Q=_o0u4u-c3n)lGT=$ocvXO?JUq1tF(ja!vVDzD=($*3bdXfR%`gj_>f zaVH<3#{52P23yBa-IuXNDI{g5iq0iaDUpPsbB1{&{)!Fy8n_x8uqD&b#W#w+!4wV z8&?G|<@6`o9Mk{1@yGci=HsK~XEp0C?m7*<~ARq+oX`LqEy8+v?n zPmOeY6Y$~Qs;Rs^CsdgGMsQZA1kbE%!0&(TLeZ>{j-rGKHAs!@oZr|(P zz8Ck`+&osJ3+E{Xd8HGbqq3k7J$bC+>kG_ErSY&|&PLY{hvFOHDM((H?!U17!p%l; z=X-(PQ&aAL;Ft|j{=)UC?E>TEMAj>PSw#UcodcO@e<8b^YtzftvPT$>8_0ccz{|y6 z0QCKm53kh`W3mss+a6LvuRFm1kYVflPtx7?r-m| zW()RtT9W=D=$p*Hm|G7C#t2}~(!M<_3Mg;NyPWiLv$JMDr%?Oh8`_8xp-ws3NnwosQ@WW6$|N4Xmqaj{ljAzl$IaC87(HA=VU)4+V1bAMlGKb7!kwbD#)FyW6oPQ0%FFT?k_9~R0t<#+KS`ese! zbR$F%f?sXUXE!9Jv5T!u(BSrW1DW5#1tU_OPFtiIJ|yw6*^ z2CgfLKQDc|^o>hO{lhjP305;QeommQpH3`>HkkFeAU{gHr~ZE_bEpGMcUNs)Fpkp)gb%$*%`L z?BbfH%MM-|LzQRBr47_{%#~iaXPX*7VBM$?FqncShyYFaWsmGnbZ&c=-#R3`M+P@a zG9tV(3WMFuZrzP!qQvB-ROejold{aiz}JlSEsBzm)xw?a#kJ$Ta{4Nu@5FxD^bwdQ zucySQJuS~x_w|FKGH{(?PWdJS>ZPwimxVzB^^w^a{?Px!D2xf6k zJ^*y-so}Ufy1OfMWnvwYAYuT^*w)Y`t zv%D!kfAJ{rJ*nj{_77d7j&Co)C%|LN(O>M;GBB)+m&+;DOZ$TcP`x6Eb)dYT{O?=W z7kHYYFT7jSX1;uELaeq$GWtMN@RU#<07zt@gm zJ^Z{*a$rTjYLq}5TVD=H5qle-=)}9T$e?$Y;uB@97i&}Xc7;^`Tj7} z!YTwrg^u7B;KQtcbwTGy+Pt_y zWB9#Vb{pk3mhPC{lII<)!}!y`ds+e}?#Y69KALL3NERm6nBVy9zH_tt*q!z1uYjoc zbdwlME}>1`|M}fC2*0Y$Uq^Dbw{)TPG~Md9#kkLYh@PCUsMwqv+~13^ z6lIG2tMx+Scmk*Vm6r}b#|y9*Ys)X7<|?vmi=6J}ju%A<7ZUi{)xo$m(P7#twRC=K z%OV4QO%65t1Cbur{92ORdN-9N596LFj-#~PT0KfP+@#ZYIi;g5sc;xuUi4G7npv}v zG&TMNXR_V9hFN&q&-vQ-h>(}!etAToFGlbRg_uF4^bpCR83rO=?Oym3C2b?XcvrFA zn%uAr8pF4?72LU=CO#Wirx6)B%{hJQV`}5SESKW!7Q#}4PT6&|?U+0b)3<~tn zst)WY)}+Mg3zW~T()zQUc*PoxLb_NW&@F8iC2;@xSsn;*P*yte553lb@gu3>YWm+Y z+Ao{}9GPrq|7Adm%)Po7sWddi#wbU{n#rGaASam6n3EmRBNY|6xh-?L7&q(c%wx`S zxDm5yCsavm7_>BjP?Ak2$5WGW4}$gXu!Pa^;X9{CfAE>MmhbmZKDSC!ka_BB_0N+C`77op9}+)2PVuYh zO)ouP)*CdfS_>O|yggh%O@nhj@7hnq8TIyF3ms_dR zlg+ia2^a*~r6(Bce>XmrZb~Kxjzm>eAj(-d-&rJ%{fzgJCa;P)~$@(8EpOFBZWqs;yHz;F;^RkFqs*kRF|)I_eC zUEK(?mW3Jw>u|3C4=rc#7L2uh;=~FI$K)0zcLByr*SG&k*Mu!PRBJ6OIB->`{Px@( zL(H#J^;>wnz3cQ%?86oI;n?W7AmjNmNV|V#Hj1il1`}OQA58Dr)p*At^qKqMMAN2< z&8<_kG+EtBmIG4tIcl|Fq9CX183SX`4$f)K_R!a!G)BlL&ll$ei)u_k4!Axw0D_(Y z{CkloS7kxJ9KgOgRUwN^qE;vQZ_?xPaj(#Nf7W&}s*HCu?L{REl1HtVeIBNK;BUIc zaCtIvc>0Y``!xmS^T9Pae?{yAhz)n7V-wyk`lCSn{2$+UUR%NxY}IkMxv<1G2d8H_R{&$ zV*>aItLd-%lSv0%=_L`N3cT)Cfo71;8TV=chllawHQ=I(3eo_Q^We0Q?muASS2HUw zTEkFD-99&xjjOQ5)Yf&o?^V`Q-rb6ifDoF2* z)ZZcx^UP^GAAXJRqpU2ISVfb;-AGB4YTpu_zYUE^6)AC8r!64FF8U85<-DFQt&(2u zq0_YQ&f#K+$yJS0kB#Ey{N@%i>}{nC{pWmqr95l0Nq|)f)cK*-ICc2Z&_p##HLvLB zT*7bippkycsa|0KvOeSCmw;xU{w9-TQV>2rjgOl+OFrv!?TH8V>S&6Wr1WNkx$b+W zJ{kQ!6b4rE>6;_HXj2(4UZMwJ1dhaKlgqUdo8-5-H#N=5cij1DD92>+ z(SsCu4ZkYKPvyjljAX!u!96R5N5!k&BR%!@KaA!v(aE#XPYIyBZ-uR3-GQQ_Y2kVt z))`3!pM2Coip+SjSroycEqekpdm*a4U|Q$!_VY*$yNFEdKy>FBAtwArq_Y!ri9?|| zoU>STO7fz7Bu4n+&G6PPntrwP!kORKyf&yhhU4uB`au}?pBJAQ#J;83FgZCkR%)jI ze(D(CL^*lM4MYUCjZ8I}(?JYqy1}y6E_Obmgy60@_;dDDFGCyEQ3|n>!W~+SE}Wq< z74z3r<+(}119@4=h7EH8g4bHw+s3)^x%8&M^0J$n(;o4pI`*+P5|&C_a#vNE8h8}7 zou5dX*S9y0=H-;*GPBTS*Kc~?nSMFwp}DXnY{{TCLxpTfq)eG7N^6HP@^brxRj!xGn`${EMMUcaHOGD!epaZ*w`$h zHhOa+qFQqDuJWBo`{veNnv8?SkmnQC?K(_ah7H$d=|^ih^Q$}Awh zl{2LDu(zv)m$%>(Br*|6>S$PR<}%>lDcm`)bIzZ^R;Gl^a;ku zBM^wr$lLRFUtNSo?EXK|^Z$C3{ok&=|8;A=W!xOb+;a-H=g&ubaK5m2h@H$WlL&wR z_p_ZRf<@fsMYjV8S>1;o7qX|uRJ9Zv@37GlMWoMWxtC@9s&n9F^Imnb53P@CR_R3^ z+&s;icRt|d{!(oBO{D?515{2B;=Xw)wN!mk9^Plnbfbqe-rZ6b7o6$w)8AZN9XsW@ z@nl4l?9_LvxB1)PM*D&RwCvd%bk_a$V&i)bAI;n zj_V~H!8+^uC7@BfPxc?Uh-Y`)k)v{8q$|Ce;cK6m)(QMGy)mq% zu<5sKIbl=mNx|8>DT4=#+;Tm5(#9>@)^O}-LLQ+*jkt17rfy$3nIkmDwhQH#jd_ zhH4ljC+>miFW&80LHFACw@l?u7xj2$DwViYPO@CQ{ibc zc*Mf2+vsh0uGL4u@@nFvb<^IxbcHz6?+pJkFk}V$oO()eimsK0DgQ3rTlHcd`IB-$ zeU<7QI_4(#qJtT%FWGTz1cEKD)95-?Zr^#ZGR4~e>oJS?7ZR{i4plar15n7u}&L9bu;gkey*_iT5NuT-bb`B`>)QA5E{q0LwBeQ;-kLt!T&P z(Wf`36|>arwIfxVVBKm&1^M->vpPPU5__;l0mt&o%wr=87r!^oTh?A9;HIrDu$wJL z@l`9z-bVdkErZPR;v#W-E-y=U{RB#ro#S)AI^t|xLhrCyw5a=F_O&F*4XTgL+3 zNqF8Gel^5|+Tn;^`eY4|;na5^Ro5;3B-n4r4s3~DYB|SkOdSrg>uW30ZUN+t zG66x5nq{4=yQzBzkiLvGlO&GvYq}bTz~6%7vZg>^DhQMS1bm7Bo%78A6hh&j>1mS~ z4Livn#FPO4CQyIu4^fV@T$L|hjvrOW>xv3bk%6~`e!cb-U6jhOd!Mdbl}a0EHwwY! zhLjR{ow2ee=~WQk#-`1Em~Pm9B-2Z{6*Za{Q9fHODL{ZIwrM)!ivzVWCqk#DRLjmR zibudb4e|9f&*-@mN?sZ3iQdDIjkgsYPbLuuLR%8oA>Pvx>44Ks{Ryy!OQHM##d_BW zu^P}{8vDnC=Ggywu&J<6787b-vEYEDm)H1#I+807mA3p^f9s0-2G@P!UCpv^9J-}F zQ2j!F_B~6S&`-B_pG2uq(5S)=*FxO*vJ~CQa3rukkU*!#mnWyCtNz8`>1tX?Tqkt0 z6dA^a^v!XJhnm}_y4E#j$LelpGg~O~NHq1idAI`PGa2a+rk7(0S` zcEY-f3~l@t{7CD^!!;#|j**dTCj;rrGk&{O<_Gpp6us~bJ_DY&VZx$(2~)xTBJa}9 zWFB08w|(TX|M3#6-s;gW#b1*g59}hy6|w$x;+3M3;=#}&7)9csLY2d{o#mJ~Q3@l> z>3tR0z~krO`_2Z@8Z7xkjM2qYI|k~Gd)4dyOfuggx9Kj8xsOKyOxKzfzP_HG3+f@4 zhBuLRSDfa`-3R7|^ez%Z5VO0bfcvW>z0UkFB8foiKXPx}OqR_ddD@}g#Xdhuce{d8x_-(5c{?}OYPbOgr&oDPL<1VQ} zOXW_S>r#_YS@iJcNlur_XXQ61=^J1a3O>-3|9&ykmQg;wN{H>9SQd#x^j#Y zLfC6v@Q><}r8CIv$nnc9Y$3JI?bl2Imj0GHTm%DkU7A`3}MQ=pR! zSmpbKHP5H)mbixnbUYHSloJX=Su4K)uvM$yV&3olaO4 zX5ETHo7JKOVQ!h-+G#hY`T6FqwxX7b5Y7UKk~J6O1X{n1QQbdlkRX0+L&pu?xRqb8 z-EU~#_ai4iJnOer)$^VD`*KXj3j;xK)OL{<-?HuSb0gg`{VK#=UC>4fGQ)bTX2D~V zd{iS6BF+Y}A&6490GWal2%ka|2*xi*P%vEX9*(qW5dGNHPq2@5a93 z!V#2Fe5$$snzVYCl&*hl94WP`)$b}^Jk2qWBa{evf01_rPz^Jjg|w5en)p;K$J~cs z!g+Po1feL(1hO6S2mxMvsd34E+szi*#9`DjgZAdgp~Vb>#B<0 zfajoCi3fuepsnkgrqfE_uwtbS^Nk7JU@<`vud+ef^iduW7Gjhoz5n-i%j4eus3m7k zv7y9y-8adcU*7`tYH?ECe_c-o3v0OcIg23H@?IYmt@I|}dWvD}^zc)(mAUG44m{f2l!Kq) z9vXK_d8MN{_PoBte$>D@?qXtaMq{R`%1OjBh!6H7xDZQF(9d10V^sxnv`*PZ;lC{< ze<(NK8fyxYD>OSP(?RnouqIX9{g**LHa+puZC%H0uS)DqN2+)0f(Mt4xL{QKV-f03 z{10HR0b&@u;zeHGcc#4kRY7=~kAC;dKFto}_d}`E!tldV_rr%zR&nCs#9JUjn6&#s z!4gjykWfT4=eeu?#Pt}3)flq5YjCKu`=x&6TAhyJ@+xH=N=Wx9J#K4x!`3{!HSraViT0~5t1*v-w>H{D~*X=B>0coenHcTknC={Vf7I4I5f>4I2$#<#5vJjQMbIiVLi2yjn0yVlf=a0`=&1wrJ{7txj@cjSKv;6g;Hg z!V7NCB~SBp4{z8%|3^W(FiS#Sw#SM86jggeVKT#~Y07DNQUq{jEs{!y!@@zzmi2T| znHdzarJr3l7h>=!ATqUg>Rjx-1xm{6r<0fA@wufbUJy`RyfTI14@wo(a5OH$YS3dv zhm6w?BDWaz6so{Aq`g855{zTD$W_^NK;5O}36p;l@FlC^mG|&D3n>TNJ{h+S#!UL_ zEsHl2QKv=FW8^7ET7V2x2;wd=d9hb#rrNwg346`H7MB zCt-;%`jgEMGNQhMv^WRa)Nkj|2mSx~kobCq=d2ME`#+q>dm0Vjm~!^H`@Zi3sAi; z@7&$;PsC>(ErC5}tq<*{LI@8R+zqfnldn=a;}IAVD*2)5j$g%R*dtBM4VOuv_o+bt_DA~klk1QVVo%)@g?h`{mJ)~W|YVcS| z;mP2U_{mwOqIO`H|80hq@EGV}rd{VhL;{Y*WKrItDyc@^x9vaon2N7HI==H^Ef!KioTfod=ux{i zd|gQcL|3s*SP)Hjk+1glWH834vmI*CU9R>1nELCZjQ9AYwPE?p2TcP{#mL@_8Y>#! zB{Or7tK#LOKexD{Z!`D}?kQ&=XZK5tGTe$QMFly2c!6{#h`tbkq|D!n#{9o|uqAEYK&uYovEkRV`B=c z{JO8g-AIrx;>9bkn!Q&ki|enF8@0Z^gK@WYI3|55`>x8^or(fzAx(LCTwa?nuuTC* zy}ndIZ10(ghl^lq2j-E&ZI&cL+1MV%+d!hekCMopN^lO!8kWzl#fk4RF!tIxX*5GqJ-dKd4Tji!#B1+dB%?*V* zVRVKo20#^CO@*AfOYY?;m*or^0J zqNkRPlY=y(_M~4yJpA3sE5DT-hfLX8*g#&0(F`o;mN4C+Zz7efKa%TVeYXJW9=~EbttF+Y^auOy2337W0au)(bMw-g>(o&gPCnN{<9OqMppUTi9)#c^JQgoB zR%^bFamE_r;F2PxB5hr!g48wk7lv(mrgh@)lDrHUA zk9`k_Dn(%@O;hE-L2Rh-j(C*@IkT?+v~fS8GS|bESshA?Nj9E62mVUy!TQ- zT0k_A%@x5(48&aDn`7y*WmSH)l4mNPTV9hGC8_4gD8pxwiE9J# zd1HeI0Wh8E@Wu}I)41$Apk;;SzxC8)gh^w6D{Ni3$OT-H%#ciTyV?SHA7%^8vP8A3 zb})VKgW&=F!c#(|-ig@8-fH-O@wVAvk__1qp>!e_lO*0sZDGYLUT8vV=4oduiY$={M$k{7Iztye!V{RBz3YKcdd zsA}`_cTCg64~lV;SblWQ2M>13ks~C#iTfHcuV^W_=ALsH>xJxjv-8fWjL;Wt5}={E zJ>@&KovOwd2gC-mV_00eCRH~3eK?UhhMehav1!nle)pLa)-d&Gj&HADajf865=?$LI*{7{V zuFe4J(1(h5n8i3pZt5YQ>N>lr$@_0no6_n}7L0)GaHcvzEb03fm%bu$?+P0Nq|cx0 zXSAFFa<$d}ut|w7)HV5CG_N#fveB8?+@2|La%Ovtyq%FGYQ?b86Yv0JU-2bV!w|=$G>OZ<}Kj8%&mjAx21EK0KU@qXUq^-RtF!Q$GZcJ6qmlaf*fY?Sf+dTu(5QRboltv5*Lly-CuyAp_#8W*-JqBE5#k1Wzl zhUz`nL)nr&TRA45W?ZgtjxEsSd`sAOZ7b|4KzA?!Z_(YVAzEq=t@tt#sMLp(K^~3S zACc;E?{-fXwUeeP_gk>JbeSO7x%~j;mh=JcuzGFgDfr%8SKKx!v_!K>F=RS9qS9r? zj%WN&P3qbfFkyAnoECSmRKt-B@IFkGnk+VF>PwWoX*jQROTxU~$un`3*2V1XHG&tNC|gc3)bPsr)Qomgr+2Y84LR*k4)^IBCxHX47=;`Q7|%-!&763z3Zg@1-y}Afp#rloGU5 zPTA={75yphX^GUh!Czk1{KoF`8caCYHFcHPQz!^CuA_8~hIQ7lis_!W9ez|BV&0a( zJ%|g73|yA*{pBd$&Gyw>k#F+BdJ?w-!u-U5x#xpx=alR8V$JkH)Z>!U z`x)zbXOn4B0i|`tWD{C|J%lfKy zt|F#c7JZf;kvBG8y9=e(X>pRUrbN0X7e*zoS8NiZoc_-EvLTV9a`unc^4;96c!t@( zW@>5SBcrUcz4))%+XGt>#9n_fS*wk~u|$f^U`mKmtD%=`B3J?pI_)X@ALU(lP*eF9 z20;;Nt{MTCwhIW-OG1;fSkSnOfe;7=R1gIO zfj}ai0HFs67<%;O?d-%ro3-= z?^N@n4M`5Rcu8+C0=Dlv|5gOOb6ouz>c=AIhl2G@`YLz&0It~Q8H(1x2V*PA<90}7 z_Ocf(2~J>)*|m2+;NVlY_+g!1O`m$h4Y9G8P?z9sb$ipvvv~yYw2!lQFV`-leD8#M zZMd`nulkXJ5V5}2=BrzFGHIGZC-}k?AdN~Vo&Z|x;|xMF*K5VPiq?nycIB*4@YjSxZpE4C+KXdffpbjr4fa7-2HZXx>B<(#A<%k4RLjl!tT zZ_GTu#q+0UD^o&~^SP~+L%t0IeKEAnkZ=67Z4A`X)#_9SbkguQQDx<0*_qkvv#o2U z;&eD>zj2GU^|bB5H*GN+RXL*Q^HeiULO)WrRR2joD8*_{xPJw6?jUeLl}fMztPdC~f zq%-M}n}!^d$goh~TX7{3H?#4MzdC4a2`}(OX6yW8q>uvQ6&5QFWuoVwlQc>wqKn z^yT^ns*;Q(%fW9Fqk8ph>?4LVDk|*WOzHW$0HPQu{^X!GI+&Fp-u?1ol*Gylp%||e zJyRy*$A&V5)3O)2&l?%qT!SLW$fyrtQ|;dMzVw2~-hnP-mso}_i5)2_&1qPn8q4Fw zj?%kS7OeNlyl`^}3H;r!ya^zz^<+C*CbVvqqaC%UCd?WS9agy^fyf-M4W+C-OfDBt zQo2yfUg63iy|yq3OV_=vS@?`(@mMUIeQc1qt}Ecn>0 zvwQqDpueg}Z&IYdX6>kznZ<}41mxRPP0w4m$tt!TAhp#JKwwe{K2^2mVVRH4da?M3 z6cl3X&e92*cN8zf+2Hd94_KufD-=Y457qc4e@FI-*8j#8BidM6bSlGqANYjs-bq^6v2f(X;Nd;5PxT_0*4Xll zQ*)Y-TLxA_QFX6P=|N1bSfk2U#pyvRCuZl2DqRK~wQ$uWXwrmI&V67mIw<(o>YYEw zW9Vld8S&}S_Gh&bzFVvdVO(C?2|HS{3l&t(Oyu!h@@N#0^EVs{Y(B;$pf77X zu>88f=mIPWa%8ZHgU+<7+W^NjYHv?AN(eMeOwyPkOjHn4qG=B2Fnd_geEPTL^MlbC z=Rme+B!rsR4ECw&l@ec&ZmdV0zg?voVGJq2*TzSTfJKHlwA8WKpchJy-_GnA<`Gacmf%0`1Gb==`X zAT<^^^S}N8%F7__k88`nFhl6)7j68kf8QrROf=MOo*75>ns=>_IFPFlZecFJ5+^7( zZX;xGaGQa}J!mDEeo{*g=gU(uYw+Lj*BBq&t zzOPRD`Qux_o!>X^di3z(sD-4H1%HIyR=L|*JfoVlXMQ+|)*bdS$s@|ud6^DfTjS!>xk41QQL4 zV+9(0x2uki5Ea|hf79gL{>7eX@fWnfu>hont&iY(Mdit>vClNak65L>moTT>4 z#*a79j?Ld6ft(gzJ3B{0`wpGWtht|EDfH64{%+4u!nPG{$Atna6)V^b=L(58*Q#~& zz?PEVnsO5`-5I0&n;fcMk}Hx%7%gwNau^5KMrpK)EFSKeg0o{e6*)esdZhxfmGU_a zBXdW-_qXk2+XXk61LivL+ve@VNa*56!I@_`@(e!%*c;1^9E5S%ZTN`t+a$$;Bv=<{ z#W|bd6%hW>BH=FHFSa@GlgalpQo$y}lW`Y&3>}QLe4$XGN0sB|G^n$ykGAIZmY4a?&(l+{_8 zjdk6Oxa{2&Q(o4k*aMsPiDO;v@NhYS7GN4z}DL*o8FE`w-{vA6aT=M@KJYq)`D4Jx-Z=D&E_(iR;K}a^wJYMa(=wOj4;Bqg9S36-hPx>3 z`LuQTK-y3L Date: Wed, 10 Jan 2024 23:18:04 +0900 Subject: [PATCH 171/521] [SPARK-46547][SS] Swallow non-fatal exception in maintenance task to avoid deadlock between maintenance thread and streaming aggregation operator MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Swallow non-fatal exception in maintenance task to avoid deadlock between maintenance thread and streaming aggregation operator ### Why are the changes needed? This change fixes a race condition that causes a deadlock between the task thread and the maintenance thread. This is primarily only possible with the streaming aggregation operator. In this case, we use 2 physical operators - `StateStoreRestoreExec` and `StateStoreSaveExec`. The first one opens the store in read-only mode and the 2nd one does the actual commit. However, the following sequence of events creates an issue 1. Task thread runs the `StateStoreRestoreExec` and gets the store instance and thereby the DB instance lock 2. Maintenance thread fails with an error for some reason 3. Maintenance thread takes the `loadedProviders` lock and tries to call `close` on all the loaded providers 4. Task thread tries to execute the StateStoreRDD for the `StateStoreSaveExec` operator and tries to acquire the `loadedProviders` lock which is held by the thread above So basically if the maintenance thread is interleaved between the `restore/save` operations, there is a deadlock condition based on the `loadedProviders` lock and the DB instance lock. The fix proposes to simply release the resources at the end of the `StateStoreRestoreExec` operator (note that `abort` for `ReadStateStore` is likely a misnomer - but we choose to follow the already provided API in this case) Relevant Logs: Link - https://github.com/anishshri-db/spark/actions/runs/7356847259/job/20027577445?pr=4 ``` 2023-12-27T09:59:02.6362466Z 09:59:02.635 WARN org.apache.spark.sql.execution.streaming.state.StateStore: Error in maintenanceThreadPool 2023-12-27T09:59:02.6365616Z java.io.FileNotFoundException: File file:/home/runner/work/spark/spark/target/tmp/spark-8ef51f34-b9de-48f2-b8df-07e14599b4c9/state/0/1 does not exist 2023-12-27T09:59:02.6367861Z at org.apache.hadoop.fs.RawLocalFileSystem.listStatus(RawLocalFileSystem.java:733) 2023-12-27T09:59:02.6369383Z at org.apache.hadoop.fs.DelegateToFileSystem.listStatus(DelegateToFileSystem.java:177) 2023-12-27T09:59:02.6370693Z at org.apache.hadoop.fs.ChecksumFs.listStatus(ChecksumFs.java:571) 2023-12-27T09:59:02.6371781Z at org.apache.hadoop.fs.FileContext$Util$1.next(FileContext.java:1940) 2023-12-27T09:59:02.6372876Z at org.apache.hadoop.fs.FileContext$Util$1.next(FileContext.java:1936) 2023-12-27T09:59:02.6373967Z at org.apache.hadoop.fs.FSLinkResolver.resolve(FSLinkResolver.java:90) 2023-12-27T09:59:02.6375104Z at org.apache.hadoop.fs.FileContext$Util.listStatus(FileContext.java:1942) 2023-12-27T09:59:02.6376676Z 09:59:02.636 WARN org.apache.spark.sql.execution.streaming.state.StateStore: Error running maintenance thread 2023-12-27T09:59:02.6379079Z java.io.FileNotFoundException: File file:/home/runner/work/spark/spark/target/tmp/spark-8ef51f34-b9de-48f2-b8df-07e14599b4c9/state/0/1 does not exist 2023-12-27T09:59:02.6381083Z at org.apache.hadoop.fs.RawLocalFileSystem.listStatus(RawLocalFileSystem.java:733) 2023-12-27T09:59:02.6382490Z at org.apache.hadoop.fs.DelegateToFileSystem.listStatus(DelegateToFileSystem.java:177) 2023-12-27T09:59:02.6383816Z at org.apache.hadoop.fs.ChecksumFs.listStatus(ChecksumFs.java:571) 2023-12-27T09:59:02.6384875Z at org.apache.hadoop.fs.FileContext$Util$1.next(FileContext.java:1940) 2023-12-27T09:59:02.6386294Z at org.apache.hadoop.fs.FileContext$Util$1.next(FileContext.java:1936) 2023-12-27T09:59:02.6387439Z at org.apache.hadoop.fs.FSLinkResolver.resolve(FSLinkResolver.java:90) 2023-12-27T09:59:02.6388674Z at org.apache.hadoop.fs.FileContext$Util.listStatus(FileContext.java:1942) ... 2023-12-27T10:01:02.4292831Z [info] - changing schema of state when restarting query - state format version 2 (RocksDBStateStore) *** FAILED *** (2 minutes) 2023-12-27T10:01:02.4295311Z [info]  Timed out waiting for stream: The code passed to failAfter did not complete within 120 seconds. 2023-12-27T10:01:02.4297271Z [info]  java.base/java.lang.Thread.getStackTrace(Thread.java:1619) 2023-12-27T10:01:02.4299084Z [info]  org.scalatest.concurrent.TimeLimits$.failAfterImpl(TimeLimits.scala:277) 2023-12-27T10:01:02.4300948Z [info]  org.scalatest.concurrent.TimeLimits.failAfter(TimeLimits.scala:231) ... 2023-12-27T10:01:02.6474472Z 10:01:02.646 WARN org.apache.spark.sql.execution.streaming.state.RocksDB StateStoreId(opId=0,partId=0,name=default): Error closing RocksDB 2023-12-27T10:01:02.6482792Z org.apache.spark.SparkException: [CANNOT_LOAD_STATE_STORE.UNRELEASED_THREAD_ERROR] An error occurred during loading state. StateStoreId(opId=0,partId=0,name=default): RocksDB instance could not be acquired by [ThreadId: Some(1858)] as it was not released by [ThreadId: Some(3835), task: partition 0.0 in stage 513.0, TID 1369] after 120009 ms. 2023-12-27T10:01:02.6488483Z Thread holding the lock has trace: app//org.apache.spark.sql.execution.streaming.state.StateStore$.getStateStoreProvider(StateStore.scala:577) 2023-12-27T10:01:02.6490896Z app//org.apache.spark.sql.execution.streaming.state.StateStore$.get(StateStore.scala:565) 2023-12-27T10:01:02.6493072Z app//org.apache.spark.sql.execution.streaming.state.StateStoreRDD.compute(StateStoreRDD.scala:128) 2023-12-27T10:01:02.6494915Z app//org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365) 2023-12-27T10:01:02.6496232Z app//org.apache.spark.rdd.RDD.iterator(RDD.scala:329) 2023-12-27T10:01:02.6497655Z app//org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) 2023-12-27T10:01:02.6499153Z app//org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365) 2023-12-27T10:01:02.6556758Z 10:01:02.654 WARN org.apache.spark.scheduler.TaskSetManager: Lost task 0.0 in stage 513.0 (TID 1369) (localhost executor driver): TaskKilled (Stage cancelled: [SPARK_JOB_CANCELLED] Job 260 cancelled part of cancelled job group cf26288c-0158-48ce-8a86-00a596dd45d8 SQLSTATE: XXKDA) ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing unit tests ``` [info] Run completed in 6 minutes, 20 seconds. [info] Total number of tests run: 80 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 80, failed 0, canceled 0, ignored 0, pending 0 [info] All tests passed. ``` ### Was this patch authored or co-authored using generative AI tooling? Yes Closes #44542 from anishshri-db/task/SPARK-46547. Authored-by: Anish Shrigondekar Signed-off-by: Jungtaek Lim (cherry picked from commit f7b0b453791707b904ed0fa5508aa4b648d56bba) Signed-off-by: Jungtaek Lim --- .../streaming/state/RocksDBStateStoreProvider.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index 10f207c7ec1fe..a19eb00a7b5ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.streaming.state import java.io._ +import scala.util.control.NonFatal + import org.apache.hadoop.conf.Configuration import org.apache.spark.{SparkConf, SparkEnv} @@ -202,7 +204,15 @@ private[sql] class RocksDBStateStoreProvider } override def doMaintenance(): Unit = { - rocksDB.doMaintenance() + try { + rocksDB.doMaintenance() + } catch { + // SPARK-46547 - Swallow non-fatal exception in maintenance task to avoid deadlock between + // maintenance thread and streaming aggregation operator + case NonFatal(ex) => + logWarning(s"Ignoring error while performing maintenance operations with exception=", + ex) + } } override def close(): Unit = { From 8a0f64274f44dd17a3e1f034c9f1f20a61ff0549 Mon Sep 17 00:00:00 2001 From: Nikhil Sheoran <125331115+nikhilsheoran-db@users.noreply.github.com> Date: Fri, 12 Jan 2024 10:20:49 +0800 Subject: [PATCH 172/521] [SPARK-46640][SQL] Fix RemoveRedundantAlias by excluding subquery attributes - In `RemoveRedundantAliases`, we have an `excluded` AttributeSet argument denoting the references for which we should not remove aliases. For a query with subquery expressions, adding the attributes references by the subquery in the `excluded` set prevents rewrites that might remove presumedly redundant aliases. (Changes in RemoveRedundantAlias) - Added a configuration flag to disable this fix, if not needed. - Added a unit test with Filter exists subquery expression to show how the alias would have been removed. - `RemoveRedundantAliases` does not take into account the outer attributes of a `SubqueryExpression` when considering redundant aliases, potentially removing them if it thinks they are redundant. - This can cause scenarios where a subquery expression has conditions like `a#x = a#x` i.e. both the attribute names and the expression ID(s) are the same. This can then lead to conflicting expression ID(s) error. - For example, in the query example below, the `RemoveRedundantAliases` would remove the alias `a#0 as a#1` and replace `a#1` with `a#0` in the Filter exists subquery expression which would create an issue if the subquery expression had an attribute with reference `a#0` (possible due to different scan relation instances possibly having the same attribute ID(s) (Ref: #40662) ``` Filter exists [a#1 && (a#1 = b#2)] : +- LocalRelation , [b#2] +- Project [a#0 AS a#1] +- LocalRelation , [a#0] ``` becomes ``` Filter exists [a#0 && (a#0 = b#2)] : +- LocalRelation , [b#2] +- LocalRelation , [a#0] ``` - The changes are needed to fix this bug. No - Added a unit test with Filter exists subquery expression to show how the alias would have been removed. No Closes #44645 from nikhilsheoran-db/SPARK-46640. Authored-by: Nikhil Sheoran <125331115+nikhilsheoran-db@users.noreply.github.com> Signed-off-by: Wenchen Fan (cherry picked from commit bbeb8d7417bafa09ad5202347175a47b3217e27f) Signed-off-by: Wenchen Fan --- .../sql/catalyst/optimizer/Optimizer.scala | 12 ++++- .../apache/spark/sql/internal/SQLConf.scala | 9 ++++ .../RemoveRedundantAliasAndProjectSuite.scala | 48 +++++++++++++++++++ 3 files changed, 68 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index ec5f00d34cd8c..df17840d567e0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -576,10 +576,20 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { } case _ => + val subQueryAttributes = if (conf.getConf(SQLConf + .EXCLUDE_SUBQUERY_EXP_REFS_FROM_REMOVE_REDUNDANT_ALIASES)) { + // Collect the references for all the subquery expressions in the plan. + AttributeSet.fromAttributeSets(plan.expressions.collect { + case e: SubqueryExpression => e.references + }) + } else { + AttributeSet.empty + } + // Remove redundant aliases in the subtree(s). val currentNextAttrPairs = mutable.Buffer.empty[(Attribute, Attribute)] val newNode = plan.mapChildren { child => - val newChild = removeRedundantAliases(child, excluded) + val newChild = removeRedundantAliases(child, excluded ++ subQueryAttributes) currentNextAttrPairs ++= createAttributeMapping(child, newChild) newChild } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index de4a89667aff6..2e41374035c8c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -4352,6 +4352,15 @@ object SQLConf { .checkValue(_ >= 0, "The threshold of cached local relations must not be negative") .createWithDefault(64 * 1024 * 1024) + val EXCLUDE_SUBQUERY_EXP_REFS_FROM_REMOVE_REDUNDANT_ALIASES = + buildConf("spark.sql.optimizer.excludeSubqueryRefsFromRemoveRedundantAliases.enabled") + .internal() + .doc("When true, exclude the references from the subquery expressions (in, exists, etc.) " + + s"while removing redundant aliases.") + .version("4.0.0") + .booleanConf + .createWithDefault(true) + val LEGACY_PERCENTILE_DISC_CALCULATION = buildConf("spark.sql.legacy.percentileDiscCalculation") .internal() .doc("If true, the old bogus percentile_disc calculation is used. The old calculation " + diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala index cd19e5062ae1f..8a0a0466ca741 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.MetadataBuilder class RemoveRedundantAliasAndProjectSuite extends PlanTest { @@ -130,4 +131,51 @@ class RemoveRedundantAliasAndProjectSuite extends PlanTest { correlated = false) comparePlans(optimized, expected) } + + test("SPARK-46640: do not remove outer references from a subquery expression") { + val a = $"a".int + val a_alias = Alias(a, "a")() + val a_alias_attr = a_alias.toAttribute + val b = $"b".int + + // The original input query + // Filter exists [a#1 && (a#1 = b#2)] + // : +- LocalRelation , [b#2] + // +- Project [a#0 AS a#1] + // +- LocalRelation , [a#0] + val query = Filter( + Exists( + LocalRelation(b), + outerAttrs = Seq(a_alias_attr), + joinCond = Seq(EqualTo(a_alias_attr, b)) + ), + Project(Seq(a_alias), LocalRelation(a)) + ) + + // The alias would not be removed if excluding subquery references is enabled. + val expectedWhenExcluded = query + + // The alias would have been removed if excluding subquery references is disabled. + // Filter exists [a#0 && (a#0 = b#2)] + // : +- LocalRelation , [b#2] + // +- LocalRelation , [a#0] + val expectedWhenNotExcluded = Filter( + Exists( + LocalRelation(b), + outerAttrs = Seq(a), + joinCond = Seq(EqualTo(a, b)) + ), + LocalRelation(a) + ) + + withSQLConf(SQLConf.EXCLUDE_SUBQUERY_EXP_REFS_FROM_REMOVE_REDUNDANT_ALIASES.key -> "true") { + val optimized = Optimize.execute(query) + comparePlans(optimized, expectedWhenExcluded) + } + + withSQLConf(SQLConf.EXCLUDE_SUBQUERY_EXP_REFS_FROM_REMOVE_REDUNDANT_ALIASES.key -> "false") { + val optimized = Optimize.execute(query) + comparePlans(optimized, expectedWhenNotExcluded) + } + } } From d422aed4a2e82d671a592b096919015bddeb751f Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Fri, 12 Jan 2024 13:19:12 +0900 Subject: [PATCH 173/521] [SPARK-46684][PYTHON][CONNECT][3.5] Fix CoGroup.applyInPandas/Arrow to pass arguments properly MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This is a backport of apache/spark#44695. Fix `CoGroup.applyInPandas/Arrow` to pass arguments properly. ### Why are the changes needed? In Spark Connect, `CoGroup.applyInPandas/Arrow` doesn't take arguments properly, so the arguments of the UDF can be broken: ```py >>> import pandas as pd >>> >>> df1 = spark.createDataFrame( ... [(1, 1.0, "a"), (2, 2.0, "b"), (1, 3.0, "c"), (2, 4.0, "d")], ("id", "v1", "v2") ... ) >>> df2 = spark.createDataFrame([(1, "x"), (2, "y"), (1, "z")], ("id", "v3")) >>> >>> def summarize(left, right): ... return pd.DataFrame( ... { ... "left_rows": [len(left)], ... "left_columns": [len(left.columns)], ... "right_rows": [len(right)], ... "right_columns": [len(right.columns)], ... } ... ) ... >>> df = ( ... df1.groupby("id") ... .cogroup(df2.groupby("id")) ... .applyInPandas( ... summarize, ... schema="left_rows long, left_columns long, right_rows long, right_columns long", ... ) ... ) >>> >>> df.show() +---------+------------+----------+-------------+ |left_rows|left_columns|right_rows|right_columns| +---------+------------+----------+-------------+ | 2| 1| 2| 1| | 2| 1| 1| 1| +---------+------------+----------+-------------+ ``` The result should be: ```py +---------+------------+----------+-------------+ |left_rows|left_columns|right_rows|right_columns| +---------+------------+----------+-------------+ |        2|           3|         2|            2| |        2|           3|         1|            2| +---------+------------+----------+-------------+ ``` ### Does this PR introduce _any_ user-facing change? This is a bug fix. ### How was this patch tested? Added the related tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44696 from ueshin/issues/SPARK-46684/3.5/cogroup. Authored-by: Takuya UESHIN Signed-off-by: Hyukjin Kwon --- .../connect/planner/SparkConnectPlanner.scala | 32 ++++++++--------- python/pyspark/sql/dataframe.py | 8 +++-- .../tests/pandas/test_pandas_cogrouped_map.py | 35 +++++++++++++++++++ 3 files changed, 57 insertions(+), 18 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 50a55f5e6411d..709e0811e5de2 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -674,8 +674,6 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { transformTypedCoGroupMap(rel, commonUdf) case proto.CommonInlineUserDefinedFunction.FunctionCase.PYTHON_UDF => - val pythonUdf = transformPythonUDF(commonUdf) - val inputCols = rel.getInputGroupingExpressionsList.asScala.toSeq.map(expr => Column(transformExpression(expr))) @@ -690,6 +688,10 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { .ofRows(session, transformRelation(rel.getOther)) .groupBy(otherCols: _*) + val pythonUdf = createUserDefinedPythonFunction(commonUdf) + .builder(input.df.logicalPlan.output ++ other.df.logicalPlan.output) + .asInstanceOf[PythonUDF] + input.flatMapCoGroupsInPandas(other, pythonUdf).logicalPlan case _ => @@ -1587,17 +1589,23 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { private def transformPythonFuncExpression( fun: proto.CommonInlineUserDefinedFunction): Expression = { + createUserDefinedPythonFunction(fun) + .builder(fun.getArgumentsList.asScala.map(transformExpression).toSeq) match { + case udaf: PythonUDAF => udaf.toAggregateExpression() + case other => other + } + } + + private def createUserDefinedPythonFunction( + fun: proto.CommonInlineUserDefinedFunction): UserDefinedPythonFunction = { val udf = fun.getPythonUdf + val function = transformPythonFunction(udf) UserDefinedPythonFunction( name = fun.getFunctionName, - func = transformPythonFunction(udf), + func = function, dataType = transformDataType(udf.getOutputType), pythonEvalType = udf.getEvalType, udfDeterministic = fun.getDeterministic) - .builder(fun.getArgumentsList.asScala.map(transformExpression).toSeq) match { - case udaf: PythonUDAF => udaf.toAggregateExpression() - case other => other - } } private def transformPythonFunction(fun: proto.PythonUDF): SimplePythonFunction = { @@ -2584,15 +2592,7 @@ class SparkConnectPlanner(val sessionHolder: SessionHolder) extends Logging { } private def handleRegisterPythonUDF(fun: proto.CommonInlineUserDefinedFunction): Unit = { - val udf = fun.getPythonUdf - val function = transformPythonFunction(udf) - val udpf = UserDefinedPythonFunction( - name = fun.getFunctionName, - func = function, - dataType = transformDataType(udf.getOutputType), - pythonEvalType = udf.getEvalType, - udfDeterministic = fun.getDeterministic) - + val udpf = createUserDefinedPythonFunction(fun) session.udf.registerPython(fun.getFunctionName, udpf) } diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 5707ae2a31fec..7c382ab1c5a54 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -942,7 +942,11 @@ def show(self, n: int = 20, truncate: Union[bool, int] = True, vertical: bool = age | 16 name | Bob """ + print(self._show_string(n, truncate, vertical)) + def _show_string( + self, n: int = 20, truncate: Union[bool, int] = True, vertical: bool = False + ) -> str: if not isinstance(n, int) or isinstance(n, bool): raise PySparkTypeError( error_class="NOT_INT", @@ -956,7 +960,7 @@ def show(self, n: int = 20, truncate: Union[bool, int] = True, vertical: bool = ) if isinstance(truncate, bool) and truncate: - print(self._jdf.showString(n, 20, vertical)) + return self._jdf.showString(n, 20, vertical) else: try: int_truncate = int(truncate) @@ -969,7 +973,7 @@ def show(self, n: int = 20, truncate: Union[bool, int] = True, vertical: bool = }, ) - print(self._jdf.showString(n, int_truncate, vertical)) + return self._jdf.showString(n, int_truncate, vertical) def __repr__(self) -> str: if not self._support_repr_html and self.sparkSession._jconf.isReplEagerEvalEnabled(): diff --git a/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py b/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py index b867156e71a5d..c3cd0f37b1038 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py @@ -445,6 +445,41 @@ def cogroup(left: pd.DataFrame, right: pd.DataFrame) -> pd.DataFrame: actual = df.orderBy("id", "day").take(days) self.assertEqual(actual, [Row(0, day, vals, vals) for day in range(days)]) + def test_with_local_data(self): + df1 = self.spark.createDataFrame( + [(1, 1.0, "a"), (2, 2.0, "b"), (1, 3.0, "c"), (2, 4.0, "d")], ("id", "v1", "v2") + ) + df2 = self.spark.createDataFrame([(1, "x"), (2, "y"), (1, "z")], ("id", "v3")) + + def summarize(left, right): + return pd.DataFrame( + { + "left_rows": [len(left)], + "left_columns": [len(left.columns)], + "right_rows": [len(right)], + "right_columns": [len(right.columns)], + } + ) + + df = ( + df1.groupby("id") + .cogroup(df2.groupby("id")) + .applyInPandas( + summarize, + schema="left_rows long, left_columns long, right_rows long, right_columns long", + ) + ) + + self.assertEqual( + df._show_string(), + "+---------+------------+----------+-------------+\n" + "|left_rows|left_columns|right_rows|right_columns|\n" + "+---------+------------+----------+-------------+\n" + "| 2| 3| 2| 2|\n" + "| 2| 3| 1| 2|\n" + "+---------+------------+----------+-------------+\n", + ) + @staticmethod def _test_with_key(left, right, isLeft): def right_assign_key(key, lft, rgt): From 2fe253e2f137dc901fd81a79c65019a8ea2312ed Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 12 Jan 2024 12:54:29 -0800 Subject: [PATCH 174/521] [SPARK-46704][CORE][UI] Fix `MasterPage` to sort `Running Drivers` table by `Duration` column correctly ### What changes were proposed in this pull request? This PR aims to fix `MasterPage` to sort `Running Drivers` table by `Duration` column correctly. ### Why are the changes needed? Since Apache Spark 3.0.0, `MasterPage` shows `Duration` column of `Running Drivers`. **BEFORE** **AFTER** ### Does this PR introduce _any_ user-facing change? Yes, this is a bug fix of UI. ### How was this patch tested? Manual. Run a Spark standalone cluster. ``` $ SPARK_MASTER_OPTS="-Dspark.master.rest.enabled=true -Dspark.deploy.maxDrivers=2" sbin/start-master.sh $ sbin/start-worker.sh spark://$(hostname):7077 ``` Submit multiple jobs via REST API. ``` $ curl -s -k -XPOST http://localhost:6066/v1/submissions/create \ --header "Content-Type:application/json;charset=UTF-8" \ --data '{ "appResource": "", "sparkProperties": { "spark.master": "spark://localhost:7077", "spark.app.name": "Test 1", "spark.submit.deployMode": "cluster", "spark.jars": "/Users/dongjoon/APACHE/spark-merge/examples/target/scala-2.13/jars/spark-examples_2.13-4.0.0-SNAPSHOT.jar" }, "clientSparkVersion": "", "mainClass": "org.apache.spark.examples.SparkPi", "environmentVariables": {}, "action": "CreateSubmissionRequest", "appArgs": [ "10000" ] }' ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44711 from dongjoon-hyun/SPARK-46704. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 25c680cfd4dc63aeb9d16a673ee431c57188b80d) Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/deploy/master/ui/MasterPage.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index e7e90aa0a37da..d8753a0ffcb59 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -354,7 +354,9 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {

    D+N=TFai~>O*VYuRhe8Yvzi~;P1HD zE3p-zzrWPwhZEAPfh6*-k7eIZD-Mp}-BPLz(#o}any#l4A}n2$+1Fc4AXzpKPK<+> znpY0*gFVV+6|HJbWql$y*(o0ZUflL9@%$kbHzqR^H?jcwLw?Y`ggpSrFWt{m*Sx6( zwkEt=`rHxT@bC~6w25 zy4A8Wy+K5C^J~vS=I%c)k?0iE0D2GNWo_qKHC%WG6L3+u@x0>gry4OAf!`$@DtsXBn$BL2e#Alt-eLu1 z*6c{06}Xl@Vz8(?nn!rD)w*i30BynvV*~ocvki?4!ngb$&)#0Ezcv9!c(S`TN`7F4 zi(D~K?rVK>Qbhg$?A}>$Q;x?f8ZY&0iD&;uv%wxay9#`o^BUX&Ye9yO_%5c($@Ue^ z4H#hB{IuO57;*mJnOi{LF$kxcLjH4}t;}`)K^Kp4t^h0Ol0+5n9>YlV$lN@a`r|S! z(JuZ0uXpv}+qmx?(>8#NT>G4+)?9p;Kv++JlJOCCgn&jDO+b%~y`I)Ki>0NOK~2FP z-!pe-r$&f>Z?XMOk++Q#c!b$Mbl?PaR`g?DPP#F{td(S*&=IBi4WA9ys zJyZ=ubmdQyknoemEnmwK9ghbTR_dzo93wvhgEf)|ampluQB2_w(++JHSO0i^M@Qf; zP-FwURRySmQ)rv zeq1NpudylUq(Lbv{tO>BE=@t61RA8b|3`BR+r}h!xD~&32a}ZA7&f2*xu&20;#tG* zqrFYa!)hkf#?5rC8-yX|&wo19>_B2jY%o5#RcLTv>J&UOkwJ(3x~fJV z21cGn&hvV?+%*Hh5(pZ?W=%jpo&)5BYySOve~*Log4&#J-@#9GseTuu;C@%(Z+J+5 zwxCHQ9i3?;HQB~gK1Nq2H1Eh5yOO4mw{|H6?8OGOJ79hbqi7nl0i}YVy>kWYdl)!I zF(cSFLS|7@NbNVnDL4~=sw=rJBdx6@*vn+Bg5ONL(OId|1LMW}>))IG#~(Ue%p_}nD7}bf}A)ePU>pqf+KBCQ;Q*5>HT|Rv6aYn`Y^FSpD zZTe%;hm)kpAl5GgU)TpU$WXUe<8{8Bq4>raM%dt|Ap{xX+XpZ+7}l@>zVCzZ)1@B< z%w`+FM^_d`k%ta(2-c172NhFkhKZRtNUO~&T-7I?Yp~nM#!sv@)1F!2Q7XGp*+1_w znP*HpnnvRBlf4&7L(+9;=2<`T0wBnBf*{l!l|OX*me)W+*AT22;8o65a!ZE2g9Kq- zF0;$dsMB7YVdOCCuw}-))-3 zFh8$N7MrilWRL-m^sYN5YC|*Co@Cp^aFAZ_$ec-CPc_*wPHxpQ|Aseyrn&ZW;qz}R zZ9`RJZi6M%%w$g4z8IGxAf3uj-x!nC65IEUjlJa7EKiO6&UiQNRb4|YpvpvCE2(NX z_KHcUR`9F5zBfsDW`DTP7ko1nSLr>tdSg1amFwK_IOo}DrS8((W)aYEE zjeIBAGjsLRa$xnN8}joOZ|tqUx$~bSoQ!8Ndym~!r&RUj2c(n;xbrZLR%f%Ckvhkc z7ZOqVDL`VG%7S%X+Uf!~ZZy0I6(_Cy)4)R?`rQHYQ#Hd}-d}jU#8a^Em6-J%Y&#cGALjI#^4`kF~m2o>s z;YS^>Zfz)x!SootY+QgEw+qAazu3q}06T;Gsq=_9We-<70_Gp`V^-WKrX9I{E$PZ4 zV?5#@*5g;0yX}KWuB3An%AkCEqj??M_}n8lZdpjo>yng?m7<6eVUyCSfj^i5S|?A_ zO5S41l*pjG%RQ8>#GOCzbj*9yWEH{W;7I(eVdAh%KT`(#`Z|@8n@3Vw#;O zc}B+DTOXyV(J3Y`-e8l<{!;KaKx7xHQ%*a`N#cs9|AwIwM$zZfm6PjHf-d14J*q4Y zl+U(M)9wXn6p&J*%#~}x-UP7{Qm|d(FQS+g+%iXUH5El!dJ80d(Td6>Qp)$PzN31u zLaAqTG3FtMgCzbCbLvlY;lS$_WyjbnRQ@;pvI7LXxq99jw_S7CR*#ptt#jBrSb@r3QpFDhdmoE`Sn5O8=1vqHpy;4@e%leE_^(YpYX!RxKhpO~K!@7{I z)JeznRLfeTXr+r*IjQ<|K(??7?>T(<$+u&mb_>}5=|oKNRN0`%PC{9ia#w&k-&PP` zmhih?oD1ML8=eg}Qvoj2B*A8tTj8E3%NqF(AV$xMyRQ?=Bi87LEj78fU&1;7PrmCg zm#6h}Rm{T_?!o;79;N-su=Tw)2t5k!lS+U){?8=0PgK5F#YopmnpjriIo5EFFw6r< z@<1Ui1GkYR7-TcE6aMS}TeP~l7%Tj7ienD=Ee!-@LGdWC z9GGeKRjJOcWDjCF-z1m_nj!VDoZr36K;?Xfc`sGwuW9w-Co>U3=q7s;Qzo6>?a3Hp#w$MRr;YoutCB!W&$Z-zx3}!+jU3XEz zs~`eP;TDHkwCHdA7|96x;$n$I$7uI#j(6R@3jp@N)a4P2AQZ$ROf;-?wTUc zdhJMw{s5p(&W0Td8PFfrbC%kXS_DUA*~(jV(HYH7u}~#;{l}Rf4$t*J*%3xfxL=^u zUF1QQeaEi?%;FIsMnr6zRc2#wrvVIi8P0?zpeJEifEUI(5ePUGqMsj6apDM|H zXK~_?D;NE#*)8YlaO3~R# zvz80p1C;Nt?JxNZ51ywS-)4vP*R0nSL7mM(L9INsPcG_sFb^UsjJ|Xih>PT@*>PwT zwZyeYDmJ{Q_(wz%p~A;Ec!)FayOm=sa)jTVH8nlu^VPX6OYU-x<-nm1&*%b1$^5+r zR()_>o0TjyOvm_w6>%~Z(LU;Ckm&#?#tPpu-IQ4-YiAEM%A?E zhpYDtXS)HzzO_2(QniY@ThyktSGlXTD{2$Osu2;J*zVS<-6FJRm57z3B{4#%T@)ok z1+ljfd+U>D9M5|^@0Wc4=g7FO>paijSq3Z+M{I`JPWWk6y?E5eos@=Dc|IWCFwT>- z_r|j*^kZ$NDO5~Pwd(x9G;M4Q)H|df_$901NM7ZpvY&U^ZCAx+EDGio^K3GGlk0Ki zdfvk5xDJ46`>IolUkk(c@t<9G@YJh9^G450@=zrwsDRPIV2{m>F+ICypd5pez0o3A zO&JzVYAle(tz#9SG(FmL&Yz4sLJ}XeedVVE9tIZqo1~uJn*!xnHH={!#)_%^!DgNq zSTm>_3K5gtXgu+gU*dhQk@h6;+mS9=WBx((&0>2P1Jvy3-9@X0vJSVYb`*qwx3c>z zVWNwJDlgA1nR-?Gtk;f~Lwjo~tf|4+i~wry#S2F|%h!JnsCfV39N_b3P)`2Bf&Qc5 zUx44O$+m46v%{rRre^T2d30QWJFcOMfG@#~S~TIucEDT^>zVI$zB2Ew=x}oKr+8nR zf%*NM0IX7xVrnmOR)aV@MVPhn8WDwoYR^H@bi{n>GaI}S?e>8sejza`}J^OAkO zwovR(A$9+`tK9E@d`KfrUP`l*2&}1X%I&Ka#Gx@o(CT-^wN1wqO5!>eQ?yoL>$D%> zXfO`D^{IvP1|h3S=jxYv*Wt9@F@OSY3QioeD&A{Cuad_p_@>I8a(RqP+Nc&`>=`wj z{S0O;yQ`I;nXmivoW^-%Xp=noA_gV%?cV zPNL5zw{oitBR2>p*1Oe#?yAI`lw`kFpsjmeW}k{&wSgCDQn4B{wAbw*c9{mg^Pxhq z+kdI(?Yp(nOACbYkDz@#(ijW943gUtl{!PCy zBPzZGEa};L=EqHm&jQJBlB0@zD=V1_Y%ctc`j|?S+cS0LLMX~9>V{S&;v*G26Cs#9 zlvEb#daH5t?2N!(Qju%bq<)SzRD87jL_Vzevz}i3BOYgg*F9RHbjoha9OhpwKU>kK z1q}iSjz%CpqQOc9H#!Jf5_+a*n12_pgu$`T367h&Y%Z;#OUwOZ-`0YOoQXU+R)E9n z{XwgP1saSucE^8OT55Cm-c&H~T1UVDh5?Z{z7rVXY?22biRCx5<}fg=szMn~RMOApg&bDAu0cb6}pITDLi}h>0Fb=`*R!26T%i z_vF~lSx^>;kb21~UBSM^+3j3;K|M6PQ9+(;p(6cw2PbDi(Dbgi&08ZxQ(Oac?Do9< zla%LIl}=KwjQnNj(w$qFliSUBIt)Qz1PXLLj=>W7%^!CHXlBc{bvv5s3D2H64nZ&h zP5Ap}e|%yX?lI6cZoa=G|HmW5#4x)#?I!j3`VSXrp^~vDT}|3TS8E+=FDYO&!gjxm z6SBk_-20wMZlbfP{(B?mo5`SCdEhZC_23Y->{29@Ep5=t3rAb(8u(M&63KSCAM+JV zcFwO*(SOzn_pTyaPE5WE>)zN!X$?42Vj_Ky6t3UxLrf~sqvSW*Up1RW_kXL)e%<1l zfp}kZxmTNuv&>OM(@L0>l2ZAoX{Z1jm=HFoCw=5$ZkGCkUP1FuW$AGCV#R;C^}-hL zjk8=_jB;{Kt}6%NxI=?l@Ht=b?7sk%+N!=%dMDZck&C*s0iX8P%`ZGNg@l@>p+aGw z=wRrLI{sf^zRi<+L6C8+kkHQFBA_0~*;BgziLXzm zGrY{vG1pn{T|3ElrSE=SaKEHvpKzJh@qx!m=`uKMAw$6!%t&QZPO52q-vgr3z$v%^0xGo(f8+s~F zR+sy)Pr&Q(flq5tu@wwj6G;@3IK}g}X z3dI+%ZX-?;_A}6z6WDfj9H?Exq8q9gzpaVlIBkTd&zm1s}7)>6i83O~O|I(sIT%C5Bh^B7ETKG7uA;h+pFpxMvGgW0Gs&q+-?JdmH}R5d)JhAF>+%I^_sYcE?21}?lsFc9_p+{*O`(A7&J$4P!~mzC$xE?zUi3t%>U=}mUvZlXEb%->SBcvNy0*VX$6KlGdmdi*>7!UL8c1e?dEGn zYx?I|UWjj3>NR;(2AMH3>~?)ITcvKlT_fk_I&g@n>7XbBqfp97HR0UolZjt;w3CP) zP@upLKVn^PyFc`c>fa}-!bX|b_5E=rJ@=pUdHXXG@AZY8rVAGvZL}Vn<>+md6q8dU zm0*$Yn$w=0KEA~n@QW(i2Q;bqXxGc!jcp*<)P~{tf%SmjUoQSXO6skM{SMUU4U~mH zG+mW37q-I>ieub-Oo`$bdej7w?#+AP0bu7q5`ZG=u*Mm_&`Gb^gmH6B;DE`^1UfQQ ztd{~bE~Qic`lg!OB%sxvl!Jb|`)Zfnn%V4AK|_*7w9$56dA3yDEdd|)D5H1MAPLs? zZzUrH;^#lb9RK0=GX?}Qhd9YP1v~j7Gd#@+Zk-tAmc18;n|C z+8TuURHetC2*pxF7Q+~ggU!{VcbaZ@jKE(-=!zgrt^&=Ic2rX;(ck`baOj&@~&`o|TqZz|~UrEVc(UBwz!PeCVn21|vQQ6ZTd>`PP@b(VWxvtF<`wwsPP`z5{fBoJS9PcX2 zi9Si?Cino_P?}+8B`ib1r|f~-xS2mXlE*by{j>MJdXb;7BOj@NJTy&Gh~o4^M607lr1$ zsR2{B<1sO%^n8ULToe2Cz}AFrnIY*~4==UPJo(dOT$oVAG>m9;!FVY)CrDF;agu#} zN~Z>gVy+mlLZFnR+!=d-gkaK`a&l7qF9mrSboY&#rNsm{pxCER7Zvs}zk?!^1T_ds zre%Ao##pnUl{j=k2h-r7CSV_1Ur6%B`E73qPJ4m1ylxuIJ;66SlC~@ZYv3Zqd75nD z7Mm)aFt^jv!zr3Fr$kED?9avFFm=M*?=Y`>{3jM|J3m)HT0&OuTzNICUZeAFXysoH zpB8U+i?b==em9qlp@d{{13E--@xb+gH9kXNKI6-QPlJz_0q;PI3{L|`x<$aF1yMlQ z_p+Y3^1%7Z%Co$e@EV&wdGvVv5q#?^-NSm$wc5%2Cvy(a-~v@QshuE4lwDK0q9E9q z^Jz|<@b>c?Kv7?GJom_g4?#}>c(S{(ep=8_oVS}B49d)SgPvQN-oifJ9Tq*oJvy6; zF{QKZQ!;(v=iJ+6ww*@l2lWuhpN)Tat{c}KrMzhGIC?%zSiw_4czsYEw7Yar4NrqfN3 z#_VCA5=>$rrwo>o3Z)90T?fcnvs3HMnCoF8;a+|-P_22x%0p)JY-gH6yt7}Eb)NDo z1BG{|+KOBpBR)AQ@z*$fYzmp0KX2Jxa;Uzp{OSErub&85U#94`E8K-O^2Sqdn*qrV zgOxr>57qHTft-2umoq6csHk@-4+5&`WikQ2$3si~(yRKjWiMu|@|m(506CWBKFwI9 z;HLiZ!?PT^X|Fkk51S9KZL^?$37YM^oZhZ}3wPvDyY)EZ=owt5%TM;(V5Rpx}WBQ=l~-gIX$*VXJc|3ril^jD}5lD=*)7XK-lU=L?3+d&y6|e z8z4p&06}CWs5wMhpAtPw6$U8K&lg&{%bQNC?%Res|8KNOn4x(82=nWT?xPpny<(L< zot2vcNq1#ma(|xu6Z6vSQuTMHFK<334P_KvtJg(vmduaq3gDo0ja)EryteFhbh+M> zXd9%GaOYEiaYiwfvr|M#&j6d@n41%*JITKzwO>T{tYn-@9m-rm7KW*18E*Y$NTR;n zKTxO$UN}@}8-1EwIeMIHv&Lb*&aE5)Z+f=2U^4|=NH~}31W7t8@hboE$Q%0g{r6Jz zmUNC)1UuM@qHklXqqj0tJgMf~ZCt4~y5IFSQlm+EOR|Ppe)c%1pJk?eBiWtPB5r9` zbvBv|c6eff^*v_$O8Wlku+=7QKjR6hGWRb7kJP%ln-uw={Y6(ZYol(Z8;Bi$Yb1N5 zf+5Pdz%VD69dwyi`Bp;K;OXG$_cDsLhCnI=zl!dYjuc%wHOuo+lP=+BNx&%e(2hw# z-M(#~DJiTsgF2JNl!WWF;oBYE=7fdwl$ob$EaAM{UVA2Qw_nxD4G1fiAGz3Bky?y9 zh*ie}_m+Mywz~w8gm+ZgA7Cehg39cjIEe+51@uGT`r!Wi$Qian?U_=46wj*yT3XP6 z(N0N<^?w)j1IkyoZ{r~3GNwSwaV5)d_m~v}%R?Hue^}MtbqvpEye+4xe^&9dy)5)T zB!^;L7Pv|x414cpr-4O;1^tffI-2jFCEHOq?^Kt6j(syDmqr`8NkkgoDe^<{;K-^hdc}EftjWmnqa?o7MK+eSe!QorwaWSKmPK zU^fnL@?-Ekqu1Ujv$wovOOS}8s9rjT$up>NfMb0|;T}ft*V!+*OO%W<>=F`7$KFjp zP}&kp2-Eu|GE?sNyB7|!sgCm{8CwGBpCJU>$uCcl@odZN(|+X>;h`vZIk~?K??am7 zK?X7{js^<)j*dC))%@DQzDd_2*F(W3XS}yHo9pdlKJ3%bVLe+iO$wbZVSgTQ8MjWK zGktDcsWY!&mJ?psbxAAYKz%pMlJAELQ)HB-(JZ~&nm4@e>Tynl*z8v6nV1yn>035- zV!6$V^Dxn0jyzH36$fSPrw?glP7CVYq7I%R*#i#N%*_2`ZKYt4m7kbubQ)3Tm*IgW zRIY&De6cpp7Qa-eQ8@{OyMuxnZIGLKVS6?u^^+L$qK45-X~^W_>@ArxBfRW{J0RCg z)D&FW#vBvPAr9`$Qv-rEC{$&igp_!~AXVO69v{XG1v9nX-8breu)eM<-c>L3$KGPi zQk9+Kg699kz-VQYN>Q$o)wwv|^tH!RJwH#!S>iE?6{3F<(#njj>z`%0RS2Z5XH5lK zql^BJD~54ylPt48eaLzuKW<}nSkNvutIqxYb7cfykg<-O6pNuGcklVoxX>Rgv$?bA zC=l0vm22A}Tk{VkpH;Md)`lC9thB}jkoGI1D~fswWhQ2XqRXN6vV;xM!&TGz4E0dO zFqK!<$UNsqpbXTdMMSJmG$Vv z@@*aStVHnLDPvMn*?Cp#m!#M}IOX5%u(B(u1z_a>KwJ1JFO^lJ$2GIzP? zq(~^)ZX9qL*W%h}7u_m?H0JF2(BQvP%fri#C1^h;V6#FhvT(w3{lU#`>SN8eBgT}q z*Ky}8REd4Oz8`~f($=0B*irt7sqMqgwci0O)py1AZ&bw5luY^8S2}Nzdw?T(d(iHw z^c-8h#_&rShxkR@#|#mcrdoepw6KX`UnkQ6~JQc40tT+rSM^wX_&VBJMj*` zcYhy5KQb(vxSYN3GOv`j@O{3*@1!(PYWOXveYG{iPNI-4aF2u@ILgKy5K*c9KAe>x zC$k=$u7O|ZH-i(!wF8M-aQ9|8=m0n7_VHgvn~!42bul{EKJ~+@{pW@mmyyh%@K>)8 zh-a+Lje8LQvC)?pPpa)URiBv?qS7F+j@dX~rZXp-8}ULAHaRagFy8n`*nHLdx<16= z6RC% zEH`>7z?rD8DQP-o1+E~v9;gUDE|uExj!Ms`|4klaOR!{k-UE-9r}~dWUSg@P zP<86!NH#SPavgXa`kh{sY5|RFChkTk?05 z1Vk3TeOe7*k?N zZ*Vzjr)cw<0ZNFruF^BYLV4!F{5~#pI3+>K0)^ENM`xa|86AKqbflYp=iSll)|VUF zuZf*z8b#d!zW?o}+}lPAM&J3@Sll1M^HG>F%vOu+=R3Ah1Srr9Jen=us9mbu19zD} z*a-oR+XWAgx^+#`&$i*&;YXDj9-WEz-U({wPd~g195dv|xnY0D{4awkQw`q#Qedi#_yjNAVbEImGx+U_IZIJg z#>z|&SFKju6isT9e{M_N4Kt~sBha)5hwW^%Kh5CnAK{`@q1c2w2|j1^8A0Hr%3Vgr zfxyJcA*O_56I0-*M2u|v8g#o+sXR62@4mk)ohPADP-Q{%h`TI_G#)35;I$!*Du@&v zs&_`RNU06_OWqyg%eRgnlR%*!iMKa2#z1A~nVv(Js3}(#V__1_dWX4^S46kWq5Z{JVqi{2LAcQdnIo!^Z@SLu zP3%qgvxIVrt)>h(b}lzOq5bl(+)CSRc(J8NVp#W`Ws1Qoa~aP0`h5009P*Z(*Uz*O z9HiXRGBQBKbQDfr$zwsr6}lYxm{|`QCt3ywVd@R%+Nu8JB;}IXBf)WPXEY@1qj=|@ zByV7g*F7T}NHX&cfxvyAB>;_zo<#cHgWpLyZ42=AoADk-xZcu)iRc+4V~r>4O<|~{ zGCYnNK7?0hQw2P(42EkPo!*5i>I_^GGXVp?-Y0S1r)>qdBVE}y1$zMmH;`|2l_k$56Vh&(B3-;c z=`NzQ%&&-OeKpn_s?bOV08K~o_f?+c=!pmhYp^crzeM;Adtg8DaznLGc{wc$!RhO>yskszaI0^Ail#c*RzxfZ1 z7$JVm-E!+MzMq~t^t$D_hD3Qp=Pd~(6S(*#!79^l2S__!;Obz26K8)II#0A*Psr?M z(#Hwtc#zzcXyv$AO5`2fcdjY6q(xq-R=l5m41R>t%EnypzcQ({`MBiK-I82eq35RV z1_19L<#|q`G_$8lCH%Hv`^-mPIh1x{@&cc4=3RHlXNzQ4z*efD(jp+8Y}96$el>l@ z8LT++OEB);d{ygknw}D{;JqxAX()KZvvO!Go#QL_NyfMOmVUWNR)utQP)6+4`!4UY z1K)E$!B&0ILMN%&C3TYCC5ef2oG%O)-L5qG%kb@0!&HkiS^XfF&hQ))C@#_i7%6R8 zl`u*njOvGcz;s9LqpCt9Bxbalo^Cts6T*aeo;uo^nj^#zSjfURn$_>2Wmy3vOjkY zG>ajUiov<}Pn*@|Jmypq+eBVS_L^l#uiusvm5@-6GyXxN`jOe?BAPQahMCFuFu`i93*|DvTZF*{4e$9l5fYotwtHGw2ns^ zX(DVOC6-t&TTV9Tf3`VU)U{t#k}yPV^UG2Bwie^mphClj5X^mSyVLjBRo zaNhs%An59`M)~9lWV;)s`S_%T$`Vfvvzo*r`tNfjiem~itFdA*p2XlUphvn6{wc^1 z>+5fIlK%T(pMD^uX#DwNbME2Jj~>pl;l4vrqu#CVp5pCV$ycfrs(pFS+O7G`^8usY z+hSzZ9KL#Qj-)Zn?b&bNe|=YsUN(*Xc7(of94mEO5!b8SUm#wtI0md#4?+cN!NQvP zg7iC@IU=1Kj_CJmqVd*1n^X6mEf$~!Lv&hV(mfd<6w|VEB zAy0D)Ei(R8`^VwHx}-qsXeN8Ip|b^Z0wC!dRd2LVX>ukn-KS*YTR?c%RltVpu2$rS zpy0WZyPOk&{8w8~y?_NL>fFx54^@$><+E_bEpx`(J-G`)4?O0sUz_`~vE~2W*A9Qi zacHp>_rs1WOjT8L+SXZWHol{&0x{7!0$h9>!ScXB#b(HL9EP0i2deT1^iOgt~$GTkfkv0MkXns;>_$Yioe_5Y`dFxQ3pL090t*_&hbw6XO-uzB>2LH+)wcyVGlJqYB zZ)NW?gZ)|r_t4&-E!Z{d7~_Y8=u7zM{dPJH>3hn*%i0&7w@JOt_Z~4T7+UBYoNs?F z_kvgJcgO3<$+GfXWj(k~e{TAiY;j1vi(hgNvM;WzPq$&X&9t#}800gW8IHVVg{<(^ zZ9m+Z!>ki$)Z;^pSDF+D&;MjbK-?b43t#OlJ$ji z>D?Pwt<&2Hb&Rb4MU1j>4Yv`&4m(Nrf<=wf!OtC?9)L|&k$_^?=+b?_(yn$)zMRk1 zON+O;TgMVwd+$tFNAUcTdf&a1tIE3&*XbZY5;wgchy)dU?{XMlDl)&Dr^eX39OAJ|^LA-U*e|);e z8iRS8>y)bZbkV5j^Y2v22gQhDC!k1=rCzFQ-#~1R&qBK2>(m2;z74?oM*aiK8{{~J5#KNk_lgY3C6fzEk&Q$HyD{5Y3qx?vs`j!ivkh+e zt47kL7}Jqe8 z?cOH|ML-BDo~ZpC->guGd|-MTOB;-b@4%;1`j+Cf^sW1BVjlT0wubM&4rLx1+n-*)+jP3fG+o=t7+A6I@ZO2d$WWX|3=p_b%N zAYoh~4-6m`byI<4gIc8Zp2Z}}1b2k_>Csvk_wSEgKL4bArfGRf9K%I!^? zG*T%3*+|Z&q$xRK6G5zBFoix_Kkl~vS`FA=gjErC&zF9!fqxmChnG!_Sk6t(1BYQk5saz@OR;+=!jbKk<{TsV1!BtZ|cLKV0vnNGWT8AS>Lti|1yY z(@)A(#!+uUJ<{ia|oOiv7Wxcp#P5 zbDl^IB!{1J)RD&@b51hP@#hDo|2L;}Qm0ne7Y?}1rhk_(GN%$cJikGO{$*%HpW8wM zI8U3Tu39sVgZpzpR7>j4&{m|-Bxrpyb@TSDsuPCq#gDlOg;&nLV#BeN?1%_mIw)Pt zw)7|!I(2lsazGv-tW!6}{dMnEzDFmMte#)xdmTKWG{?DS%k(ru$(c>yz*zSv1WeRF zi0mDp3IX(9XM7#CT-k0ysWn)SECdkA>iL7Gjq{)EB)D3rx$x8PM>mN)GBF4Zo5V<0 z`GqT+2beS>L~kw24(+${J|V}QG9n_HH~h>#$mtH&ew6^6&$Ii>cqzEn<)gTEFr&l5I%+qX?yvXfq6G; z$KM;&rpT#seXnyU#9m%U^?3Stj?>FA^Ksrw%tq~$84*P+^ZlZWS3>e8kA4WI z!2(0%(BBi&_c9R8nyzdM-^k0vnjY2uWAbVF$bvOw)OyV!>C0Sx948-4DCOyof6_ZU z2Gk$g^OmU}-WmxUxAml!;4!jki?P`42IAM|uSSMBnzSkF$9oUoJUe!(y_%x9Mod8# zgxwE`w6&NzvL;RXc8S(frtj@$~ zP%8;*i$S{aU|WwImiZ;Es<|~2rBX}kkUWD`O_Ha#z3i1mZx&wh;pA(*qARfHlv{ll zAe^v}7_y@PldKB2w#e+&L}lr(_2}7*7|6XD(p_PTe7P|WC5(?66mbdtbBl1O4$3Fg z^#`eXi@h9ulNBz7Iu1WMkB!i`&CEI5%(5$(+W%zDk9p<#&M$Xs{L9|_-b5owvKB~S zC)7}vrkb(@8|0v>`*ty;O_G?{7=^O?YR0GQpEnV_12*}MZ$cUKa1d1b0e07LXVBIi zPZ@FRuAw6SGEBQ4laJTQ>nKMytH&QSnPgK%$akD!EED-(Zoz6yFLd``2d2*_`lcm0 z!9<<~TfBsi%HS%q>BhVM_D<>C42P>C24k9pDcF`A)H;1oBEWqKQnK>hvSj^gowsWD zGxa})ntcCSCgpmIwYSeYgUm@Lb~i$gdPAG@^I^|UCGRS_O`ebh2rH4)@9PaKfqSsu zM6^@o|6ld`!gxyO1@4?KGv*EZk0jwpX-n#|BdgNfCj*>o4LqNI*csTwTMyX^6KH-ZTf${T8Dv0FnplFQy|@X61P0LRipyM%9w=m^ z&GH;z^X-EMp~_o;!!X;cHw>`3OxwZ=Y)kO8LNJ9Pfk#u8H|dgM_Kj0tM^-ToYV zmXVQTx~hVu?RoXli#P%!EscL1hR=;Wk8huUz0cLe9#_*BbNTL^$IwgQre?h21K(#>cQ|3hAxq0UI(K>|=f8ifiGRCvV4ax! zBdJt!is0?_UdfQn*mMP;Xw!cu0hMKzma2+v{5G+j&@eK*ftQO6(cb2J5ad+g4R~aX z(F&?88%{%fGL9k1?iqDFw;SuW1r_PtnsNbj)reKNg6DZyCJhTOT~uco@EBe8$#spe zsIE5D)!YYchf4f{{( zKhLtGUH$`W82lth8i<>tmbzI)9UR;bQ`*sJr-z@JF4=#*#&Ikbr&w^g{T_cW=uSt4 zc3#JGQ`jNIm^e0KphEdwRuKi?>JAH65AEquzhm7UX^~ybsZ5`lvZMlNnNGFf8sKA^h%Zg zjLy@H(sQcz*pEq!Nmd5mxf&T+h$q1)&zQA$nrq zV2Rk0>WGbh872KVxOkszJ#BgOvxOn|%{ytal9Y$01f`QItaYMS(P0_k9ttw)-oz0a zN}l~@TueYca}eykUoQWbp;i>>kAUWw!_nxx zN{eWuiGIKKeE# zW4ryTs-$QJJ)%@=K#>TgWsltTa>06qC(4f;Z^mx=|7Bna(tbVcP&XuLexTDhB()*C zc{eD+zwJ6!0z`RPpYo(PUfem&I{58VogAyEuzsIbtdF9FRvxE|QW$!~7Uaq1UE91V z{mTQRd>h6i2XepFt^J2$^OwOOeamLqan6M0eZJXeT(^aE5#Z3qOUyh4`ZjT)>~Zs= zWSn4V=up6;qTb{26Y0MUEyhyfkBFxtCtAVZAZFd59;$`^;7pQkcdXJi?Q&Q?`IE$}=brEU)|%YJP05%>r(xpE z+`2=1^d$HX#B-8qdrR8ab2{^xS?tcnS?}i)CA$JMIk-03vaQcRwN>pMy_v5)6|$&& zF{*Ymk-TookqN527_gckZNGwP0({(=7=ESDA%L~~+tUf_hA;M9u+9_bq|>TL9}`X~ z-f$(weo1NTGBLUBhSM{Tjqr6>F0`Ajo|H>7!fP$`v44Qz|J`-zoA1@$zYKipkbx+^ zYEp9RVQssSJ4o0Ps4sQdP5MPRW};?P*mGD%!XM(|+|odLT$^-b-OBE|9b5ZE=m+n1 zp&NjQj^9^6N`dUJ^8)v3YjPq6n7KR9Awwy(dUrhs&$?Pui$>$sq#R3|9(1oxZ!4HT zY2T8E0MOhbUvg|HE!!tyJy^%x!v7HF7DJv-8y603w;=nZv*Jh(sG%{XcEIi$K8(Vh-dYt>kma#aDDvU#|0PXK=f3SyUS( zlS!e8`MR+wZBTc4tFE(1d1J)4q~H;OkZdhtc&11D%dk8!-8I;I`Q5Cia+gV89fBC+ zAt@x+I@Ga9a+jkYx;UqpI|KNmO6C#wtxDw735!7zt3n;gr${8<07DEW_BgWy9K94mZS%W=iJt z?c7jkLYOG_yRH{S7aG$W-zAw$9fTXqE39eF#2cr3!ox zAVO|)l*8`>%xW47D-EYpl(tSTz{Rz!etblJa>6JFC?>M#_Fj(5Ecfkwf-}yPtV(b6 z3Cbw*3#B2=SA9<;cWh{>ZOQV)HvHoPC{NAE8hh4E9y{mE_AKMIp|7)IoYxr$9lh5D zltinW*-kPQZ`Q%A{yhae=d}c;62<7gtOkLAeI|@25_x)2rO=?)YRX zu;jV~_3R|Oqf1m&eysv9*!*qN36!dV?Xd75;EuTg@m>8=+Ml4*E`{F{LGQFeeIE5X zd>PBU4=p@t*?`w@j}1rmlKwD7M=UD(Wz(mcyE|y{4wK^@-=cnGOz)FbeP%7)9TD^K z@nIkc7utFgL=W(4kG658(`xP`c)n&QgEd}vEPhH_wo)fEbmC+4L!| zqhP_RMFTOny!zCE9aZ`-VUlkQ7a&h5%$I;PX^wcgiwB&IH)z}0fx>A28?Sa@ySZy> zTxObMeF>}@p}ducnu;Hh9*MX@8|m}v$Vv-qVcR{ix)EX{7}pn2F=6%NiuXcm(XpaO zHEaArdg^zCM7i0b&BzKMo1m9^%Hp#ryx~LY9E{2kq2MgB2qq}-W(5XfZjLapJ}NHW zkAAbOe!EFMvgOtJvr`WR5ci>zmrzT=C6LUI0fbHB<^%WrUse}w)&|=v5b>s``}j-x zPx!>#jf|X|UTV6X0)<1%~eF3nFwGw@l z@$sz21O4-)!pn!J|Fw5ApaA&gVYZ$}vQ=1u1IcUh}8-~uv zNU(cPvl=p@%tz5D=|~jz!u|6R+_5(Wrn|k^gb3wP#wl*2(0l1SE3N_i#^%eLn-ONo zzjkg61!9Z=%G)z5CmCr(3+`Ga|HcDbwxQH!%Vp6VgI@cnbA+!=YR2}{pU>#ku-yM$ zsatjAZkE_+8jsl6L}9CNqXd6cZ>Wr@B?$MRR#y`uc`PBBKRWg-0N2ZL{&2|ObUR}s zi{UKuPi5LcOQ!v$*B`N07~*9});CD^Xr%9SUa7Ns3TRznq<1!d(|UYpd8p)Z@};7$ zSvp$%wwS(h3u<#HsEe^QD941Po6NT^-*buxpzeEmn}n z`{vc76+`pvRFKqw0uD#8t1>Wn%(;hez0(zsjP?{_NDC<$JTK2YX&Uvi zrB+RqpnxaTV$A0;L_|WbMK~^GtYoS{8zOMwP2>mv0pV-Rzdu;@>v;HHcXia_xWQSv zSXnuYuBEdwx_u$&Oro|c$D4WK26qH(eA9p+P9{@Zucq zxA{h4*odIHx&r{+br}7rCcMDrLTF(1f*>H5Lk3b&y%N5jnN2xAkoH<%tuHkUQFCm( zrQLgw%6sefPiro6oznS=@d_5ag2scxYuCp;rzlfP!OkKL)as$}hRKH7uhkaxO?1H^ z6#K>{!z=Uor59g)xf!q0o~iB6_O_z6#B{pgC*KX4=mj+Nr?fUV47Wx9%0);#NH~}> zPbZJ)qqB5Tz{To?mSNnQ7{j2J>9@|iGK4nvX&v1k5Aj^q|3$*M6xpy2B0%E?FIx|E z+Gv&CK6Ml7?YuN< z6g1C*zhdJ_k)6d=H67rx4)J5-`09}2y~QC}>vLO=`AMR_HmuQUk}^U#O>VvwCO^El zbMr4ld~w}3Vy>7b;sv~kpxI)l^jihk-V5Iz1Q{jCke^D};I_iziC>EP8cWDn?!Yy- z+K5v+<2t2IkHy-r7Xi;a7A~$Y!Ph>pMCBwHzAh(jHij6oPIDL%02&8yecQ+hT7O)q@%n&!4hhhl_=D{@SITo+Ba5-#r>2w@~xK& zj5Fv*s~5;cH;?U$Z((KFgSXGc2MZdq2r6~Jfys;`a&abAsTd)P@5~bC2Ks?IYB_!B zb7Q3gue(~G2^OwG2rgGL0QK#0{7QQv{xntKmM66meW8GN3b=Ktz4AiBJNty^kCq%C zbD9Zq3W|h|X4Q*y_^v>o9J-11M#gt#={I>*WgwMg38c^-QA5n>8Aj6V!D-E+o_T`J zK=$30g-4~e^3gM_z1h41PnIQ6{GCM@8wHhOzkSEH;D>6*Sh_F!XdMIA9Kr1^VvUtFH0;J<@|7*04cUKowVhO(>0= zfV;I%Wsk~L_Nsli2kxBe5-aaoL+)f%d5O;edf7fm9ru1l$jA-MqK<`!ae6|t)3B4L3*Nv{A{s4X&B&m6W?KA(xw46WSJ zxtbL5aFrPXzDyzmT821heHfnqgxgErxBUG;O76PxklHKXR{4b0GI9A^1VTKx!(7y$&R7^179d=KevG&)@dTKl|G30AVp0+{&`%>sChRjfmU z_=&P#dumv?ue@W7+WyPnbQt)Lr&O^}M$>TbyEY#|U5U1nN|iDN5%bFQ0GA4pXE3uZ zugrVXnJIY^jg_BkHz8VgBJm}YD<(vJR!K8qr;LVf4ge{-o_Tc*^WqP@`gkp^T1pr_LfwM6K zBwE>4_*#{ODyn`h%5{=cBxN^(pNsxEF7MeGHKqAaSxD-1D3n8mk2}-9v_Kih$E;f} zKP?a6Ts9#M{X_8?{@BCWLsv`iN0+vYXSB8CG&y~wd#e7)CE$9g6IU=W-Gxmb?U{q zmU4sB_dRG#%X5U6eGY0UpQW#^ZM-llG5M<8Re*03#{sLUpx|EyjrFS36Bj4pnt~C# zYJ^uv(B72=A-mNPpkio6PDyRuTONZajn&~m&i(Ukl{}DamIf{3B?HEJ=FkgQReW9U zQil9kVNAOd^cv9^VUh*c%X6C%6kAbr;CrkfpBB=IR`Y)XK2;1Z$(6+ncvaBUt< zIpMxm0+*8mJ4)EJQ-l6YZgR!T&L7+7j+9KlKPiyqA8>neI5!Jcj%=ecsUK?pTGTz& zjO6Y7JiY8Jl!H3BL9R`b`u*Q|IFXX&&DqE&PiYJfOO0(N%j81nEN{3fFP8}>yGI0U zpuu8kQyyiQH8x}l<#6T#5*K>C!?ZgFR*xPq$!>QU^`Pak?haM($E4Nz;fyV-PC>ao zDn$Rzf?$NZE;d|j+HP#LmGYoI%4RFzSJShtmzr-94y@uibcn^A|A6mb>&vUt8px9P zSziab_R*5jE_*5eAD7QbLQ%aBd9ozaeBj~&;~kEVHk!#knaR4gVMfI(Et8YFd}!~@ zSSvLX<9rpl-0Jla2|c;34|JCV-M+Z~i5vfVAw(;{);-aGa96tYHQT>TU+C&HgEE66 z!-XFV_P4G)wYxYDt=XVZH>O}s7&Lyb{@7SoAgu~0P@`Wp?qyg9{P+KpuCKDrNii%x zkjFk*&76@O)TtSEFyk<%j;Fts4eoUO@Ps}Kz@$31;x4{Xb&d9~7FR|lMGNxCJ8Ot4 zmfu9(%TFNi)|ZkNV=z54LVbLQnSQapIy!P=8ls?K3~0Jlp{A|K4EyXBcOPj=6gmXUsjI+p1m z(D&yAjVg=nMd!9t@QftbkUP4<9BqT_jqP%%Uy3U;=s9o2IuCFvplCs8cVohGw+cPO zYfp1cm{96Gx_79ZBt4;i=M9^MTv+1M(z3a2{7{mEcw5}gHB6|9lN~|F*tzgovnmr6cs$s`pjcGAhj83Gwa!i4+uVmKmD zmQ;|}#ib{qC()@(kxrz~)8Go((a^K5Plp9F3;9LRz)bo5o$<*Iaf@uZb)P;5*o_~R zzbpO3G;7Qp>ESU5j8CNX6n9i`$NRGAY6-uf=}Nty9@^S`%Azc^J{)9%M<~$`h%h6) zC<3A5*8r|tv6(drrD2&(DjR7Vj(J6IzR6v2{2yGs^M}8UCxH zgeW2;&%N_yn)EnHC*OdjuZTEs`g&xLGq>^TKQ6m>Aou2Xn4GbNFTqA&(}9g(XnC{l zOn@_#ui2JLgH0HNrh!MYm*W4@GzIXkUCHpG!W2^j3XFWZU&^f6Tk|aKdPx3mG@B5S2_xjr z<&?ctB)%0FxSLo^_Rbd@%=I$2c8SSo)hT@zU>)SA*lg(-Ul=r&Yk$X{Sz*M0GD9EG zf-{biR~H_BNC%92=U~l;YCC-BzIC{`2`{Xb1g=Em_Y6}1 zhCd=$)1!U>eZ0}x%zEw4e*AmJ{)H{*>>*1Z&E`n&R)n_S`XR)u{StVlgx@2nQ1K6F z_pK_5MU6nP&rv;)WLB}u_WiwWts=iZC!K-Pj&x9k;U|^aL4PVB!yhNFSxk1JUbM53 z4NH1jvz6PvC5Ib!xB)T(gU0G-o$ImR|D+*B*!EWY^zV;er8%!e8t%60Q*Adl_VZwK zWr9VZhP;l_(Ej=c;$Xc)9unH>1JYY3G~5+T#pdS*L4GNYawJ|&3A3AJuEsm#RZA|m z;z-(ZXR26#id7A7w1@Su8m@zsz8%VytxSBWWXN!Zk4tKXw+1ltV%n~TwYrIqlnqBi zkbh~qiuFwqqy0!G8$9Ua>ac>`cM`Wn!dgY|;Bwzj+}6)%}^$h8OB@jG*@6~C-gCCv<5vqgrc+$ubL>{DJhBbmP)fb z5ZJ!2ZyE99fThQcEvTY%*Q?&lAKWpq5!&wG(CqlEh81Lxf<#4bQ%X}?!9)c9D>XX) zp`y+$ZqttAisY7(_}uscX8@;9?u{Bp`Xrb1bRHrU@Rw$uy6ZK|Z;>lLYI|#agObm! zM}URrEPCvPZsxiTif7&aKAq6~E(8`xx0ZO5jGN)~KO5h~ctP|^Tf%<=vG6@)jl}ID zuNLV11Qy#!0VYlcx5{iGJc2e~{hOPwPse)d*ip0g`}VpG?htpnIRZY!4UjTsY;3v! zu)MfIWSaD0*WH;+HLA(Qg{6$sPrsOO-tK?Uo@lPJ}YB{+Q7!jszd7 zj=pH3B~yNFP%4W>h_a58>vCiSv{iYpF=q^639DJ(nh8m>N@*gA>%TC#A!Tbg}ItS9KY_vW7s`TeuWH6!M8s;1J#|@C;yCSKc2_ovd&= zYu&CnzuUe+_Qmk`UdULX&SqA^1J}2GUk=dG#vfb>7)1#z)0P54CABJX)D!;1>D0Z7 z+A#ezS%M_gKvPMRo0P^&3%q2L)emt?3U6Yt?dLw05$zQ|hU=Cd)dkd83z1TX)eT0~ zN`%Uugj6*W%I1RQUjGohe43=+Hx5}myK?^89&B~cF&8!ox)>%NHs?%*v58Fr$msk=msj7hzT>qohokc@&-`tU9*zY$Hme*kEzEm8s1XsV^a@)K?%39K3%?#L zvK1U)Dg0k9Y}+)WM?5xMGzb=UcAi{j+4t^$%Q^n%;&_7rR&oNXcHQOtwS+D4Nu1r0 z>)?=75UBRn3s6iqfH!}XY>a+dx}vuRK(V>XN>`-$b5OM2g+3@Xe5a5~bGVE4azd{KiV#fcsljsl4>i3_IbD|47RF~bQNp+sn1JD2 z#drqJ$-CVH98(Ts{sLoidC@!>Z`($Q*Wb9L+r>r&oJIzCbf2;?aF1?utsfjGDg+qM zO)q;>0-wk8#22R73M-d=g}Jtrl%%cAVxX_uk9p6Wz^6qoiUTgqS;9Vi z>~6sx_t#Q{r*HflJ@uzsoaaMUkvj!%5s7n+-M2Ua>{{% zGaV7pCDwIubzUbn58D$0+UVg~J82(_LD@a8M4(#ZKNf@lD!Yg@TiYO`gm)-mU3@TW znX@UIzxqp=1Rq_TsMw*N@Fms=ORWi^590?_t15%8@Gwt3xfYoG>rtVC0$<$ammkaT z-iC3di_pgw#&wy+K}F5DN}LTU{CsnZApn2po$?|qhFqGoXxSy&XCKh=ha>DEV?@ty zciHbvl}0e@9FW`sU-N%>=jUI&vVmqA$RN7>M^_T6kHd9#IU`&AYn(F7_TX2VuJFCp zkneg%uW_uPaAFx!&FU?JqRsrcH~rd6K?wsh60;s9X6w)8`lpY z6<3OPF@loA<8m99%S7Z^A&n^@ZEdHp_ves{Y}wJ6eP?Uo9l8AM?Z_H42i&>L2lK)d zAqsu$hX&8xS1N^F`ZU}h+Xzj85ARM~-1>PQ7%?h@9jsTZl6&7!tf;G#Ag1WJMVhQz zZkyOVY3DtGCxocT zR=(^yio>`OkRog|yR65p2z8W#hp6!;w|b9emBKjjDH96)XZ>pm4XJqgJ0PY1&C;?7iok_C6y1%8368od4hz8h2$oDrHdSk?gk~XW9Uj`mXx11$+ehYGBTDQ_u z8y0}K0xo1GMAQ(?+aqC428`f+CZ^Kz+4W;oVNbJ1bzSq`d)ouGH7 zXx>B~Ed^Wi@oe4V222#$q^}bw5UJ-}sTw!U=lieK)7?>c@6-6b$NrCHkA`k`I!`KE z<;^W6g}8g+^BTf{d*m0*#szpi`dirxbz#mD8BG6Hl8tZ0=`^yn8S7?C;ZlOYf=wn- z)g_1_zXJL&*$~Hz9KWNR)~-+Rk&|l8N=;*sHl$=|n{a zoffU|7`)6rUx{tvrpxp(8Is&$$~*CiuZs*k`b(1)Z3P%Ww@k7v8A8Wwb;e`(mLE`SKCbYrc${mJp5 ze*Ji)K7UPg6`!7Fcg9e0WQ;D#F`Y?(W;Af|pB$8OjmC*M&w@MGT&DinE6!WR8=$tf4VoSD6TWv$ut84M;Mq zK5nt)`giVa;WOP}a65_DjuBu7W$egPDCo7^SBAPWywV8}m7V72;25vDR0%Xij9CCO-kyPt-dSC^p9V2{mq?vj<9 zm67$|aGQIFfPnsy3{e5z-Vpw?C#uAsyXxdC3%5c{E2oDfYC`x6*MVNE>R_)H(_QA+ zYX6O}XSs^DGT*h!EJi9TM2)UI3Gb^-L;?aAb^#?Yd7iO@+- zuQtnFTwpCI@w$iMQE*oOWBtH$Q<~#Co6D`}c@FILmw}bVj~D1t?b1H(wWWMb+Nqd` zmP5)dDfvB_R#ew$e{?w-_H5TJIU@L=g$zw>7&D`u@&o*aX3 z!k_XJP_S*VT^(8$a=CNt(lbwAmyG?x{z5cdA$qKU2+|!{MS$-iiZEa14 z4AIfw+%)Ih$B><%pGd(;MZY&+5R^D%U!0zXL6;a)Exz_$=a4!C{621RXIO-D_(HiW zgt2q_*5?4Y@zGAq_P%R)eL$@Bh(TFV)J-|`fXvg88FStFOU?Pw`^SCVmL{Hl0!r_0 znY1%M8JN&|Kj+4)w5AS@t4wW!&#wr=tySW76NZg@gGSoy8g|Bwz(GFt+@O|tvCaP9 zjRnppk5{R$IMpBfF($DLoPp2P#CsQVzc(`peUPl)O}7Q*!N>KK+hkecMu~rE(1^2< zTG)+k4o?;N^d%jZ`1KO0;~qzjy2zP{ioxKI8+|&MB#?>yCNUeTZS_M%an{5Lcc-Qu zd@X|+_j=UFhi3;Plj@#g&fZ%4#CKK7vRUG`X%FCCY;!LY9}LL)>*7-Wj(bm?D?mm9`tX-Miw#vyZgkTcc}K|RnYWMr6C+XtV7kw8cv zC5PjTfo79I%?9hK7{)A7wz)GtZZYY{g%noXhax+)s^9gM__-dx1qe3^ha8rY+Bbn+ z+;3_~#0y<0-iMkix^jndAJoO)GgSNYnjl732E%92!xmpZC>p8BCb(E%SKh~WP{z_M z%Th|1;vH-1sv?+58RP-@fxKZpmC*E}4YS&J4Ft|F{+~S?fHv9yoi^GhRhlEyubW z5?z4F=A}8HTvM;#krrj#PU(m2CaZ5Ceh%{`85c@N*{g@+=Yhw4)DJMnMOnM*pwHY4 zt=qBoobIv{v+j1zv5ERbs-&rBE9Xh5+TV`C_;Rn|W#e&o1A{D*OB&&N4K!B$MM@t+UP(01WEPq^vY zPMNkkQfEzT$L2=ZaG2okb;5ePKvR0JMe|+aQg89|brA7j>7BjlO`8M{CP`RQ(_%((nn zj^-3;Nr3oINC2Hkmz4u#{9v}r=>1i^r32}XmE}|JmWC_8c#HZKFNU0NNVmf57QYMA zS9RU{Qhrj<%$cE#8Lbgd@zCFzfn)ZY=RFUA!b@pIhl};$bKWRnEUaXbyaDzCUOt=f zd$<^@)=&Ryc#)vKZLT4;JnaT%p@G~lwx#{dE;w+6?Hnu3(aqN8%S~AVV#dBMS*N=L z&05@iJV%+Vi-Y!FZfk8z2ne+2;H5f>R-}02v{^GnyaWbro)EQ8zHm#iH?N3#ecmq@ zML$S0(pe`5PX_yGA86a?HwO>(ylEYDf ziRMjBzL?3O3wmD8@9nPF`fTFRzacO^+iFJB?WE|a!FL%irS0OhR)o!~uH4X-wGXy( zZ$4b|%GYH16q@}oLhDj?M0$Ys{=iiCC+_XfBkRyddFx?E!RD*Qs|VJ5L=f95MyS6D zFKH0na+qF0!RN`t8CGYeUsj@H-a8`14dsH;(%ME z44~EFOi^6CX#)|xvA8Z-mMm9NW{+TpRNIBvp|6 zFL~f|1C34!>-Sf*`8Ns9(5>a|$-y_vxkwVFZfrBx!bu0Y?S`bZxVJw#D>?MH0$H#Q z^Q9J~ZL*w{2)Ju4F64Vk|NMu(H9i-L;7Rj4&YuK=Yyj1SAcD1@-9mh&Vgssu-@Sih zb=!@aw}83)`h}hpm6_;H@yMy}IVqV|Z&Bijzr$=du;}jkjnJFBQawQQUu&;CYVR0q zj~h{J+)4w==S-f?=xc0Mof34pg5k$?%bpue7O!&Mj9%q3$g2++m;cghHr`Xd61^5B z`0_|U$I7ka*!KWuK_ZI=6@IGikVq(>7S^E#GF-rqkkfdf(~hv{2F`>Ntd_jM82Nl|q}D^TgFp2mot zhQzEDdCGaIh`g9RPLa4BqC9r4Sa)Q(p500lW7LxHjyM0czg(+B;#FyXK?`_WJd43M z;4mVt0!Y+P5h-$$OK(3%_u{m*TRh1^R>kIm<#2fEA{nFKuM97PTc1zH^*aUeIccSO z3Jmt)If=|4I6nJ`!V2RnSQ#z*Re#()gWtUTXF3J4)VOsX>!Umqrd3`_*bsJ>#EevB zHx;v6HOx@gaVI(;jSF6JFn=X%Jg+!T=C0=e(-C?9W7hfd$W_AHirZxoz#mRJdHm{j zzY{NCgOH*PRMzEOI_Xl+_}HYi3A<8G+%^+?Zt+HBV6uH!XR!Z82iTP0XxMpseGGc> zo2a{Tv(UV#$&Mz_g>$lTXQ5Hbwg~@yWr?PqSEiT zv-Q>-AwzlEsKG}k*eQQrF zNjsknr`_*HF#q+#acGGTamnQ*pk{OG7o(>^-%#0b3P*}NEs^>yG(~iEOSj1)-YB*9 zC2<#3jyw!?@1?VN)6k}ZpgJxww$a>kU@kBshKqD0qW1Teg4H=J^=@P$b2I-?$MYm z{6a{$5)MsVKRIuF_mk!R4_8F(-c4~&`J>-I^XncctWynh;dNHoCc%Vn#}gJ7^`Dx} z@BT|@QQ%Fy1z+J!*r?(fR%7WIVgg?6raE5Zw6bkdAhMEHk|Fvr zw-htB_fl<>iM-R(>P}2D*jf2WaFIk^%Ay^cqFxvLX%utbw_qsra6CCcb4d-fv|3q{ zjToPL(gs576^Hj`3x`>Q;fxmfGK7uI3b0XIV?E^qRmkGQncSt3pBd|oM6*eMEzi&d zRevvbGfDN%J3x{LHy_6hIy8c=C*(tmAK!L-A={p%@+tXjFr&A2-kcE6qk#0*sjNJo zSQ!yA@snZ_uz3ivJ8=Zv#)CvM?tM-`rRtRU23hanvVBSm-)KZ#BW|v+8M&O4;7~pN zM@F8ZMu5J+qT}0OE3hP)t_&_}`*uZyS2KKGH&Gh2wf8}`H1;ue|E2tNkr!u` zo%l>{&+>Bw19;>ij2&-eRQxI7)VPe@w6`BwaxvfzQcBS}b$w=8J|GqEtkH0AWN1*6 zIc~K+F`0NnsoH74xT?4k#+leo(2Edpba*`9H!L-M0+H@2mi;+U1w0+(r?!2+)aYJi z$SUPOUuu5=cGv7*H0{xzhzBe8In5bKB7_5kOvBd8V*5bVyi(m(3Kzz79Hu}C$RDHnH-m7vmEwJ(;wM6R zI*Ys5`?J?{irTIr@-lT`yfh81X2o%@#m1pX%UwBXUc#sbQ>{_hh{|sHw4;J)#b#+^ zn#;0eyvRI}5dVGZjj%$XO?v4wZS6d0<(#LXHQutmG&NG#AOi6ix}YLwYGl;)-Yhvy zDr@O2KuC62pN>utppu^l4}kY&;FR>cLK!x!(jCv#;0X3sDCWKExA3{AyHO)k+ZeS| zwZwj>VyPZG#ve7B4|RopIbnNNj$7Wjzpnh?8JDJ3%QtxvW~x`|YuV5!@~F(TBgl`S z;OR4}kilqitLu;OM6qf4$K6VV%||3ba>N;CRQ2I6%|Dr%w{OYc{E%iBN;^9`Rji3h zA($N}S8%hge}|^ueXM}*`*~_t3Av8KB!6DFwzw0EROHiiMNMH9YDRqu7;BNmK>B2Z z%Yn|3-F&fr)1lq_kja4PlJ68NL4JOeTyELg6kQZG@^3)M>TFw+BA)NJh2=1TG~ ziJaE&V&A~FKk^G|SKDhpOH<25p&}+($m9=kBoY&2*EPlx$u72?+MY+w$-Qw3x8o|+ z&CW?M?TOq<)7*8FoVFE3#Sxv`ze zH1Mca!FdO&)Zp?`R%kpDnoZrUpXaZwS796)L?*jTqe4{8#}GO?zU$c3$T|M{cxd){ z{>Ulsf=xnr-O<;1wA{L$1c3RdS)P09%@OI{DC<6ND@Gjax;so?+5fmtw>^jCX z_I)HhZb337>8P7^V^TaTUP*NgJTF%Adp|8Kq~R<4QqMF_uedvmE9`g2eYGpogk#|R z$t2Z9(R=H`Kwn?JF7bHqr$OO6457!YwhXWGssvPy$Co&rPE@(%q_rDh#6#%kk0iw4 zn!NMnlNCA-e1+y$Mpf+T*hTaJ6H<@O{bcuA)+84@S>pe z+{@@e&&OlKiA3%gg>-L#pU9#@Y!jw`pKW+w^xCvH9^+rE@_Z?>L%U6NUW~pcTdrW9 z`>r!SthsQ#ahJG;ug>seT7oXMePT*sop;dKh%t^y*h{SZ5qFvSbHOt!nl1e~?H5>F zCoV2e_`spkIM*6wWEI@A!Cb8Vbw8v)=GSu$Yr{PIY__BI?^{duNXQ>)9Wt@}DRdntR#+Z!s>3tYZT-a++o z5Tc$Miy0K>?U2L)<{3AakQTS^l#80l3a~bG&5FZ}{DNpJaz0c3CSB$@wrdwVApW=wBx zh;WuIS=!w8_z)KdL^DJDT=riAo<|X3f_U#sG_# zjD!gKI&*<gdgDa*_gHIO{c6j6=UR_2K86Mr zI__lS?lBTn&(gS~zQR%1h@@G&hVruH(S2BbQ-Su{K`%gI*0D^Wm@xtwSw6rpvb8S=m5@ND=Z_@O#~3e zqC&Oh9MyPf?LRg+ufn!gf|*d3=tBVr=w_rxCgIZ^1{xJe??0=zI==-U8vJ+XO zQiu2}oS&>C0;?lrQ)VF%C013SE59|9Z|$ZT>H(`pR8CUb6UlC&>{HFeP9=kCDe~SH z=ereNUe!^1vk~vs+VOVSq;w!hH9OxH*9Rt|zDYHbJT_B#ToZceCXbS-;f;W!w_%etR$^nDsf`&J zVxX@^6?I7U;ND8Xd*usyKkEA?id+h9S>0^g+(eJ(A!n$)#qp)BTvNV<)dnmP<(>gS zo>{nSt%==?ao4O^*&7|QP)S{N6HmNP{&ao}->I)^^9A7LVW$x2$Xa{#oG?Lf6Jy6u53g}4I+M`;sKcWA)_n}V z+3G%RG`9!?6D}#lr-|`T#Q+L03w#Z({tDdJo6+;d)>sd^WVWQ6H6~b*dv;&t z!KEA9r1-_Qtu+ShMp#rX9u*ARJ{ycan>rB7Ur4rUa5nu91Nw>IabG9t0<2_PJVNu^ z+Cu}3s|%D2q39{Z_SBj~L0)4J8c;McFZ2oj&j(7{#T<1UsQ>EMb{`MA#k9~&{%gIq zwS?b=sstolY-tt;6V@XJqcFO2soSBVM`IL!bjdK=W?UWVFAZ+=lw3Qg+HRkzNyl5p zvCJyW~@+Vh%2dI4-Zf?xYK>dzhnO=jvz?; zY)-CKoQ{%~>norVWLw6IWgYf4P9L5b9UWaNP2X#&cjVMk=-+^C|E0ON792POfW6|n zN6T^M8MMOcPG{K1K?MdXmz9Oq)TE$m2p%n~Uu~>sCMb}7GCF-h>&Hg?X{)>Ie){@< z52yY;T*zkHULn4*(4%J0Gl;I+M_LSSH*UXH|8y zPKF&|?U9#3bsW~{3$dd1){Xk$){FD8x_nX@-$KDC>q=tgc4{(6!*BvsI*F&8sG6-G zb%fpoqw^5OyzYY9Z&={gzW;!O6y;B7La z+n`$wp{}Ra8>Yp$#p*9twRzoc>ppigrR78Y*T-%AgY#4WBcfK1haJ*8Y~2R2-B{;= z!ZX3-GQ_q;_>dj?eAG^kt`*Fu4BN2OU?cUzC5Bsylb3t??FuLAe&E_KT1eKHZk_ri zz&N(W%8t6kIy|6;be>fO1-MEV<>n*#JDh5klH3km=7Y|twf$zmfB%=v;D4zNhQ?)D zLtli26Nic^K4o`Qb67}dH6w4O2qj=vroHd;nf^L;1}x~9x0~%K+YkLy-iQ)9+v&f5smFc z#2ssMQ={&mXL1(<8Sc5~bL##W|4D|?t+y0Y6*))!{K#hK9yMq8>9I}NBW#*S1N8LB z>EvudVzXn1uwbnI1(EM@BjAvy2bG|rko?NWdyVpR_gSV1$fXe;PwSa%)0reN zbW;6f?oLK*MGH2f%eL8#BaG|5DQ7!VYza;*pwG%8!2~97t7l;E!l%oFqY2w3%aPvz z_L8`*1KaSN^idp0xH-50Po)}zHGoz#S!YZt{8UvQV@!FPew2hqxYPC&or%kE*sbk0 z)|KnwtN~&4Xp0~CT$-JbS0%05UUNBFOrGSx#UjsI=Zrg6#(Oc53+fDBR0v%dPq?AC zCAJjyWxz{{kiis_6KOnWlZ^1W>@y{;kytM{C&9y!w((>2OL0KKTOvIqbgH6==C7+JE9-m$Ka zkN#{B-*)PlY3_-ty~xtgf{x21oF?53=t{HAQ<3xG-W0X1^Aq-&?BCG7DF65C- zL9sF(c1!4`QG5|Lz6hTNnkucr>N^w=?cNr%_8R2cDrhSVNb4A;S+5&H^kiFHY?SL~ z)sqV9qPlVN;rH7ac+8I@BymbLkzJWp$nP2SYF^7p6YjwfNl=k!=Lp7x5WW3w?8iCvyv^VDvxhizZLM|ku(d)uKT zXKcwvs&sRr9W7uy^Xg+(BCO_*^GlWFiW~4bXMU=SeOV_LBVixYe`YNtqO0 zS)6k4vAAoz-qfSy4U^X{V`fWRf3r6pStygsVV9l9dVw4)DG=IOqvjrD*%JgTkWKG_%ZJ%-%my|vhPwxk>R>O<} z?|5H#tptP1;EgqU1?C!qMIGq&0@6G@CRaW1O@FUM_RQYiV~Br)Wq$SsSZwp~jV^UR z^Lm?~6zg@7r}E#)7!BLVtmKK=J3eh5PmM5hqc^=b?K`9QYFXvDO3Ezm3C)QN$fcE+ z2P>`Pd(lF|X)ZnCSiF(T$VSF^r`O|Wxk7Wb-=o#(bu7n=%{$y){H2lOeyH8W7z5pQ z)Twga+QhX4F?XutxfcpHH$rNUmAmo1pJB1DhUc0wHV*>m3}_*z+qc%A*Mpts64%;< zLQA!xvN~nrbuN7hj{PzmI1YtpVUgXk;nj;fUrPuRgN#rUh=sKWtd@dz z=Cp8`{@u(j&KPZ*EBPiz0h!@G_i~NnKM0Iza=!`|Fy))>cz1a({HM`5e|3+W?JneJ z2LAfw)#-qm9}XUmjy+VuPAOgF`d+0cH@RFg@m*#pq@{)gzo*Nd3bw0i`#M>8HZhuR!M!hx#qxyd)7S;*2sT9$Hv4y z7HY#yC(6Jhw#KT_z^U7=r7Xl<)PCc5YA1ExEZwi*>iO%Pq*$5b->O}IT>NdowbxaK zTha@>#HfLZx0T*IHh98N=EZykJ8*VarYyoZU0fZQpnRr1;e0;|wUUcqAt07YM+gOe zkrj7**RVg;5vB-x9q6_7iSk7<6ChU+X4p|zgkGtKXb#PAP|-j|1@+uN{%nyCx!DNE zUEPnffWK8ZRjZ4}01RVykE-^1iq|%~KdFrcbRU>HcGfIezM0>?X(wDI^bwXbUm-B8 zDdFkMgOiu3Jz7QF)UStWpI)S`K)AJy?{lxnH1h(>&#%br$Dn-^Wk-sKcN797d3 z9I0lPNDx|9Kr+FP-noca8yULw?c4TpH{!@OX9lm-@^+Z#7CUaY(S_{)lP@`563Sza zxN!)!+f{?p4nOfVd#NH-Y*nBb!H#@8EitdS5_j&{Jy6Qd-!&-mvtL_rj`&u^u0?n% z&nZ}Q2r~D3S%wHcTn(ME8~wz%c#vL2(Ufk`>nUT3+zkcb}4vt{z9g6>r6jx?@V)NaQgAg zUz&&PIbT~l?z`l?yCQ)PB-`0;TaLMS>LXOIaX=e?P(5Dh7A~&IKh3M#?GWY2J z{P^s&_u7zmU5xF{71vbdPxpU)d4=mp_Do{*0@imevN}%e^=zqx2I$kJW1@(;eDf=D zX+YUxjpsm$g?3NDpvs5&$EMKvTEz}_FY)Jc^LI@0oc}myW-U)G3s)iC%NpQrg8D;L=+taF;5-|bRqdu*HR4(3P}GtLK{ue%=CV|

{if (showDuration) { - + }} } From 679e4b6593a1c3aa439d9c8bb6237d203fba0e58 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 12 Jan 2024 13:49:24 -0800 Subject: [PATCH 175/521] [SPARK-46700][CORE] Count the last spilling for the shuffle disk spilling bytes metric ### What changes were proposed in this pull request? This PR fixes a long-standing bug in ShuffleExternalSorter about the "spilled disk bytes" metrics. When we close the sorter, we will spill the remaining data in the buffer, with a flag `isLastFile = true`. This flag means the spilling will not increase the "spilled disk bytes" metrics. This makes sense if the sorter has never spilled before, then the final spill file will be used as the final shuffle output file, and we should keep the "spilled disk bytes" metrics as 0. However, if spilling did happen before, then we simply miscount the final spill file for the "spilled disk bytes" metrics today. This PR fixes this issue, by setting that flag when closing the sorter only if this is the first spilling. ### Why are the changes needed? make metrics accurate ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? updated tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #44709 from cloud-fan/shuffle. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun (cherry picked from commit 4ea374257c1fdb276abcd6b953ba042593e4d5a3) Signed-off-by: Dongjoon Hyun --- .../shuffle/sort/ShuffleExternalSorter.java | 34 +++++++++++-------- .../shuffle/sort/UnsafeShuffleWriter.java | 6 ---- .../sort/UnsafeShuffleWriterSuite.java | 20 +++++++---- 3 files changed, 33 insertions(+), 27 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index a82f691d085d4..b097089282ce3 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -150,11 +150,21 @@ public long[] getChecksums() { * Sorts the in-memory records and writes the sorted records to an on-disk file. * This method does not free the sort data structures. * - * @param isLastFile if true, this indicates that we're writing the final output file and that the - * bytes written should be counted towards shuffle spill metrics rather than - * shuffle write metrics. + * @param isFinalFile if true, this indicates that we're writing the final output file and that + * the bytes written should be counted towards shuffle write metrics rather + * than shuffle spill metrics. */ - private void writeSortedFile(boolean isLastFile) { + private void writeSortedFile(boolean isFinalFile) { + // Only emit the log if this is an actual spilling. + if (!isFinalFile) { + logger.info( + "Task {} on Thread {} spilling sort data of {} to disk ({} {} so far)", + taskContext.taskAttemptId(), + Thread.currentThread().getId(), + Utils.bytesToString(getMemoryUsage()), + spills.size(), + spills.size() != 1 ? " times" : " time"); + } // This call performs the actual sort. final ShuffleInMemorySorter.ShuffleSorterIterator sortedRecords = @@ -167,13 +177,14 @@ private void writeSortedFile(boolean isLastFile) { final ShuffleWriteMetricsReporter writeMetricsToUse; - if (isLastFile) { + if (isFinalFile) { // We're writing the final non-spill file, so we _do_ want to count this as shuffle bytes. writeMetricsToUse = writeMetrics; } else { // We're spilling, so bytes written should be counted towards spill rather than write. // Create a dummy WriteMetrics object to absorb these metrics, since we don't want to count // them towards shuffle bytes written. + // The actual shuffle bytes written will be counted when we merge the spill files. writeMetricsToUse = new ShuffleWriteMetrics(); } @@ -246,7 +257,7 @@ private void writeSortedFile(boolean isLastFile) { spills.add(spillInfo); } - if (!isLastFile) { // i.e. this is a spill file + if (!isFinalFile) { // i.e. this is a spill file // The current semantics of `shuffleRecordsWritten` seem to be that it's updated when records // are written to disk, not when they enter the shuffle sorting code. DiskBlockObjectWriter // relies on its `recordWritten()` method being called in order to trigger periodic updates to @@ -281,12 +292,6 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { return 0L; } - logger.info("Thread {} spilling sort data of {} to disk ({} {} so far)", - Thread.currentThread().getId(), - Utils.bytesToString(getMemoryUsage()), - spills.size(), - spills.size() > 1 ? " times" : " time"); - writeSortedFile(false); final long spillSize = freeMemory(); inMemSorter.reset(); @@ -440,8 +445,9 @@ public void insertRecord(Object recordBase, long recordOffset, int length, int p */ public SpillInfo[] closeAndGetSpills() throws IOException { if (inMemSorter != null) { - // Do not count the final file towards the spill count. - writeSortedFile(true); + // Here we are spilling the remaining data in the buffer. If there is no spill before, this + // final spill file will be the final shuffle output file. + writeSortedFile(/* isFinalFile = */spills.isEmpty()); freeMemory(); inMemSorter.free(); inMemSorter = null; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 9c54184105951..d5b4eb138b1a6 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -327,12 +327,6 @@ private long[] mergeSpillsUsingStandardWriter(SpillInfo[] spills) throws IOExcep logger.debug("Using slow merge"); mergeSpillsWithFileStream(spills, mapWriter, compressionCodec); } - // When closing an UnsafeShuffleExternalSorter that has already spilled once but also has - // in-memory records, we write out the in-memory records to a file but do not count that - // final write as bytes spilled (instead, it's accounted as shuffle write). The merge needs - // to be counted as shuffle write, but this will lead to double-counting of the final - // SpillInfo's bytes. - writeMetrics.decBytesWritten(spills[spills.length - 1].file.length()); partitionLengths = mapWriter.commitAllPartitions(sorter.getChecksums()).getPartitionLengths(); } catch (Exception e) { try { diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index d3aa93549a83a..1fa17b908699f 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -69,6 +69,7 @@ public class UnsafeShuffleWriterSuite implements ShuffleChecksumTestHelper { File tempDir; long[] partitionSizesInMergedFile; final LinkedList spillFilesCreated = new LinkedList<>(); + long totalSpilledDiskBytes = 0; SparkConf conf; final Serializer serializer = new KryoSerializer(new SparkConf().set("spark.kryo.unsafe", "false")); @@ -96,6 +97,7 @@ public void setUp() throws Exception { mergedOutputFile = File.createTempFile("mergedoutput", "", tempDir); partitionSizesInMergedFile = null; spillFilesCreated.clear(); + totalSpilledDiskBytes = 0; conf = new SparkConf() .set(package$.MODULE$.BUFFER_PAGESIZE().key(), "1m") .set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false) @@ -160,7 +162,11 @@ public void setUp() throws Exception { when(diskBlockManager.createTempShuffleBlock()).thenAnswer(invocationOnMock -> { TempShuffleBlockId blockId = new TempShuffleBlockId(UUID.randomUUID()); - File file = File.createTempFile("spillFile", ".spill", tempDir); + File file = spy(File.createTempFile("spillFile", ".spill", tempDir)); + when(file.delete()).thenAnswer(inv -> { + totalSpilledDiskBytes += file.length(); + return inv.callRealMethod(); + }); spillFilesCreated.add(file); return Tuple2$.MODULE$.apply(blockId, file); }); @@ -284,6 +290,9 @@ public void writeWithoutSpilling() throws Exception { final Option mapStatus = writer.stop(true); assertTrue(mapStatus.isDefined()); assertTrue(mergedOutputFile.exists()); + // Even if there is no spill, the sorter still writes its data to a spill file at the end, + // which will become the final shuffle file. + assertEquals(1, spillFilesCreated.size()); long sumOfPartitionSizes = 0; for (long size: partitionSizesInMergedFile) { @@ -425,9 +434,8 @@ private void testMergingSpills( assertSpillFilesWereCleanedUp(); ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics(); assertEquals(dataToWrite.size(), shuffleWriteMetrics.recordsWritten()); - assertTrue(taskMetrics.diskBytesSpilled() > 0L); - assertTrue(taskMetrics.diskBytesSpilled() < mergedOutputFile.length()); assertTrue(taskMetrics.memoryBytesSpilled() > 0L); + assertEquals(totalSpilledDiskBytes, taskMetrics.diskBytesSpilled()); assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.bytesWritten()); } @@ -517,9 +525,8 @@ public void writeEnoughDataToTriggerSpill() throws Exception { assertSpillFilesWereCleanedUp(); ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics(); assertEquals(dataToWrite.size(), shuffleWriteMetrics.recordsWritten()); - assertTrue(taskMetrics.diskBytesSpilled() > 0L); - assertTrue(taskMetrics.diskBytesSpilled() < mergedOutputFile.length()); assertTrue(taskMetrics.memoryBytesSpilled()> 0L); + assertEquals(totalSpilledDiskBytes, taskMetrics.diskBytesSpilled()); assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.bytesWritten()); } @@ -550,9 +557,8 @@ private void writeEnoughRecordsToTriggerSortBufferExpansionAndSpill() throws Exc assertSpillFilesWereCleanedUp(); ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics(); assertEquals(dataToWrite.size(), shuffleWriteMetrics.recordsWritten()); - assertTrue(taskMetrics.diskBytesSpilled() > 0L); - assertTrue(taskMetrics.diskBytesSpilled() < mergedOutputFile.length()); assertTrue(taskMetrics.memoryBytesSpilled()> 0L); + assertEquals(totalSpilledDiskBytes, taskMetrics.diskBytesSpilled()); assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.bytesWritten()); } From 10d5d8956b4cc22f17b1752dc91766398e1540ee Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 16 Jan 2024 14:16:03 +0800 Subject: [PATCH 176/521] [SPARK-46715][INFRA][3.5] Pin `sphinxcontrib-*` ### What changes were proposed in this pull request? backport https://github.com/apache/spark/pull/44727 to branch-3.5 ### Why are the changes needed? to restore doc build ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #44744 from zhengruifeng/infra_pin_shinxcontrib. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .github/workflows/build_and_test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 32f6a44102bf9..b0b72a0d1a689 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -678,7 +678,7 @@ jobs: # See also https://issues.apache.org/jira/browse/SPARK-35375. # Pin the MarkupSafe to 2.0.1 to resolve the CI error. # See also https://issues.apache.org/jira/browse/SPARK-38279. - python3.9 -m pip install 'sphinx<3.1.0' mkdocs pydata_sphinx_theme 'sphinx-copybutton==0.5.2' nbsphinx numpydoc 'jinja2<3.0.0' 'markupsafe==2.0.1' 'pyzmq<24.0.0' + python3.9 -m pip install 'sphinx<3.1.0' mkdocs pydata_sphinx_theme 'sphinx-copybutton==0.5.2' nbsphinx numpydoc 'jinja2<3.0.0' 'markupsafe==2.0.1' 'pyzmq<24.0.0' 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' 'nest-asyncio==1.5.8' 'rpds-py==0.16.2' 'alabaster==0.7.13' python3.9 -m pip install ipython_genutils # See SPARK-38517 python3.9 -m pip install sphinx_plotly_directive 'numpy>=1.20.0' pyarrow pandas 'plotly>=4.8' python3.9 -m pip install 'docutils<0.18.0' # See SPARK-39421 From 5a0bc96d5f5e42fa5e6ea9d024da572343f239a9 Mon Sep 17 00:00:00 2001 From: xieshuaihu Date: Wed, 17 Jan 2024 15:24:58 +0900 Subject: [PATCH 177/521] [SPARK-46732][CONNECT][3.5] Make Subquery/Broadcast thread work with Connect's artifact management ### What changes were proposed in this pull request? Similar with SPARK-44794, propagate JobArtifactState to broadcast/subquery thread. This is an example: ```scala val add1 = udf((i: Long) => i + 1) val tableA = spark.range(2).alias("a") val tableB = broadcast(spark.range(2).select(add1(col("id")).alias("id"))).alias("b") tableA.join(tableB). where(col("a.id")===col("b.id")). select(col("a.id").alias("a_id"), col("b.id").alias("b_id")). collect(). mkString("[", ", ", "]") ``` Before this pr, this example will throw exception `ClassNotFoundException`. Subquery and Broadcast execution use a separate ThreadPool which don't have the `JobArtifactState`. ### Why are the changes needed? Fix bug. Make Subquery/Broadcast thread work with Connect's artifact management. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Add a new test to `ReplE2ESuite` ### Was this patch authored or co-authored using generative AI tooling? No Closes #44763 from xieshuaihu/SPARK-46732backport. Authored-by: xieshuaihu Signed-off-by: Hyukjin Kwon --- .../spark/sql/application/ReplE2ESuite.scala | 16 ++++++++++++++++ .../spark/sql/execution/SQLExecution.scala | 5 +++-- 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala index 5bb8cbf3543b0..9d61b4d56e1ed 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala @@ -362,4 +362,20 @@ class ReplE2ESuite extends RemoteSparkSession with BeforeAndAfterEach { val output = runCommandsInShell(input) assertContains("noException: Boolean = true", output) } + + test("broadcast works with REPL generated code") { + val input = + """ + |val add1 = udf((i: Long) => i + 1) + |val tableA = spark.range(2).alias("a") + |val tableB = broadcast(spark.range(2).select(add1(col("id")).alias("id"))).alias("b") + |tableA.join(tableB). + | where(col("a.id")===col("b.id")). + | select(col("a.id").alias("a_id"), col("b.id").alias("b_id")). + | collect(). + | mkString("[", ", ", "]") + |""".stripMargin + val output = runCommandsInShell(input) + assertContains("""String = "[[1,1]]"""", output) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index daeac699c2791..b4cbb61352235 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution import java.util.concurrent.{ConcurrentHashMap, ExecutorService, Future => JFuture} import java.util.concurrent.atomic.AtomicLong -import org.apache.spark.{ErrorMessageFormat, SparkContext, SparkThrowable, SparkThrowableHelper} +import org.apache.spark.{ErrorMessageFormat, JobArtifactSet, SparkContext, SparkThrowable, SparkThrowableHelper} import org.apache.spark.internal.config.{SPARK_DRIVER_PREFIX, SPARK_EXECUTOR_PREFIX} import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.sql.SparkSession @@ -215,7 +215,8 @@ object SQLExecution { val activeSession = sparkSession val sc = sparkSession.sparkContext val localProps = Utils.cloneProperties(sc.getLocalProperties) - exec.submit(() => { + val artifactState = JobArtifactSet.getCurrentJobArtifactState.orNull + exec.submit(() => JobArtifactSet.withActiveJobArtifactState(artifactState) { val originalSession = SparkSession.getActiveSession val originalLocalProps = sc.getLocalProperties SparkSession.setActiveSession(activeSession) From d083da76b4d6b4f1351f2b4597840e2cc1a8683a Mon Sep 17 00:00:00 2001 From: Xinrong Meng Date: Thu, 18 Jan 2024 09:01:05 +0900 Subject: [PATCH 178/521] [SPARK-46663][PYTHON][3.5] Disable memory profiler for pandas UDFs with iterators ### What changes were proposed in this pull request? When using pandas UDFs with iterators, if users enable the profiling spark conf, a warning indicating non-support should be raised, and profiling should be disabled. However, currently, after raising the not-supported warning, the memory profiler is still being enabled. The PR proposed to fix that. ### Why are the changes needed? A bug fix to eliminate misleading behavior. ### Does this PR introduce _any_ user-facing change? The noticeable changes will affect only those using the PySpark shell. This is because, in the PySpark shell, the memory profiler will raise an error, which in turn blocks the execution of the UDF. ### How was this patch tested? Manual test. ### Was this patch authored or co-authored using generative AI tooling? Setup: ```py $ ./bin/pyspark --conf spark.python.profile=true >>> from typing import Iterator >>> from pyspark.sql.functions import * >>> import pandas as pd >>> pandas_udf("long") ... def plus_one(iterator: Iterator[pd.Series]) -> Iterator[pd.Series]: ... for s in iterator: ... yield s + 1 ... >>> df = spark.createDataFrame(pd.DataFrame([1, 2, 3], columns=["v"])) ``` Before: ``` >>> df.select(plus_one(df.v)).show() UserWarning: Profiling UDFs with iterators input/output is not supported. Traceback (most recent call last): ... OSError: could not get source code ``` After: ``` >>> df.select(plus_one(df.v)).show() /Users/xinrong.meng/spark/python/pyspark/sql/udf.py:417: UserWarning: Profiling UDFs with iterators input/output is not supported. +-----------+ |plus_one(v)| +-----------+ | 2| | 3| | 4| +-----------+ ``` Closes #44760 from xinrong-meng/PR_TOOL_PICK_PR_44668_BRANCH-3.5. Authored-by: Xinrong Meng Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/tests/test_udf_profiler.py | 53 ++++++++++++++++++- python/pyspark/sql/udf.py | 32 ++++++----- 2 files changed, 67 insertions(+), 18 deletions(-) diff --git a/python/pyspark/sql/tests/test_udf_profiler.py b/python/pyspark/sql/tests/test_udf_profiler.py index 136f423d0a35c..019e502ec67cf 100644 --- a/python/pyspark/sql/tests/test_udf_profiler.py +++ b/python/pyspark/sql/tests/test_udf_profiler.py @@ -19,11 +19,19 @@ import unittest import os import sys +import warnings from io import StringIO +from typing import Iterator from pyspark import SparkConf from pyspark.sql import SparkSession -from pyspark.sql.functions import udf +from pyspark.sql.functions import udf, pandas_udf +from pyspark.testing.sqlutils import ( + have_pandas, + have_pyarrow, + pandas_requirement_message, + pyarrow_requirement_message, +) from pyspark.profiler import UDFBasicProfiler @@ -101,6 +109,49 @@ def add2(x): df = self.spark.range(10) df.select(add1("id"), add2("id"), add1("id")).collect() + # Unsupported + def exec_pandas_udf_iter_to_iter(self): + import pandas as pd + + @pandas_udf("int") + def iter_to_iter(batch_ser: Iterator[pd.Series]) -> Iterator[pd.Series]: + for ser in batch_ser: + yield ser + 1 + + self.spark.range(10).select(iter_to_iter("id")).collect() + + # Unsupported + def exec_map(self): + import pandas as pd + + def map(pdfs: Iterator[pd.DataFrame]) -> Iterator[pd.DataFrame]: + for pdf in pdfs: + yield pdf[pdf.id == 1] + + df = self.spark.createDataFrame([(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0)], ("id", "v")) + df.mapInPandas(map, schema=df.schema).collect() + + @unittest.skipIf(not have_pandas, pandas_requirement_message) # type: ignore + @unittest.skipIf(not have_pyarrow, pyarrow_requirement_message) # type: ignore + def test_unsupported(self): + with warnings.catch_warnings(record=True) as warns: + warnings.simplefilter("always") + self.exec_pandas_udf_iter_to_iter() + user_warns = [warn.message for warn in warns if isinstance(warn.message, UserWarning)] + self.assertTrue(len(user_warns) > 0) + self.assertTrue( + "Profiling UDFs with iterators input/output is not supported" in str(user_warns[0]) + ) + + with warnings.catch_warnings(record=True) as warns: + warnings.simplefilter("always") + self.exec_map() + user_warns = [warn.message for warn in warns if isinstance(warn.message, UserWarning)] + self.assertTrue(len(user_warns) > 0) + self.assertTrue( + "Profiling UDFs with iterators input/output is not supported" in str(user_warns[0]) + ) + if __name__ == "__main__": from pyspark.sql.tests.test_udf_profiler import * # noqa: F401 diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py index 7d7784dd5226d..bdd3aba502b89 100644 --- a/python/pyspark/sql/udf.py +++ b/python/pyspark/sql/udf.py @@ -28,7 +28,6 @@ from py4j.java_gateway import JavaObject from pyspark import SparkContext -from pyspark.profiler import Profiler from pyspark.rdd import _prepare_for_python_RDD, PythonEvalType from pyspark.sql.column import Column, _to_java_column, _to_java_expr, _to_seq from pyspark.sql.types import ( @@ -338,24 +337,23 @@ def _create_judf(self, func: Callable[..., Any]) -> JavaObject: def __call__(self, *cols: "ColumnOrName") -> Column: sc = get_active_spark_context() - profiler: Optional[Profiler] = None - memory_profiler: Optional[Profiler] = None - if sc.profiler_collector: - profiler_enabled = sc._conf.get("spark.python.profile", "false") == "true" - memory_profiler_enabled = sc._conf.get("spark.python.profile.memory", "false") == "true" + profiler_enabled = sc._conf.get("spark.python.profile", "false") == "true" + memory_profiler_enabled = sc._conf.get("spark.python.profile.memory", "false") == "true" + if profiler_enabled or memory_profiler_enabled: # Disable profiling Pandas UDFs with iterators as input/output. - if profiler_enabled or memory_profiler_enabled: - if self.evalType in [ - PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF, - PythonEvalType.SQL_MAP_PANDAS_ITER_UDF, - PythonEvalType.SQL_MAP_ARROW_ITER_UDF, - ]: - profiler_enabled = memory_profiler_enabled = False - warnings.warn( - "Profiling UDFs with iterators input/output is not supported.", - UserWarning, - ) + if self.evalType in [ + PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF, + PythonEvalType.SQL_MAP_PANDAS_ITER_UDF, + PythonEvalType.SQL_MAP_ARROW_ITER_UDF, + ]: + warnings.warn( + "Profiling UDFs with iterators input/output is not supported.", + UserWarning, + ) + judf = self._judf + jPythonUDF = judf.apply(_to_seq(sc, cols, _to_java_column)) + return Column(jPythonUDF) # Disallow enabling two profilers at the same time. if profiler_enabled and memory_profiler_enabled: From b27d169c85e99ceffb20a6df8d10340749ab2129 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 18 Jan 2024 08:19:52 -0800 Subject: [PATCH 179/521] [MINOR][DOCS] Add zstandard as a candidate to fix the desc of spark.sql.avro.compression.codec ### What changes were proposed in this pull request? Add zstandard as a candidate to fix the desc of spark.sql.avro.compression.codec ### Why are the changes needed? docfix ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? doc build ### Was this patch authored or co-authored using generative AI tooling? no Closes #44783 from yaooqinn/avro_minor. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun (cherry picked from commit c040824fd75c955dbc8e5712bc473a0ddb9a8c0f) Signed-off-by: Dongjoon Hyun --- docs/sql-data-sources-avro.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-data-sources-avro.md b/docs/sql-data-sources-avro.md index c846116ebf3e3..1da75e5d7b17e 100644 --- a/docs/sql-data-sources-avro.md +++ b/docs/sql-data-sources-avro.md @@ -348,7 +348,7 @@ Configuration of Avro can be done using the `setConf` method on SparkSession or From fa6bf22112b4300dae1e7617f1480c0d12124b90 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Fri, 19 Jan 2024 11:38:53 +0900 Subject: [PATCH 180/521] [SPARK-46676][SS] dropDuplicatesWithinWatermark should not fail on canonicalization of the plan ### What changes were proposed in this pull request? This PR proposes to fix the bug on canonicalizing the plan which contains the physical node of dropDuplicatesWithinWatermark (`StreamingDeduplicateWithinWatermarkExec`). ### Why are the changes needed? Canonicalization of the plan will replace the expressions (including attributes) to remove out cosmetic, including name, "and metadata", which denotes the event time column marker. StreamingDeduplicateWithinWatermarkExec assumes that the input attributes of child node contain the event time column, and it is determined at the initialization of the node instance. Once canonicalization is being triggered, child node will lose the notion of event time column from its attributes, and copy of StreamingDeduplicateWithinWatermarkExec will be performed which instantiating a new node of `StreamingDeduplicateWithinWatermarkExec` with new child node, which no longer has an event time column, hence instantiation will fail. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New UT added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44688 from HeartSaVioR/SPARK-46676. Authored-by: Jungtaek Lim Signed-off-by: Jungtaek Lim (cherry picked from commit c1ed3e60e67f53bb323e2b9fa47789fcde70a75a) Signed-off-by: Jungtaek Lim --- .../streaming/statefulOperators.scala | 10 ++++++--- ...ingDeduplicationWithinWatermarkSuite.scala | 21 +++++++++++++++++++ 2 files changed, 28 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala index b31f6151fce23..b597c9723f5cf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala @@ -1037,10 +1037,14 @@ case class StreamingDeduplicateWithinWatermarkExec( protected val extraOptionOnStateStore: Map[String, String] = Map.empty - private val eventTimeCol: Attribute = WatermarkSupport.findEventTimeColumn(child.output, + // Below three variables are defined as lazy, as evaluating these variables does not work with + // canonicalized plan. Specifically, attributes in child won't have an event time column in + // the canonicalized plan. These variables are NOT referenced in canonicalized plan, hence + // defining these variables as lazy would avoid such error. + private lazy val eventTimeCol: Attribute = WatermarkSupport.findEventTimeColumn(child.output, allowMultipleEventTimeColumns = false).get - private val delayThresholdMs = eventTimeCol.metadata.getLong(EventTimeWatermark.delayKey) - private val eventTimeColOrdinal: Int = child.output.indexOf(eventTimeCol) + private lazy val delayThresholdMs = eventTimeCol.metadata.getLong(EventTimeWatermark.delayKey) + private lazy val eventTimeColOrdinal: Int = child.output.indexOf(eventTimeCol) protected def initializeReusedDupInfoRow(): Option[UnsafeRow] = { val timeoutToUnsafeRow = UnsafeProjection.create(schemaForValueRow) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala index 595fc1cb9cea8..9a02ab3df7dd4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationWithinWatermarkSuite.scala @@ -199,4 +199,25 @@ class StreamingDeduplicationWithinWatermarkSuite extends StateStoreMetricsTest { ) } } + + test("SPARK-46676: canonicalization of StreamingDeduplicateWithinWatermarkExec should work") { + withTempDir { checkpoint => + val dedupeInputData = MemoryStream[(String, Int)] + val dedupe = dedupeInputData.toDS() + .withColumn("eventTime", timestamp_seconds($"_2")) + .withWatermark("eventTime", "10 second") + .dropDuplicatesWithinWatermark("_1") + .select($"_1", $"eventTime".cast("long").as[Long]) + + testStream(dedupe, Append)( + StartStream(checkpointLocation = checkpoint.getCanonicalPath), + AddData(dedupeInputData, "a" -> 1), + CheckNewAnswer("a" -> 1), + Execute { q => + // This threw out error before SPARK-46676. + q.lastExecution.executedPlan.canonicalized + } + ) + } + } } From c19bf01b5208bb3aad0e6264b64597e0809b1efe Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 20 Jan 2024 20:57:09 +0800 Subject: [PATCH 181/521] [SPARK-46769][SQL] Refine timestamp related schema inference This is a refinement of https://github.com/apache/spark/pull/43243 . This PR enforces one thing: we only infer TIMESTAMP NTZ type using NTZ parser, and only infer LTZ type using LTZ parser. This consistency is important to avoid nondeterministic behaviors. Avoid non-deterministic behaviors. After https://github.com/apache/spark/pull/43243 , we can still have inconsistency if the LEGACY mode is enabled. Yes for the legacy parser. Now it's more likely to infer string type instead of inferring timestamp type "by luck" existing tests no Closes https://github.com/apache/spark/pull/44789 Closes #44800 from cloud-fan/infer. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit e4e40762ca41931646b8f201028b1f2298252d96) Signed-off-by: Wenchen Fan --- .../sql/catalyst/csv/CSVInferSchema.scala | 18 ++++---- .../sql/catalyst/json/JsonInferSchema.scala | 32 ++++++++++---- .../execution/datasources/csv/CSVSuite.scala | 42 +++++++++---------- 3 files changed, 55 insertions(+), 37 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala index ec01b56f9eb7c..2c27da3cf6e15 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.util.{DateFormatter, TimestampFormatter} import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ class CSVInferSchema(val options: CSVOptions) extends Serializable { @@ -66,6 +66,8 @@ class CSVInferSchema(val options: CSVOptions) extends Serializable { private val LENIENT_TS_FORMATTER_SUPPORTED_DATE_FORMATS = Set( "yyyy-MM-dd", "yyyy-M-d", "yyyy-M-dd", "yyyy-MM-d", "yyyy-MM", "yyyy-M", "yyyy") + private val isDefaultNTZ = SQLConf.get.timestampType == TimestampNTZType + /** * Similar to the JSON schema inference * 1. Infer type of each row @@ -199,14 +201,12 @@ class CSVInferSchema(val options: CSVOptions) extends Serializable { } private def tryParseTimestampNTZ(field: String): DataType = { - // We can only parse the value as TimestampNTZType if it does not have zone-offset or - // time-zone component and can be parsed with the timestamp formatter. - // Otherwise, it is likely to be a timestamp with timezone. - val timestampType = SQLConf.get.timestampType - if ((SQLConf.get.legacyTimeParserPolicy == LegacyBehaviorPolicy.LEGACY || - timestampType == TimestampNTZType) && - timestampNTZFormatter.parseWithoutTimeZoneOptional(field, false).isDefined) { - timestampType + // For text-based format, it's ambiguous to infer a timestamp string without timezone, as it can + // be both TIMESTAMP LTZ and NTZ. To avoid behavior changes with the new support of NTZ, here + // we only try to infer NTZ if the config is set to use NTZ by default. + if (isDefaultNTZ && + timestampNTZFormatter.parseWithoutTimeZoneOptional(field, false).isDefined) { + TimestampNTZType } else { tryParseTimestamp(field) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala index 4123c5290b6a1..f6d32f39f64ee 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable { @@ -53,6 +54,9 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable { isParsing = true, forTimestampNTZ = true) + private val isDefaultNTZ = SQLConf.get.timestampType == TimestampNTZType + private val legacyMode = SQLConf.get.legacyTimeParserPolicy == LegacyBehaviorPolicy.LEGACY + private def handleJsonErrorsByParseMode(parseMode: ParseMode, columnNameOfCorruptRecord: String, e: Throwable): Option[StructType] = { parseMode match { @@ -148,16 +152,30 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable { val bigDecimal = decimalParser(field) DecimalType(bigDecimal.precision, bigDecimal.scale) } - val timestampType = SQLConf.get.timestampType if (options.prefersDecimal && decimalTry.isDefined) { decimalTry.get - } else if (options.inferTimestamp && (SQLConf.get.legacyTimeParserPolicy == - LegacyBehaviorPolicy.LEGACY || timestampType == TimestampNTZType) && + } else if (options.inferTimestamp) { + // For text-based format, it's ambiguous to infer a timestamp string without timezone, as + // it can be both TIMESTAMP LTZ and NTZ. To avoid behavior changes with the new support + // of NTZ, here we only try to infer NTZ if the config is set to use NTZ by default. + if (isDefaultNTZ && timestampNTZFormatter.parseWithoutTimeZoneOptional(field, false).isDefined) { - timestampType - } else if (options.inferTimestamp && - timestampFormatter.parseOptional(field).isDefined) { - TimestampType + TimestampNTZType + } else if (timestampFormatter.parseOptional(field).isDefined) { + TimestampType + } else if (legacyMode) { + val utf8Value = UTF8String.fromString(field) + // There was a mistake that we use TIMESTAMP NTZ parser to infer LTZ type with legacy + // mode. The mistake makes it easier to infer TIMESTAMP LTZ type and we have to keep + // this behavior now. See SPARK-46769 for more details. + if (SparkDateTimeUtils.stringToTimestampWithoutTimeZone(utf8Value, false).isDefined) { + TimestampType + } else { + StringType + } + } else { + StringType + } } else { StringType } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 3bd45ca0dcdb3..78266acfd7de9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -1105,10 +1105,12 @@ abstract class CSVSuite test("SPARK-37326: Timestamp type inference for a column with TIMESTAMP_NTZ values") { withTempPath { path => - val exp = spark.sql(""" - select timestamp_ntz'2020-12-12 12:12:12' as col0 union all - select timestamp_ntz'2020-12-12 12:12:12' as col0 - """) + val exp = spark.sql( + """ + |select * + |from values (timestamp_ntz'2020-12-12 12:12:12'), (timestamp_ntz'2020-12-12 12:12:12') + |as t(col0) + |""".stripMargin) exp.write.format("csv").option("header", "true").save(path.getAbsolutePath) @@ -1126,6 +1128,15 @@ abstract class CSVSuite if (timestampType == SQLConf.TimestampTypes.TIMESTAMP_NTZ.toString) { checkAnswer(res, exp) + } else if (SQLConf.get.legacyTimeParserPolicy == LegacyBehaviorPolicy.LEGACY) { + // When legacy parser is enabled, we can't parse the NTZ string to LTZ, and eventually + // infer string type. + val expected = spark.read + .format("csv") + .option("inferSchema", "false") + .option("header", "true") + .load(path.getAbsolutePath) + checkAnswer(res, expected) } else { checkAnswer( res, @@ -2862,13 +2873,12 @@ abstract class CSVSuite test("SPARK-40474: Infer schema for columns with a mix of dates and timestamp") { withTempPath { path => - Seq( - "1765-03-28", + val input = Seq( "1423-11-12T23:41:00", + "1765-03-28", "2016-01-28T20:00:00" - ).toDF() - .repartition(1) - .write.text(path.getAbsolutePath) + ).toDF().repartition(1) + input.write.text(path.getAbsolutePath) if (SQLConf.get.legacyTimeParserPolicy == LegacyBehaviorPolicy.LEGACY) { val options = Map( @@ -2879,12 +2889,7 @@ abstract class CSVSuite .format("csv") .options(options) .load(path.getAbsolutePath) - val expected = Seq( - Row(Timestamp.valueOf("1765-03-28 00:00:00.0")), - Row(Timestamp.valueOf("1423-11-12 23:41:00.0")), - Row(Timestamp.valueOf("2016-01-28 20:00:00.0")) - ) - checkAnswer(df, expected) + checkAnswer(df, input) } else { // When timestampFormat is specified, infer and parse the column as strings val options1 = Map( @@ -2895,12 +2900,7 @@ abstract class CSVSuite .format("csv") .options(options1) .load(path.getAbsolutePath) - val expected1 = Seq( - Row("1765-03-28"), - Row("1423-11-12T23:41:00"), - Row("2016-01-28T20:00:00") - ) - checkAnswer(df1, expected1) + checkAnswer(df1, input) // When timestampFormat is not specified, infer and parse the column as // timestamp type if possible From b98cf95896a2b14e3692f1e7d58660583a2bb175 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 20 Jan 2024 17:51:12 -0800 Subject: [PATCH 182/521] [SPARK-46786][K8S] Fix `MountVolumesFeatureStep` to use `ReadWriteOncePod` instead of `ReadWriteOnce` This PR aims to fix a duplicated volume mounting bug by using `ReadWriteOncePod` instead of `ReadWriteOnce`. This bug fix is based on the stable K8s feature which is available since v1.22. - [KEP-2485: ReadWriteOncePod PersistentVolume AccessMode](https://github.com/kubernetes/enhancements/blob/master/keps/sig-storage/2485-read-write-once-pod-pv-access-mode/README.md) - https://kubernetes.io/docs/concepts/storage/persistent-volumes/#access-modes - v1.22 Alpha - v1.27 Beta - v1.29 Stable For the record, the minimum K8s version of GKE/EKS/AKE is **v1.24** as of today and the latest v1.29 is supported like the following. - [2024.01 (GKE Regular Channel)](https://cloud.google.com/kubernetes-engine/docs/release-schedule) - [2024.02 (AKE GA)](https://learn.microsoft.com/en-us/azure/aks/supported-kubernetes-versions?tabs=azure-cli#aks-kubernetes-release-calendar) This is a bug fix. Pass the CIs with the existing PV-related tests. No. Closes #44817 from dongjoon-hyun/SPARK-46786. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 45ec74415a4a89851968941b80c490e37ee88daf) Signed-off-by: Dongjoon Hyun --- .../spark/deploy/k8s/features/MountVolumesFeatureStep.scala | 2 +- .../apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala index 78dd6ec21ed34..cbbbb9c0bdf57 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala @@ -126,5 +126,5 @@ private[spark] object MountVolumesFeatureStep { val PVC_ON_DEMAND = "OnDemand" val PVC = "PersistentVolumeClaim" val PVC_POSTFIX = "-pvc" - val PVC_ACCESS_MODE = "ReadWriteOnce" + val PVC_ACCESS_MODE = "ReadWriteOncePod" } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala index 1d373f3f8066e..f8e76012638cf 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala @@ -166,7 +166,7 @@ private[spark] trait PVTestsSuite { k8sSuite: KubernetesSuite => } } - test("PVs with local hostpath and storageClass on statefulsets", k8sTestTag, MinikubeTag) { + ignore("PVs with local hostpath and storageClass on statefulsets", k8sTestTag, MinikubeTag) { sparkAppConf .set(s"spark.kubernetes.driver.volumes.persistentVolumeClaim.data.mount.path", CONTAINER_MOUNT_PATH) From 687c2979959f13ac2c32e596f745f79284144735 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 20 Jan 2024 18:53:21 -0800 Subject: [PATCH 183/521] [SPARK-44495][INFRA][K8S][3.5] Use the latest minikube in K8s IT ### What changes were proposed in this pull request? This is a backport of #44813 . This PR aims to recover GitHub Action K8s IT to use the latest Minikube and to make it sure that Apache Spark K8s module are tested with all Minikubes without any issues. **BEFORE** - Minikube: v1.30.1 - K8s: v1.26.3 **AFTER** - Minikube: v1.32.0 - K8s: v1.28.3 ### Why are the changes needed? - Previously, it was pinned due to the failure. - After this PR, we will track the latest Minikube and K8s version always. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44819 from dongjoon-hyun/SPARK-44495-3.5. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/build_and_test.yml | 6 ++---- .../k8s/integrationtest/KubernetesTestComponents.scala | 2 ++ .../spark/deploy/k8s/integrationtest/PVTestsSuite.scala | 4 +++- .../deploy/k8s/integrationtest/VolcanoTestsSuite.scala | 4 ++-- 4 files changed, 9 insertions(+), 7 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index b0b72a0d1a689..ad8685754b316 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -1016,9 +1016,7 @@ jobs: - name: start minikube run: | # See more in "Installation" https://minikube.sigs.k8s.io/docs/start/ - # curl -LO https://storage.googleapis.com/minikube/releases/latest/minikube-linux-amd64 - # TODO(SPARK-44495): Resume to use the latest minikube for k8s-integration-tests. - curl -LO https://storage.googleapis.com/minikube/releases/v1.30.1/minikube-linux-amd64 + curl -LO https://storage.googleapis.com/minikube/releases/latest/minikube-linux-amd64 sudo install minikube-linux-amd64 /usr/local/bin/minikube # Github Action limit cpu:2, memory: 6947MB, limit to 2U6G for better resource statistic minikube start --cpus 2 --memory 6144 @@ -1036,7 +1034,7 @@ jobs: kubectl create clusterrolebinding serviceaccounts-cluster-admin --clusterrole=cluster-admin --group=system:serviceaccounts || true kubectl apply -f https://raw.githubusercontent.com/volcano-sh/volcano/v1.7.0/installer/volcano-development.yaml || true eval $(minikube docker-env) - build/sbt -Psparkr -Pkubernetes -Pvolcano -Pkubernetes-integration-tests -Dspark.kubernetes.test.driverRequestCores=0.5 -Dspark.kubernetes.test.executorRequestCores=0.2 -Dspark.kubernetes.test.volcanoMaxConcurrencyJobNum=1 -Dtest.exclude.tags=local "kubernetes-integration-tests/test" + build/sbt -Psparkr -Pkubernetes -Pvolcano -Pkubernetes-integration-tests -Dspark.kubernetes.test.volcanoMaxConcurrencyJobNum=1 -Dtest.exclude.tags=local "kubernetes-integration-tests/test" - name: Upload Spark on K8S integration tests log files if: failure() uses: actions/upload-artifact@v3 diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala index 4aba11bdb9d8f..4ebf44ce9a4bc 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala @@ -75,6 +75,8 @@ private[spark] class KubernetesTestComponents(val kubernetesClient: KubernetesCl .set(UI_ENABLED.key, "true") .set("spark.kubernetes.submission.waitAppCompletion", "false") .set("spark.kubernetes.authenticate.driver.serviceAccountName", serviceAccountName) + .set("spark.kubernetes.driver.request.cores", "0.2") + .set("spark.kubernetes.executor.request.cores", "0.2") } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala index f8e76012638cf..a699ef674cdcd 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala @@ -24,6 +24,7 @@ import org.scalatest.concurrent.{Eventually, PatienceConfiguration} import org.scalatest.time.{Milliseconds, Span} import org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite._ +import org.apache.spark.deploy.k8s.integrationtest.backend.minikube.MinikubeTestBackend private[spark] trait PVTestsSuite { k8sSuite: KubernetesSuite => import PVTestsSuite._ @@ -54,6 +55,7 @@ private[spark] trait PVTestsSuite { k8sSuite: KubernetesSuite => setupLocalStorageClass() + val hostname = if (testBackend == MinikubeTestBackend) "minikube" else "docker-desktop" val pvBuilder = new PersistentVolumeBuilder() .withKind("PersistentVolume") .withApiVersion("v1") @@ -72,7 +74,7 @@ private[spark] trait PVTestsSuite { k8sSuite: KubernetesSuite => .withMatchExpressions(new NodeSelectorRequirementBuilder() .withKey("kubernetes.io/hostname") .withOperator("In") - .withValues("minikube", "m01", "docker-for-desktop", "docker-desktop") + .withValues(hostname) .build()).build()) .endRequired() .endNodeAffinity() diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala index 06d6f7dc100f3..e7143e32db61e 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala @@ -496,8 +496,8 @@ private[spark] object VolcanoTestsSuite extends SparkFunSuite { val DRIVER_PG_TEMPLATE_MEMORY_3G = new File( getClass.getResource("/volcano/driver-podgroup-template-memory-3g.yml").getFile ).getAbsolutePath - val DRIVER_REQUEST_CORES = sys.props.get(CONFIG_DRIVER_REQUEST_CORES).getOrElse("1") - val EXECUTOR_REQUEST_CORES = sys.props.get(CONFIG_EXECUTOR_REQUEST_CORES).getOrElse("1") + val DRIVER_REQUEST_CORES = sys.props.get(CONFIG_DRIVER_REQUEST_CORES).getOrElse("0.2") + val EXECUTOR_REQUEST_CORES = sys.props.get(CONFIG_EXECUTOR_REQUEST_CORES).getOrElse("0.2") val VOLCANO_MAX_JOB_NUM = sys.props.get(CONFIG_KEY_VOLCANO_MAX_JOB_NUM).getOrElse("2") val TEMP_DIR = "/tmp/" } From 04d32493fde779021871c88709dbbae32f18e512 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 22 Jan 2024 16:19:39 +0800 Subject: [PATCH 184/521] [SPARK-46789][K8S][TESTS] Add `VolumeSuite` to K8s IT ### What changes were proposed in this pull request? This PR aims to add `VolumeSuite` to K8s IT. ### Why are the changes needed? To improve the test coverage on various K8s volume use cases. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44827 from dongjoon-hyun/SPARK-46789. Authored-by: Dongjoon Hyun Signed-off-by: Kent Yao --- .../k8s/integrationtest/KubernetesSuite.scala | 4 +- .../k8s/integrationtest/VolumeSuite.scala | 173 ++++++++++++++++++ 2 files changed, 175 insertions(+), 2 deletions(-) create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolumeSuite.scala diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index f52af87a745ca..54ef1f6cee30d 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -45,8 +45,8 @@ import org.apache.spark.internal.config._ class KubernetesSuite extends SparkFunSuite with BeforeAndAfterAll with BeforeAndAfter with BasicTestsSuite with SparkConfPropagateSuite with SecretsTestsSuite with PythonTestsSuite with ClientModeTestsSuite with PodTemplateSuite - with PVTestsSuite with DepsTestsSuite with DecommissionSuite with RTestsSuite with Logging - with Eventually with Matchers { + with VolumeSuite with PVTestsSuite with DepsTestsSuite with DecommissionSuite with RTestsSuite + with Logging with Eventually with Matchers { import KubernetesSuite._ diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolumeSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolumeSuite.scala new file mode 100644 index 0000000000000..c57e4b4578d6c --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolumeSuite.scala @@ -0,0 +1,173 @@ +/* + * 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.deploy.k8s.integrationtest + +import scala.jdk.CollectionConverters._ + +import io.fabric8.kubernetes.api.model._ +import org.scalatest.concurrent.PatienceConfiguration +import org.scalatest.time.{Seconds, Span} + +import org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite._ +import org.apache.spark.deploy.k8s.integrationtest.backend.minikube.MinikubeTestBackend + +private[spark] trait VolumeSuite { k8sSuite: KubernetesSuite => + val IGNORE = Some((Some(PatienceConfiguration.Interval(Span(0, Seconds))), None)) + + private def checkDisk(pod: Pod, path: String, expected: String) = { + eventually(PatienceConfiguration.Timeout(Span(10, Seconds)), INTERVAL) { + implicit val podName: String = pod.getMetadata.getName + implicit val components: KubernetesTestComponents = kubernetesTestComponents + assert(Utils.executeCommand("df", path).contains(expected)) + } + } + + test("A driver-only Spark job with a tmpfs-backed localDir volume", k8sTestTag) { + sparkAppConf + .set("spark.kubernetes.driver.master", "local[10]") + .set("spark.kubernetes.local.dirs.tmpfs", "true") + runSparkApplicationAndVerifyCompletion( + containerLocalSparkDistroExamplesJar, + SPARK_PI_MAIN_CLASS, + Seq("local[10]", "Pi is roughly 3"), + Seq(), + Array.empty[String], + driverPodChecker = (driverPod: Pod) => { + doBasicDriverPodCheck(driverPod) + val path = driverPod.getSpec.getContainers.get(0).getEnv.asScala + .filter(_.getName == "SPARK_LOCAL_DIRS").map(_.getValue).head + checkDisk(driverPod, path, "tmpfs") + }, + _ => (), + isJVM = true, + executorPatience = IGNORE) + } + + test("A driver-only Spark job with a tmpfs-backed emptyDir data volume", k8sTestTag) { + sparkAppConf + .set("spark.kubernetes.driver.master", "local[10]") + .set("spark.kubernetes.driver.volumes.emptyDir.data.mount.path", "/data") + .set("spark.kubernetes.driver.volumes.emptyDir.data.options.medium", "Memory") + .set("spark.kubernetes.driver.volumes.emptyDir.data.options.sizeLimit", "1G") + runSparkApplicationAndVerifyCompletion( + containerLocalSparkDistroExamplesJar, + SPARK_PI_MAIN_CLASS, + Seq("local[10]", "Pi is roughly 3"), + Seq(), + Array.empty[String], + driverPodChecker = (driverPod: Pod) => { + doBasicDriverPodCheck(driverPod) + checkDisk(driverPod, "/data", "tmpfs") + }, + _ => (), + isJVM = true, + executorPatience = IGNORE) + } + + test("A driver-only Spark job with a disk-backed emptyDir volume", k8sTestTag) { + sparkAppConf + .set("spark.kubernetes.driver.master", "local[10]") + .set("spark.kubernetes.driver.volumes.emptyDir.data.mount.path", "/data") + .set("spark.kubernetes.driver.volumes.emptyDir.data.mount.sizeLimit", "1G") + runSparkApplicationAndVerifyCompletion( + containerLocalSparkDistroExamplesJar, + SPARK_PI_MAIN_CLASS, + Seq("local[10]", "Pi is roughly 3"), + Seq(), + Array.empty[String], + driverPodChecker = (driverPod: Pod) => { + doBasicDriverPodCheck(driverPod) + checkDisk(driverPod, "/data", "/dev/") + }, + _ => (), + isJVM = true, + executorPatience = IGNORE) + } + + test("A driver-only Spark job with an OnDemand PVC volume", k8sTestTag) { + val storageClassName = if (testBackend == MinikubeTestBackend) "standard" else "hostpath" + val DRIVER_PREFIX = "spark.kubernetes.driver.volumes.persistentVolumeClaim" + sparkAppConf + .set("spark.kubernetes.driver.master", "local[10]") + .set(s"$DRIVER_PREFIX.data.options.claimName", "OnDemand") + .set(s"$DRIVER_PREFIX.data.options.storageClass", storageClassName) + .set(s"$DRIVER_PREFIX.data.options.sizeLimit", "1Gi") + .set(s"$DRIVER_PREFIX.data.mount.path", "/data") + .set(s"$DRIVER_PREFIX.data.mount.readOnly", "false") + runSparkApplicationAndVerifyCompletion( + containerLocalSparkDistroExamplesJar, + SPARK_PI_MAIN_CLASS, + Seq("local[10]", "Pi is roughly 3"), + Seq(), + Array.empty[String], + driverPodChecker = (driverPod: Pod) => { + doBasicDriverPodCheck(driverPod) + checkDisk(driverPod, "/data", "/dev/") + }, + _ => (), + isJVM = true, + executorPatience = IGNORE) + } + + test("A Spark job with tmpfs-backed localDir volumes", k8sTestTag) { + sparkAppConf + .set("spark.kubernetes.local.dirs.tmpfs", "true") + runSparkApplicationAndVerifyCompletion( + containerLocalSparkDistroExamplesJar, + SPARK_PI_MAIN_CLASS, + Seq("Pi is roughly 3"), + Seq(), + Array.empty[String], + driverPodChecker = (driverPod: Pod) => { + doBasicDriverPodCheck(driverPod) + val path = driverPod.getSpec.getContainers.get(0).getEnv.asScala + .filter(_.getName == "SPARK_LOCAL_DIRS").map(_.getValue).head + checkDisk(driverPod, path, "tmpfs") + }, + executorPodChecker = (executorPod: Pod) => { + doBasicExecutorPodCheck(executorPod) + val path = executorPod.getSpec.getContainers.get(0).getEnv.asScala + .filter(_.getName == "SPARK_LOCAL_DIRS").map(_.getValue).head + checkDisk(executorPod, path, "tmpfs") + }, + isJVM = true) + } + + test("A Spark job with two executors with OnDemand PVC volumes", k8sTestTag) { + val storageClassName = if (testBackend == MinikubeTestBackend) "standard" else "hostpath" + val EXECUTOR_PREFIX = "spark.kubernetes.executor.volumes.persistentVolumeClaim" + sparkAppConf + .set("spark.executor.instances", "2") + .set(s"$EXECUTOR_PREFIX.data.options.claimName", "OnDemand") + .set(s"$EXECUTOR_PREFIX.data.options.storageClass", storageClassName) + .set(s"$EXECUTOR_PREFIX.data.options.sizeLimit", "1Gi") + .set(s"$EXECUTOR_PREFIX.data.mount.path", "/data") + .set(s"$EXECUTOR_PREFIX.data.mount.readOnly", "false") + runSparkApplicationAndVerifyCompletion( + containerLocalSparkDistroExamplesJar, + SPARK_PI_MAIN_CLASS, + Seq("Pi is roughly 3"), + Seq(), + Array.empty[String], + _ => (), + executorPodChecker = (executorPod: Pod) => { + doBasicExecutorPodCheck(executorPod) + checkDisk(executorPod, "/data", "/dev/") + }, + isJVM = true) + } +} From 68d9f353300ed7de0b47c26cb30236bada896d25 Mon Sep 17 00:00:00 2001 From: Bruce Robbins Date: Mon, 22 Jan 2024 11:09:01 -0800 Subject: [PATCH 185/521] [SPARK-46779][SQL] `InMemoryRelation` instances of the same cached plan should be semantically equivalent When canonicalizing `output` in `InMemoryRelation`, use `output` itself as the schema for determining the ordinals, rather than `cachedPlan.output`. `InMemoryRelation.output` and `InMemoryRelation.cachedPlan.output` don't necessarily use the same exprIds. E.g.: ``` +- InMemoryRelation [c1#340, c2#341], StorageLevel(disk, memory, deserialized, 1 replicas) +- LocalTableScan [c1#254, c2#255] ``` Because of this, `InMemoryRelation` will sometimes fail to fully canonicalize, resulting in cases where two semantically equivalent `InMemoryRelation` instances appear to be semantically nonequivalent. Example: ``` create or replace temp view data(c1, c2) as values (1, 2), (1, 3), (3, 7), (4, 5); cache table data; select c1, (select count(*) from data d1 where d1.c1 = d2.c1), count(c2) from data d2 group by all; ``` If plan change validation checking is on (i.e., `spark.sql.planChangeValidation=true`), the failure is: ``` [PLAN_VALIDATION_FAILED_RULE_EXECUTOR] The input plan of org.apache.spark.sql.internal.BaseSessionStateBuilder$$anon$2 is invalid: Aggregate: Aggregate [c1#78, scalar-subquery#77 [c1#78]], [c1#78, scalar-subquery#77 [c1#78] AS scalarsubquery(c1)#90L, count(c2#79) AS count(c2)#83L] ... is not a valid aggregate expression: [SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION] The correlated scalar subquery '"scalarsubquery(c1)"' is neither present in GROUP BY, nor in an aggregate function. ``` If plan change validation checking is off, the failure is more mysterious: ``` [INTERNAL_ERROR] Couldn't find count(1)#163L in [c1#78,_groupingexpression#149L,count(1)#82L] SQLSTATE: XX000 org.apache.spark.SparkException: [INTERNAL_ERROR] Couldn't find count(1)#163L in [c1#78,_groupingexpression#149L,count(1)#82L] SQLSTATE: XX000 ``` If you remove the cache command, the query succeeds. The above failures happen because the subquery in the aggregate expressions and the subquery in the grouping expressions seem semantically nonequivalent since the `InMemoryRelation` in one of the subquery plans failed to completely canonicalize. In `CacheManager#useCachedData`, two lookups for the same cached plan may create `InMemoryRelation` instances that have different exprIds in `output`. That's because the plan fragments used as lookup keys may have been deduplicated by `DeduplicateRelations`, and thus have different exprIds in their respective output schemas. When `CacheManager#useCachedData` creates an `InMemoryRelation` instance, it borrows the output schema of the plan fragment used as the lookup key. The failure to fully canonicalize has other effects. For example, this query fails to reuse the exchange: ``` create or replace temp view data(c1, c2) as values (1, 2), (1, 3), (2, 4), (3, 7), (7, 22); cache table data; set spark.sql.autoBroadcastJoinThreshold=-1; set spark.sql.adaptive.enabled=false; select * from data l join data r on l.c1 = r.c1; ``` No. New tests. No. Closes #44806 from bersprockets/plan_validation_issue. Authored-by: Bruce Robbins Signed-off-by: Dongjoon Hyun (cherry picked from commit b80e8cb4552268b771fc099457b9186807081c4a) Signed-off-by: Dongjoon Hyun --- .../sql/execution/columnar/InMemoryRelation.scala | 2 +- .../spark/sql/DataFrameAggregateSuite.scala | 15 +++++++++++++++ .../columnar/InMemoryRelationSuite.scala | 7 +++++++ 3 files changed, 23 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 65f7835b42cf8..5bab8e53eb163 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -405,7 +405,7 @@ case class InMemoryRelation( } override def doCanonicalize(): logical.LogicalPlan = - copy(output = output.map(QueryPlan.normalizeExpressions(_, cachedPlan.output)), + copy(output = output.map(QueryPlan.normalizeExpressions(_, output)), cacheBuilder, outputOrdering) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index d78771a8f19bc..631fcd8c0d87d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -2102,6 +2102,21 @@ class DataFrameAggregateSuite extends QueryTest Seq(Row(1)) ) } + + test("SPARK-46779: Group by subquery with a cached relation") { + withTempView("data") { + sql( + """create or replace temp view data(c1, c2) as values + |(1, 2), + |(1, 3), + |(3, 7)""".stripMargin) + sql("cache table data") + val df = sql( + """select c1, (select count(*) from data d1 where d1.c1 = d2.c1), count(c2) + |from data d2 group by all""".stripMargin) + checkAnswer(df, Row(1, 2, 2) :: Row(3, 1, 1) :: Nil) + } + } } case class B(c: Option[Double]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryRelationSuite.scala index 72b3a4bc1095a..a5c5ec40af6fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryRelationSuite.scala @@ -34,4 +34,11 @@ class InMemoryRelationSuite extends SparkFunSuite with SharedSparkSessionBase { assert(!relationCachedPlan.eq(clonedCachedPlan)) assert(relationCachedPlan === clonedCachedPlan) } + + test("SPARK-46779: InMemoryRelations with the same cached plan are semantically equivalent") { + val d = spark.range(1) + val r1 = InMemoryRelation(StorageLevel.MEMORY_ONLY, d.queryExecution, None) + val r2 = r1.withOutput(r1.output.map(_.newInstance())) + assert(r1.sameResult(r2)) + } } From a6869b25fb9a7ac0e7e5015d342435e5c1b5f044 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Mon, 22 Jan 2024 17:06:59 -0800 Subject: [PATCH 186/521] [SPARK-46801][PYTHON][TESTS] Do not treat exit code 5 as a test failure in Python testing script ### What changes were proposed in this pull request? This PR proposes to avoid treating the exit code 5 as a test failure in Python testing script. ### Why are the changes needed? ``` ... ======================================================================== Running PySpark tests ======================================================================== Running PySpark tests. Output is in /__w/spark/spark/python/unit-tests.log Will test against the following Python executables: ['python3.12'] Will test the following Python modules: ['pyspark-core', 'pyspark-streaming', 'pyspark-errors'] python3.12 python_implementation is CPython python3.12 version is: Python 3.12.1 Starting test(python3.12): pyspark.streaming.tests.test_context (temp output: /__w/spark/spark/python/target/8674ed86-36bd-47d1-863b-abb0405557f6/python3.12__pyspark.streaming.tests.test_context__umu69c3v.log) Finished test(python3.12): pyspark.streaming.tests.test_context (12s) Starting test(python3.12): pyspark.streaming.tests.test_dstream (temp output: /__w/spark/spark/python/target/847eb56b-3c5f-49ab-8a83-3326bb96bc5d/python3.12__pyspark.streaming.tests.test_dstream__rorhk0lc.log) Finished test(python3.12): pyspark.streaming.tests.test_dstream (102s) Starting test(python3.12): pyspark.streaming.tests.test_kinesis (temp output: /__w/spark/spark/python/target/78f23c83-c24d-4fa1-abbd-edb90f48dff1/python3.12__pyspark.streaming.tests.test_kinesis__q5l1pv0h.log) test_kinesis_stream (pyspark.streaming.tests.test_kinesis.KinesisStreamTests.test_kinesis_stream) ... skipped "Skipping all Kinesis Python tests as environmental variable 'ENABLE_KINESIS_TESTS' was not set." test_kinesis_stream_api (pyspark.streaming.tests.test_kinesis.KinesisStreamTests.test_kinesis_stream_api) ... skipped "Skipping all Kinesis Python tests as environmental variable 'ENABLE_KINESIS_TESTS' was not set." ---------------------------------------------------------------------- Ran 0 tests in 0.000s NO TESTS RAN (skipped=2) Had test failures in pyspark.streaming.tests.test_kinesis with python3.12; see logs. Error: running /__w/spark/spark/python/run-tests --modules=pyspark-core,pyspark-streaming,pyspark-errors --parallelism=1 --python-executables=python3.12 ; received return code 255 Error: Process completed with exit code 19. ``` Scheduled job fails because of exit 5, see https://github.com/pytest-dev/pytest/issues/2393. This isn't a test failure. ### Does this PR introduce _any_ user-facing change? No, test-only. ### How was this patch tested? Manually tested. ### Was this patch authored or co-authored using generative AI tooling? No, Closes #44841 from HyukjinKwon/SPARK-46801. Authored-by: Hyukjin Kwon Signed-off-by: Dongjoon Hyun (cherry picked from commit 52b62921cadb05da5b1183f979edf7d608256f2e) Signed-off-by: Dongjoon Hyun --- python/run-tests.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/run-tests.py b/python/run-tests.py index 19e39c822cbb4..b9031765d9437 100755 --- a/python/run-tests.py +++ b/python/run-tests.py @@ -147,8 +147,8 @@ def run_individual_python_test(target_dir, test_name, pyspark_python, keep_test_ # this code is invoked from a thread other than the main thread. os._exit(1) duration = time.time() - start_time - # Exit on the first failure. - if retcode != 0: + # Exit on the first failure but exclude the code 5 for no test ran, see SPARK-46801. + if retcode != 0 and retcode != 5: try: with FAILURE_REPORTING_LOCK: with open(LOG_FILE, 'ab') as log_file: From 6403a84b6854214a4ed7d5c0c800e877e0748964 Mon Sep 17 00:00:00 2001 From: jackylee-ch Date: Tue, 23 Jan 2024 16:10:37 +0800 Subject: [PATCH 187/521] [SPARK-46590][SQL] Fix coalesce failed with unexpected partition indeces ### What changes were proposed in this pull request? As outlined in JIRA issue [SPARK-46590](https://issues.apache.org/jira/browse/SPARK-46590), when a broadcast join follows a union within the same stage, the [collectCoalesceGroups](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CoalesceShufflePartitions.scala#L144) method will indiscriminately traverse all sub-plans, aggregating them into a single group, which is not expected. ### Why are the changes needed? In fact, for broadcastjoin, we do not expect broadcast exchange has same partition number. Therefore, we can safely disregard the broadcast join and continue traversing the subplan. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Newly added unit test. It would fail without this pr. ### Was this patch authored or co-authored using generative AI tooling? No Closes #44661 from jackylee-ch/fix_coalesce_problem_with_broadcastjoin_and_union. Authored-by: jackylee-ch Signed-off-by: Wenchen Fan (cherry picked from commit de0c4ad3947f1188f02aaa612df8278d1c7c3ce5) Signed-off-by: Wenchen Fan --- .../adaptive/CoalesceShufflePartitions.scala | 10 ++- .../adaptive/ShufflePartitionsUtil.scala | 6 +- .../CoalesceShufflePartitionsSuite.scala | 61 +++++++++++++++++++ .../ShufflePartitionsUtilSuite.scala | 31 +++++----- 4 files changed, 86 insertions(+), 22 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CoalesceShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CoalesceShufflePartitions.scala index 34399001c726f..26e5ac649dbb1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CoalesceShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CoalesceShufflePartitions.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.physical.SinglePartition import org.apache.spark.sql.execution.{ShufflePartitionSpec, SparkPlan, UnaryExecNode, UnionExec} import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, REBALANCE_PARTITIONS_BY_COL, REBALANCE_PARTITIONS_BY_NONE, REPARTITION_BY_COL, ShuffleExchangeLike, ShuffleOrigin} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, CartesianProductExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Utils @@ -146,13 +147,16 @@ case class CoalesceShufflePartitions(session: SparkSession) extends AQEShuffleRe Seq(collectShuffleStageInfos(r)) case unary: UnaryExecNode => collectCoalesceGroups(unary.child) case union: UnionExec => union.children.flatMap(collectCoalesceGroups) - // If not all leaf nodes are exchange query stages, it's not safe to reduce the number of - // shuffle partitions, because we may break the assumption that all children of a spark plan - // have same number of output partitions. + case join: CartesianProductExec => join.children.flatMap(collectCoalesceGroups) // Note that, `BroadcastQueryStageExec` is a valid case: // If a join has been optimized from shuffled join to broadcast join, then the one side is // `BroadcastQueryStageExec` and other side is `ShuffleQueryStageExec`. It can coalesce the // shuffle side as we do not expect broadcast exchange has same partition number. + case join: BroadcastHashJoinExec => join.children.flatMap(collectCoalesceGroups) + case join: BroadcastNestedLoopJoinExec => join.children.flatMap(collectCoalesceGroups) + // If not all leaf nodes are exchange query stages, it's not safe to reduce the number of + // shuffle partitions, because we may break the assumption that all children of a spark plan + // have same number of output partitions. case p if p.collectLeaves().forall(_.isInstanceOf[ExchangeQueryStageExec]) => val shuffleStages = collectShuffleStageInfos(p) // ShuffleExchanges introduced by repartition do not support partition number change. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala index dbed66683b017..9370b3d8d1d74 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsUtil.scala @@ -128,8 +128,10 @@ object ShufflePartitionsUtil extends Logging { // There should be no unexpected partition specs and the start indices should be identical // across all different shuffles. - assert(partitionIndicesSeq.distinct.length == 1 && partitionIndicesSeq.head.forall(_ >= 0), - s"Invalid shuffle partition specs: $inputPartitionSpecs") + if (partitionIndicesSeq.distinct.length > 1 || partitionIndicesSeq.head.exists(_ < 0)) { + logWarning(s"Could not apply partition coalescing because of unexpected partition indices.") + return Seq.empty + } // The indices may look like [0, 1, 2, 2, 2, 3, 4, 4, 5], and the repeated `2` and `4` mean // skewed partitions. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala index 24a98dd83f33a..e11191da6a952 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala @@ -310,6 +310,67 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite { } } + test("SPARK-46590 adaptive query execution works correctly with broadcast join and union") { + val test: SparkSession => Unit = { spark: SparkSession => + import spark.implicits._ + spark.conf.set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "1KB") + spark.conf.set(SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key, "10KB") + spark.conf.set(SQLConf.SKEW_JOIN_SKEWED_PARTITION_FACTOR, 2.0) + val df00 = spark.range(0, 1000, 2) + .selectExpr("id as key", "id as value") + .union(Seq.fill(100000)((600, 600)).toDF("key", "value")) + val df01 = spark.range(0, 1000, 3) + .selectExpr("id as key", "id as value") + val df10 = spark.range(0, 1000, 5) + .selectExpr("id as key", "id as value") + .union(Seq.fill(500000)((600, 600)).toDF("key", "value")) + val df11 = spark.range(0, 1000, 7) + .selectExpr("id as key", "id as value") + val df20 = spark.range(0, 10).selectExpr("id as key", "id as value") + + df20.join(df00.join(df01, Array("key", "value"), "left_outer") + .union(df10.join(df11, Array("key", "value"), "left_outer"))) + .write + .format("noop") + .mode("overwrite") + .save() + } + withSparkSession(test, 12000, None) + } + + test("SPARK-46590 adaptive query execution works correctly with cartesian join and union") { + val test: SparkSession => Unit = { spark: SparkSession => + import spark.implicits._ + spark.conf.set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") + spark.conf.set(SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key, "100B") + spark.conf.set(SQLConf.SKEW_JOIN_SKEWED_PARTITION_FACTOR, 2.0) + val df00 = spark.range(0, 10, 2) + .selectExpr("id as key", "id as value") + .union(Seq.fill(1000)((600, 600)).toDF("key", "value")) + val df01 = spark.range(0, 10, 3) + .selectExpr("id as key", "id as value") + val df10 = spark.range(0, 10, 5) + .selectExpr("id as key", "id as value") + .union(Seq.fill(5000)((600, 600)).toDF("key", "value")) + val df11 = spark.range(0, 10, 7) + .selectExpr("id as key", "id as value") + val df20 = spark.range(0, 10) + .selectExpr("id as key", "id as value") + .union(Seq.fill(1000)((11, 11)).toDF("key", "value")) + val df21 = spark.range(0, 10) + .selectExpr("id as key", "id as value") + + df20.join(df21.hint("shuffle_hash"), Array("key", "value"), "left_outer") + .join(df00.join(df01.hint("shuffle_hash"), Array("key", "value"), "left_outer") + .union(df10.join(df11.hint("shuffle_hash"), Array("key", "value"), "left_outer"))) + .write + .format("noop") + .mode("overwrite") + .save() + } + withSparkSession(test, 100, None) + } + test("SPARK-24705 adaptive query execution works correctly when exchange reuse enabled") { val test: SparkSession => Unit = { spark: SparkSession => spark.sql("SET spark.sql.exchange.reuse=true") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsUtilSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsUtilSuite.scala index da05373125d31..f8b7964368476 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsUtilSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsUtilSuite.scala @@ -567,14 +567,13 @@ class ShufflePartitionsUtilSuite extends SparkFunSuite with LocalSparkContext { } { - // Assertion error if shuffle partition specs contain `CoalescedShuffleSpec` that has - // `end` - `start` > 1. + // If shuffle partition specs contain `CoalescedShuffleSpec` that has + // `end` - `start` > 1, return empty result. val bytesByPartitionId1 = Array[Long](10, 10, 10, 10, 10) val bytesByPartitionId2 = Array[Long](10, 10, 10, 10, 10) val specs1 = Seq(CoalescedPartitionSpec(0, 1), CoalescedPartitionSpec(1, 5)) val specs2 = specs1 - intercept[AssertionError] { - ShufflePartitionsUtil.coalescePartitions( + val coalesced = ShufflePartitionsUtil.coalescePartitions( Array( Some(new MapOutputStatistics(0, bytesByPartitionId1)), Some(new MapOutputStatistics(1, bytesByPartitionId2))), @@ -582,17 +581,16 @@ class ShufflePartitionsUtilSuite extends SparkFunSuite with LocalSparkContext { Some(specs1), Some(specs2)), targetSize, 1, 0) - } + assert(coalesced.isEmpty) } { - // Assertion error if shuffle partition specs contain `PartialMapperShuffleSpec`. + // If shuffle partition specs contain `PartialMapperShuffleSpec`, return empty result. val bytesByPartitionId1 = Array[Long](10, 10, 10, 10, 10) val bytesByPartitionId2 = Array[Long](10, 10, 10, 10, 10) val specs1 = Seq(CoalescedPartitionSpec(0, 1), PartialMapperPartitionSpec(1, 0, 1)) val specs2 = specs1 - intercept[AssertionError] { - ShufflePartitionsUtil.coalescePartitions( + val coalesced = ShufflePartitionsUtil.coalescePartitions( Array( Some(new MapOutputStatistics(0, bytesByPartitionId1)), Some(new MapOutputStatistics(1, bytesByPartitionId2))), @@ -600,18 +598,17 @@ class ShufflePartitionsUtilSuite extends SparkFunSuite with LocalSparkContext { Some(specs1), Some(specs2)), targetSize, 1, 0) - } + assert(coalesced.isEmpty) } { - // Assertion error if partition specs of different shuffles have different lengths. + // If partition specs of different shuffles have different lengths, return empty result. val bytesByPartitionId1 = Array[Long](10, 10, 10, 10, 10) val bytesByPartitionId2 = Array[Long](10, 10, 10, 10, 10) val specs1 = Seq.tabulate(4)(i => CoalescedPartitionSpec(i, i + 1)) ++ Seq.tabulate(2)(i => PartialReducerPartitionSpec(4, i, i + 1, 10L)) val specs2 = Seq.tabulate(5)(i => CoalescedPartitionSpec(i, i + 1)) - intercept[AssertionError] { - ShufflePartitionsUtil.coalescePartitions( + val coalesced = ShufflePartitionsUtil.coalescePartitions( Array( Some(new MapOutputStatistics(0, bytesByPartitionId1)), Some(new MapOutputStatistics(1, bytesByPartitionId2))), @@ -619,11 +616,12 @@ class ShufflePartitionsUtilSuite extends SparkFunSuite with LocalSparkContext { Some(specs1), Some(specs2)), targetSize, 1, 0) - } + assert(coalesced.isEmpty) } { - // Assertion error if start indices of partition specs are not identical among all shuffles. + // If start indices of partition specs are not identical among all shuffles, + // return empty result. val bytesByPartitionId1 = Array[Long](10, 10, 10, 10, 10) val bytesByPartitionId2 = Array[Long](10, 10, 10, 10, 10) val specs1 = Seq.tabulate(4)(i => CoalescedPartitionSpec(i, i + 1)) ++ @@ -631,8 +629,7 @@ class ShufflePartitionsUtilSuite extends SparkFunSuite with LocalSparkContext { val specs2 = Seq.tabulate(2)(i => CoalescedPartitionSpec(i, i + 1)) ++ Seq.tabulate(2)(i => PartialReducerPartitionSpec(2, i, i + 1, 10L)) ++ Seq.tabulate(2)(i => CoalescedPartitionSpec(i + 3, i + 4)) - intercept[AssertionError] { - ShufflePartitionsUtil.coalescePartitions( + val coalesced = ShufflePartitionsUtil.coalescePartitions( Array( Some(new MapOutputStatistics(0, bytesByPartitionId1)), Some(new MapOutputStatistics(1, bytesByPartitionId2))), @@ -640,7 +637,7 @@ class ShufflePartitionsUtilSuite extends SparkFunSuite with LocalSparkContext { Some(specs1), Some(specs2)), targetSize, 1, 0) - } + assert(coalesced.isEmpty) } { From a559ff7bb9d3c34429f80760741f1bbd40696f32 Mon Sep 17 00:00:00 2001 From: Nikhil Sheoran <125331115+nikhilsheoran-db@users.noreply.github.com> Date: Tue, 23 Jan 2024 17:15:30 +0800 Subject: [PATCH 188/521] [SPARK-46763] Fix assertion failure in ReplaceDeduplicateWithAggregate for duplicate attributes ### What changes were proposed in this pull request? - Updated the `ReplaceDeduplicateWithAggregate` implementation to reuse aliases generated for an attribute. - Added a unit test to ensure scenarios with duplicate non-grouping keys are correctly optimized. ### Why are the changes needed? - `ReplaceDeduplicateWithAggregate` replaces `Deduplicate` with an `Aggregate` operator with grouping expressions for the deduplication keys and aggregate expressions for the non-grouping keys (to preserve the output schema and keep the non-grouping columns). - For non-grouping key `a#X`, it generates an aggregate expression of the form `first(a#X, false) AS a#Y` - In case the non-grouping keys have a repeated attribute (with the same name and exprId), the existing logic would generate two different aggregate expressions both having two different exprId. - This then leads to duplicate rewrite attributes error (in `transformUpWithNewOutput`) when transforming the remaining tree. - For example, for the query ``` Project [a#0, b#1] +- Deduplicate [b#1] +- Project [a#0, a#0, b#1] +- LocalRelation , [a#0, b#1] ``` the existing logic would transform it to ``` Project [a#3, b#1] +- Aggregate [b#1], [first(a#0, false) AS a#3, first(a#0, false) AS a#5, b#1] +- Project [a#0, a#0, b#1] +- LocalRelation , [a#0, b#1] ``` with the aggregate mapping having two entries `a#0 -> a#3, a#0 -> a#5`. The correct transformation would be ``` Project [a#3, b#1] +- Aggregate [b#1], [first(a#0, false) AS a#3, first(a#0, false) AS a#3, b#1] +- Project [a#0, a#0, b#1] +- LocalRelation , [a#0, b#1] ``` with the aggregate mapping having only one entry `a#0 -> a#3`. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added a unit test in `ResolveOperatorSuite`. ### Was this patch authored or co-authored using generative AI tooling? No Closes #44835 from nikhilsheoran-db/SPARK-46763. Authored-by: Nikhil Sheoran <125331115+nikhilsheoran-db@users.noreply.github.com> Signed-off-by: Wenchen Fan (cherry picked from commit 715b43428913d6a631f8f9043baac751b88cb5d4) Signed-off-by: Wenchen Fan --- .../sql/catalyst/optimizer/Optimizer.scala | 6 +++- .../optimizer/ReplaceOperatorSuite.scala | 31 +++++++++++++++++++ 2 files changed, 36 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index df17840d567e0..04d3eb962ed44 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -2195,11 +2195,15 @@ object ReplaceDeduplicateWithAggregate extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUpWithNewOutput { case d @ Deduplicate(keys, child) if !child.isStreaming => val keyExprIds = keys.map(_.exprId) + val generatedAliasesMap = new mutable.HashMap[Attribute, Alias](); val aggCols = child.output.map { attr => if (keyExprIds.contains(attr.exprId)) { attr } else { - Alias(new First(attr).toAggregateExpression(), attr.name)() + // Keep track of the generated aliases to avoid generating multiple aliases + // for the same attribute (in case the attribute is duplicated) + generatedAliasesMap.getOrElseUpdate(attr, + Alias(new First(attr).toAggregateExpression(), attr.name)()) } } // SPARK-22951: Physical aggregate operators distinguishes global aggregation and grouping diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala index 5d81e96a8e583..cb9577e050d04 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala @@ -265,4 +265,35 @@ class ReplaceOperatorSuite extends PlanTest { Join(basePlan, otherPlan, LeftAnti, Option(condition), JoinHint.NONE)).analyze comparePlans(result, correctAnswer) } + + test("SPARK-46763: ReplaceDeduplicateWithAggregate non-grouping keys with duplicate attributes") { + val a = $"a".int + val b = $"b".int + val first_a = Alias(new First(a).toAggregateExpression(), a.name)() + + val query = Project( + projectList = Seq(a, b), + Deduplicate( + keys = Seq(b), + child = Project( + projectList = Seq(a, a, b), + child = LocalRelation(Seq(a, b)) + ) + ) + ).analyze + + val result = Optimize.execute(query) + val correctAnswer = Project( + projectList = Seq(first_a.toAttribute, b), + Aggregate( + Seq(b), + Seq(first_a, first_a, b), + Project( + projectList = Seq(a, a, b), + child = LocalRelation(Seq(a, b)) + ) + ) + ).analyze + comparePlans(result, correctAnswer) + } } From 20da7c0f3c44b1b435171058da36d8cd59391ba8 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 23 Jan 2024 17:35:59 +0800 Subject: [PATCH 189/521] Revert "[SPARK-46417][SQL] Do not fail when calling hive.getTable and throwException is false" This reverts commit 8abf9583ac2303765255299af3e843d8248f313f. --- .../scala/org/apache/spark/sql/hive/client/HiveShim.scala | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 7025e09ae9d9e..60ff9ec42f29d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -620,13 +620,7 @@ private[client] class Shim_v0_12 extends Shim with Logging { tableName: String, throwException: Boolean): Table = { recordHiveCall() - val table = try { - hive.getTable(dbName, tableName, throwException) - } catch { - // Hive may have bugs and still throw an exception even if `throwException` is false. - case e: HiveException if !throwException => - null - } + val table = hive.getTable(dbName, tableName, throwException) if (table != null) { table.getTTable.setTableName(tableName) table.getTTable.setDbName(dbName) From 05f7aa596c7b1c05704abfad94b1b1d3085c530e Mon Sep 17 00:00:00 2001 From: Tom van Bussel Date: Tue, 23 Jan 2024 08:45:32 -0800 Subject: [PATCH 190/521] [SPARK-46794][SQL] Remove subqueries from LogicalRDD constraints This PR modifies `LogicalRDD` to filter out all subqueries from its `constraints`. Fixes a correctness bug. Spark can produce incorrect results when using a checkpointed `DataFrame` with a filter containing a scalar subquery. This subquery is included in the constraints of the resulting `LogicalRDD`, and may then be propagated as a filter when joining with the checkpointed `DataFrame`. This causes the subquery to be evaluated twice: once during checkpointing and once while evaluating the query. These two subquery evaluations may return different results, e.g. when the subquery contains a limit with an underspecified sort order. No Added a test to `DataFrameSuite`. No Closes #44833 from tomvanbussel/SPARK-46794. Authored-by: Tom van Bussel Signed-off-by: Dongjoon Hyun (cherry picked from commit d26e871136e0c6e1f84a25978319733a516b7b2e) Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/execution/ExistingRDD.scala | 7 +++++++ .../org/apache/spark/sql/DataFrameSuite.scala | 16 +++++++++++++++- 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 3dcf0efaadd8f..3b49abcb1a866 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -150,6 +150,13 @@ case class LogicalRDD( } override lazy val constraints: ExpressionSet = originConstraints.getOrElse(ExpressionSet()) + // Subqueries can have non-deterministic results even when they only contain deterministic + // expressions (e.g. consider a LIMIT 1 subquery without an ORDER BY). Propagating predicates + // containing a subquery causes the subquery to be executed twice (as the result of the subquery + // in the checkpoint computation cannot be reused), which could result in incorrect results. + // Therefore we assume that all subqueries are non-deterministic, and we do not expose any + // constraints that contain a subquery. + .filterNot(SubqueryExpression.hasSubquery) } object LogicalRDD extends Logging { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 2eba9f1810982..002719f06896d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -35,7 +35,7 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd} import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, EqualTo, ExpressionSet, GreaterThan, Literal, PythonUDF, Uuid} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, EqualTo, ExpressionSet, GreaterThan, Literal, PythonUDF, ScalarSubquery, Uuid} import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LeafNode, LocalRelation, LogicalPlan, OneRowRelation, Statistics} @@ -2258,6 +2258,20 @@ class DataFrameSuite extends QueryTest assert(newConstraints === newExpectedConstraints) } + test("SPARK-46794: exclude subqueries from LogicalRDD constraints") { + withTempDir { checkpointDir => + val subquery = + new Column(ScalarSubquery(spark.range(10).selectExpr("max(id)").logicalPlan)) + val df = spark.range(1000).filter($"id" === subquery) + assert(df.logicalPlan.constraints.exists(_.exists(_.isInstanceOf[ScalarSubquery]))) + + spark.sparkContext.setCheckpointDir(checkpointDir.getAbsolutePath) + val checkpointedDf = df.checkpoint() + assert(!checkpointedDf.logicalPlan.constraints + .exists(_.exists(_.isInstanceOf[ScalarSubquery]))) + } + } + test("SPARK-10656: completely support special chars") { val df = Seq(1 -> "a").toDF("i_$.a", "d^'a.") checkAnswer(df.select(df("*")), Row(1, "a")) From be7f1e9979c38b1358b0af2b358bacb0bd523c80 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 23 Jan 2024 16:38:45 -0800 Subject: [PATCH 191/521] [SPARK-46817][CORE] Fix `spark-daemon.sh` usage by adding `decommission` command ### What changes were proposed in this pull request? This PR aims to fix `spark-daemon.sh` usage by adding `decommission` command. ### Why are the changes needed? This was missed when SPARK-20628 added `decommission` command at Apache Spark 3.1.0. The command has been used like the following. https://github.com/apache/spark/blob/0356ac00947282b1a0885ad7eaae1e25e43671fe/sbin/decommission-worker.sh#L41 ### Does this PR introduce _any_ user-facing change? No, this is only a change on usage message. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44856 from dongjoon-hyun/SPARK-46817. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 00a92d328576c39b04cfd0fdd8a30c5a9bc37e36) Signed-off-by: Dongjoon Hyun --- sbin/spark-daemon.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 3cfd5acfe2b56..28d205f03e0fa 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -31,7 +31,7 @@ # SPARK_NO_DAEMONIZE If set, will run the proposed command in the foreground. It will not output a PID file. ## -usage="Usage: spark-daemon.sh [--config ] (start|stop|submit|status) " +usage="Usage: spark-daemon.sh [--config ] (start|stop|submit|decommission|status) " # if no args specified, show usage if [ $# -le 1 ]; then From 0956db6901bf03d2d948b23f00bcd6e74a0c251b Mon Sep 17 00:00:00 2001 From: zml1206 Date: Wed, 24 Jan 2024 15:06:55 +0800 Subject: [PATCH 192/521] [SPARK-46590][SQL][FOLLOWUP] Update CoalesceShufflePartitions comments ### What changes were proposed in this pull request? After #44661 ,In addition to Union, children of CartesianProduct, BroadcastHashJoin and BroadcastNestedLoopJoin can also be coalesced independently, update comments. ### Why are the changes needed? Improve the readability and maintainability. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? N/A ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44854 from zml1206/SPARK-46590-FOLLOWUP. Authored-by: zml1206 Signed-off-by: Wenchen Fan (cherry picked from commit fe4f8eac3efee42d53f7f24763a59c82ef03d343) Signed-off-by: Wenchen Fan --- .../adaptive/CoalesceShufflePartitions.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CoalesceShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CoalesceShufflePartitions.scala index 26e5ac649dbb1..db4a6b7dcf2eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CoalesceShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CoalesceShufflePartitions.scala @@ -66,9 +66,9 @@ case class CoalesceShufflePartitions(session: SparkSession) extends AQEShuffleRe } } - // Sub-plans under the Union operator can be coalesced independently, so we can divide them - // into independent "coalesce groups", and all shuffle stages within each group have to be - // coalesced together. + // Sub-plans under the Union/CartesianProduct/BroadcastHashJoin/BroadcastNestedLoopJoin + // operator can be coalesced independently, so we can divide them into independent + // "coalesce groups", and all shuffle stages within each group have to be coalesced together. val coalesceGroups = collectCoalesceGroups(plan) // Divide minimum task parallelism among coalesce groups according to their data sizes. @@ -137,8 +137,9 @@ case class CoalesceShufflePartitions(session: SparkSession) extends AQEShuffleRe } /** - * Gather all coalesce-able groups such that the shuffle stages in each child of a Union operator - * are in their independent groups if: + * Gather all coalesce-able groups such that the shuffle stages in each child of a + * Union/CartesianProduct/BroadcastHashJoin/BroadcastNestedLoopJoin operator are in their + * independent groups if: * 1) all leaf nodes of this child are exchange stages; and * 2) all these shuffle stages support coalescing. */ From ef33b9c50806475f287267c05278aeda3645abac Mon Sep 17 00:00:00 2001 From: Bhuwan Sahni Date: Wed, 24 Jan 2024 21:35:33 +0900 Subject: [PATCH 193/521] [SPARK-46796][SS] Ensure the correct remote files (mentioned in metadata.zip) are used on RocksDB version load This PR ensures that RocksDB loads do not run into SST file Version ID mismatch issue. RocksDB has added validation to ensure exact same SST file is used during database load from snapshot. Current streaming state suffers from certain edge cases where this condition is violated resulting in state load failure. The changes introduced are: 1. Ensure that the local SST file is exactly the same DFS file (as per mapping in metadata.zip). We keep track of the DFS file path for a local SST file, and re download the SST file in case DFS file has a different UUID in metadata zip. 2. Reset lastSnapshotVersion in RocksDB when Rocks DB is loaded. Changelog checkpoint relies on this version for future snapshots. Currently, if a older version is reloaded we were not uploading snapshots as lastSnapshotVersion was pointing to a higher snapshot of a cleanup database. We need to ensure that the correct SST files are used on executor during RocksDB load as per mapping in metadata.zip. With current implementation, its possible that the executor uses a SST file (with a different UUID) from a older version which is not the exact file mapped in the metadata.zip. This can cause version Id mismatch errors while loading RocksDB leading to streaming query failures. See https://issues.apache.org/jira/browse/SPARK-46796 for failure scenarios. No Added exhaustive unit testcases covering the scenarios. No Closes #44837 from sahnib/SPARK-46796. Authored-by: Bhuwan Sahni Signed-off-by: Jungtaek Lim (cherry picked from commit f25ebe52b9b84ece9b3c5ae30b83eaaef52ec55b) Signed-off-by: Jungtaek Lim --- .../execution/streaming/state/RocksDB.scala | 3 + .../streaming/state/RocksDBFileManager.scala | 92 +++-- .../streaming/state/RocksDBSuite.scala | 314 +++++++++++++++++- 3 files changed, 372 insertions(+), 37 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index 2398b7780726a..0c9738a6b0817 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -151,6 +151,8 @@ class RocksDB( val metadata = fileManager.loadCheckpointFromDfs(latestSnapshotVersion, workingDir) loadedVersion = latestSnapshotVersion + // reset last snapshot version + lastSnapshotVersion = 0L openDB() numKeysOnWritingVersion = if (!conf.trackTotalNumberOfRows) { @@ -191,6 +193,7 @@ class RocksDB( */ private def replayChangelog(endVersion: Long): Unit = { for (v <- loadedVersion + 1 to endVersion) { + logInfo(s"replaying changelog from version $loadedVersion -> $endVersion") var changelogReader: StateStoreChangelogReader = null try { changelogReader = fileManager.getChangelogReader(v) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala index faf9cd701aeca..300a3b8137b4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala @@ -132,6 +132,15 @@ class RocksDBFileManager( import RocksDBImmutableFile._ private val versionToRocksDBFiles = new ConcurrentHashMap[Long, Seq[RocksDBImmutableFile]] + + + // used to keep a mapping of the exact Dfs file that was used to create a local SST file. + // The reason this is a separate map because versionToRocksDBFiles can contain multiple similar + // SST files to a particular local file (for example 1.sst can map to 1-UUID1.sst in v1 and + // 1-UUID2.sst in v2). We need to capture the exact file used to ensure Version ID compatibility + // across SST files and RocksDB manifest. + private[sql] val localFilesToDfsFiles = new ConcurrentHashMap[String, RocksDBImmutableFile] + private lazy val fm = CheckpointFileManager.create(new Path(dfsRootDir), hadoopConf) private val fs = new Path(dfsRootDir).getFileSystem(hadoopConf) private val onlyZipFiles = new PathFilter { @@ -213,6 +222,7 @@ class RocksDBFileManager( versionToRocksDBFiles.keySet().removeIf(_ >= version) val metadata = if (version == 0) { if (localDir.exists) Utils.deleteRecursively(localDir) + localFilesToDfsFiles.clear() localDir.mkdirs() RocksDBCheckpointMetadata(Seq.empty, 0) } else { @@ -449,44 +459,54 @@ class RocksDBFileManager( // Get the immutable files used in previous versions, as some of those uploaded files can be // reused for this version logInfo(s"Saving RocksDB files to DFS for $version") - val prevFilesToSizes = versionToRocksDBFiles.asScala.filterKeys(_ < version) - .values.flatten.map { f => - f.localFileName -> f - }.toMap var bytesCopied = 0L var filesCopied = 0L var filesReused = 0L val immutableFiles = localFiles.map { localFile => - prevFilesToSizes - .get(localFile.getName) - .filter(_.isSameFile(localFile)) - .map { reusable => - filesReused += 1 - reusable - }.getOrElse { - val localFileName = localFile.getName - val dfsFileName = newDFSFileName(localFileName) - val dfsFile = dfsFilePath(dfsFileName) - // Note: The implementation of copyFromLocalFile() closes the output stream when there is - // any exception while copying. So this may generate partial files on DFS. But that is - // okay because until the main [version].zip file is written, those partial files are - // not going to be used at all. Eventually these files should get cleared. - fs.copyFromLocalFile( - new Path(localFile.getAbsoluteFile.toURI), dfsFile) - val localFileSize = localFile.length() - logInfo(s"Copied $localFile to $dfsFile - $localFileSize bytes") - filesCopied += 1 - bytesCopied += localFileSize - - RocksDBImmutableFile(localFile.getName, dfsFileName, localFileSize) - } + val existingDfsFile = localFilesToDfsFiles.asScala.get(localFile.getName) + if (existingDfsFile.isDefined && existingDfsFile.get.sizeBytes == localFile.length()) { + val dfsFile = existingDfsFile.get + filesReused += 1 + logInfo(s"reusing file $dfsFile for $localFile") + RocksDBImmutableFile(localFile.getName, dfsFile.dfsFileName, dfsFile.sizeBytes) + } else { + val localFileName = localFile.getName + val dfsFileName = newDFSFileName(localFileName) + val dfsFile = dfsFilePath(dfsFileName) + // Note: The implementation of copyFromLocalFile() closes the output stream when there is + // any exception while copying. So this may generate partial files on DFS. But that is + // okay because until the main [version].zip file is written, those partial files are + // not going to be used at all. Eventually these files should get cleared. + fs.copyFromLocalFile( + new Path(localFile.getAbsoluteFile.toURI), dfsFile) + val localFileSize = localFile.length() + logInfo(s"Copied $localFile to $dfsFile - $localFileSize bytes") + filesCopied += 1 + bytesCopied += localFileSize + + val immutableDfsFile = RocksDBImmutableFile(localFile.getName, dfsFileName, localFileSize) + localFilesToDfsFiles.put(localFileName, immutableDfsFile) + + immutableDfsFile + } } logInfo(s"Copied $filesCopied files ($bytesCopied bytes) from local to" + s" DFS for version $version. $filesReused files reused without copying.") versionToRocksDBFiles.put(version, immutableFiles) + // clean up deleted SST files from the localFilesToDfsFiles Map + val currentLocalFiles = localFiles.map(_.getName).toSet + val mappingsToClean = localFilesToDfsFiles.asScala + .keys + .filterNot(currentLocalFiles.contains) + + mappingsToClean.foreach { f => + logInfo(s"cleaning $f from the localFilesToDfsFiles map") + localFilesToDfsFiles.remove(f) + } + saveCheckpointMetrics = RocksDBFileManagerMetrics( bytesCopied = bytesCopied, filesCopied = filesCopied, @@ -506,11 +526,22 @@ class RocksDBFileManager( // Delete unnecessary local immutable files listRocksDBFiles(localDir)._1 .foreach { existingFile => - val isSameFile = - requiredFileNameToFileDetails.get(existingFile.getName).exists(_.isSameFile(existingFile)) + val requiredFile = requiredFileNameToFileDetails.get(existingFile.getName) + val prevDfsFile = localFilesToDfsFiles.asScala.get(existingFile.getName) + val isSameFile = if (requiredFile.isDefined && prevDfsFile.isDefined) { + requiredFile.get.dfsFileName == prevDfsFile.get.dfsFileName && + existingFile.length() == requiredFile.get.sizeBytes + } else { + false + } + if (!isSameFile) { existingFile.delete() - logInfo(s"Deleted local file $existingFile") + localFilesToDfsFiles.remove(existingFile.getName) + logInfo(s"Deleted local file $existingFile with size ${existingFile.length()} mapped" + + s" to previous dfsFile ${prevDfsFile.getOrElse("null")}") + } else { + logInfo(s"reusing $prevDfsFile present at $existingFile for $requiredFile") } } @@ -536,6 +567,7 @@ class RocksDBFileManager( } filesCopied += 1 bytesCopied += localFileSize + localFilesToDfsFiles.put(localFileName, file) logInfo(s"Copied $dfsFile to $localFile - $localFileSize bytes") } else { filesReused += 1 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 91dd858220717..04b11dfe43f0c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -24,16 +24,36 @@ import scala.language.implicitConversions import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.scalactic.source.Position import org.scalatest.Tag import org.apache.spark.sql.catalyst.util.quietly -import org.apache.spark.sql.execution.streaming.CreateAtomicTestManager +import org.apache.spark.sql.execution.streaming.{CreateAtomicTestManager, FileSystemBasedCheckpointFileManager} +import org.apache.spark.sql.execution.streaming.CheckpointFileManager.{CancellableFSDataOutputStream, RenameBasedFSDataOutputStream} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.tags.SlowSQLTest import org.apache.spark.util.{ThreadUtils, Utils} +class NoOverwriteFileSystemBasedCheckpointFileManager(path: Path, hadoopConf: Configuration) + extends FileSystemBasedCheckpointFileManager(path, hadoopConf) { + + override def createAtomic(path: Path, + overwriteIfPossible: Boolean): CancellableFSDataOutputStream = { + new RenameBasedFSDataOutputStream(this, path, overwriteIfPossible) + } + + override def renameTempFile(srcPath: Path, dstPath: Path, + overwriteIfPossible: Boolean): Unit = { + if (!fs.exists(dstPath)) { + // only write if a file does not exist at this location + super.renameTempFile(srcPath, dstPath, overwriteIfPossible) + } + } +} + trait RocksDBStateStoreChangelogCheckpointingTestUtil { val rocksdbChangelogCheckpointingConfKey: String = RocksDBConf.ROCKSDB_SQL_CONF_NAME_PREFIX + ".changelogCheckpointing.enabled" @@ -666,19 +686,19 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared // Save SAME version again with different checkpoint files and load back again to verify // whether files were overwritten. val cpFiles1_ = Seq( - "sst-file1.sst" -> 10, // same SST file as before, but same version, so should get copied + "sst-file1.sst" -> 10, // same SST file as before, this should get reused "sst-file2.sst" -> 25, // new SST file with same name as before, but different length "sst-file3.sst" -> 30, // new SST file "other-file1" -> 100, // same non-SST file as before, should not get copied "other-file2" -> 210, // new non-SST file with same name as before, but different length "other-file3" -> 300, // new non-SST file - "archive/00001.log" -> 1000, // same log file as before and version, so should get copied + "archive/00001.log" -> 1000, // same log file as before, this should get reused "archive/00002.log" -> 2500, // new log file with same name as before, but different length "archive/00003.log" -> 3000 // new log file ) saveCheckpointFiles(fileManager, cpFiles1_, version = 1, numKeys = 1001) - assert(numRemoteSSTFiles === 5, "shouldn't copy same files again") // 2 old + 3 new SST files - assert(numRemoteLogFiles === 5, "shouldn't copy same files again") // 2 old + 3 new log files + assert(numRemoteSSTFiles === 4, "shouldn't copy same files again") // 2 old + 2 new SST files + assert(numRemoteLogFiles === 4, "shouldn't copy same files again") // 2 old + 2 new log files loadAndVerifyCheckpointFiles(fileManager, verificationDir, version = 1, cpFiles1_, 1001) // Save another version and verify @@ -688,8 +708,8 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared "archive/00004.log" -> 4000 ) saveCheckpointFiles(fileManager, cpFiles2, version = 2, numKeys = 1501) - assert(numRemoteSSTFiles === 6) // 1 new file over earlier 5 files - assert(numRemoteLogFiles === 6) // 1 new file over earlier 5 files + assert(numRemoteSSTFiles === 5) // 1 new file over earlier 4 files + assert(numRemoteLogFiles === 5) // 1 new file over earlier 4 files loadAndVerifyCheckpointFiles(fileManager, verificationDir, version = 2, cpFiles2, 1501) // Loading an older version should work @@ -1152,6 +1172,286 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } + test("time travel - validate successful RocksDB load") { + val remoteDir = Utils.createTempDir().toString + val conf = dbConf.copy(minDeltasForSnapshot = 1, compactOnCommit = false) + new File(remoteDir).delete() // to make sure that the directory gets created + withDB(remoteDir, conf = conf) { db => + for (version <- 0 to 1) { + db.load(version) + db.put(version.toString, version.toString) + db.commit() + } + // upload snapshot 2.zip + db.doMaintenance() + for (version <- Seq(2)) { + db.load(version) + db.put(version.toString, version.toString) + db.commit() + } + // upload snapshot 3.zip + db.doMaintenance() + // simulate db in another executor that override the zip file + withDB(remoteDir, conf = conf) { db1 => + for (version <- 0 to 1) { + db1.load(version) + db1.put(version.toString, version.toString) + db1.commit() + } + db1.doMaintenance() + } + db.load(2) + for (version <- Seq(2)) { + db.load(version) + db.put(version.toString, version.toString) + db.commit() + } + // upload snapshot 3.zip + db.doMaintenance() + // rollback to version 2 + db.load(2) + } + } + + test("time travel 2 - validate successful RocksDB load") { + Seq(1, 2).map(minDeltasForSnapshot => { + val remoteDir = Utils.createTempDir().toString + val conf = dbConf.copy(minDeltasForSnapshot = minDeltasForSnapshot, + compactOnCommit = false) + new File(remoteDir).delete() // to make sure that the directory gets created + withDB(remoteDir, conf = conf) { db => + for (version <- 0 to 1) { + db.load(version) + db.put(version.toString, version.toString) + db.commit() + } + // upload snapshot 2.zip + db.doMaintenance() + for (version <- 2 to 3) { + db.load(version) + db.put(version.toString, version.toString) + db.commit() + } + db.load(0) + // simulate db in another executor that override the zip file + withDB(remoteDir, conf = conf) { db1 => + for (version <- 0 to 1) { + db1.load(version) + db1.put(version.toString, version.toString) + db1.commit() + } + db1.doMaintenance() + } + for (version <- 2 to 3) { + db.load(version) + db.put(version.toString, version.toString) + db.commit() + } + // upload snapshot 4.zip + db.doMaintenance() + } + withDB(remoteDir, version = 4, conf = conf) { db => + } + }) + } + + test("time travel 3 - validate successful RocksDB load") { + val remoteDir = Utils.createTempDir().toString + val conf = dbConf.copy(minDeltasForSnapshot = 0, compactOnCommit = false) + new File(remoteDir).delete() // to make sure that the directory gets created + withDB(remoteDir, conf = conf) { db => + for (version <- 0 to 2) { + db.load(version) + db.put(version.toString, version.toString) + db.commit() + } + // upload snapshot 2.zip + db.doMaintenance() + for (version <- 1 to 3) { + db.load(version) + db.put(version.toString, version.toString) + db.commit() + } + // upload snapshot 4.zip + db.doMaintenance() + } + + withDB(remoteDir, version = 4, conf = conf) { db => + } + } + + test("validate Rocks DB SST files do not have a VersionIdMismatch" + + " when metadata file is not overwritten - scenario 1") { + val fmClass = "org.apache.spark.sql.execution.streaming.state." + + "NoOverwriteFileSystemBasedCheckpointFileManager" + withTempDir { dir => + val dbConf = RocksDBConf(StateStoreConf(new SQLConf())) + val hadoopConf = new Configuration() + hadoopConf.set(STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, fmClass) + + val remoteDir = dir.getCanonicalPath + withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf) { db1 => + withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf) { db2 => + // commit version 1 via db1 + db1.load(0) + db1.put("a", "1") + db1.put("b", "1") + + db1.commit() + + // commit version 1 via db2 + db2.load(0) + db2.put("a", "1") + db2.put("b", "1") + + db2.commit() + + // commit version 2 via db2 + db2.load(1) + db2.put("a", "2") + db2.put("b", "2") + + db2.commit() + + // reload version 1, this should succeed + db2.load(1) + db1.load(1) + + // reload version 2, this should succeed + db2.load(2) + db1.load(2) + } + } + } + } + + test("validate Rocks DB SST files do not have a VersionIdMismatch" + + " when metadata file is overwritten - scenario 1") { + withTempDir { dir => + val dbConf = RocksDBConf(StateStoreConf(new SQLConf())) + val hadoopConf = new Configuration() + val remoteDir = dir.getCanonicalPath + withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf) { db1 => + withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf) { db2 => + // commit version 1 via db1 + db1.load(0) + db1.put("a", "1") + db1.put("b", "1") + + db1.commit() + + // commit version 1 via db2 + db2.load(0) + db2.put("a", "1") + db2.put("b", "1") + + db2.commit() + + // commit version 2 via db2 + db2.load(1) + db2.put("a", "2") + db2.put("b", "2") + + db2.commit() + + // reload version 1, this should succeed + db2.load(1) + db1.load(1) + + // reload version 2, this should succeed + db2.load(2) + db1.load(2) + } + } + } + } + + test("validate Rocks DB SST files do not have a VersionIdMismatch" + + " when metadata file is not overwritten - scenario 2") { + val fmClass = "org.apache.spark.sql.execution.streaming.state." + + "NoOverwriteFileSystemBasedCheckpointFileManager" + withTempDir { dir => + val dbConf = RocksDBConf(StateStoreConf(new SQLConf())) + val hadoopConf = new Configuration() + hadoopConf.set(STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, fmClass) + + val remoteDir = dir.getCanonicalPath + withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf) { db1 => + withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf) { db2 => + // commit version 1 via db2 + db2.load(0) + db2.put("a", "1") + db2.put("b", "1") + + db2.commit() + + // commit version 1 via db1 + db1.load(0) + db1.put("a", "1") + db1.put("b", "1") + + db1.commit() + + // commit version 2 via db2 + db2.load(1) + db2.put("a", "2") + db2.put("b", "2") + + db2.commit() + + // reload version 1, this should succeed + db2.load(1) + db1.load(1) + + // reload version 2, this should succeed + db2.load(2) + db1.load(2) + } + } + } + } + + test("validate Rocks DB SST files do not have a VersionIdMismatch" + + " when metadata file is overwritten - scenario 2") { + withTempDir { dir => + val dbConf = RocksDBConf(StateStoreConf(new SQLConf())) + val hadoopConf = new Configuration() + val remoteDir = dir.getCanonicalPath + withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf) { db1 => + withDB(remoteDir, conf = dbConf, hadoopConf = hadoopConf) { db2 => + // commit version 1 via db2 + db2.load(0) + db2.put("a", "1") + db2.put("b", "1") + + db2.commit() + + // commit version 1 via db1 + db1.load(0) + db1.put("a", "1") + db1.put("b", "1") + + db1.commit() + + // commit version 2 via db2 + db2.load(1) + db2.put("a", "2") + db2.put("b", "2") + + db2.commit() + + // reload version 1, this should succeed + db2.load(1) + db1.load(1) + + // reload version 2, this should succeed + db2.load(2) + db1.load(2) + } + } + } + } + private def sqlConf = SQLConf.get.clone() private def dbConf = RocksDBConf(StateStoreConf(sqlConf)) From 125b2f87d453a16325f24e7382707f2b365bba14 Mon Sep 17 00:00:00 2001 From: fred-db Date: Thu, 25 Jan 2024 08:34:37 -0800 Subject: [PATCH 194/521] [SPARK-46861][CORE] Avoid Deadlock in DAGScheduler * The DAGScheduler could currently run into a deadlock with another thread if both access the partitions of the same RDD at the same time. * To make progress in getCacheLocs, we require both exclusive access to the RDD partitions and the location cache. We first lock on the location cache, and then on the RDD. * When accessing partitions of an RDD, the RDD first acquires exclusive access on the partitions, and then might acquire exclusive access on the location cache. * If thread 1 is able to acquire access on the RDD, while thread 2 holds the access to the location cache, we can run into a deadlock situation. * To fix this, acquire locks in the same order. Change the DAGScheduler to first acquire the lock on the RDD, and then the lock on the location cache. * This is a deadlock you can run into, which can prevent any progress on the cluster. * No * Unit test that reproduces the issue. No Closes #44882 from fred-db/fix-deadlock. Authored-by: fred-db Signed-off-by: Dongjoon Hyun (cherry picked from commit 617014cc92d933c70c9865a578fceb265883badd) Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/rdd/RDD.scala | 11 ++++-- .../apache/spark/scheduler/DAGScheduler.scala | 31 ++++++++------- .../spark/scheduler/DAGSchedulerSuite.scala | 38 ++++++++++++++++++- 3 files changed, 62 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index a21d2ae773966..f695b10202758 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -223,14 +223,17 @@ abstract class RDD[T: ClassTag]( * not use `this` because RDDs are user-visible, so users might have added their own locking on * RDDs; sharing that could lead to a deadlock. * - * One thread might hold the lock on many of these, for a chain of RDD dependencies; but - * because DAGs are acyclic, and we only ever hold locks for one path in that DAG, there is no - * chance of deadlock. + * One thread might hold the lock on many of these, for a chain of RDD dependencies. Deadlocks + * are possible if we try to lock another resource while holding the stateLock, + * and the lock acquisition sequence of these locks is not guaranteed to be the same. + * This can lead lead to a deadlock as one thread might first acquire the stateLock, + * and then the resource, + * while another thread might first acquire the resource, and then the stateLock. * * Executors may reference the shared fields (though they should never mutate them, * that only happens on the driver). */ - private val stateLock = new Serializable {} + private[spark] val stateLock = new Serializable {} // Our dependencies and partitions will be gotten by calling subclass's methods below, and will // be overwritten when we're checkpointed diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index d8adaae19b90d..89d16e5793482 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -174,6 +174,9 @@ private[spark] class DAGScheduler( * locations where that RDD partition is cached. * * All accesses to this map should be guarded by synchronizing on it (see SPARK-4454). + * If you need to access any RDD while synchronizing on the cache locations, + * first synchronize on the RDD, and then synchronize on this map to avoid deadlocks. The RDD + * could try to access the cache locations after synchronizing on the RDD. */ private val cacheLocs = new HashMap[Int, IndexedSeq[Seq[TaskLocation]]] @@ -420,22 +423,24 @@ private[spark] class DAGScheduler( } private[scheduler] - def getCacheLocs(rdd: RDD[_]): IndexedSeq[Seq[TaskLocation]] = cacheLocs.synchronized { - // Note: this doesn't use `getOrElse()` because this method is called O(num tasks) times - if (!cacheLocs.contains(rdd.id)) { - // Note: if the storage level is NONE, we don't need to get locations from block manager. - val locs: IndexedSeq[Seq[TaskLocation]] = if (rdd.getStorageLevel == StorageLevel.NONE) { - IndexedSeq.fill(rdd.partitions.length)(Nil) - } else { - val blockIds = - rdd.partitions.indices.map(index => RDDBlockId(rdd.id, index)).toArray[BlockId] - blockManagerMaster.getLocations(blockIds).map { bms => - bms.map(bm => TaskLocation(bm.host, bm.executorId)) + def getCacheLocs(rdd: RDD[_]): IndexedSeq[Seq[TaskLocation]] = rdd.stateLock.synchronized { + cacheLocs.synchronized { + // Note: this doesn't use `getOrElse()` because this method is called O(num tasks) times + if (!cacheLocs.contains(rdd.id)) { + // Note: if the storage level is NONE, we don't need to get locations from block manager. + val locs: IndexedSeq[Seq[TaskLocation]] = if (rdd.getStorageLevel == StorageLevel.NONE) { + IndexedSeq.fill(rdd.partitions.length)(Nil) + } else { + val blockIds = + rdd.partitions.indices.map(index => RDDBlockId(rdd.id, index)).toArray[BlockId] + blockManagerMaster.getLocations(blockIds).map { bms => + bms.map(bm => TaskLocation(bm.host, bm.executorId)) + } } + cacheLocs(rdd.id) = locs } - cacheLocs(rdd.id) = locs + cacheLocs(rdd.id) } - cacheLocs(rdd.id) } private def clearCacheLocs(): Unit = cacheLocs.synchronized { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 9b7c5d5ace314..1818bf9b152d3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -48,7 +48,7 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.local.LocalSchedulerBackend import org.apache.spark.shuffle.{FetchFailedException, MetadataFetchFailedException} import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerId, BlockManagerMaster} -import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, CallSite, Clock, LongAccumulator, SystemClock, Utils} +import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, CallSite, Clock, LongAccumulator, SystemClock, ThreadUtils, Utils} class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler) extends DAGSchedulerEventProcessLoop(dagScheduler) { @@ -594,6 +594,42 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti assertDataStructuresEmpty() } + // Note that this test is NOT perfectly reproducible when there is a deadlock as it uses + // Thread.sleep, but it should never fail / flake when there is no deadlock. + // If this test starts to flake, this shows that there is a deadlock! + test("No Deadlock between getCacheLocs and CoalescedRDD") { + val rdd = sc.parallelize(1 to 10, numSlices = 10) + val coalescedRDD = rdd.coalesce(2) + val executionContext = ThreadUtils.newDaemonFixedThreadPool( + nThreads = 2, "test-getCacheLocs") + // Used to only make progress on getCacheLocs after we acquired the lock to the RDD. + val rddLock = new java.util.concurrent.Semaphore(0) + val partitionsFuture = executionContext.submit(new Runnable { + override def run(): Unit = { + coalescedRDD.stateLock.synchronized { + rddLock.release(1) + // Try to access the partitions of the coalescedRDD. This will cause a call to + // getCacheLocs internally. + Thread.sleep(5000) + coalescedRDD.partitions + } + } + }) + val getCacheLocsFuture = executionContext.submit(new Runnable { + override def run(): Unit = { + rddLock.acquire() + // Access the cache locations. + // If the partition location cache is locked before the stateLock is locked, + // we'll run into a deadlock. + sc.dagScheduler.getCacheLocs(coalescedRDD) + } + }) + // If any of the futures throw a TimeOutException, this shows that there is a deadlock between + // getCacheLocs and accessing partitions of an RDD. + getCacheLocsFuture.get(120, TimeUnit.SECONDS) + partitionsFuture.get(120, TimeUnit.SECONDS) + } + test("All shuffle files on the storage endpoint should be cleaned up when it is lost") { conf.set(config.SHUFFLE_SERVICE_ENABLED.key, "true") conf.set("spark.files.fetchFailure.unRegisterOutputOnHost", "true") From e5a654e818b4698260807a081e5cf3d71480ac13 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Thu, 25 Jan 2024 22:35:38 -0800 Subject: [PATCH 195/521] [SPARK-46855][INFRA][3.5] Add `sketch` to the dependencies of the `catalyst` in `module.py` ### What changes were proposed in this pull request? This pr add `sketch` to the dependencies of the `catalyst` module in `module.py` due to `sketch` is direct dependency of `catalyst` module. ### Why are the changes needed? Ensure that when modifying the `sketch` module, both `catalyst` and cascading modules will trigger tests. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #44893 from LuciferYang/SPARK-46855-35. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun --- dev/sparktestsupport/modules.py | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 33d253a47ea07..d29fc8726018d 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -168,6 +168,15 @@ def __hash__(self): ], ) +sketch = Module( + name="sketch", + dependencies=[tags], + source_file_regexes=[ + "common/sketch/", + ], + sbt_test_goals=["sketch/test"], +) + core = Module( name="core", dependencies=[kvstore, network_common, network_shuffle, unsafe, launcher], @@ -181,7 +190,7 @@ def __hash__(self): catalyst = Module( name="catalyst", - dependencies=[tags, core], + dependencies=[tags, sketch, core], source_file_regexes=[ "sql/catalyst/", ], @@ -295,15 +304,6 @@ def __hash__(self): ], ) -sketch = Module( - name="sketch", - dependencies=[tags], - source_file_regexes=[ - "common/sketch/", - ], - sbt_test_goals=["sketch/test"], -) - graphx = Module( name="graphx", dependencies=[tags, core], From cf4e8672dd1a4e6309bb530191be7d92a793cb28 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Fri, 26 Jan 2024 11:02:15 +0300 Subject: [PATCH 196/521] [SPARK-46862][SQL] Disable CSV column pruning in the multi-line mode ### What changes were proposed in this pull request? In the PR, I propose to disable the column pruning feature in the CSV datasource for the `multiLine` mode. ### Why are the changes needed? To workaround the issue in the `uniVocity` parser used by the CSV datasource: https://github.com/uniVocity/univocity-parsers/issues/529 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the affected test suites: ``` $ build/sbt "test:testOnly *CSVv1Suite" $ build/sbt "test:testOnly *CSVv2Suite" $ build/sbt "test:testOnly *CSVLegacyTimeParserSuite" $ build/sbt "testOnly *.CsvFunctionsSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44872 from MaxGekk/csv-disable-column-pruning. Authored-by: Max Gekk Signed-off-by: Max Gekk (cherry picked from commit 829e742df8251c6f5e965cb08ad454ac3ee1a389) Signed-off-by: Max Gekk --- .../spark/sql/catalyst/csv/CSVOptions.scala | 10 ++++++++ .../sql/catalyst/csv/UnivocityParser.scala | 2 +- .../v2/csv/CSVPartitionReaderFactory.scala | 2 +- .../execution/datasources/csv/CSVSuite.scala | 25 ++++++++++++++++++- 4 files changed, 36 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala index 845c815c5648b..c5a6bf5076dec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala @@ -277,6 +277,15 @@ class CSVOptions( val unescapedQuoteHandling: UnescapedQuoteHandling = UnescapedQuoteHandling.valueOf(parameters .getOrElse(UNESCAPED_QUOTE_HANDLING, "STOP_AT_DELIMITER").toUpperCase(Locale.ROOT)) + /** + * The column pruning feature can be enabled either via the CSV option `columnPruning` or + * in non-multiline mode via initialization of CSV options by the SQL config: + * `spark.sql.csv.parser.columnPruning.enabled`. + * The feature is disabled in the `multiLine` mode because of the issue: + * https://github.com/uniVocity/univocity-parsers/issues/529 + */ + val isColumnPruningEnabled: Boolean = getBool(COLUMN_PRUNING, !multiLine && columnPruning) + def asWriterSettings: CsvWriterSettings = { val writerSettings = new CsvWriterSettings() val format = writerSettings.getFormat @@ -376,4 +385,5 @@ object CSVOptions extends DataSourceOptions { val SEP = "sep" val DELIMITER = "delimiter" newOption(SEP, DELIMITER) + val COLUMN_PRUNING = newOption("columnPruning") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index b99ee630d4b22..804c5d358ad6c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -72,7 +72,7 @@ class UnivocityParser( // positions. Generally assigned by input configuration options, except when input column(s) have // default values, in which case we omit the explicit indexes in order to know how many tokens // were present in each line instead. - private def columnPruning: Boolean = options.columnPruning && + private def columnPruning: Boolean = options.isColumnPruningEnabled && !requiredSchema.exists(_.metadata.contains(EXISTS_DEFAULT_COLUMN_METADATA_KEY)) // When column pruning is enabled, the parser only parses the required columns based on diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala index 37f6ae4aaa9fc..cef5a71ca9c60 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala @@ -58,7 +58,7 @@ case class CSVPartitionReaderFactory( actualReadDataSchema, options, filters) - val schema = if (options.columnPruning) actualReadDataSchema else actualDataSchema + val schema = if (options.isColumnPruningEnabled) actualReadDataSchema else actualDataSchema val isStartOfFile = file.start == 0 val headerChecker = new CSVHeaderChecker( schema, options, source = s"CSV file: ${file.urlEncodedPath}", isStartOfFile) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 78266acfd7de9..6690bf101fa7a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -2079,6 +2079,7 @@ abstract class CSVSuite .option("header", true) .option("enforceSchema", false) .option("multiLine", multiLine) + .option("columnPruning", true) .load(dir) .select("columnA"), Row("a")) @@ -2089,6 +2090,7 @@ abstract class CSVSuite .option("header", true) .option("enforceSchema", false) .option("multiLine", multiLine) + .option("columnPruning", true) .load(dir) .count() === 1L) } @@ -3151,7 +3153,7 @@ abstract class CSVSuite } test("SPARK-40667: validate CSV Options") { - assert(CSVOptions.getAllOptions.size == 38) + assert(CSVOptions.getAllOptions.size == 39) // Please add validation on any new CSV options here assert(CSVOptions.isValidOption("header")) assert(CSVOptions.isValidOption("inferSchema")) @@ -3191,6 +3193,7 @@ abstract class CSVSuite assert(CSVOptions.isValidOption("codec")) assert(CSVOptions.isValidOption("sep")) assert(CSVOptions.isValidOption("delimiter")) + assert(CSVOptions.isValidOption("columnPruning")) // Please add validation on any new parquet options with alternative here assert(CSVOptions.getAlternativeOption("sep").contains("delimiter")) assert(CSVOptions.getAlternativeOption("delimiter").contains("sep")) @@ -3200,6 +3203,26 @@ abstract class CSVSuite assert(CSVOptions.getAlternativeOption("codec").contains("compression")) assert(CSVOptions.getAlternativeOption("preferDate").isEmpty) } + + test("SPARK-46862: column pruning in the multi-line mode") { + val data = + """"jobID","Name","City","Active" + |"1","DE","","Yes" + |"5",",","","," + |"3","SA","","No" + |"10","abcd""efgh"" \ndef","","" + |"8","SE","","No"""".stripMargin + + withTempPath { path => + Files.write(path.toPath, data.getBytes(StandardCharsets.UTF_8)) + val df = spark.read + .option("multiline", "true") + .option("header", "true") + .option("escape", "\"") + .csv(path.getCanonicalPath) + assert(df.count() === 5) + } + } } class CSVv1Suite extends CSVSuite { From a2854ba5d852e2001b96636a8964494c45fc27d3 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Sat, 27 Jan 2024 19:22:52 +0300 Subject: [PATCH 197/521] [SPARK-46862][SQL][FOLLOWUP] Fix column pruning without schema enforcing in V1 CSV datasource ### What changes were proposed in this pull request? In the PR, I propose to invoke `CSVOptons.isColumnPruningEnabled` introduced by https://github.com/apache/spark/pull/44872 while matching of CSV header to a schema in the V1 CSV datasource. ### Why are the changes needed? To fix the failure when column pruning happens and a schema is not enforced: ```scala scala> spark.read. | option("multiLine", true). | option("header", true). | option("escape", "\""). | option("enforceSchema", false). | csv("/Users/maximgekk/tmp/es-939111-data.csv"). | count() 24/01/27 12:43:14 ERROR Executor: Exception in task 0.0 in stage 3.0 (TID 3) java.lang.IllegalArgumentException: Number of column in CSV header is not equal to number of fields in the schema: Header length: 4, schema size: 0 CSV file: file:///Users/maximgekk/tmp/es-939111-data.csv ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? By running the affected test suites: ``` $ build/sbt "test:testOnly *CSVv1Suite" $ build/sbt "test:testOnly *CSVv2Suite" $ build/sbt "test:testOnly *CSVLegacyTimeParserSuite" $ build/sbt "testOnly *.CsvFunctionsSuite" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #44910 from MaxGekk/check-header-column-pruning. Authored-by: Max Gekk Signed-off-by: Max Gekk (cherry picked from commit bc51c9fea3645c6ae1d9e1e83b0f94f8b849be20) Signed-off-by: Max Gekk --- .../execution/datasources/csv/CSVFileFormat.scala | 6 +++--- .../sql/execution/datasources/csv/CSVSuite.scala | 15 +++++++++------ 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 069ad9562a7d5..0ff96f073f03b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -100,12 +100,12 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) - val columnPruning = sparkSession.sessionState.conf.csvColumnPruning val parsedOptions = new CSVOptions( options, - columnPruning, + sparkSession.sessionState.conf.csvColumnPruning, sparkSession.sessionState.conf.sessionLocalTimeZone, sparkSession.sessionState.conf.columnNameOfCorruptRecord) + val isColumnPruningEnabled = parsedOptions.isColumnPruningEnabled // Check a field requirement for corrupt records here to throw an exception in a driver side ExprUtils.verifyColumnNameOfCorruptRecord(dataSchema, parsedOptions.columnNameOfCorruptRecord) @@ -125,7 +125,7 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { actualRequiredSchema, parsedOptions, actualFilters) - val schema = if (columnPruning) actualRequiredSchema else actualDataSchema + val schema = if (isColumnPruningEnabled) actualRequiredSchema else actualDataSchema val isStartOfFile = file.start == 0 val headerChecker = new CSVHeaderChecker( schema, parsedOptions, source = s"CSV file: ${file.urlEncodedPath}", isStartOfFile) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 6690bf101fa7a..a91adb787838e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -3215,12 +3215,15 @@ abstract class CSVSuite withTempPath { path => Files.write(path.toPath, data.getBytes(StandardCharsets.UTF_8)) - val df = spark.read - .option("multiline", "true") - .option("header", "true") - .option("escape", "\"") - .csv(path.getCanonicalPath) - assert(df.count() === 5) + Seq(true, false).foreach { enforceSchema => + val df = spark.read + .option("multiLine", true) + .option("header", true) + .option("escape", "\"") + .option("enforceSchema", enforceSchema) + .csv(path.getCanonicalPath) + assert(df.count() === 5) + } } } } From accfb39e4ddf7f7b54396bd0e35256a04461c693 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 27 Jan 2024 20:24:15 -0800 Subject: [PATCH 198/521] [SPARK-46888][CORE] Fix `Master` to reject `/workers/kill/` requests if decommission is disabled MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR aims to fix `Master` to reject `/workers/kill/` request if `spark.decommission.enabled` is `false` in order to fix the dangling worker issue. Currently, `spark.decommission.enabled` is `false` by default. So, when a user asks to decommission, only Master marked it `DECOMMISSIONED` while the worker is alive. ``` $ curl -XPOST http://localhost:8080/workers/kill/\?host\=127.0.0.1 ``` **Master UI** ![Screenshot 2024-01-27 at 6 19 18 PM](https://github.com/apache/spark/assets/9700541/443bfc32-b924-438a-8bf6-c64b9afbc4be) **Worker Log** ``` 24/01/27 18:18:06 WARN Worker: Receive decommission request, but decommission feature is disabled. ``` To be consistent with the existing `Worker` behavior which ignores the request. https://github.com/apache/spark/blob/1787a5261e87e0214a3f803f6534c5e52a0138e6/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala#L859-L868 No, this is a bug fix. Pass the CI with the newly added test case. No. Closes #44915 from dongjoon-hyun/SPARK-46888. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 20b593811dc02c96c71978851e051d32bf8c3496) Signed-off-by: Dongjoon Hyun --- .../spark/deploy/master/ui/MasterWebUI.scala | 4 +++- .../spark/deploy/master/MasterSuite.scala | 21 +++++++++++++++++++ .../deploy/master/ui/MasterWebUISuite.scala | 3 ++- 3 files changed, 26 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index af94bd6d9e0f2..53e5c5ac2a8f0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -23,6 +23,7 @@ import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import org.apache.spark.deploy.DeployMessages.{DecommissionWorkersOnHosts, MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.Master import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.DECOMMISSION_ENABLED import org.apache.spark.internal.config.UI.MASTER_UI_DECOMMISSION_ALLOW_MODE import org.apache.spark.internal.config.UI.UI_KILL_ENABLED import org.apache.spark.ui.{SparkUI, WebUI} @@ -40,6 +41,7 @@ class MasterWebUI( val masterEndpointRef = master.self val killEnabled = master.conf.get(UI_KILL_ENABLED) + val decommissionDisabled = !master.conf.get(DECOMMISSION_ENABLED) val decommissionAllowMode = master.conf.get(MASTER_UI_DECOMMISSION_ALLOW_MODE) initialize() @@ -58,7 +60,7 @@ class MasterWebUI( override def doPost(req: HttpServletRequest, resp: HttpServletResponse): Unit = { val hostnames: Seq[String] = Option(req.getParameterValues("host")) .getOrElse(Array[String]()).toSeq - if (!isDecommissioningRequestAllowed(req)) { + if (decommissionDisabled || !isDecommissioningRequestAllowed(req)) { resp.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED) } else { val removedWorkers = masterEndpointRef.askSync[Integer]( diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 1cec863b1e7f9..37874de987662 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy.master +import java.net.{HttpURLConnection, URL} import java.util.Date import java.util.concurrent.{ConcurrentLinkedQueue, CountDownLatch, TimeUnit} import java.util.concurrent.atomic.AtomicInteger @@ -325,6 +326,26 @@ class MasterSuite extends SparkFunSuite } } + test("SPARK-46888: master should reject worker kill request if decommision is disabled") { + implicit val formats = org.json4s.DefaultFormats + val conf = new SparkConf() + .set(DECOMMISSION_ENABLED, false) + .set(MASTER_UI_DECOMMISSION_ALLOW_MODE, "ALLOW") + val localCluster = LocalSparkCluster(1, 1, 512, conf) + localCluster.start() + val masterUrl = s"http://${Utils.localHostNameForURI()}:${localCluster.masterWebUIPort}" + try { + eventually(timeout(30.seconds), interval(100.milliseconds)) { + val url = new URL(s"$masterUrl/workers/kill/?host=${Utils.localHostNameForURI()}") + val conn = url.openConnection().asInstanceOf[HttpURLConnection] + conn.setRequestMethod("POST") + assert(conn.getResponseCode === 405) + } + } finally { + localCluster.stop() + } + } + test("master/worker web ui available") { implicit val formats = org.json4s.DefaultFormats val conf = new SparkConf() diff --git a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala index 024511189accc..40265a12af93b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala @@ -30,12 +30,13 @@ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.DeployMessages.{DecommissionWorkersOnHosts, KillDriverResponse, RequestKillDriver} import org.apache.spark.deploy.DeployTestUtils._ import org.apache.spark.deploy.master._ +import org.apache.spark.internal.config.DECOMMISSION_ENABLED import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv} import org.apache.spark.util.Utils class MasterWebUISuite extends SparkFunSuite { - val conf = new SparkConf() + val conf = new SparkConf().set(DECOMMISSION_ENABLED, true) val securityMgr = new SecurityManager(conf) val rpcEnv = mock(classOf[RpcEnv]) val master = mock(classOf[Master]) From 343ae822616185022570f1c14b151e54ff54e265 Mon Sep 17 00:00:00 2001 From: Willi Raschkowski Date: Mon, 29 Jan 2024 22:43:21 -0800 Subject: [PATCH 199/521] [SPARK-46893][UI] Remove inline scripts from UI descriptions ### What changes were proposed in this pull request? This PR prevents malicious users from injecting inline scripts via job and stage descriptions. Spark's Web UI [already checks the security of job and stage descriptions](https://github.com/apache/spark/blob/a368280708dd3c6eb90bd3b09a36a68bdd096222/core/src/main/scala/org/apache/spark/ui/UIUtils.scala#L528-L545) before rendering them as HTML (or treating them as plain text). The UI already disallows `
Property NameDefaultMeaning
Property NameDefaultMeaningSince Version
spark.yarn.shuffle.stopOnFailure false2.1.0
spark.yarn.shuffle.service.metrics.namespace3.2.0
spark.yarn.shuffle.service.logs.namespace3.3.0
spark.shuffle.service.db.backend{formatResourcesAddresses(driver.resources)} {driver.desc.command.arguments(2)}{UIUtils.formatDuration(System.currentTimeMillis() - driver.startTime)} + {UIUtils.formatDuration(System.currentTimeMillis() - driver.startTime)} +
snappy Compression codec used in writing of AVRO files. Supported codecs: uncompressed, deflate, - snappy, bzip2 and xz. Default codec is snappy. + snappy, bzip2, xz and zstandard. Default codec is snappy. 2.4.0