From e55420228a32eed61ec6a9e9103731c5fd037909 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 18 Aug 2020 08:18:44 -0600 Subject: [PATCH 01/11] explicitly disable AQE in one test (#567) Signed-off-by: Andy Grove --- .../com/nvidia/spark/rapids/GpuCoalesceBatchesSuite.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesSuite.scala index 07fdab26670..b4573e96361 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesSuite.scala @@ -166,6 +166,10 @@ class GpuCoalesceBatchesSuite extends SparkQueryCompareTestSuite { .set(RapidsConf.TEST_ALLOWED_NONGPU.key, "FileSourceScanExec") .set("spark.rapids.sql.exec.FileSourceScanExec", "false") // force Parquet read onto CPU .set("spark.sql.shuffle.partitions", "1") + // this test isn't valid when AQE is enabled because the FileScan happens as part of + // a query stage that runs on the CPU, wrapped in a CPU Exchange, with a ColumnarToRow + // transition inserted + .set("spark.sql.adaptive.enabled", "false") val dir = Files.createTempDirectory("spark-rapids-test").toFile val path = new File(dir, From 419f05a90cec71bb85141c7d829b701c0fa849df Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 18 Aug 2020 09:59:22 -0600 Subject: [PATCH 02/11] Enable using spark-submit to convert from csv to parquet (#565) Signed-off-by: Andy Grove --- .../rapids/tests/tpcds/TpcdsLikeBench.scala | 9 ++- .../rapids/tests/tpcds/TpcdsLikeSpark.scala | 72 ++++++++++++++----- 2 files changed, 60 insertions(+), 21 deletions(-) diff --git a/integration_tests/src/main/scala/com/nvidia/spark/rapids/tests/tpcds/TpcdsLikeBench.scala b/integration_tests/src/main/scala/com/nvidia/spark/rapids/tests/tpcds/TpcdsLikeBench.scala index 1eff919b1ef..27928cdaf82 100644 --- a/integration_tests/src/main/scala/com/nvidia/spark/rapids/tests/tpcds/TpcdsLikeBench.scala +++ b/integration_tests/src/main/scala/com/nvidia/spark/rapids/tests/tpcds/TpcdsLikeBench.scala @@ -59,14 +59,17 @@ object TpcdsLikeBench extends Logging { } for (i <- 0 until numColdRuns) { - println(s"Cold run $i took ${coldRunElapsed(i)} msec.") + println(s"Cold run $i for query $query took ${coldRunElapsed(i)} msec.") } println(s"Average cold run took ${coldRunElapsed.sum.toDouble/numColdRuns} msec.") for (i <- 0 until numHotRuns) { - println(s"Hot run $i took ${hotRunElapsed(i)} msec.") + println(s"Hot run $i for query $query took ${hotRunElapsed(i)} msec.") } - println(s"Average hot run took ${hotRunElapsed.sum.toDouble/numHotRuns} msec.") + println(s"Query $query: " + + s"best: ${hotRunElapsed.min} msec; " + + s"worst: ${hotRunElapsed.max} msec; " + + s"average: ${hotRunElapsed.sum.toDouble/numHotRuns} msec.") } def main(args: Array[String]): Unit = { diff --git a/integration_tests/src/main/scala/com/nvidia/spark/rapids/tests/tpcds/TpcdsLikeSpark.scala b/integration_tests/src/main/scala/com/nvidia/spark/rapids/tests/tpcds/TpcdsLikeSpark.scala index 9bffec269e1..77ab54a5842 100644 --- a/integration_tests/src/main/scala/com/nvidia/spark/rapids/tests/tpcds/TpcdsLikeSpark.scala +++ b/integration_tests/src/main/scala/com/nvidia/spark/rapids/tests/tpcds/TpcdsLikeSpark.scala @@ -45,26 +45,31 @@ case class Table( private def setupWrite( spark: SparkSession, - inputBase: String): DataFrameWriter[Row] = { + inputBase: String, + writePartitioning: Boolean): DataFrameWriter[Row] = { val tmp = readCSV(spark, inputBase) .write .mode("overwrite") - tmp - // Disabling partitioning on writes. The original databricks code has - // partitioning enabled, but for our data sizes it does not help - // We can possibly add it back in for large scale factors. -// if (partitionColumns.isEmpty) { -// tmp -// } else { -// tmp.partitionBy(partitionColumns: _*) -// } + if (writePartitioning && partitionColumns.nonEmpty) { + tmp.partitionBy(partitionColumns: _*) + } else { + tmp + } } - def csvToParquet(spark: SparkSession, inputBase: String, outputBase: String): Unit = - setupWrite(spark, inputBase).parquet(path(outputBase)) + def csvToParquet( + spark: SparkSession, + inputBase: String, + outputBase: String, + writePartitioning: Boolean): Unit = + setupWrite(spark, inputBase, writePartitioning).parquet(path(outputBase)) - def csvToOrc(spark: SparkSession, inputBase: String, outputBase: String): Unit = - setupWrite(spark, inputBase).orc(path(outputBase)) + def csvToOrc( + spark: SparkSession, + inputBase: String, + outputBase: String, + writePartitioning: Boolean): Unit = + setupWrite(spark, inputBase, writePartitioning).orc(path(outputBase)) } case class Query(name: String, query: String) { @@ -79,12 +84,43 @@ case class Query(name: String, query: String) { */ object TpcdsLikeSpark { - def csvToParquet(spark: SparkSession, baseInput: String, baseOutput: String): Unit = { - tables.foreach(_.csvToParquet(spark, baseInput, baseOutput)) + /** + * Main method allows us to submit using spark-submit to perform conversions from CSV to + * Parquet or Orc. + */ + def main(arg: Array[String]): Unit = { + val baseInput = arg(0) + val baseOutput = arg(1) + val targetFileType = arg(2) + val withPartitioning = if (arg.length > 3) { + arg(3).toBoolean + } else { + false + } + + val spark = SparkSession.builder.appName("TPC-DS Like File Conversion").getOrCreate() + + targetFileType match { + case "parquet" => csvToParquet(spark, baseInput, baseOutput, withPartitioning) + case "orc" => csvToOrc(spark, baseInput, baseOutput, withPartitioning) + } + } - def csvToOrc(spark: SparkSession, baseInput: String, baseOutput: String): Unit = { - tables.foreach(_.csvToOrc(spark, baseInput, baseOutput)) + def csvToParquet( + spark: SparkSession, + baseInput: String, + baseOutput: String, + writePartitioning: Boolean = false): Unit = { + tables.foreach(_.csvToParquet(spark, baseInput, baseOutput, writePartitioning)) + } + + def csvToOrc( + spark: SparkSession, + baseInput: String, + baseOutput: String, + writePartitioning: Boolean = false): Unit = { + tables.foreach(_.csvToOrc(spark, baseInput, baseOutput, writePartitioning)) } def setupAllCSV(spark: SparkSession, basePath: String): Unit = { From 7bfd96d988c22a43c34b4df7d867157d415f8088 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 18 Aug 2020 13:08:42 -0500 Subject: [PATCH 03/11] xfail the Spark 3.1.0 integration tests that fail (#580) * xfail GpuTimeSub, arithmetic ops, and full outer join failures on 3.1.0 Signed-off-by: Thomas Graves * xfail the rest of the 3.1.0 tests and enable 3.1.0 unit tests in the jenkins builds Signed-off-by: Thomas Graves Co-authored-by: Thomas Graves --- .../src/main/python/arithmetic_ops_test.py | 13 ++++++++++++ .../src/main/python/date_time_test.py | 4 ++++ .../src/main/python/join_test.py | 7 ++++++- integration_tests/src/main/python/orc_test.py | 5 ++++- .../src/main/python/qa_nightly_sql.py | 21 ++++++++++--------- jenkins/spark-nightly-build.sh | 3 +-- jenkins/spark-premerge-build.sh | 3 +-- 7 files changed, 40 insertions(+), 16 deletions(-) diff --git a/integration_tests/src/main/python/arithmetic_ops_test.py b/integration_tests/src/main/python/arithmetic_ops_test.py index b31e9c9d675..7100afaa5e2 100644 --- a/integration_tests/src/main/python/arithmetic_ops_test.py +++ b/integration_tests/src/main/python/arithmetic_ops_test.py @@ -18,6 +18,7 @@ from data_gen import * from marks import incompat, approximate_float from pyspark.sql.types import * +from spark_session import with_spark_session import pyspark.sql.functions as f @pytest.mark.parametrize('data_gen', numeric_gens, ids=idfn) @@ -358,24 +359,36 @@ def test_expm1(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr('expm1(a)')) +@pytest.mark.xfail( + condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), + reason='https://issues.apache.org/jira/browse/SPARK-32640') @approximate_float @pytest.mark.parametrize('data_gen', double_gens, ids=idfn) def test_log(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr('log(a)')) +@pytest.mark.xfail( + condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), + reason='https://issues.apache.org/jira/browse/SPARK-32640') @approximate_float @pytest.mark.parametrize('data_gen', double_gens, ids=idfn) def test_log1p(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr('log1p(a)')) +@pytest.mark.xfail( + condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), + reason='https://issues.apache.org/jira/browse/SPARK-32640') @approximate_float @pytest.mark.parametrize('data_gen', double_gens, ids=idfn) def test_log2(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr('log2(a)')) +@pytest.mark.xfail( + condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), + reason='https://issues.apache.org/jira/browse/SPARK-32640') @approximate_float @pytest.mark.parametrize('data_gen', double_gens, ids=idfn) def test_log10(data_gen): diff --git a/integration_tests/src/main/python/date_time_test.py b/integration_tests/src/main/python/date_time_test.py index 5373628c7bb..c5c587021fd 100644 --- a/integration_tests/src/main/python/date_time_test.py +++ b/integration_tests/src/main/python/date_time_test.py @@ -18,11 +18,15 @@ from datetime import date, datetime, timezone from marks import incompat from pyspark.sql.types import * +from spark_session import with_spark_session import pyspark.sql.functions as f # We only support literal intervals for TimeSub vals = [(-584, 1563), (1943, 1101), (2693, 2167), (2729, 0), (44, 1534), (2635, 3319), (1885, -2828), (0, 2463), (932, 2286), (0, 0)] +@pytest.mark.xfail( + condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), + reason='https://issues.apache.org/jira/browse/SPARK-32640') @pytest.mark.parametrize('data_gen', vals, ids=idfn) def test_timesub(data_gen): days, seconds = data_gen diff --git a/integration_tests/src/main/python/join_test.py b/integration_tests/src/main/python/join_test.py index dc29e63fa8a..83a058b401c 100644 --- a/integration_tests/src/main/python/join_test.py +++ b/integration_tests/src/main/python/join_test.py @@ -18,6 +18,7 @@ from conftest import is_databricks_runtime from data_gen import * from marks import ignore_order, allow_non_gpu, incompat +from spark_session import with_spark_session all_gen = [StringGen(), ByteGen(), ShortGen(), IntegerGen(), LongGen(), BooleanGen(), DateGen(), TimestampGen(), @@ -149,7 +150,11 @@ def do_join(spark): ('b', StringGen()), ('c', BooleanGen())] @ignore_order -@pytest.mark.parametrize('join_type', ['Left', 'Right', 'Inner', 'LeftSemi', 'LeftAnti', 'FullOuter', 'Cross'], ids=idfn) +@pytest.mark.parametrize('join_type', ['Left', 'Right', 'Inner', 'LeftSemi', 'LeftAnti', + pytest.param('FullOuter', marks=pytest.mark.xfail( + condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), + reason='https://github.com/NVIDIA/spark-rapids/issues/575')), + 'Cross'], ids=idfn) def test_broadcast_join_mixed(join_type): def do_join(spark): left = gen_df(spark, _mixed_df1_with_nulls, length=500) diff --git a/integration_tests/src/main/python/orc_test.py b/integration_tests/src/main/python/orc_test.py index 9b541410454..571bb4e511f 100644 --- a/integration_tests/src/main/python/orc_test.py +++ b/integration_tests/src/main/python/orc_test.py @@ -19,7 +19,7 @@ from data_gen import * from marks import * from pyspark.sql.types import * -from spark_session import with_cpu_session +from spark_session import with_cpu_session, with_spark_session def read_orc_df(data_path): return lambda spark : spark.read.orc(data_path) @@ -187,6 +187,9 @@ def test_compress_write_round_trip(spark_tmp_path, compress): data_path, conf={'spark.sql.orc.compression.codec': compress}) +@pytest.mark.xfail( + condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), + reason='https://github.com/NVIDIA/spark-rapids/issues/576') def test_input_meta(spark_tmp_path): first_data_path = spark_tmp_path + '/ORC_DATA/key=0' with_cpu_session( diff --git a/integration_tests/src/main/python/qa_nightly_sql.py b/integration_tests/src/main/python/qa_nightly_sql.py index 5c51f77381e..9024e025d5a 100644 --- a/integration_tests/src/main/python/qa_nightly_sql.py +++ b/integration_tests/src/main/python/qa_nightly_sql.py @@ -13,6 +13,7 @@ # limitations under the License. from conftest import is_databricks_runtime +from spark_session import with_spark_session import pytest SELECT_SQL = [ @@ -744,16 +745,16 @@ ("SELECT test_table.strF as strF, test_table1.strF as strF1 from test_table RIGHT JOIN test_table1 ON test_table.strF=test_table1.strF", "test_table.strF, test_table1.strF RIGHT JOIN test_table1 ON test_table.strF=test_table1.strF"), ("SELECT test_table.dateF as dateF, test_table1.dateF as dateF1 from test_table RIGHT JOIN test_table1 ON test_table.dateF=test_table1.dateF", "test_table.dateF, test_table1.dateF RIGHT JOIN test_table1 ON test_table.dateF=test_table1.dateF"), ("SELECT test_table.timestampF as timestampF, test_table1.timestampF as timestampF1 from test_table RIGHT JOIN test_table1 ON test_table.timestampF=test_table1.timestampF", "test_table.timestampF, test_table1.timestampF RIGHT JOIN test_table1 ON test_table.timestampF=test_table1.timestampF"), -("SELECT test_table.byteF as byteF, test_table1.byteF as byteF1 from test_table FULL JOIN test_table1 ON test_table.byteF=test_table1.byteF", "test_table.byteF, test_table1.byteF FULL JOIN test_table1 ON test_table.byteF=test_table1.byteF"), -("SELECT test_table.shortF as shortF, test_table1.shortF as shortF1 from test_table FULL JOIN test_table1 ON test_table.shortF=test_table1.shortF", "test_table.shortF, test_table1.shortF FULL JOIN test_table1 ON test_table.shortF=test_table1.shortF"), -("SELECT test_table.intF as intF, test_table1.intF as intF1 from test_table FULL JOIN test_table1 ON test_table.intF=test_table1.intF", "test_table.intF, test_table1.intF FULL JOIN test_table1 ON test_table.intF=test_table1.intF"), -("SELECT test_table.longF as longF, test_table1.longF as longF1 from test_table FULL JOIN test_table1 ON test_table.longF=test_table1.longF", "test_table.longF, test_table1.longF FULL JOIN test_table1 ON test_table.longF=test_table1.longF"), -("SELECT test_table.floatF as floatF, test_table1.floatF as floatF1 from test_table FULL JOIN test_table1 ON test_table.floatF=test_table1.floatF", "test_table.floatF, test_table1.floatF FULL JOIN test_table1 ON test_table.floatF=test_table1.floatF"), -("SELECT test_table.doubleF as doubleF, test_table1.doubleF as doubleF1 from test_table FULL JOIN test_table1 ON test_table.doubleF=test_table1.doubleF", "test_table.doubleF, test_table1.doubleF FULL JOIN test_table1 ON test_table.doubleF=test_table1.doubleF"), -("SELECT test_table.booleanF as booleanF, test_table1.booleanF as booleanF1 from test_table FULL JOIN test_table1 ON test_table.booleanF=test_table1.booleanF", "test_table.booleanF, test_table1.booleanF FULL JOIN test_table1 ON test_table.booleanF=test_table1.booleanF"), -("SELECT test_table.strF as strF, test_table1.strF as strF1 from test_table FULL JOIN test_table1 ON test_table.strF=test_table1.strF", "test_table.strF, test_table1.strF FULL JOIN test_table1 ON test_table.strF=test_table1.strF"), -("SELECT test_table.dateF as dateF, test_table1.dateF as dateF1 from test_table FULL JOIN test_table1 ON test_table.dateF=test_table1.dateF", "test_table.dateF, test_table1.dateF FULL JOIN test_table1 ON test_table.dateF=test_table1.dateF"), -("SELECT test_table.timestampF as timestampF, test_table1.timestampF as timestampF1 from test_table FULL JOIN test_table1 ON test_table.timestampF=test_table1.timestampF", "test_table.timestampF, test_table1.timestampF FULL JOIN test_table1 ON test_table.timestampF=test_table1.timestampF"), +pytest.param(("SELECT test_table.byteF as byteF, test_table1.byteF as byteF1 from test_table FULL JOIN test_table1 ON test_table.byteF=test_table1.byteF", "test_table.byteF, test_table1.byteF FULL JOIN test_table1 ON test_table.byteF=test_table1.byteF"), marks=pytest.mark.xfail(condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), reason='https://github.com/NVIDIA/spark-rapids/issues/578')), +pytest.param(("SELECT test_table.shortF as shortF, test_table1.shortF as shortF1 from test_table FULL JOIN test_table1 ON test_table.shortF=test_table1.shortF", "test_table.shortF, test_table1.shortF FULL JOIN test_table1 ON test_table.shortF=test_table1.shortF"), marks=pytest.mark.xfail(condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), reason='https://github.com/NVIDIA/spark-rapids/issues/578')), +pytest.param(("SELECT test_table.intF as intF, test_table1.intF as intF1 from test_table FULL JOIN test_table1 ON test_table.intF=test_table1.intF", "test_table.intF, test_table1.intF FULL JOIN test_table1 ON test_table.intF=test_table1.intF"), marks=pytest.mark.xfail(condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), reason='https://github.com/NVIDIA/spark-rapids/issues/578')), +pytest.param(("SELECT test_table.longF as longF, test_table1.longF as longF1 from test_table FULL JOIN test_table1 ON test_table.longF=test_table1.longF", "test_table.longF, test_table1.longF FULL JOIN test_table1 ON test_table.longF=test_table1.longF"), marks=pytest.mark.xfail(condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), reason='https://github.com/NVIDIA/spark-rapids/issues/578')), +pytest.param(("SELECT test_table.floatF as floatF, test_table1.floatF as floatF1 from test_table FULL JOIN test_table1 ON test_table.floatF=test_table1.floatF", "test_table.floatF, test_table1.floatF FULL JOIN test_table1 ON test_table.floatF=test_table1.floatF"), marks=pytest.mark.xfail(condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), reason='https://github.com/NVIDIA/spark-rapids/issues/578')), +pytest.param(("SELECT test_table.doubleF as doubleF, test_table1.doubleF as doubleF1 from test_table FULL JOIN test_table1 ON test_table.doubleF=test_table1.doubleF", "test_table.doubleF, test_table1.doubleF FULL JOIN test_table1 ON test_table.doubleF=test_table1.doubleF"), marks=pytest.mark.xfail(condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), reason='https://github.com/NVIDIA/spark-rapids/issues/578')), +pytest.param(("SELECT test_table.booleanF as booleanF, test_table1.booleanF as booleanF1 from test_table FULL JOIN test_table1 ON test_table.booleanF=test_table1.booleanF", "test_table.booleanF, test_table1.booleanF FULL JOIN test_table1 ON test_table.booleanF=test_table1.booleanF"), marks=pytest.mark.xfail(condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), reason='https://github.com/NVIDIA/spark-rapids/issues/578')), +pytest.param(("SELECT test_table.strF as strF, test_table1.strF as strF1 from test_table FULL JOIN test_table1 ON test_table.strF=test_table1.strF", "test_table.strF, test_table1.strF FULL JOIN test_table1 ON test_table.strF=test_table1.strF"), marks=pytest.mark.xfail(condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), reason='https://github.com/NVIDIA/spark-rapids/issues/578')), +pytest.param(("SELECT test_table.dateF as dateF, test_table1.dateF as dateF1 from test_table FULL JOIN test_table1 ON test_table.dateF=test_table1.dateF", "test_table.dateF, test_table1.dateF FULL JOIN test_table1 ON test_table.dateF=test_table1.dateF"), marks=pytest.mark.xfail(condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), reason='https://github.com/NVIDIA/spark-rapids/issues/578')), +pytest.param(("SELECT test_table.timestampF as timestampF, test_table1.timestampF as timestampF1 from test_table FULL JOIN test_table1 ON test_table.timestampF=test_table1.timestampF", "test_table.timestampF, test_table1.timestampF FULL JOIN test_table1 ON test_table.timestampF=test_table1.timestampF"), marks=pytest.mark.xfail(condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), reason='https://github.com/NVIDIA/spark-rapids/issues/578')) ] SELECT_PRE_ORDER_SQL=[ diff --git a/jenkins/spark-nightly-build.sh b/jenkins/spark-nightly-build.sh index 41a7f38b0d6..7e445913f31 100755 --- a/jenkins/spark-nightly-build.sh +++ b/jenkins/spark-nightly-build.sh @@ -22,8 +22,7 @@ set -ex mvn -U -B -Pinclude-databricks clean deploy $MVN_URM_MIRROR -Dmaven.repo.local=$WORKSPACE/.m2 # Run unit tests against other spark versions mvn -U -B -Pspark301tests test $MVN_URM_MIRROR -Dmaven.repo.local=$WORKSPACE/.m2 -# spark310 unit tests fail - https://github.com/NVIDIA/spark-rapids/issues/382 -#mvn -U -B -Pspark310tests test $MVN_URM_MIRROR -Dmaven.repo.local=$WORKSPACE/.m2 +mvn -U -B -Pspark310tests test $MVN_URM_MIRROR -Dmaven.repo.local=$WORKSPACE/.m2 # Parse cudf and spark files from local mvn repo jenkins/printJarVersion.sh "CUDFVersion" "${WORKSPACE}/.m2/ai/rapids/cudf/${CUDF_VER}" "cudf-${CUDF_VER}" "-${CUDA_CLASSIFIER}.jar" diff --git a/jenkins/spark-premerge-build.sh b/jenkins/spark-premerge-build.sh index 9fa312ce6a1..74558df7db3 100755 --- a/jenkins/spark-premerge-build.sh +++ b/jenkins/spark-premerge-build.sh @@ -40,8 +40,7 @@ tar zxf $SPARK_HOME.tgz -C $ARTF_ROOT && \ mvn -U -B $MVN_URM_MIRROR -Pinclude-databricks clean verify -Dpytest.TEST_TAGS='' # Run the unit tests for other Spark versions but dont run full python integration tests env -u SPARK_HOME mvn -U -B $MVN_URM_MIRROR -Pspark301tests test -Dpytest.TEST_TAGS='' -# spark310 unit tests fail - https://github.com/NVIDIA/spark-rapids/issues/382 -#env -u SPARK_HOME mvn -U -B $MVN_URM_MIRROR -Pspark310tests test -Dpytest.TEST_TAGS='' +env -u SPARK_HOME mvn -U -B $MVN_URM_MIRROR -Pspark310tests test -Dpytest.TEST_TAGS='' # The jacoco coverage should have been collected, but because of how the shade plugin # works and jacoco we need to clean some things up so jacoco will only report for the From a4e86f633a8cd77af61a4753f8f842b700ad1313 Mon Sep 17 00:00:00 2001 From: Niranjan Artal <50492963+nartal1@users.noreply.github.com> Date: Tue, 18 Aug 2020 15:52:00 -0700 Subject: [PATCH 04/11] Fix unit tests when AQE is enabled (#558) * Fix scala tests when AQE is enabled Signed-off-by: Niranjan Artal * fix broadcasthashjoin tests Signed-off-by: Niranjan Artal * fix indentation Signed-off-by: Niranjan Artal * addressed review comments Signed-off-by: Niranjan Artal * addressed review comments Signed-off-by: Niranjan Artal * addressed review comments Signed-off-by: Niranjan Artal --- .../spark/rapids/BroadcastHashJoinSuite.scala | 35 +++++++++------ .../spark/rapids/HashSortOptimizeSuite.scala | 15 +++++-- .../com/nvidia/spark/rapids/TestUtils.scala | 44 +++++++++++++++++++ 3 files changed, 77 insertions(+), 17 deletions(-) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/BroadcastHashJoinSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/BroadcastHashJoinSuite.scala index f343927d055..b68763be678 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/BroadcastHashJoinSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/BroadcastHashJoinSuite.scala @@ -16,6 +16,8 @@ package com.nvidia.spark.rapids +import com.nvidia.spark.rapids.TestUtils.{findOperator, operatorCount} + import org.apache.spark.SparkConf import org.apache.spark.sql.execution.joins.HashJoin import org.apache.spark.sql.functions.broadcast @@ -34,14 +36,15 @@ class BroadcastHashJoinSuite extends SparkQueryCompareTestSuite { val df4 = longsDf(spark) val df5 = df4.join(df3, Seq("longs"), "inner") + // execute the plan so that the final adaptive plan is available when AQE is on + df5.collect() val plan = df5.queryExecution.executedPlan - assert(plan.collect { - case p if ShimLoader.getSparkShims.isGpuBroadcastHashJoin(p) => p - }.size === 1) - assert(plan.collect { - case p if ShimLoader.getSparkShims.isGpuShuffledHashJoin(p) => p - }.size === 1) + val bhjCount = operatorCount(plan, ShimLoader.getSparkShims.isGpuBroadcastHashJoin) + assert(bhjCount.size === 1) + + val shjCount = operatorCount(plan, ShimLoader.getSparkShims.isGpuShuffledHashJoin) + assert(shjCount.size === 1) }, conf) } @@ -52,17 +55,21 @@ class BroadcastHashJoinSuite extends SparkQueryCompareTestSuite { for (name <- Seq("BROADCAST", "BROADCASTJOIN", "MAPJOIN")) { val plan1 = spark.sql(s"SELECT /*+ $name(t) */ * FROM t JOIN u ON t.longs = u.longs") - .queryExecution.executedPlan val plan2 = spark.sql(s"SELECT /*+ $name(u) */ * FROM t JOIN u ON t.longs = u.longs") - .queryExecution.executedPlan - val res1 = plan1.find(ShimLoader.getSparkShims.isGpuBroadcastHashJoin(_)) - val res2 = plan2.find(ShimLoader.getSparkShims.isGpuBroadcastHashJoin(_)) + // execute the plan so that the final adaptive plan is available when AQE is on + plan1.collect() + val finalPlan1 = findOperator(plan1.queryExecution.executedPlan, + ShimLoader.getSparkShims.isGpuBroadcastHashJoin) + assert(ShimLoader.getSparkShims.getBuildSide + (finalPlan1.get.asInstanceOf[HashJoin]).toString == "GpuBuildLeft") - assert(ShimLoader.getSparkShims.getBuildSide(res1.get.asInstanceOf[HashJoin]).toString == - "GpuBuildLeft") - assert(ShimLoader.getSparkShims.getBuildSide(res2.get.asInstanceOf[HashJoin]).toString == - "GpuBuildRight") + // execute the plan so that the final adaptive plan is available when AQE is on + plan2.collect() + val finalPlan2 = findOperator(plan2.queryExecution.executedPlan, + ShimLoader.getSparkShims.isGpuBroadcastHashJoin) + assert(ShimLoader.getSparkShims. + getBuildSide(finalPlan2.get.asInstanceOf[HashJoin]).toString == "GpuBuildRight") } }) } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/HashSortOptimizeSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/HashSortOptimizeSuite.scala index 287a2a1f8e2..dc92c07e825 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/HashSortOptimizeSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/HashSortOptimizeSuite.scala @@ -16,6 +16,7 @@ package com.nvidia.spark.rapids +import com.nvidia.spark.rapids.TestUtils.{findOperator, getFinalPlan} import org.scalatest.FunSuite import org.apache.spark.SparkConf @@ -69,7 +70,10 @@ class HashSortOptimizeSuite extends FunSuite { val df2 = buildDataFrame2(spark) val rdf = df1.join(df2, df1("a") === df2("x")) val plan = rdf.queryExecution.executedPlan - val joinNode = plan.find(ShimLoader.getSparkShims.isGpuBroadcastHashJoin(_)) + // execute the plan so that the final adaptive plan is available when AQE is on + rdf.collect() + + val joinNode = findOperator(plan, ShimLoader.getSparkShims.isGpuBroadcastHashJoin(_)) assert(joinNode.isDefined, "No broadcast join node found") validateOptimizeSort(plan, joinNode.get) }) @@ -82,7 +86,9 @@ class HashSortOptimizeSuite extends FunSuite { val df2 = buildDataFrame2(spark) val rdf = df1.join(df2, df1("a") === df2("x")) val plan = rdf.queryExecution.executedPlan - val joinNode = plan.find(ShimLoader.getSparkShims.isGpuShuffledHashJoin(_)) + // execute the plan so that the final adaptive plan is available when AQE is on + rdf.collect() + val joinNode = findOperator(plan, ShimLoader.getSparkShims.isGpuShuffledHashJoin(_)) assert(joinNode.isDefined, "No broadcast join node found") validateOptimizeSort(plan, joinNode.get) }) @@ -106,7 +112,10 @@ class HashSortOptimizeSuite extends FunSuite { val df2 = buildDataFrame2(spark) val rdf = df1.join(df2, df1("a") === df2("x")).orderBy(df1("a")) val plan = rdf.queryExecution.executedPlan - val numSorts = plan.map { + // Get the final executed plan when AQE is either enabled or disabled. + val finalPlan = getFinalPlan(plan) + + val numSorts = finalPlan.map { case _: SortExec | _: GpuSortExec => 1 case _ => 0 }.sum diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/TestUtils.scala b/tests/src/test/scala/com/nvidia/spark/rapids/TestUtils.scala index 4be02c8ccdd..3fed04b3503 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/TestUtils.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/TestUtils.scala @@ -20,9 +20,12 @@ import java.io.File import ai.rapids.cudf.{ColumnVector, DType, Table} import org.scalatest.Assertions +import scala.collection.mutable.ListBuffer import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, BroadcastQueryStageExec, ShuffleQueryStageExec} import org.apache.spark.sql.rapids.GpuShuffleEnv import org.apache.spark.sql.vectorized.ColumnarBatch @@ -51,6 +54,47 @@ object TestUtils extends Assertions with Arm { } } + /** Recursively check if the predicate matches in the given plan */ + def findOperator(plan: SparkPlan, predicate: SparkPlan => Boolean): Option[SparkPlan] = { + plan match { + case _ if predicate(plan) => Some(plan) + case a: AdaptiveSparkPlanExec => findOperator(a.executedPlan, predicate) + case qs: BroadcastQueryStageExec => findOperator(qs.broadcast, predicate) + case qs: ShuffleQueryStageExec => findOperator(qs.shuffle, predicate) + case other => other.children.flatMap(p => findOperator(p, predicate)).headOption + } + } + + /** Return list of matching predicates present in the plan */ + def operatorCount(plan: SparkPlan, predicate: SparkPlan => Boolean): Seq[SparkPlan] = { + def recurse( + plan: SparkPlan, + predicate: SparkPlan => Boolean, + accum: ListBuffer[SparkPlan]): Seq[SparkPlan] = { + plan match { + case _ if predicate(plan) => + accum += plan + plan.children.flatMap(p => recurse(p, predicate, accum)).headOption + case a: AdaptiveSparkPlanExec => recurse(a.executedPlan, predicate, accum) + case qs: BroadcastQueryStageExec => recurse(qs.broadcast, predicate, accum) + case qs: ShuffleQueryStageExec => recurse(qs.shuffle, predicate, accum) + case other => other.children.flatMap(p => recurse(p, predicate, accum)).headOption + } + accum + } + + recurse(plan, predicate, new ListBuffer[SparkPlan]()) + } + + /** Return final executed plan */ + def getFinalPlan(plan: SparkPlan): SparkPlan = { + plan match { + case a: AdaptiveSparkPlanExec => + a.executedPlan + case _ => plan + } + } + /** Compre the equality of two `ColumnVector` instances */ def compareColumns(expected: ColumnVector, actual: ColumnVector): Unit = { assertResult(expected.getType)(actual.getType) From 123a9c9d99e1016caaac3a7f79d18e01e09a5730 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Wed, 19 Aug 2020 10:29:13 -0500 Subject: [PATCH 05/11] Update buffer store to return compressed batches directly, add compression NVTX ranges (#572) * Update buffer store to return compressed batches directly, add compression NVTX ranges Signed-off-by: Jason Lowe * Update parameter name for clarity Signed-off-by: Jason Lowe --- .../nvidia/spark/rapids/GpuColumnVector.java | 11 ++- .../rapids/GpuCompressedColumnVector.java | 16 +++- .../spark/rapids/GpuCoalesceBatches.scala | 2 +- .../nvidia/spark/rapids/RapidsBuffer.scala | 4 +- .../spark/rapids/RapidsBufferStore.scala | 77 ++++--------------- .../rapids/RapidsDeviceMemoryStore.scala | 7 +- .../spark/rapids/TableCompressionCodec.scala | 59 ++++++++++---- 7 files changed, 88 insertions(+), 88 deletions(-) diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuColumnVector.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuColumnVector.java index 0b27c65e395..fef3de23c3c 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuColumnVector.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuColumnVector.java @@ -395,8 +395,15 @@ public final int numNulls() { public static final long getTotalDeviceMemoryUsed(ColumnarBatch batch) { long sum = 0; - for (int i = 0; i < batch.numCols(); i++) { - sum += ((GpuColumnVector) batch.column(i)).getBase().getDeviceMemorySize(); + if (batch.numCols() > 0) { + if (batch.column(0) instanceof GpuCompressedColumnVector) { + GpuCompressedColumnVector gccv = (GpuCompressedColumnVector) batch.column(0); + sum += gccv.getBuffer().getLength(); + } else { + for (int i = 0; i < batch.numCols(); i++) { + sum += ((GpuColumnVector) batch.column(i)).getBase().getDeviceMemorySize(); + } + } } return sum; } diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuCompressedColumnVector.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuCompressedColumnVector.java index ea7e16e9d56..044a709a279 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuCompressedColumnVector.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuCompressedColumnVector.java @@ -33,9 +33,19 @@ public final class GpuCompressedColumnVector extends GpuColumnVectorBase { private final DeviceMemoryBuffer buffer; private final TableMeta tableMeta; + /** + * Build a columnar batch from a compressed table. + * NOTE: The data remains compressed and cannot be accessed directly from the columnar batch. + */ public static ColumnarBatch from(CompressedTable compressedTable) { - DeviceMemoryBuffer buffer = compressedTable.buffer(); - TableMeta tableMeta = compressedTable.meta(); + return from(compressedTable.buffer(), compressedTable.meta()); + } + + /** + * Build a columnar batch from a compressed data buffer and specified table metadata + * NOTE: The data remains compressed and cannot be accessed directly from the columnar batch. + */ + public static ColumnarBatch from(DeviceMemoryBuffer compressedBuffer, TableMeta tableMeta) { long rows = tableMeta.rowCount(); if (rows != (int) rows) { throw new IllegalStateException("Cannot support a batch larger that MAX INT rows"); @@ -49,7 +59,7 @@ public static ColumnarBatch from(CompressedTable compressedTable) { tableMeta.columnMetas(columnMeta, i); DType dtype = DType.fromNative(columnMeta.dtype()); DataType type = GpuColumnVector.getSparkType(dtype); - DeviceMemoryBuffer slicedBuffer = buffer.slice(0, buffer.getLength()); + DeviceMemoryBuffer slicedBuffer = compressedBuffer.slice(0, compressedBuffer.getLength()); columns[i] = new GpuCompressedColumnVector(type, slicedBuffer, tableMeta); } } catch (Throwable t) { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala index 812c3ece914..52735e2d017 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala @@ -480,7 +480,7 @@ class GpuCoalesceIterator(iter: Iterator[ColumnarBatch], val buffer = cv.getBuffer.slice(0, cv.getBuffer.getLength) decompressor.addBufferToDecompress(buffer, bufferMeta) } - closeOnExcept(decompressor.finish()) { outputBuffers => + withResource(decompressor.finish()) { outputBuffers => outputBuffers.zipWithIndex.foreach { case (outputBuffer, outputIndex) => val cv = compressedVecs(outputIndex) val batchIndex = compressedBatchIndices(outputIndex) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala index 097f12e2621..f69b8459444 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala @@ -76,7 +76,9 @@ trait RapidsBuffer extends AutoCloseable { * successfully acquired the buffer beforehand. * @see [[addReference]] * @note It is the responsibility of the caller to close the batch. - * @note This may be an expensive operation (e.g.: batch may need to be decompressed). + * @note If the buffer is compressed data then the resulting batch will be built using + * `GpuCompressedColumnVector`, and it is the responsibility of the caller to deal + * with decompressing the data if necessary. */ def getColumnarBatch: ColumnarBatch diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala index ac44dbb2828..866c44ebbd7 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala @@ -21,8 +21,7 @@ import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicLong import ai.rapids.cudf.{Cuda, DeviceMemoryBuffer, HostMemoryBuffer, NvtxColor, NvtxRange} -import com.nvidia.spark.rapids.StorageTier.StorageTier -import com.nvidia.spark.rapids.format.{BufferMeta, CodecBufferDescriptor, CodecType, TableMeta} +import com.nvidia.spark.rapids.format.TableMeta import org.apache.spark.internal.Logging import org.apache.spark.sql.vectorized.ColumnarBatch @@ -267,30 +266,24 @@ abstract class RapidsBufferStore( // allocated. Allocations can trigger synchronous spills which can // deadlock if another thread holds the device store lock and is trying // to spill to this store. - var deviceBuffer = DeviceMemoryBuffer.allocate(size) - try { - val buffer = getMemoryBuffer - try { - buffer match { - case h: HostMemoryBuffer => - logDebug(s"copying from host $h to device $deviceBuffer") - deviceBuffer.copyFromHostBuffer(h) - case _ => throw new IllegalStateException( - "must override getColumnarBatch if not providing a host buffer") - } - } finally { - buffer.close() - } - - if (meta.bufferMeta.codecBufferDescrsLength > 0) { - withResource(deviceBuffer) { compressedBuffer => - deviceBuffer = uncompressBuffer(compressedBuffer, meta.bufferMeta) - } + withResource(DeviceMemoryBuffer.allocate(size)) { deviceBuffer => + withResource(getMemoryBuffer) { + case h: HostMemoryBuffer => + logDebug(s"copying from host $h to device $deviceBuffer") + deviceBuffer.copyFromHostBuffer(h) + case _ => throw new IllegalStateException( + "must override getColumnarBatch if not providing a host buffer") } + columnarBatchFromDeviceBuffer(deviceBuffer) + } + } - MetaUtils.getBatchFromMeta(deviceBuffer, meta) - } finally { - deviceBuffer.close() + protected def columnarBatchFromDeviceBuffer(devBuffer: DeviceMemoryBuffer): ColumnarBatch = { + val bufferMeta = meta.bufferMeta() + if (bufferMeta == null || bufferMeta.codecBufferDescrsLength == 0) { + MetaUtils.getBatchFromMeta(devBuffer, meta) + } else { + GpuCompressedColumnVector.from(devBuffer, meta) } } @@ -343,42 +336,6 @@ abstract class RapidsBufferStore( } } - protected def uncompressBuffer( - compressedBuffer: DeviceMemoryBuffer, - meta: BufferMeta): DeviceMemoryBuffer = { - closeOnExcept(DeviceMemoryBuffer.allocate(meta.uncompressedSize)) { uncompressedBuffer => - val cbd = new CodecBufferDescriptor - (0 until meta.codecBufferDescrsLength).foreach { i => - meta.codecBufferDescrs(cbd, i) - if (cbd.codec == CodecType.UNCOMPRESSED) { - uncompressedBuffer.copyFromDeviceBufferAsync( - cbd.uncompressedOffset, - compressedBuffer, - cbd.compressedOffset, - cbd.compressedSize, - Cuda.DEFAULT_STREAM) - } else { - val startTime = System.nanoTime() - val codec = TableCompressionCodec.getCodec(cbd.codec) - codec.decompressBuffer( - uncompressedBuffer, - cbd.uncompressedOffset, - cbd.uncompressedSize, - compressedBuffer, - cbd.compressedOffset, - cbd.compressedSize) - val duration = System.nanoTime() - startTime - val compressedSize = cbd.compressedSize() - val uncompressedSize = cbd.uncompressedSize - logDebug(s"Decompressed buffer with ${codec.name} in ${duration / 1000} us," + - s"rate=${compressedSize.toFloat / duration} GB/s " + - s"from $compressedSize to $uncompressedSize") - } - } - uncompressedBuffer - } - } - override def toString: String = s"$name buffer size=$size" } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStore.scala index 48845f6c5f3..d5fc8c943b4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStore.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStore.scala @@ -140,12 +140,7 @@ class RapidsDeviceMemoryStore( if (table.isDefined) { GpuColumnVector.from(table.get) //REFCOUNT ++ of all columns } else { - val uncompressedBuffer = uncompressBuffer(contigBuffer, meta.bufferMeta) - try { - MetaUtils.getBatchFromMeta(uncompressedBuffer, meta) - } finally { - uncompressedBuffer.close() - } + columnarBatchFromDeviceBuffer(contigBuffer) } } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TableCompressionCodec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TableCompressionCodec.scala index c3215887f5e..67d07b2cc43 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TableCompressionCodec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TableCompressionCodec.scala @@ -18,10 +18,12 @@ package com.nvidia.spark.rapids import scala.collection.mutable.ArrayBuffer -import ai.rapids.cudf.{ContiguousTable, DeviceMemoryBuffer} +import ai.rapids.cudf.{ContiguousTable, DeviceMemoryBuffer, NvtxColor, NvtxRange} import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.format.{BufferMeta, CodecType, TableMeta} +import org.apache.spark.internal.Logging + /** * Compressed table descriptor * @note the buffer may be significantly oversized for the amount of compressed data @@ -122,7 +124,8 @@ object TableCompressionCodec { * temporary and output memory above this limit is allowed but will * be compressed individually. */ -abstract class BatchedTableCompressor(maxBatchMemorySize: Long) extends AutoCloseable with Arm { +abstract class BatchedTableCompressor(maxBatchMemorySize: Long) extends AutoCloseable with Arm + with Logging { // The tables that need to be compressed in the next batch private[this] val tables = new ArrayBuffer[ContiguousTable] @@ -220,23 +223,37 @@ abstract class BatchedTableCompressor(maxBatchMemorySize: Long) extends AutoClos private def compressBatch(): Unit = if (tables.nonEmpty) { require(oversizedOutBuffers.length == tables.length) require(tempBuffers.length == tables.length) - val metas = compress(oversizedOutBuffers.toArray, tables.toArray, tempBuffers.toArray) + val startTime = System.nanoTime() + val metas = withResource(new NvtxRange("batch ompress", NvtxColor.ORANGE)) { _ => + compress(oversizedOutBuffers.toArray, tables.toArray, tempBuffers.toArray) + } require(metas.length == tables.length) + val inputSize = tables.map(_.getBuffer.getLength).sum + var outputSize: Long = 0 + // copy the output data into correctly-sized buffers - metas.zipWithIndex.foreach { case (meta, i) => - val oversizedBuffer = oversizedOutBuffers(i) - val compressedSize = meta.bufferMeta.size - val buffer = if (oversizedBuffer.getLength > compressedSize) { - oversizedBuffer.sliceWithCopy(0, compressedSize) - } else { - // use this buffer as-is, don't close it at the end of this method - oversizedOutBuffers(i) = null - oversizedBuffer + withResource(new NvtxRange("copy compressed buffers", NvtxColor.PURPLE)) { _ => + metas.zipWithIndex.foreach { case (meta, i) => + val oversizedBuffer = oversizedOutBuffers(i) + val compressedSize = meta.bufferMeta.size + outputSize += compressedSize + val buffer = if (oversizedBuffer.getLength > compressedSize) { + oversizedBuffer.sliceWithCopy(0, compressedSize) + } else { + // use this buffer as-is, don't close it at the end of this method + oversizedOutBuffers(i) = null + oversizedBuffer + } + results += CompressedTable(compressedSize, meta, buffer) } - results += CompressedTable(compressedSize, meta, buffer) } + val duration = (System.nanoTime() - startTime).toFloat + logDebug(s"Compressed ${tables.length} tables from $inputSize to $outputSize " + + s"in ${duration / 1000000} msec rate=${inputSize / duration} GB/s " + + s"ratio=${outputSize.toFloat/inputSize}") + // free all the inputs to this batch tables.safeClose() tables.clear() @@ -277,7 +294,8 @@ abstract class BatchedTableCompressor(maxBatchMemorySize: Long) extends AutoClos * temporary and output memory above this limit is allowed but will * be compressed individually. */ -abstract class BatchedBufferDecompressor(maxBatchMemorySize: Long) extends AutoCloseable with Arm { +abstract class BatchedBufferDecompressor(maxBatchMemorySize: Long) extends AutoCloseable with Arm + with Logging { // The buffers of compressed data that will be decompressed in the next batch private[this] val inputBuffers = new ArrayBuffer[DeviceMemoryBuffer] @@ -343,16 +361,27 @@ abstract class BatchedBufferDecompressor(maxBatchMemorySize: Long) extends AutoC inputBuffers.safeClose() tempBuffers.safeClose() outputBuffers.safeClose() + results.safeClose() } protected def decompressBatch(): Unit = { if (inputBuffers.nonEmpty) { require(outputBuffers.length == inputBuffers.length) require(tempBuffers.length == inputBuffers.length) - decompress(outputBuffers.toArray, inputBuffers.toArray, tempBuffers.toArray) + val startTime = System.nanoTime() + withResource(new NvtxRange("batch decompress", NvtxColor.ORANGE)) { _ => + decompress(outputBuffers.toArray, inputBuffers.toArray, tempBuffers.toArray) + } + val duration = (System.nanoTime - startTime).toFloat + val inputSize = inputBuffers.map(_.getLength).sum + val outputSize = outputBuffers.map(_.getLength).sum + results ++= outputBuffers outputBuffers.clear() + logDebug(s"Decompressed ${inputBuffers.length} buffers from $inputSize " + + s"to $outputSize in ${duration / 1000000} msec rate=${outputSize / duration} GB/s") + // free all the inputs to this batch inputBuffers.safeClose() inputBuffers.clear() From 74ddc06165c1a8aa76fc719a24e30ed10445d924 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 19 Aug 2020 11:36:04 -0500 Subject: [PATCH 06/11] xfail the tpch spark 3.1.0 tests that fail (#588) Signed-off-by: Thomas Graves Co-authored-by: Thomas Graves --- integration_tests/src/main/python/tpch_test.py | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/integration_tests/src/main/python/tpch_test.py b/integration_tests/src/main/python/tpch_test.py index 1399eec9b14..fe39073bc1f 100644 --- a/integration_tests/src/main/python/tpch_test.py +++ b/integration_tests/src/main/python/tpch_test.py @@ -16,6 +16,7 @@ from asserts import assert_gpu_and_cpu_are_equal_collect from marks import approximate_float, incompat, ignore_order, allow_non_gpu +from spark_session import with_spark_session _base_conf = {'spark.rapids.sql.variableFloatAgg.enabled': 'true', 'spark.rapids.sql.hasNans': 'false'} @@ -117,18 +118,27 @@ def test_tpch_q15(tpch, conf): assert_gpu_and_cpu_are_equal_collect( lambda spark : tpch.do_test_query("q15")) +@pytest.mark.xfail( + condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), + reason='https://github.com/NVIDIA/spark-rapids/issues/586') @allow_non_gpu('BroadcastNestedLoopJoinExec', 'Or', 'IsNull', 'EqualTo', 'AttributeReference', 'BroadcastExchangeExec') @pytest.mark.parametrize('conf', [_base_conf, _adaptive_conf]) def test_tpch_q16(tpch, conf): assert_gpu_and_cpu_are_equal_collect( lambda spark : tpch.do_test_query("q16"), conf=conf) +@pytest.mark.xfail( + condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), + reason='https://github.com/NVIDIA/spark-rapids/issues/586') @approximate_float @pytest.mark.parametrize('conf', [_base_conf, _adaptive_conf]) def test_tpch_q17(tpch, conf): assert_gpu_and_cpu_are_equal_collect( lambda spark : tpch.do_test_query("q17"), conf=conf) +@pytest.mark.xfail( + condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), + reason='https://github.com/NVIDIA/spark-rapids/issues/586') @incompat @approximate_float @allow_non_gpu('TakeOrderedAndProjectExec', 'SortOrder', 'AttributeReference') @@ -143,6 +153,9 @@ def test_tpch_q19(tpch, conf): assert_gpu_and_cpu_are_equal_collect( lambda spark : tpch.do_test_query("q19"), conf=conf) +@pytest.mark.xfail( + condition=with_spark_session(lambda spark : not(spark.sparkContext.version < "3.1.0")), + reason='https://github.com/NVIDIA/spark-rapids/issues/586') @pytest.mark.parametrize('conf', [_base_conf, _adaptive_conf]) def test_tpch_q20(tpch, conf): assert_gpu_and_cpu_are_equal_collect( From ce1f9b8b6201057a37a1a4add87f1ba0ec15c5bb Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 19 Aug 2020 16:06:40 -0500 Subject: [PATCH 07/11] Move GpuParquetScan/GpuOrcScan into Shim (#590) * Move GpuParquetScan to shim Signed-off-by: Thomas Graves * Move scan overrides into shim Signed-off-by: Thomas Graves * Rename GpuParquetScan object to match Signed-off-by: Thomas Graves * Add tests for v2 datasources Signed-off-by: Thomas Graves * Move OrcScan into shims Signed-off-by: Thomas Graves * Fixes Signed-off-by: Thomas Graves * Fix imports Signed-off-by: Thomas Graves Co-authored-by: Thomas Graves --- integration_tests/src/main/python/csv_test.py | 33 ++++++--- integration_tests/src/main/python/orc_test.py | 36 ++++++---- .../src/main/python/parquet_test.py | 72 ++++++++++++------- .../rapids/shims/spark300/GpuOrcScan.scala | 68 ++++++++++++++++++ .../shims/spark300/GpuParquetScan.scala | 68 ++++++++++++++++++ .../rapids/shims/spark300/Spark300Shims.scala | 48 ++++++++++++- .../rapids/shims/spark310/GpuOrcScan.scala | 68 ++++++++++++++++++ .../shims/spark310/GpuParquetScan.scala | 68 ++++++++++++++++++ .../rapids/shims/spark310/Spark310Shims.scala | 45 ++++++++++++ .../com/nvidia/spark/rapids/GpuOrcScan.scala | 37 ++-------- .../nvidia/spark/rapids/GpuOverrides.scala | 45 ++---------- .../nvidia/spark/rapids/GpuParquetScan.scala | 33 ++------- .../spark/rapids/GpuReadOrcFileFormat.scala | 2 +- .../rapids/GpuReadParquetFileFormat.scala | 2 +- .../com/nvidia/spark/rapids/SparkShims.scala | 3 + 15 files changed, 479 insertions(+), 149 deletions(-) create mode 100644 shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuOrcScan.scala create mode 100644 shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuParquetScan.scala create mode 100644 shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/GpuOrcScan.scala create mode 100644 shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/GpuParquetScan.scala diff --git a/integration_tests/src/main/python/csv_test.py b/integration_tests/src/main/python/csv_test.py index 153c302780b..d74e8bc9aa1 100644 --- a/integration_tests/src/main/python/csv_test.py +++ b/integration_tests/src/main/python/csv_test.py @@ -119,9 +119,12 @@ def read_impl(spark): ('str.csv', _good_str_schema, ',', True) ]) @pytest.mark.parametrize('read_func', [read_csv_df, read_csv_sql]) -def test_basic_read(std_input_path, name, schema, sep, header, read_func): +@pytest.mark.parametrize('v1_enabled_list', ["", "csv"]) +def test_basic_read(std_input_path, name, schema, sep, header, read_func, v1_enabled_list): + updated_conf=_enable_ts_conf + updated_conf['spark.sql.sources.useV1SourceList']=v1_enabled_list assert_gpu_and_cpu_are_equal_collect(read_func(std_input_path + '/' + name, schema, header, sep), - conf=_enable_ts_conf) + conf=updated_conf) csv_supported_gens = [ # Spark does not escape '\r' or '\n' even though it uses it to mark end of record @@ -141,15 +144,18 @@ def test_basic_read(std_input_path, name, schema, sep, header, read_func): @approximate_float @pytest.mark.parametrize('data_gen', csv_supported_gens, ids=idfn) -def test_round_trip(spark_tmp_path, data_gen): +@pytest.mark.parametrize('v1_enabled_list', ["", "csv"]) +def test_round_trip(spark_tmp_path, data_gen, v1_enabled_list): gen = StructGen([('a', data_gen)], nullable=False) data_path = spark_tmp_path + '/CSV_DATA' schema = gen.data_type + updated_conf=_enable_ts_conf + updated_conf['spark.sql.sources.useV1SourceList']=v1_enabled_list with_cpu_session( lambda spark : gen_df(spark, gen).write.csv(data_path)) assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.read.schema(schema).csv(data_path), - conf=_enable_ts_conf) + conf=updated_conf) @allow_non_gpu('FileSourceScanExec') @pytest.mark.parametrize('read_func', [read_csv_df, read_csv_sql]) @@ -174,7 +180,8 @@ def test_csv_fallback(spark_tmp_path, read_func, disable_conf): csv_supported_date_formats = ['yyyy-MM-dd', 'yyyy/MM/dd', 'yyyy-MM', 'yyyy/MM', 'MM-yyyy', 'MM/yyyy', 'MM-dd-yyyy', 'MM/dd/yyyy'] @pytest.mark.parametrize('date_format', csv_supported_date_formats, ids=idfn) -def test_date_formats_round_trip(spark_tmp_path, date_format): +@pytest.mark.parametrize('v1_enabled_list', ["", "csv"]) +def test_date_formats_round_trip(spark_tmp_path, date_format, v1_enabled_list): gen = StructGen([('a', DateGen())], nullable=False) data_path = spark_tmp_path + '/CSV_DATA' schema = gen.data_type @@ -186,7 +193,8 @@ def test_date_formats_round_trip(spark_tmp_path, date_format): lambda spark : spark.read\ .schema(schema)\ .option('dateFormat', date_format)\ - .csv(data_path)) + .csv(data_path), + conf={'spark.sql.sources.useV1SourceList': v1_enabled_list}) csv_supported_ts_parts = ['', # Just the date "'T'HH:mm:ss.SSSXXX", @@ -199,7 +207,8 @@ def test_date_formats_round_trip(spark_tmp_path, date_format): @pytest.mark.parametrize('ts_part', csv_supported_ts_parts) @pytest.mark.parametrize('date_format', csv_supported_date_formats) -def test_ts_formats_round_trip(spark_tmp_path, date_format, ts_part): +@pytest.mark.parametrize('v1_enabled_list', ["", "csv"]) +def test_ts_formats_round_trip(spark_tmp_path, date_format, ts_part, v1_enabled_list): full_format = date_format + ts_part # Once https://github.com/NVIDIA/spark-rapids/issues/122 is fixed the full range should be used data_gen = TimestampGen(start=datetime(1902, 1, 1, tzinfo=timezone.utc), @@ -211,14 +220,17 @@ def test_ts_formats_round_trip(spark_tmp_path, date_format, ts_part): lambda spark : gen_df(spark, gen).write\ .option('timestampFormat', full_format)\ .csv(data_path)) + updated_conf=_enable_ts_conf + updated_conf['spark.sql.sources.useV1SourceList']=v1_enabled_list assert_gpu_and_cpu_are_equal_collect( lambda spark : spark.read\ .schema(schema)\ .option('timestampFormat', full_format)\ .csv(data_path), - conf=_enable_ts_conf) + conf=updated_conf) -def test_input_meta(spark_tmp_path): +@pytest.mark.parametrize('v1_enabled_list', ["", "csv"]) +def test_input_meta(spark_tmp_path, v1_enabled_list): gen = StructGen([('a', long_gen), ('b', long_gen)], nullable=False) first_data_path = spark_tmp_path + '/CSV_DATA/key=0' with_cpu_session( @@ -234,4 +246,5 @@ def test_input_meta(spark_tmp_path): .selectExpr('a', 'input_file_name()', 'input_file_block_start()', - 'input_file_block_length()')) + 'input_file_block_length()'), + conf={'spark.sql.sources.useV1SourceList': v1_enabled_list}) diff --git a/integration_tests/src/main/python/orc_test.py b/integration_tests/src/main/python/orc_test.py index 571bb4e511f..e9e121f6354 100644 --- a/integration_tests/src/main/python/orc_test.py +++ b/integration_tests/src/main/python/orc_test.py @@ -29,9 +29,11 @@ def read_orc_sql(data_path): @pytest.mark.parametrize('name', ['timestamp-date-test.orc']) @pytest.mark.parametrize('read_func', [read_orc_df, read_orc_sql]) -def test_basic_read(std_input_path, name, read_func): +@pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) +def test_basic_read(std_input_path, name, read_func, v1_enabled_list): assert_gpu_and_cpu_are_equal_collect( - read_func(std_input_path + '/' + name)) + read_func(std_input_path + '/' + name), + conf={'spark.sql.sources.useV1SourceList': v1_enabled_list}) orc_gens_list = [[byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), @@ -59,13 +61,15 @@ def test_orc_fallback(spark_tmp_path, read_func, disable_conf): @pytest.mark.parametrize('orc_gens', orc_gens_list, ids=idfn) @pytest.mark.parametrize('read_func', [read_orc_df, read_orc_sql]) -def test_read_round_trip(spark_tmp_path, orc_gens, read_func): +@pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) +def test_read_round_trip(spark_tmp_path, orc_gens, read_func, v1_enabled_list): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gens)] data_path = spark_tmp_path + '/ORC_DATA' with_cpu_session( lambda spark : gen_df(spark, gen_list).write.orc(data_path)) assert_gpu_and_cpu_are_equal_collect( - read_func(data_path)) + read_func(data_path), + conf={'spark.sql.sources.useV1SourceList': v1_enabled_list}) orc_pred_push_gens = [ byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, boolean_gen, @@ -79,7 +83,8 @@ def test_read_round_trip(spark_tmp_path, orc_gens, read_func): @pytest.mark.parametrize('orc_gen', orc_pred_push_gens, ids=idfn) @pytest.mark.parametrize('read_func', [read_orc_df, read_orc_sql]) -def test_pred_push_round_trip(spark_tmp_path, orc_gen, read_func): +@pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) +def test_pred_push_round_trip(spark_tmp_path, orc_gen, read_func, v1_enabled_list): data_path = spark_tmp_path + '/ORC_DATA' gen_list = [('a', RepeatSeqGen(orc_gen, 100)), ('b', orc_gen)] s0 = gen_scalar(orc_gen, force_no_nulls=True) @@ -87,22 +92,26 @@ def test_pred_push_round_trip(spark_tmp_path, orc_gen, read_func): lambda spark : gen_df(spark, gen_list).orderBy('a').write.orc(data_path)) rf = read_func(data_path) assert_gpu_and_cpu_are_equal_collect( - lambda spark: rf(spark).select(f.col('a') >= s0)) + lambda spark: rf(spark).select(f.col('a') >= s0), + conf={'spark.sql.sources.useV1SourceList': v1_enabled_list}) orc_compress_options = ['none', 'uncompressed', 'snappy', 'zlib'] # The following need extra jars 'lzo' # https://github.com/NVIDIA/spark-rapids/issues/143 @pytest.mark.parametrize('compress', orc_compress_options) -def test_compress_read_round_trip(spark_tmp_path, compress): +@pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) +def test_compress_read_round_trip(spark_tmp_path, compress, v1_enabled_list): data_path = spark_tmp_path + '/ORC_DATA' with_cpu_session( lambda spark : binary_op_df(spark, long_gen).write.orc(data_path), conf={'spark.sql.orc.compression.codec': compress}) assert_gpu_and_cpu_are_equal_collect( - lambda spark : spark.read.orc(data_path)) + lambda spark : spark.read.orc(data_path), + conf={'spark.sql.sources.useV1SourceList': v1_enabled_list}) -def test_simple_partitioned_read(spark_tmp_path): +@pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) +def test_simple_partitioned_read(spark_tmp_path, v1_enabled_list): # Once https://github.com/NVIDIA/spark-rapids/issues/131 is fixed # we should go with a more standard set of generators orc_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, @@ -117,10 +126,12 @@ def test_simple_partitioned_read(spark_tmp_path): lambda spark : gen_df(spark, gen_list).write.orc(second_data_path)) data_path = spark_tmp_path + '/ORC_DATA' assert_gpu_and_cpu_are_equal_collect( - lambda spark : spark.read.orc(data_path)) + lambda spark : spark.read.orc(data_path), + conf={'spark.sql.sources.useV1SourceList': v1_enabled_list}) @pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/135') -def test_merge_schema_read(spark_tmp_path): +@pytest.mark.parametrize('v1_enabled_list', ["", "orc"]) +def test_merge_schema_read(spark_tmp_path, v1_enabled_list): # Once https://github.com/NVIDIA/spark-rapids/issues/131 is fixed # we should go with a more standard set of generators orc_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, @@ -136,7 +147,8 @@ def test_merge_schema_read(spark_tmp_path): lambda spark : gen_df(spark, second_gen_list).write.orc(second_data_path)) data_path = spark_tmp_path + '/ORC_DATA' assert_gpu_and_cpu_are_equal_collect( - lambda spark : spark.read.option('mergeSchema', 'true').orc(data_path)) + lambda spark : spark.read.option('mergeSchema', 'true').orc(data_path), + conf={'spark.sql.sources.useV1SourceList': v1_enabled_list}) orc_write_gens_list = [ [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, diff --git a/integration_tests/src/main/python/parquet_test.py b/integration_tests/src/main/python/parquet_test.py index bac8cd11d29..a80208ce9e8 100644 --- a/integration_tests/src/main/python/parquet_test.py +++ b/integration_tests/src/main/python/parquet_test.py @@ -20,7 +20,7 @@ from data_gen import * from marks import * from pyspark.sql.types import * -from spark_session import with_cpu_session +from spark_session import with_cpu_session, with_gpu_session def read_parquet_df(data_path): return lambda spark : spark.read.parquet(data_path) @@ -35,14 +35,15 @@ def read_parquet_sql(data_path): @pytest.mark.parametrize('parquet_gens', parquet_gens_list, ids=idfn) @pytest.mark.parametrize('read_func', [read_parquet_df, read_parquet_sql]) -def test_read_round_trip(spark_tmp_path, parquet_gens, read_func): +@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) +def test_read_round_trip(spark_tmp_path, parquet_gens, read_func, v1_enabled_list): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] data_path = spark_tmp_path + '/PARQUET_DATA' with_cpu_session( lambda spark : gen_df(spark, gen_list).write.parquet(data_path), conf={'spark.sql.legacy.parquet.datetimeRebaseModeInWrite': 'CORRECTED'}) - assert_gpu_and_cpu_are_equal_collect( - read_func(data_path)) + assert_gpu_and_cpu_are_equal_collect(read_func(data_path), + conf={'spark.sql.sources.useV1SourceList': v1_enabled_list}) @allow_non_gpu('FileSourceScanExec') @pytest.mark.parametrize('read_func', [read_parquet_df, read_parquet_sql]) @@ -67,13 +68,15 @@ def test_parquet_fallback(spark_tmp_path, read_func, disable_conf): # https://github.com/NVIDIA/spark-rapids/issues/143 @pytest.mark.parametrize('compress', parquet_compress_options) -def test_compress_read_round_trip(spark_tmp_path, compress): +@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) +def test_compress_read_round_trip(spark_tmp_path, compress, v1_enabled_list): data_path = spark_tmp_path + '/PARQUET_DATA' with_cpu_session( lambda spark : binary_op_df(spark, long_gen).write.parquet(data_path), conf={'spark.sql.parquet.compression.codec': compress}) assert_gpu_and_cpu_are_equal_collect( - lambda spark : spark.read.parquet(data_path)) + lambda spark : spark.read.parquet(data_path), + conf={'spark.sql.sources.useV1SourceList': v1_enabled_list}) parquet_pred_push_gens = [ byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, boolean_gen, @@ -84,8 +87,9 @@ def test_compress_read_round_trip(spark_tmp_path, compress): @pytest.mark.parametrize('parquet_gen', parquet_pred_push_gens, ids=idfn) @pytest.mark.parametrize('read_func', [read_parquet_df, read_parquet_sql]) -def test_pred_push_round_trip(spark_tmp_path, parquet_gen, read_func): - data_path = spark_tmp_path + '/ORC_DATA' +@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) +def test_pred_push_round_trip(spark_tmp_path, parquet_gen, read_func, v1_enabled_list): + data_path = spark_tmp_path + '/PARQUET_DATA' gen_list = [('a', RepeatSeqGen(parquet_gen, 100)), ('b', parquet_gen)] s0 = gen_scalar(parquet_gen, force_no_nulls=True) with_cpu_session( @@ -93,13 +97,15 @@ def test_pred_push_round_trip(spark_tmp_path, parquet_gen, read_func): conf={'spark.sql.legacy.parquet.datetimeRebaseModeInWrite': 'CORRECTED'}) rf = read_func(data_path) assert_gpu_and_cpu_are_equal_collect( - lambda spark: rf(spark).select(f.col('a') >= s0)) + lambda spark: rf(spark).select(f.col('a') >= s0), + conf={'spark.sql.sources.useV1SourceList': v1_enabled_list}) parquet_ts_write_options = ['INT96', 'TIMESTAMP_MICROS', 'TIMESTAMP_MILLIS'] @pytest.mark.parametrize('ts_write', parquet_ts_write_options) @pytest.mark.parametrize('ts_rebase', ['CORRECTED', 'LEGACY']) -def test_ts_read_round_trip(spark_tmp_path, ts_write, ts_rebase): +@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) +def test_ts_read_round_trip(spark_tmp_path, ts_write, ts_rebase, v1_enabled_list): # Once https://github.com/NVIDIA/spark-rapids/issues/132 is fixed replace this with # timestamp_gen gen = TimestampGen(start=datetime(1900, 1, 1, tzinfo=timezone.utc)) @@ -109,7 +115,8 @@ def test_ts_read_round_trip(spark_tmp_path, ts_write, ts_rebase): conf={'spark.sql.legacy.parquet.datetimeRebaseModeInWrite': ts_rebase, 'spark.sql.parquet.outputTimestampType': ts_write}) assert_gpu_and_cpu_are_equal_collect( - lambda spark : spark.read.parquet(data_path)) + lambda spark : spark.read.parquet(data_path), + conf={'spark.sql.sources.useV1SourceList': v1_enabled_list}) parquet_gens_legacy_list = [[byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, DateGen(start=date(1590, 1, 1)), @@ -118,16 +125,19 @@ def test_ts_read_round_trip(spark_tmp_path, ts_write, ts_rebase): pytest.param([date_gen], marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/133'))] @pytest.mark.parametrize('parquet_gens', parquet_gens_legacy_list, ids=idfn) -def test_read_round_trip_legacy(spark_tmp_path, parquet_gens): +@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) +def test_read_round_trip_legacy(spark_tmp_path, parquet_gens, v1_enabled_list): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] data_path = spark_tmp_path + '/PARQUET_DATA' with_cpu_session( lambda spark : gen_df(spark, gen_list).write.parquet(data_path), conf={'spark.sql.legacy.parquet.datetimeRebaseModeInWrite': 'LEGACY'}) assert_gpu_and_cpu_are_equal_collect( - lambda spark : spark.read.parquet(data_path)) + lambda spark : spark.read.parquet(data_path), + conf={'spark.sql.sources.useV1SourceList': v1_enabled_list}) -def test_simple_partitioned_read(spark_tmp_path): +@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) +def test_simple_partitioned_read(spark_tmp_path, v1_enabled_list): # Once https://github.com/NVIDIA/spark-rapids/issues/133 and https://github.com/NVIDIA/spark-rapids/issues/132 are fixed # we should go with a more standard set of generators parquet_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, @@ -144,11 +154,13 @@ def test_simple_partitioned_read(spark_tmp_path): conf={'spark.sql.legacy.parquet.datetimeRebaseModeInWrite': 'CORRECTED'}) data_path = spark_tmp_path + '/PARQUET_DATA' assert_gpu_and_cpu_are_equal_collect( - lambda spark : spark.read.parquet(data_path)) + lambda spark : spark.read.parquet(data_path), + conf={'spark.sql.sources.useV1SourceList': v1_enabled_list}) @pytest.mark.xfail(condition=is_databricks_runtime(), reason='https://github.com/NVIDIA/spark-rapids/issues/192') -def test_read_merge_schema(spark_tmp_path): +@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) +def test_read_merge_schema(spark_tmp_path, v1_enabled_list): # Once https://github.com/NVIDIA/spark-rapids/issues/133 and https://github.com/NVIDIA/spark-rapids/issues/132 are fixed # we should go with a more standard set of generators parquet_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, @@ -166,14 +178,16 @@ def test_read_merge_schema(spark_tmp_path): conf={'spark.sql.legacy.parquet.datetimeRebaseModeInWrite': 'CORRECTED'}) data_path = spark_tmp_path + '/PARQUET_DATA' assert_gpu_and_cpu_are_equal_collect( - lambda spark : spark.read.option('mergeSchema', 'true').parquet(data_path)) + lambda spark : spark.read.option('mergeSchema', 'true').parquet(data_path), + conf={'spark.sql.sources.useV1SourceList': v1_enabled_list}) parquet_write_gens_list = [ [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, date_gen, timestamp_gen]] @pytest.mark.parametrize('parquet_gens', parquet_write_gens_list, ids=idfn) -def test_write_round_trip(spark_tmp_path, parquet_gens): +@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) +def test_write_round_trip(spark_tmp_path, parquet_gens, v1_enabled_list): gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gens)] data_path = spark_tmp_path + '/PARQUET_DATA' assert_gpu_and_cpu_writes_are_equal_collect( @@ -181,7 +195,8 @@ def test_write_round_trip(spark_tmp_path, parquet_gens): lambda spark, path: spark.read.parquet(path), data_path, conf={'spark.sql.legacy.parquet.datetimeRebaseModeInWrite': 'CORRECTED', - 'spark.sql.parquet.outputTimestampType': 'TIMESTAMP_MICROS'}) + 'spark.sql.parquet.outputTimestampType': 'TIMESTAMP_MICROS', + 'spark.sql.sources.useV1SourceList': v1_enabled_list}) parquet_part_write_gens = [ byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, @@ -192,7 +207,8 @@ def test_write_round_trip(spark_tmp_path, parquet_gens): # There are race conditions around when individual files are read in for partitioned data @ignore_order @pytest.mark.parametrize('parquet_gen', parquet_part_write_gens, ids=idfn) -def test_part_write_round_trip(spark_tmp_path, parquet_gen): +@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) +def test_part_write_round_trip(spark_tmp_path, parquet_gen, v1_enabled_list): gen_list = [('a', RepeatSeqGen(parquet_gen, 10)), ('b', parquet_gen)] data_path = spark_tmp_path + '/PARQUET_DATA' @@ -201,19 +217,23 @@ def test_part_write_round_trip(spark_tmp_path, parquet_gen): lambda spark, path: spark.read.parquet(path), data_path, conf={'spark.sql.legacy.parquet.datetimeRebaseModeInWrite': 'CORRECTED', - 'spark.sql.parquet.outputTimestampType': 'TIMESTAMP_MICROS'}) + 'spark.sql.parquet.outputTimestampType': 'TIMESTAMP_MICROS', + 'spark.sql.sources.useV1SourceList': v1_enabled_list}) parquet_write_compress_options = ['none', 'uncompressed', 'snappy'] @pytest.mark.parametrize('compress', parquet_write_compress_options) -def test_compress_write_round_trip(spark_tmp_path, compress): +@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) +def test_compress_write_round_trip(spark_tmp_path, compress, v1_enabled_list): data_path = spark_tmp_path + '/PARQUET_DATA' assert_gpu_and_cpu_writes_are_equal_collect( lambda spark, path : binary_op_df(spark, long_gen).coalesce(1).write.parquet(path), lambda spark, path : spark.read.parquet(path), data_path, - conf={'spark.sql.parquet.compression.codec': compress}) + conf={'spark.sql.parquet.compression.codec': compress, + 'spark.sql.sources.useV1SourceList': v1_enabled_list}) -def test_input_meta(spark_tmp_path): +@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) +def test_input_meta(spark_tmp_path, v1_enabled_list): first_data_path = spark_tmp_path + '/PARQUET_DATA/key=0' with_cpu_session( lambda spark : unary_op_df(spark, long_gen).write.parquet(first_data_path)) @@ -227,4 +247,6 @@ def test_input_meta(spark_tmp_path): .selectExpr('a', 'input_file_name()', 'input_file_block_start()', - 'input_file_block_length()')) + 'input_file_block_length()'), + conf={'spark.sql.sources.useV1SourceList': v1_enabled_list}) + diff --git a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuOrcScan.scala b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuOrcScan.scala new file mode 100644 index 00000000000..148a05838d2 --- /dev/null +++ b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuOrcScan.scala @@ -0,0 +1,68 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims.spark300 + +import com.nvidia.spark.rapids.{GpuOrcScanBase, RapidsConf} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.connector.read.PartitionReaderFactory +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.v2.FileScan +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +// FileScan changed in Spark 3.1.0 so need to compile in Shim +case class GpuOrcScan( + sparkSession: SparkSession, + hadoopConf: Configuration, + fileIndex: PartitioningAwareFileIndex, + dataSchema: StructType, + readDataSchema: StructType, + readPartitionSchema: StructType, + options: CaseInsensitiveStringMap, + pushedFilters: Array[Filter], + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression], + rapidsConf: RapidsConf) + extends GpuOrcScanBase(sparkSession, hadoopConf, dataSchema, readDataSchema, + readPartitionSchema, pushedFilters, rapidsConf) with FileScan { + + override def isSplitable(path: Path): Boolean = super.isSplitableBase(path) + + override def createReaderFactory(): PartitionReaderFactory = super.createReaderFactoryBase() + + override def equals(obj: Any): Boolean = obj match { + case o: GpuOrcScan => + super.equals(o) && dataSchema == o.dataSchema && options == o.options && + equivalentFilters(pushedFilters, o.pushedFilters) && rapidsConf == o.rapidsConf + case _ => false + } + + override def hashCode(): Int = getClass.hashCode() + + override def description(): String = { + super.description() + ", PushedFilters: " + seqToString(pushedFilters) + } + + override def withFilters( + partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = + this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) +} diff --git a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuParquetScan.scala b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuParquetScan.scala new file mode 100644 index 00000000000..fe0b0dbb114 --- /dev/null +++ b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuParquetScan.scala @@ -0,0 +1,68 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims.spark300 + +import com.nvidia.spark.rapids.{GpuParquetScanBase, RapidsConf} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.connector.read.PartitionReaderFactory +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.v2.FileScan +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +// FileScan changed in Spark 3.1.0 so need to compile in Shim +case class GpuParquetScan( + sparkSession: SparkSession, + hadoopConf: Configuration, + fileIndex: PartitioningAwareFileIndex, + dataSchema: StructType, + readDataSchema: StructType, + readPartitionSchema: StructType, + pushedFilters: Array[Filter], + options: CaseInsensitiveStringMap, + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression], + rapidsConf: RapidsConf) + extends GpuParquetScanBase(sparkSession, hadoopConf, dataSchema, + readDataSchema, readPartitionSchema, pushedFilters, rapidsConf) with FileScan { + + override def isSplitable(path: Path): Boolean = super.isSplitableBase(path) + + override def createReaderFactory(): PartitionReaderFactory = super.createReaderFactoryBase() + + override def equals(obj: Any): Boolean = obj match { + case p: GpuParquetScan => + super.equals(p) && dataSchema == p.dataSchema && options == p.options && + equivalentFilters(pushedFilters, p.pushedFilters) && rapidsConf == p.rapidsConf + case _ => false + } + + override def hashCode(): Int = getClass.hashCode() + + override def description(): String = { + super.description() + ", PushedFilters: " + seqToString(pushedFilters) + } + + override def withFilters( + partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = + this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) +} diff --git a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/Spark300Shims.scala b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/Spark300Shims.scala index cb11b3652b9..3cc0fbe24ae 100644 --- a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/Spark300Shims.scala +++ b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/Spark300Shims.scala @@ -31,14 +31,18 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning} import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.ShuffleQueryStageExec -import org.apache.spark.sql.execution.datasources.{BucketingUtils, FilePartition, FileScanRDD, HadoopFsRelation, PartitionDirectory, PartitionedFile} +import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, HadoopFsRelation, PartitionDirectory, PartitionedFile} +import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, HashJoin, SortMergeJoinExec} import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec import org.apache.spark.sql.rapids.{GpuFileSourceScanExec, GpuTimeSub, ShuffleManagerShimBase} -import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExecBase, GpuBroadcastMeta, GpuBroadcastNestedLoopJoinExecBase, GpuShuffleExchangeExecBase, GpuShuffleMeta} +import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExecBase, GpuBroadcastNestedLoopJoinExecBase, GpuShuffleExchangeExecBase} import org.apache.spark.sql.rapids.shims.spark300._ import org.apache.spark.sql.types._ import org.apache.spark.storage.{BlockId, BlockManagerId} @@ -216,6 +220,46 @@ class Spark300Shims extends SparkShims { ).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap } + override def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = Seq( + GpuOverrides.scan[ParquetScan]( + "Parquet parsing", + (a, conf, p, r) => new ScanMeta[ParquetScan](a, conf, p, r) { + override def tagSelfForGpu(): Unit = GpuParquetScanBase.tagSupport(this) + + override def convertToGpu(): Scan = + GpuParquetScan(a.sparkSession, + a.hadoopConf, + a.fileIndex, + a.dataSchema, + a.readDataSchema, + a.readPartitionSchema, + a.pushedFilters, + a.options, + a.partitionFilters, + a.dataFilters, + conf) + }), + GpuOverrides.scan[OrcScan]( + "ORC parsing", + (a, conf, p, r) => new ScanMeta[OrcScan](a, conf, p, r) { + override def tagSelfForGpu(): Unit = + GpuOrcScanBase.tagSupport(this) + + override def convertToGpu(): Scan = + GpuOrcScan(a.sparkSession, + a.hadoopConf, + a.fileIndex, + a.dataSchema, + a.readDataSchema, + a.readPartitionSchema, + a.options, + a.pushedFilters, + a.partitionFilters, + a.dataFilters, + conf) + }) + ).map(r => (r.getClassFor.asSubclass(classOf[Scan]), r)).toMap + override def getBuildSide(join: HashJoin): GpuBuildSide = { GpuJoinUtils.getGpuBuildSide(join.buildSide) } diff --git a/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/GpuOrcScan.scala b/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/GpuOrcScan.scala new file mode 100644 index 00000000000..1c65fcb4fee --- /dev/null +++ b/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/GpuOrcScan.scala @@ -0,0 +1,68 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims.spark310 + +import com.nvidia.spark.rapids.{GpuOrcScanBase, RapidsConf} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.connector.read.PartitionReaderFactory +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.v2.FileScan +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +// FileScan changed in Spark 3.1.0 so need to compile in Shim +case class GpuOrcScan( + sparkSession: SparkSession, + hadoopConf: Configuration, + fileIndex: PartitioningAwareFileIndex, + dataSchema: StructType, + readDataSchema: StructType, + readPartitionSchema: StructType, + options: CaseInsensitiveStringMap, + pushedFilters: Array[Filter], + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression], + rapidsConf: RapidsConf) + extends GpuOrcScanBase(sparkSession, hadoopConf, dataSchema, readDataSchema, + readPartitionSchema, pushedFilters, rapidsConf) with FileScan { + + override def isSplitable(path: Path): Boolean = super.isSplitableBase(path) + + override def createReaderFactory(): PartitionReaderFactory = super.createReaderFactoryBase() + + override def equals(obj: Any): Boolean = obj match { + case o: GpuOrcScan => + super.equals(o) && dataSchema == o.dataSchema && options == o.options && + equivalentFilters(pushedFilters, o.pushedFilters) && rapidsConf == o.rapidsConf + case _ => false + } + + override def hashCode(): Int = getClass.hashCode() + + override def description(): String = { + super.description() + ", PushedFilters: " + seqToString(pushedFilters) + } + + override def withFilters( + partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = + this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) +} diff --git a/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/GpuParquetScan.scala b/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/GpuParquetScan.scala new file mode 100644 index 00000000000..aac3b13098a --- /dev/null +++ b/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/GpuParquetScan.scala @@ -0,0 +1,68 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.shims.spark310 + +import com.nvidia.spark.rapids.{GpuParquetScanBase, RapidsConf} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.connector.read.PartitionReaderFactory +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.v2.FileScan +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +// FileScan changed in Spark 3.1.0 so need to compile in Shim +case class GpuParquetScan( + sparkSession: SparkSession, + hadoopConf: Configuration, + fileIndex: PartitioningAwareFileIndex, + dataSchema: StructType, + readDataSchema: StructType, + readPartitionSchema: StructType, + pushedFilters: Array[Filter], + options: CaseInsensitiveStringMap, + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression], + rapidsConf: RapidsConf) + extends GpuParquetScanBase(sparkSession, hadoopConf, dataSchema, + readDataSchema, readPartitionSchema, pushedFilters, rapidsConf) with FileScan { + + override def isSplitable(path: Path): Boolean = super.isSplitableBase(path) + + override def createReaderFactory(): PartitionReaderFactory = super.createReaderFactoryBase() + + override def equals(obj: Any): Boolean = obj match { + case p: GpuParquetScan => + super.equals(p) && dataSchema == p.dataSchema && options == p.options && + equivalentFilters(pushedFilters, p.pushedFilters) && rapidsConf == p.rapidsConf + case _ => false + } + + override def hashCode(): Int = getClass.hashCode() + + override def description(): String = { + super.description() + ", PushedFilters: " + seqToString(pushedFilters) + } + + override def withFilters( + partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = + this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) +} diff --git a/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/Spark310Shims.scala b/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/Spark310Shims.scala index 9840c8f229d..02d3e175eb6 100644 --- a/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/Spark310Shims.scala +++ b/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/Spark310Shims.scala @@ -26,8 +26,12 @@ import org.apache.spark.SparkEnv import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.HadoopFsRelation +import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, HashJoin, SortMergeJoinExec} import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec import org.apache.spark.sql.rapids.{GpuFileSourceScanExec, GpuTimeSub, ShuffleManagerShimBase} @@ -163,6 +167,47 @@ class Spark310Shims extends Spark301Shims { ).map(r => (r.getClassFor.asSubclass(classOf[SparkPlan]), r)).toMap } + override def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = Seq( + GpuOverrides.scan[ParquetScan]( + "Parquet parsing", + (a, conf, p, r) => new ScanMeta[ParquetScan](a, conf, p, r) { + override def tagSelfForGpu(): Unit = GpuParquetScanBase.tagSupport(this) + + override def convertToGpu(): Scan = + GpuParquetScan(a.sparkSession, + a.hadoopConf, + a.fileIndex, + a.dataSchema, + a.readDataSchema, + a.readPartitionSchema, + a.pushedFilters, + a.options, + a.partitionFilters, + a.dataFilters, + conf) + }), + GpuOverrides.scan[OrcScan]( + "ORC parsing", + (a, conf, p, r) => new ScanMeta[OrcScan](a, conf, p, r) { + override def tagSelfForGpu(): Unit = + GpuOrcScanBase.tagSupport(this) + + override def convertToGpu(): Scan = + GpuOrcScan(a.sparkSession, + a.hadoopConf, + a.fileIndex, + a.dataSchema, + a.readDataSchema, + a.readPartitionSchema, + a.options, + a.pushedFilters, + a.partitionFilters, + a.dataFilters, + conf) + }) + ).map(r => (r.getClassFor.asSubclass(classOf[Scan]), r)).toMap + + override def getBuildSide(join: HashJoin): GpuBuildSide = { GpuJoinUtils.getGpuBuildSide(join.buildSide) } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index 55417797bef..4a6fc01310e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -45,40 +45,34 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} import org.apache.spark.sql.execution.QueryExecutionException -import org.apache.spark.sql.execution.datasources.{PartitionedFile, PartitioningAwareFileIndex} +import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.orc.OrcUtils -import org.apache.spark.sql.execution.datasources.v2.{FilePartitionReaderFactory, FileScan} +import org.apache.spark.sql.execution.datasources.v2.FilePartitionReaderFactory import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.OrcFilters import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration -case class GpuOrcScan( +abstract class GpuOrcScanBase( sparkSession: SparkSession, hadoopConf: Configuration, - fileIndex: PartitioningAwareFileIndex, dataSchema: StructType, readDataSchema: StructType, readPartitionSchema: StructType, - options: CaseInsensitiveStringMap, pushedFilters: Array[Filter], - partitionFilters: Seq[Expression], - dataFilters: Seq[Expression], rapidsConf: RapidsConf) - extends FileScan with ScanWithMetrics { + extends ScanWithMetrics { - override def isSplitable(path: Path): Boolean = true + def isSplitableBase(path: Path): Boolean = true - override def createReaderFactory(): PartitionReaderFactory = { + def createReaderFactoryBase(): PartitionReaderFactory = { // Unset any serialized search argument setup by Spark's OrcScanBuilder as // it will be incompatible due to shading and potential ORC classifier mismatch. hadoopConf.unset(OrcConf.KRYO_SARG.getAttribute) @@ -88,26 +82,9 @@ case class GpuOrcScan( GpuOrcPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, dataSchema, readDataSchema, readPartitionSchema, pushedFilters, rapidsConf, metrics) } - - override def equals(obj: Any): Boolean = obj match { - case o: GpuOrcScan => - super.equals(o) && dataSchema == o.dataSchema && options == o.options && - equivalentFilters(pushedFilters, o.pushedFilters) && rapidsConf == o.rapidsConf - case _ => false - } - - override def hashCode(): Int = getClass.hashCode() - - override def description(): String = { - super.description() + ", PushedFilters: " + seqToString(pushedFilters) - } - - override def withFilters( - partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = - this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) } -object GpuOrcScan { +object GpuOrcScanBase { def tagSupport(scanMeta: ScanMeta[OrcScan]): Unit = { val scan = scanMeta.wrapped val schema = StructType(scan.readDataSchema ++ scan.readPartitionSchema) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 102743c5f53..3f97b4d5bfa 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -1504,8 +1504,8 @@ object GpuOverrides { .map(r => r.wrap(scan, conf, parent, r).asInstanceOf[ScanMeta[INPUT]]) .getOrElse(new RuleNotFoundScanMeta(scan, conf, parent)) - val scans : Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = Seq( - scan[CSVScan]( + val commonScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = Seq( + GpuOverrides.scan[CSVScan]( "CSV parsing", (a, conf, p, r) => new ScanMeta[CSVScan](a, conf, p, r) { override def tagSelfForGpu(): Unit = GpuCSVScan.tagSupport(this) @@ -1521,45 +1521,10 @@ object GpuOverrides { a.dataFilters, conf.maxReadBatchSizeRows, conf.maxReadBatchSizeBytes) - }), - scan[ParquetScan]( - "Parquet parsing", - (a, conf, p, r) => new ScanMeta[ParquetScan](a, conf, p, r) { - override def tagSelfForGpu(): Unit = GpuParquetScan.tagSupport(this) - - override def convertToGpu(): Scan = - GpuParquetScan(a.sparkSession, - a.hadoopConf, - a.fileIndex, - a.dataSchema, - a.readDataSchema, - a.readPartitionSchema, - a.pushedFilters, - a.options, - a.partitionFilters, - a.dataFilters, - conf) - }), - scan[OrcScan]( - "ORC parsing", - (a, conf, p, r) => new ScanMeta[OrcScan](a, conf, p, r) { - override def tagSelfForGpu(): Unit = - GpuOrcScan.tagSupport(this) + })).map(r => (r.getClassFor.asSubclass(classOf[Scan]), r)).toMap - override def convertToGpu(): Scan = - GpuOrcScan(a.sparkSession, - a.hadoopConf, - a.fileIndex, - a.dataSchema, - a.readDataSchema, - a.readPartitionSchema, - a.options, - a.pushedFilters, - a.partitionFilters, - a.dataFilters, - conf) - }) - ).map(r => (r.getClassFor.asSubclass(classOf[Scan]), r)).toMap + val scans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = + commonScans ++ ShimLoader.getSparkShims.getScans def wrapPart[INPUT <: Partitioning]( part: INPUT, diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index 544eb44b5c1..8d5b2fccf04 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -48,7 +48,6 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.execution.datasources.{PartitionedFile, PartitioningAwareFileIndex} @@ -60,52 +59,30 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.{StringType, StructType, TimestampType} -import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration -case class GpuParquetScan( +abstract class GpuParquetScanBase( sparkSession: SparkSession, hadoopConf: Configuration, - fileIndex: PartitioningAwareFileIndex, dataSchema: StructType, readDataSchema: StructType, readPartitionSchema: StructType, pushedFilters: Array[Filter], - options: CaseInsensitiveStringMap, - partitionFilters: Seq[Expression], - dataFilters: Seq[Expression], rapidsConf: RapidsConf) - extends FileScan with ScanWithMetrics { + extends ScanWithMetrics { - override def isSplitable(path: Path): Boolean = true + def isSplitableBase(path: Path): Boolean = true - override def createReaderFactory(): PartitionReaderFactory = { + def createReaderFactoryBase(): PartitionReaderFactory = { val broadcastedConf = sparkSession.sparkContext.broadcast( new SerializableConfiguration(hadoopConf)) GpuParquetPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, dataSchema, readDataSchema, readPartitionSchema, pushedFilters, rapidsConf, metrics) } - - override def equals(obj: Any): Boolean = obj match { - case p: GpuParquetScan => - super.equals(p) && dataSchema == p.dataSchema && options == p.options && - equivalentFilters(pushedFilters, p.pushedFilters) && rapidsConf == p.rapidsConf - case _ => false - } - - override def hashCode(): Int = getClass.hashCode() - - override def description(): String = { - super.description() + ", PushedFilters: " + seqToString(pushedFilters) - } - - override def withFilters( - partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = - this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) } -object GpuParquetScan { +object GpuParquetScanBase { def tagSupport(scanMeta: ScanMeta[ParquetScan]): Unit = { val scan = scanMeta.wrapped val schema = StructType(scan.readDataSchema ++ scan.readPartitionSchema) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala index 3115f08825e..a8450079afc 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala @@ -63,7 +63,7 @@ object GpuReadOrcFileFormat { if (fsse.relation.options.getOrElse("mergeSchema", "false").toBoolean) { meta.willNotWorkOnGpu("mergeSchema and schema evolution is not supported yet") } - GpuOrcScan.tagSupport( + GpuOrcScanBase.tagSupport( fsse.sqlContext.sparkSession, fsse.requiredSchema, meta diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala index d468348776c..0d0ea9a05f6 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala @@ -60,7 +60,7 @@ class GpuReadParquetFileFormat extends ParquetFileFormat with GpuReadFileFormatW object GpuReadParquetFileFormat { def tagSupport(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { val fsse = meta.wrapped - GpuParquetScan.tagSupport( + GpuParquetScanBase.tagSupport( fsse.sqlContext.sparkSession, fsse.requiredSchema, meta diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala index fe347eccc65..c95dc764f53 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning} import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.adaptive.ShuffleQueryStageExec import org.apache.spark.sql.execution.datasources.{FilePartition, HadoopFsRelation, PartitionDirectory, PartitionedFile} @@ -65,6 +66,8 @@ trait SparkShims { def getBuildSide(join: BroadcastNestedLoopJoinExec): GpuBuildSide def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] def getExecs: Map[Class[_ <: SparkPlan], ExecRule[_ <: SparkPlan]] + def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] + def getScalaUDFAsExpression( function: AnyRef, dataType: DataType, From df00904433514930a2dbbdd8ae3ec9043dc3dc3d Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 20 Aug 2020 15:02:27 -0500 Subject: [PATCH 08/11] Filter nulls from joins where possible to improve performance. (#594) * Filter nulls from joins where possible to improve performance. Signed-off-by: Robert (Bobby) Evans * Addressed review comments Signed-off-by: Robert (Bobby) Evans * Updated patch for other shims --- .../src/main/python/tpcds_test.py | 16 ++++- .../spark300/GpuBroadcastHashJoinExec.scala | 13 ++-- .../rapids/shims/spark300/GpuHashJoin.scala | 69 ++++++++++++++++++- .../spark300/GpuShuffledHashJoinExec.scala | 26 +++---- .../spark300db/GpuBroadcastHashJoinExec.scala | 13 ++-- .../rapids/shims/spark300db/GpuHashJoin.scala | 64 ++++++++++++++++- .../spark300db/GpuShuffledHashJoinExec.scala | 26 +++---- .../spark301/GpuBroadcastHashJoinExec.scala | 13 ++-- .../rapids/shims/spark310/GpuHashJoin.scala | 64 ++++++++++++++++- .../spark310/GpuShuffledHashJoinExec.scala | 26 +++---- .../spark/rapids/basicPhysicalOperators.scala | 34 ++++----- 11 files changed, 292 insertions(+), 72 deletions(-) diff --git a/integration_tests/src/main/python/tpcds_test.py b/integration_tests/src/main/python/tpcds_test.py index cc91b42bf5d..5b8780af375 100644 --- a/integration_tests/src/main/python/tpcds_test.py +++ b/integration_tests/src/main/python/tpcds_test.py @@ -23,8 +23,8 @@ 'q30', 'q31', 'q32', 'q33', 'q34', 'q35', 'q36', 'q37', 'q38', 'q39a', 'q39b', 'q40', 'q41', 'q42', 'q43', 'q44', 'q45', 'q46', 'q47', 'q48', 'q49', 'q50', 'q51', 'q52', 'q53', 'q54', 'q55', 'q56', 'q57', 'q58', 'q59', - 'q60', 'q61', 'q62', 'q63', 'q64', 'q65', 'q66', 'q67', 'q68', 'q69', - 'q70', 'q71', 'q72', 'q73', 'q74', 'q75', 'q76', 'q77', 'q78', 'q79', + 'q60', 'q61', 'q62', 'q63', 'q64', 'q65', 'q66', 'q68', 'q69', + 'q71', 'q72', 'q73', 'q74', 'q75', 'q76', 'q77', 'q78', 'q79', 'q80', 'q81', 'q82', 'q83', 'q84', 'q85', 'q86', 'q87', 'q88', 'q89', 'q90', 'q91', 'q92', 'q93', 'q94', 'q95', 'q96', 'q97', 'q98', 'q99', 'ss_max', 'ss_maxb'] @@ -35,5 +35,17 @@ @allow_non_gpu(any=True) @pytest.mark.parametrize('query', queries) def test_tpcds(tpcds, query): + assert_gpu_and_cpu_are_equal_collect( + lambda spark : tpcds.do_test_query(query), + conf={'spark.rapids.sql.variableFloatAgg.enabled': 'true'}) + +no_var_agg_queries = ['q67', 'q70'] + +@incompat +@ignore_order +@approximate_float +@allow_non_gpu(any=True) +@pytest.mark.parametrize('query', no_var_agg_queries) +def test_tpcds_no_var_agg(tpcds, query): assert_gpu_and_cpu_are_equal_collect( lambda spark : tpcds.do_test_query(query)) diff --git a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuBroadcastHashJoinExec.scala b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuBroadcastHashJoinExec.scala index 610b408d418..2293d9471a3 100644 --- a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuBroadcastHashJoinExec.scala +++ b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuBroadcastHashJoinExec.scala @@ -137,10 +137,15 @@ case class GpuBroadcastHashJoinExec( val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) lazy val builtTable = { - // TODO clean up intermediate results... - val keys = GpuProjectExec.project(broadcastRelation.value.batch, gpuBuildKeys) - val combined = combine(keys, broadcastRelation.value.batch) - val ret = GpuColumnVector.from(combined) + val ret = withResource( + GpuProjectExec.project(broadcastRelation.value.batch, gpuBuildKeys)) { keys => + val combined = GpuHashJoin.incRefCount(combine(keys, broadcastRelation.value.batch)) + val filtered = filterBuiltTableIfNeeded(combined) + withResource(filtered) { filtered => + GpuColumnVector.from(filtered) + } + } + // Don't warn for a leak, because we cannot control when we are done with this (0 until ret.getNumberOfColumns).foreach(ret.getColumn(_).noWarnLeakExpected()) ret diff --git a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuHashJoin.scala b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuHashJoin.scala index b3eb7a39fa4..1b2686b6334 100644 --- a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuHashJoin.scala +++ b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuHashJoin.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, FullOuter, InnerLike, JoinType, LeftAnti, LeftExistence, LeftOuter, LeftSemi, RightOuter} import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, HashJoin} import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.rapids.GpuAnd import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} object GpuHashJoin { @@ -39,6 +40,11 @@ object GpuHashJoin { } case _ => meta.willNotWorkOnGpu(s"$joinType currently is not supported") } + + def incRefCount(cb: ColumnarBatch): ColumnarBatch = { + GpuColumnVector.extractBases(cb).foreach(_.incRefCount()) + cb + } } trait GpuHashJoin extends GpuExec with HashJoin { @@ -110,6 +116,63 @@ trait GpuHashJoin extends GpuExec with HashJoin { output.indices.map (v => v + joinLength) } + // Spark adds in rules to filter out nulls for some types of joins, but it does not + // guarantee 100% that all nulls will be filtered out by the time they get to + // this point, but because of https://github.com/rapidsai/cudf/issues/6052 + // we need to filter out the nulls ourselves until it is fixed. + // InnerLike | LeftSemi => + // filter left and right keys + // RightOuter => + // filter left keys + // LeftOuter | LeftAnti => + // filter right keys + + private[this] lazy val shouldFilterBuiltTableForNulls: Boolean = { + val builtAnyNullable = gpuBuildKeys.exists(_.nullable) + (joinType, buildSide) match { + case (_: InnerLike | LeftSemi, _) => builtAnyNullable + case (RightOuter, BuildLeft) => builtAnyNullable + case (LeftOuter | LeftAnti, BuildRight) => builtAnyNullable + case _ => false + } + } + + private[this] lazy val shouldFilterStreamTableForNulls: Boolean = { + val streamedAnyNullable = gpuStreamedKeys.exists(_.nullable) + (joinType, buildSide) match { + case (_: InnerLike | LeftSemi, _) => streamedAnyNullable + case (RightOuter, BuildRight) => streamedAnyNullable + case (LeftOuter | LeftAnti, BuildLeft) => streamedAnyNullable + case _ => false + } + } + + private[this] def mkNullFilterExpr(exprs: Seq[GpuExpression]): GpuExpression = + exprs.zipWithIndex.map { kv => + GpuIsNotNull(GpuBoundReference(kv._2, kv._1.dataType, kv._1.nullable)) + }.reduce(GpuAnd) + + private[this] lazy val builtTableNullFilterExpression: GpuExpression = + mkNullFilterExpr(gpuBuildKeys) + + private[this] lazy val streamedTableNullFilterExpression: GpuExpression = + mkNullFilterExpr(gpuStreamedKeys) + + /** + * Filter the builtBatch if needed. builtBatch will be closed. + * @param builtBatch + * @return + */ + def filterBuiltTableIfNeeded(builtBatch: ColumnarBatch): ColumnarBatch = + if (shouldFilterBuiltTableForNulls) { + GpuFilter(builtBatch, builtTableNullFilterExpression) + } else { + builtBatch + } + + private[this] def filterStreamedTable(streamedBatch:ColumnarBatch): ColumnarBatch = + GpuFilter(streamedBatch, streamedTableNullFilterExpression) + def doJoin(builtTable: Table, stream: Iterator[ColumnarBatch], boundCondition: Option[Expression], @@ -134,7 +197,11 @@ trait GpuHashJoin extends GpuExec with HashJoin { override def hasNext: Boolean = { while (nextCb.isEmpty && (first || stream.hasNext)) { if (stream.hasNext) { - val cb = stream.next() + val cb = if (shouldFilterStreamTableForNulls) { + filterStreamedTable(stream.next()) + } else { + stream.next() + } val startTime = System.nanoTime() nextCb = doJoin(builtTable, cb, boundCondition, joinOutputRows, numOutputRows, numOutputBatches, joinTime, filterTime) diff --git a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuShuffledHashJoinExec.scala b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuShuffledHashJoinExec.scala index 08875868787..6997608f646 100644 --- a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuShuffledHashJoinExec.scala +++ b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuShuffledHashJoinExec.scala @@ -117,20 +117,20 @@ case class GpuShuffledHashJoinExec( streamedPlan.executeColumnar().zipPartitions(buildPlan.executeColumnar()) { (streamIter, buildIter) => { var combinedSize = 0 + val startTime = System.nanoTime() - val buildBatch = - ConcatAndConsumeAll.getSingleBatchWithVerification(buildIter, localBuildOutput) - val keys = GpuProjectExec.project(buildBatch, gpuBuildKeys) - val builtTable = try { - // Combine does not inc any reference counting - val combined = combine(keys, buildBatch) - combinedSize = - GpuColumnVector.extractColumns(combined) - .map(_.getBase.getDeviceMemorySize).sum.toInt - GpuColumnVector.from(combined) - } finally { - keys.close() - buildBatch.close() + val builtTable = withResource(ConcatAndConsumeAll.getSingleBatchWithVerification( + buildIter, localBuildOutput)) { buildBatch: ColumnarBatch => + withResource(GpuProjectExec.project(buildBatch, gpuBuildKeys)) { keys => + val combined = GpuHashJoin.incRefCount(combine(keys, buildBatch)) + val filtered = filterBuiltTableIfNeeded(combined) + combinedSize = + GpuColumnVector.extractColumns(filtered) + .map(_.getBase.getDeviceMemorySize).sum.toInt + withResource(filtered) { filtered => + GpuColumnVector.from(filtered) + } + } } val delta = System.nanoTime() - startTime diff --git a/shims/spark300db/src/main/scala/com/nvidia/spark/rapids/shims/spark300db/GpuBroadcastHashJoinExec.scala b/shims/spark300db/src/main/scala/com/nvidia/spark/rapids/shims/spark300db/GpuBroadcastHashJoinExec.scala index a004e0fccb6..a8307fe789c 100644 --- a/shims/spark300db/src/main/scala/com/nvidia/spark/rapids/shims/spark300db/GpuBroadcastHashJoinExec.scala +++ b/shims/spark300db/src/main/scala/com/nvidia/spark/rapids/shims/spark300db/GpuBroadcastHashJoinExec.scala @@ -138,10 +138,15 @@ case class GpuBroadcastHashJoinExec( val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) lazy val builtTable = { - // TODO clean up intermediate results... - val keys = GpuProjectExec.project(broadcastRelation.value.batch, gpuBuildKeys) - val combined = combine(keys, broadcastRelation.value.batch) - val ret = GpuColumnVector.from(combined) + val ret = withResource( + GpuProjectExec.project(broadcastRelation.value.batch, gpuBuildKeys)) { keys => + val combined = GpuHashJoin.incRefCount(combine(keys, broadcastRelation.value.batch)) + val filtered = filterBuiltTableIfNeeded(combined) + withResource(filtered) { filtered => + GpuColumnVector.from(filtered) + } + } + // Don't warn for a leak, because we cannot control when we are done with this (0 until ret.getNumberOfColumns).foreach(ret.getColumn(_).noWarnLeakExpected()) ret diff --git a/shims/spark300db/src/main/scala/com/nvidia/spark/rapids/shims/spark300db/GpuHashJoin.scala b/shims/spark300db/src/main/scala/com/nvidia/spark/rapids/shims/spark300db/GpuHashJoin.scala index b80db78fb28..2e59d034e1a 100644 --- a/shims/spark300db/src/main/scala/com/nvidia/spark/rapids/shims/spark300db/GpuHashJoin.scala +++ b/shims/spark300db/src/main/scala/com/nvidia/spark/rapids/shims/spark300db/GpuHashJoin.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, FullOuter, InnerLike, JoinType, LeftAnti, LeftExistence, LeftOuter, LeftSemi, RightOuter} import org.apache.spark.sql.execution.joins.HashJoin import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.rapids.GpuAnd import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} object GpuHashJoin { @@ -40,6 +41,11 @@ object GpuHashJoin { } case _ => meta.willNotWorkOnGpu(s"$joinType currently is not supported") } + + def incRefCount(cb: ColumnarBatch): ColumnarBatch = { + GpuColumnVector.extractBases(cb).foreach(_.incRefCount()) + cb + } } trait GpuHashJoin extends GpuExec with HashJoin { @@ -111,6 +117,58 @@ trait GpuHashJoin extends GpuExec with HashJoin { output.indices.map (v => v + joinLength) } + // Spark adds in rules to filter out nulls for some types of joins, but it does not + // guarantee 100% that all nulls will be filtered out by the time they get to + // this point, but because of https://github.com/rapidsai/cudf/issues/6052 + // we need to filter out the nulls ourselves until it is fixed. + // InnerLike | LeftSemi => + // filter left and right keys + // RightOuter => + // filter left keys + // LeftOuter | LeftAnti => + // filter right keys + + private[this] lazy val shouldFilterBuiltTableForNulls: Boolean = { + val builtAnyNullable = gpuBuildKeys.exists(_.nullable) + (joinType, buildSide) match { + case (_: InnerLike | LeftSemi, _) => builtAnyNullable + case (RightOuter, BuildLeft) => builtAnyNullable + case (LeftOuter | LeftAnti, BuildRight) => builtAnyNullable + case _ => false + } + } + + private[this] lazy val shouldFilterStreamTableForNulls: Boolean = { + val streamedAnyNullable = gpuStreamedKeys.exists(_.nullable) + (joinType, buildSide) match { + case (_: InnerLike | LeftSemi, _) => streamedAnyNullable + case (RightOuter, BuildRight) => streamedAnyNullable + case (LeftOuter | LeftAnti, BuildLeft) => streamedAnyNullable + case _ => false + } + } + + private[this] def mkNullFilterExpr(exprs: Seq[GpuExpression]): GpuExpression = + exprs.zipWithIndex.map { kv => + GpuIsNotNull(GpuBoundReference(kv._2, kv._1.dataType, kv._1.nullable)) + }.reduce(GpuAnd) + + private[this] lazy val builtTableNullFilterExpression: GpuExpression = + mkNullFilterExpr(gpuBuildKeys) + + private[this] lazy val streamedTableNullFilterExpression: GpuExpression = + mkNullFilterExpr(gpuStreamedKeys) + + def filterBuiltTableIfNeeded(builtBatch: ColumnarBatch): ColumnarBatch = + if (shouldFilterBuiltTableForNulls) { + GpuFilter(builtBatch, builtTableNullFilterExpression) + } else { + builtBatch + } + + private[this] def filterStreamedTable(streamedBatch:ColumnarBatch): ColumnarBatch = + GpuFilter(streamedBatch, streamedTableNullFilterExpression) + def doJoin(builtTable: Table, stream: Iterator[ColumnarBatch], boundCondition: Option[Expression], @@ -135,7 +193,11 @@ trait GpuHashJoin extends GpuExec with HashJoin { override def hasNext: Boolean = { while (nextCb.isEmpty && (first || stream.hasNext)) { if (stream.hasNext) { - val cb = stream.next() + val cb = if (shouldFilterStreamTableForNulls) { + filterStreamedTable(stream.next()) + } else { + stream.next() + } val startTime = System.nanoTime() nextCb = doJoin(builtTable, cb, boundCondition, joinOutputRows, numOutputRows, numOutputBatches, joinTime, filterTime) diff --git a/shims/spark300db/src/main/scala/com/nvidia/spark/rapids/shims/spark300db/GpuShuffledHashJoinExec.scala b/shims/spark300db/src/main/scala/com/nvidia/spark/rapids/shims/spark300db/GpuShuffledHashJoinExec.scala index ad481219fc6..2aa4f83ad13 100644 --- a/shims/spark300db/src/main/scala/com/nvidia/spark/rapids/shims/spark300db/GpuShuffledHashJoinExec.scala +++ b/shims/spark300db/src/main/scala/com/nvidia/spark/rapids/shims/spark300db/GpuShuffledHashJoinExec.scala @@ -118,20 +118,20 @@ case class GpuShuffledHashJoinExec( streamedPlan.executeColumnar().zipPartitions(buildPlan.executeColumnar()) { (streamIter, buildIter) => { var combinedSize = 0 + val startTime = System.nanoTime() - val buildBatch = - ConcatAndConsumeAll.getSingleBatchWithVerification(buildIter, localBuildOutput) - val keys = GpuProjectExec.project(buildBatch, gpuBuildKeys) - val builtTable = try { - // Combine does not inc any reference counting - val combined = combine(keys, buildBatch) - combinedSize = - GpuColumnVector.extractColumns(combined) - .map(_.getBase.getDeviceMemorySize).sum.toInt - GpuColumnVector.from(combined) - } finally { - keys.close() - buildBatch.close() + val builtTable = withResource(ConcatAndConsumeAll.getSingleBatchWithVerification( + buildIter, localBuildOutput)) { buildBatch: ColumnarBatch => + withResource(GpuProjectExec.project(buildBatch, gpuBuildKeys)) { keys => + val combined = GpuHashJoin.incRefCount(combine(keys, buildBatch)) + val filtered = filterBuiltTableIfNeeded(combined) + combinedSize = + GpuColumnVector.extractColumns(filtered) + .map(_.getBase.getDeviceMemorySize).sum.toInt + withResource(filtered) { filtered => + GpuColumnVector.from(filtered) + } + } } val delta = System.nanoTime() - startTime diff --git a/shims/spark301/src/main/scala/com/nvidia/spark/rapids/shims/spark301/GpuBroadcastHashJoinExec.scala b/shims/spark301/src/main/scala/com/nvidia/spark/rapids/shims/spark301/GpuBroadcastHashJoinExec.scala index ef46a37e832..45d5c6e4303 100644 --- a/shims/spark301/src/main/scala/com/nvidia/spark/rapids/shims/spark301/GpuBroadcastHashJoinExec.scala +++ b/shims/spark301/src/main/scala/com/nvidia/spark/rapids/shims/spark301/GpuBroadcastHashJoinExec.scala @@ -143,10 +143,15 @@ case class GpuBroadcastHashJoinExec( val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) lazy val builtTable = { - // TODO clean up intermediate results... - val keys = GpuProjectExec.project(broadcastRelation.value.batch, gpuBuildKeys) - val combined = combine(keys, broadcastRelation.value.batch) - val ret = GpuColumnVector.from(combined) + val ret = withResource( + GpuProjectExec.project(broadcastRelation.value.batch, gpuBuildKeys)) { keys => + val combined = GpuHashJoin.incRefCount(combine(keys, broadcastRelation.value.batch)) + val filtered = filterBuiltTableIfNeeded(combined) + withResource(filtered) { filtered => + GpuColumnVector.from(filtered) + } + } + // Don't warn for a leak, because we cannot control when we are done with this (0 until ret.getNumberOfColumns).foreach(ret.getColumn(_).noWarnLeakExpected()) ret diff --git a/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/GpuHashJoin.scala b/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/GpuHashJoin.scala index 27d10c5c0b3..4ed12d95727 100644 --- a/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/GpuHashJoin.scala +++ b/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/GpuHashJoin.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, FullOuter, InnerLike, JoinType, LeftAnti, LeftExistence, LeftOuter, LeftSemi, RightOuter} import org.apache.spark.sql.execution.joins.HashJoinWithoutCodegen import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.rapids.GpuAnd import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} object GpuHashJoin { @@ -40,6 +41,11 @@ object GpuHashJoin { } case _ => meta.willNotWorkOnGpu(s"$joinType currently is not supported") } + + def incRefCount(cb: ColumnarBatch): ColumnarBatch = { + GpuColumnVector.extractBases(cb).foreach(_.incRefCount()) + cb + } } trait GpuHashJoin extends GpuExec with HashJoinWithoutCodegen { @@ -111,6 +117,58 @@ trait GpuHashJoin extends GpuExec with HashJoinWithoutCodegen { output.indices.map (v => v + joinLength) } + // Spark adds in rules to filter out nulls for some types of joins, but it does not + // guarantee 100% that all nulls will be filtered out by the time they get to + // this point, but because of https://github.com/rapidsai/cudf/issues/6052 + // we need to filter out the nulls ourselves until it is fixed. + // InnerLike | LeftSemi => + // filter left and right keys + // RightOuter => + // filter left keys + // LeftOuter | LeftAnti => + // filter right keys + + private[this] lazy val shouldFilterBuiltTableForNulls: Boolean = { + val builtAnyNullable = gpuBuildKeys.exists(_.nullable) + (joinType, buildSide) match { + case (_: InnerLike | LeftSemi, _) => builtAnyNullable + case (RightOuter, BuildLeft) => builtAnyNullable + case (LeftOuter | LeftAnti, BuildRight) => builtAnyNullable + case _ => false + } + } + + private[this] lazy val shouldFilterStreamTableForNulls: Boolean = { + val streamedAnyNullable = gpuStreamedKeys.exists(_.nullable) + (joinType, buildSide) match { + case (_: InnerLike | LeftSemi, _) => streamedAnyNullable + case (RightOuter, BuildRight) => streamedAnyNullable + case (LeftOuter | LeftAnti, BuildLeft) => streamedAnyNullable + case _ => false + } + } + + private[this] def mkNullFilterExpr(exprs: Seq[GpuExpression]): GpuExpression = + exprs.zipWithIndex.map { kv => + GpuIsNotNull(GpuBoundReference(kv._2, kv._1.dataType, kv._1.nullable)) + }.reduce(GpuAnd) + + private[this] lazy val builtTableNullFilterExpression: GpuExpression = + mkNullFilterExpr(gpuBuildKeys) + + private[this] lazy val streamedTableNullFilterExpression: GpuExpression = + mkNullFilterExpr(gpuStreamedKeys) + + def filterBuiltTableIfNeeded(builtBatch: ColumnarBatch): ColumnarBatch = + if (shouldFilterBuiltTableForNulls) { + GpuFilter(builtBatch, builtTableNullFilterExpression) + } else { + builtBatch + } + + private[this] def filterStreamedTable(streamedBatch:ColumnarBatch): ColumnarBatch = + GpuFilter(streamedBatch, streamedTableNullFilterExpression) + def doJoin(builtTable: Table, stream: Iterator[ColumnarBatch], boundCondition: Option[Expression], @@ -135,7 +193,11 @@ trait GpuHashJoin extends GpuExec with HashJoinWithoutCodegen { override def hasNext: Boolean = { while (nextCb.isEmpty && (first || stream.hasNext)) { if (stream.hasNext) { - val cb = stream.next() + val cb = if (shouldFilterStreamTableForNulls) { + filterStreamedTable(stream.next()) + } else { + stream.next() + } val startTime = System.nanoTime() nextCb = doJoin(builtTable, cb, boundCondition, joinOutputRows, numOutputRows, numOutputBatches, joinTime, filterTime) diff --git a/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/GpuShuffledHashJoinExec.scala b/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/GpuShuffledHashJoinExec.scala index 7e281c698b1..719639fbdfc 100644 --- a/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/GpuShuffledHashJoinExec.scala +++ b/shims/spark310/src/main/scala/com/nvidia/spark/rapids/shims/spark310/GpuShuffledHashJoinExec.scala @@ -118,20 +118,20 @@ case class GpuShuffledHashJoinExec( streamedPlan.executeColumnar().zipPartitions(buildPlan.executeColumnar()) { (streamIter, buildIter) => { var combinedSize = 0 + val startTime = System.nanoTime() - val buildBatch = - ConcatAndConsumeAll.getSingleBatchWithVerification(buildIter, localBuildOutput) - val keys = GpuProjectExec.project(buildBatch, gpuBuildKeys) - val builtTable = try { - // Combine does not inc any reference counting - val combined = combine(keys, buildBatch) - combinedSize = - GpuColumnVector.extractColumns(combined) - .map(_.getBase.getDeviceMemorySize).sum.toInt - GpuColumnVector.from(combined) - } finally { - keys.close() - buildBatch.close() + val builtTable = withResource(ConcatAndConsumeAll.getSingleBatchWithVerification( + buildIter, localBuildOutput)) { buildBatch: ColumnarBatch => + withResource(GpuProjectExec.project(buildBatch, gpuBuildKeys)) { keys => + val combined = GpuHashJoin.incRefCount(combine(keys, buildBatch)) + val filtered = filterBuiltTableIfNeeded(combined) + combinedSize = + GpuColumnVector.extractColumns(filtered) + .map(_.getBase.getDeviceMemorySize).sum.toInt + withResource(filtered) { filtered => + GpuColumnVector.from(filtered) + } + } } val delta = System.nanoTime() - startTime diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala index ae4c9f46614..a783e905939 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala @@ -94,32 +94,34 @@ case class GpuProjectExec(projectList: Seq[Expression], child: SparkPlan) /** * Run a filter on a batch. The batch will be consumed. */ -object GpuFilter { +object GpuFilter extends Arm { def apply( batch: ColumnarBatch, boundCondition: Expression, numOutputRows: SQLMetric, numOutputBatches: SQLMetric, filterTime: SQLMetric): ColumnarBatch = { - val nvtxRange = new NvtxWithMetrics("filter batch", NvtxColor.YELLOW, filterTime) - try { - var filterConditionCv: GpuColumnVector = null - var tbl: cudf.Table = null - var filtered: cudf.Table = null - val filteredBatch = try { - filterConditionCv = boundCondition.columnarEval(batch).asInstanceOf[GpuColumnVector] - tbl = GpuColumnVector.from(batch) - filtered = tbl.filter(filterConditionCv.getBase) - GpuColumnVector.from(filtered) - } finally { - Seq(filtered, tbl, filterConditionCv, batch).safeClose() - } - + withResource(new NvtxWithMetrics("filter batch", NvtxColor.YELLOW, filterTime)) { _ => + val filteredBatch = GpuFilter(batch, boundCondition) numOutputBatches += 1 numOutputRows += filteredBatch.numRows() filteredBatch + } + } + + def apply( + batch: ColumnarBatch, + boundCondition: Expression) : ColumnarBatch = { + var filterConditionCv: GpuColumnVector = null + var tbl: cudf.Table = null + var filtered: cudf.Table = null + try { + filterConditionCv = boundCondition.columnarEval(batch).asInstanceOf[GpuColumnVector] + tbl = GpuColumnVector.from(batch) + filtered = tbl.filter(filterConditionCv.getBase) + GpuColumnVector.from(filtered) } finally { - nvtxRange.close() + Seq(filtered, tbl, filterConditionCv, batch).safeClose() } } } From ef552393079bb546a690e5314a0eae98a82f5716 Mon Sep 17 00:00:00 2001 From: Peixin Li Date: Thu, 20 Aug 2020 16:31:51 +0800 Subject: [PATCH 09/11] changelog generator Signed-off-by: Peixin Li --- .github/workflows/changelog/changelog | 294 ++++++++++++++++++++++++++ CHANGELOG.md | 166 +++++++++++++++ 2 files changed, 460 insertions(+) create mode 100755 .github/workflows/changelog/changelog create mode 100644 CHANGELOG.md diff --git a/.github/workflows/changelog/changelog b/.github/workflows/changelog/changelog new file mode 100755 index 00000000000..4e922c9eccd --- /dev/null +++ b/.github/workflows/changelog/changelog @@ -0,0 +1,294 @@ +#!/usr/bin/env python + +# Copyright (c) 2020, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +"""A simple changelog generator + +NOTE: This is a repo-specific script, so you may not use it in other places. + +e.g. + cd spark-rapids/ + .github/workflows/changelog/changelog --token= --base_refs=branch-0.1,branch-0.2,branch-0.3 +""" +import os +import sys +from argparse import ArgumentParser +from collections import OrderedDict +from datetime import date + +import requests + +parser = ArgumentParser(description="Changelog Generator") +parser.add_argument("--base_refs", help="list of base refs, separated by comma", + default="branch-0.1,branch-0.2,branch-0.3") +parser.add_argument("--token", help="github token, will use GITHUB_TOKEN if empty", default='') +parser.add_argument("--path", help="path for generated changelog file", default='./CHANGELOG.md') +args = parser.parse_args() + +GITHUB_TOKEN = args.token if args.token else os.environ.get('GITHUB_TOKEN') +assert GITHUB_TOKEN, 'env GITHUB_TOKEN should not be empty' + +# Constants +RELEASE = "Release " +PULL_REQUESTS = "pullRequests" +ISSUES = "issues" +# Subtitles +INVALID = 'Invalid' +BUGS_FIXED = 'Bugs Fixed' +PERFORMANCE = 'Performance' +FEATURES = 'Features' +PRS = 'PRs' +# Labels +LABEL_WONTFIX, LABEL_INVALID, LABEL_DUPLICATE = 'wontfix', 'invalid', 'duplicate' +LABEL_BUG = 'bug' +LABEL_PERFORMANCE, LABEL_SHUFFLE = 'performance', 'shuffle' +LABEL_FEATURE, LABEL_SQL = 'feature request', 'SQL' +# Global Vars +changelog = {} # changelog dict +no_project_prs = [] # list of merge pr w/o project + +query_pr = """ +query ($baseRefName: String!, $after: String) { + repository(name: "spark-rapids", owner: "NVIDIA") { + pullRequests(states: [MERGED], baseRefName: $baseRefName, first: 100, after: $after) { + totalCount + nodes { + number + title + headRefName + baseRefName + state + url + labels(first: 10) { + nodes { + name + } + } + projectCards(first: 10) { + nodes { + project { + name + } + column { + name + } + } + } + mergedAt + } + pageInfo { + hasNextPage + endCursor + } + } + } +} +""" + +query_issue = """ +query ($after: String) { + repository(name: "spark-rapids", owner: "NVIDIA") { + issues(states: [CLOSED], labels: ["SQL", "feature request", "performance", "bug", "shuffle"], first: 100, after: $after) { + totalCount + nodes { + number + title + state + url + labels(first: 10) { + nodes { + name + } + } + projectCards(first: 10) { + nodes { + project { + name + } + column { + name + } + } + } + closedAt + } + pageInfo { + hasNextPage + endCursor + } + } + } +} +""" + + +def process_changelog(resource_type: str): + if resource_type == PULL_REQUESTS: + items = process_pr() + time_field = 'mergedAt' + elif resource_type == ISSUES: + items = process_issue() + time_field = 'closedAt' + else: + return + + for item in items: + if len(item['projectCards']['nodes']) == 0: + if resource_type == PULL_REQUESTS: + no_project_prs.append(item) + continue + + project = item['projectCards']['nodes'][0]['project']['name'] + if not release_project(project): + continue + + if project not in changelog: + changelog[project] = { + FEATURES: [], + PERFORMANCE: [], + BUGS_FIXED: [], + PRS: [], + } + + labels = set() + for label in item['labels']['nodes']: + labels.add(label['name']) + category = rules(labels) + if category == INVALID: + continue + if resource_type == PULL_REQUESTS: + category = PRS + + changelog[project][category].append({ + "number": item['number'], + "title": item['title'], + "url": item['url'], + "time": item[time_field], + }) + + +def process_pr(): + pr = [] + for ref in [x.strip() for x in args.base_refs.split(',')]: + pr.extend(fetch(PULL_REQUESTS, {'baseRefName': ref})) + return pr + + +def process_issue(): + return fetch(ISSUES) + + +def fetch(resource_type: str, variables=None): + items = [] + if resource_type == PULL_REQUESTS and variables: + q = query_pr + elif resource_type == ISSUES: + q = query_issue + variables = {} + else: + return items + + has_next = True + while has_next: + res = post(q, variables) + if res.status_code == 200: + d = res.json() + has_next = d['data']['repository'][resource_type]["pageInfo"]["hasNextPage"] + variables['after'] = d['data']['repository'][resource_type]["pageInfo"]["endCursor"] + items.extend(d['data']['repository'][resource_type]['nodes']) + else: + raise Exception("Query failed to run by returning code of {}. {}".format(res.status_code, q)) + return items + + +def post(query: str, variable: dict): + return requests.post('https://api.github.com/graphql', + json={'query': query, 'variables': variable}, + headers={"Authorization": f"token {GITHUB_TOKEN}"}) + + +def release_project(project_name: str): + if project_name.startswith(RELEASE): + return True + return False + + +def rules(labels: set): + if LABEL_WONTFIX in labels or LABEL_INVALID in labels or LABEL_DUPLICATE in labels: + return INVALID + if LABEL_BUG in labels: + return BUGS_FIXED + if LABEL_PERFORMANCE in labels or LABEL_SHUFFLE in labels: + return PERFORMANCE + if LABEL_FEATURE in labels or LABEL_SQL in labels: + return FEATURES + return INVALID + + +def form_changelog(): + sorted_dict = OrderedDict(sorted(changelog.items(), reverse=True)) + subsections = "" + for project_name, issues in sorted_dict.items(): + subsections += f"\n## {project_name}\n" + subsections += form_subsection(issues, FEATURES) + subsections += form_subsection(issues, PERFORMANCE) + subsections += form_subsection(issues, BUGS_FIXED) + subsections += form_subsection(issues, PRS) + markdown = f"""# Change log +Generated on {date.today()} +{subsections} +""" + with open(args.path, "w") as file: + file.write(markdown) + + +def form_subsection(issues: dict, subtitle: str): + if len(issues[subtitle]) == 0: + return '' + subsection = f"\n### {subtitle}\n" + subsection += "|||\n|:---|:---|" + for issue in sorted(issues[subtitle], key=lambda x: x['time'], reverse=True): + subsection += f"\n|[#{issue['number']}]({issue['url']})|{issue['title']}|" + return subsection + + +def print_no_project_pr(): + if len(no_project_prs) != 0: + print("Merged Pull Requests w/o Project:") + for pr in no_project_prs: + print(f"{pr['baseRefName']} #{pr['number']} {pr['title']} {pr['url']}") + + +def main(): + print('Generating changelog ...') + + try: + print('Processing pull requests ...') + process_changelog(PULL_REQUESTS) + print('Processing issues ...') + process_changelog(ISSUES) + # form doc + form_changelog() + except Exception as e: # pylint: disable=broad-except + print(e) + sys.exit(1) + + print('Done.') + # post action + print_no_project_pr() + + +if __name__ == '__main__': + main() diff --git a/CHANGELOG.md b/CHANGELOG.md new file mode 100644 index 00000000000..ca384d939e1 --- /dev/null +++ b/CHANGELOG.md @@ -0,0 +1,166 @@ +# Change log +Generated on 2020-08-21 + +## Release 0.2 + +### Features +||| +|:---|:---| +|[#452](https://github.com/NVIDIA/spark-rapids/issues/452)|[FEA] Update HashSortOptimizerSuite to work with AQE| +|[#454](https://github.com/NVIDIA/spark-rapids/issues/454)|[FEA] Update GpuCoalesceBatchesSuite to work with AQE enabled| +|[#566](https://github.com/NVIDIA/spark-rapids/issues/566)|[FEA] Add support for StringSplit with an array index.| +|[#524](https://github.com/NVIDIA/spark-rapids/issues/524)|[FEA] Add GPU specific metrics to GpuFileSourceScanExec| +|[#494](https://github.com/NVIDIA/spark-rapids/issues/494)|[FEA] Add some AQE-specific tests to the PySpark test suite| +|[#146](https://github.com/NVIDIA/spark-rapids/issues/146)|[FEA] Python tests should support running with Adaptive Query Execution enabled| +|[#488](https://github.com/NVIDIA/spark-rapids/issues/488)|[FEA] Ability to limit total GPU memory used| +|[#70](https://github.com/NVIDIA/spark-rapids/issues/70)|[FEA] Support StringSplit| +|[#403](https://github.com/NVIDIA/spark-rapids/issues/403)|[FEA] Add in support for GetArrayItem| +|[#493](https://github.com/NVIDIA/spark-rapids/issues/493)|[FEA] Implement shuffle optimization when AQE is enabled| +|[#500](https://github.com/NVIDIA/spark-rapids/issues/500)|[FEA] Add maven profiles for testing with AQE on or off| +|[#471](https://github.com/NVIDIA/spark-rapids/issues/471)|[FEA] create a formal process for updateing the github-pages branch| +|[#479](https://github.com/NVIDIA/spark-rapids/issues/479)|[FEA] Please consider to support spark.sql.extensions=com.nvidia.spark.rapids.SQLExecPlugin| +|[#233](https://github.com/NVIDIA/spark-rapids/issues/233)|[FEA] Audit DataWritingCommandExec | +|[#240](https://github.com/NVIDIA/spark-rapids/issues/240)|[FEA] Audit Api validation script follow on - Optimize StringToTypeTag | +|[#388](https://github.com/NVIDIA/spark-rapids/issues/388)|[FEA] Audit WindowExec| +|[#425](https://github.com/NVIDIA/spark-rapids/issues/425)|[FEA] Add tests for configs in BatchScan Readers| +|[#453](https://github.com/NVIDIA/spark-rapids/issues/453)|[FEA] Update HashAggregatesSuite to work with AQE| +|[#184](https://github.com/NVIDIA/spark-rapids/issues/184)|[FEA] Enable NoScalaDoc scalastyle rule| +|[#438](https://github.com/NVIDIA/spark-rapids/issues/438)|[FEA] Enable StringLPad| +|[#232](https://github.com/NVIDIA/spark-rapids/issues/232)|[FEA] Audit SortExec | +|[#236](https://github.com/NVIDIA/spark-rapids/issues/236)|[FEA] Audit ShuffleExchangeExec | +|[#355](https://github.com/NVIDIA/spark-rapids/issues/355)|[FEA] Support Multiple Spark versions in the same jar| +|[#317](https://github.com/NVIDIA/spark-rapids/issues/317)|[FEA] Write test wrapper to run SQL queries via pyspark| +|[#235](https://github.com/NVIDIA/spark-rapids/issues/235)|[FEA] Audit BroadcastExchangeExec| +|[#234](https://github.com/NVIDIA/spark-rapids/issues/234)|[FEA] Audit BatchScanExec| +|[#238](https://github.com/NVIDIA/spark-rapids/issues/238)|[FEA] Audit ShuffledHashJoinExec | +|[#237](https://github.com/NVIDIA/spark-rapids/issues/237)|[FEA] Audit BroadcastHashJoinExec | +|[#316](https://github.com/NVIDIA/spark-rapids/issues/316)|[FEA] Add some basic Dataframe tests for CoalesceExec| +|[#145](https://github.com/NVIDIA/spark-rapids/issues/145)|[FEA] Scala tests should support running with Adaptive Query Execution enabled| +|[#231](https://github.com/NVIDIA/spark-rapids/issues/231)|[FEA] Audit ProjectExec | +|[#229](https://github.com/NVIDIA/spark-rapids/issues/229)|[FEA] Audit FileSourceScanExec | +### Performance +||| +|:---|:---| +|[#15](https://github.com/NVIDIA/spark-rapids/issues/15)|[FEA] Multiple threads shareing the same GPU| +|[#272](https://github.com/NVIDIA/spark-rapids/issues/272)|[DOC] Getting started guide for UCX shuffle| +### Bugs Fixed +||| +|:---|:---| +|[#569](https://github.com/NVIDIA/spark-rapids/issues/569)|[BUG] left_semi_join operation is abnormal and serious time-consuming| +|[#341](https://github.com/NVIDIA/spark-rapids/issues/341)|[BUG] Document compression formats for readers/writers| +|[#362](https://github.com/NVIDIA/spark-rapids/issues/362)|[BUG] Implement getReaderForRange in the RapidsShuffleManager| +|[#528](https://github.com/NVIDIA/spark-rapids/issues/528)|[BUG] HashAggregateSuite "Avg Distinct with filter" no longer valid when testing against Spark 3.1.0| +|[#416](https://github.com/NVIDIA/spark-rapids/issues/416)|[BUG] Fix Spark 3.1.0 integration tests| +|[#556](https://github.com/NVIDIA/spark-rapids/issues/556)|[BUG] NPE when removing shuffle| +|[#553](https://github.com/NVIDIA/spark-rapids/issues/553)|[BUG] GpuColumnVector build warnings from raw type access| +|[#492](https://github.com/NVIDIA/spark-rapids/issues/492)|[BUG] Re-enable AQE integration tests| +|[#275](https://github.com/NVIDIA/spark-rapids/issues/275)|[BUG] TpchLike query 2 fails when AQE is enabled| +|[#508](https://github.com/NVIDIA/spark-rapids/issues/508)|[BUG] GpuUnion publishes metrics on the UI that are all 0| +|[#269](https://github.com/NVIDIA/spark-rapids/issues/269)|Needed to add `--conf spark.driver.extraClassPath=` | +|[#473](https://github.com/NVIDIA/spark-rapids/issues/473)|[BUG] PartMerge:countDistinct:sum fails sporadically| +|[#531](https://github.com/NVIDIA/spark-rapids/issues/531)|[BUG] Temporary RMM workaround needs to be removed| +|[#525](https://github.com/NVIDIA/spark-rapids/issues/525)|[BUG] GpuFilterExec reports incorrect nullability of output in some cases| +|[#382](https://github.com/NVIDIA/spark-rapids/issues/382)|[BUG] Spark3.1 StringFallbackSuite regexp_replace null cpu fall back test fails.| +|[#441](https://github.com/NVIDIA/spark-rapids/issues/441)|[BUG] test_broadcast_nested_loop_join_special_case fails on databricks| +|[#347](https://github.com/NVIDIA/spark-rapids/issues/347)|[BUG] Failed to read Parquet file generated by GPU-enabled Spark.| +|[#433](https://github.com/NVIDIA/spark-rapids/issues/433)|`InSet` operator produces an error for Strings| +|[#144](https://github.com/NVIDIA/spark-rapids/issues/144)|[BUG] spark.sql.legacy.parquet.datetimeRebaseModeInWrite is ignored| +|[#323](https://github.com/NVIDIA/spark-rapids/issues/323)|[BUG] GpuBroadcastNestedLoopJoinExec can fail if there are no columns| +|[#280](https://github.com/NVIDIA/spark-rapids/issues/280)|[BUG] Full Outer Join does not work on nullable keys| +|[#149](https://github.com/NVIDIA/spark-rapids/issues/149)|[BUG] Spark driver fails to load native libs when running on node without CUDA| +### PRs +||| +|:---|:---| +|[#594](https://github.com/NVIDIA/spark-rapids/pull/594)|Filter nulls from joins where possible to improve performance.| +|[#590](https://github.com/NVIDIA/spark-rapids/pull/590)|Move GpuParquetScan/GpuOrcScan into Shim| +|[#572](https://github.com/NVIDIA/spark-rapids/pull/572)|Update buffer store to return compressed batches directly, add compression NVTX ranges| +|[#564](https://github.com/NVIDIA/spark-rapids/pull/564)|Add GPU decode time metric to scans| +|[#562](https://github.com/NVIDIA/spark-rapids/pull/562)|getCatalog can be called from the driver, and can return null| +|[#555](https://github.com/NVIDIA/spark-rapids/pull/555)|Fix build warnings for ColumnViewAccess| +|[#547](https://github.com/NVIDIA/spark-rapids/pull/547)|Add GPU metrics to GpuFileSourceScanExec| +|[#462](https://github.com/NVIDIA/spark-rapids/pull/462)|Implement optimized AQE support so that exchanges run on GPU where possible| +|[#539](https://github.com/NVIDIA/spark-rapids/pull/539)|Update script to audit multiple Spark versions| +|[#543](https://github.com/NVIDIA/spark-rapids/pull/543)|Add metrics to GpuUnion operator| +|[#497](https://github.com/NVIDIA/spark-rapids/pull/497)|Add UDF compiler implementations| +|[#487](https://github.com/NVIDIA/spark-rapids/pull/487)|Add framework for batch compression of shuffle partitions| +|[#537](https://github.com/NVIDIA/spark-rapids/pull/537)|Use fresh SparkSession when capturing to avoid late capture of previous query| +|[#538](https://github.com/NVIDIA/spark-rapids/pull/538)|Revert "Temporary workaround for RMM initial pool size bug (#530)"| +|[#517](https://github.com/NVIDIA/spark-rapids/pull/517)|Add config to limit maximum RMM pool size| +|[#527](https://github.com/NVIDIA/spark-rapids/pull/527)|Add support for split and getArrayIndex| +|[#534](https://github.com/NVIDIA/spark-rapids/pull/534)|Fixes bugs around GpuShuffleEnv initialization| +|[#529](https://github.com/NVIDIA/spark-rapids/pull/529)|[BUG] Degenerate table metas were not getting copied to the heap| +|[#530](https://github.com/NVIDIA/spark-rapids/pull/530)|Temporary workaround for RMM initial pool size bug| +|[#526](https://github.com/NVIDIA/spark-rapids/pull/526)|Fix bug with nullability reporting in GpuFilterExec| +|[#522](https://github.com/NVIDIA/spark-rapids/pull/522)|Use SQLConf instead of SparkConf when looking up SQL configs| +|[#518](https://github.com/NVIDIA/spark-rapids/pull/518)|Fix init order issue in GpuShuffleEnv when RAPIDS shuffle configured| +|[#503](https://github.com/NVIDIA/spark-rapids/pull/503)|Remove unused async buffer spill support| +|[#466](https://github.com/NVIDIA/spark-rapids/pull/466)|Optimize and fix Api validation script| +|[#319](https://github.com/NVIDIA/spark-rapids/pull/319)|Update partitioning logic in ShuffledBatchRDD| +|[#448](https://github.com/NVIDIA/spark-rapids/pull/448)|Preliminary support for keeping broadcast exchanges on GPU when AQE is enabled| +|[#478](https://github.com/NVIDIA/spark-rapids/pull/478)|Fall back to CPU for binary as string in parquet| +|[#477](https://github.com/NVIDIA/spark-rapids/pull/477)|Fix special case joins in broadcast nested loop join| +|[#434](https://github.com/NVIDIA/spark-rapids/pull/434)|Add UDF compiler skeleton| +|[#456](https://github.com/NVIDIA/spark-rapids/pull/456)|Add closeOnExcept to clean up code that closes resources only on exceptions| +|[#426](https://github.com/NVIDIA/spark-rapids/pull/426)|Update BufferMeta to support multiple codec buffers per table| +|[#445](https://github.com/NVIDIA/spark-rapids/pull/445)|Added in StringRPad and StringLPad| +|[#437](https://github.com/NVIDIA/spark-rapids/pull/437)|Fix bug with InSet and Strings| +|[#435](https://github.com/NVIDIA/spark-rapids/pull/435)|Add in checks for Parquet LEGACY date/time rebase| +|[#432](https://github.com/NVIDIA/spark-rapids/pull/432)|Fix batch use-after-close in partitioning, shuffle env init| +|[#406](https://github.com/NVIDIA/spark-rapids/pull/406)|Update cast handling to deal with new libcudf casting limitations| +|[#397](https://github.com/NVIDIA/spark-rapids/pull/397)|Fix small bug in ShuffleBufferCatalog.hasActiveShuffle| +|[#389](https://github.com/NVIDIA/spark-rapids/pull/389)|Added in support for RangeExec| +|[#390](https://github.com/NVIDIA/spark-rapids/pull/390)|Ucx getting started| +|[#387](https://github.com/NVIDIA/spark-rapids/pull/387)|Remove the term whitelist| +|[#376](https://github.com/NVIDIA/spark-rapids/pull/376)|Add our session extension correctly if there are other extensions configured| +|[#374](https://github.com/NVIDIA/spark-rapids/pull/374)|Fix rat issue for notebooks| +|[#364](https://github.com/NVIDIA/spark-rapids/pull/364)|Update Databricks patch for changes to GpuSortMergeJoin| +|[#353](https://github.com/NVIDIA/spark-rapids/pull/353)|Fix broadcast nested loop join for the no column case| +|[#342](https://github.com/NVIDIA/spark-rapids/pull/342)|Implement build-side rules for shuffle hash join| +|[#349](https://github.com/NVIDIA/spark-rapids/pull/349)|Updated join code to treat null equality properly| +|[#331](https://github.com/NVIDIA/spark-rapids/pull/331)|Adds iterator and client unit tests, and prepares for more fetch failure handling| +|[#338](https://github.com/NVIDIA/spark-rapids/pull/338)|Move GpuPartitioning to a separate file| +|[#324](https://github.com/NVIDIA/spark-rapids/pull/324)|Add in basic support for GpuCartesianProductExec| +|[#321](https://github.com/NVIDIA/spark-rapids/pull/321)|update databricks patch for GpuBroadcastNestedLoopJoinExec| +|[#258](https://github.com/NVIDIA/spark-rapids/pull/258)|Enable running TPCH tests with AQE enabled| +|[#309](https://github.com/NVIDIA/spark-rapids/pull/309)|Update databricks patch to latest changes| +|[#296](https://github.com/NVIDIA/spark-rapids/pull/296)|Added in basic support for broadcast nested loop join| +|[#289](https://github.com/NVIDIA/spark-rapids/pull/289)|Clean up GpuCollectLimitMeta and add in metrics| +|[#287](https://github.com/NVIDIA/spark-rapids/pull/287)|Add in support for right join and fix issues build right| +|[#281](https://github.com/NVIDIA/spark-rapids/pull/281)|Add in support for Full Outer Join on non-null keys| +|[#263](https://github.com/NVIDIA/spark-rapids/pull/263)|More time ops| +|[#256](https://github.com/NVIDIA/spark-rapids/pull/256)|Remove special cases for contains, startsWith, and endWith| +|[#253](https://github.com/NVIDIA/spark-rapids/pull/253)|Remove GpuAttributeReference and GpuSortOrder| +|[#162](https://github.com/NVIDIA/spark-rapids/pull/162)|Integration tests for corner cases in window functions.| +## Release 0.1 + +### Features +||| +|:---|:---| +|[#74](https://github.com/NVIDIA/spark-rapids/issues/74)|[FEA] Support ToUnixTimestamp| +|[#21](https://github.com/NVIDIA/spark-rapids/issues/21)|[FEA] NormalizeNansAndZeros| +|[#105](https://github.com/NVIDIA/spark-rapids/issues/105)|[FEA] integration tests for equi-joins| +### Bugs Fixed +||| +|:---|:---| +|[#116](https://github.com/NVIDIA/spark-rapids/issues/116)|[BUG] calling replace with a NULL throws an exception| +|[#168](https://github.com/NVIDIA/spark-rapids/issues/168)|[BUG] GpuUnitTests Date tests leak column vectors| +|[#209](https://github.com/NVIDIA/spark-rapids/issues/209)|[BUG] Developers section in pom need to be updated| +|[#204](https://github.com/NVIDIA/spark-rapids/issues/204)|[BUG] Code coverage docs are out of date| +|[#154](https://github.com/NVIDIA/spark-rapids/issues/154)|[BUG] Incorrect output from partial-only averages with nulls| +|[#61](https://github.com/NVIDIA/spark-rapids/issues/61)|[BUG] Cannot disable Parquet, ORC, CSV reading when using FileSourceScanExec| +### PRs +||| +|:---|:---| +|[#177](https://github.com/NVIDIA/spark-rapids/pull/177)|Fixed leaks in unit test and use ColumnarBatch for testing| +|[#202](https://github.com/NVIDIA/spark-rapids/pull/202)|Fixed issue for contains when searching for an empty string| +|[#201](https://github.com/NVIDIA/spark-rapids/pull/201)|Fix name of scan| +|[#200](https://github.com/NVIDIA/spark-rapids/pull/200)|Fix issue with GpuAttributeReference not overrideing references| +|[#197](https://github.com/NVIDIA/spark-rapids/pull/197)|Fix metrics for writes| +|[#186](https://github.com/NVIDIA/spark-rapids/pull/186)|Fixed issue with nullability on concat| +|[#193](https://github.com/NVIDIA/spark-rapids/pull/193)|Add RapidsBufferCatalog tests| +|[#189](https://github.com/NVIDIA/spark-rapids/pull/189)|Handle AggregateExpression having resultIds parameter instead of a single resultId| +|[#190](https://github.com/NVIDIA/spark-rapids/pull/190)|FileSourceScanExec can have logicalRelation parameter on some distributions| +|[#185](https://github.com/NVIDIA/spark-rapids/pull/185)|Update type of parameter of GpuExpandExec to make it consistent| +|[#180](https://github.com/NVIDIA/spark-rapids/pull/180)|Add MetaUtils unit tests| +|[#157](https://github.com/NVIDIA/spark-rapids/pull/157)|Add partial and final only hash aggregate tests and fix nulls corner case for Average| +|[#158](https://github.com/NVIDIA/spark-rapids/pull/158)|Orc merge schema fallback and FileScan format configs| From 7cb34ffe96178e848c283c3ab8a982cb4328e6f3 Mon Sep 17 00:00:00 2001 From: Peixin Li Date: Fri, 21 Aug 2020 16:49:59 +0800 Subject: [PATCH 10/11] only filter out labels for issue --- .github/workflows/changelog/changelog | 2 +- CHANGELOG.md | 118 ++++++++++++++++++++++++++ 2 files changed, 119 insertions(+), 1 deletion(-) diff --git a/.github/workflows/changelog/changelog b/.github/workflows/changelog/changelog index 4e922c9eccd..47adde3b97c 100755 --- a/.github/workflows/changelog/changelog +++ b/.github/workflows/changelog/changelog @@ -166,7 +166,7 @@ def process_changelog(resource_type: str): for label in item['labels']['nodes']: labels.add(label['name']) category = rules(labels) - if category == INVALID: + if resource_type == ISSUES and category == INVALID: continue if resource_type == PULL_REQUESTS: category = PRS diff --git a/CHANGELOG.md b/CHANGELOG.md index ca384d939e1..606c4195fcb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -73,16 +73,28 @@ Generated on 2020-08-21 |:---|:---| |[#594](https://github.com/NVIDIA/spark-rapids/pull/594)|Filter nulls from joins where possible to improve performance.| |[#590](https://github.com/NVIDIA/spark-rapids/pull/590)|Move GpuParquetScan/GpuOrcScan into Shim| +|[#588](https://github.com/NVIDIA/spark-rapids/pull/588)|xfail the tpch spark 3.1.0 tests that fail| |[#572](https://github.com/NVIDIA/spark-rapids/pull/572)|Update buffer store to return compressed batches directly, add compression NVTX ranges| +|[#558](https://github.com/NVIDIA/spark-rapids/pull/558)|Fix unit tests when AQE is enabled| +|[#580](https://github.com/NVIDIA/spark-rapids/pull/580)|xfail the Spark 3.1.0 integration tests that fail | +|[#565](https://github.com/NVIDIA/spark-rapids/pull/565)|Minor improvements to TPC-DS benchmarking code| +|[#567](https://github.com/NVIDIA/spark-rapids/pull/567)|Explicitly disable AQE in one test| +|[#571](https://github.com/NVIDIA/spark-rapids/pull/571)|Fix Databricks shim layer for GpuFileSourceScanExec and GpuBroadcastExchangeExec| |[#564](https://github.com/NVIDIA/spark-rapids/pull/564)|Add GPU decode time metric to scans| |[#562](https://github.com/NVIDIA/spark-rapids/pull/562)|getCatalog can be called from the driver, and can return null| |[#555](https://github.com/NVIDIA/spark-rapids/pull/555)|Fix build warnings for ColumnViewAccess| +|[#560](https://github.com/NVIDIA/spark-rapids/pull/560)|Fix databricks build for AQE support| +|[#557](https://github.com/NVIDIA/spark-rapids/pull/557)|Fix tests failing on Spark 3.1| |[#547](https://github.com/NVIDIA/spark-rapids/pull/547)|Add GPU metrics to GpuFileSourceScanExec| |[#462](https://github.com/NVIDIA/spark-rapids/pull/462)|Implement optimized AQE support so that exchanges run on GPU where possible| +|[#550](https://github.com/NVIDIA/spark-rapids/pull/550)|Document Parquet and ORC compression support| |[#539](https://github.com/NVIDIA/spark-rapids/pull/539)|Update script to audit multiple Spark versions| |[#543](https://github.com/NVIDIA/spark-rapids/pull/543)|Add metrics to GpuUnion operator| +|[#549](https://github.com/NVIDIA/spark-rapids/pull/549)|Move spark shim properties to top level pom| |[#497](https://github.com/NVIDIA/spark-rapids/pull/497)|Add UDF compiler implementations| |[#487](https://github.com/NVIDIA/spark-rapids/pull/487)|Add framework for batch compression of shuffle partitions| +|[#544](https://github.com/NVIDIA/spark-rapids/pull/544)|Add in driverExtraClassPath for standalone mode docs| +|[#546](https://github.com/NVIDIA/spark-rapids/pull/546)|Fix Spark 3.1.0 shim build error in GpuHashJoin| |[#537](https://github.com/NVIDIA/spark-rapids/pull/537)|Use fresh SparkSession when capturing to avoid late capture of previous query| |[#538](https://github.com/NVIDIA/spark-rapids/pull/538)|Revert "Temporary workaround for RMM initial pool size bug (#530)"| |[#517](https://github.com/NVIDIA/spark-rapids/pull/517)|Add config to limit maximum RMM pool size| @@ -91,46 +103,125 @@ Generated on 2020-08-21 |[#529](https://github.com/NVIDIA/spark-rapids/pull/529)|[BUG] Degenerate table metas were not getting copied to the heap| |[#530](https://github.com/NVIDIA/spark-rapids/pull/530)|Temporary workaround for RMM initial pool size bug| |[#526](https://github.com/NVIDIA/spark-rapids/pull/526)|Fix bug with nullability reporting in GpuFilterExec| +|[#521](https://github.com/NVIDIA/spark-rapids/pull/521)|Fix typo with databricks shim classname SparkShimServiceProvider| |[#522](https://github.com/NVIDIA/spark-rapids/pull/522)|Use SQLConf instead of SparkConf when looking up SQL configs| |[#518](https://github.com/NVIDIA/spark-rapids/pull/518)|Fix init order issue in GpuShuffleEnv when RAPIDS shuffle configured| +|[#514](https://github.com/NVIDIA/spark-rapids/pull/514)|Added clarification of RegExpReplace, DateDiff, made descriptive text consistent| +|[#506](https://github.com/NVIDIA/spark-rapids/pull/506)|Add in basic support for running tpcds like queries| +|[#504](https://github.com/NVIDIA/spark-rapids/pull/504)|Add ability to ignore tests depending on spark shim version| |[#503](https://github.com/NVIDIA/spark-rapids/pull/503)|Remove unused async buffer spill support| +|[#501](https://github.com/NVIDIA/spark-rapids/pull/501)|disable codegen in 3.1 shim for hash join| |[#466](https://github.com/NVIDIA/spark-rapids/pull/466)|Optimize and fix Api validation script| +|[#481](https://github.com/NVIDIA/spark-rapids/pull/481)|Codeowners| +|[#439](https://github.com/NVIDIA/spark-rapids/pull/439)|Check a PR has been committed using git signoff| |[#319](https://github.com/NVIDIA/spark-rapids/pull/319)|Update partitioning logic in ShuffledBatchRDD| +|[#491](https://github.com/NVIDIA/spark-rapids/pull/491)|Temporarily ignore AQE integration tests| +|[#490](https://github.com/NVIDIA/spark-rapids/pull/490)|Fix Spark 3.1.0 build for HashJoin changes| +|[#482](https://github.com/NVIDIA/spark-rapids/pull/482)|Prevent bad practice in python tests| +|[#485](https://github.com/NVIDIA/spark-rapids/pull/485)|Show plan in assertion message if test fails| +|[#480](https://github.com/NVIDIA/spark-rapids/pull/480)|Fix link from README to getting-started.md| |[#448](https://github.com/NVIDIA/spark-rapids/pull/448)|Preliminary support for keeping broadcast exchanges on GPU when AQE is enabled| |[#478](https://github.com/NVIDIA/spark-rapids/pull/478)|Fall back to CPU for binary as string in parquet| |[#477](https://github.com/NVIDIA/spark-rapids/pull/477)|Fix special case joins in broadcast nested loop join| +|[#469](https://github.com/NVIDIA/spark-rapids/pull/469)|Update HashAggregateSuite to work with AQE| +|[#475](https://github.com/NVIDIA/spark-rapids/pull/475)|Udf compiler pom followup| |[#434](https://github.com/NVIDIA/spark-rapids/pull/434)|Add UDF compiler skeleton| +|[#474](https://github.com/NVIDIA/spark-rapids/pull/474)|Re-enable noscaladoc check| +|[#461](https://github.com/NVIDIA/spark-rapids/pull/461)|Fix comments style to pass scala style check| +|[#468](https://github.com/NVIDIA/spark-rapids/pull/468)|fix broken link| |[#456](https://github.com/NVIDIA/spark-rapids/pull/456)|Add closeOnExcept to clean up code that closes resources only on exceptions| +|[#464](https://github.com/NVIDIA/spark-rapids/pull/464)|Turn off noscaladoc rule until codebase is fixed| +|[#449](https://github.com/NVIDIA/spark-rapids/pull/449)|Enforce NoScalaDoc rule in scalastyle checks| +|[#450](https://github.com/NVIDIA/spark-rapids/pull/450)|Enable scalastyle for shuffle plugin| +|[#451](https://github.com/NVIDIA/spark-rapids/pull/451)|Databricks remove unneeded files and fix build to not fail on rm when file missing| +|[#442](https://github.com/NVIDIA/spark-rapids/pull/442)|Shim layer support for Spark 3.0.0 Databricks| +|[#447](https://github.com/NVIDIA/spark-rapids/pull/447)|Add scalastyle plugin to shim module| |[#426](https://github.com/NVIDIA/spark-rapids/pull/426)|Update BufferMeta to support multiple codec buffers per table| +|[#440](https://github.com/NVIDIA/spark-rapids/pull/440)|Run mortgage test both with AQE on and off| |[#445](https://github.com/NVIDIA/spark-rapids/pull/445)|Added in StringRPad and StringLPad| +|[#422](https://github.com/NVIDIA/spark-rapids/pull/422)|Documentation updates| |[#437](https://github.com/NVIDIA/spark-rapids/pull/437)|Fix bug with InSet and Strings| |[#435](https://github.com/NVIDIA/spark-rapids/pull/435)|Add in checks for Parquet LEGACY date/time rebase| |[#432](https://github.com/NVIDIA/spark-rapids/pull/432)|Fix batch use-after-close in partitioning, shuffle env init| +|[#423](https://github.com/NVIDIA/spark-rapids/pull/423)|Fix duplicates includes in assembly jar| +|[#418](https://github.com/NVIDIA/spark-rapids/pull/418)|CI Add unit tests running for Spark 3.0.1| +|[#421](https://github.com/NVIDIA/spark-rapids/pull/421)|Make it easier to run TPCxBB benchmarks from spark shell| +|[#413](https://github.com/NVIDIA/spark-rapids/pull/413)|Fix download link| +|[#414](https://github.com/NVIDIA/spark-rapids/pull/414)|Shim Layer to support multiple Spark versions | |[#406](https://github.com/NVIDIA/spark-rapids/pull/406)|Update cast handling to deal with new libcudf casting limitations| +|[#405](https://github.com/NVIDIA/spark-rapids/pull/405)|Change slave->worker| +|[#395](https://github.com/NVIDIA/spark-rapids/pull/395)|Databricks doc updates| +|[#401](https://github.com/NVIDIA/spark-rapids/pull/401)|Extended the FAQ| +|[#398](https://github.com/NVIDIA/spark-rapids/pull/398)|Add tests for GpuPartition| +|[#352](https://github.com/NVIDIA/spark-rapids/pull/352)|Change spark tgz package name| |[#397](https://github.com/NVIDIA/spark-rapids/pull/397)|Fix small bug in ShuffleBufferCatalog.hasActiveShuffle| +|[#286](https://github.com/NVIDIA/spark-rapids/pull/286)|[REVIEW] Updated join tests for cache| +|[#393](https://github.com/NVIDIA/spark-rapids/pull/393)|Contributor license agreement| |[#389](https://github.com/NVIDIA/spark-rapids/pull/389)|Added in support for RangeExec| |[#390](https://github.com/NVIDIA/spark-rapids/pull/390)|Ucx getting started| +|[#391](https://github.com/NVIDIA/spark-rapids/pull/391)|Hide slack channel in Jenkins scripts| |[#387](https://github.com/NVIDIA/spark-rapids/pull/387)|Remove the term whitelist| +|[#365](https://github.com/NVIDIA/spark-rapids/pull/365)|[REVIEW] Timesub tests| +|[#383](https://github.com/NVIDIA/spark-rapids/pull/383)|Test utility to compare SQL query results between CPU and GPU| +|[#380](https://github.com/NVIDIA/spark-rapids/pull/380)|Fix databricks notebook link| +|[#378](https://github.com/NVIDIA/spark-rapids/pull/378)|Added in FAQ and fixed spelling| +|[#377](https://github.com/NVIDIA/spark-rapids/pull/377)|Update heading in configs.md| +|[#373](https://github.com/NVIDIA/spark-rapids/pull/373)|Modifying branch name to conform with rapidsai branch name change| |[#376](https://github.com/NVIDIA/spark-rapids/pull/376)|Add our session extension correctly if there are other extensions configured| |[#374](https://github.com/NVIDIA/spark-rapids/pull/374)|Fix rat issue for notebooks| |[#364](https://github.com/NVIDIA/spark-rapids/pull/364)|Update Databricks patch for changes to GpuSortMergeJoin| +|[#371](https://github.com/NVIDIA/spark-rapids/pull/371)|fix typo and use regional bucket per GCP's update| +|[#359](https://github.com/NVIDIA/spark-rapids/pull/359)|Karthik changes| |[#353](https://github.com/NVIDIA/spark-rapids/pull/353)|Fix broadcast nested loop join for the no column case| +|[#313](https://github.com/NVIDIA/spark-rapids/pull/313)|Additional tests for broadcast hash join| |[#342](https://github.com/NVIDIA/spark-rapids/pull/342)|Implement build-side rules for shuffle hash join| |[#349](https://github.com/NVIDIA/spark-rapids/pull/349)|Updated join code to treat null equality properly| +|[#335](https://github.com/NVIDIA/spark-rapids/pull/335)|Integration tests on spark 3.0.1-SNAPSHOT & 3.1.0-SNAPSHOT| +|[#346](https://github.com/NVIDIA/spark-rapids/pull/346)|Update the Title Header for Fine Tuning| +|[#344](https://github.com/NVIDIA/spark-rapids/pull/344)|Fix small typo in readme| |[#331](https://github.com/NVIDIA/spark-rapids/pull/331)|Adds iterator and client unit tests, and prepares for more fetch failure handling| +|[#337](https://github.com/NVIDIA/spark-rapids/pull/337)|Fix Scala compile phase to allow Java classes referencing Scala classes| +|[#332](https://github.com/NVIDIA/spark-rapids/pull/332)|Match GPU overwritten functions with SQL functions from FunctionRegistry| +|[#339](https://github.com/NVIDIA/spark-rapids/pull/339)|Fix databricks build| |[#338](https://github.com/NVIDIA/spark-rapids/pull/338)|Move GpuPartitioning to a separate file| +|[#310](https://github.com/NVIDIA/spark-rapids/pull/310)|Update release Jenkinsfile for Databricks| +|[#330](https://github.com/NVIDIA/spark-rapids/pull/330)|Hide private info in Jenkins scripts| |[#324](https://github.com/NVIDIA/spark-rapids/pull/324)|Add in basic support for GpuCartesianProductExec| +|[#328](https://github.com/NVIDIA/spark-rapids/pull/328)|Enable slack notification for Databricks build| |[#321](https://github.com/NVIDIA/spark-rapids/pull/321)|update databricks patch for GpuBroadcastNestedLoopJoinExec| +|[#322](https://github.com/NVIDIA/spark-rapids/pull/322)|Add oss.sonatype.org to download the cudf jar| +|[#320](https://github.com/NVIDIA/spark-rapids/pull/320)|Don't mount passwd/group to the container| |[#258](https://github.com/NVIDIA/spark-rapids/pull/258)|Enable running TPCH tests with AQE enabled| +|[#318](https://github.com/NVIDIA/spark-rapids/pull/318)|Build docker image with Dockerfile| |[#309](https://github.com/NVIDIA/spark-rapids/pull/309)|Update databricks patch to latest changes| +|[#312](https://github.com/NVIDIA/spark-rapids/pull/312)|Trigger branch-0.2 integration test| +|[#307](https://github.com/NVIDIA/spark-rapids/pull/307)|[Jenkins] Update the release script and Jenkinsfile| +|[#304](https://github.com/NVIDIA/spark-rapids/pull/304)|[DOC][Minor] Fix typo in spark config name.| +|[#303](https://github.com/NVIDIA/spark-rapids/pull/303)|Update compatibility doc for -0.0 issues| +|[#301](https://github.com/NVIDIA/spark-rapids/pull/301)|Add info about branches in README.md| |[#296](https://github.com/NVIDIA/spark-rapids/pull/296)|Added in basic support for broadcast nested loop join| +|[#297](https://github.com/NVIDIA/spark-rapids/pull/297)|Databricks CI improvements and support runtime env parameter to xfail certain tests| +|[#292](https://github.com/NVIDIA/spark-rapids/pull/292)|Move artifacts version in version-def.sh| +|[#254](https://github.com/NVIDIA/spark-rapids/pull/254)|Cleanup QA tests| |[#289](https://github.com/NVIDIA/spark-rapids/pull/289)|Clean up GpuCollectLimitMeta and add in metrics| |[#287](https://github.com/NVIDIA/spark-rapids/pull/287)|Add in support for right join and fix issues build right| +|[#273](https://github.com/NVIDIA/spark-rapids/pull/273)|Added releases to the README.md| +|[#285](https://github.com/NVIDIA/spark-rapids/pull/285)|modify run_pyspark_from_build.sh to be bash 3 friendly| |[#281](https://github.com/NVIDIA/spark-rapids/pull/281)|Add in support for Full Outer Join on non-null keys| +|[#274](https://github.com/NVIDIA/spark-rapids/pull/274)|Add RapidsDiskStore tests| +|[#259](https://github.com/NVIDIA/spark-rapids/pull/259)|Add RapidsHostMemoryStore tests| +|[#282](https://github.com/NVIDIA/spark-rapids/pull/282)|Update Databricks patch for 0.2 branch| +|[#261](https://github.com/NVIDIA/spark-rapids/pull/261)|Add conditional xfail test for DISTINCT aggregates with NaN| |[#263](https://github.com/NVIDIA/spark-rapids/pull/263)|More time ops| |[#256](https://github.com/NVIDIA/spark-rapids/pull/256)|Remove special cases for contains, startsWith, and endWith| |[#253](https://github.com/NVIDIA/spark-rapids/pull/253)|Remove GpuAttributeReference and GpuSortOrder| +|[#271](https://github.com/NVIDIA/spark-rapids/pull/271)|Update the versions for 0.2.0 properly for the databricks build| |[#162](https://github.com/NVIDIA/spark-rapids/pull/162)|Integration tests for corner cases in window functions.| +|[#264](https://github.com/NVIDIA/spark-rapids/pull/264)|Add a local mvn repo for nightly pipeline| +|[#262](https://github.com/NVIDIA/spark-rapids/pull/262)|Refer to branch-0.2| +|[#255](https://github.com/NVIDIA/spark-rapids/pull/255)|Revert change to make dependencies of shaded jar optional| +|[#257](https://github.com/NVIDIA/spark-rapids/pull/257)|Fix link to RAPIDS cudf in index.md| +|[#252](https://github.com/NVIDIA/spark-rapids/pull/252)|Update to 0.2.0-SNAPSHOT and cudf-0.15-SNAPSHOT| ## Release 0.1 ### Features @@ -151,16 +242,43 @@ Generated on 2020-08-21 ### PRs ||| |:---|:---| +|[#249](https://github.com/NVIDIA/spark-rapids/pull/249)|Compatability -> Compatibility| +|[#247](https://github.com/NVIDIA/spark-rapids/pull/247)|Add index.md for default doc page, fix table formatting for configs| +|[#241](https://github.com/NVIDIA/spark-rapids/pull/241)|Let default branch to master per the release rule| |[#177](https://github.com/NVIDIA/spark-rapids/pull/177)|Fixed leaks in unit test and use ColumnarBatch for testing| +|[#243](https://github.com/NVIDIA/spark-rapids/pull/243)|Jenkins file for Databricks release| +|[#225](https://github.com/NVIDIA/spark-rapids/pull/225)|Make internal project dependencies optional for shaded artifact| +|[#242](https://github.com/NVIDIA/spark-rapids/pull/242)|Add site pages| +|[#221](https://github.com/NVIDIA/spark-rapids/pull/221)|Databricks Build Support| +|[#215](https://github.com/NVIDIA/spark-rapids/pull/215)|Remove CudfColumnVector| +|[#213](https://github.com/NVIDIA/spark-rapids/pull/213)|Add RapidsDeviceMemoryStore tests| +|[#214](https://github.com/NVIDIA/spark-rapids/pull/214)|[REVIEW] Test failure to pass Attribute as GpuAttribute| +|[#211](https://github.com/NVIDIA/spark-rapids/pull/211)|Add project leads to pom developer list| +|[#210](https://github.com/NVIDIA/spark-rapids/pull/210)|Updated coverage docs| +|[#196](https://github.com/NVIDIA/spark-rapids/pull/196)|Update Spark dependency to the released 3.0.0 artifacts| |[#202](https://github.com/NVIDIA/spark-rapids/pull/202)|Fixed issue for contains when searching for an empty string| |[#201](https://github.com/NVIDIA/spark-rapids/pull/201)|Fix name of scan| |[#200](https://github.com/NVIDIA/spark-rapids/pull/200)|Fix issue with GpuAttributeReference not overrideing references| |[#197](https://github.com/NVIDIA/spark-rapids/pull/197)|Fix metrics for writes| |[#186](https://github.com/NVIDIA/spark-rapids/pull/186)|Fixed issue with nullability on concat| |[#193](https://github.com/NVIDIA/spark-rapids/pull/193)|Add RapidsBufferCatalog tests| +|[#188](https://github.com/NVIDIA/spark-rapids/pull/188)|rebrand to com.nvidia instead of ai.rapids| |[#189](https://github.com/NVIDIA/spark-rapids/pull/189)|Handle AggregateExpression having resultIds parameter instead of a single resultId| |[#190](https://github.com/NVIDIA/spark-rapids/pull/190)|FileSourceScanExec can have logicalRelation parameter on some distributions| |[#185](https://github.com/NVIDIA/spark-rapids/pull/185)|Update type of parameter of GpuExpandExec to make it consistent| +|[#172](https://github.com/NVIDIA/spark-rapids/pull/172)|Merge qa test to integration test| |[#180](https://github.com/NVIDIA/spark-rapids/pull/180)|Add MetaUtils unit tests| +|[#171](https://github.com/NVIDIA/spark-rapids/pull/171)|Cleanup scaladoc warnings about missing links| +|[#176](https://github.com/NVIDIA/spark-rapids/pull/176)|Updated join tests to cover more data.| +|[#169](https://github.com/NVIDIA/spark-rapids/pull/169)|Remove dependency on shaded Spark artifact| +|[#174](https://github.com/NVIDIA/spark-rapids/pull/174)|Added in fallback tests| +|[#165](https://github.com/NVIDIA/spark-rapids/pull/165)|Move input metadata tests to pyspark| +|[#173](https://github.com/NVIDIA/spark-rapids/pull/173)|Fix setting local mode for tests| +|[#160](https://github.com/NVIDIA/spark-rapids/pull/160)|Integration tests for normalizing NaN/zeroes.| +|[#163](https://github.com/NVIDIA/spark-rapids/pull/163)|Ignore the order locally for repartition tests| |[#157](https://github.com/NVIDIA/spark-rapids/pull/157)|Add partial and final only hash aggregate tests and fix nulls corner case for Average| +|[#159](https://github.com/NVIDIA/spark-rapids/pull/159)|Add integration tests for joins| |[#158](https://github.com/NVIDIA/spark-rapids/pull/158)|Orc merge schema fallback and FileScan format configs| +|[#164](https://github.com/NVIDIA/spark-rapids/pull/164)|Fix compiler warnings| +|[#152](https://github.com/NVIDIA/spark-rapids/pull/152)|Moved cudf to 0.14 for CI| +|[#151](https://github.com/NVIDIA/spark-rapids/pull/151)|Switch CICD pipelines to Github| From a48aff4550d061269ab6a33ae5e88ad07a34d0c1 Mon Sep 17 00:00:00 2001 From: Peixin Li Date: Mon, 24 Aug 2020 09:51:56 +0800 Subject: [PATCH 11/11] add nightly workflow on github actions --- .github/workflows/changelog.yml | 63 +++++++++++++++++++++++++++ .github/workflows/changelog/changelog | 3 +- 2 files changed, 65 insertions(+), 1 deletion(-) create mode 100644 .github/workflows/changelog.yml diff --git a/.github/workflows/changelog.yml b/.github/workflows/changelog.yml new file mode 100644 index 00000000000..57e78fab503 --- /dev/null +++ b/.github/workflows/changelog.yml @@ -0,0 +1,63 @@ +# Copyright (c) 2020, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# A workflow to create PR to update CHANGELOG nightly +name: CHANGELOG generation nightly + +on: + schedule: + - cron: '5/* * * * *' # nightly + +jobs: + changelog-gen: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v2 + + - name: Update CHANGELOG.md + id: upt + run: echo "::set-output name=stdout::$(.github/workflows/changelog/changelog --base_refs=branch-0.1,branch-0.2,branch-0.3)" + env: + GITHUB_TOKEN: ${{ secrets.PAT }} + + - name: Get date + id: dt + run: echo "::set-output name=date::$(date +'%Y-%m-%d')" + + - name: Create PR + uses: peter-evans/create-pull-request@v3 + with: + token: ${{ secrets.PAT }} + commit-message: + committer: Peixin Li + author: Peixin Li + signoff: true + branch: changelog-night-update + title: '[DOC] Changelog update ${{ steps.dt.outputs.date }}' + body: | + changelog-gen runs on ${{ steps.dt.outputs.date }} + + script run: + ```bash + ${{ steps.upt.outputs.stdout }} + ``` + + Please review newest CHANGELOG.md, then merge or close the PR. + labels: | + documentation + reviewers: pxLi + team-reviewers: | + owners + maintainers + draft: false diff --git a/.github/workflows/changelog/changelog b/.github/workflows/changelog/changelog index 47adde3b97c..ba879186c9e 100755 --- a/.github/workflows/changelog/changelog +++ b/.github/workflows/changelog/changelog @@ -142,7 +142,8 @@ def process_changelog(resource_type: str): items = process_issue() time_field = 'closedAt' else: - return + print(f"[process_changelog] Invalid type: {resource_type}") + sys.exit(1) for item in items: if len(item['projectCards']['nodes']) == 0: