From 2516fd8439df42b1c161fbd346a0c346cc075f0f Mon Sep 17 00:00:00 2001 From: Andy Lam Date: Thu, 23 May 2024 15:46:09 -0700 Subject: [PATCH 01/16] [SPARK-45009][SQL][FOLLOW UP] Add error class and tests for decorrelation of predicate subqueries in join condition which reference both join child ### What changes were proposed in this pull request? This is a follow up PR for https://github.com/apache/spark/pull/42725, which decorrelates predicate subqueries in join conditions. I forgot to add the error class definition for the case where the subquery references both join children, and test cases for it. ### Why are the changes needed? To show a clear error message when the condition is hit. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added SQL test and golden files. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46708 from andylam-db/follow-up-decorrelate-subqueries-in-join-cond. Authored-by: Andy Lam Signed-off-by: Gengliang Wang --- .../resources/error/error-conditions.json | 6 +++ .../exists-in-join-condition.sql.out | 44 +++++++++++++++++++ .../in-subquery-in-join-condition.sql.out | 44 +++++++++++++++++++ .../exists-in-join-condition.sql | 4 ++ .../in-subquery-in-join-condition.sql | 4 ++ .../exists-in-join-condition.sql.out | 30 +++++++++++++ .../in-subquery-in-join-condition.sql.out | 30 +++++++++++++ 7 files changed, 162 insertions(+) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index c1c0cd6bfb39e..883c51bffadec 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4731,6 +4731,12 @@ "" ] }, + "UNSUPPORTED_CORRELATED_EXPRESSION_IN_JOIN_CONDITION" : { + "message" : [ + "Correlated subqueries in the join predicate cannot reference both join inputs:", + "" + ] + }, "UNSUPPORTED_CORRELATED_REFERENCE_DATA_TYPE" : { "message" : [ "Correlated column reference '' cannot be type." diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-in-join-condition.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-in-join-condition.sql.out index 1b09e8798a325..3b55a7293bcfa 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-in-join-condition.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-in-join-condition.sql.out @@ -1004,3 +1004,47 @@ Sort [x1#x ASC NULLS FIRST, x2#x ASC NULLS FIRST, y1#x ASC NULLS FIRST, y2#x ASC +- View (`y`, [y1#x, y2#x]) +- Project [cast(col1#x as int) AS y1#x, cast(col2#x as int) AS y2#x] +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from x join y on x1 = y1 and exists (select * from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2 +-- !query analysis +Sort [x1#x ASC NULLS FIRST, x2#x ASC NULLS FIRST, y1#x ASC NULLS FIRST, y2#x ASC NULLS FIRST], true ++- Project [x1#x, x2#x, y1#x, y2#x] + +- Join Inner, ((x1#x = y1#x) AND exists#x [x2#x && y2#x]) + : +- Project [z1#x, z2#x] + : +- Filter ((z2#x = outer(x2#x)) AND (z2#x = outer(y2#x))) + : +- SubqueryAlias z + : +- View (`z`, [z1#x, z2#x]) + : +- Project [cast(col1#x as int) AS z1#x, cast(col2#x as int) AS z2#x] + : +- LocalRelation [col1#x, col2#x] + :- SubqueryAlias x + : +- View (`x`, [x1#x, x2#x]) + : +- Project [cast(col1#x as int) AS x1#x, cast(col2#x as int) AS x2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias y + +- View (`y`, [y1#x, y2#x]) + +- Project [cast(col1#x as int) AS y1#x, cast(col2#x as int) AS y2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from x join y on x1 = y1 and not exists (select * from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2 +-- !query analysis +Sort [x1#x ASC NULLS FIRST, x2#x ASC NULLS FIRST, y1#x ASC NULLS FIRST, y2#x ASC NULLS FIRST], true ++- Project [x1#x, x2#x, y1#x, y2#x] + +- Join Inner, ((x1#x = y1#x) AND NOT exists#x [x2#x && y2#x]) + : +- Project [z1#x, z2#x] + : +- Filter ((z2#x = outer(x2#x)) AND (z2#x = outer(y2#x))) + : +- SubqueryAlias z + : +- View (`z`, [z1#x, z2#x]) + : +- Project [cast(col1#x as int) AS z1#x, cast(col2#x as int) AS z2#x] + : +- LocalRelation [col1#x, col2#x] + :- SubqueryAlias x + : +- View (`x`, [x1#x, x2#x]) + : +- Project [cast(col1#x as int) AS x1#x, cast(col2#x as int) AS x2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias y + +- View (`y`, [y1#x, y2#x]) + +- Project [cast(col1#x as int) AS y1#x, cast(col2#x as int) AS y2#x] + +- LocalRelation [col1#x, col2#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-subquery-in-join-condition.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-subquery-in-join-condition.sql.out index 422ac4d5c2cbf..ce6a1a3d7ed53 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-subquery-in-join-condition.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-subquery-in-join-condition.sql.out @@ -916,3 +916,47 @@ Sort [x1#x ASC NULLS FIRST, x2#x ASC NULLS FIRST, y1#x ASC NULLS FIRST, y2#x ASC +- View (`y`, [y1#x, y2#x]) +- Project [cast(col1#x as int) AS y1#x, cast(col2#x as int) AS y2#x] +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from x left join y on x1 = y1 and x2 IN (select z1 from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2 +-- !query analysis +Sort [x1#x ASC NULLS FIRST, x2#x ASC NULLS FIRST, y1#x ASC NULLS FIRST, y2#x ASC NULLS FIRST], true ++- Project [x1#x, x2#x, y1#x, y2#x] + +- Join LeftOuter, ((x1#x = y1#x) AND x2#x IN (list#x [x2#x && y2#x])) + : +- Project [z1#x] + : +- Filter ((z2#x = outer(x2#x)) AND (z2#x = outer(y2#x))) + : +- SubqueryAlias z + : +- View (`z`, [z1#x, z2#x]) + : +- Project [cast(col1#x as int) AS z1#x, cast(col2#x as int) AS z2#x] + : +- LocalRelation [col1#x, col2#x] + :- SubqueryAlias x + : +- View (`x`, [x1#x, x2#x]) + : +- Project [cast(col1#x as int) AS x1#x, cast(col2#x as int) AS x2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias y + +- View (`y`, [y1#x, y2#x]) + +- Project [cast(col1#x as int) AS y1#x, cast(col2#x as int) AS y2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select * from x left join y on x1 = y1 and x2 not IN (select z1 from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2 +-- !query analysis +Sort [x1#x ASC NULLS FIRST, x2#x ASC NULLS FIRST, y1#x ASC NULLS FIRST, y2#x ASC NULLS FIRST], true ++- Project [x1#x, x2#x, y1#x, y2#x] + +- Join LeftOuter, ((x1#x = y1#x) AND NOT x2#x IN (list#x [x2#x && y2#x])) + : +- Project [z1#x] + : +- Filter ((z2#x = outer(x2#x)) AND (z2#x = outer(y2#x))) + : +- SubqueryAlias z + : +- View (`z`, [z1#x, z2#x]) + : +- Project [cast(col1#x as int) AS z1#x, cast(col2#x as int) AS z2#x] + : +- LocalRelation [col1#x, col2#x] + :- SubqueryAlias x + : +- View (`x`, [x1#x, x2#x]) + : +- Project [cast(col1#x as int) AS x1#x, cast(col2#x as int) AS x2#x] + : +- LocalRelation [col1#x, col2#x] + +- SubqueryAlias y + +- View (`y`, [y1#x, y2#x]) + +- Project [cast(col1#x as int) AS y1#x, cast(col2#x as int) AS y2#x] + +- LocalRelation [col1#x, col2#x] diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-in-join-condition.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-in-join-condition.sql index ad2e7ad563e08..bc732cc3d320d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-in-join-condition.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-in-join-condition.sql @@ -89,3 +89,7 @@ select * from x inner join y on x1 = y1 and exists (select * from z where z1 = y select * from x inner join y on x1 = y1 and not exists (select * from z where z1 = y1) order by x1, x2, y1, y2; select * from x left join y on x1 = y1 and exists (select * from z where z1 = y1) order by x1, x2, y1, y2; select * from x left join y on x1 = y1 and not exists (select * from z where z1 = y1) order by x1, x2, y1, y2; + +-- Correlated subquery references both left and right children, errors +select * from x join y on x1 = y1 and exists (select * from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2; +select * from x join y on x1 = y1 and not exists (select * from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2; diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-subquery-in-join-condition.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-subquery-in-join-condition.sql index d519abdbacc05..c906390c99c32 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-subquery-in-join-condition.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-subquery-in-join-condition.sql @@ -84,3 +84,7 @@ select * from x inner join y on x1 = y1 and y2 IN (select z1 from z where z1 = y select * from x inner join y on x1 = y1 and y2 not IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2; select * from x left join y on x1 = y1 and y2 IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2; select * from x left join y on x1 = y1 and y2 not IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2; + +-- Correlated subquery references both left and right children, errors +select * from x left join y on x1 = y1 and x2 IN (select z1 from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 and x2 not IN (select z1 from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2; diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-in-join-condition.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-in-join-condition.sql.out index b490704bebc57..c9c68a5f0602b 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-in-join-condition.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-in-join-condition.sql.out @@ -472,3 +472,33 @@ struct 1 1 1 4 2 1 NULL NULL 3 4 NULL NULL + + +-- !query +select * from x join y on x1 = y1 and exists (select * from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_EXPRESSION_IN_JOIN_CONDITION", + "sqlState" : "0A000", + "messageParameters" : { + "subqueryExpression" : "exists(x.x2, y.y2, (z.z2 = x.x2), (z.z2 = y.y2))" + } +} + + +-- !query +select * from x join y on x1 = y1 and not exists (select * from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_EXPRESSION_IN_JOIN_CONDITION", + "sqlState" : "0A000", + "messageParameters" : { + "subqueryExpression" : "exists(x.x2, y.y2, (z.z2 = x.x2), (z.z2 = y.y2))" + } +} diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-subquery-in-join-condition.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-subquery-in-join-condition.sql.out index 9f829d522ad25..13af4c81173ae 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-subquery-in-join-condition.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-subquery-in-join-condition.sql.out @@ -434,3 +434,33 @@ struct 1 1 1 4 2 1 NULL NULL 3 4 NULL NULL + + +-- !query +select * from x left join y on x1 = y1 and x2 IN (select z1 from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_EXPRESSION_IN_JOIN_CONDITION", + "sqlState" : "0A000", + "messageParameters" : { + "subqueryExpression" : "(x.x2 IN (listquery(x.x2, y.y2, (z.z2 = x.x2), (z.z2 = y.y2))))" + } +} + + +-- !query +select * from x left join y on x1 = y1 and x2 not IN (select z1 from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_EXPRESSION_IN_JOIN_CONDITION", + "sqlState" : "0A000", + "messageParameters" : { + "subqueryExpression" : "(x.x2 IN (listquery(x.x2, y.y2, (z.z2 = x.x2), (z.z2 = y.y2))))" + } +} From 6afa6cc3c16e21f94087ebb6adb01bd1ff397086 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 24 May 2024 10:13:49 +0800 Subject: [PATCH 02/16] [SPARK-48399][SQL] Teradata: ByteType should map to BYTEINT instead of BYTE(binary) ### What changes were proposed in this pull request? According to the doc of Teradata and Teradata jdbc, BYTE represents binary type in Teradata, while BYTEINT is used for tinyint. - https://docs.teradata.com/r/Enterprise_IntelliFlex_VMware/SQL-Data-Types-and-Literals/Numeric-Data-Types/BYTEINT-Data-Type - https://teradata-docs.s3.amazonaws.com/doc/connectivity/jdbc/reference/current/frameset.html ### Why are the changes needed? Bugfix ### Does this PR introduce _any_ user-facing change? Yes, ByteType used to be stored as binary type in Teradata, now it has become BYTEINT. (The use-case seems rare, the migration guide or legacy config are pending reviewer's comments) ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #46715 from yaooqinn/SPARK-48399. Authored-by: Kent Yao Signed-off-by: Kent Yao --- .../apache/spark/sql/jdbc/TeradataDialect.scala | 1 + .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 15 +++++---------- 2 files changed, 6 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala index 7acd22a3f10be..95a9f60b64ed8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala @@ -42,6 +42,7 @@ private case class TeradataDialect() extends JdbcDialect { override def getJDBCType(dt: DataType): Option[JdbcType] = dt match { case StringType => Some(JdbcType("VARCHAR(255)", java.sql.Types.VARCHAR)) case BooleanType => Option(JdbcType("CHAR(1)", java.sql.Types.CHAR)) + case ByteType => Option(JdbcType("BYTEINT", java.sql.Types.TINYINT)) 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 0a792f44d3e22..e4116b565818e 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 @@ -1477,16 +1477,11 @@ class JDBCSuite extends QueryTest with SharedSparkSession { } } - test("SPARK-15648: teradataDialect StringType data mapping") { - val teradataDialect = JdbcDialects.get("jdbc:teradata://127.0.0.1/db") - assert(teradataDialect.getJDBCType(StringType). - map(_.databaseTypeDefinition).get == "VARCHAR(255)") - } - - test("SPARK-15648: teradataDialect BooleanType data mapping") { - val teradataDialect = JdbcDialects.get("jdbc:teradata://127.0.0.1/db") - assert(teradataDialect.getJDBCType(BooleanType). - map(_.databaseTypeDefinition).get == "CHAR(1)") + test("SPARK-48399: TeradataDialect jdbc data mapping") { + val dialect = JdbcDialects.get("jdbc:teradata://127.0.0.1/db") + assert(dialect.getJDBCType(StringType).map(_.databaseTypeDefinition).get == "VARCHAR(255)") + assert(dialect.getJDBCType(BooleanType).map(_.databaseTypeDefinition).get == "CHAR(1)") + assert(dialect.getJDBCType(ByteType).map(_.databaseTypeDefinition).get == "BYTEINT") } test("SPARK-38846: TeradataDialect catalyst type mapping") { From 3b9b52dff6149e499c59bb30641df777bd712d9b Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 24 May 2024 11:52:37 +0800 Subject: [PATCH 03/16] [SPARK-48405][BUILD] Upgrade `commons-compress` to 1.26.2 ### What changes were proposed in this pull request? The pr aims to upgrade `commons-compress` to `1.26.2`. ### Why are the changes needed? The full release notes: https://commons.apache.org/proper/commons-compress/changes-report.html#a1.26.2 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46725 from panbingkun/SPARK-48405. Authored-by: panbingkun Signed-off-by: Kent Yao --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 79ce883dc672c..35f6103e9fa45 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -42,7 +42,7 @@ commons-codec/1.17.0//commons-codec-1.17.0.jar commons-collections/3.2.2//commons-collections-3.2.2.jar commons-collections4/4.4//commons-collections4-4.4.jar commons-compiler/3.1.9//commons-compiler-3.1.9.jar -commons-compress/1.26.1//commons-compress-1.26.1.jar +commons-compress/1.26.2//commons-compress-1.26.2.jar commons-crypto/1.1.0//commons-crypto-1.1.0.jar commons-dbcp/1.4//commons-dbcp-1.4.jar commons-io/2.16.1//commons-io-2.16.1.jar diff --git a/pom.xml b/pom.xml index 6bbcf05b59e54..ecd05ee996e1b 100644 --- a/pom.xml +++ b/pom.xml @@ -187,7 +187,7 @@ 1.1.10.5 3.0.3 1.17.0 - 1.26.1 + 1.26.2 2.16.1 2.6 From f42ed6c760043b0213ebf0348a22dec7c0bb8244 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Fri, 24 May 2024 14:23:23 +0800 Subject: [PATCH 04/16] [SPARK-48406][BUILD] Upgrade commons-cli to 1.8.0 ### What changes were proposed in this pull request? This pr aims to upgrade Apache `commons-cli` from 1.6.0 to 1.8.0. ### Why are the changes needed? The full release notes as follows: - https://commons.apache.org/proper/commons-cli/changes-report.html#a1.7.0 - https://commons.apache.org/proper/commons-cli/changes-report.html#a1.8.0 ### 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 #46727 from LuciferYang/commons-cli-180. Authored-by: yangjie01 Signed-off-by: Kent Yao --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 35f6103e9fa45..46c5108e4eba4 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -37,7 +37,7 @@ cats-kernel_2.13/2.8.0//cats-kernel_2.13-2.8.0.jar checker-qual/3.42.0//checker-qual-3.42.0.jar chill-java/0.10.0//chill-java-0.10.0.jar chill_2.13/0.10.0//chill_2.13-0.10.0.jar -commons-cli/1.6.0//commons-cli-1.6.0.jar +commons-cli/1.8.0//commons-cli-1.8.0.jar commons-codec/1.17.0//commons-codec-1.17.0.jar commons-collections/3.2.2//commons-collections-3.2.2.jar commons-collections4/4.4//commons-collections4-4.4.jar diff --git a/pom.xml b/pom.xml index ecd05ee996e1b..e8d47afa1cca5 100644 --- a/pom.xml +++ b/pom.xml @@ -210,7 +210,7 @@ 4.17.0 3.1.0 1.1.0 - 1.6.0 + 1.8.0 1.78 1.13.0 6.0.0 From a29c9653f3d48d97875ae446d82896bdf0de61ca Mon Sep 17 00:00:00 2001 From: ulysses-you Date: Fri, 24 May 2024 14:31:52 +0800 Subject: [PATCH 05/16] [SPARK-46090][SQL] Support plan fragment level SQL configs in AQE ### What changes were proposed in this pull request? This pr introduces `case class AdaptiveRuleContext(isSubquery: Boolean, isFinalStage: Boolean)` which can be used inside adaptive sql extension rules through thread local, so that developers can modify the next plan fragment configs using `AdaptiveRuleContext.get()`. The plan fragment configs can be propagated through multi-phases, e.g., if set a config in `queryPostPlannerStrategyRules` then the config can be gotten in `queryStagePrepRules`, `queryStageOptimizerRules` and `columnarRules`. The configs will be cleanup before going to execute, so in next round the configs will be empty. ### Why are the changes needed? To support modify the plan fragment level SQL configs through AQE rules. ### Does this PR introduce _any_ user-facing change? no, only affect developers. ### How was this patch tested? add new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #44013 from ulysses-you/rule-context. Lead-authored-by: ulysses-you Co-authored-by: Kent Yao Signed-off-by: youxiduo --- .../adaptive/AdaptiveRuleContext.scala | 89 +++++++++ .../adaptive/AdaptiveSparkPlanExec.scala | 42 ++++- .../adaptive/AdaptiveRuleContextSuite.scala | 176 ++++++++++++++++++ 3 files changed, 299 insertions(+), 8 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveRuleContext.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveRuleContextSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveRuleContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveRuleContext.scala new file mode 100644 index 0000000000000..fce20b79e1136 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveRuleContext.scala @@ -0,0 +1,89 @@ +/* + * 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.execution.adaptive + +import scala.collection.mutable + +import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.catalyst.SQLConfHelper + +/** + * Provide the functionality to modify the next plan fragment configs in AQE rules. + * The configs will be cleanup before going to execute next plan fragment. + * To get instance, use: {{{ AdaptiveRuleContext.get() }}} + * + * @param isSubquery if the input query plan is subquery + * @param isFinalStage if the next stage is final stage + */ +@Experimental +@DeveloperApi +case class AdaptiveRuleContext(isSubquery: Boolean, isFinalStage: Boolean) { + + /** + * Set SQL configs for next plan fragment. The configs will affect all of rules in AQE, + * i.e., the runtime optimizer, planner, queryStagePreparationRules, queryStageOptimizerRules, + * columnarRules. + * This configs will be cleared before going to get the next plan fragment. + */ + private val nextPlanFragmentConf = new mutable.HashMap[String, String]() + + private[sql] def withFinalStage(isFinalStage: Boolean): AdaptiveRuleContext = { + if (this.isFinalStage == isFinalStage) { + this + } else { + val newRuleContext = copy(isFinalStage = isFinalStage) + newRuleContext.setConfigs(this.configs()) + newRuleContext + } + } + + def setConfig(key: String, value: String): Unit = { + nextPlanFragmentConf.put(key, value) + } + + def setConfigs(kvs: Map[String, String]): Unit = { + kvs.foreach(kv => nextPlanFragmentConf.put(kv._1, kv._2)) + } + + private[sql] def configs(): Map[String, String] = nextPlanFragmentConf.toMap + + private[sql] def clearConfigs(): Unit = nextPlanFragmentConf.clear() +} + +object AdaptiveRuleContext extends SQLConfHelper { + private val ruleContextThreadLocal = new ThreadLocal[AdaptiveRuleContext] + + /** + * If a rule is applied inside AQE then the returned value is always defined, else return None. + */ + def get(): Option[AdaptiveRuleContext] = Option(ruleContextThreadLocal.get()) + + private[sql] def withRuleContext[T](ruleContext: AdaptiveRuleContext)(block: => T): T = { + assert(ruleContext != null) + val origin = ruleContextThreadLocal.get() + ruleContextThreadLocal.set(ruleContext) + try { + val conf = ruleContext.configs() + withSQLConf(conf.toSeq: _*) { + block + } + } finally { + ruleContextThreadLocal.set(origin) + } + } +} 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 f30ffaf515664..f21960aeedd64 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 @@ -85,6 +85,25 @@ case class AdaptiveSparkPlanExec( case _ => logDebug(_) } + @transient private var ruleContext = new AdaptiveRuleContext( + isSubquery = isSubquery, + isFinalStage = false) + + private def withRuleContext[T](f: => T): T = + AdaptiveRuleContext.withRuleContext(ruleContext) { f } + + private def applyPhysicalRulesWithRuleContext( + plan: => SparkPlan, + rules: Seq[Rule[SparkPlan]], + loggerAndBatchName: Option[(PlanChangeLogger[SparkPlan], String)] = None): SparkPlan = { + // Apply the last rules if exists before going to apply the next batch of rules, + // so that we can propagate the configs. + val newPlan = plan + withRuleContext { + applyPhysicalRules(newPlan, rules, loggerAndBatchName) + } + } + @transient private val planChangeLogger = new PlanChangeLogger[SparkPlan]() // The logical plan optimizer for re-optimizing the current logical plan. @@ -161,7 +180,9 @@ case class AdaptiveSparkPlanExec( collapseCodegenStagesRule ) - private def optimizeQueryStage(plan: SparkPlan, isFinalStage: Boolean): SparkPlan = { + private def optimizeQueryStage( + plan: SparkPlan, + isFinalStage: Boolean): SparkPlan = withRuleContext { val rules = if (isFinalStage && !conf.getConf(SQLConf.ADAPTIVE_EXECUTION_APPLY_FINAL_STAGE_SHUFFLE_OPTIMIZATIONS)) { queryStageOptimizerRules.filterNot(_.isInstanceOf[AQEShuffleReadRule]) @@ -197,7 +218,7 @@ case class AdaptiveSparkPlanExec( } private def applyQueryPostPlannerStrategyRules(plan: SparkPlan): SparkPlan = { - applyPhysicalRules( + applyPhysicalRulesWithRuleContext( plan, context.session.sessionState.adaptiveRulesHolder.queryPostPlannerStrategyRules, Some((planChangeLogger, "AQE Query Post Planner Strategy Rules")) @@ -205,7 +226,7 @@ case class AdaptiveSparkPlanExec( } @transient val initialPlan = context.session.withActive { - applyPhysicalRules( + applyPhysicalRulesWithRuleContext( applyQueryPostPlannerStrategyRules(inputPlan), queryStagePreparationRules, Some((planChangeLogger, "AQE Preparations"))) @@ -282,6 +303,7 @@ case class AdaptiveSparkPlanExec( val errors = new mutable.ArrayBuffer[Throwable]() var stagesToReplace = Seq.empty[QueryStageExec] while (!result.allChildStagesMaterialized) { + ruleContext.clearConfigs() currentPhysicalPlan = result.newPlan if (result.newStages.nonEmpty) { stagesToReplace = result.newStages ++ stagesToReplace @@ -373,11 +395,13 @@ case class AdaptiveSparkPlanExec( result = createQueryStages(currentPhysicalPlan) } + ruleContext = ruleContext.withFinalStage(isFinalStage = true) // Run the final plan when there's no more unfinished stages. - currentPhysicalPlan = applyPhysicalRules( + currentPhysicalPlan = applyPhysicalRulesWithRuleContext( optimizeQueryStage(result.newPlan, isFinalStage = true), postStageCreationRules(supportsColumnar), Some((planChangeLogger, "AQE Post Stage Creation"))) + ruleContext.clearConfigs() _isFinalPlan = true executionId.foreach(onUpdatePlan(_, Seq(currentPhysicalPlan))) currentPhysicalPlan @@ -595,7 +619,7 @@ case class AdaptiveSparkPlanExec( val queryStage = plan match { case e: Exchange => val optimized = e.withNewChildren(Seq(optimizeQueryStage(e.child, isFinalStage = false))) - val newPlan = applyPhysicalRules( + val newPlan = applyPhysicalRulesWithRuleContext( optimized, postStageCreationRules(outputsColumnar = plan.supportsColumnar), Some((planChangeLogger, "AQE Post Stage Creation"))) @@ -722,9 +746,11 @@ case class AdaptiveSparkPlanExec( private def reOptimize(logicalPlan: LogicalPlan): Option[(SparkPlan, LogicalPlan)] = { try { logicalPlan.invalidateStatsCache() - val optimized = optimizer.execute(logicalPlan) - val sparkPlan = context.session.sessionState.planner.plan(ReturnAnswer(optimized)).next() - val newPlan = applyPhysicalRules( + val optimized = withRuleContext { optimizer.execute(logicalPlan) } + val sparkPlan = withRuleContext { + context.session.sessionState.planner.plan(ReturnAnswer(optimized)).next() + } + val newPlan = applyPhysicalRulesWithRuleContext( applyQueryPostPlannerStrategyRules(sparkPlan), preprocessingRules ++ queryStagePreparationRules, Some((planChangeLogger, "AQE Replanning"))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveRuleContextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveRuleContextSuite.scala new file mode 100644 index 0000000000000..04c9e6c946b45 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveRuleContextSuite.scala @@ -0,0 +1,176 @@ +/* + * 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.execution.adaptive + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.{SparkSession, SparkSessionExtensionsProvider} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{ColumnarRule, RangeExec, SparkPlan, SparkStrategy} +import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec + +class AdaptiveRuleContextSuite extends SparkFunSuite with AdaptiveSparkPlanHelper { + + private def stop(spark: SparkSession): Unit = { + spark.stop() + SparkSession.clearActiveSession() + SparkSession.clearDefaultSession() + } + + private def withSession( + builders: Seq[SparkSessionExtensionsProvider])(f: SparkSession => Unit): Unit = { + val builder = SparkSession.builder().master("local[1]") + builders.foreach(builder.withExtensions) + val spark = builder.getOrCreate() + try f(spark) finally { + stop(spark) + } + } + + test("test adaptive rule context") { + withSession( + Seq(_.injectRuntimeOptimizerRule(_ => MyRuleContextForRuntimeOptimization), + _.injectPlannerStrategy(_ => MyRuleContextForPlannerStrategy), + _.injectQueryPostPlannerStrategyRule(_ => MyRuleContextForPostPlannerStrategyRule), + _.injectQueryStagePrepRule(_ => MyRuleContextForPreQueryStageRule), + _.injectQueryStageOptimizerRule(_ => MyRuleContextForQueryStageRule), + _.injectColumnar(_ => MyRuleContextForColumnarRule))) { spark => + val df = spark.range(1, 10, 1, 3).selectExpr("id % 3 as c").groupBy("c").count() + df.collect() + assert(collectFirst(df.queryExecution.executedPlan) { + case s: ShuffleExchangeExec if s.numPartitions == 2 => s + }.isDefined) + } + } + + test("test adaptive rule context with subquery") { + withSession( + Seq(_.injectQueryStagePrepRule(_ => MyRuleContextForQueryStageWithSubquery))) { spark => + spark.sql("select (select count(*) from range(10)), id from range(10)").collect() + } + } +} + +object MyRuleContext { + def checkAndGetRuleContext(): AdaptiveRuleContext = { + val ruleContextOpt = AdaptiveRuleContext.get() + assert(ruleContextOpt.isDefined) + ruleContextOpt.get + } + + def checkRuleContextForQueryStage(plan: SparkPlan): SparkPlan = { + val ruleContext = checkAndGetRuleContext() + assert(!ruleContext.isSubquery) + val stage = plan.find(_.isInstanceOf[ShuffleQueryStageExec]) + if (stage.isDefined && stage.get.asInstanceOf[ShuffleQueryStageExec].isMaterialized) { + assert(ruleContext.isFinalStage) + assert(!ruleContext.configs().get("spark.sql.shuffle.partitions").contains("2")) + } else { + assert(!ruleContext.isFinalStage) + assert(ruleContext.configs().get("spark.sql.shuffle.partitions").contains("2")) + } + plan + } +} + +object MyRuleContextForRuntimeOptimization extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = { + MyRuleContext.checkAndGetRuleContext() + plan + } +} + +object MyRuleContextForPlannerStrategy extends SparkStrategy { + override def apply(plan: LogicalPlan): Seq[SparkPlan] = { + plan match { + case _: LogicalQueryStage => + val ruleContext = MyRuleContext.checkAndGetRuleContext() + assert(!ruleContext.configs().get("spark.sql.shuffle.partitions").contains("2")) + Nil + case _ => Nil + } + } +} + +object MyRuleContextForPostPlannerStrategyRule extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = { + val ruleContext = MyRuleContext.checkAndGetRuleContext() + if (plan.find(_.isInstanceOf[RangeExec]).isDefined) { + ruleContext.setConfig("spark.sql.shuffle.partitions", "2") + } + plan + } +} + +object MyRuleContextForPreQueryStageRule extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = { + val ruleContext = MyRuleContext.checkAndGetRuleContext() + assert(!ruleContext.isFinalStage) + plan + } +} + +object MyRuleContextForQueryStageRule extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = { + MyRuleContext.checkRuleContextForQueryStage(plan) + } +} + +object MyRuleContextForColumnarRule extends ColumnarRule { + override def preColumnarTransitions: Rule[SparkPlan] = { + plan: SparkPlan => { + if (plan.isInstanceOf[AdaptiveSparkPlanExec]) { + // skip if we are not inside AQE + assert(AdaptiveRuleContext.get().isEmpty) + plan + } else { + MyRuleContext.checkRuleContextForQueryStage(plan) + } + } + } + + override def postColumnarTransitions: Rule[SparkPlan] = { + plan: SparkPlan => { + if (plan.isInstanceOf[AdaptiveSparkPlanExec]) { + // skip if we are not inside AQE + assert(AdaptiveRuleContext.get().isEmpty) + plan + } else { + MyRuleContext.checkRuleContextForQueryStage(plan) + } + } + } +} + +object MyRuleContextForQueryStageWithSubquery extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = { + val ruleContext = MyRuleContext.checkAndGetRuleContext() + if (plan.exists(_.isInstanceOf[HashAggregateExec])) { + assert(ruleContext.isSubquery) + if (plan.exists(_.isInstanceOf[RangeExec])) { + assert(!ruleContext.isFinalStage) + } else { + assert(ruleContext.isFinalStage) + } + } else { + assert(!ruleContext.isSubquery) + } + plan + } +} From 3346afd4b250c3aead5a237666d4942018a463e0 Mon Sep 17 00:00:00 2001 From: ulysses-you Date: Fri, 24 May 2024 14:53:26 +0800 Subject: [PATCH 06/16] [SPARK-46090][SQL][FOLLOWUP] Add DeveloperApi import ### What changes were proposed in this pull request? Add DeveloperApi import ### Why are the changes needed? Fix compile issue ### Does this PR introduce _any_ user-facing change? Fix compile issue ### How was this patch tested? pass CI ### Was this patch authored or co-authored using generative AI tooling? no Closes #46730 from ulysses-you/hot-fix. Authored-by: ulysses-you Signed-off-by: Kent Yao --- .../spark/sql/execution/adaptive/AdaptiveRuleContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveRuleContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveRuleContext.scala index fce20b79e1136..23817be71c89c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveRuleContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveRuleContext.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.adaptive import scala.collection.mutable -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.sql.catalyst.SQLConfHelper /** From ef43bbbc11638b6ad3f02b9f4d74a6357ef09f13 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 24 May 2024 16:20:54 +0800 Subject: [PATCH 07/16] [SPARK-48384][BUILD] Exclude `io.netty:netty-tcnative-boringssl-static` from `zookeeper` ### What changes were proposed in this pull request? The pr aims to exclude `io.netty:netty-tcnative-boringssl-static` from `zookeeper`. ### Why are the changes needed? 1.According to: https://github.com/netty/netty-tcnative/blob/c9b4b6ab62cdbfb4aab6ab3efb8dd7cf73f353ad/boringssl-static/pom.xml#L970-L982 the `io.netty:netty-tcnative-boringssl-static` is composed of: `io.netty:netty-tcnative-boringssl-static:linux-aarch_64` `io.netty:netty-tcnative-boringssl-static:linux-x86_64` `io.netty:netty-tcnative-boringssl-static:osx-aarch_64` `io.netty:netty-tcnative-boringssl-static:osx-x86_64` `io.netty:netty-tcnative-boringssl-staticwindows-x86_64` and our `common/network-common/pom.xml` file already explicitly depends on them. 2.Their versions are different in `dev/deps/spark-deps-hadoop-3-hive-2.3` image Let's keep one version to avoid conflicts. 3.In the `pom.xml` file, zookeeper already has other `netty` related exclusions, eg: https://github.com/apache/spark/blob/master/pom.xml#L1838-L1842 image ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46695 from panbingkun/SPARK-48384. Authored-by: panbingkun Signed-off-by: yangjie01 --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 1 - pom.xml | 4 ++++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 46c5108e4eba4..61d7861f4469b 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -207,7 +207,6 @@ netty-common/4.1.109.Final//netty-common-4.1.109.Final.jar netty-handler-proxy/4.1.109.Final//netty-handler-proxy-4.1.109.Final.jar netty-handler/4.1.109.Final//netty-handler-4.1.109.Final.jar netty-resolver/4.1.109.Final//netty-resolver-4.1.109.Final.jar -netty-tcnative-boringssl-static/2.0.61.Final//netty-tcnative-boringssl-static-2.0.61.Final.jar netty-tcnative-boringssl-static/2.0.65.Final/linux-aarch_64/netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar netty-tcnative-boringssl-static/2.0.65.Final/linux-x86_64/netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar netty-tcnative-boringssl-static/2.0.65.Final/osx-aarch_64/netty-tcnative-boringssl-static-2.0.65.Final-osx-aarch_64.jar diff --git a/pom.xml b/pom.xml index e8d47afa1cca5..eef7237ac12f9 100644 --- a/pom.xml +++ b/pom.xml @@ -1839,6 +1839,10 @@ io.netty netty-transport-native-epoll + + io.netty + netty-tcnative-boringssl-static + com.github.spotbugs spotbugs-annotations From b15b6cf1f537756eafbe8dd31a3b03dc500077f3 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 24 May 2024 17:04:38 +0800 Subject: [PATCH 08/16] [SPARK-48409][BUILD][TESTS] Upgrade MySQL & Postgres & Mariadb docker image version ### What changes were proposed in this pull request? The pr aims to upgrade some db docker image version, include: - `MySQL` from `8.3.0` to `8.4.0` - `Postgres` from `10.5.12` to `10.5.25` - `Mariadb` from `16.2-alpine` to `16.3-alpine` ### Why are the changes needed? Tests dependencies upgrading. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46704 from panbingkun/db_images_upgrade. Authored-by: panbingkun Signed-off-by: Kent Yao --- .../apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala | 6 +++--- .../org/apache/spark/sql/jdbc/MySQLDatabaseOnDocker.scala | 2 +- .../apache/spark/sql/jdbc/PostgresIntegrationSuite.scala | 6 +++--- .../apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala | 6 +++--- .../spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala | 6 +++--- .../spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala | 6 +++--- .../apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala | 6 +++--- .../apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala | 6 +++--- 8 files changed, 22 insertions(+), 22 deletions(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala index 6825c001f7670..efb2fa09f6a3f 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala @@ -25,9 +25,9 @@ import org.apache.spark.sql.execution.datasources.jdbc.connection.SecureConnecti import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., mariadb:10.5.12): + * To run this test suite for a specific version (e.g., mariadb:10.5.25): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 MARIADB_DOCKER_IMAGE_NAME=mariadb:10.5.12 + * ENABLE_DOCKER_INTEGRATION_TESTS=1 MARIADB_DOCKER_IMAGE_NAME=mariadb:10.5.25 * ./build/sbt -Pdocker-integration-tests * "docker-integration-tests/testOnly org.apache.spark.sql.jdbc.MariaDBKrbIntegrationSuite" * }}} @@ -38,7 +38,7 @@ class MariaDBKrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { override protected val keytabFileName = "mariadb.keytab" override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("MARIADB_DOCKER_IMAGE_NAME", "mariadb:10.5.12") + override val imageName = sys.env.getOrElse("MARIADB_DOCKER_IMAGE_NAME", "mariadb:10.5.25") override val env = Map( "MYSQL_ROOT_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLDatabaseOnDocker.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLDatabaseOnDocker.scala index 568eb5f109731..570a81ac3947f 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLDatabaseOnDocker.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLDatabaseOnDocker.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.jdbc class MySQLDatabaseOnDocker extends DatabaseOnDocker { - override val imageName = sys.env.getOrElse("MYSQL_DOCKER_IMAGE_NAME", "mysql:8.3.0") + override val imageName = sys.env.getOrElse("MYSQL_DOCKER_IMAGE_NAME", "mysql:8.4.0") override val env = Map( "MYSQL_ROOT_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala index 5ad4f15216b74..12a71dbd7c7f8 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala @@ -32,9 +32,9 @@ import org.apache.spark.sql.types._ import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., postgres:16.2): + * To run this test suite for a specific version (e.g., postgres:16.3-alpine): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:16.2 + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:16.3-alpine * ./build/sbt -Pdocker-integration-tests * "docker-integration-tests/testOnly org.apache.spark.sql.jdbc.PostgresIntegrationSuite" * }}} @@ -42,7 +42,7 @@ import org.apache.spark.tags.DockerTest @DockerTest class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:16.2-alpine") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:16.3-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala index d08be3b5f40e3..af1cd464ad5fe 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala @@ -25,9 +25,9 @@ import org.apache.spark.sql.execution.datasources.jdbc.connection.SecureConnecti import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., postgres:16.2): + * To run this test suite for a specific version (e.g., postgres:16.3-alpine): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:16.2 + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:16.3-alpine * ./build/sbt -Pdocker-integration-tests * "docker-integration-tests/testOnly *PostgresKrbIntegrationSuite" * }}} @@ -38,7 +38,7 @@ class PostgresKrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { override protected val keytabFileName = "postgres.keytab" override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:16.2") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:16.3-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala index 7ae03e974845b..8b27e9cb0e0a3 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala @@ -28,9 +28,9 @@ import org.apache.spark.tags.DockerTest /** * This suite is used to generate subqueries, and test Spark against Postgres. - * To run this test suite for a specific version (e.g., postgres:16.2): + * To run this test suite for a specific version (e.g., postgres:16.3-alpine): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:16.2 + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:16.3-alpine * ./build/sbt -Pdocker-integration-tests * "docker-integration-tests/testOnly org.apache.spark.sql.jdbc.GeneratedSubquerySuite" * }}} @@ -39,7 +39,7 @@ import org.apache.spark.tags.DockerTest class GeneratedSubquerySuite extends DockerJDBCIntegrationSuite with QueryGeneratorHelper { override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:16.2-alpine") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:16.3-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala index f2a7e14cfc4b9..de28e16b325ce 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala @@ -30,9 +30,9 @@ import org.apache.spark.tags.DockerTest * confidence, and you won't have to manually verify the golden files generated with your test. * 2. Add this line to your .sql file: --ONLY_IF spark * - * Note: To run this test suite for a specific version (e.g., postgres:16.2): + * Note: To run this test suite for a specific version (e.g., postgres:16.3-alpine): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:16.2 + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:16.3-alpine * ./build/sbt -Pdocker-integration-tests * "testOnly org.apache.spark.sql.jdbc.PostgreSQLQueryTestSuite" * }}} @@ -45,7 +45,7 @@ class PostgreSQLQueryTestSuite extends CrossDbmsQueryTestSuite { protected val customInputFilePath: String = new File(inputFilePath, "subquery").getAbsolutePath override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:16.2-alpine") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:16.3-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala index 7fef3ccd6b3f6..7c439d449d86f 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala @@ -28,9 +28,9 @@ import org.apache.spark.sql.types._ import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., postgres:16.2) + * To run this test suite for a specific version (e.g., postgres:16.3-alpine) * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:16.2 + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:16.3-alpine * ./build/sbt -Pdocker-integration-tests "testOnly *v2.PostgresIntegrationSuite" * }}} */ @@ -38,7 +38,7 @@ import org.apache.spark.tags.DockerTest class PostgresIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest { override val catalogName: String = "postgresql" override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:16.2-alpine") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:16.3-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala index 838de5acab0df..8a2d0ded84381 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala @@ -26,16 +26,16 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., postgres:16.2): + * To run this test suite for a specific version (e.g., postgres:16.3-alpine): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:16.2 + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:16.3-alpine * ./build/sbt -Pdocker-integration-tests "testOnly *v2.PostgresNamespaceSuite" * }}} */ @DockerTest class PostgresNamespaceSuite extends DockerJDBCIntegrationSuite with V2JDBCNamespaceTest { override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:16.2-alpine") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:16.3-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) From bd95040c3170aaed61ee5e9090d1b8580351ee80 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 24 May 2024 17:36:46 +0800 Subject: [PATCH 09/16] [SPARK-48412][PYTHON] Refactor data type json parse ### What changes were proposed in this pull request? Refactor data type json parse ### Why are the changes needed? the `_all_atomic_types` causes confusions: - it is only used in json parse, so it should use the `jsonValue` instead of `typeName` (and so it causes the `typeName` not consistent with Scala, will fix in separate PR); - not all atomic types are included in it (e.g. `YearMonthIntervalType`); - not all atomic types should be placed in it (e.g. `VarcharType` which has to be excluded here and there) ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ci, added tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #46733 from zhengruifeng/refactor_json_parse. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/tests/test_types.py | 42 +++++++++++++++++-- python/pyspark/sql/types.py | 57 ++++++++++++++++++-------- 2 files changed, 79 insertions(+), 20 deletions(-) diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index 6c64a9471363a..d665053d94904 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -1136,12 +1136,46 @@ def test_struct_type(self): self.assertRaises(IndexError, lambda: struct1[9]) self.assertRaises(TypeError, lambda: struct1[9.9]) + def test_parse_datatype_json_string(self): + from pyspark.sql.types import _parse_datatype_json_string + + for dataType in [ + StringType(), + CharType(5), + VarcharType(10), + BinaryType(), + BooleanType(), + DecimalType(), + DecimalType(10, 2), + FloatType(), + DoubleType(), + ByteType(), + ShortType(), + IntegerType(), + LongType(), + DateType(), + TimestampType(), + TimestampNTZType(), + NullType(), + VariantType(), + YearMonthIntervalType(), + YearMonthIntervalType(YearMonthIntervalType.YEAR), + YearMonthIntervalType(YearMonthIntervalType.YEAR, YearMonthIntervalType.MONTH), + DayTimeIntervalType(), + DayTimeIntervalType(DayTimeIntervalType.DAY), + DayTimeIntervalType(DayTimeIntervalType.HOUR, DayTimeIntervalType.SECOND), + CalendarIntervalType(), + ]: + json_str = dataType.json() + parsed = _parse_datatype_json_string(json_str) + self.assertEqual(dataType, parsed) + def test_parse_datatype_string(self): - from pyspark.sql.types import _all_atomic_types, _parse_datatype_string + from pyspark.sql.types import _all_mappable_types, _parse_datatype_string + + for k, t in _all_mappable_types.items(): + self.assertEqual(t(), _parse_datatype_string(k)) - for k, t in _all_atomic_types.items(): - if k != "varchar" and k != "char": - self.assertEqual(t(), _parse_datatype_string(k)) self.assertEqual(IntegerType(), _parse_datatype_string("int")) self.assertEqual(StringType(), _parse_datatype_string("string")) self.assertEqual(CharType(1), _parse_datatype_string("char(1)")) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 17b019240f826..b9db59e0a58ac 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1756,13 +1756,45 @@ def toJson(self, zone_id: str = "UTC") -> str: TimestampNTZType, NullType, VariantType, + YearMonthIntervalType, + DayTimeIntervalType, ] -_all_atomic_types: Dict[str, Type[DataType]] = dict((t.typeName(), t) for t in _atomic_types) -_complex_types: List[Type[Union[ArrayType, MapType, StructType]]] = [ArrayType, MapType, StructType] -_all_complex_types: Dict[str, Type[Union[ArrayType, MapType, StructType]]] = dict( - (v.typeName(), v) for v in _complex_types -) +_complex_types: List[Type[Union[ArrayType, MapType, StructType]]] = [ + ArrayType, + MapType, + StructType, +] +_all_complex_types: Dict[str, Type[Union[ArrayType, MapType, StructType]]] = { + "array": ArrayType, + "map": MapType, + "struct": StructType, +} + +# Datatypes that can be directly parsed by mapping a json string without regex. +# This dict should be only used in json parsing. +# Note that: +# 1, CharType and VarcharType are not listed here, since they need regex; +# 2, DecimalType can be parsed by both mapping ('decimal') and regex ('decimal(10, 2)'); +# 3, CalendarIntervalType is not an atomic type, but can be mapped by 'interval'; +_all_mappable_types: Dict[str, Type[DataType]] = { + "string": StringType, + "binary": BinaryType, + "boolean": BooleanType, + "decimal": DecimalType, + "float": FloatType, + "double": DoubleType, + "byte": ByteType, + "short": ShortType, + "integer": IntegerType, + "long": LongType, + "date": DateType, + "timestamp": TimestampType, + "timestamp_ntz": TimestampNTZType, + "void": NullType, + "variant": VariantType, + "interval": CalendarIntervalType, +} _LENGTH_CHAR = re.compile(r"char\(\s*(\d+)\s*\)") _LENGTH_VARCHAR = re.compile(r"varchar\(\s*(\d+)\s*\)") @@ -1887,11 +1919,8 @@ def _parse_datatype_json_string(json_string: str) -> DataType: ... python_datatype = _parse_datatype_json_string(scala_datatype.json()) ... assert datatype == python_datatype ... - >>> for cls in _all_atomic_types.values(): - ... if cls is not VarcharType and cls is not CharType: - ... check_datatype(cls()) - ... else: - ... check_datatype(cls(1)) + >>> for cls in _all_mappable_types.values(): + ... check_datatype(cls()) >>> # Simple ArrayType. >>> simple_arraytype = ArrayType(StringType(), True) @@ -1938,14 +1967,12 @@ def _parse_datatype_json_value( collationsMap: Optional[Dict[str, str]] = None, ) -> DataType: if not isinstance(json_value, dict): - if json_value in _all_atomic_types.keys(): + if json_value in _all_mappable_types.keys(): if collationsMap is not None and fieldPath in collationsMap: _assert_valid_type_for_collation(fieldPath, json_value, collationsMap) collation_name = collationsMap[fieldPath] return StringType(collation_name) - return _all_atomic_types[json_value]() - elif json_value == "decimal": - return DecimalType() + return _all_mappable_types[json_value]() elif _FIXED_DECIMAL.match(json_value): m = _FIXED_DECIMAL.match(json_value) return DecimalType(int(m.group(1)), int(m.group(2))) # type: ignore[union-attr] @@ -1965,8 +1992,6 @@ def _parse_datatype_json_value( if first_field is not None and second_field is None: return YearMonthIntervalType(first_field) return YearMonthIntervalType(first_field, second_field) - elif json_value == "interval": - return CalendarIntervalType() elif _LENGTH_CHAR.match(json_value): m = _LENGTH_CHAR.match(json_value) return CharType(int(m.group(1))) # type: ignore[union-attr] From 7ae939ae12a68b8664af4e4d9bfe11902ec3494d Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 24 May 2024 18:20:14 +0800 Subject: [PATCH 10/16] [SPARK-48168][SQL] Add bitwise shifting operators support ### What changes were proposed in this pull request? This PR introduces three bitwise shifting operators as aliases for existing shifting functions. ### Why are the changes needed? The bit shifting functions named in alphabet form vary from one platform to anthor. Take our shiftleft as an example, - Hive, shiftleft (where we copied it from) - MsSQL Server LEFT_SHIFT - MySQL, N/A - PostgreSQL, N/A - Presto, bitwise_left_shift The [bit shifting operators](https://en.wikipedia.org/wiki/Bitwise_operations_in_C) share a much more common and consistent way for users to port their queries. For self-consistent with existing bit operators in Spark, `AND &`, `OR |`, `XOR ^` and `NOT ~`, we now add `<<`, `>>` and `>>>`. For other systems that we can refer to: https://learn.microsoft.com/en-us/sql/t-sql/functions/left-shift-transact-sql?view=sql-server-ver16 https://www.postgresql.org/docs/9.4/functions-bitstring.html https://dev.mysql.com/doc/refman/8.0/en/bit-functions.html ### Does this PR introduce _any_ user-facing change? Yes, new operators were added but no behavior change ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #46440 from yaooqinn/SPARK-48168. Authored-by: Kent Yao Signed-off-by: youxiduo --- .../function_shiftleft.explain | 2 +- .../function_shiftright.explain | 2 +- .../function_shiftrightunsigned.explain | 2 +- .../grouping_and_grouping_id.explain | 2 +- .../spark/sql/catalyst/parser/SqlBaseLexer.g4 | 40 +++++- .../sql/catalyst/parser/SqlBaseParser.g4 | 8 ++ .../catalyst/analysis/FunctionRegistry.scala | 3 + .../expressions/mathExpressions.scala | 134 +++++++++--------- .../sql/catalyst/parser/AstBuilder.scala | 11 ++ .../spark/sql/catalyst/SQLKeywordSuite.scala | 2 +- .../sql-functions/sql-expression-schema.md | 9 +- .../analyzer-results/bitwise.sql.out | 112 +++++++++++++++ .../analyzer-results/group-analytics.sql.out | 10 +- .../analyzer-results/grouping_set.sql.out | 6 +- .../postgreSQL/groupingsets.sql.out | 44 +++--- .../analyzer-results/postgreSQL/int2.sql.out | 4 +- .../analyzer-results/postgreSQL/int4.sql.out | 4 +- .../analyzer-results/postgreSQL/int8.sql.out | 4 +- .../udf/udf-group-analytics.sql.out | 10 +- .../resources/sql-tests/inputs/bitwise.sql | 12 ++ .../sql-tests/results/bitwise.sql.out | 128 +++++++++++++++++ .../sql-tests/results/postgreSQL/int2.sql.out | 4 +- .../sql-tests/results/postgreSQL/int4.sql.out | 4 +- .../sql-tests/results/postgreSQL/int8.sql.out | 2 +- .../approved-plans-v1_4/q17/explain.txt | 2 +- .../approved-plans-v1_4/q25/explain.txt | 2 +- .../approved-plans-v1_4/q27.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q27/explain.txt | 2 +- .../approved-plans-v1_4/q29/explain.txt | 2 +- .../approved-plans-v1_4/q36.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q36/explain.txt | 2 +- .../approved-plans-v1_4/q39a/explain.txt | 2 +- .../approved-plans-v1_4/q39b/explain.txt | 2 +- .../approved-plans-v1_4/q49/explain.txt | 6 +- .../approved-plans-v1_4/q5/explain.txt | 2 +- .../approved-plans-v1_4/q64/explain.txt | 4 +- .../approved-plans-v1_4/q70.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q70/explain.txt | 2 +- .../approved-plans-v1_4/q72/explain.txt | 2 +- .../approved-plans-v1_4/q85/explain.txt | 2 +- .../approved-plans-v1_4/q86.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q86/explain.txt | 2 +- .../approved-plans-v2_7/q24.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q49/explain.txt | 6 +- .../approved-plans-v2_7/q5a/explain.txt | 2 +- .../approved-plans-v2_7/q64/explain.txt | 4 +- .../approved-plans-v2_7/q72/explain.txt | 2 +- .../sql/expressions/ExpressionInfoSuite.scala | 5 +- 48 files changed, 469 insertions(+), 153 deletions(-) diff --git a/connector/connect/common/src/test/resources/query-tests/explain-results/function_shiftleft.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/function_shiftleft.explain index f89a8be7ceedb..6d5eb29944d52 100644 --- a/connector/connect/common/src/test/resources/query-tests/explain-results/function_shiftleft.explain +++ b/connector/connect/common/src/test/resources/query-tests/explain-results/function_shiftleft.explain @@ -1,2 +1,2 @@ -Project [shiftleft(cast(b#0 as int), 2) AS shiftleft(b, 2)#0] +Project [(cast(b#0 as int) << 2) AS (b << 2)#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_shiftright.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/function_shiftright.explain index b436f52e912b5..b1c2c35ac2d0e 100644 --- a/connector/connect/common/src/test/resources/query-tests/explain-results/function_shiftright.explain +++ b/connector/connect/common/src/test/resources/query-tests/explain-results/function_shiftright.explain @@ -1,2 +1,2 @@ -Project [shiftright(cast(b#0 as int), 2) AS shiftright(b, 2)#0] +Project [(cast(b#0 as int) >> 2) AS (b >> 2)#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_shiftrightunsigned.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/function_shiftrightunsigned.explain index 282ad156b3825..508c78a7f421f 100644 --- a/connector/connect/common/src/test/resources/query-tests/explain-results/function_shiftrightunsigned.explain +++ b/connector/connect/common/src/test/resources/query-tests/explain-results/function_shiftrightunsigned.explain @@ -1,2 +1,2 @@ -Project [shiftrightunsigned(cast(b#0 as int), 2) AS shiftrightunsigned(b, 2)#0] +Project [(cast(b#0 as int) >>> 2) AS (b >>> 2)#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/grouping_and_grouping_id.explain b/connector/connect/common/src/test/resources/query-tests/explain-results/grouping_and_grouping_id.explain index 3b7d6fb2b7072..f46fa38989ed4 100644 --- a/connector/connect/common/src/test/resources/query-tests/explain-results/grouping_and_grouping_id.explain +++ b/connector/connect/common/src/test/resources/query-tests/explain-results/grouping_and_grouping_id.explain @@ -1,4 +1,4 @@ -Aggregate [a#0, b#0, spark_grouping_id#0L], [a#0, b#0, cast((shiftright(spark_grouping_id#0L, 1) & 1) as tinyint) AS grouping(a)#0, cast((shiftright(spark_grouping_id#0L, 0) & 1) as tinyint) AS grouping(b)#0, spark_grouping_id#0L AS grouping_id(a, b)#0L] +Aggregate [a#0, b#0, spark_grouping_id#0L], [a#0, b#0, cast(((spark_grouping_id#0L >> 1) & 1) as tinyint) AS grouping(a)#0, cast(((spark_grouping_id#0L >> 0) & 1) as tinyint) AS grouping(b)#0, spark_grouping_id#0L AS grouping_id(a, b)#0L] +- Expand [[id#0L, a#0, b#0, a#0, b#0, 0], [id#0L, a#0, b#0, a#0, null, 1], [id#0L, a#0, b#0, null, b#0, 2], [id#0L, a#0, b#0, null, null, 3]], [id#0L, a#0, b#0, a#0, b#0, spark_grouping_id#0L] +- Project [id#0L, a#0, b#0, a#0 AS a#0, b#0 AS b#0] +- LocalRelation , [id#0L, a#0, b#0] diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 index 25a06a5b98cf7..a9705c1733df5 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 @@ -69,6 +69,35 @@ lexer grammar SqlBaseLexer; public void markUnclosedComment() { has_unclosed_bracketed_comment = true; } + + /** + * When greater than zero, it's in the middle of parsing ARRAY/MAP/STRUCT type. + */ + public int complex_type_level_counter = 0; + + /** + * Increase the counter by one when hits KEYWORD 'ARRAY', 'MAP', 'STRUCT'. + */ + public void incComplexTypeLevelCounter() { + complex_type_level_counter++; + } + + /** + * Decrease the counter by one when hits close tag '>' && the counter greater than zero + * which means we are in the middle of complex type parsing. Otherwise, it's a dangling + * GT token and we do nothing. + */ + public void decComplexTypeLevelCounter() { + if (complex_type_level_counter > 0) complex_type_level_counter--; + } + + /** + * If the counter is zero, it's a shift right operator. It can be closing tags of an complex + * type definition, such as MAP>. + */ + public boolean isShiftRightOperator() { + return complex_type_level_counter == 0 ? true : false; + } } SEMICOLON: ';'; @@ -100,7 +129,7 @@ ANTI: 'ANTI'; ANY: 'ANY'; ANY_VALUE: 'ANY_VALUE'; ARCHIVE: 'ARCHIVE'; -ARRAY: 'ARRAY'; +ARRAY: 'ARRAY' {incComplexTypeLevelCounter();}; AS: 'AS'; ASC: 'ASC'; AT: 'AT'; @@ -259,7 +288,7 @@ LOCKS: 'LOCKS'; LOGICAL: 'LOGICAL'; LONG: 'LONG'; MACRO: 'MACRO'; -MAP: 'MAP'; +MAP: 'MAP' {incComplexTypeLevelCounter();}; MATCHED: 'MATCHED'; MERGE: 'MERGE'; MICROSECOND: 'MICROSECOND'; @@ -362,7 +391,7 @@ STATISTICS: 'STATISTICS'; STORED: 'STORED'; STRATIFY: 'STRATIFY'; STRING: 'STRING'; -STRUCT: 'STRUCT'; +STRUCT: 'STRUCT' {incComplexTypeLevelCounter();}; SUBSTR: 'SUBSTR'; SUBSTRING: 'SUBSTRING'; SYNC: 'SYNC'; @@ -439,8 +468,11 @@ NEQ : '<>'; NEQJ: '!='; LT : '<'; LTE : '<=' | '!>'; -GT : '>'; +GT : '>' {decComplexTypeLevelCounter();}; GTE : '>=' | '!<'; +SHIFT_LEFT: '<<'; +SHIFT_RIGHT: '>>' {isShiftRightOperator()}?; +SHIFT_RIGHT_UNSIGNED: '>>>' {isShiftRightOperator()}?; PLUS: '+'; MINUS: '-'; 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 a87ecd02fb3a4..f0c0adb881212 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 @@ -395,6 +395,7 @@ describeFuncName | comparisonOperator | arithmeticOperator | predicateOperator + | shiftOperator | BANG ; @@ -989,6 +990,13 @@ valueExpression | left=valueExpression operator=HAT right=valueExpression #arithmeticBinary | left=valueExpression operator=PIPE right=valueExpression #arithmeticBinary | left=valueExpression comparisonOperator right=valueExpression #comparison + | left=valueExpression shiftOperator right=valueExpression #shiftExpression + ; + +shiftOperator + : SHIFT_LEFT + | SHIFT_RIGHT + | SHIFT_RIGHT_UNSIGNED ; datetimeUnit diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 78126ce30af5e..3a418497fa537 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -800,6 +800,9 @@ object FunctionRegistry { expression[BitwiseNot]("~"), expression[BitwiseOr]("|"), expression[BitwiseXor]("^"), + expression[ShiftLeft]("<<", true, Some("4.0.0")), + expression[ShiftRight](">>", true, Some("4.0.0")), + expression[ShiftRightUnsigned](">>>", true, Some("4.0.0")), expression[BitwiseCount]("bit_count"), expression[BitAndAgg]("bit_and"), expression[BitOrAgg]("bit_or"), 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 dc50c18f2ebbf..4bb0c658eacf1 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 @@ -1261,6 +1261,41 @@ case class Pow(left: Expression, right: Expression) newLeft: Expression, newRight: Expression): Expression = copy(left = newLeft, right = newRight) } +sealed trait BitShiftOperation + extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant { + + def symbol: String + def shiftInt: (Int, Int) => Int + def shiftLong: (Long, Int) => Long + + override def inputTypes: Seq[AbstractDataType] = + Seq(TypeCollection(IntegerType, LongType), IntegerType) + + override def dataType: DataType = left.dataType + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, (left, right) => s"$left $symbol $right") + } + + override protected def nullSafeEval(input1: Any, input2: Any): Any = input1 match { + case l: jl.Long => shiftLong(l, input2.asInstanceOf[Int]) + case i: jl.Integer => shiftInt(i, input2.asInstanceOf[Int]) + } + + override def toString: String = { + getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse(symbol) match { + case alias if alias == symbol => s"($left $symbol $right)" + case _ => super.toString + } + } + + override def sql: String = { + getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse(symbol) match { + case alias if alias == symbol => s"(${left.sql} $symbol ${right.sql})" + case _ => super.sql + } + } +} /** * Bitwise left shift. @@ -1269,38 +1304,28 @@ case class Pow(left: Expression, right: Expression) * @param right number of bits to left shift. */ @ExpressionDescription( - usage = "_FUNC_(base, expr) - Bitwise left shift.", + usage = "base << exp - Bitwise left shift.", examples = """ Examples: - > SELECT _FUNC_(2, 1); + > SELECT shiftleft(2, 1); + 4 + > SELECT 2 << 1; 4 """, + note = """ + `<<` operator is added in Spark 4.0.0 as an alias for `shiftleft`. + """, since = "1.5.0", group = "bitwise_funcs") -case class ShiftLeft(left: Expression, right: Expression) - extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant { - - override def inputTypes: Seq[AbstractDataType] = - Seq(TypeCollection(IntegerType, LongType), IntegerType) - - override def dataType: DataType = left.dataType - - protected override def nullSafeEval(input1: Any, input2: Any): Any = { - input1 match { - case l: jl.Long => l << input2.asInstanceOf[jl.Integer] - case i: jl.Integer => i << input2.asInstanceOf[jl.Integer] - } - } - - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - defineCodeGen(ctx, ev, (left, right) => s"$left << $right") - } - +case class ShiftLeft(left: Expression, right: Expression) extends BitShiftOperation { + override def symbol: String = "<<" + override def shiftInt: (Int, Int) => Int = (x: Int, y: Int) => x << y + override def shiftLong: (Long, Int) => Long = (x: Long, y: Int) => x << y + val shift: (Number, Int) => Any = (x: Number, y: Int) => x.longValue() << y override protected def withNewChildrenInternal( newLeft: Expression, newRight: Expression): ShiftLeft = copy(left = newLeft, right = newRight) } - /** * Bitwise (signed) right shift. * @@ -1308,38 +1333,27 @@ case class ShiftLeft(left: Expression, right: Expression) * @param right number of bits to right shift. */ @ExpressionDescription( - usage = "_FUNC_(base, expr) - Bitwise (signed) right shift.", + usage = "base >> expr - Bitwise (signed) right shift.", examples = """ Examples: - > SELECT _FUNC_(4, 1); + > SELECT shiftright(4, 1); + 2 + > SELECT 4 >> 1; 2 """, + note = """ + `>>` operator is added in Spark 4.0.0 as an alias for `shiftright`. + """, since = "1.5.0", group = "bitwise_funcs") -case class ShiftRight(left: Expression, right: Expression) - extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant { - - override def inputTypes: Seq[AbstractDataType] = - Seq(TypeCollection(IntegerType, LongType), IntegerType) - - override def dataType: DataType = left.dataType - - protected override def nullSafeEval(input1: Any, input2: Any): Any = { - input1 match { - case l: jl.Long => l >> input2.asInstanceOf[jl.Integer] - case i: jl.Integer => i >> input2.asInstanceOf[jl.Integer] - } - } - - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - defineCodeGen(ctx, ev, (left, right) => s"$left >> $right") - } - +case class ShiftRight(left: Expression, right: Expression) extends BitShiftOperation { + override def symbol: String = ">>" + override def shiftInt: (Int, Int) => Int = (x: Int, y: Int) => x >> y + override def shiftLong: (Long, Int) => Long = (x: Long, y: Int) => x >> y override protected def withNewChildrenInternal( newLeft: Expression, newRight: Expression): ShiftRight = copy(left = newLeft, right = newRight) } - /** * Bitwise unsigned right shift, for integer and long data type. * @@ -1347,33 +1361,23 @@ case class ShiftRight(left: Expression, right: Expression) * @param right the number of bits to right shift. */ @ExpressionDescription( - usage = "_FUNC_(base, expr) - Bitwise unsigned right shift.", + usage = "base >>> expr - Bitwise unsigned right shift.", examples = """ Examples: - > SELECT _FUNC_(4, 1); + > SELECT shiftrightunsigned(4, 1); 2 + > SELECT 4 >>> 1; + 2 + """, + note = """ + `>>>` operator is added in Spark 4.0.0 as an alias for `shiftrightunsigned`. """, since = "1.5.0", group = "bitwise_funcs") -case class ShiftRightUnsigned(left: Expression, right: Expression) - extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant { - - override def inputTypes: Seq[AbstractDataType] = - Seq(TypeCollection(IntegerType, LongType), IntegerType) - - override def dataType: DataType = left.dataType - - protected override def nullSafeEval(input1: Any, input2: Any): Any = { - input1 match { - case l: jl.Long => l >>> input2.asInstanceOf[jl.Integer] - case i: jl.Integer => i >>> input2.asInstanceOf[jl.Integer] - } - } - - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - defineCodeGen(ctx, ev, (left, right) => s"$left >>> $right") - } - +case class ShiftRightUnsigned(left: Expression, right: Expression) extends BitShiftOperation { + override def symbol: String = ">>>" + override def shiftInt: (Int, Int) => Int = (x: Int, y: Int) => x >>> y + override def shiftLong: (Long, Int) => Long = (x: Long, y: Int) => x >>> y override protected def withNewChildrenInternal( newLeft: Expression, newRight: Expression): ShiftRightUnsigned = copy(left = newLeft, right = newRight) 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 b6816f5bb2925..52c32530f2e92 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 @@ -2196,6 +2196,17 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { } } + override def visitShiftExpression(ctx: ShiftExpressionContext): Expression = withOrigin(ctx) { + val left = expression(ctx.left) + val right = expression(ctx.right) + val operator = ctx.shiftOperator().getChild(0).asInstanceOf[TerminalNode] + operator.getSymbol.getType match { + case SqlBaseParser.SHIFT_LEFT => ShiftLeft(left, right) + case SqlBaseParser.SHIFT_RIGHT => ShiftRight(left, right) + case SqlBaseParser.SHIFT_RIGHT_UNSIGNED => ShiftRightUnsigned(left, right) + } + } + /** * Create a unary arithmetic expression. The following arithmetic operators are supported: * - Plus: '+' diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SQLKeywordSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SQLKeywordSuite.scala index 8806431ab4395..9977dcd83d6af 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SQLKeywordSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SQLKeywordSuite.scala @@ -98,7 +98,7 @@ trait SQLKeywordUtils extends SparkFunSuite with SQLHelper { } (symbol, literals) :: Nil } else { - val literal = literalDef.replaceAll("'", "").trim + val literal = literalDef.split("\\{")(0).replaceAll("'", "").trim // The case where a symbol string and its literal string are different, // e.g., `SETMINUS: 'MINUS';`. if (symbol != literal) { diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index ca864dddf19b1..bf46fe91eb903 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -289,9 +289,12 @@ | org.apache.spark.sql.catalyst.expressions.Sha1 | sha | SELECT sha('Spark') | struct | | org.apache.spark.sql.catalyst.expressions.Sha1 | sha1 | SELECT sha1('Spark') | struct | | org.apache.spark.sql.catalyst.expressions.Sha2 | sha2 | SELECT sha2('Spark', 256) | struct | -| org.apache.spark.sql.catalyst.expressions.ShiftLeft | shiftleft | SELECT shiftleft(2, 1) | struct | -| org.apache.spark.sql.catalyst.expressions.ShiftRight | shiftright | SELECT shiftright(4, 1) | struct | -| org.apache.spark.sql.catalyst.expressions.ShiftRightUnsigned | shiftrightunsigned | SELECT shiftrightunsigned(4, 1) | struct | +| org.apache.spark.sql.catalyst.expressions.ShiftLeft | << | SELECT shiftleft(2, 1) | struct<(2 << 1):int> | +| org.apache.spark.sql.catalyst.expressions.ShiftLeft | shiftleft | SELECT shiftleft(2, 1) | struct<(2 << 1):int> | +| org.apache.spark.sql.catalyst.expressions.ShiftRight | >> | SELECT shiftright(4, 1) | struct<(4 >> 1):int> | +| org.apache.spark.sql.catalyst.expressions.ShiftRight | shiftright | SELECT shiftright(4, 1) | struct<(4 >> 1):int> | +| org.apache.spark.sql.catalyst.expressions.ShiftRightUnsigned | >>> | SELECT shiftrightunsigned(4, 1) | struct<(4 >>> 1):int> | +| org.apache.spark.sql.catalyst.expressions.ShiftRightUnsigned | shiftrightunsigned | SELECT shiftrightunsigned(4, 1) | struct<(4 >>> 1):int> | | org.apache.spark.sql.catalyst.expressions.Shuffle | shuffle | SELECT shuffle(array(1, 20, 3, 5)) | struct> | | org.apache.spark.sql.catalyst.expressions.Signum | sign | SELECT sign(40) | struct | | org.apache.spark.sql.catalyst.expressions.Signum | signum | SELECT signum(40) | struct | diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/bitwise.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/bitwise.sql.out index b8958f4a331a6..fee226c0c3411 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/bitwise.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/bitwise.sql.out @@ -306,3 +306,115 @@ select getbit(11L, 64) -- !query analysis Project [getbit(11, 64) AS getbit(11, 64)#x] +- OneRowRelation + + +-- !query +SELECT 20181117 >> 2 +-- !query analysis +Project [(20181117 >> 2) AS (20181117 >> 2)#x] ++- OneRowRelation + + +-- !query +SELECT 20181117 << 2 +-- !query analysis +Project [(20181117 << 2) AS (20181117 << 2)#x] ++- OneRowRelation + + +-- !query +SELECT 20181117 >>> 2 +-- !query analysis +Project [(20181117 >>> 2) AS (20181117 >>> 2)#x] ++- OneRowRelation + + +-- !query +SELECT 20181117 > > 2 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'>'", + "hint" : "" + } +} + + +-- !query +SELECT 20181117 < < 2 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'<'", + "hint" : "" + } +} + + +-- !query +SELECT 20181117 > >> 2 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'>>'", + "hint" : "" + } +} + + +-- !query +SELECT 20181117 <<< 2 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'<'", + "hint" : "" + } +} + + +-- !query +SELECT 20181117 >>>> 2 +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'>'", + "hint" : "" + } +} + + +-- !query +select cast(null as array>), 20181117 >> 2 +-- !query analysis +Project [cast(null as array>) AS NULL#x, (20181117 >> 2) AS (20181117 >> 2)#x] ++- OneRowRelation + + +-- !query +select cast(null as array>), 20181117 >>> 2 +-- !query analysis +Project [cast(null as array>) AS NULL#x, (20181117 >>> 2) AS (20181117 >>> 2)#x] ++- OneRowRelation + + +-- !query +select cast(null as map>), 20181117 >> 2 +-- !query analysis +Project [cast(null as map>) AS NULL#x, (20181117 >> 2) AS (20181117 >> 2)#x] ++- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out index cdb6372bec099..f8967d7df0b0c 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-analytics.sql.out @@ -316,7 +316,7 @@ Sort [course#x ASC NULLS FIRST, sum#xL ASC NULLS FIRST], true SELECT course, year, GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year) -- !query analysis -Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(course)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(year)#x, spark_grouping_id#xL AS grouping_id(course, year)#xL] +Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(course)#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(year)#x, spark_grouping_id#xL AS grouping_id(course, year)#xL] +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#x] +- SubqueryAlias coursesales @@ -382,7 +382,7 @@ HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 ORDER BY course, yea -- !query analysis Sort [course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true +- Project [course#x, year#x] - +- Filter ((cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) = 1) AND (spark_grouping_id#xL > cast(0 as bigint))) + +- Filter ((cast(cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) as int) = 1) AND (spark_grouping_id#xL > cast(0 as bigint))) +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, spark_grouping_id#xL, spark_grouping_id#xL] +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#x] @@ -435,8 +435,8 @@ SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY ORDER BY GROUPING(course), GROUPING(year), course, year -- !query analysis Project [course#x, year#x, grouping(course)#x, grouping(year)#x] -+- Sort [cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) ASC NULLS FIRST, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) ASC NULLS FIRST, course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true - +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(course)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(year)#x, spark_grouping_id#xL, spark_grouping_id#xL] ++- Sort [cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) ASC NULLS FIRST, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) ASC NULLS FIRST, course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true + +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(course)#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(year)#x, spark_grouping_id#xL, spark_grouping_id#xL] +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#x] +- SubqueryAlias coursesales @@ -452,7 +452,7 @@ SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(co ORDER BY GROUPING(course), GROUPING(year), course, year -- !query analysis Project [course#x, year#x, grouping_id(course, year)#xL] -+- Sort [cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) ASC NULLS FIRST, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) ASC NULLS FIRST, course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true ++- Sort [cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) ASC NULLS FIRST, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) ASC NULLS FIRST, course#x ASC NULLS FIRST, year#x ASC NULLS FIRST], true +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, spark_grouping_id#xL AS grouping_id(course, year)#xL, spark_grouping_id#xL, spark_grouping_id#xL] +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out index b73ee16c8bdef..cbbcb77325348 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/grouping_set.sql.out @@ -72,7 +72,7 @@ Aggregate [c1#x, spark_grouping_id#xL], [c1#x, sum(c2#x) AS sum(c2)#xL] -- !query SELECT c1, sum(c2), grouping(c1) FROM (VALUES ('x', 10, 0), ('y', 20, 0)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1) -- !query analysis -Aggregate [c1#x, spark_grouping_id#xL], [c1#x, sum(c2#x) AS sum(c2)#xL, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(c1)#x] +Aggregate [c1#x, spark_grouping_id#xL], [c1#x, sum(c2#x) AS sum(c2)#xL, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(c1)#x] +- Expand [[c1#x, c2#x, c3#x, c1#x, 0]], [c1#x, c2#x, c3#x, c1#x, spark_grouping_id#xL] +- Project [c1#x, c2#x, c3#x, c1#x AS c1#x] +- SubqueryAlias t @@ -98,7 +98,7 @@ Filter (grouping__id#xL > cast(1 as bigint)) -- !query SELECT grouping(c1) FROM (VALUES ('x', 'a', 10), ('y', 'b', 20)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1,c2) -- !query analysis -Aggregate [c1#x, c2#x, spark_grouping_id#xL], [cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(c1)#x] +Aggregate [c1#x, c2#x, spark_grouping_id#xL], [cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(c1)#x] +- Expand [[c1#x, c2#x, c3#x, c1#x, null, 1], [c1#x, c2#x, c3#x, null, c2#x, 2]], [c1#x, c2#x, c3#x, c1#x, c2#x, spark_grouping_id#xL] +- Project [c1#x, c2#x, c3#x, c1#x AS c1#x, c2#x AS c2#x] +- SubqueryAlias t @@ -223,7 +223,7 @@ Aggregate [k1#x, k2#x, spark_grouping_id#xL, _gen_grouping_pos#x], [spark_groupi -- !query SELECT grouping(k1), k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)) -- !query analysis -Aggregate [k1#x, k2#x, spark_grouping_id#xL, _gen_grouping_pos#x], [cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(k1)#x, k1#x, k2#x, avg(v#x) AS avg(v)#x] +Aggregate [k1#x, k2#x, spark_grouping_id#xL, _gen_grouping_pos#x], [cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(k1)#x, k1#x, k2#x, avg(v#x) AS avg(v)#x] +- Expand [[k1#x, k2#x, v#x, k1#x, null, 1, 0], [k1#x, k2#x, v#x, k1#x, k2#x, 0, 1], [k1#x, k2#x, v#x, k1#x, k2#x, 0, 2]], [k1#x, k2#x, v#x, k1#x, k2#x, spark_grouping_id#xL, _gen_grouping_pos#x] +- Project [k1#x, k2#x, v#x, k1#x AS k1#x, k2#x AS k2#x] +- SubqueryAlias t diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/groupingsets.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/groupingsets.sql.out index 27e9707425833..d2a25fabe2059 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/groupingsets.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/groupingsets.sql.out @@ -82,7 +82,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`gstest_empty`, false select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) from gstest1 group by rollup (a,b) -- !query analysis -Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(a)#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] +- Expand [[a#x, b#x, v#x, a#x, b#x, 0], [a#x, b#x, v#x, a#x, null, 1], [a#x, b#x, v#x, null, null, 3]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL] +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] +- SubqueryAlias gstest1 @@ -96,7 +96,7 @@ select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) from gstest1 group by rollup (a,b) order by a,b -- !query analysis Sort [a#x ASC NULLS FIRST, b#x ASC NULLS FIRST], true -+- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] ++- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(a)#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] +- Expand [[a#x, b#x, v#x, a#x, b#x, 0], [a#x, b#x, v#x, a#x, null, 1], [a#x, b#x, v#x, null, null, 3]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL] +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] +- SubqueryAlias gstest1 @@ -110,7 +110,7 @@ select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) from gstest1 group by rollup (a,b) order by b desc, a -- !query analysis Sort [b#x DESC NULLS LAST, a#x ASC NULLS FIRST], true -+- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] ++- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(a)#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] +- Expand [[a#x, b#x, v#x, a#x, b#x, 0], [a#x, b#x, v#x, a#x, null, 1], [a#x, b#x, v#x, null, null, 3]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL] +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] +- SubqueryAlias gstest1 @@ -124,7 +124,7 @@ select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) from gstest1 group by rollup (a,b) order by coalesce(a,0)+coalesce(b,0), a -- !query analysis Sort [(coalesce(a#x, 0) + coalesce(b#x, 0)) ASC NULLS FIRST, a#x ASC NULLS FIRST], true -+- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] ++- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(a)#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] +- Expand [[a#x, b#x, v#x, a#x, b#x, 0], [a#x, b#x, v#x, a#x, null, 1], [a#x, b#x, v#x, null, null, 3]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL] +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] +- SubqueryAlias gstest1 @@ -209,7 +209,7 @@ select t1.a, t2.b, grouping(t1.a), grouping(t2.b), sum(t1.v), max(t2.a) from gstest1 t1, gstest2 t2 group by grouping sets ((t1.a, t2.b), ()) -- !query analysis -Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, max(a#x) AS max(a)#x] +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(a)#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, max(a#x) AS max(a)#x] +- Expand [[a#x, b#x, v#x, a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, b#x, 0], [a#x, b#x, v#x, a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, null, null, 3]], [a#x, b#x, v#x, a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, b#x, spark_grouping_id#xL] +- Project [a#x, b#x, v#x, a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x AS a#x, b#x AS b#x] +- Join Inner @@ -228,7 +228,7 @@ select t1.a, t2.b, grouping(t1.a), grouping(t2.b), sum(t1.v), max(t2.a) from gstest1 t1 join gstest2 t2 on (t1.a=t2.a) group by grouping sets ((t1.a, t2.b), ()) -- !query analysis -Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, max(a#x) AS max(a)#x] +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(a)#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, max(a#x) AS max(a)#x] +- Expand [[a#x, b#x, v#x, a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, b#x, 0], [a#x, b#x, v#x, a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, null, null, 3]], [a#x, b#x, v#x, a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, b#x, spark_grouping_id#xL] +- Project [a#x, b#x, v#x, a#x, b#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x AS a#x, b#x AS b#x] +- Join Inner, (a#x = a#x) @@ -247,7 +247,7 @@ select a, b, grouping(a), grouping(b), sum(t1.v), max(t2.c) from gstest1 t1 join gstest2 t2 using (a,b) group by grouping sets ((a, b), ()) -- !query analysis -Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, max(c#x) AS max(c)#x] +Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(a)#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, max(c#x) AS max(c)#x] +- Expand [[a#x, b#x, v#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, b#x, 0], [a#x, b#x, v#x, c#x, d#x, e#x, f#x, g#x, h#x, null, null, 3]], [a#x, b#x, v#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x, b#x, spark_grouping_id#xL] +- Project [a#x, b#x, v#x, c#x, d#x, e#x, f#x, g#x, h#x, a#x AS a#x, b#x AS b#x] +- Project [a#x, b#x, v#x, c#x, d#x, e#x, f#x, g#x, h#x] @@ -402,8 +402,8 @@ order by 2,1 -- !query analysis Sort [grouping(ten)#x ASC NULLS FIRST, ten#x ASC NULLS FIRST], true +- Project [ten#x, grouping(ten)#x] - +- Filter (cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) >= 0) - +- Aggregate [ten#x, spark_grouping_id#xL], [ten#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(ten)#x, spark_grouping_id#xL] + +- Filter (cast(cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) as int) >= 0) + +- Aggregate [ten#x, spark_grouping_id#xL], [ten#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(ten)#x, spark_grouping_id#xL] +- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, 0]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, spark_grouping_id#xL] +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x] +- SubqueryAlias spark_catalog.default.onek @@ -417,8 +417,8 @@ order by 2,1 -- !query analysis Sort [grouping(ten)#x ASC NULLS FIRST, ten#x ASC NULLS FIRST], true +- Project [ten#x, grouping(ten)#x] - +- Filter (cast(cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) as int) > 0) - +- Aggregate [ten#x, four#x, spark_grouping_id#xL], [ten#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(ten)#x, spark_grouping_id#xL] + +- Filter (cast(cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) as int) > 0) + +- Aggregate [ten#x, four#x, spark_grouping_id#xL], [ten#x, cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(ten)#x, spark_grouping_id#xL] +- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, null, 1], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, null, four#x, 2]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, four#x, spark_grouping_id#xL] +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x, four#x AS four#x] +- SubqueryAlias spark_catalog.default.onek @@ -432,8 +432,8 @@ order by 2,1 -- !query analysis Sort [grouping(ten)#x ASC NULLS FIRST, ten#x ASC NULLS FIRST], true +- Project [ten#x, grouping(ten)#x] - +- Filter (cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) > 0) - +- Aggregate [ten#x, spark_grouping_id#xL], [ten#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(ten)#x, spark_grouping_id#xL] + +- Filter (cast(cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) as int) > 0) + +- Aggregate [ten#x, spark_grouping_id#xL], [ten#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(ten)#x, spark_grouping_id#xL] +- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, 0], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, null, 1]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, spark_grouping_id#xL] +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x] +- SubqueryAlias spark_catalog.default.onek @@ -447,8 +447,8 @@ order by 2,1 -- !query analysis Sort [grouping(ten)#x ASC NULLS FIRST, ten#x ASC NULLS FIRST], true +- Project [ten#x, grouping(ten)#x] - +- Filter (cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) > 0) - +- Aggregate [ten#x, spark_grouping_id#xL], [ten#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(ten)#x, spark_grouping_id#xL] + +- Filter (cast(cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) as int) > 0) + +- Aggregate [ten#x, spark_grouping_id#xL], [ten#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(ten)#x, spark_grouping_id#xL] +- Expand [[unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, 0], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, null, 1]], [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x, spark_grouping_id#xL] +- Project [unique1#x, unique2#x, two#x, four#x, ten#x, twenty#x, hundred#x, thousand#x, twothousand#x, fivethous#x, tenthous#x, odd#x, even#x, stringu1#x, stringu2#x, string4#x, ten#x AS ten#x] +- SubqueryAlias spark_catalog.default.onek @@ -482,7 +482,7 @@ select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) from gstest1 group by grouping sets ((a),(b)) order by 3,4,1,2 /* 3,1,2 */ -- !query analysis Sort [grouping(a)#x ASC NULLS FIRST, grouping(b)#x ASC NULLS FIRST, a#x ASC NULLS FIRST, b#x ASC NULLS FIRST], true -+- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] ++- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(a)#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] +- Expand [[a#x, b#x, v#x, a#x, null, 1], [a#x, b#x, v#x, null, b#x, 2]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL] +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] +- SubqueryAlias gstest1 @@ -496,7 +496,7 @@ select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) from gstest1 group by cube(a,b) order by 3,4,1,2 /* 3,1,2 */ -- !query analysis Sort [grouping(a)#x ASC NULLS FIRST, grouping(b)#x ASC NULLS FIRST, a#x ASC NULLS FIRST, b#x ASC NULLS FIRST], true -+- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] ++- Aggregate [a#x, b#x, spark_grouping_id#xL], [a#x, b#x, cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(a)#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] +- Expand [[a#x, b#x, v#x, a#x, b#x, 0], [a#x, b#x, v#x, a#x, null, 1], [a#x, b#x, v#x, null, b#x, 2], [a#x, b#x, v#x, null, null, 3]], [a#x, b#x, v#x, a#x, b#x, spark_grouping_id#xL] +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x] +- SubqueryAlias gstest1 @@ -526,7 +526,7 @@ select unhashable_col, unsortable_col, order by 3, 4, 6 /* 3, 5 */ -- !query analysis Sort [grouping(unhashable_col)#x ASC NULLS FIRST, grouping(unsortable_col)#x ASC NULLS FIRST, sum(v)#xL ASC NULLS FIRST], true -+- Aggregate [unhashable_col#x, unsortable_col#x, spark_grouping_id#xL], [unhashable_col#x, unsortable_col#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(unhashable_col)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(unsortable_col)#x, count(1) AS count(1)#xL, sum(v#x) AS sum(v)#xL] ++- Aggregate [unhashable_col#x, unsortable_col#x, spark_grouping_id#xL], [unhashable_col#x, unsortable_col#x, cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(unhashable_col)#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(unsortable_col)#x, count(1) AS count(1)#xL, sum(v#x) AS sum(v)#xL] +- Expand [[id#x, v#x, unhashable_col#x, unsortable_col#x, unhashable_col#x, null, 1], [id#x, v#x, unhashable_col#x, unsortable_col#x, null, unsortable_col#x, 2]], [id#x, v#x, unhashable_col#x, unsortable_col#x, unhashable_col#x, unsortable_col#x, spark_grouping_id#xL] +- Project [id#x, v#x, unhashable_col#x, unsortable_col#x, unhashable_col#x AS unhashable_col#x, unsortable_col#x AS unsortable_col#x] +- SubqueryAlias spark_catalog.default.gstest4 @@ -541,7 +541,7 @@ select unhashable_col, unsortable_col, order by 3, 4, 6 /* 3,5 */ -- !query analysis Sort [grouping(unhashable_col)#x ASC NULLS FIRST, grouping(unsortable_col)#x ASC NULLS FIRST, sum(v)#xL ASC NULLS FIRST], true -+- Aggregate [v#x, unhashable_col#x, unsortable_col#x, spark_grouping_id#xL], [unhashable_col#x, unsortable_col#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(unhashable_col)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(unsortable_col)#x, count(1) AS count(1)#xL, sum(v#x) AS sum(v)#xL] ++- Aggregate [v#x, unhashable_col#x, unsortable_col#x, spark_grouping_id#xL], [unhashable_col#x, unsortable_col#x, cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(unhashable_col)#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(unsortable_col)#x, count(1) AS count(1)#xL, sum(v#x) AS sum(v)#xL] +- Expand [[id#x, v#x, unhashable_col#x, unsortable_col#x, v#x, unhashable_col#x, null, 1], [id#x, v#x, unhashable_col#x, unsortable_col#x, v#x, null, unsortable_col#x, 2]], [id#x, v#x, unhashable_col#x, unsortable_col#x, v#x, unhashable_col#x, unsortable_col#x, spark_grouping_id#xL] +- Project [id#x, v#x, unhashable_col#x, unsortable_col#x, v#x AS v#x, unhashable_col#x AS unhashable_col#x, unsortable_col#x AS unsortable_col#x] +- SubqueryAlias spark_catalog.default.gstest4 @@ -593,7 +593,7 @@ select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) from gstest1 group by grouping sets ((a,b),(a+1,b+1),(a+2,b+2)) order by 3,4,7 /* 3,6 */ -- !query analysis Sort [grouping(a)#x ASC NULLS FIRST, grouping(b)#x ASC NULLS FIRST, max(v)#x ASC NULLS FIRST], true -+- Aggregate [a#x, b#x, (a#x + 1)#x, (b#x + 1)#x, (a#x + 2)#x, (b#x + 2)#x, spark_grouping_id#xL], [a#x, b#x, cast((shiftright(spark_grouping_id#xL, 5) & 1) as tinyint) AS grouping(a)#x, cast((shiftright(spark_grouping_id#xL, 4) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] ++- Aggregate [a#x, b#x, (a#x + 1)#x, (b#x + 1)#x, (a#x + 2)#x, (b#x + 2)#x, spark_grouping_id#xL], [a#x, b#x, cast(((spark_grouping_id#xL >> 5) & 1) as tinyint) AS grouping(a)#x, cast(((spark_grouping_id#xL >> 4) & 1) as tinyint) AS grouping(b)#x, sum(v#x) AS sum(v)#xL, count(1) AS count(1)#xL, max(v#x) AS max(v)#x] +- Expand [[a#x, b#x, v#x, a#x, b#x, null, null, null, null, 15], [a#x, b#x, v#x, null, null, (a#x + 1)#x, (b#x + 1)#x, null, null, 51], [a#x, b#x, v#x, null, null, null, null, (a#x + 2)#x, (b#x + 2)#x, 60]], [a#x, b#x, v#x, a#x, b#x, (a#x + 1)#x, (b#x + 1)#x, (a#x + 2)#x, (b#x + 2)#x, spark_grouping_id#xL] +- Project [a#x, b#x, v#x, a#x AS a#x, b#x AS b#x, (a#x + 1) AS (a#x + 1)#x, (b#x + 1) AS (b#x + 1)#x, (a#x + 2) AS (a#x + 2)#x, (b#x + 2) AS (b#x + 2)#x] +- SubqueryAlias gstest1 @@ -634,7 +634,7 @@ select v||'a', case grouping(v||'a') when 1 then 1 else 0 end, count(*) group by rollup(i, v||'a') order by 1,3 -- !query analysis Sort [concat(v, a)#x ASC NULLS FIRST, count(1)#xL ASC NULLS FIRST], true -+- Aggregate [i#x, concat(v#x, a)#x, spark_grouping_id#xL], [concat(v#x, a)#x AS concat(v, a)#x, CASE WHEN (cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) = 1) THEN 1 ELSE 0 END AS CASE WHEN (grouping(concat(v, a)) = 1) THEN 1 ELSE 0 END#x, count(1) AS count(1)#xL] ++- Aggregate [i#x, concat(v#x, a)#x, spark_grouping_id#xL], [concat(v#x, a)#x AS concat(v, a)#x, CASE WHEN (cast(cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) as int) = 1) THEN 1 ELSE 0 END AS CASE WHEN (grouping(concat(v, a)) = 1) THEN 1 ELSE 0 END#x, count(1) AS count(1)#xL] +- Expand [[i#x, v#x, i#x, concat(v#x, a)#x, 0], [i#x, v#x, i#x, null, 1], [i#x, v#x, null, null, 3]], [i#x, v#x, i#x, concat(v#x, a)#x, spark_grouping_id#xL] +- Project [i#x, v#x, i#x AS i#x, concat(v#x, a) AS concat(v#x, a)#x] +- SubqueryAlias u @@ -647,7 +647,7 @@ select v||'a', case when grouping(v||'a') = 1 then 1 else 0 end, count(*) group by rollup(i, v||'a') order by 1,3 -- !query analysis Sort [concat(v, a)#x ASC NULLS FIRST, count(1)#xL ASC NULLS FIRST], true -+- Aggregate [i#x, concat(v#x, a)#x, spark_grouping_id#xL], [concat(v#x, a)#x AS concat(v, a)#x, CASE WHEN (cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) = 1) THEN 1 ELSE 0 END AS CASE WHEN (grouping(concat(v, a)) = 1) THEN 1 ELSE 0 END#x, count(1) AS count(1)#xL] ++- Aggregate [i#x, concat(v#x, a)#x, spark_grouping_id#xL], [concat(v#x, a)#x AS concat(v, a)#x, CASE WHEN (cast(cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) as int) = 1) THEN 1 ELSE 0 END AS CASE WHEN (grouping(concat(v, a)) = 1) THEN 1 ELSE 0 END#x, count(1) AS count(1)#xL] +- Expand [[i#x, v#x, i#x, concat(v#x, a)#x, 0], [i#x, v#x, i#x, null, 1], [i#x, v#x, null, null, 3]], [i#x, v#x, i#x, concat(v#x, a)#x, spark_grouping_id#xL] +- Project [i#x, v#x, i#x AS i#x, concat(v#x, a) AS concat(v#x, a)#x] +- SubqueryAlias u diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int2.sql.out index 9dda3c0dc42d4..3fa919434da79 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int2.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int2.sql.out @@ -274,14 +274,14 @@ Project [ AS five#x, f1#x, (cast(f1#x as double) / cast(cast(2 as int) as double -- !query SELECT string(shiftleft(smallint(-1), 15)) -- !query analysis -Project [cast(shiftleft(cast(cast(-1 as smallint) as int), 15) as string) AS shiftleft(-1, 15)#x] +Project [cast((cast(cast(-1 as smallint) as int) << 15) as string) AS (-1 << 15)#x] +- OneRowRelation -- !query SELECT string(smallint(shiftleft(smallint(-1), 15))+1) -- !query analysis -Project [cast((cast(cast(shiftleft(cast(cast(-1 as smallint) as int), 15) as smallint) as int) + 1) as string) AS (shiftleft(-1, 15) + 1)#x] +Project [cast((cast(cast((cast(cast(-1 as smallint) as int) << 15) as smallint) as int) + 1) as string) AS ((-1 << 15) + 1)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int4.sql.out index d261b59a4c5e2..f6a8b24f917d2 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int4.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int4.sql.out @@ -411,14 +411,14 @@ Project [(cast((2 + 2) as double) / cast(2 as double)) AS two#x] -- !query SELECT string(shiftleft(int(-1), 31)) -- !query analysis -Project [cast(shiftleft(cast(-1 as int), 31) as string) AS shiftleft(-1, 31)#x] +Project [cast((cast(-1 as int) << 31) as string) AS (-1 << 31)#x] +- OneRowRelation -- !query SELECT string(int(shiftleft(int(-1), 31))+1) -- !query analysis -Project [cast((cast(shiftleft(cast(-1 as int), 31) as int) + 1) as string) AS (shiftleft(-1, 31) + 1)#x] +Project [cast((cast((cast(-1 as int) << 31) as int) + 1) as string) AS ((-1 << 31) + 1)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int8.sql.out index 72972469fa6ef..dfc96427b57ed 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int8.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int8.sql.out @@ -659,14 +659,14 @@ Project [id#xL] -- !query SELECT string(shiftleft(bigint(-1), 63)) -- !query analysis -Project [cast(shiftleft(cast(-1 as bigint), 63) as string) AS shiftleft(-1, 63)#x] +Project [cast((cast(-1 as bigint) << 63) as string) AS (-1 << 63)#x] +- OneRowRelation -- !query SELECT string(int(shiftleft(bigint(-1), 63))+1) -- !query analysis -Project [cast((cast(shiftleft(cast(-1 as bigint), 63) as int) + 1) as string) AS (shiftleft(-1, 63) + 1)#x] +Project [cast((cast((cast(-1 as bigint) << 63) as int) + 1) as string) AS ((-1 << 63) + 1)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-analytics.sql.out index fbee3e2c8c89f..7d6eb0a83bf4e 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-analytics.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-analytics.sql.out @@ -189,7 +189,7 @@ Sort [cast(udf(cast(course#x as string)) as string) ASC NULLS FIRST, sum#xL ASC SELECT udf(course), udf(year), GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year) -- !query analysis -Aggregate [course#x, year#x, spark_grouping_id#xL], [cast(udf(cast(course#x as string)) as string) AS udf(course)#x, cast(udf(cast(year#x as string)) as int) AS udf(year)#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(course)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(year)#x, spark_grouping_id#xL AS grouping_id(course, year)#xL] +Aggregate [course#x, year#x, spark_grouping_id#xL], [cast(udf(cast(course#x as string)) as string) AS udf(course)#x, cast(udf(cast(year#x as string)) as int) AS udf(year)#x, cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(course)#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(year)#x, spark_grouping_id#xL AS grouping_id(course, year)#xL] +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#x] +- SubqueryAlias coursesales @@ -255,7 +255,7 @@ HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 ORDER BY course, udf -- !query analysis Sort [course#x ASC NULLS FIRST, cast(udf(cast(year#x as string)) as int) ASC NULLS FIRST], true +- Project [course#x, year#x] - +- Filter ((cast(cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) as int) = 1) AND (spark_grouping_id#xL > cast(0 as bigint))) + +- Filter ((cast(cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) as int) = 1) AND (spark_grouping_id#xL > cast(0 as bigint))) +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, spark_grouping_id#xL, spark_grouping_id#xL] +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#x] @@ -308,8 +308,8 @@ SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY ORDER BY GROUPING(course), GROUPING(year), course, udf(year) -- !query analysis Project [course#x, year#x, grouping(course)#x, grouping(year)#x] -+- Sort [cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) ASC NULLS FIRST, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) ASC NULLS FIRST, course#x ASC NULLS FIRST, cast(udf(cast(year#x as string)) as int) ASC NULLS FIRST], true - +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) AS grouping(course)#x, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) AS grouping(year)#x, spark_grouping_id#xL, spark_grouping_id#xL] ++- Sort [cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) ASC NULLS FIRST, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) ASC NULLS FIRST, course#x ASC NULLS FIRST, cast(udf(cast(year#x as string)) as int) ASC NULLS FIRST], true + +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) AS grouping(course)#x, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) AS grouping(year)#x, spark_grouping_id#xL, spark_grouping_id#xL] +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#x] +- SubqueryAlias coursesales @@ -325,7 +325,7 @@ SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(co ORDER BY GROUPING(course), GROUPING(year), course, udf(year) -- !query analysis Project [course#x, year#x, grouping_id(course, year)#xL] -+- Sort [cast((shiftright(spark_grouping_id#xL, 1) & 1) as tinyint) ASC NULLS FIRST, cast((shiftright(spark_grouping_id#xL, 0) & 1) as tinyint) ASC NULLS FIRST, course#x ASC NULLS FIRST, cast(udf(cast(year#x as string)) as int) ASC NULLS FIRST], true ++- Sort [cast(((spark_grouping_id#xL >> 1) & 1) as tinyint) ASC NULLS FIRST, cast(((spark_grouping_id#xL >> 0) & 1) as tinyint) ASC NULLS FIRST, course#x ASC NULLS FIRST, cast(udf(cast(year#x as string)) as int) ASC NULLS FIRST], true +- Aggregate [course#x, year#x, spark_grouping_id#xL], [course#x, year#x, spark_grouping_id#xL AS grouping_id(course, year)#xL, spark_grouping_id#xL, spark_grouping_id#xL] +- Expand [[course#x, year#x, earnings#x, course#x, year#x, 0], [course#x, year#x, earnings#x, course#x, null, 1], [course#x, year#x, earnings#x, null, year#x, 2], [course#x, year#x, earnings#x, null, null, 3]], [course#x, year#x, earnings#x, course#x, year#x, spark_grouping_id#xL] +- Project [course#x, year#x, earnings#x, course#x AS course#x, year#x AS year#x] diff --git a/sql/core/src/test/resources/sql-tests/inputs/bitwise.sql b/sql/core/src/test/resources/sql-tests/inputs/bitwise.sql index f9dfd161d0c07..5823b22ef6453 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/bitwise.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/bitwise.sql @@ -75,3 +75,15 @@ select getbit(11L, 2 + 1), getbit(11L, 3 - 1), getbit(10L + 1, 1 * 1), getbit(ca select getbit(11L, 63); select getbit(11L, -1); select getbit(11L, 64); + +SELECT 20181117 >> 2; +SELECT 20181117 << 2; +SELECT 20181117 >>> 2; +SELECT 20181117 > > 2; +SELECT 20181117 < < 2; +SELECT 20181117 > >> 2; +SELECT 20181117 <<< 2; +SELECT 20181117 >>>> 2; +select cast(null as array>), 20181117 >> 2; +select cast(null as array>), 20181117 >>> 2; +select cast(null as map>), 20181117 >> 2; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out b/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out index 2c8b733004aac..a7ebaea293bf9 100644 --- a/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out @@ -322,3 +322,131 @@ org.apache.spark.SparkIllegalArgumentException "upper" : "64" } } + + +-- !query +SELECT 20181117 >> 2 +-- !query schema +struct<(20181117 >> 2):int> +-- !query output +5045279 + + +-- !query +SELECT 20181117 << 2 +-- !query schema +struct<(20181117 << 2):int> +-- !query output +80724468 + + +-- !query +SELECT 20181117 >>> 2 +-- !query schema +struct<(20181117 >>> 2):int> +-- !query output +5045279 + + +-- !query +SELECT 20181117 > > 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'>'", + "hint" : "" + } +} + + +-- !query +SELECT 20181117 < < 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'<'", + "hint" : "" + } +} + + +-- !query +SELECT 20181117 > >> 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'>>'", + "hint" : "" + } +} + + +-- !query +SELECT 20181117 <<< 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'<'", + "hint" : "" + } +} + + +-- !query +SELECT 20181117 >>>> 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'>'", + "hint" : "" + } +} + + +-- !query +select cast(null as array>), 20181117 >> 2 +-- !query schema +struct>,(20181117 >> 2):int> +-- !query output +NULL 5045279 + + +-- !query +select cast(null as array>), 20181117 >>> 2 +-- !query schema +struct>,(20181117 >>> 2):int> +-- !query output +NULL 5045279 + + +-- !query +select cast(null as map>), 20181117 >> 2 +-- !query schema +struct>,(20181117 >> 2):int> +-- !query output +NULL 5045279 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out index ca55b6accc665..1c96f8dfa5e54 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out @@ -289,7 +289,7 @@ struct -- !query SELECT string(shiftleft(smallint(-1), 15)) -- !query schema -struct +struct<(-1 << 15):string> -- !query output -32768 @@ -297,7 +297,7 @@ struct -- !query SELECT string(smallint(shiftleft(smallint(-1), 15))+1) -- !query schema -struct<(shiftleft(-1, 15) + 1):string> +struct<((-1 << 15) + 1):string> -- !query output -32767 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out index 16c18c86f2919..afe0211bd1947 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out @@ -520,7 +520,7 @@ struct -- !query SELECT string(shiftleft(int(-1), 31)) -- !query schema -struct +struct<(-1 << 31):string> -- !query output -2147483648 @@ -528,7 +528,7 @@ struct -- !query SELECT string(int(shiftleft(int(-1), 31))+1) -- !query schema -struct<(shiftleft(-1, 31) + 1):string> +struct<((-1 << 31) + 1):string> -- !query output -2147483647 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out index f6e4bd8bd7e08..6e7ca4afab67d 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out @@ -883,7 +883,7 @@ struct -- !query SELECT string(shiftleft(bigint(-1), 63)) -- !query schema -struct +struct<(-1 << 63):string> -- !query output -9223372036854775808 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index 850b20431e487..6908b8137b0c4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -102,7 +102,7 @@ Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) (13) BroadcastExchange Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] +Arguments: HashedRelationBroadcastMode(List(((cast(input[0, int, false] as bigint) << 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] (14) BroadcastHashJoin [codegen id : 8] Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt index e2caa9f171b86..15b74bac0fbec 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt @@ -102,7 +102,7 @@ Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) (13) BroadcastExchange Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] +Arguments: HashedRelationBroadcastMode(List(((cast(input[0, int, false] as bigint) << 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] (14) BroadcastHashJoin [codegen id : 8] Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt index 6cc9c3a4834ee..4b7d4f2f068d6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt @@ -167,7 +167,7 @@ Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, s Keys [3]: [i_item_id#19, s_state#20, spark_grouping_id#21] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [4]: [avg(ss_quantity#4)#38, avg(UnscaledValue(ss_list_price#5))#39, avg(UnscaledValue(ss_coupon_amt#7))#40, avg(UnscaledValue(ss_sales_price#6))#41] -Results [7]: [i_item_id#19, s_state#20, cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint) AS g_state#42, avg(ss_quantity#4)#38 AS agg1#43, cast((avg(UnscaledValue(ss_list_price#5))#39 / 100.0) as decimal(11,6)) AS agg2#44, cast((avg(UnscaledValue(ss_coupon_amt#7))#40 / 100.0) as decimal(11,6)) AS agg3#45, cast((avg(UnscaledValue(ss_sales_price#6))#41 / 100.0) as decimal(11,6)) AS agg4#46] +Results [7]: [i_item_id#19, s_state#20, cast(((spark_grouping_id#21 >> 0) & 1) as tinyint) AS g_state#42, avg(ss_quantity#4)#38 AS agg1#43, cast((avg(UnscaledValue(ss_list_price#5))#39 / 100.0) as decimal(11,6)) AS agg2#44, cast((avg(UnscaledValue(ss_coupon_amt#7))#40 / 100.0) as decimal(11,6)) AS agg3#45, cast((avg(UnscaledValue(ss_sales_price#6))#41 / 100.0) as decimal(11,6)) AS agg4#46] (30) TakeOrderedAndProject Input [7]: [i_item_id#19, s_state#20, g_state#42, agg1#43, agg2#44, agg3#45, agg4#46] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt index 6cc9c3a4834ee..4b7d4f2f068d6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt @@ -167,7 +167,7 @@ Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, s Keys [3]: [i_item_id#19, s_state#20, spark_grouping_id#21] Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] Aggregate Attributes [4]: [avg(ss_quantity#4)#38, avg(UnscaledValue(ss_list_price#5))#39, avg(UnscaledValue(ss_coupon_amt#7))#40, avg(UnscaledValue(ss_sales_price#6))#41] -Results [7]: [i_item_id#19, s_state#20, cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint) AS g_state#42, avg(ss_quantity#4)#38 AS agg1#43, cast((avg(UnscaledValue(ss_list_price#5))#39 / 100.0) as decimal(11,6)) AS agg2#44, cast((avg(UnscaledValue(ss_coupon_amt#7))#40 / 100.0) as decimal(11,6)) AS agg3#45, cast((avg(UnscaledValue(ss_sales_price#6))#41 / 100.0) as decimal(11,6)) AS agg4#46] +Results [7]: [i_item_id#19, s_state#20, cast(((spark_grouping_id#21 >> 0) & 1) as tinyint) AS g_state#42, avg(ss_quantity#4)#38 AS agg1#43, cast((avg(UnscaledValue(ss_list_price#5))#39 / 100.0) as decimal(11,6)) AS agg2#44, cast((avg(UnscaledValue(ss_coupon_amt#7))#40 / 100.0) as decimal(11,6)) AS agg3#45, cast((avg(UnscaledValue(ss_sales_price#6))#41 / 100.0) as decimal(11,6)) AS agg4#46] (30) TakeOrderedAndProject Input [7]: [i_item_id#19, s_state#20, g_state#42, agg1#43, agg2#44, agg3#45, agg4#46] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt index 76a6ab9c7215b..27534390f0a24 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt @@ -102,7 +102,7 @@ Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) (13) BroadcastExchange Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] +Arguments: HashedRelationBroadcastMode(List(((cast(input[0, int, false] as bigint) << 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] (14) BroadcastHashJoin [codegen id : 8] Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/explain.txt index ea59f2b926c9d..63cb718a827f3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/explain.txt @@ -137,7 +137,7 @@ Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] Keys [3]: [i_category#13, i_class#14, spark_grouping_id#15] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#20, sum(UnscaledValue(ss_ext_sales_price#3))#21] -Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS gross_margin#22, i_category#13, i_class#14, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS lochierarchy#23, (MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS _w0#24, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS _w1#25, CASE WHEN (cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint) = 0) THEN i_category#13 END AS _w2#26] +Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS gross_margin#22, i_category#13, i_class#14, (cast(((spark_grouping_id#15 >> 1) & 1) as tinyint) + cast(((spark_grouping_id#15 >> 0) & 1) as tinyint)) AS lochierarchy#23, (MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS _w0#24, (cast(((spark_grouping_id#15 >> 1) & 1) as tinyint) + cast(((spark_grouping_id#15 >> 0) & 1) as tinyint)) AS _w1#25, CASE WHEN (cast(((spark_grouping_id#15 >> 0) & 1) as tinyint) = 0) THEN i_category#13 END AS _w2#26] (24) Exchange Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt index 6cc55ab063f68..eb59673575aba 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt @@ -137,7 +137,7 @@ Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] Keys [3]: [i_category#13, i_class#14, spark_grouping_id#15] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#20, sum(UnscaledValue(ss_ext_sales_price#3))#21] -Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS gross_margin#22, i_category#13, i_class#14, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS lochierarchy#23, (MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS _w0#24, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS _w1#25, CASE WHEN (cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint) = 0) THEN i_category#13 END AS _w2#26] +Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS gross_margin#22, i_category#13, i_class#14, (cast(((spark_grouping_id#15 >> 1) & 1) as tinyint) + cast(((spark_grouping_id#15 >> 0) & 1) as tinyint)) AS lochierarchy#23, (MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS _w0#24, (cast(((spark_grouping_id#15 >> 1) & 1) as tinyint) + cast(((spark_grouping_id#15 >> 0) & 1) as tinyint)) AS _w1#25, CASE WHEN (cast(((spark_grouping_id#15 >> 0) & 1) as tinyint) = 0) THEN i_category#13 END AS _w2#26] (24) Exchange Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt index 220598440e092..995b723c6e287 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt @@ -237,7 +237,7 @@ Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] (41) BroadcastExchange Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(((cast(input[1, int, true] as bigint) << 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] (42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt index 585e748860557..dba61c77b774e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt @@ -237,7 +237,7 @@ Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] (41) BroadcastExchange Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(((cast(input[1, int, true] as bigint) << 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] (42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [i_item_sk#6, w_warehouse_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index 9eea658d789e4..93f79d66f0973 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -99,7 +99,7 @@ Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_ne (5) BroadcastExchange Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=1] +Arguments: HashedRelationBroadcastMode(List(((cast(input[1, int, true] as bigint) << 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=1] (6) Scan parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] @@ -209,7 +209,7 @@ Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, c (29) BroadcastExchange Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(((cast(input[1, int, true] as bigint) << 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=4] (30) Scan parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] @@ -319,7 +319,7 @@ Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, (53) BroadcastExchange Input [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(((cast(input[1, int, true] as bigint) << 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=7] (54) Scan parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index 313959456c809..93103073d6f85 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -304,7 +304,7 @@ Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, (49) BroadcastExchange Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(((cast(input[0, int, true] as bigint) << 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] (50) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt index 69023c88202af..3a049ca71e742 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt @@ -201,7 +201,7 @@ Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AN (4) BroadcastExchange Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=1] +Arguments: HashedRelationBroadcastMode(List(((cast(input[0, int, false] as bigint) << 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=1] (5) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] @@ -714,7 +714,7 @@ Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113) (115) BroadcastExchange Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=16] +Arguments: HashedRelationBroadcastMode(List(((cast(input[0, int, false] as bigint) << 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=16] (116) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index d64f560f144e0..b6b480018aa46 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -224,7 +224,7 @@ Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS total_sum#26, s_state#20, s_county#21, (cast((shiftright(spark_grouping_id#22, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint)) AS lochierarchy#27, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS _w0#28, (cast((shiftright(spark_grouping_id#22, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint)) AS _w1#29, CASE WHEN (cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint) = 0) THEN s_state#20 END AS _w2#30] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS total_sum#26, s_state#20, s_county#21, (cast(((spark_grouping_id#22 >> 1) & 1) as tinyint) + cast(((spark_grouping_id#22 >> 0) & 1) as tinyint)) AS lochierarchy#27, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS _w0#28, (cast(((spark_grouping_id#22 >> 1) & 1) as tinyint) + cast(((spark_grouping_id#22 >> 0) & 1) as tinyint)) AS _w1#29, CASE WHEN (cast(((spark_grouping_id#22 >> 0) & 1) as tinyint) = 0) THEN s_state#20 END AS _w2#30] (39) Exchange Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index dade1b4f55c5f..9495128a50e13 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -224,7 +224,7 @@ Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24] Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS total_sum#26, s_state#20, s_county#21, (cast((shiftright(spark_grouping_id#22, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint)) AS lochierarchy#27, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS _w0#28, (cast((shiftright(spark_grouping_id#22, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint)) AS _w1#29, CASE WHEN (cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint) = 0) THEN s_state#20 END AS _w2#30] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS total_sum#26, s_state#20, s_county#21, (cast(((spark_grouping_id#22 >> 1) & 1) as tinyint) + cast(((spark_grouping_id#22 >> 0) & 1) as tinyint)) AS lochierarchy#27, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS _w0#28, (cast(((spark_grouping_id#22 >> 1) & 1) as tinyint) + cast(((spark_grouping_id#22 >> 0) & 1) as tinyint)) AS _w1#29, CASE WHEN (cast(((spark_grouping_id#22 >> 0) & 1) as tinyint) = 0) THEN s_state#20 END AS _w2#30] (39) Exchange Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt index 12ba2db6323e4..31da928a5d7a3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt @@ -264,7 +264,7 @@ Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) (42) BroadcastExchange Input [2]: [d_date_sk#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(((cast(input[1, int, false] as bigint) << 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] (43) BroadcastHashJoin [codegen id : 10] Left keys [2]: [d_week_seq#24, inv_date_sk#13] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt index af6632f4fb608..31c804c73eaef 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt @@ -66,7 +66,7 @@ Condition : ((((isnotnull(ws_item_sk#1) AND isnotnull(ws_order_number#3)) AND is (4) BroadcastExchange Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[2, int, false] as bigint) & 4294967295))),false), [plan_id=1] +Arguments: HashedRelationBroadcastMode(List(((cast(input[0, int, false] as bigint) << 32) | (cast(input[2, int, false] as bigint) & 4294967295))),false), [plan_id=1] (5) Scan parquet spark_catalog.default.web_returns Output [9]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16, wr_returned_date_sk#17] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.sf100/explain.txt index d1802b2e4a7c6..c496d20204875 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.sf100/explain.txt @@ -98,7 +98,7 @@ Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] Keys [3]: [i_category#9, i_class#10, spark_grouping_id#11] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#14] -Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS total_sum#15, i_category#9, i_class#10, (cast((shiftright(spark_grouping_id#11, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint)) AS lochierarchy#16, MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS _w0#17, (cast((shiftright(spark_grouping_id#11, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint)) AS _w1#18, CASE WHEN (cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint) = 0) THEN i_category#9 END AS _w2#19] +Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS total_sum#15, i_category#9, i_class#10, (cast(((spark_grouping_id#11 >> 1) & 1) as tinyint) + cast(((spark_grouping_id#11 >> 0) & 1) as tinyint)) AS lochierarchy#16, MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS _w0#17, (cast(((spark_grouping_id#11 >> 1) & 1) as tinyint) + cast(((spark_grouping_id#11 >> 0) & 1) as tinyint)) AS _w1#18, CASE WHEN (cast(((spark_grouping_id#11 >> 0) & 1) as tinyint) = 0) THEN i_category#9 END AS _w2#19] (17) Exchange Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt index d1802b2e4a7c6..c496d20204875 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt @@ -98,7 +98,7 @@ Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] Keys [3]: [i_category#9, i_class#10, spark_grouping_id#11] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#14] -Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS total_sum#15, i_category#9, i_class#10, (cast((shiftright(spark_grouping_id#11, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint)) AS lochierarchy#16, MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS _w0#17, (cast((shiftright(spark_grouping_id#11, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint)) AS _w1#18, CASE WHEN (cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint) = 0) THEN i_category#9 END AS _w2#19] +Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS total_sum#15, i_category#9, i_class#10, (cast(((spark_grouping_id#11 >> 1) & 1) as tinyint) + cast(((spark_grouping_id#11 >> 0) & 1) as tinyint)) AS lochierarchy#16, MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS _w0#17, (cast(((spark_grouping_id#11 >> 1) & 1) as tinyint) + cast(((spark_grouping_id#11 >> 0) & 1) as tinyint)) AS _w1#18, CASE WHEN (cast(((spark_grouping_id#11 >> 0) & 1) as tinyint) = 0) THEN i_category#9 END AS _w2#19] (17) Exchange Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt index 9d80077e99372..e437dea8ca9a0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt @@ -125,7 +125,7 @@ Input [11]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#6, ca_state# (17) BroadcastExchange Input [7]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#7, c_customer_sk#10, c_first_name#12, c_last_name#13] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [plan_id=3] +Arguments: HashedRelationBroadcastMode(List(((cast(input[0, int, true] as bigint) << 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [plan_id=3] (18) Scan parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#15, ss_customer_sk#16, ss_store_sk#17, ss_ticket_number#18, ss_net_paid#19, ss_sold_date_sk#20] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index fea7a9fe207df..ec609603ea35b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -99,7 +99,7 @@ Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_ne (5) BroadcastExchange Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=1] +Arguments: HashedRelationBroadcastMode(List(((cast(input[1, int, true] as bigint) << 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=1] (6) Scan parquet spark_catalog.default.web_returns Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] @@ -209,7 +209,7 @@ Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, c (29) BroadcastExchange Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(((cast(input[1, int, true] as bigint) << 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=4] (30) Scan parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#42, cr_order_number#43, cr_return_quantity#44, cr_return_amount#45, cr_returned_date_sk#46] @@ -319,7 +319,7 @@ Input [6]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, (53) BroadcastExchange Input [5]: [ss_item_sk#70, ss_ticket_number#71, ss_quantity#72, ss_net_paid#73, ss_sold_date_sk#75] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(((cast(input[1, int, true] as bigint) << 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=7] (54) Scan parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#76, sr_ticket_number#77, sr_return_quantity#78, sr_return_amt#79, sr_returned_date_sk#80] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index 34c6ecf3cf2fa..1b9bf5123e965 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -317,7 +317,7 @@ Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, (49) BroadcastExchange Input [5]: [wr_item_sk#92, wr_order_number#93, wr_return_amt#94, wr_net_loss#95, wr_returned_date_sk#96] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] +Arguments: HashedRelationBroadcastMode(List(((cast(input[0, int, true] as bigint) << 32) | (cast(input[1, int, true] as bigint) & 4294967295))),false), [plan_id=5] (50) Scan parquet spark_catalog.default.web_sales Output [4]: [ws_item_sk#97, ws_web_site_sk#98, ws_order_number#99, ws_sold_date_sk#100] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index 40eddbbacf38a..4579b8bbe8197 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt @@ -201,7 +201,7 @@ Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AN (4) BroadcastExchange Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=1] +Arguments: HashedRelationBroadcastMode(List(((cast(input[0, int, false] as bigint) << 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=1] (5) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] @@ -714,7 +714,7 @@ Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113) (115) BroadcastExchange Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=16] +Arguments: HashedRelationBroadcastMode(List(((cast(input[0, int, false] as bigint) << 32) | (cast(input[7, int, false] as bigint) & 4294967295))),false), [plan_id=16] (116) Scan parquet spark_catalog.default.store_returns Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt index 13d7d1bc9c4d8..47974c9691023 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt @@ -264,7 +264,7 @@ Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) (42) BroadcastExchange Input [2]: [d_date_sk#25, d_week_seq#26] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, false] as bigint), 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(((cast(input[1, int, false] as bigint) << 32) | (cast(input[0, int, false] as bigint) & 4294967295))),false), [plan_id=6] (43) BroadcastHashJoin [codegen id : 10] Left keys [2]: [d_week_seq#24, inv_date_sk#13] 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 19251330cffe3..e80f4af1dc462 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 @@ -141,7 +141,10 @@ class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { // Examples demonstrate alternative syntax, see SPARK-45574 "org.apache.spark.sql.catalyst.expressions.Cast", // Examples demonstrate alternative syntax, see SPARK-47012 - "org.apache.spark.sql.catalyst.expressions.Collate" + "org.apache.spark.sql.catalyst.expressions.Collate", + classOf[ShiftLeft].getName, + classOf[ShiftRight].getName, + classOf[ShiftRightUnsigned].getName ) spark.sessionState.functionRegistry.listFunction().foreach { funcId => val info = spark.sessionState.catalog.lookupFunctionInfo(funcId) From 3cb30c2366b27c5a65ec02121c30bd1a4eb20584 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 24 May 2024 09:43:03 -0700 Subject: [PATCH 11/16] [SPARK-47579][SQL][FOLLOWUP] Restore the `--help` print format of spark sql shell ### What changes were proposed in this pull request? Restore the print format of spark sql shell ### Why are the changes needed? bugfix ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? manually ![image](https://github.com/apache/spark/assets/8326978/17b9d009-5d93-4d84-9367-7308b4cda426) ![image](https://github.com/apache/spark/assets/8326978/a5e333bd-0e22-4d5a-83f1-843767f6d5f5) ### Was this patch authored or co-authored using generative AI tooling? no Closes #46735 from yaooqinn/SPARK-47579. Authored-by: Kent Yao Signed-off-by: Gengliang Wang --- .../src/main/scala/org/apache/spark/internal/LogKey.scala | 1 - .../scala/org/apache/spark/deploy/SparkSubmitArguments.scala | 3 ++- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala index 1f67a211c01fa..99fc58b035030 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala @@ -585,7 +585,6 @@ object LogKeys { case object SESSION_KEY extends LogKey case object SET_CLIENT_INFO_REQUEST extends LogKey case object SHARD_ID extends LogKey - case object SHELL_OPTIONS extends LogKey case object SHORT_USER_NAME extends LogKey case object SHUFFLE_BLOCK_INFO extends LogKey case object SHUFFLE_DB_BACKEND_KEY extends LogKey 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 61235a7019070..e47596a6ae430 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -588,7 +588,8 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S ) if (SparkSubmit.isSqlShell(mainClass)) { - logInfo(log"CLI options:\n${MDC(SHELL_OPTIONS, getSqlShellOptions())}") + logInfo("CLI options:") + logInfo(getSqlShellOptions()) } throw SparkUserAppException(exitCode) From 7d96334902f22a80af63ce1253d5abda63178c4e Mon Sep 17 00:00:00 2001 From: Bo Zhang Date: Fri, 24 May 2024 15:54:21 -0700 Subject: [PATCH 12/16] [SPARK-48325][CORE] Always specify messages in ExecutorRunner.killProcess ### What changes were proposed in this pull request? This change is to always specify the message in `ExecutorRunner.killProcess`. ### Why are the changes needed? This is to get the occurrence rate for different cases when killing the executor process, in order to analyze executor running stability. ### 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 #46641 from bozhang2820/spark-48325. Authored-by: Bo Zhang Signed-off-by: Dongjoon Hyun --- .../apache/spark/deploy/worker/ExecutorRunner.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 7bb8b74eb0218..bd98f19cdb605 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -88,7 +88,7 @@ private[deploy] class ExecutorRunner( if (state == ExecutorState.LAUNCHING || state == ExecutorState.RUNNING) { state = ExecutorState.FAILED } - killProcess(Some("Worker shutting down")) } + killProcess("Worker shutting down") } } /** @@ -96,7 +96,7 @@ private[deploy] class ExecutorRunner( * * @param message the exception message which caused the executor's death */ - private def killProcess(message: Option[String]): Unit = { + private def killProcess(message: String): Unit = { var exitCode: Option[Int] = None if (process != null) { logInfo("Killing process!") @@ -113,7 +113,7 @@ private[deploy] class ExecutorRunner( } } try { - worker.send(ExecutorStateChanged(appId, execId, state, message, exitCode)) + worker.send(ExecutorStateChanged(appId, execId, state, Some(message), exitCode)) } catch { case e: IllegalStateException => logWarning(log"${MDC(ERROR, e.getMessage())}", e) } @@ -206,11 +206,11 @@ private[deploy] class ExecutorRunner( case interrupted: InterruptedException => logInfo("Runner thread for executor " + fullId + " interrupted") state = ExecutorState.KILLED - killProcess(None) + killProcess(s"Runner thread for executor $fullId interrupted") case e: Exception => logError("Error running executor", e) state = ExecutorState.FAILED - killProcess(Some(e.toString)) + killProcess(s"Error running executor: $e") } } } From 1a536f01ead35b770467381c476e093338d81e7c Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 24 May 2024 15:56:19 -0700 Subject: [PATCH 13/16] [SPARK-48407][SQL][DOCS] Teradata: Document Type Conversion rules between Spark SQL and teradata ### What changes were proposed in this pull request? This PR adds documentation for the builtin teradata jdbc dialect's data type conversion rules ### Why are the changes needed? doc improvement ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ![image](https://github.com/apache/spark/assets/8326978/e1ec0de5-cd83-4339-896a-50c58ad01c4d) ### Was this patch authored or co-authored using generative AI tooling? no Closes #46728 from yaooqinn/SPARK-48407. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- docs/sql-data-sources-jdbc.md | 214 ++++++++++++++++++++++++++++++++++ 1 file changed, 214 insertions(+) diff --git a/docs/sql-data-sources-jdbc.md b/docs/sql-data-sources-jdbc.md index 371dc05950717..9ffd96cd40ee5 100644 --- a/docs/sql-data-sources-jdbc.md +++ b/docs/sql-data-sources-jdbc.md @@ -1991,3 +1991,217 @@ The Spark Catalyst data types below are not supported with suitable DB2 types. - NullType - ObjectType - VariantType + +### Mapping Spark SQL Data Types from Teradata + +The below table describes the data type conversions from Teradata data types to Spark SQL Data Types, +when reading data from a Teradata table using the built-in jdbc data source with the [Teradata JDBC Driver](https://mvnrepository.com/artifact/com.teradata.jdbc/terajdbc) +as the activated JDBC Driver. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Teradata Data TypeSpark SQL Data TypeRemarks
BYTEINTByteType
SMALLINTShortType
INTEGER, INTIntegerType
BIGINTLongType
REAL, DOUBLE PRECISION, FLOATDoubleType
DECIMAL, NUMERIC, NUMBERDecimalType
DATEDateType
TIMESTAMP, TIMESTAMP WITH TIME ZONETimestampType(Default)preferTimestampNTZ=false or spark.sql.timestampType=TIMESTAMP_LTZ
TIMESTAMP, TIMESTAMP WITH TIME ZONETimestampNTZTypepreferTimestampNTZ=true or spark.sql.timestampType=TIMESTAMP_NTZ
TIME, TIME WITH TIME ZONETimestampType(Default)preferTimestampNTZ=false or spark.sql.timestampType=TIMESTAMP_LTZ
TIME, TIME WITH TIME ZONETimestampNTZTypepreferTimestampNTZ=true or spark.sql.timestampType=TIMESTAMP_NTZ
CHARACTER(n), CHAR(n), GRAPHIC(n)CharType(n)
VARCHAR(n), VARGRAPHIC(n)VarcharType(n)
BYTE(n), VARBYTE(n)BinaryType
CLOBStringType
BLOBBinaryType
INTERVAL Data Types-The INTERVAL data types are unknown yet
Period Data Types, ARRAY, UDT-Not Supported
+ +### Mapping Spark SQL Data Types to Teradata + +The below table describes the data type conversions from Spark SQL Data Types to Teradata data types, +when creating, altering, or writing data to a Teradata table using the built-in jdbc data source with +the [Teradata JDBC Driver](https://mvnrepository.com/artifact/com.teradata.jdbc/terajdbc) as the activated JDBC Driver. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Spark SQL Data TypeTeradata Data TypeRemarks
BooleanTypeCHAR(1)
ByteTypeBYTEINT
ShortTypeSMALLINT
IntegerTypeINTEGER
LongTypeBIGINT
FloatTypeREAL
DoubleTypeDOUBLE PRECISION
DecimalType(p, s)DECIMAL(p,s)
DateTypeDATE
TimestampTypeTIMESTAMP
TimestampNTZTypeTIMESTAMP
StringTypeVARCHAR(255)
BinaryTypeBLOB
CharType(n)CHAR(n)
VarcharType(n)VARCHAR(n)
+ +The Spark Catalyst data types below are not supported with suitable Teradata types. + +- DayTimeIntervalType +- YearMonthIntervalType +- CalendarIntervalType +- ArrayType +- MapType +- StructType +- UserDefinedType +- NullType +- ObjectType +- VariantType From 6cd1ccc56321dfa52672cd25f4cfdf2bbc86b3ea Mon Sep 17 00:00:00 2001 From: Yi Wu Date: Fri, 24 May 2024 16:01:17 -0700 Subject: [PATCH 14/16] [SPARK-48394][CORE] Cleanup mapIdToMapIndex on mapoutput unregister ### What changes were proposed in this pull request? This PR cleans up `mapIdToMapIndex` when the corresponding mapstatus is unregistered in three places: * `removeMapOutput` * `removeOutputsByFilter` * `addMapOutput` (old mapstatus overwritten) ### Why are the changes needed? There is only one valid mapstatus for the same `mapIndex` at the same time in Spark. `mapIdToMapIndex` should also follows the same rule to avoid chaos. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46706 from Ngone51/SPARK-43043-followup. Lead-authored-by: Yi Wu Co-authored-by: wuyi Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/MapOutputTracker.scala | 26 ++++++--- .../apache/spark/MapOutputTrackerSuite.scala | 55 +++++++++++++++++++ 2 files changed, 72 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index fdc2b0a4c20f0..a660bccd2e68f 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -44,7 +44,6 @@ import org.apache.spark.shuffle.MetadataFetchFailedException import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId, ShuffleMergedBlockId} import org.apache.spark.util._ import org.apache.spark.util.ArrayImplicits._ -import org.apache.spark.util.collection.OpenHashMap import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} /** @@ -153,17 +152,22 @@ private class ShuffleStatus( /** * Mapping from a mapId to the mapIndex, this is required to reduce the searching overhead within * the function updateMapOutput(mapId, bmAddress). + * + * Exposed for testing. */ - private[this] val mapIdToMapIndex = new OpenHashMap[Long, Int]() + private[spark] val mapIdToMapIndex = new HashMap[Long, Int]() /** * Register a map output. If there is already a registered location for the map output then it * will be replaced by the new location. */ def addMapOutput(mapIndex: Int, status: MapStatus): Unit = withWriteLock { - if (mapStatuses(mapIndex) == null) { + val currentMapStatus = mapStatuses(mapIndex) + if (currentMapStatus == null) { _numAvailableMapOutputs += 1 invalidateSerializedMapOutputStatusCache() + } else { + mapIdToMapIndex.remove(currentMapStatus.mapId) } mapStatuses(mapIndex) = status mapIdToMapIndex(status.mapId) = mapIndex @@ -193,8 +197,8 @@ private class ShuffleStatus( mapStatus.updateLocation(bmAddress) invalidateSerializedMapOutputStatusCache() case None => - if (mapIndex.map(mapStatusesDeleted).exists(_.mapId == mapId)) { - val index = mapIndex.get + val index = mapStatusesDeleted.indexWhere(x => x != null && x.mapId == mapId) + if (index >= 0 && mapStatuses(index) == null) { val mapStatus = mapStatusesDeleted(index) mapStatus.updateLocation(bmAddress) mapStatuses(index) = mapStatus @@ -222,9 +226,11 @@ private class ShuffleStatus( */ def removeMapOutput(mapIndex: Int, bmAddress: BlockManagerId): Unit = withWriteLock { logDebug(s"Removing existing map output ${mapIndex} ${bmAddress}") - if (mapStatuses(mapIndex) != null && mapStatuses(mapIndex).location == bmAddress) { + val currentMapStatus = mapStatuses(mapIndex) + if (currentMapStatus != null && currentMapStatus.location == bmAddress) { _numAvailableMapOutputs -= 1 - mapStatusesDeleted(mapIndex) = mapStatuses(mapIndex) + mapIdToMapIndex.remove(currentMapStatus.mapId) + mapStatusesDeleted(mapIndex) = currentMapStatus mapStatuses(mapIndex) = null invalidateSerializedMapOutputStatusCache() } @@ -290,9 +296,11 @@ private class ShuffleStatus( */ def removeOutputsByFilter(f: BlockManagerId => Boolean): Unit = withWriteLock { for (mapIndex <- mapStatuses.indices) { - if (mapStatuses(mapIndex) != null && f(mapStatuses(mapIndex).location)) { + val currentMapStatus = mapStatuses(mapIndex) + if (currentMapStatus != null && f(currentMapStatus.location)) { _numAvailableMapOutputs -= 1 - mapStatusesDeleted(mapIndex) = mapStatuses(mapIndex) + mapIdToMapIndex.remove(currentMapStatus.mapId) + mapStatusesDeleted(mapIndex) = currentMapStatus mapStatuses(mapIndex) = null invalidateSerializedMapOutputStatusCache() } diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 7aec8eeaad423..26dc218c30c74 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -1110,4 +1110,59 @@ class MapOutputTrackerSuite extends SparkFunSuite with LocalSparkContext { rpcEnv.shutdown() } } + + test( + "SPARK-48394: mapIdToMapIndex should cleanup unused mapIndexes after removeOutputsByFilter" + ) { + val rpcEnv = createRpcEnv("test") + val tracker = newTrackerMaster() + try { + tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) + tracker.registerShuffle(0, 1, 1) + tracker.registerMapOutput(0, 0, MapStatus(BlockManagerId("exec-1", "hostA", 1000), + Array(2L), 0)) + tracker.removeOutputsOnHost("hostA") + assert(tracker.shuffleStatuses(0).mapIdToMapIndex.filter(_._2 == 0).size == 0) + } finally { + tracker.stop() + rpcEnv.shutdown() + } + } + + test("SPARK-48394: mapIdToMapIndex should cleanup unused mapIndexes after unregisterMapOutput") { + val rpcEnv = createRpcEnv("test") + val tracker = newTrackerMaster() + try { + tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) + tracker.registerShuffle(0, 1, 1) + tracker.registerMapOutput(0, 0, MapStatus(BlockManagerId("exec-1", "hostA", 1000), + Array(2L), 0)) + tracker.unregisterMapOutput(0, 0, BlockManagerId("exec-1", "hostA", 1000)) + assert(tracker.shuffleStatuses(0).mapIdToMapIndex.filter(_._2 == 0).size == 0) + } finally { + tracker.stop() + rpcEnv.shutdown() + } + } + + test("SPARK-48394: mapIdToMapIndex should cleanup unused mapIndexes after registerMapOutput") { + val rpcEnv = createRpcEnv("test") + val tracker = newTrackerMaster() + try { + tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) + tracker.registerShuffle(0, 1, 1) + tracker.registerMapOutput(0, 0, MapStatus(BlockManagerId("exec-1", "hostA", 1000), + Array(2L), 0)) + // Another task also finished working on partition 0. + tracker.registerMapOutput(0, 0, MapStatus(BlockManagerId("exec-2", "hostB", 1000), + Array(2L), 1)) + assert(tracker.shuffleStatuses(0).mapIdToMapIndex.filter(_._2 == 0).size == 1) + } finally { + tracker.stop() + rpcEnv.shutdown() + } + } } From cff7014b337be18f14289ad8c50c3e08c214a9e4 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 24 May 2024 20:38:07 -0700 Subject: [PATCH 15/16] [SPARK-47579][CORE][PART3] Spark core: Migrate logInfo with variables to structured logging framework ### What changes were proposed in this pull request? The PR aims to migrate logInfo in Core module with variables to structured logging framework. ### Why are the changes needed? To enhance Apache Spark's logging system by implementing structured logging. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? GA tests ### Was this patch authored or co-authored using generative AI tooling? Yes, Generated-by: Github Copilot Github Copilot provides a few suggestions. Closes #46739 from gengliangwang/logInfoCoreP3. Authored-by: Gengliang Wang Signed-off-by: Gengliang Wang --- .../shuffle/RetryingBlockTransferor.java | 6 +- .../org/apache/spark/internal/LogKey.scala | 51 ++++- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../spark/deploy/LocalSparkCluster.scala | 5 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 7 +- .../deploy/client/StandaloneAppClient.scala | 19 +- .../deploy/master/RecoveryModeFactory.scala | 7 +- .../deploy/rest/RestSubmissionClient.scala | 30 ++- .../spark/deploy/worker/CommandUtils.scala | 5 +- .../spark/deploy/worker/ExecutorRunner.scala | 2 +- .../apache/spark/deploy/worker/Worker.scala | 96 ++++---- .../spark/memory/ExecutionMemoryPool.scala | 2 +- .../org/apache/spark/rdd/HadoopRDD.scala | 2 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 2 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 2 +- .../spark/rdd/SequenceFileRDDFunctions.scala | 7 +- .../apache/spark/scheduler/DAGScheduler.scala | 210 ++++++++++-------- .../spark/scheduler/HealthTracker.scala | 35 +-- .../scheduler/OutputCommitCoordinator.scala | 16 +- .../spark/scheduler/SchedulableBuilder.scala | 32 ++- .../spark/scheduler/StatsReportListener.scala | 9 +- .../spark/scheduler/TaskSchedulerImpl.scala | 69 +++--- .../spark/scheduler/TaskSetExcludeList.scala | 9 +- .../spark/scheduler/TaskSetManager.scala | 67 +++--- .../CoarseGrainedSchedulerBackend.scala | 48 ++-- .../cluster/StandaloneSchedulerBackend.scala | 25 ++- .../scheduler/dynalloc/ExecutorMonitor.scala | 18 +- .../apache/spark/storage/BlockManager.scala | 30 +-- .../storage/BlockManagerDecommissioner.scala | 60 +++-- .../spark/storage/BlockManagerMaster.scala | 10 +- .../storage/BlockManagerMasterEndpoint.scala | 58 ++--- .../spark/storage/PushBasedFetchHelper.scala | 3 +- .../storage/ShuffleBlockFetcherIterator.scala | 35 +-- .../spark/storage/memory/MemoryStore.scala | 37 +-- 34 files changed, 614 insertions(+), 402 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockTransferor.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockTransferor.java index 83be2db5d0b73..31c454f63a92e 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockTransferor.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockTransferor.java @@ -183,12 +183,12 @@ private void transferAllOutstanding() { if (numRetries > 0) { logger.error("Exception while beginning {} of {} outstanding blocks (after {} retries)", e, MDC.of(LogKeys.TRANSFER_TYPE$.MODULE$, listener.getTransferType()), - MDC.of(LogKeys.NUM_BLOCK_IDS$.MODULE$, blockIdsToTransfer.length), + MDC.of(LogKeys.NUM_BLOCKS$.MODULE$, blockIdsToTransfer.length), MDC.of(LogKeys.NUM_RETRY$.MODULE$, numRetries)); } else { logger.error("Exception while beginning {} of {} outstanding blocks", e, MDC.of(LogKeys.TRANSFER_TYPE$.MODULE$, listener.getTransferType()), - MDC.of(LogKeys.NUM_BLOCK_IDS$.MODULE$, blockIdsToTransfer.length)); + MDC.of(LogKeys.NUM_BLOCKS$.MODULE$, blockIdsToTransfer.length)); } if (shouldRetry(e) && initiateRetry(e)) { // successfully initiated a retry @@ -219,7 +219,7 @@ synchronized boolean initiateRetry(Throwable e) { MDC.of(LogKeys.TRANSFER_TYPE$.MODULE$, listener.getTransferType()), MDC.of(LogKeys.NUM_RETRY$.MODULE$, retryCount), MDC.of(LogKeys.MAX_ATTEMPTS$.MODULE$, maxRetries), - MDC.of(LogKeys.NUM_BLOCK_IDS$.MODULE$, outstandingBlocksIds.size()), + MDC.of(LogKeys.NUM_BLOCKS$.MODULE$, outstandingBlocksIds.size()), MDC.of(LogKeys.RETRY_WAIT_TIME$.MODULE$, retryWaitTime)); try { diff --git a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala index 99fc58b035030..534f009119226 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala @@ -72,6 +72,7 @@ object LogKeys { case object CACHE_UNTIL_LAST_PRODUCED_SIZE extends LogKey case object CALL_SITE_LONG_FORM extends LogKey case object CALL_SITE_SHORT_FORM extends LogKey + case object CANCEL_FUTURE_JOBS extends LogKey case object CATALOG_NAME extends LogKey case object CATEGORICAL_FEATURES extends LogKey case object CHECKPOINT_FILE extends LogKey @@ -118,10 +119,10 @@ object LogKeys { case object CONTAINER_ID extends LogKey case object CONTAINER_STATE extends LogKey case object CONTEXT extends LogKey - case object CONTEXT_CREATION_SITE extends LogKey case object COST extends LogKey case object COUNT extends LogKey case object CREATED_POOL_NAME extends LogKey + case object CREATION_SITE extends LogKey case object CREDENTIALS_RENEWAL_INTERVAL_RATIO extends LogKey case object CROSS_VALIDATION_METRIC extends LogKey case object CROSS_VALIDATION_METRICS extends LogKey @@ -132,8 +133,9 @@ object LogKeys { case object CSV_SCHEMA_FIELD_NAMES extends LogKey case object CSV_SOURCE extends LogKey case object CURRENT_BATCH_ID extends LogKey + case object CURRENT_DISK_SIZE extends LogKey case object CURRENT_FILE extends LogKey - case object CURRENT_MEMORY_BYTES extends LogKey + case object CURRENT_MEMORY_SIZE extends LogKey case object CURRENT_PATH extends LogKey case object CURRENT_TIME extends LogKey case object DATA extends LogKey @@ -146,7 +148,6 @@ object LogKeys { case object DEFAULT_COMPACT_INTERVAL extends LogKey case object DEFAULT_ISOLATION_LEVEL extends LogKey case object DEFAULT_NAME extends LogKey - case object DEFAULT_SCHEDULING_MODE extends LogKey case object DEFAULT_VALUE extends LogKey case object DELAY extends LogKey case object DELEGATE extends LogKey @@ -207,6 +208,8 @@ object LogKeys { case object EXPR extends LogKey case object EXPR_TERMS extends LogKey case object EXTENDED_EXPLAIN_GENERATOR extends LogKey + case object FAILED_STAGE extends LogKey + case object FAILED_STAGE_NAME extends LogKey case object FAILURES extends LogKey case object FALLBACK_VERSION extends LogKey case object FEATURE_COLUMN extends LogKey @@ -231,6 +234,7 @@ object LogKeys { case object FINAL_OUTPUT_PATH extends LogKey case object FINAL_PATH extends LogKey case object FINISH_TRIGGER_DURATION extends LogKey + case object FREE_MEMORY_SIZE extends LogKey case object FROM_OFFSET extends LogKey case object FROM_TIME extends LogKey case object FUNCTION_NAME extends LogKey @@ -250,6 +254,7 @@ object LogKeys { case object HIVE_OPERATION_STATE extends LogKey case object HIVE_OPERATION_TYPE extends LogKey case object HOST extends LogKey + case object HOST_LOCAL_BLOCKS_SIZE extends LogKey case object HOST_NAMES extends LogKey case object HOST_PORT extends LogKey case object HOST_PORT2 extends LogKey @@ -265,6 +270,7 @@ object LogKeys { case object INITIAL_HEARTBEAT_INTERVAL extends LogKey case object INIT_MODE extends LogKey case object INPUT extends LogKey + case object INPUT_SPLIT extends LogKey case object INTERVAL extends LogKey case object ISOLATION_LEVEL extends LogKey case object ISSUE_DATE extends LogKey @@ -299,11 +305,14 @@ object LogKeys { case object LOAD_FACTOR extends LogKey case object LOAD_TIME extends LogKey case object LOCALE extends LogKey + case object LOCAL_BLOCKS_SIZE extends LogKey case object LOCAL_SCRATCH_DIR extends LogKey case object LOCATION extends LogKey case object LOGICAL_PLAN_COLUMNS extends LogKey case object LOGICAL_PLAN_LEAVES extends LogKey case object LOG_ID extends LogKey + case object LOG_KEY_FILE extends LogKey + case object LOG_LEVEL extends LogKey case object LOG_OFFSET extends LogKey case object LOG_TYPE extends LogKey case object LOWER_BOUND extends LogKey @@ -351,6 +360,7 @@ object LogKeys { case object MIN_SIZE extends LogKey case object MIN_TIME extends LogKey case object MIN_VERSION_NUM extends LogKey + case object MISSING_PARENT_STAGES extends LogKey case object MODEL_WEIGHTS extends LogKey case object MODULE_NAME extends LogKey case object NAMESPACE extends LogKey @@ -368,8 +378,9 @@ object LogKeys { case object NORM extends LogKey case object NUM_ADDED_PARTITIONS extends LogKey case object NUM_APPS extends LogKey + case object NUM_ATTEMPT extends LogKey case object NUM_BIN extends LogKey - case object NUM_BLOCK_IDS extends LogKey + case object NUM_BLOCKS extends LogKey case object NUM_BROADCAST_BLOCK extends LogKey case object NUM_BYTES extends LogKey case object NUM_BYTES_CURRENT extends LogKey @@ -388,12 +399,16 @@ object LogKeys { case object NUM_CORES extends LogKey case object NUM_DATA_FILE extends LogKey case object NUM_DATA_FILES extends LogKey + case object NUM_DECOMMISSIONED extends LogKey case object NUM_DRIVERS extends LogKey case object NUM_DROPPED_PARTITIONS extends LogKey case object NUM_EFFECTIVE_RULE_OF_RUNS extends LogKey case object NUM_ELEMENTS_SPILL_THRESHOLD extends LogKey case object NUM_EVENTS extends LogKey case object NUM_EXAMPLES extends LogKey + case object NUM_EXECUTORS extends LogKey + case object NUM_EXECUTORS_EXITED extends LogKey + case object NUM_EXECUTORS_KILLED extends LogKey case object NUM_EXECUTOR_CORES extends LogKey case object NUM_EXECUTOR_CORES_REMAINING extends LogKey case object NUM_EXECUTOR_CORES_TOTAL extends LogKey @@ -407,6 +422,7 @@ object LogKeys { case object NUM_FILES_FAILED_TO_DELETE extends LogKey case object NUM_FILES_REUSED extends LogKey case object NUM_FREQUENT_ITEMS extends LogKey + case object NUM_HOST_LOCAL_BLOCKS extends LogKey case object NUM_INDEX_FILE extends LogKey case object NUM_INDEX_FILES extends LogKey case object NUM_ITERATIONS extends LogKey @@ -415,8 +431,10 @@ object LogKeys { case object NUM_LEADING_SINGULAR_VALUES extends LogKey case object NUM_LEFT_PARTITION_VALUES extends LogKey case object NUM_LOADED_ENTRIES extends LogKey + case object NUM_LOCAL_BLOCKS extends LogKey case object NUM_LOCAL_DIRS extends LogKey case object NUM_LOCAL_FREQUENT_PATTERN extends LogKey + case object NUM_MERGERS extends LogKey case object NUM_MERGER_LOCATIONS extends LogKey case object NUM_META_FILES extends LogKey case object NUM_NODES extends LogKey @@ -433,7 +451,10 @@ object LogKeys { case object NUM_POINT extends LogKey case object NUM_PREFIXES extends LogKey case object NUM_PRUNED extends LogKey + case object NUM_PUSH_MERGED_LOCAL_BLOCKS extends LogKey case object NUM_RECORDS_READ extends LogKey + case object NUM_REMAINED extends LogKey + case object NUM_REMOTE_BLOCKS extends LogKey case object NUM_REMOVED_WORKERS extends LogKey case object NUM_REPLICAS extends LogKey case object NUM_REQUESTS extends LogKey @@ -449,11 +470,14 @@ object LogKeys { case object NUM_SPILL_INFOS extends LogKey case object NUM_SPILL_WRITERS extends LogKey case object NUM_SUB_DIRS extends LogKey + case object NUM_SUCCESSFUL_TASKS extends LogKey case object NUM_TASKS extends LogKey case object NUM_TASK_CPUS extends LogKey case object NUM_TRAIN_WORD extends LogKey + case object NUM_UNFINISHED_DECOMMISSIONED extends LogKey case object NUM_VERSIONS_RETAIN extends LogKey case object NUM_WEIGHTED_EXAMPLES extends LogKey + case object NUM_WORKERS extends LogKey case object OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD extends LogKey case object OBJECT_ID extends LogKey case object OFFSET extends LogKey @@ -470,16 +494,20 @@ object LogKeys { case object OPTIONS extends LogKey case object OP_ID extends LogKey case object OP_TYPE extends LogKey + case object ORIGINAL_DISK_SIZE extends LogKey + case object ORIGINAL_MEMORY_SIZE extends LogKey case object OS_ARCH extends LogKey case object OS_NAME extends LogKey case object OS_VERSION extends LogKey case object OUTPUT extends LogKey case object OVERHEAD_MEMORY_SIZE extends LogKey case object PAGE_SIZE extends LogKey + case object PARENT_STAGES extends LogKey case object PARSE_MODE extends LogKey case object PARTITIONED_FILE_READER extends LogKey case object PARTITIONER extends LogKey case object PARTITION_ID extends LogKey + case object PARTITION_IDS extends LogKey case object PARTITION_SPECIFICATION extends LogKey case object PARTITION_SPECS extends LogKey case object PATH extends LogKey @@ -511,12 +539,14 @@ object LogKeys { case object PROTOCOL_VERSION extends LogKey case object PROVIDER extends LogKey case object PUSHED_FILTERS extends LogKey + case object PUSH_MERGED_LOCAL_BLOCKS_SIZE extends LogKey case object PVC_METADATA_NAME extends LogKey case object PYTHON_EXEC extends LogKey case object PYTHON_PACKAGES extends LogKey case object PYTHON_VERSION extends LogKey case object PYTHON_WORKER_MODULE extends LogKey case object PYTHON_WORKER_RESPONSE extends LogKey + case object QUANTILES extends LogKey case object QUERY_CACHE_VALUE extends LogKey case object QUERY_HINT extends LogKey case object QUERY_ID extends LogKey @@ -542,11 +572,13 @@ object LogKeys { case object REDACTED_STATEMENT extends LogKey case object REDUCE_ID extends LogKey case object REGISTERED_EXECUTOR_FILE extends LogKey + case object REGISTER_MERGE_RESULTS extends LogKey case object RELATION_NAME extends LogKey case object RELATION_OUTPUT extends LogKey case object RELATIVE_TOLERANCE extends LogKey case object REMAINING_PARTITIONS extends LogKey case object REMOTE_ADDRESS extends LogKey + case object REMOTE_BLOCKS_SIZE extends LogKey case object REMOVE_FROM_MASTER extends LogKey case object REPORT_DETAILS extends LogKey case object REQUESTER_SIZE extends LogKey @@ -574,6 +606,7 @@ object LogKeys { case object RUN_ID extends LogKey case object SCALA_VERSION extends LogKey case object SCHEDULER_POOL_NAME extends LogKey + case object SCHEDULING_MODE extends LogKey case object SCHEMA extends LogKey case object SCHEMA2 extends LogKey case object SERVER_NAME extends LogKey @@ -615,13 +648,17 @@ object LogKeys { case object SPILL_TIMES extends LogKey case object SQL_TEXT extends LogKey case object SRC_PATH extends LogKey + case object STAGE extends LogKey + case object STAGES extends LogKey case object STAGE_ATTEMPT extends LogKey case object STAGE_ID extends LogKey + case object STAGE_NAME extends LogKey case object START_INDEX extends LogKey case object STATEMENT_ID extends LogKey case object STATE_STORE_ID extends LogKey case object STATE_STORE_PROVIDER extends LogKey case object STATE_STORE_VERSION extends LogKey + case object STATS extends LogKey case object STATUS extends LogKey case object STDERR extends LogKey case object STOP_SITE_SHORT_FORM extends LogKey @@ -647,13 +684,18 @@ object LogKeys { case object TABLE_NAME extends LogKey case object TABLE_TYPE extends LogKey case object TABLE_TYPES extends LogKey + case object TAG extends LogKey case object TARGET_NUM_EXECUTOR extends LogKey case object TARGET_NUM_EXECUTOR_DELTA extends LogKey case object TARGET_PATH extends LogKey case object TASK_ATTEMPT_ID extends LogKey case object TASK_ID extends LogKey + case object TASK_LOCALITY extends LogKey case object TASK_NAME extends LogKey case object TASK_REQUIREMENTS extends LogKey + case object TASK_RESOURCE_ASSIGNMENTS extends LogKey + case object TASK_SET_ID extends LogKey + case object TASK_SET_MANAGER extends LogKey case object TASK_SET_NAME extends LogKey case object TASK_STATE extends LogKey case object TEMP_FILE extends LogKey @@ -685,6 +727,7 @@ object LogKeys { case object TOPIC_PARTITION_OFFSET_RANGE extends LogKey case object TOTAL extends LogKey case object TOTAL_EFFECTIVE_TIME extends LogKey + case object TOTAL_SIZE extends LogKey case object TOTAL_TIME extends LogKey case object TOTAL_TIME_READ extends LogKey case object TO_TIME extends LogKey diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 6018c87b01224..c70576b8adc10 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2930,7 +2930,7 @@ object SparkContext extends Logging { log" constructor). This may indicate an error, since only one SparkContext should be" + log" running in this JVM (see SPARK-2243)." + log" The other SparkContext was created at:\n" + - log"${MDC(LogKeys.CONTEXT_CREATION_SITE, otherContextCreationSite)}" + log"${MDC(LogKeys.CREATION_SITE, otherContextCreationSite)}" logWarning(warnMsg) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index 9c57269b28f47..263b1a233b808 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkConf import org.apache.spark.deploy.master.Master import org.apache.spark.deploy.worker.Worker -import org.apache.spark.internal.{config, Logging} +import org.apache.spark.internal.{config, Logging, LogKeys, MDC} import org.apache.spark.rpc.RpcEnv import org.apache.spark.util.Utils @@ -51,7 +51,8 @@ class LocalSparkCluster private ( private val workerDirs = ArrayBuffer[String]() def start(): Array[String] = { - logInfo("Starting a local Spark cluster with " + numWorkers + " workers.") + logInfo(log"Starting a local Spark cluster with " + + log"${MDC(LogKeys.NUM_WORKERS, numWorkers)} workers.") // Disable REST server on Master in this mode unless otherwise specified val _conf = conf.clone() diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 2edd80db2637f..ca932ef5dc05c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -37,7 +37,7 @@ import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.config.BUFFER_SIZE import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils @@ -142,8 +142,9 @@ private[spark] class SparkHadoopUtil extends Logging { if (!new File(keytabFilename).exists()) { throw new SparkException(s"Keytab file: ${keytabFilename} does not exist") } else { - logInfo("Attempting to login to Kerberos " + - s"using principal: ${principalName} and keytab: ${keytabFilename}") + logInfo(log"Attempting to login to Kerberos using principal: " + + log"${MDC(LogKeys.PRINCIPAL, principalName)} and keytab: " + + log"${MDC(LogKeys.KEYTAB, keytabFilename)}") UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala index ec231610b8575..b34e5c408c3be 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala @@ -29,7 +29,7 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.LogKeys._ import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc._ @@ -105,7 +105,8 @@ private[spark] class StandaloneAppClient( if (registered.get) { return } - logInfo("Connecting to master " + masterAddress.toSparkURL + "...") + logInfo( + log"Connecting to master ${MDC(LogKeys.MASTER_URL, masterAddress.toSparkURL)}...") val masterRef = rpcEnv.setupEndpointRef(masterAddress, Master.ENDPOINT_NAME) masterRef.send(RegisterApplication(appDescription, self)) } catch { @@ -175,14 +176,16 @@ private[spark] class StandaloneAppClient( case ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) => val fullId = s"$appId/$id" - logInfo("Executor added: %s on %s (%s) with %d core(s)".format(fullId, workerId, hostPort, - cores)) + logInfo(log"Executor added: ${MDC(LogKeys.EXECUTOR_ID, fullId)} on " + + log"${MDC(LogKeys.WORKER_ID, workerId)} (${MDC(LogKeys.HOST_PORT, hostPort)}) " + + log"with ${MDC(LogKeys.NUM_CORES, cores)} core(s)") listener.executorAdded(fullId, workerId, hostPort, cores, memory) case ExecutorUpdated(id, state, message, exitStatus, workerHost) => val fullId = s"$appId/$id" val messageText = message.map(s => " (" + s + ")").getOrElse("") - logInfo("Executor updated: %s is now %s%s".format(fullId, state, messageText)) + logInfo(log"Executor updated: ${MDC(LogKeys.EXECUTOR_ID, fullId)} is now " + + log"${MDC(LogKeys.EXECUTOR_STATE, state)}${MDC(LogKeys.MESSAGE, messageText)}") if (ExecutorState.isFinished(state)) { listener.executorRemoved(fullId, message.getOrElse(""), exitStatus, workerHost) } else if (state == ExecutorState.DECOMMISSIONED) { @@ -191,11 +194,13 @@ private[spark] class StandaloneAppClient( } case WorkerRemoved(id, host, message) => - logInfo("Master removed worker %s: %s".format(id, message)) + logInfo(log"Master removed worker ${MDC(LogKeys.WORKER_ID, id)}: " + + log"${MDC(LogKeys.MESSAGE, message)}") listener.workerRemoved(id, host, message) case MasterChanged(masterRef, masterWebUiUrl) => - logInfo("Master has changed, new master is at " + masterRef.address.toSparkURL) + logInfo(log"Master has changed, new master is at " + + log"${MDC(LogKeys.MASTER_URL, masterRef.address.toSparkURL)}") master = Some(masterRef) alreadyDisconnected = false masterRef.send(MasterChangeAcknowledged(appId.get)) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala index 106acc9a79446..964b115865aef 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.master import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.config.Deploy.{RECOVERY_COMPRESSION_CODEC, RECOVERY_DIRECTORY} import org.apache.spark.io.CompressionCodec import org.apache.spark.serializer.Serializer @@ -57,7 +57,7 @@ private[master] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: val recoveryDir = conf.get(RECOVERY_DIRECTORY) def createPersistenceEngine(): PersistenceEngine = { - logInfo("Persisting recovery state to directory: " + recoveryDir) + logInfo(log"Persisting recovery state to directory: ${MDC(LogKeys.PATH, recoveryDir)}") val codec = conf.get(RECOVERY_COMPRESSION_CODEC).map(c => CompressionCodec.createCodec(conf, c)) new FileSystemPersistenceEngine(recoveryDir, serializer, codec) } @@ -76,7 +76,8 @@ private[master] class RocksDBRecoveryModeFactory(conf: SparkConf, serializer: Se def createPersistenceEngine(): PersistenceEngine = { val recoveryDir = conf.get(RECOVERY_DIRECTORY) - logInfo("Persisting recovery state to directory: " + recoveryDir) + logInfo(log"Persisting recovery state to directory: " + + log"${MDC(LogKeys.PATH, recoveryDir)}") new RocksDBPersistenceEngine(recoveryDir, serializer) } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala index 247504f5ebbb9..4fb95033cecef 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala @@ -79,7 +79,7 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { * it to the user. Otherwise, report the error message provided by the server. */ def createSubmission(request: CreateSubmissionRequest): SubmitRestProtocolResponse = { - logInfo(s"Submitting a request to launch an application in $master.") + logInfo(log"Submitting a request to launch an application in ${MDC(MASTER_URL, master)}.") var handled: Boolean = false var response: SubmitRestProtocolResponse = null for (m <- masters if !handled) { @@ -109,7 +109,9 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { /** Request that the server kill the specified submission. */ def killSubmission(submissionId: String): SubmitRestProtocolResponse = { - logInfo(s"Submitting a request to kill submission $submissionId in $master.") + logInfo(log"Submitting a request to kill submission " + + log"${MDC(SUBMISSION_ID, submissionId)} in " + + log"${MDC(MASTER_URL, master)}.") var handled: Boolean = false var response: SubmitRestProtocolResponse = null for (m <- masters if !handled) { @@ -138,7 +140,7 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { /** Request that the server kill all submissions. */ def killAllSubmissions(): SubmitRestProtocolResponse = { - logInfo(s"Submitting a request to kill all submissions in $master.") + logInfo(log"Submitting a request to kill all submissions in ${MDC(MASTER_URL, master)}.") var handled: Boolean = false var response: SubmitRestProtocolResponse = null for (m <- masters if !handled) { @@ -167,7 +169,7 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { /** Request that the server clears all submissions and applications. */ def clear(): SubmitRestProtocolResponse = { - logInfo(s"Submitting a request to clear $master.") + logInfo(log"Submitting a request to clear ${MDC(MASTER_URL, master)}.") var handled: Boolean = false var response: SubmitRestProtocolResponse = null for (m <- masters if !handled) { @@ -196,7 +198,7 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { /** Check the readiness of Master. */ def readyz(): SubmitRestProtocolResponse = { - logInfo(s"Submitting a request to check the status of $master.") + logInfo(log"Submitting a request to check the status of ${MDC(MASTER_URL, master)}.") var handled: Boolean = false var response: SubmitRestProtocolResponse = new ErrorResponse for (m <- masters if !handled) { @@ -227,7 +229,9 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { def requestSubmissionStatus( submissionId: String, quiet: Boolean = false): SubmitRestProtocolResponse = { - logInfo(s"Submitting a request for the status of submission $submissionId in $master.") + logInfo(log"Submitting a request for the status of submission " + + log"${MDC(SUBMISSION_ID, submissionId)} in " + + log"${MDC(MASTER_URL, master)}.") var handled: Boolean = false var response: SubmitRestProtocolResponse = null @@ -440,7 +444,8 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { if (submitResponse.success) { val submissionId = submitResponse.submissionId if (submissionId != null) { - logInfo(s"Submission successfully created as $submissionId. Polling submission state...") + logInfo(log"Submission successfully created as ${MDC(SUBMISSION_ID, submissionId)}. " + + log"Polling submission state...") pollSubmissionStatus(submissionId) } else { // should never happen @@ -470,13 +475,17 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { val exception = Option(statusResponse.message) // Log driver state, if present driverState match { - case Some(state) => logInfo(s"State of driver $submissionId is now $state.") + case Some(state) => + logInfo(log"State of driver ${MDC(SUBMISSION_ID, submissionId)} is now " + + log"${MDC(DRIVER_STATE, state)}.") case _ => logError(log"State of driver ${MDC(SUBMISSION_ID, submissionId)} was not found!") } // Log worker node, if present (workerId, workerHostPort) match { - case (Some(id), Some(hp)) => logInfo(s"Driver is running on worker $id at $hp.") + case (Some(id), Some(hp)) => + logInfo( + log"Driver is running on worker ${MDC(WORKER_ID, id)} at ${MDC(HOST_PORT, hp)}.") case _ => } // Log exception stack trace, if present @@ -490,7 +499,8 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { /** Log the response sent by the server in the REST application submission protocol. */ private def handleRestResponse(response: SubmitRestProtocolResponse): Unit = { - logInfo(s"Server responded with ${response.messageType}:\n${response.toJson}") + logInfo(log"Server responded with ${MDC(CLASS_NAME, response.messageType)}:\n" + + log"${MDC(RESULT, response.toJson)}") } /** Log an appropriate error if the response sent by the server is not of the expected type. */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index d1190ca46c2a8..a3e7276fc83e1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -24,7 +24,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.{SecurityManager, SSLOptions} import org.apache.spark.deploy.Command -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.launcher.WorkerCommandBuilder import org.apache.spark.util.Utils @@ -120,7 +120,8 @@ object CommandUtils extends Logging { Utils.copyStream(in, out, true) } catch { case e: IOException => - logInfo("Redirection to " + file + " closed: " + e.getMessage) + logInfo(log"Redirection to ${MDC(LogKeys.FILE_NAME, file)} closed: " + + log"${MDC(LogKeys.ERROR, e.getMessage)}") } } }.start() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index bd98f19cdb605..8d0fb7a54f72a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -204,7 +204,7 @@ private[deploy] class ExecutorRunner( worker.send(ExecutorStateChanged(appId, execId, state, Some(message), Some(exitCode))) } catch { case interrupted: InterruptedException => - logInfo("Runner thread for executor " + fullId + " interrupted") + logInfo(log"Runner thread for executor ${MDC(EXECUTOR_ID, fullId)} interrupted") state = ExecutorState.KILLED killProcess(s"Runner thread for executor $fullId interrupted") case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index f030475131d24..7ff7974ab59f6 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -38,7 +38,6 @@ import org.apache.spark.deploy.StandaloneResourceUtils._ import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.internal.{config, Logging, MDC} -import org.apache.spark.internal.LogKeys import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.internal.config.UI._ @@ -74,8 +73,8 @@ private[deploy] class Worker( // If worker decommissioning is enabled register a handler on the configured signal to shutdown. if (conf.get(config.DECOMMISSION_ENABLED)) { val signal = conf.get(config.Worker.WORKER_DECOMMISSION_SIGNAL) - logInfo(s"Registering SIG$signal handler to trigger decommissioning.") - SignalUtils.register(signal, log"Failed to register SIG${MDC(LogKeys.SIGNAL, signal)} " + + logInfo(log"Registering SIG${MDC(SIGNAL, signal)} handler to trigger decommissioning.") + SignalUtils.register(signal, log"Failed to register SIG${MDC(SIGNAL, signal)} " + log"handler - disabling worker decommission feature.") { self.send(WorkerDecommissionSigReceived) true @@ -106,8 +105,12 @@ private[deploy] class Worker( private val INITIAL_REGISTRATION_RETRIES = conf.get(WORKER_INITIAL_REGISTRATION_RETRIES) private val TOTAL_REGISTRATION_RETRIES = conf.get(WORKER_MAX_REGISTRATION_RETRIES) if (INITIAL_REGISTRATION_RETRIES > TOTAL_REGISTRATION_RETRIES) { - logInfo(s"${WORKER_INITIAL_REGISTRATION_RETRIES.key} ($INITIAL_REGISTRATION_RETRIES) is " + - s"capped by ${WORKER_MAX_REGISTRATION_RETRIES.key} ($TOTAL_REGISTRATION_RETRIES)") + logInfo( + log"${MDC(CONFIG, WORKER_INITIAL_REGISTRATION_RETRIES.key)} " + + log"(${MDC(VALUE, INITIAL_REGISTRATION_RETRIES)}) is capped by " + + log"${MDC(CONFIG2, WORKER_MAX_REGISTRATION_RETRIES.key)} " + + log"(${MDC(MAX_ATTEMPTS, TOTAL_REGISTRATION_RETRIES)})" + ) } private val FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND = 0.500 private val REGISTRATION_RETRY_FUZZ_MULTIPLIER = { @@ -236,10 +239,11 @@ private[deploy] class Worker( override def onStart(): Unit = { assert(!registered) - logInfo("Starting Spark worker %s:%d with %d cores, %s RAM".format( - host, port, cores, Utils.megabytesToString(memory))) - logInfo(s"Running Spark version ${org.apache.spark.SPARK_VERSION}") - logInfo("Spark home: " + sparkHome) + logInfo(log"Starting Spark worker ${MDC(HOST, host)}:${MDC(PORT, port)} " + + log"with ${MDC(NUM_CORES, cores)} cores, " + + log"${MDC(MEMORY_SIZE, Utils.megabytesToString(memory))} RAM") + logInfo(log"Running Spark version ${MDC(SPARK_VERSION, org.apache.spark.SPARK_VERSION)}") + logInfo(log"Spark home: ${MDC(PATH, sparkHome)}") createWorkDir() startExternalShuffleService() setupWorkerResources() @@ -300,8 +304,9 @@ private[deploy] class Worker( master = Some(masterRef) connected = true if (reverseProxy) { - logInfo("WorkerWebUI is available at %s/proxy/%s".format( - activeMasterWebUiUrl.stripSuffix("/"), workerId)) + logInfo( + log"WorkerWebUI is available at ${MDC(WEB_URL, activeMasterWebUiUrl.stripSuffix("/"))}" + + log"/proxy/${MDC(WORKER_ID, workerId)}") // if reverseProxyUrl is not set, then we continue to generate relative URLs // starting with "/" throughout the UI and do not use activeMasterWebUiUrl val proxyUrl = conf.get(UI_REVERSE_PROXY_URL.key, "").stripSuffix("/") @@ -318,7 +323,7 @@ private[deploy] class Worker( registerMasterThreadPool.submit(new Runnable { override def run(): Unit = { try { - logInfo("Connecting to master " + masterAddress + "...") + logInfo(log"Connecting to master ${MDC(MASTER_URL, masterAddress)}...") val masterEndpoint = rpcEnv.setupEndpointRef(masterAddress, Master.ENDPOINT_NAME) sendRegisterMessageToMaster(masterEndpoint) } catch { @@ -342,7 +347,8 @@ private[deploy] class Worker( if (registered) { cancelLastRegistrationRetry() } else if (connectionAttemptCount <= TOTAL_REGISTRATION_RETRIES) { - logInfo(s"Retrying connection to master (attempt # $connectionAttemptCount)") + logInfo(log"Retrying connection to master (attempt # " + + log"${MDC(NUM_ATTEMPT, connectionAttemptCount)})") /** * Re-register with the active master this worker has been communicating with. If there * is none, then it means this worker is still bootstrapping and hasn't established a @@ -376,7 +382,7 @@ private[deploy] class Worker( registerMasterFutures = Array(registerMasterThreadPool.submit(new Runnable { override def run(): Unit = { try { - logInfo("Connecting to master " + masterAddress + "...") + logInfo(log"Connecting to master ${MDC(MASTER_URL, masterAddress)}...") val masterEndpoint = rpcEnv.setupEndpointRef(masterAddress, Master.ENDPOINT_NAME) sendRegisterMessageToMaster(masterEndpoint) } catch { @@ -483,7 +489,7 @@ private[deploy] class Worker( log"${MDC(MASTER_URL, preferredMasterAddress)}") } - logInfo(s"Successfully registered with master $preferredMasterAddress") + logInfo(log"Successfully registered with master ${MDC(MASTER_URL, preferredMasterAddress)}") registered = true changeMaster(masterRef, masterWebUiUrl, masterAddress) forwardMessageScheduler.scheduleAtFixedRate( @@ -491,7 +497,8 @@ private[deploy] class Worker( 0, HEARTBEAT_MILLIS, TimeUnit.MILLISECONDS) if (CLEANUP_ENABLED) { logInfo( - s"Worker cleanup enabled; old application directories will be deleted in: $workDir") + log"Worker cleanup enabled; old application directories will be deleted in: " + + log"${MDC(PATH, workDir)}") forwardMessageScheduler.scheduleAtFixedRate( () => Utils.tryLogNonFatalError { self.send(WorkDirCleanup) }, CLEANUP_INTERVAL_MILLIS, CLEANUP_INTERVAL_MILLIS, TimeUnit.MILLISECONDS) @@ -539,7 +546,7 @@ private[deploy] class Worker( dir.isDirectory && !isAppStillRunning && !Utils.doesDirectoryContainAnyNewFiles(dir, APP_DATA_RETENTION_SECONDS) }.foreach { dir => - logInfo(s"Removing directory: ${dir.getPath}") + logInfo(log"Removing directory: ${MDC(PATH, dir.getPath)}") Utils.deleteRecursively(dir) // Remove some registeredExecutors information of DB in external shuffle service when @@ -562,7 +569,8 @@ private[deploy] class Worker( } case MasterChanged(masterRef, masterWebUiUrl) => - logInfo("Master has changed, new master is at " + masterRef.address.toSparkURL) + logInfo(log"Master has changed, new master is at " + + log"${MDC(MASTER_URL, masterRef.address.toSparkURL)}") changeMaster(masterRef, masterWebUiUrl, masterRef.address) val executorResponses = executors.values.map { e => @@ -575,7 +583,8 @@ private[deploy] class Worker( workerId, executorResponses.toList, driverResponses.toSeq)) case ReconnectWorker(masterUrl) => - logInfo(s"Master with url $masterUrl requested this worker to reconnect.") + logInfo( + log"Master with url ${MDC(MASTER_URL, masterUrl)} requested this worker to reconnect.") registerWithMaster() case LaunchExecutor(masterUrl, appId, execId, rpId, appDesc, cores_, memory_, resources_) => @@ -586,7 +595,8 @@ private[deploy] class Worker( logWarning("Asked to launch an executor while decommissioned. Not launching executor.") } else { try { - logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) + logInfo(log"Asked to launch executor ${MDC(APP_ID, appId)}/${MDC(EXECUTOR_ID, execId)}" + + log" for ${MDC(APP_DESC, appDesc.name)}") // Create the executor's working directory val executorDir = new File(workDir, appId + "/" + execId) @@ -645,8 +655,8 @@ private[deploy] class Worker( } catch { case e: Exception => logError( - log"Failed to launch executor ${MDC(APP_ID, appId)}/${MDC(EXECUTOR_ID, execId)} " + - log"for ${MDC(APP_DESC, appDesc.name)}.", e) + log"Failed to launch executor ${MDC(APP_ID, appId)}/" + + log"${MDC(EXECUTOR_ID, execId)} for ${MDC(APP_DESC, appDesc.name)}.", e) if (executors.contains(appId + "/" + execId)) { executors(appId + "/" + execId).kill() executors -= appId + "/" + execId @@ -667,15 +677,15 @@ private[deploy] class Worker( val fullId = appId + "/" + execId executors.get(fullId) match { case Some(executor) => - logInfo("Asked to kill executor " + fullId) + logInfo(log"Asked to kill executor ${MDC(EXECUTOR_ID, fullId)}") executor.kill() case None => - logInfo("Asked to kill unknown executor " + fullId) + logInfo(log"Asked to kill unknown executor ${MDC(EXECUTOR_ID, fullId)}") } } case LaunchDriver(driverId, driverDesc, resources_) => - logInfo(s"Asked to launch driver $driverId") + logInfo(log"Asked to launch driver ${MDC(DRIVER_ID, driverId)}") val driver = new DriverRunner( conf, driverId, @@ -695,7 +705,7 @@ private[deploy] class Worker( addResourcesUsed(resources_) case KillDriver(driverId) => - logInfo(s"Asked to kill driver $driverId") + logInfo(log"Asked to kill driver ${MDC(DRIVER_ID, driverId)}") drivers.get(driverId) match { case Some(runner) => runner.kill() @@ -735,7 +745,7 @@ private[deploy] class Worker( override def onDisconnected(remoteAddress: RpcAddress): Unit = { if (master.exists(_.address == remoteAddress) || masterAddressToConnect.contains(remoteAddress)) { - logInfo(s"$remoteAddress Disassociated !") + logInfo(log"${MDC(REMOTE_ADDRESS, remoteAddress)} Disassociated !") masterDisconnected() } } @@ -753,7 +763,7 @@ private[deploy] class Worker( try { appDirectories.remove(id).foreach { dirList => concurrent.Future { - logInfo(s"Cleaning up local directories for application $id") + logInfo(log"Cleaning up local directories for application ${MDC(APP_ID, id)}") dirList.foreach { dir => Utils.deleteRecursively(new File(dir)) } @@ -874,7 +884,7 @@ private[deploy] class Worker( private[deploy] def decommissionSelf(): Unit = { if (conf.get(config.DECOMMISSION_ENABLED) && !decommissioned) { decommissioned = true - logInfo(s"Decommission worker $workerId.") + logInfo(log"Decommission worker ${MDC(WORKER_ID, workerId)}.") } else if (decommissioned) { logWarning(log"Worker ${MDC(WORKER_ID, workerId)} already started decommissioning.") } else { @@ -898,10 +908,10 @@ private[deploy] class Worker( logWarning(log"Driver ${MDC(DRIVER_ID, driverId)} " + log"exited successfully while master is disconnected.") case _ => - logInfo(s"Driver $driverId exited successfully") + logInfo(log"Driver ${MDC(DRIVER_ID, driverId)} exited successfully") } case DriverState.KILLED => - logInfo(s"Driver $driverId was killed by user") + logInfo(log"Driver ${MDC(DRIVER_ID, driverId)} was killed by user") case _ => logDebug(s"Driver $driverId changed state to $state") } @@ -921,13 +931,22 @@ private[deploy] class Worker( if (ExecutorState.isFinished(state)) { val appId = executorStateChanged.appId val fullId = appId + "/" + executorStateChanged.execId - val message = executorStateChanged.message - val exitStatus = executorStateChanged.exitStatus + val message = executorStateChanged.message match { + case Some(msg) => + log" message ${MDC(MESSAGE, msg)}" + case None => + log"" + } + val exitStatus = executorStateChanged.exitStatus match { + case Some(status) => + log" exitStatus ${MDC(EXIT_CODE, status)}" + case None => + log"" + } executors.get(fullId) match { case Some(executor) => - logInfo("Executor " + fullId + " finished with state " + state + - message.map(" message " + _).getOrElse("") + - exitStatus.map(" exitStatus " + _).getOrElse("")) + logInfo(log"Executor ${MDC(EXECUTOR_ID, fullId)} finished with state " + + log"${MDC(EXECUTOR_STATE, state)}" + message + exitStatus) executors -= fullId finishedExecutors(fullId) = executor trimFinishedExecutorsIfNecessary() @@ -939,9 +958,8 @@ private[deploy] class Worker( shuffleService.executorRemoved(executorStateChanged.execId.toString, appId) } case None => - logInfo("Unknown Executor " + fullId + " finished with state " + state + - message.map(" message " + _).getOrElse("") + - exitStatus.map(" exitStatus " + _).getOrElse("")) + logInfo(log"Unknown Executor ${MDC(EXECUTOR_ID, fullId)} finished with state " + + log"${MDC(EXECUTOR_STATE, state)}" + message + exitStatus) } maybeCleanupApplication(appId) } diff --git a/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala index 0158dd6ba7757..7098961d1649a 100644 --- a/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala @@ -156,7 +156,7 @@ private[memory] class ExecutionMemoryPool( val memoryToFree = if (curMem < numBytes) { logWarning( log"Internal error: release called on ${MDC(NUM_BYTES, numBytes)} " + - log"bytes but task only has ${MDC(CURRENT_MEMORY_BYTES, curMem)} bytes " + + log"bytes but task only has ${MDC(CURRENT_MEMORY_SIZE, curMem)} bytes " + log"of memory from the ${MDC(MEMORY_POOL_NAME, poolName)} pool") curMem } else { diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index ff899a2e56dc0..cbfce378879ec 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -270,7 +270,7 @@ class HadoopRDD[K, V]( val iter = new NextIterator[(K, V)] { private val split = theSplit.asInstanceOf[HadoopPartition] - logInfo("Input split: " + split.inputSplit) + logInfo(log"Input split: ${MDC(INPUT_SPLIT, split.inputSplit)}") private val jobConf = getJobConf() private val inputMetrics = context.taskMetrics().inputMetrics diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index a87d02287302d..3a1ce4bd1dfde 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -197,7 +197,7 @@ class NewHadoopRDD[K, V]( override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = { val iter = new Iterator[(K, V)] { private val split = theSplit.asInstanceOf[NewHadoopPartition] - logInfo("Input split: " + split.serializableHadoopSplit) + logInfo(log"Input split: ${MDC(INPUT_SPLIT, split.serializableHadoopSplit)}") private val conf = getConf private val inputMetrics = context.taskMetrics().inputMetrics 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 1f44f7e782c48..ac93abf3fe7a0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -211,7 +211,7 @@ abstract class RDD[T: ClassTag]( * @return This RDD. */ def unpersist(blocking: Boolean = false): this.type = { - logInfo(s"Removing RDD $id from persistence list") + logInfo(log"Removing RDD ${MDC(RDD_ID, id)} from persistence list") sc.unpersistRDD(id, blocking) storageLevel = StorageLevel.NONE this diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala index 2f6ff0acdf024..118660ef69476 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.SequenceFileOutputFormat -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{Logging, LogKeys, MDC} /** * Extra functions available on RDDs of (key, value) pairs to create a Hadoop SequenceFile, @@ -58,8 +58,9 @@ class SequenceFileRDDFunctions[K: IsWritable: ClassTag, V: IsWritable: ClassTag] val convertKey = self.keyClass != _keyWritableClass val convertValue = self.valueClass != _valueWritableClass - logInfo("Saving as sequence file of type " + - s"(${_keyWritableClass.getSimpleName},${_valueWritableClass.getSimpleName})" ) + logInfo(log"Saving as sequence file of type " + + log"(${MDC(LogKeys.KEY, _keyWritableClass.getSimpleName)}," + + log"${MDC(LogKeys.VALUE, _valueWritableClass.getSimpleName)})") val format = classOf[SequenceFileOutputFormat[Writable, Writable]] val jobConf = new JobConf(self.context.hadoopConfiguration) if (!convertKey && !convertValue) { 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 cc9ae5eb1ebe5..7c096dd110e50 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -535,8 +535,9 @@ private[spark] class DAGScheduler( if (!mapOutputTracker.containsShuffle(shuffleDep.shuffleId)) { // Kind of ugly: need to register RDDs with the cache and map output tracker here // since we can't do it in the RDD constructor because # of partitions is unknown - logInfo(s"Registering RDD ${rdd.id} (${rdd.getCreationSite}) as input to " + - s"shuffle ${shuffleDep.shuffleId}") + logInfo(log"Registering RDD ${MDC(RDD_ID, rdd.id)} " + + log"(${MDC(CREATION_SITE, rdd.getCreationSite)}) as input to " + + log"shuffle ${MDC(SHUFFLE_ID, shuffleDep.shuffleId)}") mapOutputTracker.registerShuffle(shuffleDep.shuffleId, rdd.partitions.length, shuffleDep.partitioner.numPartitions) } @@ -1097,7 +1098,7 @@ private[spark] class DAGScheduler( * Cancel a job that is running or waiting in the queue. */ def cancelJob(jobId: Int, reason: Option[String]): Unit = { - logInfo("Asked to cancel job " + jobId) + logInfo(log"Asked to cancel job ${MDC(JOB_ID, jobId)}") eventProcessLoop.post(JobCancelled(jobId, reason)) } @@ -1106,7 +1107,8 @@ private[spark] class DAGScheduler( * @param cancelFutureJobs if true, future submitted jobs in this job group will be cancelled */ def cancelJobGroup(groupId: String, cancelFutureJobs: Boolean = false): Unit = { - logInfo(s"Asked to cancel job group $groupId with cancelFutureJobs=$cancelFutureJobs") + logInfo(log"Asked to cancel job group ${MDC(GROUP_ID, groupId)} with " + + log"cancelFutureJobs=${MDC(CANCEL_FUTURE_JOBS, cancelFutureJobs)}") eventProcessLoop.post(JobGroupCancelled(groupId, cancelFutureJobs)) } @@ -1115,7 +1117,7 @@ private[spark] class DAGScheduler( */ def cancelJobsWithTag(tag: String): Unit = { SparkContext.throwIfInvalidTag(tag) - logInfo(s"Asked to cancel jobs with tag $tag") + logInfo(log"Asked to cancel jobs with tag ${MDC(TAG, tag)}") eventProcessLoop.post(JobTagCancelled(tag)) } @@ -1209,7 +1211,7 @@ private[spark] class DAGScheduler( // If cancelFutureJobs is true, store the cancelled job group id into internal states. // When a job belonging to this job group is submitted, skip running it. if (cancelFutureJobs) { - logInfo(s"Add job group $groupId into cancelled job groups") + logInfo(log"Add job group ${MDC(GROUP_ID, groupId)} into cancelled job groups") cancelledJobGroups.add(groupId) } @@ -1314,7 +1316,7 @@ private[spark] class DAGScheduler( if (jobGroupIdOpt.exists(cancelledJobGroups.contains(_))) { listener.jobFailed( SparkCoreErrors.sparkJobCancelledAsPartOfJobGroupError(jobId, jobGroupIdOpt.get)) - logInfo(s"Skip running a job that belongs to the cancelled job group ${jobGroupIdOpt.get}.") + logInfo(log"Skip running a job that belongs to the cancelled job group ${MDC(GROUP_ID, jobGroupIdOpt.get)}") return } @@ -1362,11 +1364,13 @@ private[spark] class DAGScheduler( val job = new ActiveJob(jobId, finalStage, callSite, listener, artifacts, properties) clearCacheLocs() - logInfo("Got job %s (%s) with %d output partitions".format( - job.jobId, callSite.shortForm, partitions.length)) - logInfo("Final stage: " + finalStage + " (" + finalStage.name + ")") - logInfo("Parents of final stage: " + finalStage.parents) - logInfo("Missing parents: " + getMissingParentStages(finalStage)) + logInfo( + log"Got job ${MDC(JOB_ID, job.jobId)} (${MDC(CALL_SITE_SHORT_FORM, callSite.shortForm)}) " + + log"with ${MDC(NUM_PARTITIONS, partitions.length)} output partitions") + logInfo(log"Final stage: ${MDC(STAGE_ID, finalStage)} " + + log"(${MDC(STAGE_NAME, finalStage.name)})") + logInfo(log"Parents of final stage: ${MDC(STAGE_ID, finalStage.parents)}") + logInfo(log"Missing parents: ${MDC(MISSING_PARENT_STAGES, getMissingParentStages(finalStage))}") val jobSubmissionTime = clock.getTimeMillis() jobIdToActiveJob(jobId) = job @@ -1403,11 +1407,13 @@ private[spark] class DAGScheduler( val job = new ActiveJob(jobId, finalStage, callSite, listener, artifacts, properties) clearCacheLocs() - logInfo("Got map stage job %s (%s) with %d output partitions".format( - jobId, callSite.shortForm, dependency.rdd.partitions.length)) - logInfo("Final stage: " + finalStage + " (" + finalStage.name + ")") - logInfo("Parents of final stage: " + finalStage.parents) - logInfo("Missing parents: " + getMissingParentStages(finalStage)) + logInfo(log"Got map stage job ${MDC(JOB_ID, jobId)} " + + log"(${MDC(CALL_SITE_SHORT_FORM, callSite.shortForm)}) with " + + log"${MDC(NUM_PARTITIONS, dependency.rdd.partitions.length)} output partitions") + logInfo(log"Final stage: ${MDC(STAGE_ID, finalStage)} " + + log"(${MDC(STAGE_NAME, finalStage.name)})") + logInfo(log"Parents of final stage: ${MDC(PARENT_STAGES, finalStage.parents.toString)}") + logInfo(log"Missing parents: ${MDC(MISSING_PARENT_STAGES, getMissingParentStages(finalStage))}") val jobSubmissionTime = clock.getTimeMillis() jobIdToActiveJob(jobId) = job @@ -1444,7 +1450,8 @@ private[spark] class DAGScheduler( val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) if (missing.isEmpty) { - logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents") + logInfo(log"Submitting ${MDC(STAGE_ID, stage)} (${MDC(RDD_ID, stage.rdd)}), " + + log"which has no missing parents") submitMissingTasks(stage, jobId.get) } else { for (parent <- missing) { @@ -1495,13 +1502,16 @@ private[spark] class DAGScheduler( val shuffleId = stage.shuffleDep.shuffleId val shuffleMergeId = stage.shuffleDep.shuffleMergeId if (stage.shuffleDep.shuffleMergeEnabled) { - logInfo(s"Shuffle merge enabled before starting the stage for $stage with shuffle" + - s" $shuffleId and shuffle merge $shuffleMergeId with" + - s" ${stage.shuffleDep.getMergerLocs.size} merger locations") + logInfo(log"Shuffle merge enabled before starting the stage for ${MDC(STAGE_ID, stage)}" + + log" with shuffle ${MDC(SHUFFLE_ID, shuffleId)} and shuffle merge" + + log" ${MDC(SHUFFLE_MERGE_ID, shuffleMergeId)} with" + + log" ${MDC(NUM_MERGER_LOCATIONS, stage.shuffleDep.getMergerLocs.size.toString)} merger locations") } else { - logInfo(s"Shuffle merge disabled for $stage with shuffle $shuffleId" + - s" and shuffle merge $shuffleMergeId, but can get enabled later adaptively" + - s" once enough mergers are available") + logInfo(log"Shuffle merge disabled for ${MDC(STAGE_ID, stage)} with " + + log"shuffle ${MDC(SHUFFLE_ID, shuffleId)} and " + + log"shuffle merge ${MDC(SHUFFLE_MERGE_ID, shuffleMergeId)}, " + + log"but can get enabled later adaptively once enough " + + log"mergers are available") } } @@ -1558,8 +1568,8 @@ private[spark] class DAGScheduler( // merger locations but the corresponding shuffle map stage did not complete // successfully, we would still enable push for its retry. s.shuffleDep.setShuffleMergeAllowed(false) - logInfo(s"Push-based shuffle disabled for $stage (${stage.name}) since it" + - " is already shuffle merge finalized") + logInfo(log"Push-based shuffle disabled for ${MDC(STAGE_ID, stage)} " + + log"(${MDC(STAGE_NAME, stage.name)}) since it is already shuffle merge finalized") } } case s: ResultStage => @@ -1681,8 +1691,9 @@ private[spark] class DAGScheduler( } if (tasks.nonEmpty) { - logInfo(s"Submitting ${tasks.size} missing tasks from $stage (${stage.rdd}) (first 15 " + - s"tasks are for partitions ${tasks.take(15).map(_.partitionId)})") + logInfo(log"Submitting ${MDC(NUM_TASKS, tasks.size)} missing tasks from " + + log"${MDC(STAGE_ID, stage)} (${MDC(RDD_ID, stage.rdd)}) (first 15 tasks are " + + log"for partitions ${MDC(PARTITION_IDS, tasks.take(15).map(_.partitionId))})") val shuffleId = stage match { case s: ShuffleMapStage => Some(s.shuffleDep.shuffleId) case _: ResultStage => None @@ -1751,9 +1762,10 @@ private[spark] class DAGScheduler( case Some(accum) => accum.getClass.getName case None => "Unknown class" } - logError( - log"Failed to update accumulator ${MDC(ACCUMULATOR_ID, id)} (${MDC(CLASS_NAME, accumClassName)}) " + - log"for task ${MDC(PARTITION_ID, task.partitionId)}", e) + logError( + log"Failed to update accumulator ${MDC(ACCUMULATOR_ID, id)} " + + log"(${MDC(CLASS_NAME, accumClassName)}) for task " + + log"${MDC(PARTITION_ID, task.partitionId)}", e) } } } @@ -1926,8 +1938,8 @@ private[spark] class DAGScheduler( try { // killAllTaskAttempts will fail if a SchedulerBackend does not implement // killTask. - logInfo(s"Job ${job.jobId} is finished. Cancelling potential speculative " + - "or zombie tasks for this job") + logInfo(log"Job ${MDC(JOB_ID, job.jobId)} is finished. Cancelling " + + log"potential speculative or zombie tasks for this job") // ResultStage is only used by this job. It's safe to kill speculative or // zombie tasks in this stage. taskScheduler.killAllTaskAttempts( @@ -1954,7 +1966,7 @@ private[spark] class DAGScheduler( } } case None => - logInfo("Ignoring result from " + rt + " because its job has finished") + logInfo(log"Ignoring result from ${MDC(RESULT, rt)} because its job has finished") } case smt: ShuffleMapTask => @@ -1969,7 +1981,8 @@ private[spark] class DAGScheduler( logDebug("ShuffleMapTask finished on " + execId) if (executorFailureEpoch.contains(execId) && smt.epoch <= executorFailureEpoch(execId)) { - logInfo(s"Ignoring possibly bogus $smt completion from executor $execId") + logInfo(log"Ignoring possibly bogus ${MDC(STAGE_ID, smt)} completion from " + + log"executor ${MDC(EXECUTOR_ID, 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 @@ -1978,7 +1991,7 @@ private[spark] class DAGScheduler( shuffleStage.shuffleDep.shuffleId, smt.partitionId, status) } } else { - logInfo(s"Ignoring $smt completion from an older attempt of indeterminate stage") + logInfo(log"Ignoring ${MDC(TASK_NAME, smt)} completion from an older attempt of indeterminate stage") } if (runningStages.contains(shuffleStage) && shuffleStage.pendingPartitions.isEmpty) { @@ -1996,17 +2009,22 @@ private[spark] class DAGScheduler( val mapStage = shuffleIdToMapStage(shuffleId) if (failedStage.latestInfo.attemptNumber() != task.stageAttemptId) { - logInfo(s"Ignoring fetch failure from $task as it's from $failedStage attempt" + - s" ${task.stageAttemptId} and there is a more recent attempt for that stage " + - s"(attempt ${failedStage.latestInfo.attemptNumber()}) running") + logInfo(log"Ignoring fetch failure from " + + log"${MDC(TASK_ID, task)} as it's from " + + log"${MDC(STAGE_ID, failedStage)} attempt " + + log"${MDC(STAGE_ATTEMPT, task.stageAttemptId)} and there is a more recent attempt for " + + log"that stage (attempt " + + log"${MDC(NUM_ATTEMPT, failedStage.latestInfo.attemptNumber())}) running") } else { val ignoreStageFailure = ignoreDecommissionFetchFailure && isExecutorDecommissioningOrDecommissioned(taskScheduler, bmAddress) if (ignoreStageFailure) { - logInfo(s"Ignoring fetch failure from $task of $failedStage attempt " + - s"${task.stageAttemptId} when count ${config.STAGE_MAX_CONSECUTIVE_ATTEMPTS.key} " + - s"as executor ${bmAddress.executorId} is decommissioned and " + - s" ${config.STAGE_IGNORE_DECOMMISSION_FETCH_FAILURE.key}=true") + logInfo(log"Ignoring fetch failure from ${MDC(TASK_ID, task)} of " + + log"${MDC(STAGE, failedStage)} attempt " + + log"${MDC(STAGE_ATTEMPT, task.stageAttemptId)} when count " + + log"${MDC(MAX_ATTEMPTS, config.STAGE_MAX_CONSECUTIVE_ATTEMPTS.key)} " + + log"as executor ${MDC(EXECUTOR_ID, bmAddress.executorId)} is decommissioned and " + + log"${MDC(CONFIG, config.STAGE_IGNORE_DECOMMISSION_FETCH_FAILURE.key)}=true") } else { failedStage.failedAttemptIds.add(task.stageAttemptId) } @@ -2019,8 +2037,10 @@ private[spark] class DAGScheduler( // multiple tasks running concurrently on different executors). In that case, it is // possible the fetch failure has already been handled by the scheduler. if (runningStages.contains(failedStage)) { - logInfo(s"Marking $failedStage (${failedStage.name}) as failed " + - s"due to a fetch failure from $mapStage (${mapStage.name})") + logInfo(log"Marking ${MDC(FAILED_STAGE, failedStage)} " + + log"(${MDC(FAILED_STAGE_NAME, failedStage.name)}) as failed " + + log"due to a fetch failure from ${MDC(STAGE, mapStage)} " + + log"(${MDC(STAGE_NAME, mapStage.name)})") markStageAsFinished(failedStage, errorMessage = Some(failureMessage), willRetry = !shouldAbortStage) } else { @@ -2148,9 +2168,9 @@ private[spark] class DAGScheduler( case _ => } - logInfo(s"The shuffle map stage $mapStage with indeterminate output was failed, " + - s"we will roll back and rerun below stages which include itself and all its " + - s"indeterminate child stages: $rollingBackStages") + logInfo(log"The shuffle map stage ${MDC(SHUFFLE_ID, mapStage)} with indeterminate output was failed, " + + log"we will roll back and rerun below stages which include itself and all its " + + log"indeterminate child stages: ${MDC(STAGES, rollingBackStages)}") } // We expect one executor failure to trigger many FetchFailures in rapid succession, @@ -2162,9 +2182,9 @@ private[spark] class DAGScheduler( // producing a resubmit for each failed stage makes debugging and logging a little // simpler while not producing an overwhelming number of scheduler events. logInfo( - s"Resubmitting $mapStage (${mapStage.name}) and " + - s"$failedStage (${failedStage.name}) due to fetch failure" - ) + log"Resubmitting ${MDC(STAGE, mapStage)} " + + log"(${MDC(STAGE_NAME, mapStage.name)}) and ${MDC(FAILED_STAGE, failedStage)} " + + log"(${MDC(FAILED_STAGE_NAME, failedStage.name)}) due to fetch failure") messageScheduler.schedule( new Runnable { override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages) @@ -2223,12 +2243,13 @@ private[spark] class DAGScheduler( // Always fail the current stage and retry all the tasks when a barrier task fail. val failedStage = stageIdToStage(task.stageId) if (failedStage.latestInfo.attemptNumber() != task.stageAttemptId) { - logInfo(s"Ignoring task failure from $task as it's from $failedStage attempt" + - s" ${task.stageAttemptId} and there is a more recent attempt for that stage " + - s"(attempt ${failedStage.latestInfo.attemptNumber()}) running") + logInfo(log"Ignoring task failure from ${MDC(TASK_ID, task)} as it's from " + + log"${MDC(FAILED_STAGE, failedStage)} attempt ${MDC(STAGE_ATTEMPT, task.stageAttemptId)} " + + log"and there is a more recent attempt for that stage (attempt " + + log"${MDC(NUM_ATTEMPT, failedStage.latestInfo.attemptNumber())}) running") } else { - logInfo(s"Marking $failedStage (${failedStage.name}) as failed due to a barrier task " + - "failed.") + logInfo(log"Marking ${MDC(STAGE_ID, failedStage.id)} (${MDC(STAGE_NAME, failedStage.name)}) " + + log"as failed due to a barrier task failed.") val message = s"Stage failed because barrier task $task finished unsuccessfully.\n" + failure.toErrorString try { @@ -2283,8 +2304,8 @@ private[spark] class DAGScheduler( val noResubmitEnqueued = !failedStages.contains(failedStage) failedStages += failedStage if (noResubmitEnqueued) { - logInfo(s"Resubmitting $failedStage (${failedStage.name}) due to barrier stage " + - "failure.") + logInfo(log"Resubmitting ${MDC(FAILED_STAGE, failedStage)} " + + log"(${MDC(FAILED_STAGE_NAME, failedStage.name)}) due to barrier stage failure.") messageScheduler.schedule(new Runnable { override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages) }, DAGScheduler.RESUBMIT_TIMEOUT, TimeUnit.MILLISECONDS) @@ -2361,8 +2382,8 @@ private[spark] class DAGScheduler( // delay should be 0 and registerMergeResults should be true. assert(delay == 0 && registerMergeResults) if (task.getDelay(TimeUnit.NANOSECONDS) > 0 && task.cancel(false)) { - logInfo(s"$stage (${stage.name}) scheduled for finalizing shuffle merge immediately " + - s"after cancelling previously scheduled task.") + logInfo(log"${MDC(STAGE, stage)} (${MDC(STAGE_NAME, stage.name)}) scheduled " + + log"for finalizing shuffle merge immediately after cancelling previously scheduled task.") shuffleDep.setFinalizeTask( shuffleMergeFinalizeScheduler.schedule( new Runnable { @@ -2373,13 +2394,15 @@ private[spark] class DAGScheduler( ) ) } else { - logInfo(s"$stage (${stage.name}) existing scheduled task for finalizing shuffle merge" + - s"would either be in-progress or finished. No need to schedule shuffle merge" + - s" finalization again.") + logInfo( + log"${MDC(STAGE, stage)} (${MDC(STAGE_NAME, stage.name)}) existing scheduled task " + + log"for finalizing shuffle merge would either be in-progress or finished. " + + log"No need to schedule shuffle merge finalization again.") } case None => // If no previous finalization task is scheduled, schedule the finalization task. - logInfo(s"$stage (${stage.name}) scheduled for finalizing shuffle merge in $delay s") + logInfo(log"${MDC(STAGE, stage)} (${MDC(STAGE_NAME, stage.name)}) scheduled for " + + log"finalizing shuffle merge in ${MDC(DELAY, delay * 1000L)} ms") shuffleDep.setFinalizeTask( shuffleMergeFinalizeScheduler.schedule( new Runnable { @@ -2408,8 +2431,9 @@ private[spark] class DAGScheduler( private[scheduler] def finalizeShuffleMerge( stage: ShuffleMapStage, registerMergeResults: Boolean = true): Unit = { - logInfo(s"$stage (${stage.name}) finalizing the shuffle merge with registering merge " + - s"results set to $registerMergeResults") + logInfo( + log"${MDC(STAGE, stage)} (${MDC(STAGE_NAME, stage.name)}) finalizing the shuffle merge with" + + log" registering merge results set to ${MDC(REGISTER_MERGE_RESULTS, registerMergeResults)}") val shuffleId = stage.shuffleDep.shuffleId val shuffleMergeId = stage.shuffleDep.shuffleMergeId val numMergers = stage.shuffleDep.getMergerLocs.length @@ -2479,8 +2503,9 @@ private[spark] class DAGScheduler( } catch { case _: TimeoutException => timedOut = true - logInfo(s"Timed out on waiting for merge results from all " + - s"$numMergers mergers for shuffle $shuffleId") + logInfo(log"Timed out on waiting for merge results from all " + + log"${MDC(NUM_MERGERS, numMergers)} mergers for " + + log"shuffle ${MDC(SHUFFLE_ID, shuffleId)}") } finally { if (timedOut || !registerMergeResults) { cancelFinalizeShuffleMergeFutures(scheduledFutures, @@ -2511,9 +2536,9 @@ private[spark] class DAGScheduler( private def processShuffleMapStageCompletion(shuffleStage: ShuffleMapStage): Unit = { markStageAsFinished(shuffleStage) logInfo("looking for newly runnable stages") - logInfo("running: " + runningStages) - logInfo("waiting: " + waitingStages) - logInfo("failed: " + failedStages) + logInfo(log"running: ${MDC(STAGES, runningStages)}") + logInfo(log"waiting: ${MDC(STAGES, waitingStages)}") + logInfo(log"failed: ${MDC(STAGES, failedStages)}") // This call to increment the epoch may not be strictly necessary, but it is retained // for now in order to minimize the changes in behavior from an earlier version of the @@ -2529,9 +2554,10 @@ private[spark] class DAGScheduler( if (!shuffleStage.isAvailable) { // Some tasks had failed; let's resubmit this shuffleStage. // TODO: Lower-level scheduler should also deal with this - logInfo("Resubmitting " + shuffleStage + " (" + shuffleStage.name + - ") because some of its tasks had failed: " + - shuffleStage.findMissingPartitions().mkString(", ")) + logInfo(log"Resubmitting ${MDC(STAGE, shuffleStage)} " + + log"(${MDC(STAGE_NAME, shuffleStage.name)}) " + + log"because some of its tasks had failed: " + + log"${MDC(PARTITION_IDS, shuffleStage.findMissingPartitions().mkString(", "))}") submitStage(shuffleStage) } else { markMapStageJobsAsFinished(shuffleStage) @@ -2603,7 +2629,7 @@ private[spark] class DAGScheduler( } private def handleResubmittedFailure(task: Task[_], stage: Stage): Unit = { - logInfo(s"Resubmitted $task, so marking it as still running.") + logInfo(log"Resubmitted ${MDC(TASK_ID, task)}, so marking it as still running.") stage match { case sms: ShuffleMapStage => sms.pendingPartitions += task.partitionId @@ -2679,7 +2705,7 @@ private[spark] class DAGScheduler( if (!isShuffleMerger && (!executorFailureEpoch.contains(execId) || executorFailureEpoch(execId) < currentEpoch)) { executorFailureEpoch(execId) = currentEpoch - logInfo(s"Executor lost: $execId (epoch $currentEpoch)") + logInfo(log"Executor lost: ${MDC(EXECUTOR_ID, execId)} (epoch ${MDC(EPOCH, currentEpoch)})") if (pushBasedShuffleEnabled) { // Remove fetchFailed host in the shuffle push merger list for push based shuffle hostToUnregisterOutputs.foreach( @@ -2703,10 +2729,12 @@ private[spark] class DAGScheduler( if (remove) { hostToUnregisterOutputs match { case Some(host) => - logInfo(s"Shuffle files lost for host: $host (epoch $currentEpoch)") + logInfo(log"Shuffle files lost for host: ${MDC(HOST, host)} (epoch " + + log"${MDC(EPOCH, currentEpoch)}") mapOutputTracker.removeOutputsOnHost(host) case None => - logInfo(s"Shuffle files lost for executor: $execId (epoch $currentEpoch)") + logInfo(log"Shuffle files lost for executor: ${MDC(EXECUTOR_ID, execId)} " + + log"(epoch ${MDC(EPOCH, currentEpoch)})") mapOutputTracker.removeOutputsOnExecutor(execId) } } @@ -2728,7 +2756,8 @@ private[spark] class DAGScheduler( workerId: String, host: String, message: String): Unit = { - logInfo("Shuffle files lost for worker %s on host %s".format(workerId, host)) + logInfo(log"Shuffle files lost for worker ${MDC(WORKER_ID, workerId)} " + + log"on host ${MDC(HOST, host)}") mapOutputTracker.removeOutputsOnHost(host) clearCacheLocs() } @@ -2736,7 +2765,7 @@ private[spark] class DAGScheduler( private[scheduler] def handleExecutorAdded(execId: String, host: String): Unit = { // remove from executorFailureEpoch(execId) ? if (executorFailureEpoch.contains(execId)) { - logInfo("Host added was in lost list earlier: " + host) + logInfo(log"Host added was in lost list earlier: ${MDC(HOST, host)}") executorFailureEpoch -= execId } shuffleFileLostEpoch -= execId @@ -2749,10 +2778,10 @@ private[spark] class DAGScheduler( }.foreach { case (_, stage: ShuffleMapStage) => configureShufflePushMergerLocations(stage) if (stage.shuffleDep.getMergerLocs.nonEmpty) { - logInfo(s"Shuffle merge enabled adaptively for $stage with shuffle" + - s" ${stage.shuffleDep.shuffleId} and shuffle merge" + - s" ${stage.shuffleDep.shuffleMergeId} with ${stage.shuffleDep.getMergerLocs.size}" + - s" merger locations") + logInfo(log"Shuffle merge enabled adaptively for ${MDC(STAGE, stage)} with shuffle" + + log" ${MDC(SHUFFLE_ID, stage.shuffleDep.shuffleId)} and shuffle merge" + + log" ${MDC(SHUFFLE_MERGE_ID, stage.shuffleDep.shuffleMergeId)} with " + + log"${MDC(NUM_MERGER_LOCATIONS, stage.shuffleDep.getMergerLocs.size)} merger locations") } } } @@ -2772,7 +2801,7 @@ private[spark] class DAGScheduler( handleJobCancellation(jobId, Option(reasonStr)) } case None => - logInfo("No active jobs to kill for Stage " + stageId) + logInfo(log"No active jobs to kill for Stage ${MDC(STAGE_ID, stageId)}") } } @@ -2795,11 +2824,12 @@ private[spark] class DAGScheduler( errorMessage: Option[String] = None, willRetry: Boolean = false): Unit = { val serviceTime = stage.latestInfo.submissionTime match { - case Some(t) => "%.03f".format((clock.getTimeMillis() - t) / 1000.0) + case Some(t) => clock.getTimeMillis() - t case _ => "Unknown" } if (errorMessage.isEmpty) { - logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) + logInfo(log"${MDC(STAGE, stage)} (${MDC(STAGE_NAME, stage.name)}) " + + log"finished in ${MDC(TIME_UNITS, serviceTime)} ms") stage.latestInfo.completionTime = Some(clock.getTimeMillis()) // Clear failure count for this stage, now that it's succeeded. @@ -2809,7 +2839,8 @@ private[spark] class DAGScheduler( stage.clearFailures() } else { stage.latestInfo.stageFailed(errorMessage.get) - logInfo(s"$stage (${stage.name}) failed in $serviceTime s due to ${errorMessage.get}") + logInfo(log"${MDC(STAGE, stage)} (${MDC(STAGE_NAME, stage.name)}) failed in " + + log"${MDC(TIME_UNITS, serviceTime)} ms due to ${MDC(ERROR, errorMessage.get)}") } updateStageInfoForPushBasedShuffle(stage) if (!willRetry) { @@ -2855,7 +2886,8 @@ private[spark] class DAGScheduler( failJobAndIndependentStages(job, finalException) } if (dependentJobs.isEmpty) { - logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") + logInfo(log"Ignoring failure of ${MDC(FAILED_STAGE, failedStage)} because all jobs " + + log"depending on it are done") } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/HealthTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/HealthTracker.scala index cecf5d498ac4b..1606072153906 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/HealthTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/HealthTracker.scala @@ -111,8 +111,8 @@ private[scheduler] class HealthTracker ( val execsToInclude = executorIdToExcludedStatus.filter(_._2.expiryTime < now).keys if (execsToInclude.nonEmpty) { // Include any executors that have been excluded longer than the excludeOnFailure timeout. - logInfo(s"Removing executors $execsToInclude from exclude list because the " + - s"the executors have reached the timed out") + logInfo(log"Removing executors ${MDC(EXECUTOR_IDS, execsToInclude)} from " + + log"exclude list because the executors have reached the timed out") execsToInclude.foreach { exec => val status = executorIdToExcludedStatus.remove(exec).get val failedExecsOnNode = nodeToExcludedExecs(status.node) @@ -128,8 +128,8 @@ private[scheduler] class HealthTracker ( val nodesToInclude = nodeIdToExcludedExpiryTime.filter(_._2 < now).keys if (nodesToInclude.nonEmpty) { // Include any nodes that have been excluded longer than the excludeOnFailure timeout. - logInfo(s"Removing nodes $nodesToInclude from exclude list because the " + - s"nodes have reached has timed out") + logInfo(log"Removing nodes ${MDC(NODES, nodesToInclude)} from exclude list because the " + + log"nodes have reached has timed out") nodesToInclude.foreach { node => nodeIdToExcludedExpiryTime.remove(node) // post both to keep backwards compatibility @@ -173,8 +173,8 @@ private[scheduler] class HealthTracker ( force = true) } case None => - logInfo(s"Not attempting to kill excluded executor id $exec " + - s"since allocation client is not defined.") + logInfo(log"Not attempting to kill excluded executor id ${MDC(EXECUTOR_ID, exec)}" + + log" since allocation client is not defined.") } } @@ -196,14 +196,15 @@ private[scheduler] class HealthTracker ( allocationClient match { case Some(a) => if (EXCLUDE_ON_FAILURE_DECOMMISSION_ENABLED) { - logInfo(s"Decommissioning all executors on excluded host $node " + - s"since ${config.EXCLUDE_ON_FAILURE_KILL_ENABLED.key} is set.") + logInfo(log"Decommissioning all executors on excluded host ${MDC(HOST, node)} " + + log"since ${MDC(CONFIG, config.EXCLUDE_ON_FAILURE_KILL_ENABLED.key)} " + + log"is set.") if (!a.decommissionExecutorsOnHost(node)) { logError(log"Decommissioning executors on ${MDC(HOST, node)} failed.") } } else { - logInfo(s"Killing all executors on excluded host $node " + - s"since ${config.EXCLUDE_ON_FAILURE_KILL_ENABLED.key} is set.") + logInfo(log"Killing all executors on excluded host ${MDC(HOST, node)} " + + log"since ${MDC(CONFIG, config.EXCLUDE_ON_FAILURE_KILL_ENABLED.key)} is set.") if (!a.killExecutorsOnHost(node)) { logError(log"Killing executors on node ${MDC(HOST, node)} failed.") } @@ -231,7 +232,8 @@ private[scheduler] class HealthTracker ( if (conf.get(config.SHUFFLE_SERVICE_ENABLED)) { if (!nodeIdToExcludedExpiryTime.contains(host)) { - logInfo(s"excluding node $host due to fetch failure of external shuffle service") + logInfo(log"excluding node ${MDC(HOST, host)} due to fetch failure of " + + log"external shuffle service") nodeIdToExcludedExpiryTime.put(host, expiryTimeForNewExcludes) // post both to keep backwards compatibility @@ -242,7 +244,7 @@ private[scheduler] class HealthTracker ( updateNextExpiryTime() } } else if (!executorIdToExcludedStatus.contains(exec)) { - logInfo(s"Excluding executor $exec due to fetch failure") + logInfo(log"Excluding executor ${MDC(EXECUTOR_ID, exec)} due to fetch failure") executorIdToExcludedStatus.put(exec, ExcludedExecutor(host, expiryTimeForNewExcludes)) // We hardcoded number of failure tasks to 1 for fetch failure, because there's no @@ -280,8 +282,8 @@ private[scheduler] class HealthTracker ( // some of the logic around expiry times a little more confusing. But it also wouldn't be a // problem to re-exclude, with a later expiry time. if (newTotal >= MAX_FAILURES_PER_EXEC && !executorIdToExcludedStatus.contains(exec)) { - logInfo(s"Excluding executor id: $exec because it has $newTotal" + - s" task failures in successful task sets") + logInfo(log"Excluding executor id: ${MDC(EXECUTOR_ID, exec)} because it has " + + log"${MDC(TOTAL, newTotal)} task failures in successful task sets") val node = failuresInTaskSet.node executorIdToExcludedStatus.put(exec, ExcludedExecutor(node, expiryTimeForNewExcludes)) // post both to keep backwards compatibility @@ -299,8 +301,9 @@ private[scheduler] class HealthTracker ( // time. if (excludedExecsOnNode.size >= MAX_FAILED_EXEC_PER_NODE && !nodeIdToExcludedExpiryTime.contains(node)) { - logInfo(s"Excluding node $node because it has ${excludedExecsOnNode.size} " + - s"executors excluded: ${excludedExecsOnNode}") + logInfo(log"Excluding node ${MDC(HOST, node)} because it has " + + log"${MDC(NUM_EXECUTORS, excludedExecsOnNode.size)} executors " + + log"excluded: ${MDC(EXECUTOR_IDS, excludedExecsOnNode)}") nodeIdToExcludedExpiryTime.put(node, expiryTimeForNewExcludes) // post both to keep backwards compatibility listenerBus.post(SparkListenerNodeBlacklisted(now, node, excludedExecsOnNode.size)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala index cd5d6b8f9c90d..d9020da4cdcb3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler import scala.collection.mutable import org.apache.spark._ -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.util.{RpcUtils, ThreadUtils} @@ -124,7 +124,7 @@ private[spark] class OutputCommitCoordinator( stageStates.get(stage) match { case Some(state) => require(state.authorizedCommitters.length == maxPartitionId + 1) - logInfo(s"Reusing state from previous attempt of stage $stage.") + logInfo(log"Reusing state from previous attempt of stage ${MDC(LogKeys.STAGE_ID, stage)}") case _ => stageStates(stage) = new StageState(maxPartitionId + 1) @@ -151,8 +151,10 @@ private[spark] class OutputCommitCoordinator( case Success => // The task output has been committed successfully case _: TaskCommitDenied => - logInfo(s"Task was denied committing, stage: $stage.$stageAttempt, " + - s"partition: $partition, attempt: $attemptNumber") + logInfo(log"Task was denied committing, stage: ${MDC(LogKeys.STAGE_ID, stage)}." + + log"${MDC(LogKeys.STAGE_ATTEMPT, stageAttempt)}, " + + log"partition: ${MDC(LogKeys.PARTITION_ID, partition)}, " + + log"attempt: ${MDC(LogKeys.NUM_ATTEMPT, attemptNumber)}") case _ => // Mark the attempt as failed to exclude from future commit protocol val taskId = TaskIdentifier(stageAttempt, attemptNumber) @@ -182,8 +184,10 @@ private[spark] class OutputCommitCoordinator( attemptNumber: Int): Boolean = synchronized { stageStates.get(stage) match { case Some(state) if attemptFailed(state, stageAttempt, partition, attemptNumber) => - logInfo(s"Commit denied for stage=$stage.$stageAttempt, partition=$partition: " + - s"task attempt $attemptNumber already marked as failed.") + logInfo(log"Commit denied for stage=${MDC(LogKeys.STAGE_ID, stage)}." + + log"${MDC(LogKeys.STAGE_ATTEMPT, stageAttempt)}, partition=" + + log"${MDC(LogKeys.PARTITION_ID, partition)}: task attempt " + + log"${MDC(LogKeys.NUM_ATTEMPT, attemptNumber)} already marked as failed.") false case Some(state) => val existing = state.authorizedCommitters(partition) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index b5cc6261cea38..6f64dff3f39d6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -80,20 +80,23 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool, sc: SparkContext fileData = schedulerAllocFile.map { f => val filePath = new Path(f) val fis = filePath.getFileSystem(sc.hadoopConfiguration).open(filePath) - logInfo(s"Creating Fair Scheduler pools from $f") + logInfo(log"Creating Fair Scheduler pools from ${MDC(LogKeys.FILE_NAME, f)}") Some((fis, f)) }.getOrElse { val is = Utils.getSparkClassLoader.getResourceAsStream(DEFAULT_SCHEDULER_FILE) if (is != null) { - logInfo(s"Creating Fair Scheduler pools from default file: $DEFAULT_SCHEDULER_FILE") + logInfo(log"Creating Fair Scheduler pools from default file: " + + log"${MDC(LogKeys.FILE_NAME, DEFAULT_SCHEDULER_FILE)}") Some((is, DEFAULT_SCHEDULER_FILE)) } else { val schedulingMode = SchedulingMode.withName(sc.conf.get(SCHEDULER_MODE)) rootPool.addSchedulable(new Pool( DEFAULT_POOL_NAME, schedulingMode, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)) - logInfo("Fair scheduler configuration not found, created default pool: " + - "%s, schedulingMode: %s, minShare: %d, weight: %d".format( - DEFAULT_POOL_NAME, schedulingMode, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)) + logInfo(log"Fair scheduler configuration not found, created default pool: " + + log"${MDC(LogKeys.DEFAULT_NAME, DEFAULT_POOL_NAME)}, " + + log"schedulingMode: ${MDC(LogKeys.SCHEDULING_MODE, schedulingMode)}, " + + log"minShare: ${MDC(LogKeys.MIN_SHARE, DEFAULT_MINIMUM_SHARE)}, " + + log"weight: ${MDC(LogKeys.WEIGHT, DEFAULT_WEIGHT)}") None } } @@ -122,8 +125,10 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool, sc: SparkContext val pool = new Pool(DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) rootPool.addSchedulable(pool) - logInfo("Created default pool: %s, schedulingMode: %s, minShare: %d, weight: %d".format( - DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)) + logInfo(log"Created default pool: ${MDC(LogKeys.POOL_NAME, DEFAULT_POOL_NAME)}, " + + log"schedulingMode: ${MDC(LogKeys.SCHEDULING_MODE, DEFAULT_SCHEDULING_MODE)}, " + + log"minShare: ${MDC(LogKeys.MIN_SHARE, DEFAULT_MINIMUM_SHARE)}, " + + log"weight: ${MDC(LogKeys.WEIGHT, DEFAULT_WEIGHT)}") } } @@ -142,8 +147,10 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool, sc: SparkContext rootPool.addSchedulable(new Pool(poolName, schedulingMode, minShare, weight)) - logInfo("Created pool: %s, schedulingMode: %s, minShare: %d, weight: %d".format( - poolName, schedulingMode, minShare, weight)) + logInfo(log"Created pool: ${MDC(LogKeys.POOL_NAME, poolName)}, " + + log"schedulingMode: ${MDC(LogKeys.SCHEDULING_MODE, schedulingMode)}, " + + log"minShare: ${MDC(LogKeys.MIN_SHARE, minShare)}, " + + log"weight: ${MDC(LogKeys.WEIGHT, weight)}") } } @@ -159,7 +166,7 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool, sc: SparkContext log"${MDC(XML_SCHEDULING_MODE, xmlSchedulingMode)} found in " + log"Fair Scheduler configuration file: ${MDC(FILE_NAME, fileName)}, using " + log"the default schedulingMode: " + - log"${MDC(LogKeys.DEFAULT_SCHEDULING_MODE, defaultValue)} for pool: " + + log"${MDC(LogKeys.SCHEDULING_MODE, defaultValue)} for pool: " + log"${MDC(POOL_NAME, poolName)}" try { if (SchedulingMode.withName(xmlSchedulingMode) != SchedulingMode.NONE) { @@ -215,11 +222,12 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool, sc: SparkContext log"when that file doesn't contain ${MDC(POOL_NAME, poolName)}. " + log"Created ${MDC(CREATED_POOL_NAME, poolName)} with default " + log"configuration (schedulingMode: " + - log"${MDC(LogKeys.DEFAULT_SCHEDULING_MODE, DEFAULT_SCHEDULING_MODE)}, " + + log"${MDC(LogKeys.SCHEDULING_MODE, DEFAULT_SCHEDULING_MODE)}, " + log"minShare: ${MDC(MIN_SHARE, DEFAULT_MINIMUM_SHARE)}, " + log"weight: ${MDC(WEIGHT, DEFAULT_WEIGHT)}") } parentPool.addSchedulable(manager) - logInfo("Added task set " + manager.name + " tasks to pool " + poolName) + logInfo(log"Added task set ${MDC(LogKeys.TASK_SET_MANAGER, manager.name)} tasks to pool " + + log"${MDC(LogKeys.POOL_NAME, poolName)}") } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala b/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala index 1f12b46412bc5..e46dde5561a26 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala @@ -21,7 +21,7 @@ import scala.collection.mutable import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.util.{Distribution, Utils} @@ -46,7 +46,8 @@ class StatsReportListener extends SparkListener with Logging { override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { implicit val sc = stageCompleted - this.logInfo(s"Finished stage: ${getStatusDetail(stageCompleted.stageInfo)}") + this.logInfo( + log"Finished stage: ${MDC(LogKeys.STAGE, getStatusDetail(stageCompleted.stageInfo))}") showMillisDistribution("task runtime:", (info, _) => info.duration, taskInfoMetrics.toSeq) // Shuffle write @@ -111,9 +112,9 @@ private[spark] object StatsReportListener extends Logging { def showDistribution(heading: String, d: Distribution, formatNumber: Double => String): Unit = { val stats = d.statCounter val quantiles = d.getQuantiles(probabilities).map(formatNumber) - logInfo(heading + stats) + logInfo(log"${MDC(LogKeys.DESCRIPTION, heading)}${MDC(LogKeys.STATS, stats)}") logInfo(percentilesHeader) - logInfo("\t" + quantiles.mkString("\t")) + logInfo(log"\t" + log"${MDC(LogKeys.QUANTILES, quantiles.mkString("\t"))}") } def showDistribution( diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 15bdd58288f1e..ad0e0ddb687ee 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -250,8 +250,9 @@ private[spark] class TaskSchedulerImpl( override def submitTasks(taskSet: TaskSet): Unit = { val tasks = taskSet.tasks - logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks " - + "resource profile " + taskSet.resourceProfileId) + logInfo(log"Adding task set ${MDC(LogKeys.TASK_SET_ID, taskSet.id)} with " + + log"${MDC(LogKeys.NUM_TASKS, tasks.length)} tasks resource profile " + + log"${MDC(LogKeys.RESOURCE_PROFILE_ID, taskSet.resourceProfileId)}") this.synchronized { val manager = createTaskSetManager(taskSet, maxTaskFailures) val stage = taskSet.stageId @@ -306,9 +307,10 @@ private[spark] class TaskSchedulerImpl( stageId: Int, interruptThread: Boolean, reason: String): Unit = synchronized { - logInfo("Cancelling stage " + stageId) + logInfo(log"Canceling stage ${MDC(LogKeys.STAGE_ID, stageId)}") // Kill all running tasks for the stage. - logInfo(s"Killing all running tasks in stage $stageId: $reason") + logInfo(log"Killing all running tasks in stage ${MDC(LogKeys.STAGE_ID, stageId)}: " + + log"${MDC(LogKeys.REASON, reason)}") taskSetsByStageIdAndAttempt.get(stageId).foreach { attempts => attempts.foreach { case (_, tsm) => // There are two possible cases here: @@ -322,7 +324,8 @@ private[spark] class TaskSchedulerImpl( } } tsm.suspend() - logInfo("Stage %s.%s was cancelled".format(stageId, tsm.taskSet.stageAttemptId)) + logInfo(log"Stage ${MDC(LogKeys.STAGE_ID, stageId)}." + + log"${MDC(LogKeys.STAGE_ATTEMPT, tsm.taskSet.stageAttemptId)} was cancelled") } } } @@ -331,7 +334,7 @@ private[spark] class TaskSchedulerImpl( taskId: Long, interruptThread: Boolean, reason: String): Boolean = synchronized { - logInfo(s"Killing task $taskId: $reason") + logInfo(log"Killing task ${MDC(LogKeys.TASK_ID, taskId)}: ${MDC(LogKeys.REASON, reason)}") val execId = taskIdToExecutorId.get(taskId) if (execId.isDefined) { backend.killTask(taskId, execId.get, interruptThread, reason) @@ -361,8 +364,8 @@ private[spark] class TaskSchedulerImpl( } noRejectsSinceLastReset -= manager.taskSet manager.parent.removeSchedulable(manager) - logInfo(s"Removed TaskSet ${manager.taskSet.id}, whose tasks have all completed, from pool" + - s" ${manager.parent.name}") + logInfo(log"Removed TaskSet ${MDC(LogKeys.TASK_SET_NAME, manager.taskSet.id)}, whose tasks " + + log"have all completed, from pool ${MDC(LogKeys.POOL_NAME, manager.parent.name)}") } /** @@ -559,9 +562,10 @@ private[spark] class TaskSchedulerImpl( // Skip the launch process. // TODO SPARK-24819 If the job requires more slots than available (both busy and free // slots), fail the job on submit. - logInfo(s"Skip current round of resource offers for barrier stage ${taskSet.stageId} " + - s"because the barrier taskSet requires ${taskSet.numTasks} slots, while the total " + - s"number of available slots is $numBarrierSlotsAvailable.") + logInfo(log"Skip current round of resource offers for barrier stage " + + log"${MDC(LogKeys.STAGE_ID, taskSet.stageId)} because the barrier taskSet requires " + + log"${MDC(LogKeys.TASK_SET_NAME, taskSet.numTasks)} slots, while the total " + + log"number of available slots is ${MDC(LogKeys.NUM_SLOTS, numBarrierSlotsAvailable)}.") } else { var launchedAnyTask = false var noDelaySchedulingRejects = true @@ -619,18 +623,18 @@ private[spark] class TaskSchedulerImpl( // in order to provision more executors to make them schedulable if (Utils.isDynamicAllocationEnabled(conf)) { if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) { - logInfo("Notifying ExecutorAllocationManager to allocate more executors to" + - " schedule the unschedulable task before aborting" + - s" stage ${taskSet.stageId}.") + logInfo(log"Notifying ExecutorAllocationManager to allocate more executors to" + + log" schedule the unschedulable task before aborting" + + log" stage ${MDC(LogKeys.STAGE_ID, taskSet.stageId)}.") dagScheduler.unschedulableTaskSetAdded(taskSet.taskSet.stageId, taskSet.taskSet.stageAttemptId) updateUnschedulableTaskSetTimeoutAndStartAbortTimer(taskSet, taskIndex) } } else { // Abort Immediately - logInfo("Cannot schedule any task because all executors excluded from " + - "failures. No idle executors can be found to kill. Aborting stage " + - s"${taskSet.stageId}.") + logInfo(log"Cannot schedule any task because all executors excluded from " + + log"failures. No idle executors can be found to kill. Aborting stage " + + log"${MDC(LogKeys.STAGE_ID, taskSet.stageId)}.") taskSet.abortSinceCompletelyExcludedOnFailure(taskIndex) } } @@ -643,8 +647,8 @@ private[spark] class TaskSchedulerImpl( // non-excluded executor and the abort timer doesn't kick in because of a constant // submission of new TaskSets. See the PR for more details. if (unschedulableTaskSetToExpiryTime.nonEmpty) { - logInfo("Clearing the expiry times for all unschedulable taskSets as a task was " + - "recently scheduled.") + logInfo(log"Clearing the expiry times for all unschedulable taskSets as a task " + + log"was recently scheduled.") // Notify ExecutorAllocationManager as well as other subscribers that a task now // recently becomes schedulable dagScheduler.unschedulableTaskSetRemoved(taskSet.taskSet.stageId, @@ -679,8 +683,8 @@ private[spark] class TaskSchedulerImpl( val curTime = clock.getTimeMillis() if (curTime - taskSet.lastResourceOfferFailLogTime > TaskSetManager.BARRIER_LOGGING_INTERVAL) { - logInfo("Releasing the assigned resource offers since only partial tasks can " + - "be launched. Waiting for later round resource offers.") + logInfo(log"Releasing the assigned resource offers since only partial tasks can " + + log"be launched. Waiting for later round resource offers.") taskSet.lastResourceOfferFailLogTime = curTime } barrierPendingLaunchTasks.foreach { task => @@ -722,8 +726,8 @@ private[spark] class TaskSchedulerImpl( .mkString(",") addressesWithDescs.foreach(_._2.properties.setProperty("addresses", addressesStr)) - logInfo(s"Successfully scheduled all the ${addressesWithDescs.length} tasks for " + - s"barrier stage ${taskSet.stageId}.") + logInfo(log"Successfully scheduled all the ${MDC(LogKeys.NUM_TASKS, addressesWithDescs.length)} " + + log"tasks for barrier stage ${MDC(LogKeys.STAGE_ID, taskSet.stageId)}.") } taskSet.barrierPendingLaunchTasks.clear() } @@ -743,8 +747,8 @@ private[spark] class TaskSchedulerImpl( taskIndex: Int): Unit = { val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000 unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout - logInfo(s"Waiting for $timeout ms for completely " + - s"excluded task to be schedulable again before aborting stage ${taskSet.stageId}.") + logInfo(log"Waiting for ${MDC(LogKeys.TIMEOUT, timeout)} ms for completely " + + log"excluded task to be schedulable again before aborting stage ${MDC(LogKeys.STAGE_ID, taskSet.stageId)}.") abortTimer.schedule( createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout, TimeUnit.MILLISECONDS) } @@ -756,8 +760,8 @@ private[spark] class TaskSchedulerImpl( override def run(): Unit = TaskSchedulerImpl.this.synchronized { if (unschedulableTaskSetToExpiryTime.contains(taskSet) && unschedulableTaskSetToExpiryTime(taskSet) <= clock.getTimeMillis()) { - logInfo("Cannot schedule any task because all executors excluded due to failures. " + - s"Wait time for scheduling expired. Aborting stage ${taskSet.stageId}.") + logInfo(log"Cannot schedule any task because all executors excluded due to failures. " + + log"Wait time for scheduling expired. Aborting stage ${MDC(LogKeys.STAGE_ID, taskSet.stageId)}.") taskSet.abortSinceCompletelyExcludedOnFailure(taskIndex) } else { this.cancel() @@ -1043,7 +1047,8 @@ private[spark] class TaskSchedulerImpl( } override def workerRemoved(workerId: String, host: String, message: String): Unit = { - logInfo(s"Handle removed worker $workerId: $message") + logInfo(log"Handle removed worker ${MDC(LogKeys.WORKER_ID, workerId)}: " + + log"${MDC(LogKeys.MESSAGE, message)}") dagScheduler.workerRemoved(workerId, host, message) } @@ -1054,10 +1059,12 @@ private[spark] class TaskSchedulerImpl( case LossReasonPending => logDebug(s"Executor $executorId on $hostPort lost, but reason not yet known.") case ExecutorKilled => - logInfo(s"Executor $executorId on $hostPort killed by driver.") + logInfo(log"Executor ${MDC(LogKeys.EXECUTOR_ID, executorId)} on " + + log"${MDC(LogKeys.HOST_PORT, hostPort)} killed by driver.") case _: ExecutorDecommission => - logInfo(s"Executor $executorId on $hostPort is decommissioned" + - s"${getDecommissionDuration(executorId)}.") + logInfo(log"Executor ${MDC(LogKeys.EXECUTOR_ID, executorId)} on " + + log"${MDC(LogKeys.HOST_PORT, hostPort)} is decommissioned" + + log"${MDC(DURATION, getDecommissionDuration(executorId))}.") case _ => logError(log"Lost executor ${MDC(LogKeys.EXECUTOR_ID, executorId)} on " + log"${MDC(LogKeys.HOST, hostPort)}: ${MDC(LogKeys.REASON, reason)}") diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetExcludeList.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetExcludeList.scala index f479e5e32bc2f..c9aa74e0852be 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetExcludeList.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetExcludeList.scala @@ -19,8 +19,7 @@ package org.apache.spark.scheduler import scala.collection.mutable.{HashMap, HashSet} import org.apache.spark.SparkConf -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config +import org.apache.spark.internal.{config, Logging, LogKeys, MDC} import org.apache.spark.util.Clock /** @@ -134,7 +133,8 @@ private[scheduler] class TaskSetExcludelist( val numFailures = execFailures.numUniqueTasksWithFailures if (numFailures >= MAX_FAILURES_PER_EXEC_STAGE) { if (excludedExecs.add(exec)) { - logInfo(s"Excluding executor ${exec} for stage $stageId") + logInfo(log"Excluding executor ${MDC(LogKeys.EXECUTOR_ID, exec)} for stage " + + log"${MDC(LogKeys.STAGE_ID, stageId)}") // This executor has been excluded for this stage. Let's check if it // the whole node should be excluded. val excludedExecutorsOnNode = @@ -149,7 +149,8 @@ private[scheduler] class TaskSetExcludelist( val numFailExec = excludedExecutorsOnNode.size if (numFailExec >= MAX_FAILED_EXEC_PER_NODE_STAGE) { if (excludedNodes.add(host)) { - logInfo(s"Excluding ${host} for stage $stageId") + logInfo(log"Excluding ${MDC(LogKeys.HOST, host)} for " + + log"stage ${MDC(LogKeys.STAGE_ID, stageId)}") // SparkListenerNodeBlacklistedForStage is deprecated but post both events // to keep backward compatibility listenerBus.post( diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index b7ff443231f30..6573ab2f23d62 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -30,8 +30,7 @@ import org.apache.spark.InternalAccumulator import org.apache.spark.InternalAccumulator.{input, shuffleRead} import org.apache.spark.TaskState.TaskState import org.apache.spark.errors.SparkCoreErrors -import org.apache.spark.internal.{config, Logging, MDC} -import org.apache.spark.internal.LogKeys +import org.apache.spark.internal.{config, Logging, LogKeys, MDC} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ import org.apache.spark.scheduler.SchedulingMode._ @@ -280,8 +279,9 @@ private[spark] class TaskSetManager( for (e <- set) { pendingTaskSetToAddTo.forExecutor.getOrElseUpdate(e, new ArrayBuffer) += index } - logInfo(s"Pending task $index has a cached location at ${e.host} " + - ", where there are executors " + set.mkString(",")) + logInfo(log"Pending task ${MDC(INDEX, index)} has a cached location at " + + log"${MDC(HOST, e.host)}, where there are executors " + + log"${MDC(EXECUTOR_IDS, set.mkString(","))}") case None => logDebug(s"Pending task $index has a cached location at ${e.host} " + ", but there are no executors alive there.") } @@ -554,10 +554,16 @@ private[spark] class TaskSetManager( // a good proxy to task serialization time. // val timeTaken = clock.getTime() - startTime val tName = taskName(taskId) - logInfo(s"Starting $tName ($host, executor ${info.executorId}, " + - s"partition ${task.partitionId}, $taskLocality, ${serializedTask.limit()} bytes) " + - (if (taskResourceAssignments.nonEmpty) s"taskResourceAssignments ${taskResourceAssignments}" - else "")) + logInfo(log"Starting ${MDC(TASK_NAME, tName)} (${MDC(HOST, host)}," + + log"executor ${MDC(LogKeys.EXECUTOR_ID, info.executorId)}, " + + log"partition ${MDC(PARTITION_ID, task.partitionId)}, " + + log"${MDC(TASK_LOCALITY, taskLocality)}, " + + log"${MDC(SIZE, serializedTask.limit())} bytes) " + + (if (taskResourceAssignments.nonEmpty) { + log"taskResourceAssignments ${MDC(TASK_RESOURCE_ASSIGNMENTS, taskResourceAssignments)}" + } else { + log"" + })) sched.dagScheduler.taskStarted(task, info) new TaskDescription( @@ -829,8 +835,11 @@ private[spark] class TaskSetManager( // Kill any other attempts for the same task (since those are unnecessary now that one // attempt completed successfully). for (attemptInfo <- taskAttempts(index) if attemptInfo.running) { - logInfo(s"Killing attempt ${attemptInfo.attemptNumber} for ${taskName(attemptInfo.taskId)}" + - s" on ${attemptInfo.host} as the attempt ${info.attemptNumber} succeeded on ${info.host}") + logInfo(log"Killing attempt ${MDC(NUM_ATTEMPT, attemptInfo.attemptNumber)} for " + + log"${MDC(TASK_NAME, taskName(attemptInfo.taskId))} on " + + log"${MDC(HOST, attemptInfo.host)} as the attempt " + + log"${MDC(TASK_ATTEMPT_ID, info.attemptNumber)} succeeded on " + + log"${MDC(HOST, info.host)}") killedByOtherAttempt += attemptInfo.taskId sched.backend.killTask( attemptInfo.taskId, @@ -840,8 +849,10 @@ private[spark] class TaskSetManager( } if (!successful(index)) { tasksSuccessful += 1 - logInfo(s"Finished ${taskName(info.taskId)} in ${info.duration} ms " + - s"on ${info.host} (executor ${info.executorId}) ($tasksSuccessful/$numTasks)") + logInfo(log"Finished ${MDC(TASK_NAME, taskName(info.taskId))} in " + + log"${MDC(DURATION, info.duration)} ms on ${MDC(HOST, info.host)} " + + log"(executor ${MDC(LogKeys.EXECUTOR_ID, info.executorId)}) " + + log"(${MDC(NUM_SUCCESSFUL_TASKS, tasksSuccessful)}/${MDC(NUM_TASKS, numTasks)})") // Mark successful and stop if all the tasks have succeeded. successful(index) = true numFailures(index) = 0 @@ -849,8 +860,9 @@ private[spark] class TaskSetManager( isZombie = true } } else { - logInfo(s"Ignoring task-finished event for ${taskName(info.taskId)} " + - s"because it has already completed successfully") + logInfo(log"Ignoring task-finished event for " + + log"${MDC(TASK_NAME, taskName(info.taskId))} " + + log"because it has already completed successfully") } // This method is called by "TaskSchedulerImpl.handleSuccessfulTask" which holds the // "TaskSchedulerImpl" lock until exiting. To avoid the SPARK-7655 issue, we should not @@ -1007,8 +1019,10 @@ private[spark] class TaskSetManager( logWarning(failureReason) } else { logInfo( - s"Lost $task on ${info.host}, executor ${info.executorId}: " + - s"${ef.className} (${ef.description}) [duplicate $dupCount]") + log"Lost ${MDC(TASK_NAME, task)} on ${MDC(HOST, info.host)}, " + + log"executor ${MDC(LogKeys.EXECUTOR_ID, info.executorId)}: " + + log"${MDC(CLASS_NAME, ef.className)} " + + log"(${MDC(DESCRIPTION, ef.description)}) [duplicate ${MDC(COUNT, dupCount)}]") } ef.exception @@ -1020,9 +1034,9 @@ private[spark] class TaskSetManager( None case e: ExecutorLostFailure if !e.exitCausedByApp => - logInfo(s"${taskName(tid)} failed because while it was being computed, its executor " + - "exited for a reason unrelated to the task. Not counting this failure towards the " + - "maximum number of failures for the task.") + logInfo(log"${MDC(TASK_NAME, taskName(tid))} failed because while it was being computed," + + log" its executor exited for a reason unrelated to the task. " + + log"Not counting this failure towards the maximum number of failures for the task.") None case _: TaskFailedReason => // TaskResultLost and others @@ -1052,10 +1066,10 @@ private[spark] class TaskSetManager( } if (successful(index)) { - logInfo(s"${taskName(info.taskId)} failed, but the task will not" + - " be re-executed (either because the task failed with a shuffle data fetch failure," + - " so the previous stage needs to be re-run, or because a different copy of the task" + - " has already succeeded).") + logInfo(log"${MDC(LogKeys.TASK_NAME, taskName(info.taskId))} failed, but the task will not" + + log" be re-executed (either because the task failed with a shuffle data fetch failure," + + log" so the previous stage needs to be re-run, or because a different copy of the task" + + log" has already succeeded).") } else { addPendingTask(index) } @@ -1238,9 +1252,10 @@ private[spark] class TaskSetManager( if (speculated) { addPendingTask(index, speculatable = true) logInfo( - ("Marking task %d in stage %s (on %s) as speculatable because it ran more" + - " than %.0f ms(%d speculatable tasks in this taskset now)") - .format(index, taskSet.id, info.host, threshold, speculatableTasks.size + 1)) + log"Marking task ${MDC(INDEX, index)} in stage ${MDC(STAGE_ID, taskSet.id)} (on " + + log"${MDC(HOST, info.host)}) as speculatable because it ran more than " + + log"${MDC(TIMEOUT, threshold)} ms(${MDC(NUM_TASKS, speculatableTasks.size + 1)}" + + log"speculatable tasks in this taskset now)") speculatableTasks += index sched.dagScheduler.speculativeTaskSubmitted(tasks(index), index) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index d359b65caa931..deaa1b4e47906 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -42,6 +42,7 @@ import org.apache.spark.rpc._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend.ENDPOINT_NAME +import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_MILLIS import org.apache.spark.status.api.v1.ThreadStackTrace import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils, Utils} import org.apache.spark.util.ArrayImplicits._ @@ -258,7 +259,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // If the cluster manager gives us an executor on an excluded node (because it // already started allocating those resources before we informed it of our exclusion, // or if it ignored our exclusion), then we reject that executor immediately. - logInfo(s"Rejecting $executorId as it has been excluded.") + logInfo(log"Rejecting ${MDC(LogKeys.EXECUTOR_ID, executorId)} as it has been excluded.") context.sendFailure( new IllegalStateException(s"Executor is excluded due to failures: $executorId")) } else { @@ -269,8 +270,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } else { context.senderAddress } - logInfo(s"Registered executor $executorRef ($executorAddress) with ID $executorId, " + - s" ResourceProfileId $resourceProfileId") + logInfo(log"Registered executor ${MDC(LogKeys.RPC_ENDPOINT_REF, executorRef)} " + + log"(${MDC(LogKeys.RPC_ADDRESS, executorAddress)}) " + + log"with ID ${MDC(LogKeys.EXECUTOR_ID, executorId)}, " + + log"ResourceProfileId ${MDC(LogKeys.RESOURCE_PROFILE_ID, resourceProfileId)}") addressToExecutorId(executorAddress) = executorId totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) @@ -324,7 +327,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp case UpdateExecutorsLogLevel(logLevel) => currentLogLevel = Some(logLevel) - logInfo(s"Asking each executor to refresh the log level to $logLevel") + logInfo(log"Asking each executor to refresh the log level to " + + log"${MDC(LogKeys.LOG_LEVEL, logLevel)}") for ((_, executorData) <- executorDataMap) { executorData.executorEndpoint.send(UpdateExecutorLogLevel(logLevel)) } @@ -497,7 +501,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // forever. Therefore, we should also post `SparkListenerExecutorRemoved` here. listenerBus.post(SparkListenerExecutorRemoved( System.currentTimeMillis(), executorId, reason.toString)) - logInfo(s"Asked to remove non-existent executor $executorId") + logInfo( + log"Asked to remove non-existent executor ${MDC(LogKeys.EXECUTOR_ID, executorId)}") } } @@ -526,7 +531,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } if (shouldDisable) { - logInfo(s"Disabling executor $executorId.") + logInfo(log"Disabling executor ${MDC(LogKeys.EXECUTOR_ID, executorId)}.") scheduler.executorLost(executorId, LossReasonPending) } @@ -570,7 +575,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp return executorsToDecommission.toImmutableArraySeq } - logInfo(s"Decommission executors: ${executorsToDecommission.mkString(", ")}") + logInfo(log"Decommission executors: " + + log"${MDC(LogKeys.EXECUTOR_IDS, executorsToDecommission.mkString(", "))}") // If we don't want to replace the executors we are decommissioning if (adjustTargetNumExecutors) { @@ -589,7 +595,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp if (!triggeredByExecutor) { executorsToDecommission.foreach { executorId => - logInfo(s"Notify executor $executorId to decommission.") + logInfo(log"Notify executor ${MDC(LogKeys.EXECUTOR_ID, executorId)} to decommission.") executorDataMap(executorId).executorEndpoint.send(DecommissionExecutor) } } @@ -601,7 +607,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp executorsToDecommission.filter(executorsPendingDecommission.contains) } if (stragglers.nonEmpty) { - logInfo(s"${stragglers.toList} failed to decommission in ${cleanupInterval}, killing.") + logInfo( + log"${MDC(LogKeys.EXECUTOR_IDS, stragglers.toList)} failed to decommission in " + + log"${MDC(LogKeys.INTERVAL, cleanupInterval)}, killing.") killExecutors(stragglers.toImmutableArraySeq, false, false, true) } } @@ -718,13 +726,14 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp override def isReady(): Boolean = { if (sufficientResourcesRegistered()) { - logInfo("SchedulerBackend is ready for scheduling beginning after " + - s"reached minRegisteredResourcesRatio: $minRegisteredRatio") + logInfo(log"SchedulerBackend is ready for scheduling beginning after " + + log"reached minRegisteredResourcesRatio: ${MDC(LogKeys.MIN_SIZE, minRegisteredRatio)}") return true } if ((System.nanoTime() - createTimeNs) >= maxRegisteredWaitingTimeNs) { - logInfo("SchedulerBackend is ready for scheduling beginning after waiting " + - s"maxRegisteredResourcesWaitingTime: $maxRegisteredWaitingTimeNs(ns)") + logInfo(log"SchedulerBackend is ready for scheduling beginning after waiting " + + log"maxRegisteredResourcesWaitingTime: " + + log"${MDC(LogKeys.TIMEOUT, maxRegisteredWaitingTimeNs / NANOS_PER_MILLIS.toDouble)}(ms)") return true } false @@ -801,7 +810,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp "Attempted to request a negative number of additional executor(s) " + s"$numAdditionalExecutors from the cluster manager. Please specify a positive number!") } - logInfo(s"Requesting $numAdditionalExecutors additional executor(s) from the cluster manager") + logInfo(log"Requesting ${MDC(LogKeys.NUM_EXECUTORS, numAdditionalExecutors)} additional " + + log"executor(s) from the cluster manager") val response = synchronized { val defaultProf = scheduler.sc.resourceProfileManager.defaultResourceProfile @@ -951,7 +961,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp adjustTargetNumExecutors: Boolean, countFailures: Boolean, force: Boolean): Seq[String] = { - logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}") + logInfo( + log"Requesting to kill executor(s) ${MDC(LogKeys.EXECUTOR_IDS, executorIds.mkString(", "))}") val response = withLock { val (knownExecutors, unknownExecutors) = executorIds.partition(executorDataMap.contains) @@ -966,7 +977,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp .filter { id => force || !scheduler.isExecutorBusy(id) } executorsToKill.foreach { id => executorsPendingToRemove(id) = !countFailures } - logInfo(s"Actual list of executor(s) to be killed is ${executorsToKill.mkString(", ")}") + logInfo(log"Actual list of executor(s) to be killed is " + + log"${MDC(LogKeys.EXECUTOR_IDS, executorsToKill.mkString(", "))}") // If we do not wish to replace the executors we kill, sync the target number of executors // with the cluster manager to avoid allocating new ones. When computing the new target, @@ -1007,7 +1019,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp * @return whether the decommission request is acknowledged. */ final override def decommissionExecutorsOnHost(host: String): Boolean = { - logInfo(s"Requesting to kill any and all executors on host $host") + logInfo(log"Requesting to kill any and all executors on host ${MDC(LogKeys.HOST, host)}") // A potential race exists if a new executor attempts to register on a host // that is on the exclude list and is no longer valid. To avoid this race, // all executor registration and decommissioning happens in the event loop. This way, either @@ -1023,7 +1035,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp * @return whether the kill request is acknowledged. */ final override def killExecutorsOnHost(host: String): Boolean = { - logInfo(s"Requesting to kill any and all executors on host $host") + logInfo(log"Requesting to kill any and all executors on host ${MDC(LogKeys.HOST, host)}") // A potential race exists if a new executor attempts to register on a host // that is on the exclude list and is no longer valid. To avoid this race, // all executor registration and killing happens in the event loop. This way, either diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 8f15dec6739a8..f4caecd7d6741 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -27,8 +27,7 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{StandaloneAppClient, StandaloneAppClientListener} import org.apache.spark.executor.ExecutorExitCode -import org.apache.spark.internal.{config, Logging, MDC} -import org.apache.spark.internal.LogKeys.REASON +import org.apache.spark.internal.{config, Logging, LogKeys, MDC} import org.apache.spark.internal.config.EXECUTOR_REMOVE_DELAY import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} @@ -145,7 +144,7 @@ private[spark] class StandaloneSchedulerBackend( } override def connected(appId: String): Unit = { - logInfo("Connected to Spark cluster with app ID " + appId) + logInfo(log"Connected to Spark cluster with app ID ${MDC(LogKeys.APP_ID, appId)}") this.appId = appId notifyContext() launcherBackend.setAppId(appId) @@ -162,7 +161,7 @@ private[spark] class StandaloneSchedulerBackend( notifyContext() if (!stopping.get) { launcherBackend.setState(SparkAppHandle.State.KILLED) - logError(log"Application has been killed. Reason: ${MDC(REASON, reason)}") + logError(log"Application has been killed. Reason: ${MDC(LogKeys.REASON, reason)}") try { scheduler.error(reason) } finally { @@ -174,8 +173,9 @@ private[spark] class StandaloneSchedulerBackend( override def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int): Unit = { - logInfo("Granted executor ID %s on hostPort %s with %d core(s), %s RAM".format( - fullId, hostPort, cores, Utils.megabytesToString(memory))) + logInfo(log"Granted executor ID ${MDC(LogKeys.EXECUTOR_ID, fullId)} on hostPort " + + log"${MDC(LogKeys.HOST_PORT, hostPort)} with ${MDC(LogKeys.NUM_CORES, cores)} core(s), " + + log"${MDC(LogKeys.MEMORY_SIZE, Utils.megabytesToString(memory))} RAM") } override def executorRemoved( @@ -192,23 +192,28 @@ private[spark] class StandaloneSchedulerBackend( case Some(code) => ExecutorExited(code, exitCausedByApp = true, message) case None => ExecutorProcessLost(message, workerHost, causedByApp = workerHost.isEmpty) } - logInfo("Executor %s removed: %s".format(fullId, message)) + logInfo( + log"Executor ${MDC(LogKeys.EXECUTOR_ID, fullId)} removed: ${MDC(LogKeys.MESSAGE, message)}") removeExecutor(fullId.split("/")(1), reason) } override def executorDecommissioned(fullId: String, decommissionInfo: ExecutorDecommissionInfo): Unit = { - logInfo(s"Asked to decommission executor $fullId") + logInfo(log"Asked to decommission executor ${MDC(LogKeys.EXECUTOR_ID, fullId)}") val execId = fullId.split("/")(1) decommissionExecutors( Array((execId, decommissionInfo)), adjustTargetNumExecutors = false, triggeredByExecutor = false) - logInfo("Executor %s decommissioned: %s".format(fullId, decommissionInfo)) + logInfo( + log"Executor ${MDC(LogKeys.EXECUTOR_ID, fullId)} " + + log"decommissioned: ${MDC(LogKeys.DESCRIPTION, decommissionInfo)}" + ) } override def workerRemoved(workerId: String, host: String, message: String): Unit = { - logInfo("Worker %s removed: %s".format(workerId, message)) + logInfo(log"Worker ${MDC(LogKeys.WORKER_ID, workerId)} removed: " + + log"${MDC(LogKeys.MESSAGE, message)}") removeWorker(workerId, host, message) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala index c389b0c988f4d..57505c87f879e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala @@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark._ import org.apache.spark.errors.SparkCoreErrors -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.config._ import org.apache.spark.resource.ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID import org.apache.spark.scheduler._ @@ -342,7 +342,8 @@ private[spark] class ExecutorMonitor( override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { val exec = ensureExecutorIsTracked(event.executorId, event.executorInfo.resourceProfileId) exec.updateRunningTasks(0) - logInfo(s"New executor ${event.executorId} has registered (new total is ${executors.size()})") + logInfo(log"New executor ${MDC(LogKeys.EXECUTOR_ID, event.executorId)} has registered " + + log"(new total is ${MDC(LogKeys.COUNT, executors.size())})") } private def decrementExecResourceProfileCount(rpId: Int): Unit = { @@ -365,11 +366,14 @@ private[spark] class ExecutorMonitor( } else { metrics.exitedUnexpectedly.inc() } - logInfo(s"Executor ${event.executorId} is removed. Remove reason statistics: (" + - s"gracefully decommissioned: ${metrics.gracefullyDecommissioned.getCount()}, " + - s"decommision unfinished: ${metrics.decommissionUnfinished.getCount()}, " + - s"driver killed: ${metrics.driverKilled.getCount()}, " + - s"unexpectedly exited: ${metrics.exitedUnexpectedly.getCount()}).") + // scalastyle:off line.size.limit + logInfo(log"Executor ${MDC(LogKeys.EXECUTOR_ID, event.executorId)} is removed. " + + log"Remove reason statistics: (gracefully decommissioned: " + + log"${MDC(LogKeys.NUM_DECOMMISSIONED, metrics.gracefullyDecommissioned.getCount())}, " + + log"decommission unfinished: ${MDC(LogKeys.NUM_UNFINISHED_DECOMMISSIONED, metrics.decommissionUnfinished.getCount())}, " + + log"driver killed: ${MDC(LogKeys.NUM_EXECUTORS_KILLED, metrics.driverKilled.getCount())}, " + + log"unexpectedly exited: ${MDC(LogKeys.NUM_EXECUTORS_EXITED, metrics.exitedUnexpectedly.getCount())}).") + // scalastyle:on line.size.limit if (!removed.pendingRemoval || !removed.decommissioning) { nextTimeout.set(Long.MinValue) } 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 26e2acf4392ca..8655b72310795 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -535,7 +535,7 @@ private[spark] class BlockManager( val priorityClass = conf.get(config.STORAGE_REPLICATION_POLICY) val clazz = Utils.classForName(priorityClass) val ret = clazz.getConstructor().newInstance().asInstanceOf[BlockReplicationPolicy] - logInfo(s"Using $priorityClass for block replication policy") + logInfo(log"Using ${MDC(CLASS_NAME, priorityClass)} for block replication policy") ret } @@ -547,7 +547,7 @@ private[spark] class BlockManager( // the registration with the ESS. Therefore, this registration should be prior to // the BlockManager registration. See SPARK-39647. if (externalShuffleServiceEnabled) { - logInfo(s"external shuffle service port = $externalShuffleServicePort") + logInfo(log"external shuffle service port = ${MDC(PORT, externalShuffleServicePort)}") shuffleServerId = BlockManagerId(executorId, blockTransferService.hostName, externalShuffleServicePort) if (!isDriver && !(Utils.isTesting && conf.get(Tests.TEST_SKIP_ESS_REGISTER))) { @@ -585,7 +585,7 @@ private[spark] class BlockManager( } } - logInfo(s"Initialized BlockManager: $blockManagerId") + logInfo(log"Initialized BlockManager: ${MDC(BLOCK_MANAGER_ID, blockManagerId)}") } def shuffleMetricsSource: Source = { @@ -646,7 +646,7 @@ private[spark] class BlockManager( * will be made then. */ private def reportAllBlocks(): Unit = { - logInfo(s"Reporting ${blockInfoManager.size} blocks to the master.") + logInfo(log"Reporting ${MDC(NUM_BLOCKS, blockInfoManager.size)} blocks to the master.") for ((blockId, info) <- blockInfoManager.entries) { val status = getCurrentBlockStatus(blockId, info) if (info.tellMaster && !tryToReportBlockStatus(blockId, status)) { @@ -664,7 +664,7 @@ private[spark] class BlockManager( */ def reregister(): Unit = { // TODO: We might need to rate limit re-registering. - logInfo(s"BlockManager $blockManagerId re-registering with master") + logInfo(log"BlockManager ${MDC(BLOCK_MANAGER_ID, blockManagerId)} re-registering with master") val id = master.registerBlockManager(blockManagerId, diskBlockManager.localDirsString, maxOnHeapMemory, maxOffHeapMemory, storageEndpoint, isReRegister = true) if (id.executorId != BlockManagerId.INVALID_EXECUTOR_ID) { @@ -875,7 +875,7 @@ private[spark] class BlockManager( droppedMemorySize: Long = 0L): Unit = { val needReregister = !tryToReportBlockStatus(blockId, status, droppedMemorySize) if (needReregister) { - logInfo(s"Got told to re-register updating block $blockId") + logInfo(log"Got told to re-register updating block ${MDC(BLOCK_ID, blockId)}") // Re-registering will report our new block for free. asyncReregister() } @@ -1139,8 +1139,9 @@ private[spark] class BlockManager( None } } - logInfo(s"Read $blockId from the disk of a same host executor is " + - (if (res.isDefined) "successful." else "failed.")) + logInfo( + log"Read ${MDC(BLOCK_ID, blockId)} from the disk of a same host executor is " + + log"${MDC(STATUS, if (res.isDefined) "successful." else "failed.")}") res }.orElse { fetchRemoteManagedBuffer(blockId, blockSize, locationsAndStatus).map(bufferTransformer) @@ -1308,12 +1309,12 @@ private[spark] class BlockManager( def get[T: ClassTag](blockId: BlockId): Option[BlockResult] = { val local = getLocalValues(blockId) if (local.isDefined) { - logInfo(s"Found block $blockId locally") + logInfo(log"Found block ${MDC(BLOCK_ID, blockId)} locally") return local } val remote = getRemoteValues[T](blockId) if (remote.isDefined) { - logInfo(s"Found block $blockId remotely") + logInfo(log"Found block ${MDC(BLOCK_ID, blockId)} remotely") return remote } None @@ -1820,7 +1821,8 @@ private[spark] class BlockManager( existingReplicas: Set[BlockManagerId], maxReplicas: Int, maxReplicationFailures: Option[Int] = None): Boolean = { - logInfo(s"Using $blockManagerId to pro-actively replicate $blockId") + logInfo(log"Using ${MDC(BLOCK_MANAGER_ID, blockManagerId)} to pro-actively replicate " + + log"${MDC(BLOCK_ID, blockId)}") blockInfoManager.lockForReading(blockId).forall { info => val data = doGetLocalBytes(blockId, info) val storageLevel = StorageLevel( @@ -1977,14 +1979,14 @@ private[spark] class BlockManager( private[storage] override def dropFromMemory[T: ClassTag]( blockId: BlockId, data: () => Either[Array[T], ChunkedByteBuffer]): StorageLevel = { - logInfo(s"Dropping block $blockId from memory") + logInfo(log"Dropping block ${MDC(BLOCK_ID, blockId)} from memory") val info = blockInfoManager.assertBlockIsLockedForWriting(blockId) var blockIsUpdated = false val level = info.level // Drop to disk, if storage level requires if (level.useDisk && !diskStore.contains(blockId)) { - logInfo(s"Writing block $blockId to disk") + logInfo(log"Writing block ${MDC(BLOCK_ID, blockId)} to disk") data() match { case Left(elements) => diskStore.put(blockId) { channel => @@ -2028,7 +2030,7 @@ private[spark] class BlockManager( */ def removeRdd(rddId: Int): Int = { // TODO: Avoid a linear scan by creating another mapping of RDD.id to blocks. - logInfo(s"Removing RDD $rddId") + logInfo(log"Removing RDD ${MDC(RDD_ID, rddId)}") val blocksToRemove = blockInfoManager.entries.flatMap(_._1.asRDDId).filter(_.rddId == rddId) blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster = false) } blocksToRemove.size diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala index fc98fbf6e18b3..19807453ee28c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerDecommissioner.scala @@ -73,14 +73,15 @@ private[storage] class BlockManagerDecommissioner( private def allowRetry(shuffleBlock: ShuffleBlockInfo, failureNum: Int): Boolean = { if (failureNum < maxReplicationFailuresForDecommission) { - logInfo(s"Add $shuffleBlock back to migration queue for " + - s"retry ($failureNum / $maxReplicationFailuresForDecommission)") + logInfo(log"Add ${MDC(SHUFFLE_BLOCK_INFO, shuffleBlock)} back to migration queue for " + + log" retry (${MDC(FAILURES, failureNum)} / " + + log"${MDC(MAX_ATTEMPTS, maxReplicationFailuresForDecommission)})") // The block needs to retry so we should not mark it as finished shufflesToMigrate.add((shuffleBlock, failureNum)) } else { logWarning(log"Give up migrating ${MDC(SHUFFLE_BLOCK_INFO, shuffleBlock)} " + log"since it's been failed for " + - log"${MDC(NUM_FAILURES, maxReplicationFailuresForDecommission)} times") + log"${MDC(MAX_ATTEMPTS, maxReplicationFailuresForDecommission)} times") false } } @@ -98,7 +99,7 @@ private[storage] class BlockManagerDecommissioner( } override def run(): Unit = { - logInfo(s"Starting shuffle block migration thread for $peer") + logInfo(log"Starting shuffle block migration thread for ${MDC(PEER, peer)}") // Once a block fails to transfer to an executor stop trying to transfer more blocks while (keepRunning) { try { @@ -107,10 +108,12 @@ private[storage] class BlockManagerDecommissioner( var isTargetDecommissioned = false // We only migrate a shuffle block when both index file and data file exist. if (blocks.isEmpty) { - logInfo(s"Ignore deleted shuffle block $shuffleBlockInfo") + logInfo(log"Ignore deleted shuffle block ${MDC(SHUFFLE_BLOCK_INFO, shuffleBlockInfo)}") } else { - logInfo(s"Got migration sub-blocks $blocks. Trying to migrate $shuffleBlockInfo " + - s"to $peer ($retryCount / $maxReplicationFailuresForDecommission)") + logInfo(log"Got migration sub-blocks ${MDC(BLOCK_IDS, blocks)}. Trying to migrate " + + log"${MDC(SHUFFLE_BLOCK_INFO, shuffleBlockInfo)} to ${MDC(PEER, peer)} " + + log"(${MDC(NUM_RETRY, retryCount)} / " + + log"${MDC(MAX_ATTEMPTS, maxReplicationFailuresForDecommission)}") // Migrate the components of the blocks. try { val startTime = System.currentTimeMillis() @@ -130,9 +133,10 @@ private[storage] class BlockManagerDecommissioner( logDebug(s"Migrated sub-block $blockId") } } - logInfo(s"Migrated $shuffleBlockInfo (" + - s"size: ${Utils.bytesToString(blocks.map(b => b._2.size()).sum)}) to $peer " + - s"in ${System.currentTimeMillis() - startTime} ms") + logInfo(log"Migrated ${MDC(SHUFFLE_BLOCK_INFO, shuffleBlockInfo)} (" + + log"size: ${MDC(SIZE, Utils.bytesToString(blocks.map(b => b._2.size()).sum))}) " + + log"to ${MDC(PEER, peer)} in " + + log"${MDC(DURATION, System.currentTimeMillis() - startTime)} ms") } catch { case e @ ( _ : IOException | _ : SparkException) => // If a block got deleted before netty opened the file handle, then trying to @@ -181,7 +185,11 @@ private[storage] class BlockManagerDecommissioner( } } catch { case _: InterruptedException => - logInfo(s"Stop shuffle block migration${if (keepRunning) " unexpectedly"}.") + if (keepRunning) { + logInfo("Stop shuffle block migration unexpectedly.") + } else { + logInfo("Stop shuffle block migration.") + } keepRunning = false case NonFatal(e) => keepRunning = false @@ -234,12 +242,16 @@ private[storage] class BlockManagerDecommissioner( logInfo("Attempting to migrate all cached RDD blocks") rddBlocksLeft = decommissionRddCacheBlocks() lastRDDMigrationTime = startTime - logInfo(s"Finished current round RDD blocks migration, " + - s"waiting for ${sleepInterval}ms before the next round migration.") + logInfo(log"Finished current round RDD blocks migration, " + + log"waiting for ${MDC(SLEEP_TIME, sleepInterval)}ms before the next round migration.") Thread.sleep(sleepInterval) } catch { case _: InterruptedException => - logInfo(s"Stop RDD blocks migration${if (!stopped && !stoppedRDD) " unexpectedly"}.") + if (!stopped && !stoppedRDD) { + logInfo("Stop RDD blocks migration unexpectedly.") + } else { + logInfo("Stop RDD blocks migration.") + } stoppedRDD = true case NonFatal(e) => logError("Error occurred during RDD blocks migration.", e) @@ -265,8 +277,9 @@ private[storage] class BlockManagerDecommissioner( val startTime = System.nanoTime() shuffleBlocksLeft = refreshMigratableShuffleBlocks() lastShuffleMigrationTime = startTime - logInfo(s"Finished current round refreshing migratable shuffle blocks, " + - s"waiting for ${sleepInterval}ms before the next round refreshing.") + logInfo(log"Finished current round refreshing migratable shuffle blocks, " + + log"waiting for ${MDC(SLEEP_TIME, sleepInterval)}ms before the " + + log"next round refreshing.") Thread.sleep(sleepInterval) } catch { case _: InterruptedException if stopped => @@ -302,8 +315,9 @@ private[storage] class BlockManagerDecommissioner( shufflesToMigrate.addAll(newShufflesToMigrate.map(x => (x, 0)).asJava) migratingShuffles ++= newShufflesToMigrate val remainedShuffles = migratingShuffles.size - numMigratedShuffles.get() - logInfo(s"${newShufflesToMigrate.size} of ${localShuffles.size} local shuffles " + - s"are added. In total, $remainedShuffles shuffles are remained.") + logInfo(log"${MDC(COUNT, newShufflesToMigrate.size)} of " + + log"${MDC(TOTAL, localShuffles.size)} local shuffles are added. " + + log"In total, ${MDC(NUM_REMAINED, remainedShuffles)} shuffles are remained.") // Update the threads doing migrations val livePeerSet = bm.getPeers(false).toSet @@ -350,8 +364,9 @@ private[storage] class BlockManagerDecommissioner( // Refresh peers and validate we have somewhere to move blocks. if (replicateBlocksInfo.nonEmpty) { - logInfo(s"Need to replicate ${replicateBlocksInfo.size} RDD blocks " + - "for block manager decommissioning") + logInfo( + log"Need to replicate ${MDC(NUM_REPLICAS, replicateBlocksInfo.size)} RDD blocks " + + log"for block manager decommissioning") } else { logWarning("Asked to decommission RDD cache blocks, but no blocks to migrate") return false @@ -378,9 +393,10 @@ private[storage] class BlockManagerDecommissioner( blockToReplicate.maxReplicas, maxReplicationFailures = Some(maxReplicationFailuresForDecommission)) if (replicatedSuccessfully) { - logInfo(s"Block ${blockToReplicate.blockId} migrated successfully, Removing block now") + logInfo(log"Block ${MDC(BLOCK_ID, blockToReplicate.blockId)} migrated " + + log"successfully, Removing block now") bm.removeBlock(blockToReplicate.blockId) - logInfo(s"Block ${blockToReplicate.blockId} removed") + logInfo(log"Block ${MDC(BLOCK_ID, blockToReplicate.blockId)} removed") } else { logWarning(log"Failed to migrate block ${MDC(BLOCK_ID, blockToReplicate.blockId)}") } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 95af44deef93d..276bd63e14237 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -42,7 +42,7 @@ class BlockManagerMaster( /** Remove a dead executor from the driver endpoint. This is only called on the driver side. */ def removeExecutor(execId: String): Unit = { tell(RemoveExecutor(execId)) - logInfo("Removed " + execId + " successfully in removeExecutor") + logInfo(log"Removed ${MDC(EXECUTOR_ID, execId)} successfully in removeExecutor") } /** Decommission block managers corresponding to given set of executors @@ -62,7 +62,7 @@ class BlockManagerMaster( */ def removeExecutorAsync(execId: String): Unit = { driverEndpoint.ask[Boolean](RemoveExecutor(execId)) - logInfo("Removal of executor " + execId + " requested") + logInfo(log"Removal of executor ${MDC(EXECUTOR_ID, execId)} requested") } /** @@ -77,7 +77,7 @@ class BlockManagerMaster( maxOffHeapMemSize: Long, storageEndpoint: RpcEndpointRef, isReRegister: Boolean = false): BlockManagerId = { - logInfo(s"Registering BlockManager $id") + logInfo(log"Registering BlockManager ${MDC(BLOCK_MANAGER_ID, id)}") val updatedId = driverEndpoint.askSync[BlockManagerId]( RegisterBlockManager( id, @@ -90,9 +90,9 @@ class BlockManagerMaster( ) if (updatedId.executorId == BlockManagerId.INVALID_EXECUTOR_ID) { assert(isReRegister, "Got invalid executor id from non re-register case") - logInfo(s"Re-register BlockManager $id failed") + logInfo(log"Re-register BlockManager ${MDC(BLOCK_MANAGER_ID, id)} failed") } else { - logInfo(s"Registered BlockManager $updatedId") + logInfo(log"Registered BlockManager ${MDC(BLOCK_MANAGER_ID, updatedId)}") } updatedId } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index be7082807182b..73f89ea0e86e5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -110,7 +110,7 @@ class BlockManagerMasterEndpoint( val clazz = Utils.classForName(topologyMapperClassName) val mapper = clazz.getConstructor(classOf[SparkConf]).newInstance(conf).asInstanceOf[TopologyMapper] - logInfo(s"Using $topologyMapperClassName for getting topology information") + logInfo(log"Using ${MDC(CLASS_NAME, topologyMapperClassName)} for getting topology information") mapper } @@ -218,7 +218,8 @@ class BlockManagerMasterEndpoint( // executor is notified(see BlockManager.decommissionSelf), so we don't need to send the // notification here. val bms = executorIds.flatMap(blockManagerIdByExecutor.get) - logInfo(s"Mark BlockManagers (${bms.mkString(", ")}) as being decommissioning.") + logInfo(log"Mark BlockManagers (${MDC(BLOCK_MANAGER_IDS, bms.mkString(", "))}) as " + + log"being decommissioning.") decommissioningBlockManagerSet ++= bms context.reply(true) @@ -535,7 +536,7 @@ class BlockManagerMasterEndpoint( } listenerBus.post(SparkListenerBlockManagerRemoved(System.currentTimeMillis(), blockManagerId)) - logInfo(s"Removing block manager $blockManagerId") + logInfo(log"Removing block manager ${MDC(BLOCK_MANAGER_ID, blockManagerId)}") } @@ -551,7 +552,7 @@ class BlockManagerMasterEndpoint( } private def removeExecutor(execId: String): Unit = { - logInfo("Trying to remove executor " + execId + " from BlockManagerMaster.") + logInfo(log"Trying to remove executor ${MDC(EXECUTOR_ID, execId)} from BlockManagerMaster.") blockManagerIdByExecutor.get(execId).foreach(removeBlockManager) } @@ -707,8 +708,9 @@ class BlockManagerMasterEndpoint( removeExecutor(id.executorId) case None => } - logInfo("Registering block manager %s with %s RAM, %s".format( - id.hostPort, Utils.bytesToString(maxOnHeapMemSize + maxOffHeapMemSize), id)) + logInfo(log"Registering block manager ${MDC(HOST_PORT, id.hostPort)} with " + + log"${MDC(MEMORY_SIZE, Utils.bytesToString(maxOnHeapMemSize + maxOffHeapMemSize))} RAM, " + + log"${MDC(BLOCK_MANAGER_ID, id)}") blockManagerIdByExecutor(id.executorId) = id @@ -738,8 +740,8 @@ class BlockManagerMasterEndpoint( assert(!blockManagerInfo.contains(id), "BlockManager re-registration shouldn't succeed when the executor is lost") - logInfo(s"BlockManager ($id) re-registration is rejected since " + - s"the executor (${id.executorId}) has been lost") + logInfo(log"BlockManager (${MDC(BLOCK_MANAGER_ID, id)}) re-registration is rejected since " + + log"the executor (${MDC(EXECUTOR_ID, id.executorId)}) has been lost") // Use "invalid" as the return executor id to indicate the block manager that // re-registration failed. It's a bit hacky but fine since the returned block @@ -1057,26 +1059,30 @@ private[spark] class BlockManagerInfo( _blocks.put(blockId, blockStatus) _remainingMem -= memSize if (blockExists) { - logInfo(s"Updated $blockId in memory on ${blockManagerId.hostPort}" + - s" (current size: ${Utils.bytesToString(memSize)}," + - s" original size: ${Utils.bytesToString(originalMemSize)}," + - s" free: ${Utils.bytesToString(_remainingMem)})") + logInfo(log"Updated ${MDC(BLOCK_ID, blockId)} in memory on " + + log"${MDC(HOST_PORT, blockManagerId.hostPort)} (current size: " + + log"${MDC(CURRENT_MEMORY_SIZE, Utils.bytesToString(memSize))}, original " + + log"size: ${MDC(ORIGINAL_MEMORY_SIZE, Utils.bytesToString(originalMemSize))}, " + + log"free: ${MDC(FREE_MEMORY_SIZE, Utils.bytesToString(_remainingMem))})") } else { - logInfo(s"Added $blockId in memory on ${blockManagerId.hostPort}" + - s" (size: ${Utils.bytesToString(memSize)}," + - s" free: ${Utils.bytesToString(_remainingMem)})") + logInfo(log"Added ${MDC(BLOCK_ID, blockId)} in memory on " + + log"${MDC(HOST_PORT, blockManagerId.hostPort)} " + + log"(size: ${MDC(CURRENT_MEMORY_SIZE, Utils.bytesToString(memSize))}, " + + log"free: ${MDC(FREE_MEMORY_SIZE, Utils.bytesToString(_remainingMem))})") } } if (storageLevel.useDisk) { blockStatus = BlockStatus(storageLevel, memSize = 0, diskSize = diskSize) _blocks.put(blockId, blockStatus) if (blockExists) { - logInfo(s"Updated $blockId on disk on ${blockManagerId.hostPort}" + - s" (current size: ${Utils.bytesToString(diskSize)}," + - s" original size: ${Utils.bytesToString(originalDiskSize)})") + logInfo(log"Updated ${MDC(BLOCK_ID, blockId)} on disk on " + + log"${MDC(HOST_PORT, blockManagerId.hostPort)} " + + log"(current size: ${MDC(CURRENT_DISK_SIZE, Utils.bytesToString(diskSize))}," + + log" original size: ${MDC(ORIGINAL_DISK_SIZE, Utils.bytesToString(originalDiskSize))})") } else { - logInfo(s"Added $blockId on disk on ${blockManagerId.hostPort}" + - s" (size: ${Utils.bytesToString(diskSize)})") + logInfo(log"Added ${MDC(BLOCK_ID, blockId)} on disk on " + + log"${MDC(HOST_PORT, blockManagerId.hostPort)} (size: " + + log"${MDC(CURRENT_DISK_SIZE, Utils.bytesToString(diskSize))})") } } @@ -1092,13 +1098,15 @@ private[spark] class BlockManagerInfo( blockStatus.remove(blockId) } if (originalLevel.useMemory) { - logInfo(s"Removed $blockId on ${blockManagerId.hostPort} in memory" + - s" (size: ${Utils.bytesToString(originalMemSize)}," + - s" free: ${Utils.bytesToString(_remainingMem)})") + logInfo(log"Removed ${MDC(BLOCK_ID, blockId)} on " + + log"${MDC(HOST_PORT, blockManagerId.hostPort)} in memory " + + log"(size: ${MDC(ORIGINAL_MEMORY_SIZE, Utils.bytesToString(originalMemSize))}, " + + log"free: ${MDC(FREE_MEMORY_SIZE, Utils.bytesToString(_remainingMem))})") } if (originalLevel.useDisk) { - logInfo(s"Removed $blockId on ${blockManagerId.hostPort} on disk" + - s" (size: ${Utils.bytesToString(originalDiskSize)})") + logInfo(log"Removed ${MDC(BLOCK_ID, blockId)} on " + + log"${MDC(HOST_PORT, blockManagerId.hostPort)} on disk" + + log" (size: ${MDC(ORIGINAL_DISK_SIZE, Utils.bytesToString(originalDiskSize))})") } } } diff --git a/core/src/main/scala/org/apache/spark/storage/PushBasedFetchHelper.scala b/core/src/main/scala/org/apache/spark/storage/PushBasedFetchHelper.scala index 059bb52714106..8a3ca3066961c 100644 --- a/core/src/main/scala/org/apache/spark/storage/PushBasedFetchHelper.scala +++ b/core/src/main/scala/org/apache/spark/storage/PushBasedFetchHelper.scala @@ -342,7 +342,8 @@ private class PushBasedFetchHelper( // Fallback for all the pending fetch requests val pendingShuffleChunks = iterator.removePendingChunks(shuffleChunkId, address) pendingShuffleChunks.foreach { pendingBlockId => - logInfo(s"Falling back immediately for shuffle chunk $pendingBlockId") + logInfo( + log"Falling back immediately for shuffle chunk ${MDC(BLOCK_ID, pendingBlockId)}") shuffleMetrics.incMergedFetchFallbackCount(1) val bitmapOfPendingChunk: RoaringBitmap = chunksMetaMap.remove(pendingBlockId).get chunkBitmap.or(bitmapOfPendingChunk) 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 b89d24f4c6ddc..ff1799d8ff3e1 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -342,8 +342,8 @@ final class ShuffleBlockFetcherIterator( if (isNettyOOMOnShuffle.compareAndSet(false, true)) { // The fetcher can fail remaining blocks in batch for the same error. So we only // log the warning once to avoid flooding the logs. - logInfo(s"Block $blockId has failed $failureTimes times " + - s"due to Netty OOM, will retry") + logInfo(log"Block ${MDC(BLOCK_ID, blockId)} has failed " + + log"${MDC(FAILURES, failureTimes)} times due to Netty OOM, will retry") } remainingBlocks -= blockId deferredBlocks += blockId @@ -448,14 +448,17 @@ final class ShuffleBlockFetcherIterator( s"the number of host-local blocks ${numHostLocalBlocks} " + s"the number of push-merged-local blocks ${pushMergedLocalBlocks.size} " + s"+ the number of remote blocks ${numRemoteBlocks} ") - logInfo(s"Getting $blocksToFetchCurrentIteration " + - s"(${Utils.bytesToString(totalBytes)}) non-empty blocks including " + - s"${localBlocks.size} (${Utils.bytesToString(localBlockBytes)}) local and " + - s"${numHostLocalBlocks} (${Utils.bytesToString(hostLocalBlockBytes)}) " + - s"host-local and ${pushMergedLocalBlocks.size} " + - s"(${Utils.bytesToString(pushMergedLocalBlockBytes)}) " + - s"push-merged-local and $numRemoteBlocks (${Utils.bytesToString(remoteBlockBytes)}) " + - s"remote blocks") + logInfo( + log"Getting ${MDC(NUM_BLOCKS, blocksToFetchCurrentIteration)} " + + log"(${MDC(TOTAL_SIZE, Utils.bytesToString(totalBytes))}) non-empty blocks including " + + log"${MDC(NUM_LOCAL_BLOCKS, localBlocks.size)} " + + log"(${MDC(LOCAL_BLOCKS_SIZE, Utils.bytesToString(localBlockBytes))}) local and " + + log"${MDC(NUM_HOST_LOCAL_BLOCKS, numHostLocalBlocks)} " + + log"(${MDC(HOST_LOCAL_BLOCKS_SIZE, Utils.bytesToString(hostLocalBlockBytes))}) " + + log"host-local and ${MDC(NUM_PUSH_MERGED_LOCAL_BLOCKS, pushMergedLocalBlocks.size)} " + + log"(${MDC(PUSH_MERGED_LOCAL_BLOCKS_SIZE, Utils.bytesToString(pushMergedLocalBlockBytes))})" + + log" push-merged-local and ${MDC(NUM_REMOTE_BLOCKS, numRemoteBlocks)} " + + log"(${MDC(REMOTE_BLOCKS_SIZE, Utils.bytesToString(remoteBlockBytes))}) remote blocks") this.hostLocalBlocks ++= hostLocalBlocksByExecutor.values .flatMap { infos => infos.map(info => (info._1, info._3)) } collectedRemoteRequests @@ -719,8 +722,10 @@ final class ShuffleBlockFetcherIterator( val numDeferredRequest = deferredFetchRequests.values.map(_.size).sum val numFetches = remoteRequests.size - fetchRequests.size - numDeferredRequest - logInfo(s"Started $numFetches remote fetches in ${Utils.getUsedTimeNs(startTimeNs)}" + - (if (numDeferredRequest > 0 ) s", deferred $numDeferredRequest requests" else "")) + logInfo(log"Started ${MDC(COUNT, numFetches)} remote fetches in " + + log"${MDC(DURATION, Utils.getUsedTimeNs(startTimeNs))}" + + (if (numDeferredRequest > 0) log", deferred ${MDC(NUM_REQUESTS, numDeferredRequest)} requests" + else log"")) // Get Local Blocks fetchLocalBlocks(localBlocks) @@ -1141,7 +1146,8 @@ final class ShuffleBlockFetcherIterator( case otherCause => s"Block $blockId is corrupted due to $otherCause" } - logInfo(s"Finished corruption diagnosis in $duration ms. $diagnosisResponse") + logInfo(log"Finished corruption diagnosis in ${MDC(DURATION, duration)} ms. " + + log"${MDC(STATUS, diagnosisResponse)}") diagnosisResponse case shuffleBlockChunk: ShuffleBlockChunkId => // TODO SPARK-36284 Add shuffle checksum support for push-based shuffle @@ -1277,7 +1283,8 @@ final class ShuffleBlockFetcherIterator( originalLocalBlocks, originalHostLocalBlocksByExecutor, originalMergedLocalBlocks) // Add the remote requests into our queue in a random order fetchRequests ++= Utils.randomize(originalRemoteReqs) - logInfo(s"Created ${originalRemoteReqs.size} fallback remote requests for push-merged") + logInfo(log"Created ${MDC(NUM_REQUESTS, originalRemoteReqs.size)} fallback remote requests " + + log"for push-merged") // fetch all the fallback blocks that are local. fetchLocalBlocks(originalLocalBlocks) // Merged local blocks should be empty during fallback diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 1283b9340a455..6746bbd490c42 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -117,7 +117,8 @@ private[spark] class MemoryStore( log"needed to store a block in memory. Please configure Spark with more memory.") } - logInfo("MemoryStore started with capacity %s".format(Utils.bytesToString(maxMemory))) + logInfo(log"MemoryStore started with capacity " + + log"${MDC(MEMORY_SIZE, Utils.bytesToString(maxMemory))}") /** Total storage memory used including unroll memory, in bytes. */ private def memoryUsed: Long = memoryManager.storageMemoryUsed @@ -158,8 +159,9 @@ private[spark] class MemoryStore( entries.synchronized { entries.put(blockId, entry) } - logInfo("Block %s stored as bytes in memory (estimated size %s, free %s)".format( - blockId, Utils.bytesToString(size), Utils.bytesToString(maxMemory - blocksMemoryUsed))) + logInfo(log"Block ${MDC(BLOCK_ID, blockId)} stored as bytes in memory " + + log"(estimated size ${MDC(SIZE, Utils.bytesToString(size))}, " + + log"free ${MDC(MEMORY_SIZE, Utils.bytesToString(maxMemory - blocksMemoryUsed))})") true } else { false @@ -250,7 +252,8 @@ private[spark] class MemoryStore( // SPARK-45025 - if a thread interrupt was received, we log a warning and return used memory // to avoid getting killed by task reaper eventually. if (shouldCheckThreadInterruption && Thread.currentThread().isInterrupted) { - logInfo(s"Failed to unroll block=$blockId since thread interrupt was received") + logInfo( + log"Failed to unroll block=${MDC(BLOCK_ID, blockId)} since thread interrupt was received") Left(unrollMemoryUsedByThisBlock) } else if (keepUnrolling) { // Make sure that we have enough memory to store the block. By this point, it is possible that @@ -279,8 +282,9 @@ private[spark] class MemoryStore( entries.put(blockId, entry) } - logInfo("Block %s stored as values in memory (estimated size %s, free %s)".format(blockId, - Utils.bytesToString(entry.size), Utils.bytesToString(maxMemory - blocksMemoryUsed))) + logInfo(log"Block ${MDC(BLOCK_ID, blockId)} stored as values in memory " + + log"(estimated size ${MDC(MEMORY_SIZE, Utils.bytesToString(entry.size))}, free " + + log"${MDC(FREE_MEMORY_SIZE, Utils.bytesToString(maxMemory - blocksMemoryUsed))})") Right(entry.size) } else { // We ran out of space while unrolling the values for this block @@ -521,8 +525,8 @@ private[spark] class MemoryStore( if (freedMemory >= space) { var lastSuccessfulBlock = -1 try { - logInfo(s"${selectedBlocks.size} blocks selected for dropping " + - s"(${Utils.bytesToString(freedMemory)} bytes)") + logInfo(log"${MDC(NUM_BLOCKS, selectedBlocks.size)} blocks selected for dropping " + + log"(${MDC(MEMORY_SIZE, Utils.bytesToString(freedMemory))} bytes)") selectedBlocks.indices.foreach { idx => val blockId = selectedBlocks(idx) val entry = entries.synchronized { @@ -537,8 +541,9 @@ private[spark] class MemoryStore( } lastSuccessfulBlock = idx } - logInfo(s"After dropping ${selectedBlocks.size} blocks, " + - s"free memory is ${Utils.bytesToString(maxMemory - blocksMemoryUsed)}") + logInfo( + log"After dropping ${MDC(NUM_BLOCKS, selectedBlocks.size)} blocks, free memory is" + + log"${MDC(FREE_MEMORY_SIZE, Utils.bytesToString(maxMemory - blocksMemoryUsed))}") freedMemory } finally { // like BlockManager.doPut, we use a finally rather than a catch to avoid having to deal @@ -553,7 +558,7 @@ private[spark] class MemoryStore( } } else { blockId.foreach { id => - logInfo(s"Will not store $id") + logInfo(log"Will not store ${MDC(BLOCK_ID, id)}") } selectedBlocks.foreach { id => blockInfoManager.unlock(id) @@ -649,11 +654,11 @@ private[spark] class MemoryStore( */ private def logMemoryUsage(): Unit = { logInfo( - s"Memory use = ${Utils.bytesToString(blocksMemoryUsed)} (blocks) + " + - s"${Utils.bytesToString(currentUnrollMemory)} (scratch space shared across " + - s"$numTasksUnrolling tasks(s)) = ${Utils.bytesToString(memoryUsed)}. " + - s"Storage limit = ${Utils.bytesToString(maxMemory)}." - ) + log"Memory use = ${MDC(CURRENT_MEMORY_SIZE, Utils.bytesToString(blocksMemoryUsed))} " + + log"(blocks) + ${MDC(FREE_MEMORY_SIZE, Utils.bytesToString(currentUnrollMemory))} " + + log"(scratch space shared across ${MDC(NUM_TASKS, numTasksUnrolling)} " + + log"tasks(s)) = ${MDC(STORAGE_MEMORY_SIZE, Utils.bytesToString(memoryUsed))}. " + + log"Storage limit = ${MDC(MAX_MEMORY_SIZE, Utils.bytesToString(maxMemory))}.") } /** From 416d7f24fc354e912773ceb160210ad6a0c5fe99 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 24 May 2024 20:53:00 -0700 Subject: [PATCH 16/16] [SPARK-48239][INFRA][FOLLOWUP] install the missing `jq` library ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/46534 . We missed the `jq` library which is needed to create git tags. ### Why are the changes needed? fix bug ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? manual ### Was this patch authored or co-authored using generative AI tooling? no Closes #46743 from cloud-fan/script. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- dev/create-release/release-util.sh | 3 +++ dev/create-release/spark-rm/Dockerfile | 1 + 2 files changed, 4 insertions(+) diff --git a/dev/create-release/release-util.sh b/dev/create-release/release-util.sh index 0394fb49c2fa0..b5edbf40d487d 100755 --- a/dev/create-release/release-util.sh +++ b/dev/create-release/release-util.sh @@ -128,6 +128,9 @@ function get_release_info { RC_COUNT=1 fi + if [ "$GIT_BRANCH" = "master" ]; then + RELEASE_VERSION="$RELEASE_VERSION-preview1" + fi export NEXT_VERSION export RELEASE_VERSION=$(read_config "Release" "$RELEASE_VERSION") diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index adaa4df3f5791..5fdaf58feee2e 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -58,6 +58,7 @@ RUN apt-get update && apt-get install -y \ texinfo \ texlive-latex-extra \ qpdf \ + jq \ r-base \ ruby \ ruby-dev \