From 3950fa0aef36e2ab4dc213193708f9f1e07cf791 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Fri, 8 Dec 2023 07:29:49 +0800 Subject: [PATCH 01/19] Avoid leading and trailing zeros in test_timestamp_seconds_rounding_necessary (#9970) * Avoid leading and tailing zeros in test_timestamp_seconds_rounding_necessary Signed-off-by: Haoyang Li * add comments and fix a typo Signed-off-by: Haoyang Li * add test comment Signed-off-by: Haoyang Li --------- Signed-off-by: Haoyang Li --- integration_tests/src/main/python/data_gen.py | 17 ++++++++++++++--- .../src/main/python/date_time_test.py | 8 ++++++-- 2 files changed, 20 insertions(+), 5 deletions(-) diff --git a/integration_tests/src/main/python/data_gen.py b/integration_tests/src/main/python/data_gen.py index 895e56050cc..4e4da4c9fa5 100644 --- a/integration_tests/src/main/python/data_gen.py +++ b/integration_tests/src/main/python/data_gen.py @@ -244,7 +244,8 @@ def start(self, rand): class DecimalGen(DataGen): """Generate Decimals, with some built in corner cases.""" - def __init__(self, precision=None, scale=None, nullable=True, special_cases=None, avoid_positive_values=False): + def __init__(self, precision=None, scale=None, nullable=True, special_cases=None, avoid_positive_values=False, full_precision=False): + """full_precision: If True, generate decimals with full precision without leading and trailing zeros.""" if precision is None: #Maximum number of decimal digits a Long can represent is 18 precision = 18 @@ -259,12 +260,14 @@ def __init__(self, precision=None, scale=None, nullable=True, special_cases=None self.scale = scale self.precision = precision self.avoid_positive_values = avoid_positive_values + self.full_precision = full_precision def __repr__(self): return super().__repr__() + '(' + str(self.precision) + ',' + str(self.scale) + ')' def _cache_repr(self): - return super()._cache_repr() + '(' + str(self.precision) + ',' + str(self.scale) + ',' + str(self.avoid_positive_values) + ')' + return super()._cache_repr() + '(' + str(self.precision) + ',' + str(self.scale) + ',' +\ + str(self.avoid_positive_values) + ',' + str(self.full_precision) + ')' def start(self, rand): def random_decimal(rand): @@ -272,7 +275,15 @@ def random_decimal(rand): sign = "-" else: sign = rand.choice(["-", ""]) - int_part = "".join([rand.choice("0123456789") for _ in range(self.precision)]) + if self.full_precision: + if self.precision == 1: + int_part = rand.choice("123456789") + else: + int_part = rand.choice("123456789") + \ + "".join([rand.choice("0123456789") for _ in range(self.precision - 2)]) + \ + rand.choice("123456789") + else: + int_part = "".join([rand.choice("0123456789") for _ in range(self.precision)]) result = f"{sign}{int_part}e{str(-self.scale)}" return Decimal(result) diff --git a/integration_tests/src/main/python/date_time_test.py b/integration_tests/src/main/python/date_time_test.py index d68dd93efac..9bc910b6350 100644 --- a/integration_tests/src/main/python/date_time_test.py +++ b/integration_tests/src/main/python/date_time_test.py @@ -573,9 +573,13 @@ def test_timestamp_seconds_long_overflow(): lambda spark : unary_op_df(spark, long_gen).selectExpr("timestamp_seconds(a)").collect(), conf={}, error_message='long overflow') - -@pytest.mark.parametrize('data_gen', [DecimalGen(7, 7), DecimalGen(20, 7)], ids=idfn) + +# For Decimal(20, 7) case, the data is both 'Overflow' and 'Rounding necessary', this case is to verify +# that 'Rounding necessary' check is before 'Overflow' check. So we should make sure that every decimal +# value in test data is 'Rounding necessary' by setting full_precision=True to avoid leading and trailing zeros. +# Otherwise, the test data will bypass the 'Rounding necessary' check and throw an 'Overflow' error. @pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@pytest.mark.parametrize('data_gen', [DecimalGen(7, 7, full_precision=True), DecimalGen(20, 7, full_precision=True)], ids=idfn) def test_timestamp_seconds_rounding_necessary(data_gen): assert_gpu_and_cpu_error( lambda spark : unary_op_df(spark, data_gen).selectExpr("timestamp_seconds(a)").collect(), From a5c37fb157f9c73aeffd8205ea907212a13885dc Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Fri, 8 Dec 2023 13:00:29 +0800 Subject: [PATCH 02/19] Avoid allocating GPU memory out of RMM managed pool in test (#9985) Signed-off-by: Ferdinand Xu --- .../spark/rapids/CachedBatchWriterSuite.scala | 109 ++++++++++-------- 1 file changed, 59 insertions(+), 50 deletions(-) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/CachedBatchWriterSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/CachedBatchWriterSuite.scala index e7418b06916..038eabed3fc 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/CachedBatchWriterSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/CachedBatchWriterSuite.scala @@ -20,7 +20,7 @@ import scala.collection.JavaConverters.asScalaIteratorConverter import scala.collection.mutable.{ArrayBuffer, ListBuffer} import scala.collection.mutable -import ai.rapids.cudf.{ColumnVector, CompressionType, DType, Table, TableWriter} +import ai.rapids.cudf.{ColumnVector, CompressionType, DType, Rmm, Table, TableWriter} import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableFromBatchColumns import org.apache.hadoop.mapreduce.{RecordWriter, TaskAttemptContext} @@ -47,6 +47,8 @@ import org.apache.spark.storage.StorageLevel.MEMORY_ONLY class CachedBatchWriterSuite extends SparkQueryCompareTestSuite { class TestResources extends AutoCloseable { + assert(Rmm.isInitialized, "Need to use this within Spark GPU session, or it may fail to " + + "release column vector.") val byteCv1 = ColumnVector.fromBytes(1) val byteCv3 = ColumnVector.fromBytes(3) val byteCv456 = ColumnVector.fromBytes(4, 5, 6) @@ -59,68 +61,75 @@ class CachedBatchWriterSuite extends SparkQueryCompareTestSuite { } test("convert columnar batch to cached batch on single col table with 0 rows in a batch") { - withResource(new TestResources()) { resources => - val (_, spyGpuCol0) = getCudfAndGpuVectors(resources) - val cb = new ColumnarBatch(Array(spyGpuCol0), 0) - val ser = new ParquetCachedBatchSerializer - val dummySchema = new StructType( - Array(StructField("empty", ByteType, false), - StructField("empty", ByteType, false), - StructField("empty", ByteType, false))) - val listOfPCB = ser.compressColumnarBatchWithParquet(cb, dummySchema, dummySchema, - BYTES_ALLOWED_PER_BATCH, false) - assert(listOfPCB.isEmpty) - } + withGpuSparkSession(_ => + withResource(new TestResources()) { resources => + val (_, spyGpuCol0) = getCudfAndGpuVectors(resources) + val cb = new ColumnarBatch(Array(spyGpuCol0), 0) + val ser = new ParquetCachedBatchSerializer + val dummySchema = new StructType( + Array( + StructField("empty", ByteType, false), + StructField("empty", ByteType, false), + StructField("empty", ByteType, false))) + val listOfPCB = ser.compressColumnarBatchWithParquet( + cb, dummySchema, dummySchema, + BYTES_ALLOWED_PER_BATCH, false) + assert(listOfPCB.isEmpty) + }) } test("convert large columnar batch to cached batch on single col table") { - withResource(new TestResources()) { resources => - val (spyCol0, spyGpuCol0) = getCudfAndGpuVectors(resources) - val splitAt = 2086912 - testCompressColBatch(resources, Array(spyCol0), Array(spyGpuCol0), splitAt) - verify(spyCol0).split(splitAt) - } + withGpuSparkSession(_ => + withResource(new TestResources()) { resources => + val (spyCol0, spyGpuCol0) = getCudfAndGpuVectors(resources) + val splitAt = 2086912 + testCompressColBatch(resources, Array(spyCol0), Array(spyGpuCol0), splitAt) + verify(spyCol0).split(splitAt) + }) } test("convert large columnar batch to cached batch on multi-col table") { - withResource(new TestResources()) { resources => - val (spyCol0, spyGpuCol0) = getCudfAndGpuVectors(resources) - val splitAt = Seq(695637, 1391274, 2086911, 2782548) - testCompressColBatch(resources, Array(spyCol0, spyCol0, spyCol0), - Array(spyGpuCol0, spyGpuCol0, spyGpuCol0), splitAt: _*) - verify(spyCol0, times(3)).split(splitAt: _*) - } + withGpuSparkSession(_ => + withResource(new TestResources()) { resources => + val (spyCol0, spyGpuCol0) = getCudfAndGpuVectors(resources) + val splitAt = Seq(695637, 1391274, 2086911, 2782548) + testCompressColBatch(resources, Array(spyCol0, spyCol0, spyCol0), + Array(spyGpuCol0, spyGpuCol0, spyGpuCol0), splitAt: _*) + verify(spyCol0, times(3)).split(splitAt: _*) + }) } test("convert large InternalRow iterator to cached batch single col") { - withResource(new TestResources()) { resources => - val (_, spyGpuCol0) = getCudfAndGpuVectors(resources) - val cb = new ColumnarBatch(Array(spyGpuCol0), ROWS) - val mockByteType = mock(classOf[ByteType]) - when(mockByteType.defaultSize).thenReturn(1024) - val schema = Seq(AttributeReference("field0", mockByteType, true)()) - testColumnarBatchToCachedBatchIterator(cb, schema) - } + withGpuSparkSession(_ => + withResource(new TestResources()) { resources => + val (_, spyGpuCol0) = getCudfAndGpuVectors(resources) + val cb = new ColumnarBatch(Array(spyGpuCol0), ROWS) + val mockByteType = mock(classOf[ByteType]) + when(mockByteType.defaultSize).thenReturn(1024) + val schema = Seq(AttributeReference("field0", mockByteType, true)()) + testColumnarBatchToCachedBatchIterator(cb, schema) + }) } test("convert large InternalRow iterator to cached batch multi-col") { - withResource(new TestResources()) { resources1 => - val (_, spyGpuCol0) = getCudfAndGpuVectors(resources1) - withResource(new TestResources()) { resources2 => - val (_, spyGpuCol1) = getCudfAndGpuVectors(resources2) - withResource(new TestResources()) { resources3 => - val (_, spyGpuCol2) = getCudfAndGpuVectors(resources3) - val cb = new ColumnarBatch(Array(spyGpuCol0, spyGpuCol1, spyGpuCol2), ROWS) - val mockByteType = mock(classOf[ByteType]) - when(mockByteType.defaultSize).thenReturn(1024) - val schema = Seq(AttributeReference("field0", mockByteType, true)(), - AttributeReference("field1", mockByteType, true)(), - AttributeReference("field2", mockByteType, true)()) - - testColumnarBatchToCachedBatchIterator(cb, schema) + withGpuSparkSession(_ => + withResource(new TestResources()) { resources1 => + val (_, spyGpuCol0) = getCudfAndGpuVectors(resources1) + withResource(new TestResources()) { resources2 => + val (_, spyGpuCol1) = getCudfAndGpuVectors(resources2) + withResource(new TestResources()) { resources3 => + val (_, spyGpuCol2) = getCudfAndGpuVectors(resources3) + val cb = new ColumnarBatch(Array(spyGpuCol0, spyGpuCol1, spyGpuCol2), ROWS) + val mockByteType = mock(classOf[ByteType]) + when(mockByteType.defaultSize).thenReturn(1024) + val schema = Seq(AttributeReference("field0", mockByteType, true)(), + AttributeReference("field1", mockByteType, true)(), + AttributeReference("field2", mockByteType, true)()) + + testColumnarBatchToCachedBatchIterator(cb, schema) + } } - } - } + }) } test("test useCompression conf is honored") { From c593338948f595a6f2cf97f952647cc368fbed1a Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Mon, 11 Dec 2023 08:11:39 +0800 Subject: [PATCH 03/19] Optimize from_unixtime (#10000) Signed-off-by: Chong Gao --- .../spark/sql/rapids/datetimeExpressions.scala | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala index d85fe582eff..913594e810c 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala @@ -1009,16 +1009,14 @@ case class GpuFromUnixTime( override def doColumnar(lhs: GpuColumnVector, rhs: GpuScalar): ColumnVector = { // we aren't using rhs as it was already converted in the GpuOverrides while creating the // expressions map and passed down here as strfFormat - withResource(lhs.getBase.asTimestampSeconds) { secondsVector => - withResource(secondsVector.asTimestampMicroseconds) { tsVector => - if (GpuOverrides.isUTCTimezone(zoneId)) { - // UTC time zone - tsVector.asStrings(strfFormat) - } else { - // Non-UTC TZ - withResource(GpuTimeZoneDB.fromUtcTimestampToTimestamp(tsVector, zoneId.normalized())) { - shifted => shifted.asStrings(strfFormat) - } + withResource(lhs.getBase.asTimestampSeconds) { secondCV => + if (GpuOverrides.isUTCTimezone(zoneId)) { + // UTC time zone + secondCV.asStrings(strfFormat) + } else { + // Non-UTC TZ + withResource(GpuTimeZoneDB.fromUtcTimestampToTimestamp(secondCV, zoneId.normalized())) { + shifted => shifted.asStrings(strfFormat) } } } From 87555e9ef97f0512e14a40055b9d5984fbf18768 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Sun, 10 Dec 2023 19:25:20 -0600 Subject: [PATCH 04/19] Use Spark 3.3.3 instead of 3.3.2 for Scala 2.13 premerge builds (#10002) * Use Spark 3.3.3 instead of 3.3.2 for Scala 2.13 premerge builds Signed-off-by: Jason Lowe * Generate Scala 2.13 poms --------- Signed-off-by: Jason Lowe --- pom.xml | 2 +- scala2.13/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 744e9dd2985..2b33c3339ba 100644 --- a/pom.xml +++ b/pom.xml @@ -791,7 +791,7 @@ 320 - 332, + 333, 340 diff --git a/scala2.13/pom.xml b/scala2.13/pom.xml index 7b09cf61707..e8a81bb77bd 100644 --- a/scala2.13/pom.xml +++ b/scala2.13/pom.xml @@ -791,7 +791,7 @@ 320 - 332, + 333, 340 From 4ab7b68c0c1fd21ebd7701a35d85fe4284c9ccc2 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Mon, 11 Dec 2023 10:57:46 +0800 Subject: [PATCH 05/19] Enable ORC timestamp and decimal predicate push down tests (#9827) Signed-off-by: Haoyang Li --- .../scala/org/apache/spark/sql/rapids/OrcFilterSuite.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/OrcFilterSuite.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/OrcFilterSuite.scala index d7cd5cbfd6f..fe86900b32f 100644 --- a/tests/src/test/scala/org/apache/spark/sql/rapids/OrcFilterSuite.scala +++ b/tests/src/test/scala/org/apache/spark/sql/rapids/OrcFilterSuite.scala @@ -75,7 +75,6 @@ class OrcFilterSuite extends SparkQueryCompareTestSuite { } test("Support for pushing down filters for decimal types gpu write gpu read") { - assume(false, "https://github.com/rapidsai/cudf/issues/13933") withTempPath { file => withGpuSparkSession(spark => { val data = (0 until 10).map(i => Tuple1(BigDecimal.valueOf(i))) @@ -87,7 +86,6 @@ class OrcFilterSuite extends SparkQueryCompareTestSuite { } test("Support for pushing down filters for decimal types gpu write cpu read") { - assume(false, "https://github.com/rapidsai/cudf/issues/13933") withTempPath { file => withGpuSparkSession(spark => { val data = (0 until 10).map(i => Tuple1(BigDecimal.valueOf(i))) @@ -132,7 +130,6 @@ class OrcFilterSuite extends SparkQueryCompareTestSuite { } test("Support for pushing down filters for timestamp types gpu write cpu read") { - assume(false, "https://github.com/rapidsai/cudf/issues/13899") withTempPath { file => withGpuSparkSession(spark => { val timeString = "2015-08-20 14:57:00" @@ -151,7 +148,6 @@ class OrcFilterSuite extends SparkQueryCompareTestSuite { } test("Support for pushing down filters for timestamp types gpu write gpu read") { - assume(false, "https://github.com/rapidsai/cudf/issues/13899") withTempPath { file => withGpuSparkSession(spark => { val timeString = "2015-08-20 14:57:00" From d6bc3002bc587f306f8f9865dcadc24100f4a146 Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Mon, 11 Dec 2023 13:24:33 +0800 Subject: [PATCH 06/19] Fix Java OOM in non-UTC case with lots of xfail (#9944) (#10007) Signed-off-by: Ferdinand Xu Co-authored-by: Ferdinand Xu --- integration_tests/src/main/python/asserts.py | 41 +++++++++++-------- .../rapids/ExecutionPlanCaptureCallback.scala | 6 +++ ...mmedExecutionPlanCaptureCallbackImpl.scala | 9 ++++ 3 files changed, 39 insertions(+), 17 deletions(-) diff --git a/integration_tests/src/main/python/asserts.py b/integration_tests/src/main/python/asserts.py index 27b33848c2c..7d97c81af04 100644 --- a/integration_tests/src/main/python/asserts.py +++ b/integration_tests/src/main/python/asserts.py @@ -351,24 +351,31 @@ def assert_gpu_fallback_write(write_func, jvm.org.apache.spark.sql.rapids.ExecutionPlanCaptureCallback.startCapture() gpu_start = time.time() gpu_path = base_path + '/GPU' - with_gpu_session(lambda spark : write_func(spark, gpu_path), conf=conf) - gpu_end = time.time() - jvm.org.apache.spark.sql.rapids.ExecutionPlanCaptureCallback.assertCapturedAndGpuFellBack(cpu_fallback_class_name_list, 10000) - print('### WRITE: GPU TOOK {} CPU TOOK {} ###'.format( - gpu_end - gpu_start, cpu_end - cpu_start)) - - (cpu_bring_back, cpu_collect_type) = _prep_func_for_compare( - lambda spark: read_func(spark, cpu_path), 'COLLECT') - (gpu_bring_back, gpu_collect_type) = _prep_func_for_compare( - lambda spark: read_func(spark, gpu_path), 'COLLECT') - - from_cpu = with_cpu_session(cpu_bring_back, conf=conf) - from_gpu = with_cpu_session(gpu_bring_back, conf=conf) - if should_sort_locally(): - from_cpu.sort(key=_RowCmp) - from_gpu.sort(key=_RowCmp) + try: + with_gpu_session(lambda spark : write_func(spark, gpu_path), conf=conf) + gpu_end = time.time() + jvm.org.apache.spark.sql.rapids.ExecutionPlanCaptureCallback.assertCapturedAndGpuFellBack(cpu_fallback_class_name_list, 10000) + print('### WRITE: GPU TOOK {} CPU TOOK {} ###'.format( + gpu_end - gpu_start, cpu_end - cpu_start)) + + (cpu_bring_back, cpu_collect_type) = _prep_func_for_compare( + lambda spark: read_func(spark, cpu_path), 'COLLECT') + (gpu_bring_back, gpu_collect_type) = _prep_func_for_compare( + lambda spark: read_func(spark, gpu_path), 'COLLECT') + + from_cpu = with_cpu_session(cpu_bring_back, conf=conf) + from_gpu = with_cpu_session(gpu_bring_back, conf=conf) + if should_sort_locally(): + from_cpu.sort(key=_RowCmp) + from_gpu.sort(key=_RowCmp) + + assert_equal(from_cpu, from_gpu) + finally: + # Ensure `shouldCapture` state is restored. This may happen when GpuPlan is failed to be executed, + # then `shouldCapture` state is failed to restore in `assertCapturedAndGpuFellBack` method. + # This mostly happen within a xfail case where error may be ignored. + jvm.org.apache.spark.sql.rapids.ExecutionPlanCaptureCallback.endCapture() - assert_equal(from_cpu, from_gpu) def assert_cpu_and_gpu_are_equal_collect_with_capture(func, exist_classes='', diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExecutionPlanCaptureCallback.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExecutionPlanCaptureCallback.scala index c082de95241..bfdebc086c0 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExecutionPlanCaptureCallback.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExecutionPlanCaptureCallback.scala @@ -26,6 +26,8 @@ trait ExecutionPlanCaptureCallbackBase { def captureIfNeeded(qe: QueryExecution): Unit def startCapture(): Unit def startCapture(timeoutMillis: Long): Unit + def endCapture(): Unit + def endCapture(timeoutMillis: Long): Unit def getResultsWithTimeout(timeoutMs: Long = 10000): Array[SparkPlan] def extractExecutedPlan(plan: SparkPlan): SparkPlan def assertContains(gpuPlan: SparkPlan, className: String): Unit @@ -57,6 +59,10 @@ object ExecutionPlanCaptureCallback extends ExecutionPlanCaptureCallbackBase { override def startCapture(timeoutMillis: Long): Unit = impl.startCapture(timeoutMillis) + override def endCapture(): Unit = impl.endCapture() + + override def endCapture(timeoutMillis: Long): Unit = impl.endCapture(timeoutMillis) + override def getResultsWithTimeout(timeoutMs: Long = 10000): Array[SparkPlan] = impl.getResultsWithTimeout(timeoutMs) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ShimmedExecutionPlanCaptureCallbackImpl.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ShimmedExecutionPlanCaptureCallbackImpl.scala index 8f811496b3d..00379026f05 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ShimmedExecutionPlanCaptureCallbackImpl.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ShimmedExecutionPlanCaptureCallbackImpl.scala @@ -57,6 +57,15 @@ class ShimmedExecutionPlanCaptureCallbackImpl extends ExecutionPlanCaptureCallba } } + override def endCapture(): Unit = endCapture(10000) + + override def endCapture(timeoutMillis: Long): Unit = synchronized { + if (shouldCapture) { + shouldCapture = false + execPlans.clear() + } + } + override def getResultsWithTimeout(timeoutMs: Long = 10000): Array[SparkPlan] = { try { val spark = SparkSession.active From ac0e29252bf2316207edaf872e14ba151c059644 Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Mon, 11 Dec 2023 16:17:13 +0800 Subject: [PATCH 07/19] Using fix seed to unblock 23.12 release; Move the blocked issues to 24.02 (#10009) Signed-off-by: Chong Gao Co-authored-by: Chong Gao --- integration_tests/src/main/python/conditionals_test.py | 1 + integration_tests/src/main/python/delta_lake_delete_test.py | 2 ++ integration_tests/src/main/python/delta_lake_update_test.py | 1 + integration_tests/src/main/python/hash_aggregate_test.py | 1 + integration_tests/src/main/python/schema_evolution_test.py | 3 ++- 5 files changed, 7 insertions(+), 1 deletion(-) diff --git a/integration_tests/src/main/python/conditionals_test.py b/integration_tests/src/main/python/conditionals_test.py index 48d5a05c099..0370d24da11 100644 --- a/integration_tests/src/main/python/conditionals_test.py +++ b/integration_tests/src/main/python/conditionals_test.py @@ -219,6 +219,7 @@ def test_conditional_with_side_effects_col_scalar(data_gen): @pytest.mark.parametrize('data_gen', [mk_str_gen('[0-9]{1,20}')], ids=idfn) @pytest.mark.skipif(not is_jvm_charset_utf8(), reason="regular expressions require UTF-8") +@datagen_overrides(seed=0, reason='https://github.com/NVIDIA/spark-rapids/issues/9992') def test_conditional_with_side_effects_cast(data_gen): test_conf=copy_and_update( ansi_enabled_conf, {'spark.rapids.sql.regexp.enabled': True}) diff --git a/integration_tests/src/main/python/delta_lake_delete_test.py b/integration_tests/src/main/python/delta_lake_delete_test.py index 413479b3a12..fe2659bf8b7 100644 --- a/integration_tests/src/main/python/delta_lake_delete_test.py +++ b/integration_tests/src/main/python/delta_lake_delete_test.py @@ -153,6 +153,7 @@ def generate_dest_data(spark): @pytest.mark.parametrize("use_cdf", [True, False], ids=idfn) @pytest.mark.parametrize("partition_columns", [None, ["a"]], ids=idfn) @pytest.mark.skipif(is_before_spark_320(), reason="Delta Lake writes are not supported before Spark 3.2.x") +@datagen_overrides(seed=0, reason='https://github.com/NVIDIA/spark-rapids/issues/9884') def test_delta_delete_rows(spark_tmp_path, use_cdf, partition_columns): # Databricks changes the number of files being written, so we cannot compare logs unless there's only one slice num_slices_to_test = 1 if is_databricks_runtime() else 10 @@ -171,6 +172,7 @@ def generate_dest_data(spark): @pytest.mark.parametrize("use_cdf", [True, False], ids=idfn) @pytest.mark.parametrize("partition_columns", [None, ["a"]], ids=idfn) @pytest.mark.skipif(is_before_spark_320(), reason="Delta Lake writes are not supported before Spark 3.2.x") +@datagen_overrides(seed=0, reason='https://github.com/NVIDIA/spark-rapids/issues/9884') def test_delta_delete_dataframe_api(spark_tmp_path, use_cdf, partition_columns): from delta.tables import DeltaTable data_path = spark_tmp_path + "/DELTA_DATA" diff --git a/integration_tests/src/main/python/delta_lake_update_test.py b/integration_tests/src/main/python/delta_lake_update_test.py index 0fc65658332..b1348fdfe17 100644 --- a/integration_tests/src/main/python/delta_lake_update_test.py +++ b/integration_tests/src/main/python/delta_lake_update_test.py @@ -122,6 +122,7 @@ def generate_dest_data(spark): @pytest.mark.parametrize("use_cdf", [True, False], ids=idfn) @pytest.mark.parametrize("partition_columns", [None, ["a"]], ids=idfn) @pytest.mark.skipif(is_before_spark_320(), reason="Delta Lake writes are not supported before Spark 3.2.x") +@datagen_overrides(seed=0, reason='https://github.com/NVIDIA/spark-rapids/issues/9884') def test_delta_update_rows(spark_tmp_path, use_cdf, partition_columns): # Databricks changes the number of files being written, so we cannot compare logs unless there's only one slice num_slices_to_test = 1 if is_databricks_runtime() else 10 diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index 0c99fc4516a..6fada82dd37 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -396,6 +396,7 @@ def test_hash_reduction_sum(data_gen, conf): @pytest.mark.parametrize('data_gen', numeric_gens + decimal_gens + [ DecimalGen(precision=38, scale=0), DecimalGen(precision=38, scale=-10)], ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) +@datagen_overrides(seed=0, reason='https://github.com/NVIDIA/spark-rapids/issues/9779') def test_hash_reduction_sum_full_decimal(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen, length=100).selectExpr("SUM(a)"), diff --git a/integration_tests/src/main/python/schema_evolution_test.py b/integration_tests/src/main/python/schema_evolution_test.py index d9f4c0f0899..248d915523e 100644 --- a/integration_tests/src/main/python/schema_evolution_test.py +++ b/integration_tests/src/main/python/schema_evolution_test.py @@ -16,7 +16,7 @@ from conftest import is_not_utc from data_gen import * from datetime import date, datetime, timezone -from marks import ignore_order +from marks import ignore_order, datagen_overrides import pytest from spark_session import is_databricks_runtime, is_databricks113_or_later @@ -63,6 +63,7 @@ def get_ddl(col_gen_pairs): @ignore_order(local=True) @pytest.mark.parametrize("format", _formats) @pytest.mark.xfail(condition = is_not_utc(), reason = 'xfail non-UTC time zone tests because of https://github.com/NVIDIA/spark-rapids/issues/9653') +@datagen_overrides(seed=0, reason='https://github.com/NVIDIA/spark-rapids/issues/9807') def test_column_add_after_partition(spark_tmp_table_factory, format): # Databricks 10.4 appears to be missing https://issues.apache.org/jira/browse/SPARK-39417 # so avoid generating nulls for numeric partitions From 7c307d489b099bd1d73f8ec4b9b953a05ca24dec Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Tue, 12 Dec 2023 08:00:17 +0800 Subject: [PATCH 08/19] Support split broadcast join condition into ast and non-ast [databricks] (#9760) * Support split broadcast join condition into ast and non-ast Signed-off-by: Ferdinand Xu * Fix * Fix compile * Fix * Fix * Address comments * Fix Update all versions * Fix * Fix * Fix Spark 311 * Update sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExecBase.scala Co-authored-by: Jason Lowe * Address comments * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/AstUtil.scala Co-authored-by: Jason Lowe * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/AstUtil.scala Co-authored-by: Jason Lowe * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/AstUtil.scala Co-authored-by: Jason Lowe * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/AstUtil.scala Co-authored-by: Jason Lowe * Fix scala 2.13, code style, refactor * Minor fix * minor * Fix scala 2.13 * Fix DBX * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/AstUtil.scala Co-authored-by: Navin Kumar <97137715+NVnavkumar@users.noreply.github.com> * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/AstUtil.scala Co-authored-by: Jason Lowe * Fix scala 2.13, refactor * Revert unnecessary changes * Fix failed UT * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/AstUtil.scala Co-authored-by: Jason Lowe * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/AstUtil.scala Co-authored-by: Jason Lowe * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/AstUtil.scala Co-authored-by: Jason Lowe * Update sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala Co-authored-by: Jason Lowe * Update sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala Co-authored-by: Jason Lowe * Update sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala Co-authored-by: Jason Lowe * Update sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala Co-authored-by: Jason Lowe * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/AstUtil.scala Co-authored-by: Jason Lowe * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/AstUtil.scala Co-authored-by: Jason Lowe --------- Signed-off-by: Ferdinand Xu Co-authored-by: Jason Lowe Co-authored-by: Navin Kumar <97137715+NVnavkumar@users.noreply.github.com> --- .../src/main/python/join_test.py | 37 +++- .../com/nvidia/spark/rapids/AstUtil.scala | 166 +++++++++++++++++- .../GpuBroadcastHashJoinExecBase.scala | 24 +++ .../sql/rapids/execution/GpuHashJoin.scala | 110 ++++++++---- .../execution/GpuSubPartitionHashJoin.scala | 2 +- .../execution/GpuBroadcastHashJoinExec.scala | 36 ++-- .../execution/GpuBroadcastHashJoinExec.scala | 48 +++-- .../execution/GpuBroadcastHashJoinExec.scala | 35 ++-- 8 files changed, 374 insertions(+), 84 deletions(-) diff --git a/integration_tests/src/main/python/join_test.py b/integration_tests/src/main/python/join_test.py index 6660e663c92..9e7f5a13cb9 100644 --- a/integration_tests/src/main/python/join_test.py +++ b/integration_tests/src/main/python/join_test.py @@ -19,8 +19,9 @@ from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_fallback_collect, assert_cpu_and_gpu_are_equal_collect_with_capture from conftest import is_databricks_runtime, is_emr_runtime, is_not_utc from data_gen import * -from marks import ignore_order, allow_non_gpu, incompat, validate_execs_in_gpu_plan -from spark_session import with_cpu_session, is_before_spark_330, is_databricks_runtime +from marks import ignore_order, allow_non_gpu, incompat, validate_execs_in_gpu_plan, \ + datagen_overrides +from spark_session import with_cpu_session, is_before_spark_330, is_databricks113_or_later, is_databricks_runtime pytestmark = [pytest.mark.nightly_resource_consuming_test] @@ -434,6 +435,38 @@ def do_join(spark): return broadcast(left).join(right, left.a > f.log(right.r_a), join_type) assert_gpu_fallback_collect(do_join, 'BroadcastNestedLoopJoinExec') +# Allowing non Gpu for ShuffleExchangeExec is mainly for Databricks where its exchange is CPU based ('Exchange SinglePartition, EXECUTOR_BROADCAST'). +db_113_cpu_bhj_join_allow=["ShuffleExchangeExec"] if is_databricks113_or_later() else [] + + +@allow_non_gpu(*db_113_cpu_bhj_join_allow) +@ignore_order(local=True) +@pytest.mark.parametrize('data_gen', [IntegerGen(), LongGen()], ids=idfn) +@pytest.mark.parametrize('join_type', ['Left', 'Inner', 'LeftSemi', 'LeftAnti'], ids=idfn) +def test_broadcast_hash_join_on_non_ast_condition_without_fallback(data_gen, join_type): + # This is to test BHJ with a condition not fully supported by AST. With extra project nodes wrapped, join can still run on GPU other than fallback. + def do_join(spark): + left, right = create_df(spark, data_gen, 50, 25) + # AST does not support cast or logarithm yet + return left.join(right.hint("broadcast"), ((left.b == right.r_b) & (f.round(left.a).cast('integer') > f.round(f.log(right.r_a).cast('integer')))), join_type) + assert_gpu_and_cpu_are_equal_collect(do_join, conf = {"spark.rapids.sql.castFloatToIntegralTypes.enabled": True}) + + +@allow_non_gpu('BroadcastHashJoinExec', 'BroadcastExchangeExec') +@ignore_order(local=True) +@pytest.mark.parametrize('data_gen', [IntegerGen(), LongGen()], ids=idfn) +@pytest.mark.parametrize('join_type', ['Left', 'LeftSemi', 'LeftAnti'], ids=idfn) +def test_broadcast_hash_join_on_non_ast_condition_fallback(data_gen, join_type): + # This is to test BHJ with a condition not fully supported by AST. Since AST doesn't support double, this query fallback to CPU. + # Inner join is not included since it can be supported by GPU via a post filter. + def do_join(spark): + left, right = create_df(spark, data_gen, 50, 25) + # AST does not support cast or logarithm yet and also it's not able to be split as project + # node those both sides are involved in join condition + return left.join(right.hint("broadcast"), ((left.b == right.r_b) & (left.a.cast('double') > right.r_a.cast('double'))), join_type) + assert_gpu_fallback_collect(do_join, 'BroadcastHashJoinExec') + + @ignore_order(local=True) @pytest.mark.parametrize('data_gen', [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AstUtil.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AstUtil.scala index 5062d8e4a99..3f68f5d3d60 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AstUtil.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AstUtil.scala @@ -16,11 +16,15 @@ package com.nvidia.spark.rapids +import java.io.Serializable + +import com.nvidia.spark.rapids.Arm.withResource import scala.collection.mutable import scala.collection.mutable.ListBuffer -import org.apache.spark.sql.catalyst.expressions.{AttributeSeq, Expression, ExprId, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSeq, Expression, ExprId, NamedExpression} import org.apache.spark.sql.rapids.catalyst.expressions.{GpuEquivalentExpressions, GpuExpressionEquals} +import org.apache.spark.sql.vectorized.ColumnarBatch object AstUtil { @@ -119,4 +123,164 @@ object AstUtil { } } } + + /** + * Transforms the original join condition into extra filter/project when necessary. + * It's targeted for some cases join condition is not fully evaluated by ast. + * Based on join condition, it can be transformed into three major strategies: + * (1) [NoopJoinCondSplit]: noop when join condition can be fully evaluated with ast. + * (2) [JoinCondSplitAsPostFilter]: entire join condition is pulled out as a post filter + * after join condition. + * (3) [JoinCondSplitAsProject]: extract not supported join condition into pre-project nodes + * on each join child. One extra project node is introduced to remove intermediate attributes. + */ + abstract class JoinCondSplitStrategy(left: Seq[NamedExpression], + right: Seq[NamedExpression], buildSide: GpuBuildSide) extends Serializable { + + // Actual output of build/stream side project due to join condition split + private[this] val (buildOutputAttr, streamOutputAttr) = buildSide match { + case GpuBuildLeft => (joinLeftOutput, joinRightOutput) + case GpuBuildRight => (joinRightOutput, joinLeftOutput) + } + + // This is the left side child of join. In `split as project` strategy, it may be different + // from original left child with extracted join condition attribute. + def leftOutput(): Seq[NamedExpression] = left + + // This is the right side child of join. In `split as project` strategy, it may be different + // from original right child with extracted join condition attribute. + def rightOutput(): Seq[NamedExpression] = right + + def astCondition(): Option[Expression] + + def processBuildSideAndClose(input: ColumnarBatch): ColumnarBatch = input + + def processStreamSideAndClose(input: ColumnarBatch): ColumnarBatch = input + + def processPostJoin(iter: Iterator[ColumnarBatch]): Iterator[ColumnarBatch] = iter + + // This is the left side child of join. In `split as project` strategy, it may be different + // from original left child with extracted join condition attribute. + def joinLeftOutput(): Seq[Attribute] = leftOutput.map(expr => expr.toAttribute) + + // This is the right side child of join. In `split as project` strategy, it may be different + // from original right child with extracted join condition attribute. + def joinRightOutput(): Seq[Attribute] = rightOutput.map(expr => expr.toAttribute) + + // Updated build attribute list after join condition split as project node. + // It may include extra attributes from split join condition. + def buildSideOutput(): Seq[Attribute] = buildOutputAttr + + // Updated stream attribute list after join condition split as project node. + // It may include extra attributes from split join condition. + def streamedSideOutput(): Seq[Attribute] = streamOutputAttr + } + + // For the case entire join condition can be evaluated as ast. + case class NoopJoinCondSplit(condition: Option[Expression], left: Seq[NamedExpression], + right: Seq[NamedExpression], buildSide: GpuBuildSide) + extends JoinCondSplitStrategy(left, right, buildSide) { + override def astCondition(): Option[Expression] = condition + } + + // For inner joins we can apply a post-join condition for any conditions that cannot be + // evaluated directly in a mixed join that leverages a cudf AST expression. + case class JoinCondSplitAsPostFilter(expr: Option[Expression], + attributeSeq: Seq[Attribute], left: Seq[NamedExpression], + right: Seq[NamedExpression], buildSide: GpuBuildSide) + extends JoinCondSplitStrategy(left, right, buildSide) { + private[this] val postFilter = expr.map { e => + GpuBindReferences.bindGpuReferencesTiered( + Seq(e), attributeSeq, false) + } + + override def astCondition(): Option[Expression] = None + + override def processPostJoin(iter: Iterator[ColumnarBatch]): Iterator[ColumnarBatch] = { + postFilter.map { filter => + iter.flatMap { cb => + GpuFilter.filterAndClose(cb, filter, NoopMetric, NoopMetric, NoopMetric) + } + }.getOrElse(iter) + } + } + + /** + * This is the split strategy targeting on the case where ast not supported join condition can be + * extracted and wrapped into extra project node(s). + * + * @param astCond remained join condition after extracting ast not supported parts + * @param left original expressions from join's left child. It's left project input + * attribute. + * @param leftProj extra expressions extracted from original join condition which is not + * supported by ast. It will be evaluated as a project on left side batch. + * @param right original expressions from join's right child. It's left project input + * attribute. + * @param rightProj extra expressions extracted from original join condition which is not + * supported by ast. It will be evaluated as a project on right side batch. + * @param post eliminate the extra columns introduced by join condition split + * @param buildSide indicates which side is build + */ + case class JoinCondSplitAsProject( + astCond: Option[Expression], + left: Seq[NamedExpression], leftProj: Seq[NamedExpression], + right: Seq[NamedExpression], rightProj: Seq[NamedExpression], + post: Seq[NamedExpression], buildSide: GpuBuildSide + ) extends JoinCondSplitStrategy(left ++ leftProj, right ++ rightProj, buildSide) { + private[this] val leftInput = left.map(_.toAttribute) + private[this] val rightInput = right.map(_.toAttribute) + + // Used to build build/stream side project + private[this] val (buildOutput, streamOutput, buildInput, streamInput) = buildSide match { + case GpuBuildLeft => + (leftOutput, rightOutput, leftInput, rightInput) + case GpuBuildRight => + (rightOutput, leftOutput, rightInput, leftInput) + } + + private[this] val buildProj = if (!buildOutput.isEmpty) { + Some(GpuBindReferences.bindGpuReferencesTiered(buildOutput, buildInput, false)) + } else None + + private[this] val streamProj = if (!streamOutput.isEmpty) { + Some(GpuBindReferences.bindGpuReferencesTiered(streamOutput, streamInput, false)) + } else None + + // Remove the intermediate attributes from left and right side project nodes. Output attributes + // need to be updated based on join type. And its attributes covers both original plan and + // extra project node. + private[this] val postProj = if (!post.isEmpty) { + Some( + GpuBindReferences.bindGpuReferencesTiered( + post, (leftOutput ++ rightOutput).map(_.toAttribute), false)) + } else None + + override def astCondition(): Option[Expression] = astCond + + override def processBuildSideAndClose(input: ColumnarBatch): ColumnarBatch = { + buildProj.map { pj => + withResource(input) { cb => + pj.project(cb) + } + }.getOrElse(input) + } + + override def processStreamSideAndClose(input: ColumnarBatch): ColumnarBatch = { + streamProj.map { pj => + withResource(input) { cb => + pj.project(cb) + } + }.getOrElse(input) + } + + override def processPostJoin(iter: Iterator[ColumnarBatch]): Iterator[ColumnarBatch] = { + postProj.map { proj => + iter.map { cb => + withResource(cb) { b => + proj.project(b) + } + } + }.getOrElse(iter) + } + } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExecBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExecBase.scala index 4982c6e3c9c..7531223fba6 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExecBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExecBase.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.rapids.execution import ai.rapids.cudf.{NvtxColor, NvtxRange} import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} +import com.nvidia.spark.rapids.AstUtil.JoinCondSplitStrategy import com.nvidia.spark.rapids.shims.{GpuBroadcastJoinMeta, ShimBinaryExecNode} import org.apache.spark.TaskContext @@ -55,6 +56,28 @@ abstract class GpuBroadcastHashJoinMetaBase( override val childExprs: Seq[BaseExprMeta[_]] = leftKeys ++ rightKeys ++ conditionMeta + private var taggedForAstCheck = false + + // Avoid checking multiple times + private var isAstCond = false + + /** + * Check whether condition can be ast-able. It includes two cases: 1) all join conditions are + * ast-able; 2) join conditions are ast-able after split and push down to child plans. + */ + def canJoinCondAstAble(): Boolean = { + if (!taggedForAstCheck) { + val Seq(leftPlan, rightPlan) = childPlans + isAstCond = conditionMeta match { + case Some(e) => AstUtil.canExtractNonAstConditionIfNeed( + e, leftPlan.outputAttributes.map(_.exprId), rightPlan.outputAttributes.map(_.exprId)) + case None => true + } + taggedForAstCheck = true + } + isAstCond + } + override def tagPlanForGpu(): Unit = { GpuHashJoin.tagJoin(this, join.joinType, buildSide, join.leftKeys, join.rightKeys, conditionMeta) @@ -103,6 +126,7 @@ abstract class GpuBroadcastHashJoinExecBase( joinType: JoinType, buildSide: GpuBuildSide, override val condition: Option[Expression], + override val joinCondSplitStrategy: JoinCondSplitStrategy, left: SparkPlan, right: SparkPlan) extends ShimBinaryExecNode with GpuHashJoin { import GpuMetric._ diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala index cbaa1cbe47c..f86f75104a3 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala @@ -19,6 +19,7 @@ import ai.rapids.cudf.{ColumnView, DType, GatherMap, GroupByAggregation, NullEqu import ai.rapids.cudf.ast.CompiledExpression import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} +import com.nvidia.spark.rapids.AstUtil.{JoinCondSplitStrategy, NoopJoinCondSplit} import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableProducingSeq import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{withRestoreOnRetry, withRetryNoSplit} import com.nvidia.spark.rapids.jni.GpuOOM @@ -116,9 +117,11 @@ object GpuHashJoin { joinType match { case _: InnerLike => case RightOuter | LeftOuter | LeftSemi | LeftAnti | ExistenceJoin(_) => - conditionMeta.foreach(meta.requireAstForGpuOn) + // First to check whether can be split if not ast-able. If false, then check requireAst to + // send not-work-on-GPU reason if not replace-able. + conditionMeta.foreach(cond => if (!canJoinCondAstAble(meta)) meta.requireAstForGpuOn(cond)) case FullOuter => - conditionMeta.foreach(meta.requireAstForGpuOn) + conditionMeta.foreach(cond => if (!canJoinCondAstAble(meta)) meta.requireAstForGpuOn(cond)) // FullOuter join cannot support with struct keys as two issues below // * https://github.com/NVIDIA/spark-rapids/issues/2126 // * https://github.com/rapidsai/cudf/issues/7947 @@ -138,6 +141,15 @@ object GpuHashJoin { } } + // Check whether the entire tree is ast-able or being able to split non-ast-able conditions + // into child nodes. Now only support broad hash join. + private[this] def canJoinCondAstAble(meta: SparkPlanMeta[_]): Boolean = { + meta match { + case meta: GpuBroadcastHashJoinMeta => meta.canJoinCondAstAble + case _ => false + } + } + /** Determine if this type of join supports using the right side of the join as the build side. */ def canBuildRight(joinType: JoinType): Boolean = joinType match { case _: InnerLike | LeftOuter | LeftSemi | LeftAnti | FullOuter | _: ExistenceJoin => true @@ -254,6 +266,25 @@ object GpuHashJoin { keys.forall(_.dataType.isInstanceOf[IntegralType]) && keys.map(_.dataType.defaultSize).sum <= 8 } + + def output(joinType: JoinType, left: Seq[Attribute], right: Seq[Attribute]): Seq[Attribute] = { + joinType match { + case _: InnerLike => + left ++ right + case LeftOuter => + left ++ right.map(_.withNullability(true)) + case RightOuter => + left.map(_.withNullability(true)) ++ right + case j: ExistenceJoin => + left :+ j.exists + case LeftExistence(_) => + left + case FullOuter => + left.map(_.withNullability(true)) ++ right.map(_.withNullability(true)) + case x => + throw new IllegalArgumentException(s"GpuHashJoin should not take $x as the JoinType") + } + } } abstract class BaseHashJoinIterator( @@ -866,6 +897,8 @@ trait GpuHashJoin extends GpuExec { def leftKeys: Seq[Expression] def rightKeys: Seq[Expression] def buildSide: GpuBuildSide + def joinCondSplitStrategy: JoinCondSplitStrategy = NoopJoinCondSplit( + condition, left.output, right.output, buildSide) protected lazy val (buildPlan, streamedPlan) = buildSide match { case GpuBuildLeft => (left, right) @@ -885,22 +918,7 @@ trait GpuHashJoin extends GpuExec { } override def output: Seq[Attribute] = { - joinType match { - case _: InnerLike => - left.output ++ right.output - case LeftOuter => - left.output ++ right.output.map(_.withNullability(true)) - case RightOuter => - left.output.map(_.withNullability(true)) ++ right.output - case j: ExistenceJoin => - left.output :+ j.exists - case LeftExistence(_) => - left.output - case FullOuter => - left.output.map(_.withNullability(true)) ++ right.output.map(_.withNullability(true)) - case x => - throw new IllegalArgumentException(s"GpuHashJoin should not take $x as the JoinType") - } + GpuHashJoin.output(joinType, left.output, right.output) } // If we have a single batch streamed in then we will produce a single batch of output @@ -953,8 +971,10 @@ trait GpuHashJoin extends GpuExec { GpuHashJoin.anyNullableStructChild(buildKeys) protected lazy val (boundBuildKeys, boundStreamKeys) = { - val lkeys = GpuBindReferences.bindGpuReferences(leftKeys, left.output) - val rkeys = GpuBindReferences.bindGpuReferences(rightKeys, right.output) + val lkeys = + GpuBindReferences.bindGpuReferences(leftKeys, joinCondSplitStrategy.joinLeftOutput) + val rkeys = + GpuBindReferences.bindGpuReferences(rightKeys, joinCondSplitStrategy.joinRightOutput) buildSide match { case GpuBuildLeft => (lkeys, rkeys) @@ -963,14 +983,25 @@ trait GpuHashJoin extends GpuExec { } protected lazy val (numFirstConditionTableColumns, boundCondition) = { - val (joinLeft, joinRight) = joinType match { - case RightOuter => (right, left) - case _ => (left, right) + val joinLeft = joinType match { + case RightOuter => + if(buildSide == GpuBuildRight) { + joinCondSplitStrategy.buildSideOutput + } else { + joinCondSplitStrategy.streamedSideOutput + } + case _ => + if (buildSide == GpuBuildRight) { + joinCondSplitStrategy.streamedSideOutput + } else { + joinCondSplitStrategy.buildSideOutput + } } val boundCondition = condition.map { c => - GpuBindReferences.bindGpuReference(c, joinLeft.output ++ joinRight.output) + GpuBindReferences.bindGpuReference(c, + joinCondSplitStrategy.streamedSideOutput ++ joinCondSplitStrategy.buildSideOutput) } - (joinLeft.output.size, boundCondition) + (joinLeft.size, boundCondition) } def doJoin( @@ -994,13 +1025,14 @@ trait GpuHashJoin extends GpuExec { builtBatch } - val spillableBuiltBatch = withResource(nullFiltered) { + val spillableBuiltBatch = withResource(joinCondSplitStrategy + .processBuildSideAndClose(nullFiltered)) { LazySpillableColumnarBatch(_, "built") } val lazyStream = stream.map { cb => - withResource(cb) { cb => - LazySpillableColumnarBatch(cb, "stream_batch") + withResource(joinCondSplitStrategy.processStreamSideAndClose(cb)) { updatedBatch => + LazySpillableColumnarBatch(updatedBatch, "stream_batch") } } @@ -1019,25 +1051,29 @@ trait GpuHashJoin extends GpuExec { opTime, joinTime) case FullOuter => - new HashFullJoinIterator(spillableBuiltBatch, boundBuildKeys, lazyStream, - boundStreamKeys, streamedPlan.output, boundCondition, numFirstConditionTableColumns, - targetSize, buildSide, compareNullsEqual, opTime, joinTime) + new HashFullJoinIterator( + spillableBuiltBatch, boundBuildKeys, lazyStream, + boundStreamKeys, joinCondSplitStrategy.streamedSideOutput, boundCondition, + numFirstConditionTableColumns, targetSize, buildSide, compareNullsEqual, opTime, + joinTime) case _ => if (boundCondition.isDefined) { // ConditionalHashJoinIterator will close the compiled condition val compiledCondition = boundCondition.get.convertToAst(numFirstConditionTableColumns).compile() - new ConditionalHashJoinIterator(spillableBuiltBatch, boundBuildKeys, lazyStream, - boundStreamKeys, streamedPlan.output, compiledCondition, + new ConditionalHashJoinIterator( + spillableBuiltBatch, boundBuildKeys, lazyStream, + boundStreamKeys, joinCondSplitStrategy.streamedSideOutput, compiledCondition, targetSize, joinType, buildSide, compareNullsEqual, opTime, joinTime) } else { - new HashJoinIterator(spillableBuiltBatch, boundBuildKeys, lazyStream, boundStreamKeys, - streamedPlan.output, targetSize, joinType, buildSide, compareNullsEqual, - opTime, joinTime) + new HashJoinIterator( + spillableBuiltBatch, boundBuildKeys, lazyStream, boundStreamKeys, + joinCondSplitStrategy.streamedSideOutput, targetSize, joinType, buildSide, + compareNullsEqual, opTime, joinTime) } } - joinIterator.map { cb => + joinCondSplitStrategy.processPostJoin(joinIterator).map { cb => joinOutputRows += cb.numRows() numOutputRows += cb.numRows() numOutputBatches += 1 diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubPartitionHashJoin.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubPartitionHashJoin.scala index 33ca8c906f6..816aa4ac07d 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubPartitionHashJoin.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubPartitionHashJoin.scala @@ -614,7 +614,7 @@ trait GpuSubPartitionHashJoin extends Logging { self: GpuHashJoin => } } // Leverage the original join iterators - val joinIter = doJoin(buildCb, streamIter, targetSize, + val joinIter = doJoin(buildCb, streamIter, targetSize, numOutputRows, joinOutputRows, numOutputBatches, opTime, joinTime) Some(joinIter) } diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala index 0b1be70234b..cd640057a58 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala @@ -35,6 +35,7 @@ spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.AstUtil.{JoinCondSplitAsPostFilter, JoinCondSplitAsProject, JoinCondSplitStrategy, NoopJoinCondSplit} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType @@ -48,12 +49,6 @@ class GpuBroadcastHashJoinMeta( rule: DataFromReplacementRule) extends GpuBroadcastHashJoinMetaBase(join, conf, parent, rule) { override def convertToGpu(): GpuExec = { - val condition = conditionMeta.map(_.convertToGpu()) - val (joinCondition, filterCondition) = if (conditionMeta.forall(_.canThisBeAst)) { - (condition, None) - } else { - (None, condition) - } val Seq(left, right) = childPlans.map(_.convertIfNeeded()) // The broadcast part of this must be a BroadcastExchangeExec val buildSideMeta = buildSide match { @@ -61,16 +56,32 @@ class GpuBroadcastHashJoinMeta( case GpuBuildRight => right } verifyBuildSideWasReplaced(buildSideMeta) - val joinExec = GpuBroadcastHashJoinExec( + // First to check whether we can extract some non-supported AST conditions. If not, will do a + // post-join filter right after hash join node. Otherwise, do split as project. + val nonAstJoinCond = if (!canJoinCondAstAble()) { + JoinCondSplitAsPostFilter( + conditionMeta.map(_.convertToGpu()), GpuHashJoin.output( + join.joinType, left.output, right.output), left.output, right.output, buildSide) + } else { + val (remain, leftExpr, rightExpr) = AstUtil.extractNonAstFromJoinCond( + conditionMeta, left.output, right.output, true) + if (leftExpr.isEmpty && rightExpr.isEmpty) { + NoopJoinCondSplit(remain, left.output, right.output, buildSide) + } else { + JoinCondSplitAsProject( + remain, left.output, leftExpr, right.output, rightExpr, + GpuHashJoin.output(join.joinType, left.output, right.output), buildSide) + } + } + + GpuBroadcastHashJoinExec( leftKeys.map(_.convertToGpu()), rightKeys.map(_.convertToGpu()), join.joinType, buildSide, - joinCondition, + nonAstJoinCond.astCondition(), + nonAstJoinCond, left, right) - // For inner joins we can apply a post-join condition for any conditions that cannot be - // evaluated directly in a mixed join that leverages a cudf AST expression - filterCondition.map(c => GpuFilterExec(c, joinExec)()).getOrElse(joinExec) } } @@ -80,6 +91,7 @@ case class GpuBroadcastHashJoinExec( joinType: JoinType, buildSide: GpuBuildSide, override val condition: Option[Expression], + override val joinCondSplitStrategy: JoinCondSplitStrategy, left: SparkPlan, right: SparkPlan) extends GpuBroadcastHashJoinExecBase( - leftKeys, rightKeys, joinType, buildSide, condition, left, right) \ No newline at end of file + leftKeys, rightKeys, joinType, buildSide, condition, joinCondSplitStrategy, left, right) \ No newline at end of file diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala index ca4b0dfa31a..5db2c11c3b1 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala @@ -24,6 +24,7 @@ package org.apache.spark.sql.rapids.execution import ai.rapids.cudf.{NvtxColor, NvtxRange} import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} +import com.nvidia.spark.rapids.AstUtil.{JoinCondSplitAsPostFilter, JoinCondSplitAsProject, JoinCondSplitStrategy, NoopJoinCondSplit} import org.apache.spark.TaskContext import org.apache.spark.rapids.shims.GpuShuffleExchangeExec @@ -45,12 +46,6 @@ class GpuBroadcastHashJoinMeta( rule: DataFromReplacementRule) extends GpuBroadcastHashJoinMetaBase(join, conf, parent, rule) { override def convertToGpu(): GpuExec = { - val condition = conditionMeta.map(_.convertToGpu()) - val (joinCondition, filterCondition) = if (conditionMeta.forall(_.canThisBeAst)) { - (condition, None) - } else { - (None, condition) - } val Seq(left, right) = childPlans.map(_.convertIfNeeded()) // The broadcast part of this must be a BroadcastExchangeExec val buildSideMeta = buildSide match { @@ -58,18 +53,32 @@ class GpuBroadcastHashJoinMeta( case GpuBuildRight => right } verifyBuildSideWasReplaced(buildSideMeta) - val joinExec = GpuBroadcastHashJoinExec( + // First to check whether we can extract some non-supported AST conditions. If not, will do a + // post-join filter right after hash join node. Otherwise, do split as project. + val nonAstJoinCond = if (!canJoinCondAstAble()) { + JoinCondSplitAsPostFilter( + conditionMeta.map(_.convertToGpu()), GpuHashJoin.output( + join.joinType, left.output, right.output), left.output, right.output, buildSide) + } else { + val (remain, leftExpr, rightExpr) = AstUtil.extractNonAstFromJoinCond( + conditionMeta, left.output, right.output, true) + if (leftExpr.isEmpty && rightExpr.isEmpty) { + NoopJoinCondSplit(remain, left.output, right.output, buildSide) + } else { + JoinCondSplitAsProject( + remain, left.output, leftExpr, right.output, rightExpr, + GpuHashJoin.output(join.joinType, left.output, right.output), buildSide) + } + } + + GpuBroadcastHashJoinExec( leftKeys.map(_.convertToGpu()), rightKeys.map(_.convertToGpu()), join.joinType, buildSide, - joinCondition, - left, - right, - join.isExecutorBroadcast) - // For inner joins we can apply a post-join condition for any conditions that cannot be - // evaluated directly in a mixed join that leverages a cudf AST expression - filterCondition.map(c => GpuFilterExec(c, joinExec)()).getOrElse(joinExec) + nonAstJoinCond.astCondition(), + nonAstJoinCond, + left, right, join.isExecutorBroadcast) } } @@ -79,11 +88,12 @@ case class GpuBroadcastHashJoinExec( joinType: JoinType, buildSide: GpuBuildSide, override val condition: Option[Expression], + override val joinCondSplitStrategy: JoinCondSplitStrategy, left: SparkPlan, - right: SparkPlan, + right: SparkPlan, executorBroadcast: Boolean) - extends GpuBroadcastHashJoinExecBase( - leftKeys, rightKeys, joinType, buildSide, condition, left, right) { + extends GpuBroadcastHashJoinExecBase(leftKeys, rightKeys, joinType, buildSide, + condition, joinCondSplitStrategy, left, right) { import GpuMetric._ override lazy val additionalMetrics: Map[String, GpuMetric] = Map( @@ -147,8 +157,8 @@ case class GpuBroadcastHashJoinExec( GpuSemaphore.acquireIfNecessary(TaskContext.get()) } } - val buildBatch = GpuExecutorBroadcastHelper.getExecutorBroadcastBatch(buildRelation, - buildSchema, buildOutput, metricsMap, targetSize) + val buildBatch = GpuExecutorBroadcastHelper.getExecutorBroadcastBatch( + buildRelation, buildSchema, buildOutput, metricsMap, targetSize) (buildBatch, bufferedStreamIter) } } diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala index 4985d791829..e2a8bb51ba5 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala @@ -22,6 +22,7 @@ spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.AstUtil.{JoinCondSplitAsPostFilter, JoinCondSplitAsProject, JoinCondSplitStrategy, NoopJoinCondSplit} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.JoinType @@ -35,12 +36,6 @@ class GpuBroadcastHashJoinMeta( rule: DataFromReplacementRule) extends GpuBroadcastHashJoinMetaBase(join, conf, parent, rule) { override def convertToGpu(): GpuExec = { - val condition = conditionMeta.map(_.convertToGpu()) - val (joinCondition, filterCondition) = if (conditionMeta.forall(_.canThisBeAst)) { - (condition, None) - } else { - (None, condition) - } val Seq(left, right) = childPlans.map(_.convertIfNeeded()) // The broadcast part of this must be a BroadcastExchangeExec val buildSideMeta = buildSide match { @@ -48,16 +43,31 @@ class GpuBroadcastHashJoinMeta( case GpuBuildRight => right } verifyBuildSideWasReplaced(buildSideMeta) - val joinExec = GpuBroadcastHashJoinExec( + // First to check whether we can extract some non-supported AST conditions. If not, will do a + // post-join filter right after hash join node. Otherwise, do split as project. + val nonAstJoinCond = if (!canJoinCondAstAble()) { + JoinCondSplitAsPostFilter(conditionMeta.map(_.convertToGpu()), GpuHashJoin.output( + join.joinType, left.output, right.output), left.output, right.output, buildSide) + } else { + val (remain, leftExpr, rightExpr) = AstUtil.extractNonAstFromJoinCond( + conditionMeta, left.output, right.output, true) + if(leftExpr.isEmpty && rightExpr.isEmpty) { + NoopJoinCondSplit(remain, left.output, right.output, buildSide) + } else { + JoinCondSplitAsProject( + remain, left.output, leftExpr, right.output, rightExpr, + GpuHashJoin.output(join.joinType, left.output, right.output), buildSide) + } + } + + GpuBroadcastHashJoinExec( leftKeys.map(_.convertToGpu()), rightKeys.map(_.convertToGpu()), join.joinType, buildSide, - joinCondition, + nonAstJoinCond.astCondition(), + nonAstJoinCond, left, right) - // For inner joins we can apply a post-join condition for any conditions that cannot be - // evaluated directly in a mixed join that leverages a cudf AST expression - filterCondition.map(c => GpuFilterExec(c, joinExec)()).getOrElse(joinExec) } } @@ -67,6 +77,7 @@ case class GpuBroadcastHashJoinExec( joinType: JoinType, buildSide: GpuBuildSide, override val condition: Option[Expression], + override val joinCondSplitStrategy: JoinCondSplitStrategy, left: SparkPlan, right: SparkPlan) extends GpuBroadcastHashJoinExecBase( - leftKeys, rightKeys, joinType, buildSide, condition, left, right) + leftKeys, rightKeys, joinType, buildSide, condition, joinCondSplitStrategy, left, right) From e33f4008ee00e12f667d3c0bf28370bd3edea89c Mon Sep 17 00:00:00 2001 From: Peixin Date: Tue, 12 Dec 2023 08:49:04 +0800 Subject: [PATCH 09/19] Update private and JNI version to released 23.12.0 (#9954) Signed-off-by: Peixin Li --- pom.xml | 4 ++-- scala2.13/pom.xml | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index e8086a35d06..765c51ff3e4 100644 --- a/pom.xml +++ b/pom.xml @@ -660,8 +660,8 @@ spark${buildver} cuda11 ${cuda.version} - 23.12.0-SNAPSHOT - 23.12.0-SNAPSHOT + 23.12.0 + 23.12.0 2.12 2.8.0 incremental diff --git a/scala2.13/pom.xml b/scala2.13/pom.xml index e065c698522..6ca4cdf7e65 100644 --- a/scala2.13/pom.xml +++ b/scala2.13/pom.xml @@ -660,8 +660,8 @@ spark${buildver} cuda11 ${cuda.version} - 23.12.0-SNAPSHOT - 23.12.0-SNAPSHOT + 23.12.0 + 23.12.0 2.13 2.8.0 incremental From 1b7940e3a582056ae9e34abdd636082d76d1db70 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Mon, 11 Dec 2023 19:04:10 -0600 Subject: [PATCH 10/19] Add documentation for how to run tests with a fixed datagen seed (#10014) Signed-off-by: Jason Lowe --- integration_tests/README.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/integration_tests/README.md b/integration_tests/README.md index af203f44ad9..f45564c6130 100644 --- a/integration_tests/README.md +++ b/integration_tests/README.md @@ -330,6 +330,19 @@ Basically, you need first to upload the test resources onto the cloud path `reso `root-dir` of each executor(e.g. via `spark-submit --files root-dir ...`). After that you must set both `LOCAL_ROOTDIR=root-dir` and `INPUT_PATH=resource-path` to run the shell-script, e.g. `LOCAL_ROOTDIR=root-dir INPUT_PATH=resource-path bash [run_pyspark_from_build.sh](run_pyspark_from_build.sh)`. +### Running with a fixed data generation seed + +By default the tests are run with a different random data generator seed to increase the chance of +uncovering bugs due to specific inputs. The seed used for a test is printed as part of the test +name, see the `DATAGEN_SEED=` part of the test name printed as tests are run. If a problem is found +with a specific data generation seed, the seed can be set explicitly when running the tests by +exporting the `DATAGEN_SEED` environment variable to the desired seed before running the +integration tests. For example: + +```shell +$ DATAGEN_SEED=1702166057 SPARK_HOME=~/spark-3.4.0-bin-hadoop3 integration_tests/run_pyspark_from_build.sh +``` + ### Reviewing integration tests in Spark History Server If the integration tests are run using [run_pyspark_from_build.sh](run_pyspark_from_build.sh) we have From e8256f78a791544640e311dfa147488aefe44927 Mon Sep 17 00:00:00 2001 From: Suraj Aralihalli Date: Tue, 12 Dec 2023 06:45:22 +0530 Subject: [PATCH 11/19] [DOC] Update docs for 23.12.0 release [skip ci] (#9943) * update docs for 23.12 release Signed-off-by: Suraj Aralihalli * update docs for 23.12 release Signed-off-by: Suraj Aralihalli * drop pascal support - docs Signed-off-by: Suraj Aralihalli * update download docs Signed-off-by: Suraj Aralihalli * update download docs Signed-off-by: Suraj Aralihalli * add scala 2.13 support Signed-off-by: Suraj Aralihalli * address feedback for release notes Signed-off-by: Suraj Aralihalli * address feedback for release notes Signed-off-by: Suraj Aralihalli * update verify signature Signed-off-by: Suraj Aralihalli * jars -> jar Signed-off-by: Suraj Aralihalli --------- Signed-off-by: Suraj Aralihalli --- CONTRIBUTING.md | 8 ++--- docs/archive.md | 80 +++++++++++++++++++++++++++++++++++++++++++++ docs/dev/testing.md | 4 +-- docs/download.md | 51 ++++++++++++++++------------- 4 files changed, 115 insertions(+), 28 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 1cc52e5472a..f832cd2facd 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -130,15 +130,15 @@ mvn -pl dist -PnoSnapshots package -DskipTests Verify that shim-specific classes are hidden from a conventional classloader. ```bash -$ javap -cp dist/target/rapids-4-spark_2.12-23.10.0-SNAPSHOT-cuda11.jar com.nvidia.spark.rapids.shims.SparkShimImpl +$ javap -cp dist/target/rapids-4-spark_2.12-23.12.0-SNAPSHOT-cuda11.jar com.nvidia.spark.rapids.shims.SparkShimImpl Error: class not found: com.nvidia.spark.rapids.shims.SparkShimImpl ``` However, its bytecode can be loaded if prefixed with `spark3XY` not contained in the package name ```bash -$ javap -cp dist/target/rapids-4-spark_2.12-23.10.0-SNAPSHOT-cuda11.jar spark320.com.nvidia.spark.rapids.shims.SparkShimImpl | head -2 -Warning: File dist/target/rapids-4-spark_2.12-23.10.0-SNAPSHOT-cuda11.jar(/spark320/com/nvidia/spark/rapids/shims/SparkShimImpl.class) does not contain class spark320.com.nvidia.spark.rapids.shims.SparkShimImpl +$ javap -cp dist/target/rapids-4-spark_2.12-23.12.0-SNAPSHOT-cuda11.jar spark320.com.nvidia.spark.rapids.shims.SparkShimImpl | head -2 +Warning: File dist/target/rapids-4-spark_2.12-23.12.0-SNAPSHOT-cuda11.jar(/spark320/com/nvidia/spark/rapids/shims/SparkShimImpl.class) does not contain class spark320.com.nvidia.spark.rapids.shims.SparkShimImpl Compiled from "SparkShims.scala" public final class com.nvidia.spark.rapids.shims.SparkShimImpl { ``` @@ -181,7 +181,7 @@ mvn package -pl dist -am -Dbuildver=340 -DallowConventionalDistJar=true Verify `com.nvidia.spark.rapids.shims.SparkShimImpl` is conventionally loadable: ```bash -$ javap -cp dist/target/rapids-4-spark_2.12-23.10.0-SNAPSHOT-cuda11.jar com.nvidia.spark.rapids.shims.SparkShimImpl | head -2 +$ javap -cp dist/target/rapids-4-spark_2.12-23.12.0-SNAPSHOT-cuda11.jar com.nvidia.spark.rapids.shims.SparkShimImpl | head -2 Compiled from "SparkShims.scala" public final class com.nvidia.spark.rapids.shims.SparkShimImpl { ``` diff --git a/docs/archive.md b/docs/archive.md index dae04b46bb0..83108f7e200 100644 --- a/docs/archive.md +++ b/docs/archive.md @@ -5,6 +5,86 @@ nav_order: 15 --- Below are archived releases for RAPIDS Accelerator for Apache Spark. +## Release v23.10.0 +### Hardware Requirements: + +The plugin is tested on the following architectures: + + GPU Models: NVIDIA P100, V100, T4, A10/A100, L4 and H100 GPUs + +### Software Requirements: + + OS: Ubuntu 20.04, Ubuntu 22.04, CentOS 7, or Rocky Linux 8 + + NVIDIA Driver*: R470+ + + Runtime: + Scala 2.12 + Python, Java Virtual Machine (JVM) compatible with your spark-version. + + * Check the Spark documentation for Python and Java version compatibility with your specific + Spark version. For instance, visit `https://spark.apache.org/docs/3.4.1` for Spark 3.4.1. + Please be aware that we do not currently support Spark builds with Scala 2.13. + + Supported Spark versions: + Apache Spark 3.2.0, 3.2.1, 3.2.2, 3.2.3, 3.2.4 + Apache Spark 3.3.0, 3.3.1, 3.3.2, 3.3.3 + Apache Spark 3.4.0, 3.4.1 + Apache Spark 3.5.0 + + Supported Databricks runtime versions for Azure and AWS: + Databricks 10.4 ML LTS (GPU, Scala 2.12, Spark 3.2.1) + Databricks 11.3 ML LTS (GPU, Scala 2.12, Spark 3.3.0) + Databricks 12.2 ML LTS (GPU, Scala 2.12, Spark 3.3.2) + + Supported Dataproc versions: + GCP Dataproc 2.0 + GCP Dataproc 2.1 + +*Some hardware may have a minimum driver version greater than R470. Check the GPU spec sheet +for your hardware's minimum driver version. + +*For Cloudera and EMR support, please refer to the +[Distributions](https://docs.nvidia.com/spark-rapids/user-guide/latest/faq.html#which-distributions-are-supported) section of the FAQ. + +#### RAPIDS Accelerator's Support Policy for Apache Spark +The RAPIDS Accelerator maintains support for Apache Spark versions available for download from [Apache Spark](https://spark.apache.org/downloads.html) + +### Download v23.10.0 +* Download the [RAPIDS + Accelerator for Apache Spark 23.10.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/23.10.0/rapids-4-spark_2.12-23.10.0.jar) + +This package is built against CUDA 11.8. It is tested on V100, T4, A10, A100, L4 and H100 GPUs with +CUDA 11.8 through CUDA 12.0. + +### Verify signature +* Download the [RAPIDS Accelerator for Apache Spark 23.10.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/23.10.0/rapids-4-spark_2.12-23.10.0.jar) + and [RAPIDS Accelerator for Apache Spark 23.10.0 jars.asc](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/23.10.0/rapids-4-spark_2.12-23.10.0.jar.asc) +* Download the [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com). +* Import the public key: `gpg --import PUB_KEY` +* Verify the signature: `gpg --verify rapids-4-spark_2.12-23.10.0.jar.asc rapids-4-spark_2.12-23.10.0.jar` + +The output of signature verify: + + gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) " + +### Release Notes +New functionality and performance improvements for this release include: +* Introduced support for Spark 3.5.0. +* Improved memory management for better control in YARN and K8s on CSP. +* Strengthened Parquet and ORC tests for enhanced stability and support. +* Reduce GPU out-of-memory (OOM) occurrences. +* Enhanced driver log with actionable insights. +* Qualification and Profiling tool: + * Enhanced user experience with the availability of the 'ascli' tool for qualification and + profiling across all platforms. + * The qualification tool now accommodates CPU-fallback transitions and broadens the speedup factor coverage. + * Extended diagnostic support for user tools to cover EMR, Databricks AWS, and Databricks Azure. + * Introduced support for cluster configuration recommendations in the profiling tool for supported platforms. + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + ## Release v23.08.2 ### Hardware Requirements: diff --git a/docs/dev/testing.md b/docs/dev/testing.md index 9d92ae4aacf..318d3d0584e 100644 --- a/docs/dev/testing.md +++ b/docs/dev/testing.md @@ -5,5 +5,5 @@ nav_order: 2 parent: Developer Overview --- An overview of testing can be found within the repository at: -* [Unit tests](https://github.com/NVIDIA/spark-rapids/tree/branch-23.10/tests#readme) -* [Integration testing](https://github.com/NVIDIA/spark-rapids/tree/branch-23.10/integration_tests#readme) +* [Unit tests](https://github.com/NVIDIA/spark-rapids/tree/branch-23.12/tests#readme) +* [Integration testing](https://github.com/NVIDIA/spark-rapids/tree/branch-23.12/integration_tests#readme) diff --git a/docs/download.md b/docs/download.md index 1c7e26fc090..e68af9c65ae 100644 --- a/docs/download.md +++ b/docs/download.md @@ -18,12 +18,12 @@ cuDF jar, that is either preinstalled in the Spark classpath on all nodes or sub that uses the RAPIDS Accelerator For Apache Spark. See the [getting-started guide](https://docs.nvidia.com/spark-rapids/user-guide/latest/getting-started/overview.html) for more details. -## Release v23.10.0 +## Release v23.12.0 ### Hardware Requirements: The plugin is tested on the following architectures: - GPU Models: NVIDIA P100, V100, T4, A10/A100, L4 and H100 GPUs + GPU Models: NVIDIA V100, T4, A10/A100, L4 and H100 GPUs ### Software Requirements: @@ -32,12 +32,11 @@ The plugin is tested on the following architectures: NVIDIA Driver*: R470+ Runtime: - Scala 2.12 + Scala 2.12, 2.13 Python, Java Virtual Machine (JVM) compatible with your spark-version. * Check the Spark documentation for Python and Java version compatibility with your specific - Spark version. For instance, visit `https://spark.apache.org/docs/3.4.1` for Spark 3.4.1. - Please be aware that we do not currently support Spark builds with Scala 2.13. + Spark version. For instance, visit `https://spark.apache.org/docs/3.4.1` for Spark 3.4.1. Supported Spark versions: Apache Spark 3.2.0, 3.2.1, 3.2.2, 3.2.3, 3.2.4 @@ -53,6 +52,9 @@ The plugin is tested on the following architectures: Supported Dataproc versions: GCP Dataproc 2.0 GCP Dataproc 2.1 + + Supported Dataproc Serverless versions: + Spark runtime 1.1 LTS *Some hardware may have a minimum driver version greater than R470. Check the GPU spec sheet for your hardware's minimum driver version. @@ -60,22 +62,28 @@ for your hardware's minimum driver version. *For Cloudera and EMR support, please refer to the [Distributions](https://docs.nvidia.com/spark-rapids/user-guide/latest/faq.html#which-distributions-are-supported) section of the FAQ. -#### RAPIDS Accelerator's Support Policy for Apache Spark +### RAPIDS Accelerator's Support Policy for Apache Spark The RAPIDS Accelerator maintains support for Apache Spark versions available for download from [Apache Spark](https://spark.apache.org/downloads.html) -### Download v23.10.0 -* Download the [RAPIDS - Accelerator for Apache Spark 23.10.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/23.10.0/rapids-4-spark_2.12-23.10.0.jar) +### Download RAPIDS Accelerator for Apache Spark v23.12.0 +- **Scala 2.12:** + - [RAPIDS Accelerator for Apache Spark 23.12.0 - Scala 2.12 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/23.12.0/rapids-4-spark_2.12-23.12.0.jar) + - [RAPIDS Accelerator for Apache Spark 23.12.0 - Scala 2.12 jar.asc](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/23.12.0/rapids-4-spark_2.12-23.12.0.jar.asc) + +- **Scala 2.13:** + - [RAPIDS Accelerator for Apache Spark 23.12.0 - Scala 2.13 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/23.12.0/rapids-4-spark_2.13-23.12.0.jar) + - [RAPIDS Accelerator for Apache Spark 23.12.0 - Scala 2.13 jar.asc](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/23.12.0/rapids-4-spark_2.13-23.12.0.jar.asc) This package is built against CUDA 11.8. It is tested on V100, T4, A10, A100, L4 and H100 GPUs with CUDA 11.8 through CUDA 12.0. ### Verify signature -* Download the [RAPIDS Accelerator for Apache Spark 23.10.0 jar](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/23.10.0/rapids-4-spark_2.12-23.10.0.jar) - and [RAPIDS Accelerator for Apache Spark 23.10.0 jars.asc](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/23.10.0/rapids-4-spark_2.12-23.10.0.jar.asc) * Download the [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com). * Import the public key: `gpg --import PUB_KEY` -* Verify the signature: `gpg --verify rapids-4-spark_2.12-23.10.0.jar.asc rapids-4-spark_2.12-23.10.0.jar` +* Verify the signature for Scala 2.12 jar: + `gpg --verify rapids-4-spark_2.12-23.12.0.jar.asc rapids-4-spark_2.12-23.12.0.jar` +* Verify the signature for Scala 2.13 jar: + `gpg --verify rapids-4-spark_2.13-23.12.0.jar.asc rapids-4-spark_2.13-23.12.0.jar` The output of signature verify: @@ -83,17 +91,16 @@ The output of signature verify: ### Release Notes New functionality and performance improvements for this release include: -* Introduced support for Spark 3.5.0. -* Improved memory management for better control in YARN and K8s on CSP. -* Strengthened Parquet and ORC tests for enhanced stability and support. -* Reduce GPU out-of-memory (OOM) occurrences. -* Enhanced driver log with actionable insights. +* Introduced support for chunked reading of ORC files. +* Enhanced support for additional time zones and added stack function support. +* Enhanced performance for join and aggregation operations. +* Kernel optimizations have been implemented to improve Parquet read performance. +* RAPIDS Accelerator also built and tested with Scala 2.13. +* Last version to support Pascal-based Nvidia GPUs; discontinued in the next release. * Qualification and Profiling tool: - * Enhanced user experience with the availability of the 'ascli' tool for qualification and - profiling across all platforms. - * The qualification tool now accommodates CPU-fallback transitions and broadens the speedup factor coverage. - * Extended diagnostic support for user tools to cover EMR, Databricks AWS, and Databricks Azure. - * Introduced support for cluster configuration recommendations in the profiling tool for supported platforms. + * Profiling Tool now processes Spark Driver log for GPU runs, enhancing feature analysis. + * Auto-tuner recommendations include AQE settings for optimized performance. + * New configurations in Profiler for enabling off-default features: udfCompiler, incompatibleDateFormats, hasExtendedYearValues. For a detailed list of changes, please refer to the [CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). From e51c412f9f3e79c2dc839e64a7b46363344b9b64 Mon Sep 17 00:00:00 2001 From: Peixin Date: Tue, 12 Dec 2023 09:27:10 +0800 Subject: [PATCH 12/19] Init 23.12 changelog [skip ci] (#9935) * Initiate 23.12 changelog Signed-off-by: Peixin Li * re-generated at 12/05 * re-generate at 12/06 * re-generated at 12/11 Signed-off-by: Peixin Li * re-generated at 12/12 Signed-off-by: Peixin Li * to include latest doc PRs --------- Signed-off-by: Peixin Li --- CHANGELOG.md | 536 ++++++++++-------- ...o_23.06.md => CHANGELOG_23.02_to_23.08.md} | 236 ++++++++ 2 files changed, 535 insertions(+), 237 deletions(-) rename docs/archives/{CHANGELOG_23.02_to_23.06.md => CHANGELOG_23.02_to_23.08.md} (76%) diff --git a/CHANGELOG.md b/CHANGELOG.md index c1c0f82ce50..3e6f5c0fdce 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,293 @@ # Change log -Generated on 2023-10-24 +Generated on 2023-12-12 + +## Release 23.12 + +### Features +||| +|:---|:---| +|[#6832](https://github.com/NVIDIA/spark-rapids/issues/6832)|[FEA] Convert Timestamp/Timezone tests/checks to be per operator instead of generic | +|[#9805](https://github.com/NVIDIA/spark-rapids/issues/9805)|[FEA] Support ```current_date``` expression function with CST (UTC + 8) timezone support| +|[#9515](https://github.com/NVIDIA/spark-rapids/issues/9515)|[FEA] Support temporal types in to_json| +|[#9872](https://github.com/NVIDIA/spark-rapids/issues/9872)|[FEA][JSON] Support Decimal type in `to_json`| +|[#9802](https://github.com/NVIDIA/spark-rapids/issues/9802)|[FEA] Support FromUTCTimestamp on the GPU with a non-UTC timezone| +|[#6831](https://github.com/NVIDIA/spark-rapids/issues/6831)|[FEA] Support timestamp transitions to and from UTC for single time zones with no repeating rules| +|[#9590](https://github.com/NVIDIA/spark-rapids/issues/9590)|[FEA][JSON] Support temporal types in `from_json`| +|[#9804](https://github.com/NVIDIA/spark-rapids/issues/9804)|[FEA] Support CPU path for from_utc_timestamp function with timezone| +|[#9461](https://github.com/NVIDIA/spark-rapids/issues/9461)|[FEA] Validate nvcomp-3.0 with spark rapids plugin| +|[#8832](https://github.com/NVIDIA/spark-rapids/issues/8832)|[FEA] rewrite join conditions where only part of it can fit on the AST| +|[#9059](https://github.com/NVIDIA/spark-rapids/issues/9059)|[FEA] Support spark.sql.parquet.datetimeRebaseModeInRead=LEGACY| +|[#9037](https://github.com/NVIDIA/spark-rapids/issues/9037)|[FEA] Support spark.sql.parquet.int96RebaseModeInWrite= LEGACY| +|[#9632](https://github.com/NVIDIA/spark-rapids/issues/9632)|[FEA] Take into account `org.apache.spark.timeZone` in Parquet/Avro from Spark 3.2| +|[#8770](https://github.com/NVIDIA/spark-rapids/issues/8770)|[FEA] add more metrics to Eventlogs or Executor logs| +|[#9597](https://github.com/NVIDIA/spark-rapids/issues/9597)|[FEA][JSON] Support boolean type in `from_json`| +|[#9516](https://github.com/NVIDIA/spark-rapids/issues/9516)|[FEA] Add support for JSON data source option `ignoreNullFields=false` in `to_json`| +|[#9520](https://github.com/NVIDIA/spark-rapids/issues/9520)|[FEA] Add support for `LAST()` as running window function| +|[#9518](https://github.com/NVIDIA/spark-rapids/issues/9518)|[FEA] Add support for relevant JSON data source options in `to_json`| +|[#9218](https://github.com/NVIDIA/spark-rapids/issues/9218)|[FEA] Support stack function| +|[#9532](https://github.com/NVIDIA/spark-rapids/issues/9532)|[FEA] Support Delta Lake 2.3.0| +|[#1525](https://github.com/NVIDIA/spark-rapids/issues/1525)|[FEA] Support Scala 2.13| +|[#7279](https://github.com/NVIDIA/spark-rapids/issues/7279)|[FEA] Support OverwriteByExpressionExecV1 for Delta Lake| +|[#9326](https://github.com/NVIDIA/spark-rapids/issues/9326)|[FEA] Specify `recover_with_null` when reading JSON files| +|[#8780](https://github.com/NVIDIA/spark-rapids/issues/8780)|[FEA] Support to_json function| +|[#7278](https://github.com/NVIDIA/spark-rapids/issues/7278)|[FEA] Support AppendDataExecV1 for Delta Lake| +|[#6266](https://github.com/NVIDIA/spark-rapids/issues/6266)|[FEA] Support Percentile| +|[#7277](https://github.com/NVIDIA/spark-rapids/issues/7277)|[FEA] Support AtomicReplaceTableAsSelect for Delta Lake| +|[#7276](https://github.com/NVIDIA/spark-rapids/issues/7276)|[FEA] Support AtomicCreateTableAsSelect for Delta Lake| + +### Performance +||| +|:---|:---| +|[#8137](https://github.com/NVIDIA/spark-rapids/issues/8137)|[FEA] Upgrade to UCX 1.15| +|[#8157](https://github.com/NVIDIA/spark-rapids/issues/8157)|[FEA] Add string comparison to AST expressions| +|[#9398](https://github.com/NVIDIA/spark-rapids/issues/9398)|[FEA] Compress/encrypt spill to disk| + +### Bugs Fixed +||| +|:---|:---| +|[#9687](https://github.com/NVIDIA/spark-rapids/issues/9687)|[BUG] `test_in_set` fails when DATAGEN_SEED=1698940723| +|[#9659](https://github.com/NVIDIA/spark-rapids/issues/9659)|[BUG] executor crash intermittantly in scala2.13-built spark332 integration tests| +|[#9923](https://github.com/NVIDIA/spark-rapids/issues/9923)|[BUG] Failed case about ```test_timestamp_seconds_rounding_necessary[Decimal(20,7)][DATAGEN_SEED=1701412018] – src.main.python.date_time_test```| +|[#9982](https://github.com/NVIDIA/spark-rapids/issues/9982)|[BUG] test "convert large InternalRow iterator to cached batch single col" failed with arena pool| +|[#9683](https://github.com/NVIDIA/spark-rapids/issues/9683)|[BUG] test_map_scalars_supported_key_types fails with DATAGEN_SEED=1698940723| +|[#9976](https://github.com/NVIDIA/spark-rapids/issues/9976)|[BUG] test_part_write_round_trip[Float] Failed on -0.0 partition| +|[#9948](https://github.com/NVIDIA/spark-rapids/issues/9948)|[BUG] parquet reader data corruption in nested schema after https://github.com/rapidsai/cudf/pull/13302| +|[#9867](https://github.com/NVIDIA/spark-rapids/issues/9867)|[BUG] Unable to use Spark Rapids with Spark Thrift Server| +|[#9934](https://github.com/NVIDIA/spark-rapids/issues/9934)|[BUG] test_delta_multi_part_write_round_trip_unmanaged and test_delta_part_write_round_trip_unmanaged failed DATA_SEED=1701608331 | +|[#9933](https://github.com/NVIDIA/spark-rapids/issues/9933)|[BUG] collection_ops_test.py::test_sequence_too_long_sequence[Long(not_null)][DATAGEN_SEED=1701553915, INJECT_OOM]| +|[#9837](https://github.com/NVIDIA/spark-rapids/issues/9837)|[BUG] test_part_write_round_trip failed| +|[#9932](https://github.com/NVIDIA/spark-rapids/issues/9932)|[BUG] Failed test_multi_tier_ast[DATAGEN_SEED=1701445668] on CI| +|[#9829](https://github.com/NVIDIA/spark-rapids/issues/9829)|[BUG] Java OOM when testing non-UTC time zone with lots of cases fallback.| +|[#9403](https://github.com/NVIDIA/spark-rapids/issues/9403)|[BUG] test_cogroup_apply_udf[Short(not_null)] failed with pandas 2.1.X| +|[#9684](https://github.com/NVIDIA/spark-rapids/issues/9684)|[BUG] test_coalesce fails with DATAGEN_SEED=1698940723| +|[#9685](https://github.com/NVIDIA/spark-rapids/issues/9685)|[BUG] test_case_when fails with DATAGEN_SEED=1698940723| +|[#9776](https://github.com/NVIDIA/spark-rapids/issues/9776)|[BUG] fastparquet compatibility tests fail with data mismatch if TZ is not set and system timezone is not UTC| +|[#9733](https://github.com/NVIDIA/spark-rapids/issues/9733)|[BUG] Complex AST expressions can crash with non-matching operand type error| +|[#9877](https://github.com/NVIDIA/spark-rapids/issues/9877)|[BUG] Fix resource leak in to_json| +|[#9722](https://github.com/NVIDIA/spark-rapids/issues/9722)|[BUG] test_floor_scale_zero fails with DATAGEN_SEED=1700009407| +|[#9846](https://github.com/NVIDIA/spark-rapids/issues/9846)|[BUG] test_ceil_scale_zero may fail with different datagen_seed| +|[#9781](https://github.com/NVIDIA/spark-rapids/issues/9781)|[BUG] test_cast_string_date_valid_format fails on DATAGEN_SEED=1700250017| +|[#9714](https://github.com/NVIDIA/spark-rapids/issues/9714)|Scala Map class not found when executing the benchmark on Spark 3.5.0 with Scala 2.13| +|[#9856](https://github.com/NVIDIA/spark-rapids/issues/9856)|collection_ops_test.py failed on Dataproc-2.1 with: Column 'None' does not exist| +|[#9397](https://github.com/NVIDIA/spark-rapids/issues/9397)|[BUG] RapidsShuffleManager MULTITHREADED on Databricks, we see loss of executors due to Rpc issues| +|[#9738](https://github.com/NVIDIA/spark-rapids/issues/9738)|[BUG] `test_delta_part_write_round_trip_unmanaged` and `test_delta_multi_part_write_round_trip_unmanaged` fail with `DATAGEN_SEED=1700105176`| +|[#9771](https://github.com/NVIDIA/spark-rapids/issues/9771)|[BUG] ast_test.py::test_X[(String, True)][DATAGEN_SEED=1700205785] failed| +|[#9782](https://github.com/NVIDIA/spark-rapids/issues/9782)|[BUG] Error messages appear in a clean build| +|[#9798](https://github.com/NVIDIA/spark-rapids/issues/9798)|[BUG] GpuCheckOverflowInTableInsert should be added to databricks shim| +|[#9820](https://github.com/NVIDIA/spark-rapids/issues/9820)|[BUG] test_parquet_write_roundtrip_datetime_with_legacy_rebase fails with "year 0 is out of range"| +|[#9817](https://github.com/NVIDIA/spark-rapids/issues/9817)|[BUG] FAILED dpp_test.py::test_dpp_reuse_broadcast_exchange[false-0-parquet][DATAGEN_SEED=1700572856, IGNORE_ORDER]| +|[#9768](https://github.com/NVIDIA/spark-rapids/issues/9768)|[BUG] `cast decimal to string` ScalaTest relies on a side effects | +|[#9711](https://github.com/NVIDIA/spark-rapids/issues/9711)|[BUG] test_lte fails with DATAGEN_SEED=1699987762| +|[#9751](https://github.com/NVIDIA/spark-rapids/issues/9751)|[BUG] cmp_test test_gte failed with DATAGEN_SEED=1700149611| +|[#9469](https://github.com/NVIDIA/spark-rapids/issues/9469)|[BUG] [main] ERROR com.nvidia.spark.rapids.GpuOverrideUtil - Encountered an exception applying GPU overrides java.lang.IllegalStateException: the broadcast must be on the GPU too| +|[#9648](https://github.com/NVIDIA/spark-rapids/issues/9648)|[BUG] Existence default values in schema are not being honored| +|[#9676](https://github.com/NVIDIA/spark-rapids/issues/9676)|Fix Delta Lake Integration tests; `test_delta_atomic_create_table_as_select` and `test_delta_atomic_replace_table_as_select`| +|[#9701](https://github.com/NVIDIA/spark-rapids/issues/9701)|[BUG] test_ts_formats_round_trip and test_datetime_roundtrip_with_legacy_rebase fail with DATAGEN_SEED=1699915317| +|[#9691](https://github.com/NVIDIA/spark-rapids/issues/9691)|[BUG] Repeated Maven invocations w/o changes recompile too many Scala sources despite recompileMode=incremental | +|[#9547](https://github.com/NVIDIA/spark-rapids/issues/9547)|Update buildall and doc to generate bloop projects for test debugging| +|[#9697](https://github.com/NVIDIA/spark-rapids/issues/9697)|[BUG] Iceberg multiple file readers can not read files if the file paths contain encoded URL unsafe chars| +|[#9681](https://github.com/NVIDIA/spark-rapids/issues/9681)|Databricks Build Failing For 330db+| +|[#9521](https://github.com/NVIDIA/spark-rapids/issues/9521)|[BUG] Multi Threaded Shuffle Writer needs flow control| +|[#9675](https://github.com/NVIDIA/spark-rapids/issues/9675)|Failing Delta Lake Tests for Databricks 13.3 Due to WriteIntoDeltaCommand| +|[#9669](https://github.com/NVIDIA/spark-rapids/issues/9669)|[BUG] Rebase exception states not in UTC but timezone is Etc/UTC| +|[#7940](https://github.com/NVIDIA/spark-rapids/issues/7940)|[BUG] UCX peer connection issue in multi-nic single node cluster| +|[#9650](https://github.com/NVIDIA/spark-rapids/issues/9650)|[BUG] Github workflow for missing scala2.13 updates fails to detect when pom is new| +|[#9621](https://github.com/NVIDIA/spark-rapids/issues/9621)|[BUG] Scala 2.13 with-classifier profile is picking up Scala2.12 spark.version| +|[#9636](https://github.com/NVIDIA/spark-rapids/issues/9636)|[BUG] All parquet integration tests failed "Part of the plan is not columnar class" in databricks runtimes| +|[#9108](https://github.com/NVIDIA/spark-rapids/issues/9108)|[BUG] nullability on some decimal operations is wrong| +|[#9625](https://github.com/NVIDIA/spark-rapids/issues/9625)|[BUG] Typo in github Maven check install-modules | +|[#9603](https://github.com/NVIDIA/spark-rapids/issues/9603)|[BUG] fastparquet_compatibility_test fails on dataproc| +|[#8729](https://github.com/NVIDIA/spark-rapids/issues/8729)|[BUG] nightly integration test failed OOM kill in JDK11 ENV| +|[#9589](https://github.com/NVIDIA/spark-rapids/issues/9589)|[BUG] Scala 2.13 build hard-codes Java 8 target | +|[#9581](https://github.com/NVIDIA/spark-rapids/issues/9581)|Delta Lake 2.4 missing equals/hashCode override for file format and some metrics for merge| +|[#9507](https://github.com/NVIDIA/spark-rapids/issues/9507)|[BUG] Spark 3.2+/ParquetFilterSuite/Parquet filter pushdown - timestamp/ FAILED | +|[#9540](https://github.com/NVIDIA/spark-rapids/issues/9540)|[BUG] Job failed with SparkUpgradeException no matter which value are set for spark.sql.parquet.datetimeRebaseModeInRead| +|[#9545](https://github.com/NVIDIA/spark-rapids/issues/9545)|[BUG] Dataproc 2.0 test_reading_file_rewritten_with_fastparquet tests failing| +|[#9552](https://github.com/NVIDIA/spark-rapids/issues/9552)|[BUG] Inconsistent CDH dependency overrides across submodules| +|[#9571](https://github.com/NVIDIA/spark-rapids/issues/9571)|[BUG] non-deterministic compiled SQLExecPlugin.class with scala 2.13 deployment| +|[#9569](https://github.com/NVIDIA/spark-rapids/issues/9569)|[BUG] test_window_running failed in 3.1.2+3.1.3| +|[#9480](https://github.com/NVIDIA/spark-rapids/issues/9480)|[BUG] mapInPandas doesn't invoke udf on empty partitions| +|[#8644](https://github.com/NVIDIA/spark-rapids/issues/8644)|[BUG] Parquet file with malformed dictionary does not error when loaded| +|[#9310](https://github.com/NVIDIA/spark-rapids/issues/9310)|[BUG] Improve support for reading JSON files with malformed rows| +|[#9457](https://github.com/NVIDIA/spark-rapids/issues/9457)|[BUG] CDH 332 unit tests failing| +|[#9404](https://github.com/NVIDIA/spark-rapids/issues/9404)|[BUG] Spark reports a decimal error when create lit scalar when generate Decimal(34, -5) data.| +|[#9110](https://github.com/NVIDIA/spark-rapids/issues/9110)|[BUG] GPU Reader fails due to partition column creating column larger then cudf column size limit| +|[#8631](https://github.com/NVIDIA/spark-rapids/issues/8631)|[BUG] Parquet load failure on repeated_no_annotation.parquet| +|[#9364](https://github.com/NVIDIA/spark-rapids/issues/9364)|[BUG] CUDA illegal access error is triggering split and retry logic| + +### PRs +||| +|:---|:---| +|[#9943](https://github.com/NVIDIA/spark-rapids/pull/9943)|[DOC] Update docs for 23.12.0 release [skip ci]| +|[#10014](https://github.com/NVIDIA/spark-rapids/pull/10014)|Add documentation for how to run tests with a fixed datagen seed [skip ci]| +|[#9954](https://github.com/NVIDIA/spark-rapids/pull/9954)|Update private and JNI version to released 23.12.0| +|[#10009](https://github.com/NVIDIA/spark-rapids/pull/10009)|Using fix seed to unblock 23.12 release; Move the blocked issues to 24.02| +|[#10007](https://github.com/NVIDIA/spark-rapids/pull/10007)|Fix Java OOM in non-UTC case with lots of xfail (#9944)| +|[#9985](https://github.com/NVIDIA/spark-rapids/pull/9985)|Avoid allocating GPU memory out of RMM managed pool in test| +|[#9970](https://github.com/NVIDIA/spark-rapids/pull/9970)|Avoid leading and trailing zeros in test_timestamp_seconds_rounding_necessary| +|[#9978](https://github.com/NVIDIA/spark-rapids/pull/9978)|Avoid using floating point values as partition values in tests| +|[#9979](https://github.com/NVIDIA/spark-rapids/pull/9979)|Add compatibility notes for writing ORC with lost Gregorian days [skip ci]| +|[#9949](https://github.com/NVIDIA/spark-rapids/pull/9949)|Override the seed for `test_map_scalars_supported_key_types ` for version of Spark before 3.4.0 [Databricks]| +|[#9961](https://github.com/NVIDIA/spark-rapids/pull/9961)|Avoid using floating point for partition values in Delta Lake tests| +|[#9960](https://github.com/NVIDIA/spark-rapids/pull/9960)|Fix LongGen accidentally using special cases when none are desired| +|[#9950](https://github.com/NVIDIA/spark-rapids/pull/9950)|Avoid generating NaNs as partition values in test_part_write_round_trip| +|[#9940](https://github.com/NVIDIA/spark-rapids/pull/9940)|Fix 'year 0 is out of range' by setting a fix seed| +|[#9946](https://github.com/NVIDIA/spark-rapids/pull/9946)|Fix test_multi_tier_ast to ignore ordering of output rows| +|[#9928](https://github.com/NVIDIA/spark-rapids/pull/9928)|Test `inset` with `NaN` only for Spark from 3.1.3| +|[#9906](https://github.com/NVIDIA/spark-rapids/pull/9906)|Fix test_initcap to use the intended limited character set| +|[#9831](https://github.com/NVIDIA/spark-rapids/pull/9831)|Skip fastparquet timestamp tests when plugin cannot read/write timestamps| +|[#9893](https://github.com/NVIDIA/spark-rapids/pull/9893)|Add multiple expression tier regression test for AST| +|[#9873](https://github.com/NVIDIA/spark-rapids/pull/9873)|Add support for decimal in `to_json`| +|[#9890](https://github.com/NVIDIA/spark-rapids/pull/9890)|Remove Databricks 13.3 from release 23.12| +|[#9874](https://github.com/NVIDIA/spark-rapids/pull/9874)|Fix zero-scale floor and ceil tests| +|[#9879](https://github.com/NVIDIA/spark-rapids/pull/9879)|Fix resource leak in to_json| +|[#9600](https://github.com/NVIDIA/spark-rapids/pull/9600)|Add date and timestamp support to to_json| +|[#9871](https://github.com/NVIDIA/spark-rapids/pull/9871)|Fix test_cast_string_date_valid_format generating year 0| +|[#9885](https://github.com/NVIDIA/spark-rapids/pull/9885)|Preparation for non-UTC nightly CI [skip ci]| +|[#9810](https://github.com/NVIDIA/spark-rapids/pull/9810)|Support from_utc_timestamp on the GPU for non-UTC timezones (non-DST)| +|[#9865](https://github.com/NVIDIA/spark-rapids/pull/9865)|Fix problems with nulls in sequence tests| +|[#9864](https://github.com/NVIDIA/spark-rapids/pull/9864)|Add compatibility documentation with respect to decimal overflow detection [skip ci]| +|[#9860](https://github.com/NVIDIA/spark-rapids/pull/9860)|Fixing FAQ deadlink in plugin code [skip ci]| +|[#9840](https://github.com/NVIDIA/spark-rapids/pull/9840)|Avoid using NaNs as Delta Lake partition values| +|[#9773](https://github.com/NVIDIA/spark-rapids/pull/9773)|xfail all the impacted cases when using non-UTC time zone| +|[#9849](https://github.com/NVIDIA/spark-rapids/pull/9849)|Instantly Delete pre-merge content of stage workspace if success| +|[#9848](https://github.com/NVIDIA/spark-rapids/pull/9848)|Force datagen_seed for test_ceil_scale_zero and test_decimal_round| +|[#9677](https://github.com/NVIDIA/spark-rapids/pull/9677)|Enable build for Databricks 13.3| +|[#9809](https://github.com/NVIDIA/spark-rapids/pull/9809)|Re-enable AST string integration cases| +|[#9835](https://github.com/NVIDIA/spark-rapids/pull/9835)|Avoid pre-Gregorian dates in schema_evolution_test| +|[#9786](https://github.com/NVIDIA/spark-rapids/pull/9786)|Check paths for existence to prevent ignorable error messages during build| +|[#9824](https://github.com/NVIDIA/spark-rapids/pull/9824)|UCX 1.15 upgrade| +|[#9800](https://github.com/NVIDIA/spark-rapids/pull/9800)|Add GpuCheckOverflowInTableInsert to Databricks 11.3+| +|[#9821](https://github.com/NVIDIA/spark-rapids/pull/9821)|Update timestamp gens to avoid "year 0 is out of range" errors| +|[#9826](https://github.com/NVIDIA/spark-rapids/pull/9826)|Set seed to 0 for test_hash_reduction_sum| +|[#9720](https://github.com/NVIDIA/spark-rapids/pull/9720)|Support timestamp in `from_json`| +|[#9818](https://github.com/NVIDIA/spark-rapids/pull/9818)|Specify nullable=False when generating filter values in dpp tests| +|[#9689](https://github.com/NVIDIA/spark-rapids/pull/9689)|Support CPU path for from_utc_timestamp function with timezone | +|[#9769](https://github.com/NVIDIA/spark-rapids/pull/9769)|Use withGpuSparkSession to customize SparkConf| +|[#9780](https://github.com/NVIDIA/spark-rapids/pull/9780)|Fix NaN handling in GpuLessThanOrEqual and GpuGreaterThanOrEqual| +|[#9795](https://github.com/NVIDIA/spark-rapids/pull/9795)|xfail AST string tests| +|[#9666](https://github.com/NVIDIA/spark-rapids/pull/9666)|Add support for parsing strings as dates in `from_json`| +|[#9673](https://github.com/NVIDIA/spark-rapids/pull/9673)|Fix the broadcast joins issues caused by InputFileBlockRule| +|[#9785](https://github.com/NVIDIA/spark-rapids/pull/9785)|Force datagen_seed for 9781 and 9784 [skip ci]| +|[#9765](https://github.com/NVIDIA/spark-rapids/pull/9765)|Let GPU scans fall back when default values exist in schema| +|[#9729](https://github.com/NVIDIA/spark-rapids/pull/9729)|Fix Delta Lake atomic table operations on spark341db| +|[#9770](https://github.com/NVIDIA/spark-rapids/pull/9770)|[BUG] Fix the doc for Maven and Scala 2.13 test example [skip ci]| +|[#9761](https://github.com/NVIDIA/spark-rapids/pull/9761)|Fix bug in tagging of JsonToStructs| +|[#9758](https://github.com/NVIDIA/spark-rapids/pull/9758)|Remove forced seed from Delta Lake part_write_round_trip_unmanaged tests| +|[#9652](https://github.com/NVIDIA/spark-rapids/pull/9652)|Add time zone config to set non-UTC| +|[#9736](https://github.com/NVIDIA/spark-rapids/pull/9736)|Fix `TimestampGen` to generate value not too close to the minimum allowed timestamp| +|[#9698](https://github.com/NVIDIA/spark-rapids/pull/9698)|Speed up build: unnecessary invalidation in the incremental recompile mode| +|[#9748](https://github.com/NVIDIA/spark-rapids/pull/9748)|Fix Delta Lake part_write_round_trip_unmanaged tests with floating point| +|[#9702](https://github.com/NVIDIA/spark-rapids/pull/9702)|Support split BroadcastNestedLoopJoin condition for AST and non-AST| +|[#9746](https://github.com/NVIDIA/spark-rapids/pull/9746)|Force test_hypot to be single seed for now| +|[#9745](https://github.com/NVIDIA/spark-rapids/pull/9745)|Avoid generating null filter values in test_delta_dfp_reuse_broadcast_exchange| +|[#9741](https://github.com/NVIDIA/spark-rapids/pull/9741)|Set seed=0 for the delta lake part roundtrip tests| +|[#9660](https://github.com/NVIDIA/spark-rapids/pull/9660)|Fully support date/time legacy rebase for nested input| +|[#9672](https://github.com/NVIDIA/spark-rapids/pull/9672)|Support String type for AST| +|[#9732](https://github.com/NVIDIA/spark-rapids/pull/9732)|Temporarily force `datagen_seed=0` for `test_re_replace_all` to unblock CI| +|[#9726](https://github.com/NVIDIA/spark-rapids/pull/9726)|Fix leak in BatchWithPartitionData| +|[#9717](https://github.com/NVIDIA/spark-rapids/pull/9717)|Encode the file path from Iceberg when converting to a PartitionedFile| +|[#9441](https://github.com/NVIDIA/spark-rapids/pull/9441)|Add a random seed specific to datagen cases| +|[#9649](https://github.com/NVIDIA/spark-rapids/pull/9649)|Support `spark.sql.parquet.datetimeRebaseModeInRead=LEGACY` and `spark.sql.parquet.int96RebaseModeInRead=LEGACY`| +|[#9612](https://github.com/NVIDIA/spark-rapids/pull/9612)|Escape quotes and newlines when converting strings to json format in to_json| +|[#9644](https://github.com/NVIDIA/spark-rapids/pull/9644)|Add Partial Delta Lake Support for Databricks 13.3| +|[#9690](https://github.com/NVIDIA/spark-rapids/pull/9690)|Changed `extractExecutedPlan` to consider ResultQueryStageExec for Databricks 13.3| +|[#9686](https://github.com/NVIDIA/spark-rapids/pull/9686)|Removed Maven Profiles From `tests/pom.xml`| +|[#9509](https://github.com/NVIDIA/spark-rapids/pull/9509)|Fine-grained spill metrics| +|[#9658](https://github.com/NVIDIA/spark-rapids/pull/9658)|Support `spark.sql.parquet.int96RebaseModeInWrite=LEGACY`| +|[#9695](https://github.com/NVIDIA/spark-rapids/pull/9695)|Revert "Support split non-AST-able join condition for BroadcastNested…| +|[#9693](https://github.com/NVIDIA/spark-rapids/pull/9693)|Enable automerge from 23.12 to 24.02 [skip ci]| +|[#9679](https://github.com/NVIDIA/spark-rapids/pull/9679)|[Doc] update the dead link in download page [skip ci]| +|[#9678](https://github.com/NVIDIA/spark-rapids/pull/9678)|Add flow control for multithreaded shuffle writer| +|[#9635](https://github.com/NVIDIA/spark-rapids/pull/9635)|Support split non-AST-able join condition for BroadcastNestedLoopJoin| +|[#9646](https://github.com/NVIDIA/spark-rapids/pull/9646)|Fix Integration Test Failures for Databricks 13.3 Support| +|[#9670](https://github.com/NVIDIA/spark-rapids/pull/9670)|Normalize file timezone and handle missing file timezone in datetimeRebaseUtils| +|[#9657](https://github.com/NVIDIA/spark-rapids/pull/9657)|Update verify check to handle new pom files [skip ci]| +|[#9663](https://github.com/NVIDIA/spark-rapids/pull/9663)|Making User Guide info in bold and adding it as top right link in github.io [skip ci]| +|[#9609](https://github.com/NVIDIA/spark-rapids/pull/9609)|Add valid retry solution to mvn-verify [skip ci]| +|[#9655](https://github.com/NVIDIA/spark-rapids/pull/9655)|Document problem with handling of invalid characters in CSV reader| +|[#9620](https://github.com/NVIDIA/spark-rapids/pull/9620)|Add support for parsing boolean values in `from_json`| +|[#9615](https://github.com/NVIDIA/spark-rapids/pull/9615)|Bloop updates - require JDK11 in buildall + docs, build bloop for all targets.| +|[#9631](https://github.com/NVIDIA/spark-rapids/pull/9631)|Refactor Parquet readers| +|[#9637](https://github.com/NVIDIA/spark-rapids/pull/9637)|Added Support For Various Execs for Databricks 13.3 | +|[#9640](https://github.com/NVIDIA/spark-rapids/pull/9640)|Add support for `ignoreNullFields=false` in `to_json`| +|[#9623](https://github.com/NVIDIA/spark-rapids/pull/9623)|Running window optimization for `LAST()`| +|[#9641](https://github.com/NVIDIA/spark-rapids/pull/9641)|Revert "Support rebase checking for nested dates and timestamps (#9617)"| +|[#9423](https://github.com/NVIDIA/spark-rapids/pull/9423)|Re-enable `from_json` / `JsonToStructs`| +|[#9624](https://github.com/NVIDIA/spark-rapids/pull/9624)|Add jenkins-level retry for pre-merge build in databricks runtimes| +|[#9608](https://github.com/NVIDIA/spark-rapids/pull/9608)|Fix nullability issues for some decimal operations| +|[#9617](https://github.com/NVIDIA/spark-rapids/pull/9617)|Support rebase checking for nested dates and timestamps| +|[#9611](https://github.com/NVIDIA/spark-rapids/pull/9611)|Move simple classes after refactoring to sql-plugin-api| +|[#9618](https://github.com/NVIDIA/spark-rapids/pull/9618)|Remove unused dataTypes argument from HostShuffleCoalesceIterator| +|[#9626](https://github.com/NVIDIA/spark-rapids/pull/9626)|Fix ENV typo in pre-merge github actions [skip ci]| +|[#9593](https://github.com/NVIDIA/spark-rapids/pull/9593)|PythonRunner and RapidsErrorUtils Changes For Databricks 13.3| +|[#9607](https://github.com/NVIDIA/spark-rapids/pull/9607)|Integration tests: Install specific fastparquet version.| +|[#9610](https://github.com/NVIDIA/spark-rapids/pull/9610)|Propagate local properties to broadcast execs| +|[#9544](https://github.com/NVIDIA/spark-rapids/pull/9544)|Support batching for `RANGE` running window aggregations. Including on| +|[#9601](https://github.com/NVIDIA/spark-rapids/pull/9601)|Remove usage of deprecated scala.Proxy| +|[#9591](https://github.com/NVIDIA/spark-rapids/pull/9591)|Enable implicit JDK profile activation| +|[#9586](https://github.com/NVIDIA/spark-rapids/pull/9586)|Merge metrics and file format fixes to Delta 2.4 support| +|[#9594](https://github.com/NVIDIA/spark-rapids/pull/9594)|Revert "Ignore failing Parquet filter test to unblock CI (#9519)"| +|[#9454](https://github.com/NVIDIA/spark-rapids/pull/9454)|Support encryption and compression in disk store| +|[#9439](https://github.com/NVIDIA/spark-rapids/pull/9439)|Support stack function| +|[#9583](https://github.com/NVIDIA/spark-rapids/pull/9583)|Fix fastparquet tests to work with HDFS| +|[#9508](https://github.com/NVIDIA/spark-rapids/pull/9508)|Consolidate deps switching in an intermediate pom| +|[#9562](https://github.com/NVIDIA/spark-rapids/pull/9562)|Delta Lake 2.3.0 support| +|[#9576](https://github.com/NVIDIA/spark-rapids/pull/9576)|Move Stack classes to wrapper classes to fix non-deterministic build issue| +|[#9572](https://github.com/NVIDIA/spark-rapids/pull/9572)|Add retry for CrossJoinIterator and ConditionalNestedLoopJoinIterator| +|[#9575](https://github.com/NVIDIA/spark-rapids/pull/9575)|Fix `test_window_running*()` for `NTH_VALUE IGNORE NULLS`.| +|[#9574](https://github.com/NVIDIA/spark-rapids/pull/9574)|Fix broken #endif scala comments [skip ci]| +|[#9568](https://github.com/NVIDIA/spark-rapids/pull/9568)|Enforce Apache 3.3.0+ for Scala 2.13| +|[#9557](https://github.com/NVIDIA/spark-rapids/pull/9557)|Support launching Map Pandas UDF on empty partitions| +|[#9489](https://github.com/NVIDIA/spark-rapids/pull/9489)|Batching support for ROW-based `FIRST()` window function| +|[#9510](https://github.com/NVIDIA/spark-rapids/pull/9510)|Add Databricks 13.3 shim boilerplate code and refactor Databricks 12.2 shim| +|[#9554](https://github.com/NVIDIA/spark-rapids/pull/9554)|Fix fastparquet installation for| +|[#9536](https://github.com/NVIDIA/spark-rapids/pull/9536)|Add CPU POC of TimeZoneDB; Test some time zones by comparing CPU POC and Spark| +|[#9558](https://github.com/NVIDIA/spark-rapids/pull/9558)|Support integration test against scala2.13 spark binaries[skip ci]| +|[#8592](https://github.com/NVIDIA/spark-rapids/pull/8592)|Scala 2.13 Support| +|[#9551](https://github.com/NVIDIA/spark-rapids/pull/9551)|Enable malformed Parquet failure test| +|[#9546](https://github.com/NVIDIA/spark-rapids/pull/9546)|Support OverwriteByExpressionExecV1 for Delta Lake tables| +|[#9527](https://github.com/NVIDIA/spark-rapids/pull/9527)|Support Split And Retry for GpuProjectAstExec| +|[#9541](https://github.com/NVIDIA/spark-rapids/pull/9541)|Move simple classes to API| +|[#9548](https://github.com/NVIDIA/spark-rapids/pull/9548)|Append new authorized user to blossom-ci whitelist [skip ci]| +|[#9418](https://github.com/NVIDIA/spark-rapids/pull/9418)|Fix STRUCT comparison between Pandas and Spark dataframes in fastparquet tests| +|[#9468](https://github.com/NVIDIA/spark-rapids/pull/9468)|Add SplitAndRetry to GpuRunningWindowIterator| +|[#9486](https://github.com/NVIDIA/spark-rapids/pull/9486)|Add partial support for `to_json`| +|[#9538](https://github.com/NVIDIA/spark-rapids/pull/9538)|Fix tiered project breaking higher order functions| +|[#9539](https://github.com/NVIDIA/spark-rapids/pull/9539)|Add delta-24x to delta-lake/README.md [skip ci]| +|[#9534](https://github.com/NVIDIA/spark-rapids/pull/9534)|Add pyarrow tests for Databricks runtime| +|[#9444](https://github.com/NVIDIA/spark-rapids/pull/9444)|Remove redundant pass-through shuffle manager classes| +|[#9531](https://github.com/NVIDIA/spark-rapids/pull/9531)|Fix relative path for spark-shell nightly test [skip ci]| +|[#9525](https://github.com/NVIDIA/spark-rapids/pull/9525)|Follow-up to dbdeps consolidation| +|[#9506](https://github.com/NVIDIA/spark-rapids/pull/9506)|Move ProxyShuffleInternalManagerBase to api| +|[#9504](https://github.com/NVIDIA/spark-rapids/pull/9504)|Add a spark-shell smoke test to premerge and nightly| +|[#9519](https://github.com/NVIDIA/spark-rapids/pull/9519)|Ignore failing Parquet filter test to unblock CI| +|[#9478](https://github.com/NVIDIA/spark-rapids/pull/9478)|Support AppendDataExecV1 for Delta Lake tables| +|[#9366](https://github.com/NVIDIA/spark-rapids/pull/9366)|Add tests to check compatibility with `fastparquet`| +|[#9419](https://github.com/NVIDIA/spark-rapids/pull/9419)|Add retry to RoundRobin Partitioner and Range Partitioner| +|[#9502](https://github.com/NVIDIA/spark-rapids/pull/9502)|Install Dependencies Needed For Databricks 13.3| +|[#9296](https://github.com/NVIDIA/spark-rapids/pull/9296)|Implement `percentile` aggregation| +|[#9488](https://github.com/NVIDIA/spark-rapids/pull/9488)|Add Shim JSON Headers for Databricks 13.3| +|[#9443](https://github.com/NVIDIA/spark-rapids/pull/9443)|Add AtomicReplaceTableAsSelectExec support for Delta Lake| +|[#9476](https://github.com/NVIDIA/spark-rapids/pull/9476)|Refactor common Delta Lake test code| +|[#9463](https://github.com/NVIDIA/spark-rapids/pull/9463)|Fix Cloudera 3.3.2 shim for handling CheckOverflowInTableInsert and orc zstd support| +|[#9460](https://github.com/NVIDIA/spark-rapids/pull/9460)|Update links in old release notes to new doc locations [skip ci]| +|[#9405](https://github.com/NVIDIA/spark-rapids/pull/9405)|Wrap scalar generation into spark session in integration test| +|[#9459](https://github.com/NVIDIA/spark-rapids/pull/9459)|Fix 332cdh build [skip ci]| +|[#9425](https://github.com/NVIDIA/spark-rapids/pull/9425)|Add support for AtomicCreateTableAsSelect with Delta Lake| +|[#9434](https://github.com/NVIDIA/spark-rapids/pull/9434)|Add retry support to `HostToGpuCoalesceIterator.concatAllAndPutOnGPU`| +|[#9453](https://github.com/NVIDIA/spark-rapids/pull/9453)|Update codeowner and blossom-ci ACL [skip ci]| +|[#9396](https://github.com/NVIDIA/spark-rapids/pull/9396)|Add support for Cloudera CDS-3.3.2| +|[#9380](https://github.com/NVIDIA/spark-rapids/pull/9380)|Fix parsing of Parquet legacy list-of-struct format| +|[#9438](https://github.com/NVIDIA/spark-rapids/pull/9438)|Fix auto merge conflict 9437 [skip ci]| +|[#9424](https://github.com/NVIDIA/spark-rapids/pull/9424)|Refactor aggregate functions| +|[#9414](https://github.com/NVIDIA/spark-rapids/pull/9414)|Add retry to GpuHashJoin.filterNulls| +|[#9388](https://github.com/NVIDIA/spark-rapids/pull/9388)|Add developer documentation about working with data sources [skip ci]| +|[#9369](https://github.com/NVIDIA/spark-rapids/pull/9369)|Improve JSON empty row fix to use less memory| +|[#9373](https://github.com/NVIDIA/spark-rapids/pull/9373)|Fix auto merge conflict 9372| +|[#9308](https://github.com/NVIDIA/spark-rapids/pull/9308)|Initiate arm64 CI support [skip ci]| +|[#9292](https://github.com/NVIDIA/spark-rapids/pull/9292)|Init project version 23.12.0-SNAPSHOT| +|[#9291](https://github.com/NVIDIA/spark-rapids/pull/9291)|Automerge from 23.10 to 23.12 [skip ci]| ## Release 23.10 @@ -22,11 +310,14 @@ Generated on 2023-10-24 ### Performance ||| |:---|:---| +|[#9512](https://github.com/NVIDIA/spark-rapids/issues/9512)|[DOC] Multi-Threaded shuffle documentation is not accurate on the read side| |[#7803](https://github.com/NVIDIA/spark-rapids/issues/7803)|[FEA] Accelerate Bloom filtered joins | ### Bugs Fixed ||| |:---|:---| +|[#8662](https://github.com/NVIDIA/spark-rapids/issues/8662)|[BUG] Dataproc spark-rapids.sh fails due to cuda driver version issue| +|[#9428](https://github.com/NVIDIA/spark-rapids/issues/9428)|[Audit] SPARK-44448 Wrong results for dense_rank() <= k| |[#9485](https://github.com/NVIDIA/spark-rapids/issues/9485)|[BUG] GpuSemaphore can deadlock if there are multiple threads per task| |[#9498](https://github.com/NVIDIA/spark-rapids/issues/9498)|[BUG] spark 3.5.0 shim spark-shell is broken in spark-rapids 23.10 and 23.12| |[#9060](https://github.com/NVIDIA/spark-rapids/issues/9060)|[BUG] OOM error in split and retry with multifile coalesce reader with parquet data| @@ -83,13 +374,19 @@ Generated on 2023-10-24 ### PRs ||| |:---|:---| +|[#9304](https://github.com/NVIDIA/spark-rapids/pull/9304)|Specify recoverWithNull when reading JSON files| +|[#9474](https://github.com/NVIDIA/spark-rapids/pull/9474)| Improve configuration handling in BatchWithPartitionData| +|[#9289](https://github.com/NVIDIA/spark-rapids/pull/9289)|Add tests to check compatibility with pyarrow| +|[#9522](https://github.com/NVIDIA/spark-rapids/pull/9522)|Update 23.10 changelog [skip ci]| |[#9501](https://github.com/NVIDIA/spark-rapids/pull/9501)|Fix GpuSemaphore to support multiple threads per task| |[#9500](https://github.com/NVIDIA/spark-rapids/pull/9500)|Fix Spark 3.5.0 shell classloader issue with the plugin| +|[#9230](https://github.com/NVIDIA/spark-rapids/pull/9230)|Fix reading partition value columns larger than cudf column size limit| |[#9427](https://github.com/NVIDIA/spark-rapids/pull/9427)|[DOC] Update docs for 23.10.0 release [skip ci]| |[#9421](https://github.com/NVIDIA/spark-rapids/pull/9421)|Init changelog of 23.10 [skip ci]| |[#9445](https://github.com/NVIDIA/spark-rapids/pull/9445)|Only run test_csv_infer_schema_timestamp_ntz tests with PySpark >= 3.4.1| |[#9420](https://github.com/NVIDIA/spark-rapids/pull/9420)|Update private and jni dep version to released 23.10.0| |[#9415](https://github.com/NVIDIA/spark-rapids/pull/9415)|[BUG] fix docker modified check in premerge [skip ci]| +|[#9407](https://github.com/NVIDIA/spark-rapids/pull/9407)|[Doc]Update docs for 23.08.2 version[skip ci]| |[#9392](https://github.com/NVIDIA/spark-rapids/pull/9392)|Only run test_json_ts_formats_round_trip_ntz tests with PySpark >= 3.4.1| |[#9401](https://github.com/NVIDIA/spark-rapids/pull/9401)|Remove using mamba before they fix the incompatibility issue [skip ci]| |[#9381](https://github.com/NVIDIA/spark-rapids/pull/9381)|Change the executor core calculation to take into account the cluster manager| @@ -178,6 +475,7 @@ Generated on 2023-10-24 |[#9102](https://github.com/NVIDIA/spark-rapids/pull/9102)|Add retry support to GpuOutOfCoreSortIterator.mergeSortEnoughToOutput | |[#9089](https://github.com/NVIDIA/spark-rapids/pull/9089)|Add application to run Scale Test| |[#9143](https://github.com/NVIDIA/spark-rapids/pull/9143)|[DOC] update spark.rapids.sql.concurrentGpuTasks default value in tuning guide [skip ci]| +|[#8476](https://github.com/NVIDIA/spark-rapids/pull/8476)|Use retry with split in GpuCachedDoublePassWindowIterator| |[#9141](https://github.com/NVIDIA/spark-rapids/pull/9141)|Removed resultDecimalType in GpuIntegralDecimalDivide| |[#9099](https://github.com/NVIDIA/spark-rapids/pull/9099)|Spark 3.5.0 follow-on work (rc2 support + Python UDAF)| |[#9140](https://github.com/NVIDIA/spark-rapids/pull/9140)|Bump Jython to 2.7.3| @@ -234,241 +532,5 @@ Generated on 2023-10-24 |[#8805](https://github.com/NVIDIA/spark-rapids/pull/8805)|Bump up dep versions to 23.10.0-SNAPSHOT| |[#8796](https://github.com/NVIDIA/spark-rapids/pull/8796)|Init version 23.10.0-SNAPSHOT| -## Release 23.08 - -### Features -||| -|:---|:---| -|[#5509](https://github.com/NVIDIA/spark-rapids/issues/5509)|[FEA] Support order-by on Array| -|[#7876](https://github.com/NVIDIA/spark-rapids/issues/7876)|[FEA] Add initial support for Databricks 12.2 ML LTS| -|[#8547](https://github.com/NVIDIA/spark-rapids/issues/8547)|[FEA] Add support for Delta Lake 2.4 with Spark 3.4| -|[#8633](https://github.com/NVIDIA/spark-rapids/issues/8633)|[FEA] Add support for xxHash64 function| -|[#4929](https://github.com/NVIDIA/spark-rapids/issues/4929)|[FEA] Support min/max aggregation/reduction for arrays of structs and arrays of strings| -|[#8668](https://github.com/NVIDIA/spark-rapids/issues/8668)|[FEA] Support min and max for arrays| -|[#4887](https://github.com/NVIDIA/spark-rapids/issues/4887)|[FEA] Hash partitioning on ArrayType| -|[#6680](https://github.com/NVIDIA/spark-rapids/issues/6680)|[FEA] Support hashaggregate for Array[Any]| -|[#8085](https://github.com/NVIDIA/spark-rapids/issues/8085)|[FEA] Add support for MillisToTimestamp| -|[#7801](https://github.com/NVIDIA/spark-rapids/issues/7801)|[FEA] Window Expression orderBy column is not supported in a window range function, found DoubleType| -|[#8556](https://github.com/NVIDIA/spark-rapids/issues/8556)|[FEA] [Delta Lake] Add support for new metrics in MERGE| -|[#308](https://github.com/NVIDIA/spark-rapids/issues/308)|[FEA] Spark 3.1 adding support for TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS functions| -|[#8122](https://github.com/NVIDIA/spark-rapids/issues/8122)|[FEA] Add spark 3.4.1 snapshot shim| -|[#8525](https://github.com/NVIDIA/spark-rapids/issues/8525)|[FEA] Add support for org.apache.spark.sql.functions.flatten| -|[#8202](https://github.com/NVIDIA/spark-rapids/issues/8202)|[FEA] List supported Spark builds when the Shim is not found| - -### Performance -||| -|:---|:---| -|[#8231](https://github.com/NVIDIA/spark-rapids/issues/8231)|[FEA] Add filecache support to ORC scans| -|[#8141](https://github.com/NVIDIA/spark-rapids/issues/8141)|[FEA] Explore how to best deal with large numbers of aggregations in the short term| - -### Bugs Fixed -||| -|:---|:---| -|[#9034](https://github.com/NVIDIA/spark-rapids/issues/9034)|[BUG] java.lang.ClassCastException: com.nvidia.spark.rapids.RuleNotFoundExprMeta cannot be cast to com.nvidia.spark.rapids.GeneratorExprMeta| -|[#9032](https://github.com/NVIDIA/spark-rapids/issues/9032)|[BUG] Multiple NDS queries fail with Spark-3.4.1 with bloom filter exception| -|[#8962](https://github.com/NVIDIA/spark-rapids/issues/8962)|[BUG] Nightly build failed: ExecutionPlanCaptureCallback$.class is not bitwise-identical across shims| -|[#9021](https://github.com/NVIDIA/spark-rapids/issues/9021)|[BUG] test_map_scalars_supported_key_types failed in dataproc 2.1| -|[#9020](https://github.com/NVIDIA/spark-rapids/issues/9020)|[BUG] auto-disable snapshot shims test in github action for pre-release branch| -|[#9010](https://github.com/NVIDIA/spark-rapids/issues/9010)|[BUG] Customer failure 23.08: Cannot compute hash of a table with a LIST of STRUCT columns.| -|[#8922](https://github.com/NVIDIA/spark-rapids/issues/8922)|[BUG] integration map_test:test_map_scalars_supported_key_types failures| -|[#8982](https://github.com/NVIDIA/spark-rapids/issues/8982)|[BUG] Nightly prerelease failures - OrcSuite| -|[#8978](https://github.com/NVIDIA/spark-rapids/issues/8978)|[BUG] compiling error due to OrcSuite&OrcStatisticShim in databricks runtimes| -|[#8610](https://github.com/NVIDIA/spark-rapids/issues/8610)|[BUG] query 95 @ SF30K fails with OOM exception| -|[#8955](https://github.com/NVIDIA/spark-rapids/issues/8955)|[BUG] Bloom filter join tests can fail with multiple join columns| -|[#45](https://github.com/NVIDIA/spark-rapids/issues/45)|[BUG] very large shuffles can fail| -|[#8779](https://github.com/NVIDIA/spark-rapids/issues/8779)|[BUG] Put shared Databricks test script together for ease of maintenance| -|[#8930](https://github.com/NVIDIA/spark-rapids/issues/8930)|[BUG] checkoutSCM plugin is unstable for pre-merge CI, it is often unable to clone submodules| -|[#8923](https://github.com/NVIDIA/spark-rapids/issues/8923)|[BUG] Mortgage test failing with 'JavaPackage' error on AWS Databricks| -|[#8303](https://github.com/NVIDIA/spark-rapids/issues/8303)|[BUG] GpuExpression columnarEval can return scalars from subqueries that may be unhandled| -|[#8318](https://github.com/NVIDIA/spark-rapids/issues/8318)|[BUG][Databricks 12.2] GpuRowBasedHiveGenericUDF ClassCastException| -|[#8822](https://github.com/NVIDIA/spark-rapids/issues/8822)|[BUG] Early terminate CI if submodule init failed| -|[#8847](https://github.com/NVIDIA/spark-rapids/issues/8847)|[BUG] github actions CI messed up w/ JDK versions intermittently| -|[#8716](https://github.com/NVIDIA/spark-rapids/issues/8716)|[BUG] `test_hash_groupby_collect_set_on_nested_type` and `test_hash_reduction_collect_set_on_nested_type` failed| -|[#8827](https://github.com/NVIDIA/spark-rapids/issues/8827)|[BUG] databricks cudf_udf night build failing with pool size exceeded errors| -|[#8630](https://github.com/NVIDIA/spark-rapids/issues/8630)|[BUG] Parquet with RLE encoded booleans loads corrupted data| -|[#8735](https://github.com/NVIDIA/spark-rapids/issues/8735)|[BUG] test_orc_column_name_with_dots fails in nightly EGX tests| -|[#6980](https://github.com/NVIDIA/spark-rapids/issues/6980)|[BUG] Partitioned writes release GPU semaphore with unspillable GPU memory| -|[#8784](https://github.com/NVIDIA/spark-rapids/issues/8784)|[BUG] hash_aggregate_test.py::test_min_max_in_groupby_and_reduction failed on "TypeError: object of type 'NoneType' has no len()"| -|[#8756](https://github.com/NVIDIA/spark-rapids/issues/8756)|[BUG] [Databricks 12.2] RapidsDeltaWrite queries that reference internal metadata fail to run| -|[#8636](https://github.com/NVIDIA/spark-rapids/issues/8636)|[BUG] AWS Databricks 12.2 integration tests failed due to Iceberg check| -|[#8754](https://github.com/NVIDIA/spark-rapids/issues/8754)|[BUG] databricks build broke after adding bigDataGen| -|[#8726](https://github.com/NVIDIA/spark-rapids/issues/8726)|[BUG] Test "parquet_write_test.py::test_hive_timestamp_value[INJECT_OOM]" failed on Databricks | -|[#8690](https://github.com/NVIDIA/spark-rapids/issues/8690)|[BUG buildall script does not support JDK11 profile| -|[#8702](https://github.com/NVIDIA/spark-rapids/issues/8702)|[BUG] test_min_max_for_single_level_struct failed| -|[#8727](https://github.com/NVIDIA/spark-rapids/issues/8727)|[BUG] test_column_add_after_partition failed in databricks 10.4 runtime| -|[#8669](https://github.com/NVIDIA/spark-rapids/issues/8669)|[BUG] SpillableColumnarBatch doesn't always take ownership| -|[#8655](https://github.com/NVIDIA/spark-rapids/issues/8655)|[BUG] There are some potential device memory leaks in `AbstractGpuCoalesceIterator`| -|[#8685](https://github.com/NVIDIA/spark-rapids/issues/8685)|[BUG] install build fails with Maven 3.9.3| -|[#8156](https://github.com/NVIDIA/spark-rapids/issues/8156)|[BUG] Install phase for modules with Spark build classifier fails for install plugin versions 3.0.0+| -|[#1130](https://github.com/NVIDIA/spark-rapids/issues/1130)|[BUG] TIMESTAMP_MILLIS not handled in isDateTimeRebaseNeeded| -|[#7676](https://github.com/NVIDIA/spark-rapids/issues/7676)|[BUG] SparkShimsImpl class initialization in SparkShimsSuite for 340 too eager| -|[#8278](https://github.com/NVIDIA/spark-rapids/issues/8278)|[BUG] NDS query 16 hangs at SF30K| -|[#8665](https://github.com/NVIDIA/spark-rapids/issues/8665)|[BUG] EGX nightly tests fail to detect Spark version on startup| -|[#8647](https://github.com/NVIDIA/spark-rapids/issues/8647)|[BUG] array_test.py::test_array_min_max[Float][INJECT_OOM] failed mismatched CPU and GPU output in nightly| -|[#8640](https://github.com/NVIDIA/spark-rapids/issues/8640)|[BUG] Optimize Databricks pre-merge scripts, move it out into a new CI file| -|[#8308](https://github.com/NVIDIA/spark-rapids/issues/8308)|[BUG] Device Memory leak seen in integration_tests when AssertEmptyNulls are enabled| -|[#8602](https://github.com/NVIDIA/spark-rapids/issues/8602)|[BUG] AutoCloseable Broadcast results are getting closed by Spark| -|[#8603](https://github.com/NVIDIA/spark-rapids/issues/8603)|[BUG] SerializeConcatHostBuffersDeserializeBatch.writeObject fails with ArrayIndexOutOfBoundsException on rows-only table| -|[#8615](https://github.com/NVIDIA/spark-rapids/issues/8615)|[BUG] RapidsShuffleThreadedWriterSuite temp shuffle file test failure| -|[#6872](https://github.com/NVIDIA/spark-rapids/issues/6872)|[BUG] awk: cmd. line:1: warning: regexp escape sequence `\ ' is not a known regexp operator| -|[#8588](https://github.com/NVIDIA/spark-rapids/issues/8588)|[BUG] Spark 3.3.x integration tests failed due to missing jars| -|[#7775](https://github.com/NVIDIA/spark-rapids/issues/7775)|[BUG] scala version hardcoded irrespective of Spark dependency| -|[#8548](https://github.com/NVIDIA/spark-rapids/issues/8548)|[BUG] cache_test:test_batch_no_cols test FAILED on spark-3.3.0+| -|[#8579](https://github.com/NVIDIA/spark-rapids/issues/8579)|[BUG] build failed on Databricks clusters "GpuDeleteCommand.scala:104: type mismatch" | -|[#8187](https://github.com/NVIDIA/spark-rapids/issues/8187)|[BUG] Integration test test_window_running_no_part can produce non-empty nulls (cudf scan)| -|[#8493](https://github.com/NVIDIA/spark-rapids/issues/8493)|[BUG] branch-23.08 fails to build on Databricks 12.2| - -### PRs -||| -|:---|:---| -|[#9407](https://github.com/NVIDIA/spark-rapids/pull/9407)|[Doc]Update docs for 23.08.2 version[skip ci]| -|[#9382](https://github.com/NVIDIA/spark-rapids/pull/9382)|Bump up project version to 23.08.2| -|[#8476](https://github.com/NVIDIA/spark-rapids/pull/8476)|Use retry with split in GpuCachedDoublePassWindowIterator| -|[#9048](https://github.com/NVIDIA/spark-rapids/pull/9048)|Update 23.08 changelog 23/08/15 [skip ci]| -|[#9044](https://github.com/NVIDIA/spark-rapids/pull/9044)|[DOC] update release version from v2308.0 to 2308.1 [skip ci]| -|[#9036](https://github.com/NVIDIA/spark-rapids/pull/9036)|Fix meta class cast exception when generator not supported| -|[#9042](https://github.com/NVIDIA/spark-rapids/pull/9042)|Bump up project version to 23.08.1-SNAPSHOT| -|[#9035](https://github.com/NVIDIA/spark-rapids/pull/9035)|Handle null values when merging Bloom filters| -|[#9029](https://github.com/NVIDIA/spark-rapids/pull/9029)|Update 23.08 changelog to latest [skip ci]| -|[#9023](https://github.com/NVIDIA/spark-rapids/pull/9023)|Allow WindowLocalExec to run on CPU for a map test.| -|[#9024](https://github.com/NVIDIA/spark-rapids/pull/9024)|Do not trigger snapshot spark version test in pre-release maven-verify checks [skip ci]| -|[#8975](https://github.com/NVIDIA/spark-rapids/pull/8975)|Init 23.08 changelog [skip ci]| -|[#9016](https://github.com/NVIDIA/spark-rapids/pull/9016)|Fix issue where murmur3 tried to work on array of structs| -|[#9014](https://github.com/NVIDIA/spark-rapids/pull/9014)|Updating link to download jar [skip ci]| -|[#9006](https://github.com/NVIDIA/spark-rapids/pull/9006)|Revert test changes to fix binary dedup error| -|[#9001](https://github.com/NVIDIA/spark-rapids/pull/9001)|[Doc]update the emr getting started doc for emr-6120 release[skip ci]| -|[#8949](https://github.com/NVIDIA/spark-rapids/pull/8949)|Update JNI and private version to released 23.08.0| -|[#8977](https://github.com/NVIDIA/spark-rapids/pull/8977)|Create an anonymous subclass of AdaptiveSparkPlanHelper in ExecutionPlanCaptureCallback.scala| -|[#8972](https://github.com/NVIDIA/spark-rapids/pull/8972)|[Doc]Add best practice doc[skip ci]| -|[#8948](https://github.com/NVIDIA/spark-rapids/pull/8948)|[Doc]update download docs for 2308 version[skip ci]| -|[#8971](https://github.com/NVIDIA/spark-rapids/pull/8971)|Fix test_map_scalars_supported_key_types| -|[#8990](https://github.com/NVIDIA/spark-rapids/pull/8990)|Remove doc references to 312db [skip ci]| -|[#8960](https://github.com/NVIDIA/spark-rapids/pull/8960)|[Doc] address profiling tool formatted issue [skip ci]| -|[#8983](https://github.com/NVIDIA/spark-rapids/pull/8983)|Revert OrcSuite to fix deployment build| -|[#8979](https://github.com/NVIDIA/spark-rapids/pull/8979)|Fix Databricks build error for new added ORC test cases| -|[#8920](https://github.com/NVIDIA/spark-rapids/pull/8920)|Add test case to test orc dictionary encoding with lots of rows for nested types| -|[#8940](https://github.com/NVIDIA/spark-rapids/pull/8940)|Add test case for ORC statistics test| -|[#8909](https://github.com/NVIDIA/spark-rapids/pull/8909)|Match Spark's NaN handling in collect_set| -|[#8892](https://github.com/NVIDIA/spark-rapids/pull/8892)|Experimental support for BloomFilterAggregate expression in a reduction context| -|[#8957](https://github.com/NVIDIA/spark-rapids/pull/8957)|Fix building dockerfile.cuda hanging at tzdata installation [skip ci]| -|[#8944](https://github.com/NVIDIA/spark-rapids/pull/8944)|Fix issues around bloom filter with multple columns| -|[#8744](https://github.com/NVIDIA/spark-rapids/pull/8744)|Add test for selecting a single complex field array and its parent struct array| -|[#8936](https://github.com/NVIDIA/spark-rapids/pull/8936)|Device synchronize prior to freeing a set of RapidsBuffer| -|[#8935](https://github.com/NVIDIA/spark-rapids/pull/8935)|Don't go over shuffle limits on CPU| -|[#8927](https://github.com/NVIDIA/spark-rapids/pull/8927)|Skipping test_map_scalars_supported_key_types because of distributed …| -|[#8931](https://github.com/NVIDIA/spark-rapids/pull/8931)|Clone submodule using git command instead of checkoutSCM plugin| -|[#8917](https://github.com/NVIDIA/spark-rapids/pull/8917)|Databricks shim version for integration test| -|[#8775](https://github.com/NVIDIA/spark-rapids/pull/8775)|Support BloomFilterMightContain expression| -|[#8833](https://github.com/NVIDIA/spark-rapids/pull/8833)|Binary and ternary handling of scalar audit and some fixes| -|[#7233](https://github.com/NVIDIA/spark-rapids/pull/7233)|[FEA] Support `order by` on single-level array| -|[#8893](https://github.com/NVIDIA/spark-rapids/pull/8893)|Fix regression in Hive Generic UDF support on Databricks 12.2| -|[#8828](https://github.com/NVIDIA/spark-rapids/pull/8828)|Put shared part together for Databricks test scripts| -|[#8872](https://github.com/NVIDIA/spark-rapids/pull/8872)|Terminate CI if fail to clone submodule| -|[#8787](https://github.com/NVIDIA/spark-rapids/pull/8787)|Add in support for ExponentialDistribution| -|[#8868](https://github.com/NVIDIA/spark-rapids/pull/8868)|Add a test case for testing ORC version V_0_11 and V_0_12| -|[#8795](https://github.com/NVIDIA/spark-rapids/pull/8795)|Add ORC writing test cases for not implicitly lowercase columns| -|[#8871](https://github.com/NVIDIA/spark-rapids/pull/8871)|Adjust parallelism in spark-tests script to reduce memory footprint [skip ci]| -|[#8869](https://github.com/NVIDIA/spark-rapids/pull/8869)|Specify expected JAVA_HOME and bin for mvn-verify-check [skip ci]| -|[#8785](https://github.com/NVIDIA/spark-rapids/pull/8785)|Add test cases for ORC writing according to options orc.compress and compression| -|[#8810](https://github.com/NVIDIA/spark-rapids/pull/8810)|Fall back to CPU for deletion vectors writes on Databricks| -|[#8830](https://github.com/NVIDIA/spark-rapids/pull/8830)|Update documentation to add Databricks 12.2 as a supported platform [skip ci]| -|[#8799](https://github.com/NVIDIA/spark-rapids/pull/8799)|Add tests to cover some odd corner cases with nulls and empty arrays| -|[#8783](https://github.com/NVIDIA/spark-rapids/pull/8783)|Fix collect_set_on_nested_type tests failed| -|[#8855](https://github.com/NVIDIA/spark-rapids/pull/8855)|Fix bug: Check GPU file instead of CPU file [skip ci]| -|[#8852](https://github.com/NVIDIA/spark-rapids/pull/8852)|Update test scripts and dockerfiles to match cudf conda pkg change [skip ci]| -|[#8848](https://github.com/NVIDIA/spark-rapids/pull/8848)|Try mitigate mismatched JDK versions in mvn-verify checks [skip ci]| -|[#8825](https://github.com/NVIDIA/spark-rapids/pull/8825)|Add a case to test ORC writing/reading with lots of nulls| -|[#8802](https://github.com/NVIDIA/spark-rapids/pull/8802)|Treat unbounded windows as truly non-finite.| -|[#8798](https://github.com/NVIDIA/spark-rapids/pull/8798)|Add ORC writing test cases for dictionary compression| -|[#8829](https://github.com/NVIDIA/spark-rapids/pull/8829)|Enable rle_boolean_encoding.parquet test| -|[#8667](https://github.com/NVIDIA/spark-rapids/pull/8667)|Make state spillable in partitioned writer| -|[#8801](https://github.com/NVIDIA/spark-rapids/pull/8801)|Fix shuffling an empty Struct() column with UCX| -|[#8748](https://github.com/NVIDIA/spark-rapids/pull/8748)|Add driver log warning when GPU is limiting scheduling resource| -|[#8786](https://github.com/NVIDIA/spark-rapids/pull/8786)|Add support for row-based execution in RapidsDeltaWrite| -|[#8791](https://github.com/NVIDIA/spark-rapids/pull/8791)|Auto merge to branch-23.10 from branch-23.08[skip ci]| -|[#8790](https://github.com/NVIDIA/spark-rapids/pull/8790)|Update ubuntu dockerfiles default to 20.04 and deprecating centos one [skip ci]| -|[#8777](https://github.com/NVIDIA/spark-rapids/pull/8777)|Install python packages with shared scripts on Databricks| -|[#8772](https://github.com/NVIDIA/spark-rapids/pull/8772)|Test concurrent writer update file metrics| -|[#8646](https://github.com/NVIDIA/spark-rapids/pull/8646)|Add testing of Parquet files from apache/parquet-testing| -|[#8684](https://github.com/NVIDIA/spark-rapids/pull/8684)|Add 'submodule update --init' when build spark-rapids| -|[#8769](https://github.com/NVIDIA/spark-rapids/pull/8769)|Remove iceberg scripts from Databricks test scripts| -|[#8773](https://github.com/NVIDIA/spark-rapids/pull/8773)|Add a test case for reading/write null to ORC| -|[#8749](https://github.com/NVIDIA/spark-rapids/pull/8749)|Add test cases for read/write User Defined Type (UDT) to ORC| -|[#8768](https://github.com/NVIDIA/spark-rapids/pull/8768)|Add support for xxhash64| -|[#8751](https://github.com/NVIDIA/spark-rapids/pull/8751)|Ensure columnarEval always returns a GpuColumnVector| -|[#8765](https://github.com/NVIDIA/spark-rapids/pull/8765)|Add in support for maps to big data gen| -|[#8758](https://github.com/NVIDIA/spark-rapids/pull/8758)|Normal and Multi Distributions for BigDataGen| -|[#8755](https://github.com/NVIDIA/spark-rapids/pull/8755)|Add in dependency for databricks on integration tests| -|[#8737](https://github.com/NVIDIA/spark-rapids/pull/8737)|Fix parquet_write_test.py::test_hive_timestamp_value failure for Databricks| -|[#8745](https://github.com/NVIDIA/spark-rapids/pull/8745)|Conventional jar layout is not required for JDK9+| -|[#8706](https://github.com/NVIDIA/spark-rapids/pull/8706)|Add a tool to support generating large amounts of data| -|[#8747](https://github.com/NVIDIA/spark-rapids/pull/8747)|xfail hash_groupby_collect_set and hash_reduction_collect_set on nested type cases| -|[#8689](https://github.com/NVIDIA/spark-rapids/pull/8689)|Support nested arrays for `min`/`max` aggregations in groupby and reduction| -|[#8699](https://github.com/NVIDIA/spark-rapids/pull/8699)|Regression test for array of struct with a single field name "element" in Parquet| -|[#8733](https://github.com/NVIDIA/spark-rapids/pull/8733)|Avoid generating numeric null partition values on Databricks 10.4| -|[#8728](https://github.com/NVIDIA/spark-rapids/pull/8728)|Use specific mamba version and install libarchive explictly [skip ci]| -|[#8594](https://github.com/NVIDIA/spark-rapids/pull/8594)|String generation from complex regex in integration tests| -|[#8700](https://github.com/NVIDIA/spark-rapids/pull/8700)|Add regression test to ensure Parquet doesn't interpret timestamp values differently from Hive 0.14.0+| -|[#8711](https://github.com/NVIDIA/spark-rapids/pull/8711)|Factor out modules shared among shim profiles| -|[#8697](https://github.com/NVIDIA/spark-rapids/pull/8697)|Spillable columnar batch takes ownership and improve code coverage| -|[#8705](https://github.com/NVIDIA/spark-rapids/pull/8705)|Add schema evolution integration tests for partitioned data| -|[#8673](https://github.com/NVIDIA/spark-rapids/pull/8673)|Fix some potential memory leaks| -|[#8707](https://github.com/NVIDIA/spark-rapids/pull/8707)|Update config docs for new filecache configs [skip ci]| -|[#8695](https://github.com/NVIDIA/spark-rapids/pull/8695)|Always create the main artifact along with a shim-classifier artifact| -|[#8704](https://github.com/NVIDIA/spark-rapids/pull/8704)|Add tests for column names with dots| -|[#8703](https://github.com/NVIDIA/spark-rapids/pull/8703)|Comment out min/max agg test for nested structs to unblock CI| -|[#8698](https://github.com/NVIDIA/spark-rapids/pull/8698)|Cache last ORC stripe footer to avoid redundant remote reads| -|[#8687](https://github.com/NVIDIA/spark-rapids/pull/8687)|Handle TIMESTAMP_MILLIS for rebase check| -|[#8688](https://github.com/NVIDIA/spark-rapids/pull/8688)|Enable the 340 shim test| -|[#8656](https://github.com/NVIDIA/spark-rapids/pull/8656)|Return result from filecache message instead of null| -|[#8659](https://github.com/NVIDIA/spark-rapids/pull/8659)|Filter out nulls for build batches when needed in hash joins| -|[#8682](https://github.com/NVIDIA/spark-rapids/pull/8682)|[DOC] Update CUDA requirements in documentation and Dockerfiles[skip ci]| -|[#8637](https://github.com/NVIDIA/spark-rapids/pull/8637)|Support Float order-by columns for RANGE window functions| -|[#8681](https://github.com/NVIDIA/spark-rapids/pull/8681)|changed container name to adapt to blossom-lib refactor [skip ci]| -|[#8573](https://github.com/NVIDIA/spark-rapids/pull/8573)|Add support for Delta Lake 2.4.0| -|[#8671](https://github.com/NVIDIA/spark-rapids/pull/8671)|Fix use-after-freed bug in `GpuFloatArrayMin`| -|[#8650](https://github.com/NVIDIA/spark-rapids/pull/8650)|Support TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS| -|[#8495](https://github.com/NVIDIA/spark-rapids/pull/8495)|Speed up PCBS CPU read path by not recalculating as much| -|[#8389](https://github.com/NVIDIA/spark-rapids/pull/8389)|Add filecache support for ORC| -|[#8658](https://github.com/NVIDIA/spark-rapids/pull/8658)|Check if need to run Databricks pre-merge| -|[#8649](https://github.com/NVIDIA/spark-rapids/pull/8649)|Add Spark 3.4.1 shim| -|[#8624](https://github.com/NVIDIA/spark-rapids/pull/8624)|Rename numBytesAdded/Removed metrics and add deletion vector metrics in Databricks 12.2 shims| -|[#8645](https://github.com/NVIDIA/spark-rapids/pull/8645)|Fix "PytestUnknownMarkWarning: Unknown pytest.mark.inject_oom" warning| -|[#8608](https://github.com/NVIDIA/spark-rapids/pull/8608)|Matrix stages to dynamically build Databricks shims| -|[#8517](https://github.com/NVIDIA/spark-rapids/pull/8517)|Revert "Disable asserts for non-empty nulls (#8183)"| -|[#8628](https://github.com/NVIDIA/spark-rapids/pull/8628)|Enable Delta Write fallback tests on Databricks 12.2| -|[#8632](https://github.com/NVIDIA/spark-rapids/pull/8632)|Fix GCP examples and getting started guide [skip ci]| -|[#8638](https://github.com/NVIDIA/spark-rapids/pull/8638)|Support nested structs for `min`/`max` aggregations in groupby and reduction| -|[#8639](https://github.com/NVIDIA/spark-rapids/pull/8639)|Add iceberg test for nightly DB12.2 IT pipeline[skip ci]| -|[#8618](https://github.com/NVIDIA/spark-rapids/pull/8618)|Heuristic to speed up partial aggregates that get larger| -|[#8605](https://github.com/NVIDIA/spark-rapids/pull/8605)|[Doc] Fix demo link in index.md [skip ci]| -|[#8619](https://github.com/NVIDIA/spark-rapids/pull/8619)|Enable output batches metric for GpuShuffleCoalesceExec by default| -|[#8617](https://github.com/NVIDIA/spark-rapids/pull/8617)|Fixes broadcast spill serialization/deserialization| -|[#8531](https://github.com/NVIDIA/spark-rapids/pull/8531)|filecache: Modify FileCacheLocalityManager.init to pass in Spark context| -|[#8613](https://github.com/NVIDIA/spark-rapids/pull/8613)|Try print JVM core dump files if any test failures in CI| -|[#8616](https://github.com/NVIDIA/spark-rapids/pull/8616)|Wait for futures in multi-threaded writers even on exception| -|[#8578](https://github.com/NVIDIA/spark-rapids/pull/8578)|Add in metric to see how much computation time is lost due to retry| -|[#8590](https://github.com/NVIDIA/spark-rapids/pull/8590)|Drop ".dev0" suffix from Spark SNASHOT distro builds| -|[#8604](https://github.com/NVIDIA/spark-rapids/pull/8604)|Upgrade scalatest version to 3.2.16| -|[#8555](https://github.com/NVIDIA/spark-rapids/pull/8555)|Support `flatten` SQL function| -|[#8599](https://github.com/NVIDIA/spark-rapids/pull/8599)|Fix broken links in advanced_configs.md| -|[#8589](https://github.com/NVIDIA/spark-rapids/pull/8589)|Revert to the JVM-based Spark version extraction in pytests| -|[#8582](https://github.com/NVIDIA/spark-rapids/pull/8582)|Fix databricks shims build errors caused by DB updates| -|[#8564](https://github.com/NVIDIA/spark-rapids/pull/8564)|Fold `verify-all-modules-with-headSparkVersion` into `verify-all-modules` [skip ci]| -|[#8553](https://github.com/NVIDIA/spark-rapids/pull/8553)|Handle empty batch in ParquetCachedBatchSerializer| -|[#8575](https://github.com/NVIDIA/spark-rapids/pull/8575)|Corrected typos in CONTRIBUTING.md [skip ci]| -|[#8574](https://github.com/NVIDIA/spark-rapids/pull/8574)|Remove maxTaskFailures=4 for pre-3.1.1 Spark| -|[#8503](https://github.com/NVIDIA/spark-rapids/pull/8503)|Remove hard-coded version numbers for dependencies when building on| -|[#8544](https://github.com/NVIDIA/spark-rapids/pull/8544)|Fix auto merge conflict 8543 [skip ci]| -|[#8521](https://github.com/NVIDIA/spark-rapids/pull/8521)|List supported Spark versions when no shim found| -|[#8520](https://github.com/NVIDIA/spark-rapids/pull/8520)|Add support for first, last, nth, and collect_list aggregations for BinaryType| -|[#8509](https://github.com/NVIDIA/spark-rapids/pull/8509)|Remove legacy spark version check| -|[#8494](https://github.com/NVIDIA/spark-rapids/pull/8494)|Fix 23.08 build on Databricks 12.2| -|[#8487](https://github.com/NVIDIA/spark-rapids/pull/8487)|Move MockTaskContext to tests project| -|[#8426](https://github.com/NVIDIA/spark-rapids/pull/8426)|Pre-merge CI to support Databricks 12.2| -|[#8282](https://github.com/NVIDIA/spark-rapids/pull/8282)|Databricks 12.2 Support| -|[#8407](https://github.com/NVIDIA/spark-rapids/pull/8407)|Bump up dep version to 23.08.0-SNAPSHOT| -|[#8359](https://github.com/NVIDIA/spark-rapids/pull/8359)|Init version 23.08.0-SNAPSHOT| - ## Older Releases Changelog of older releases can be found at [docs/archives](/docs/archives) diff --git a/docs/archives/CHANGELOG_23.02_to_23.06.md b/docs/archives/CHANGELOG_23.02_to_23.08.md similarity index 76% rename from docs/archives/CHANGELOG_23.02_to_23.06.md rename to docs/archives/CHANGELOG_23.02_to_23.08.md index 9b57989dbb2..7423e378aca 100644 --- a/docs/archives/CHANGELOG_23.02_to_23.06.md +++ b/docs/archives/CHANGELOG_23.02_to_23.08.md @@ -1,6 +1,242 @@ # Change log Generated on 2023-10-12 +## Release 23.08 + +### Features +||| +|:---|:---| +|[#5509](https://github.com/NVIDIA/spark-rapids/issues/5509)|[FEA] Support order-by on Array| +|[#7876](https://github.com/NVIDIA/spark-rapids/issues/7876)|[FEA] Add initial support for Databricks 12.2 ML LTS| +|[#8547](https://github.com/NVIDIA/spark-rapids/issues/8547)|[FEA] Add support for Delta Lake 2.4 with Spark 3.4| +|[#8633](https://github.com/NVIDIA/spark-rapids/issues/8633)|[FEA] Add support for xxHash64 function| +|[#4929](https://github.com/NVIDIA/spark-rapids/issues/4929)|[FEA] Support min/max aggregation/reduction for arrays of structs and arrays of strings| +|[#8668](https://github.com/NVIDIA/spark-rapids/issues/8668)|[FEA] Support min and max for arrays| +|[#4887](https://github.com/NVIDIA/spark-rapids/issues/4887)|[FEA] Hash partitioning on ArrayType| +|[#6680](https://github.com/NVIDIA/spark-rapids/issues/6680)|[FEA] Support hashaggregate for Array[Any]| +|[#8085](https://github.com/NVIDIA/spark-rapids/issues/8085)|[FEA] Add support for MillisToTimestamp| +|[#7801](https://github.com/NVIDIA/spark-rapids/issues/7801)|[FEA] Window Expression orderBy column is not supported in a window range function, found DoubleType| +|[#8556](https://github.com/NVIDIA/spark-rapids/issues/8556)|[FEA] [Delta Lake] Add support for new metrics in MERGE| +|[#308](https://github.com/NVIDIA/spark-rapids/issues/308)|[FEA] Spark 3.1 adding support for TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS functions| +|[#8122](https://github.com/NVIDIA/spark-rapids/issues/8122)|[FEA] Add spark 3.4.1 snapshot shim| +|[#8525](https://github.com/NVIDIA/spark-rapids/issues/8525)|[FEA] Add support for org.apache.spark.sql.functions.flatten| +|[#8202](https://github.com/NVIDIA/spark-rapids/issues/8202)|[FEA] List supported Spark builds when the Shim is not found| + +### Performance +||| +|:---|:---| +|[#8231](https://github.com/NVIDIA/spark-rapids/issues/8231)|[FEA] Add filecache support to ORC scans| +|[#8141](https://github.com/NVIDIA/spark-rapids/issues/8141)|[FEA] Explore how to best deal with large numbers of aggregations in the short term| + +### Bugs Fixed +||| +|:---|:---| +|[#9034](https://github.com/NVIDIA/spark-rapids/issues/9034)|[BUG] java.lang.ClassCastException: com.nvidia.spark.rapids.RuleNotFoundExprMeta cannot be cast to com.nvidia.spark.rapids.GeneratorExprMeta| +|[#9032](https://github.com/NVIDIA/spark-rapids/issues/9032)|[BUG] Multiple NDS queries fail with Spark-3.4.1 with bloom filter exception| +|[#8962](https://github.com/NVIDIA/spark-rapids/issues/8962)|[BUG] Nightly build failed: ExecutionPlanCaptureCallback$.class is not bitwise-identical across shims| +|[#9021](https://github.com/NVIDIA/spark-rapids/issues/9021)|[BUG] test_map_scalars_supported_key_types failed in dataproc 2.1| +|[#9020](https://github.com/NVIDIA/spark-rapids/issues/9020)|[BUG] auto-disable snapshot shims test in github action for pre-release branch| +|[#9010](https://github.com/NVIDIA/spark-rapids/issues/9010)|[BUG] Customer failure 23.08: Cannot compute hash of a table with a LIST of STRUCT columns.| +|[#8922](https://github.com/NVIDIA/spark-rapids/issues/8922)|[BUG] integration map_test:test_map_scalars_supported_key_types failures| +|[#8982](https://github.com/NVIDIA/spark-rapids/issues/8982)|[BUG] Nightly prerelease failures - OrcSuite| +|[#8978](https://github.com/NVIDIA/spark-rapids/issues/8978)|[BUG] compiling error due to OrcSuite&OrcStatisticShim in databricks runtimes| +|[#8610](https://github.com/NVIDIA/spark-rapids/issues/8610)|[BUG] query 95 @ SF30K fails with OOM exception| +|[#8955](https://github.com/NVIDIA/spark-rapids/issues/8955)|[BUG] Bloom filter join tests can fail with multiple join columns| +|[#45](https://github.com/NVIDIA/spark-rapids/issues/45)|[BUG] very large shuffles can fail| +|[#8779](https://github.com/NVIDIA/spark-rapids/issues/8779)|[BUG] Put shared Databricks test script together for ease of maintenance| +|[#8930](https://github.com/NVIDIA/spark-rapids/issues/8930)|[BUG] checkoutSCM plugin is unstable for pre-merge CI, it is often unable to clone submodules| +|[#8923](https://github.com/NVIDIA/spark-rapids/issues/8923)|[BUG] Mortgage test failing with 'JavaPackage' error on AWS Databricks| +|[#8303](https://github.com/NVIDIA/spark-rapids/issues/8303)|[BUG] GpuExpression columnarEval can return scalars from subqueries that may be unhandled| +|[#8318](https://github.com/NVIDIA/spark-rapids/issues/8318)|[BUG][Databricks 12.2] GpuRowBasedHiveGenericUDF ClassCastException| +|[#8822](https://github.com/NVIDIA/spark-rapids/issues/8822)|[BUG] Early terminate CI if submodule init failed| +|[#8847](https://github.com/NVIDIA/spark-rapids/issues/8847)|[BUG] github actions CI messed up w/ JDK versions intermittently| +|[#8716](https://github.com/NVIDIA/spark-rapids/issues/8716)|[BUG] `test_hash_groupby_collect_set_on_nested_type` and `test_hash_reduction_collect_set_on_nested_type` failed| +|[#8827](https://github.com/NVIDIA/spark-rapids/issues/8827)|[BUG] databricks cudf_udf night build failing with pool size exceeded errors| +|[#8630](https://github.com/NVIDIA/spark-rapids/issues/8630)|[BUG] Parquet with RLE encoded booleans loads corrupted data| +|[#8735](https://github.com/NVIDIA/spark-rapids/issues/8735)|[BUG] test_orc_column_name_with_dots fails in nightly EGX tests| +|[#6980](https://github.com/NVIDIA/spark-rapids/issues/6980)|[BUG] Partitioned writes release GPU semaphore with unspillable GPU memory| +|[#8784](https://github.com/NVIDIA/spark-rapids/issues/8784)|[BUG] hash_aggregate_test.py::test_min_max_in_groupby_and_reduction failed on "TypeError: object of type 'NoneType' has no len()"| +|[#8756](https://github.com/NVIDIA/spark-rapids/issues/8756)|[BUG] [Databricks 12.2] RapidsDeltaWrite queries that reference internal metadata fail to run| +|[#8636](https://github.com/NVIDIA/spark-rapids/issues/8636)|[BUG] AWS Databricks 12.2 integration tests failed due to Iceberg check| +|[#8754](https://github.com/NVIDIA/spark-rapids/issues/8754)|[BUG] databricks build broke after adding bigDataGen| +|[#8726](https://github.com/NVIDIA/spark-rapids/issues/8726)|[BUG] Test "parquet_write_test.py::test_hive_timestamp_value[INJECT_OOM]" failed on Databricks | +|[#8690](https://github.com/NVIDIA/spark-rapids/issues/8690)|[BUG buildall script does not support JDK11 profile| +|[#8702](https://github.com/NVIDIA/spark-rapids/issues/8702)|[BUG] test_min_max_for_single_level_struct failed| +|[#8727](https://github.com/NVIDIA/spark-rapids/issues/8727)|[BUG] test_column_add_after_partition failed in databricks 10.4 runtime| +|[#8669](https://github.com/NVIDIA/spark-rapids/issues/8669)|[BUG] SpillableColumnarBatch doesn't always take ownership| +|[#8655](https://github.com/NVIDIA/spark-rapids/issues/8655)|[BUG] There are some potential device memory leaks in `AbstractGpuCoalesceIterator`| +|[#8685](https://github.com/NVIDIA/spark-rapids/issues/8685)|[BUG] install build fails with Maven 3.9.3| +|[#8156](https://github.com/NVIDIA/spark-rapids/issues/8156)|[BUG] Install phase for modules with Spark build classifier fails for install plugin versions 3.0.0+| +|[#1130](https://github.com/NVIDIA/spark-rapids/issues/1130)|[BUG] TIMESTAMP_MILLIS not handled in isDateTimeRebaseNeeded| +|[#7676](https://github.com/NVIDIA/spark-rapids/issues/7676)|[BUG] SparkShimsImpl class initialization in SparkShimsSuite for 340 too eager| +|[#8278](https://github.com/NVIDIA/spark-rapids/issues/8278)|[BUG] NDS query 16 hangs at SF30K| +|[#8665](https://github.com/NVIDIA/spark-rapids/issues/8665)|[BUG] EGX nightly tests fail to detect Spark version on startup| +|[#8647](https://github.com/NVIDIA/spark-rapids/issues/8647)|[BUG] array_test.py::test_array_min_max[Float][INJECT_OOM] failed mismatched CPU and GPU output in nightly| +|[#8640](https://github.com/NVIDIA/spark-rapids/issues/8640)|[BUG] Optimize Databricks pre-merge scripts, move it out into a new CI file| +|[#8308](https://github.com/NVIDIA/spark-rapids/issues/8308)|[BUG] Device Memory leak seen in integration_tests when AssertEmptyNulls are enabled| +|[#8602](https://github.com/NVIDIA/spark-rapids/issues/8602)|[BUG] AutoCloseable Broadcast results are getting closed by Spark| +|[#8603](https://github.com/NVIDIA/spark-rapids/issues/8603)|[BUG] SerializeConcatHostBuffersDeserializeBatch.writeObject fails with ArrayIndexOutOfBoundsException on rows-only table| +|[#8615](https://github.com/NVIDIA/spark-rapids/issues/8615)|[BUG] RapidsShuffleThreadedWriterSuite temp shuffle file test failure| +|[#6872](https://github.com/NVIDIA/spark-rapids/issues/6872)|[BUG] awk: cmd. line:1: warning: regexp escape sequence `\ ' is not a known regexp operator| +|[#8588](https://github.com/NVIDIA/spark-rapids/issues/8588)|[BUG] Spark 3.3.x integration tests failed due to missing jars| +|[#7775](https://github.com/NVIDIA/spark-rapids/issues/7775)|[BUG] scala version hardcoded irrespective of Spark dependency| +|[#8548](https://github.com/NVIDIA/spark-rapids/issues/8548)|[BUG] cache_test:test_batch_no_cols test FAILED on spark-3.3.0+| +|[#8579](https://github.com/NVIDIA/spark-rapids/issues/8579)|[BUG] build failed on Databricks clusters "GpuDeleteCommand.scala:104: type mismatch" | +|[#8187](https://github.com/NVIDIA/spark-rapids/issues/8187)|[BUG] Integration test test_window_running_no_part can produce non-empty nulls (cudf scan)| +|[#8493](https://github.com/NVIDIA/spark-rapids/issues/8493)|[BUG] branch-23.08 fails to build on Databricks 12.2| + +### PRs +||| +|:---|:---| +|[#9407](https://github.com/NVIDIA/spark-rapids/pull/9407)|[Doc]Update docs for 23.08.2 version[skip ci]| +|[#9382](https://github.com/NVIDIA/spark-rapids/pull/9382)|Bump up project version to 23.08.2| +|[#8476](https://github.com/NVIDIA/spark-rapids/pull/8476)|Use retry with split in GpuCachedDoublePassWindowIterator| +|[#9048](https://github.com/NVIDIA/spark-rapids/pull/9048)|Update 23.08 changelog 23/08/15 [skip ci]| +|[#9044](https://github.com/NVIDIA/spark-rapids/pull/9044)|[DOC] update release version from v2308.0 to 2308.1 [skip ci]| +|[#9036](https://github.com/NVIDIA/spark-rapids/pull/9036)|Fix meta class cast exception when generator not supported| +|[#9042](https://github.com/NVIDIA/spark-rapids/pull/9042)|Bump up project version to 23.08.1-SNAPSHOT| +|[#9035](https://github.com/NVIDIA/spark-rapids/pull/9035)|Handle null values when merging Bloom filters| +|[#9029](https://github.com/NVIDIA/spark-rapids/pull/9029)|Update 23.08 changelog to latest [skip ci]| +|[#9023](https://github.com/NVIDIA/spark-rapids/pull/9023)|Allow WindowLocalExec to run on CPU for a map test.| +|[#9024](https://github.com/NVIDIA/spark-rapids/pull/9024)|Do not trigger snapshot spark version test in pre-release maven-verify checks [skip ci]| +|[#8975](https://github.com/NVIDIA/spark-rapids/pull/8975)|Init 23.08 changelog [skip ci]| +|[#9016](https://github.com/NVIDIA/spark-rapids/pull/9016)|Fix issue where murmur3 tried to work on array of structs| +|[#9014](https://github.com/NVIDIA/spark-rapids/pull/9014)|Updating link to download jar [skip ci]| +|[#9006](https://github.com/NVIDIA/spark-rapids/pull/9006)|Revert test changes to fix binary dedup error| +|[#9001](https://github.com/NVIDIA/spark-rapids/pull/9001)|[Doc]update the emr getting started doc for emr-6120 release[skip ci]| +|[#8949](https://github.com/NVIDIA/spark-rapids/pull/8949)|Update JNI and private version to released 23.08.0| +|[#8977](https://github.com/NVIDIA/spark-rapids/pull/8977)|Create an anonymous subclass of AdaptiveSparkPlanHelper in ExecutionPlanCaptureCallback.scala| +|[#8972](https://github.com/NVIDIA/spark-rapids/pull/8972)|[Doc]Add best practice doc[skip ci]| +|[#8948](https://github.com/NVIDIA/spark-rapids/pull/8948)|[Doc]update download docs for 2308 version[skip ci]| +|[#8971](https://github.com/NVIDIA/spark-rapids/pull/8971)|Fix test_map_scalars_supported_key_types| +|[#8990](https://github.com/NVIDIA/spark-rapids/pull/8990)|Remove doc references to 312db [skip ci]| +|[#8960](https://github.com/NVIDIA/spark-rapids/pull/8960)|[Doc] address profiling tool formatted issue [skip ci]| +|[#8983](https://github.com/NVIDIA/spark-rapids/pull/8983)|Revert OrcSuite to fix deployment build| +|[#8979](https://github.com/NVIDIA/spark-rapids/pull/8979)|Fix Databricks build error for new added ORC test cases| +|[#8920](https://github.com/NVIDIA/spark-rapids/pull/8920)|Add test case to test orc dictionary encoding with lots of rows for nested types| +|[#8940](https://github.com/NVIDIA/spark-rapids/pull/8940)|Add test case for ORC statistics test| +|[#8909](https://github.com/NVIDIA/spark-rapids/pull/8909)|Match Spark's NaN handling in collect_set| +|[#8892](https://github.com/NVIDIA/spark-rapids/pull/8892)|Experimental support for BloomFilterAggregate expression in a reduction context| +|[#8957](https://github.com/NVIDIA/spark-rapids/pull/8957)|Fix building dockerfile.cuda hanging at tzdata installation [skip ci]| +|[#8944](https://github.com/NVIDIA/spark-rapids/pull/8944)|Fix issues around bloom filter with multple columns| +|[#8744](https://github.com/NVIDIA/spark-rapids/pull/8744)|Add test for selecting a single complex field array and its parent struct array| +|[#8936](https://github.com/NVIDIA/spark-rapids/pull/8936)|Device synchronize prior to freeing a set of RapidsBuffer| +|[#8935](https://github.com/NVIDIA/spark-rapids/pull/8935)|Don't go over shuffle limits on CPU| +|[#8927](https://github.com/NVIDIA/spark-rapids/pull/8927)|Skipping test_map_scalars_supported_key_types because of distributed …| +|[#8931](https://github.com/NVIDIA/spark-rapids/pull/8931)|Clone submodule using git command instead of checkoutSCM plugin| +|[#8917](https://github.com/NVIDIA/spark-rapids/pull/8917)|Databricks shim version for integration test| +|[#8775](https://github.com/NVIDIA/spark-rapids/pull/8775)|Support BloomFilterMightContain expression| +|[#8833](https://github.com/NVIDIA/spark-rapids/pull/8833)|Binary and ternary handling of scalar audit and some fixes| +|[#7233](https://github.com/NVIDIA/spark-rapids/pull/7233)|[FEA] Support `order by` on single-level array| +|[#8893](https://github.com/NVIDIA/spark-rapids/pull/8893)|Fix regression in Hive Generic UDF support on Databricks 12.2| +|[#8828](https://github.com/NVIDIA/spark-rapids/pull/8828)|Put shared part together for Databricks test scripts| +|[#8872](https://github.com/NVIDIA/spark-rapids/pull/8872)|Terminate CI if fail to clone submodule| +|[#8787](https://github.com/NVIDIA/spark-rapids/pull/8787)|Add in support for ExponentialDistribution| +|[#8868](https://github.com/NVIDIA/spark-rapids/pull/8868)|Add a test case for testing ORC version V_0_11 and V_0_12| +|[#8795](https://github.com/NVIDIA/spark-rapids/pull/8795)|Add ORC writing test cases for not implicitly lowercase columns| +|[#8871](https://github.com/NVIDIA/spark-rapids/pull/8871)|Adjust parallelism in spark-tests script to reduce memory footprint [skip ci]| +|[#8869](https://github.com/NVIDIA/spark-rapids/pull/8869)|Specify expected JAVA_HOME and bin for mvn-verify-check [skip ci]| +|[#8785](https://github.com/NVIDIA/spark-rapids/pull/8785)|Add test cases for ORC writing according to options orc.compress and compression| +|[#8810](https://github.com/NVIDIA/spark-rapids/pull/8810)|Fall back to CPU for deletion vectors writes on Databricks| +|[#8830](https://github.com/NVIDIA/spark-rapids/pull/8830)|Update documentation to add Databricks 12.2 as a supported platform [skip ci]| +|[#8799](https://github.com/NVIDIA/spark-rapids/pull/8799)|Add tests to cover some odd corner cases with nulls and empty arrays| +|[#8783](https://github.com/NVIDIA/spark-rapids/pull/8783)|Fix collect_set_on_nested_type tests failed| +|[#8855](https://github.com/NVIDIA/spark-rapids/pull/8855)|Fix bug: Check GPU file instead of CPU file [skip ci]| +|[#8852](https://github.com/NVIDIA/spark-rapids/pull/8852)|Update test scripts and dockerfiles to match cudf conda pkg change [skip ci]| +|[#8848](https://github.com/NVIDIA/spark-rapids/pull/8848)|Try mitigate mismatched JDK versions in mvn-verify checks [skip ci]| +|[#8825](https://github.com/NVIDIA/spark-rapids/pull/8825)|Add a case to test ORC writing/reading with lots of nulls| +|[#8802](https://github.com/NVIDIA/spark-rapids/pull/8802)|Treat unbounded windows as truly non-finite.| +|[#8798](https://github.com/NVIDIA/spark-rapids/pull/8798)|Add ORC writing test cases for dictionary compression| +|[#8829](https://github.com/NVIDIA/spark-rapids/pull/8829)|Enable rle_boolean_encoding.parquet test| +|[#8667](https://github.com/NVIDIA/spark-rapids/pull/8667)|Make state spillable in partitioned writer| +|[#8801](https://github.com/NVIDIA/spark-rapids/pull/8801)|Fix shuffling an empty Struct() column with UCX| +|[#8748](https://github.com/NVIDIA/spark-rapids/pull/8748)|Add driver log warning when GPU is limiting scheduling resource| +|[#8786](https://github.com/NVIDIA/spark-rapids/pull/8786)|Add support for row-based execution in RapidsDeltaWrite| +|[#8791](https://github.com/NVIDIA/spark-rapids/pull/8791)|Auto merge to branch-23.10 from branch-23.08[skip ci]| +|[#8790](https://github.com/NVIDIA/spark-rapids/pull/8790)|Update ubuntu dockerfiles default to 20.04 and deprecating centos one [skip ci]| +|[#8777](https://github.com/NVIDIA/spark-rapids/pull/8777)|Install python packages with shared scripts on Databricks| +|[#8772](https://github.com/NVIDIA/spark-rapids/pull/8772)|Test concurrent writer update file metrics| +|[#8646](https://github.com/NVIDIA/spark-rapids/pull/8646)|Add testing of Parquet files from apache/parquet-testing| +|[#8684](https://github.com/NVIDIA/spark-rapids/pull/8684)|Add 'submodule update --init' when build spark-rapids| +|[#8769](https://github.com/NVIDIA/spark-rapids/pull/8769)|Remove iceberg scripts from Databricks test scripts| +|[#8773](https://github.com/NVIDIA/spark-rapids/pull/8773)|Add a test case for reading/write null to ORC| +|[#8749](https://github.com/NVIDIA/spark-rapids/pull/8749)|Add test cases for read/write User Defined Type (UDT) to ORC| +|[#8768](https://github.com/NVIDIA/spark-rapids/pull/8768)|Add support for xxhash64| +|[#8751](https://github.com/NVIDIA/spark-rapids/pull/8751)|Ensure columnarEval always returns a GpuColumnVector| +|[#8765](https://github.com/NVIDIA/spark-rapids/pull/8765)|Add in support for maps to big data gen| +|[#8758](https://github.com/NVIDIA/spark-rapids/pull/8758)|Normal and Multi Distributions for BigDataGen| +|[#8755](https://github.com/NVIDIA/spark-rapids/pull/8755)|Add in dependency for databricks on integration tests| +|[#8737](https://github.com/NVIDIA/spark-rapids/pull/8737)|Fix parquet_write_test.py::test_hive_timestamp_value failure for Databricks| +|[#8745](https://github.com/NVIDIA/spark-rapids/pull/8745)|Conventional jar layout is not required for JDK9+| +|[#8706](https://github.com/NVIDIA/spark-rapids/pull/8706)|Add a tool to support generating large amounts of data| +|[#8747](https://github.com/NVIDIA/spark-rapids/pull/8747)|xfail hash_groupby_collect_set and hash_reduction_collect_set on nested type cases| +|[#8689](https://github.com/NVIDIA/spark-rapids/pull/8689)|Support nested arrays for `min`/`max` aggregations in groupby and reduction| +|[#8699](https://github.com/NVIDIA/spark-rapids/pull/8699)|Regression test for array of struct with a single field name "element" in Parquet| +|[#8733](https://github.com/NVIDIA/spark-rapids/pull/8733)|Avoid generating numeric null partition values on Databricks 10.4| +|[#8728](https://github.com/NVIDIA/spark-rapids/pull/8728)|Use specific mamba version and install libarchive explictly [skip ci]| +|[#8594](https://github.com/NVIDIA/spark-rapids/pull/8594)|String generation from complex regex in integration tests| +|[#8700](https://github.com/NVIDIA/spark-rapids/pull/8700)|Add regression test to ensure Parquet doesn't interpret timestamp values differently from Hive 0.14.0+| +|[#8711](https://github.com/NVIDIA/spark-rapids/pull/8711)|Factor out modules shared among shim profiles| +|[#8697](https://github.com/NVIDIA/spark-rapids/pull/8697)|Spillable columnar batch takes ownership and improve code coverage| +|[#8705](https://github.com/NVIDIA/spark-rapids/pull/8705)|Add schema evolution integration tests for partitioned data| +|[#8673](https://github.com/NVIDIA/spark-rapids/pull/8673)|Fix some potential memory leaks| +|[#8707](https://github.com/NVIDIA/spark-rapids/pull/8707)|Update config docs for new filecache configs [skip ci]| +|[#8695](https://github.com/NVIDIA/spark-rapids/pull/8695)|Always create the main artifact along with a shim-classifier artifact| +|[#8704](https://github.com/NVIDIA/spark-rapids/pull/8704)|Add tests for column names with dots| +|[#8703](https://github.com/NVIDIA/spark-rapids/pull/8703)|Comment out min/max agg test for nested structs to unblock CI| +|[#8698](https://github.com/NVIDIA/spark-rapids/pull/8698)|Cache last ORC stripe footer to avoid redundant remote reads| +|[#8687](https://github.com/NVIDIA/spark-rapids/pull/8687)|Handle TIMESTAMP_MILLIS for rebase check| +|[#8688](https://github.com/NVIDIA/spark-rapids/pull/8688)|Enable the 340 shim test| +|[#8656](https://github.com/NVIDIA/spark-rapids/pull/8656)|Return result from filecache message instead of null| +|[#8659](https://github.com/NVIDIA/spark-rapids/pull/8659)|Filter out nulls for build batches when needed in hash joins| +|[#8682](https://github.com/NVIDIA/spark-rapids/pull/8682)|[DOC] Update CUDA requirements in documentation and Dockerfiles[skip ci]| +|[#8637](https://github.com/NVIDIA/spark-rapids/pull/8637)|Support Float order-by columns for RANGE window functions| +|[#8681](https://github.com/NVIDIA/spark-rapids/pull/8681)|changed container name to adapt to blossom-lib refactor [skip ci]| +|[#8573](https://github.com/NVIDIA/spark-rapids/pull/8573)|Add support for Delta Lake 2.4.0| +|[#8671](https://github.com/NVIDIA/spark-rapids/pull/8671)|Fix use-after-freed bug in `GpuFloatArrayMin`| +|[#8650](https://github.com/NVIDIA/spark-rapids/pull/8650)|Support TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS| +|[#8495](https://github.com/NVIDIA/spark-rapids/pull/8495)|Speed up PCBS CPU read path by not recalculating as much| +|[#8389](https://github.com/NVIDIA/spark-rapids/pull/8389)|Add filecache support for ORC| +|[#8658](https://github.com/NVIDIA/spark-rapids/pull/8658)|Check if need to run Databricks pre-merge| +|[#8649](https://github.com/NVIDIA/spark-rapids/pull/8649)|Add Spark 3.4.1 shim| +|[#8624](https://github.com/NVIDIA/spark-rapids/pull/8624)|Rename numBytesAdded/Removed metrics and add deletion vector metrics in Databricks 12.2 shims| +|[#8645](https://github.com/NVIDIA/spark-rapids/pull/8645)|Fix "PytestUnknownMarkWarning: Unknown pytest.mark.inject_oom" warning| +|[#8608](https://github.com/NVIDIA/spark-rapids/pull/8608)|Matrix stages to dynamically build Databricks shims| +|[#8517](https://github.com/NVIDIA/spark-rapids/pull/8517)|Revert "Disable asserts for non-empty nulls (#8183)"| +|[#8628](https://github.com/NVIDIA/spark-rapids/pull/8628)|Enable Delta Write fallback tests on Databricks 12.2| +|[#8632](https://github.com/NVIDIA/spark-rapids/pull/8632)|Fix GCP examples and getting started guide [skip ci]| +|[#8638](https://github.com/NVIDIA/spark-rapids/pull/8638)|Support nested structs for `min`/`max` aggregations in groupby and reduction| +|[#8639](https://github.com/NVIDIA/spark-rapids/pull/8639)|Add iceberg test for nightly DB12.2 IT pipeline[skip ci]| +|[#8618](https://github.com/NVIDIA/spark-rapids/pull/8618)|Heuristic to speed up partial aggregates that get larger| +|[#8605](https://github.com/NVIDIA/spark-rapids/pull/8605)|[Doc] Fix demo link in index.md [skip ci]| +|[#8619](https://github.com/NVIDIA/spark-rapids/pull/8619)|Enable output batches metric for GpuShuffleCoalesceExec by default| +|[#8617](https://github.com/NVIDIA/spark-rapids/pull/8617)|Fixes broadcast spill serialization/deserialization| +|[#8531](https://github.com/NVIDIA/spark-rapids/pull/8531)|filecache: Modify FileCacheLocalityManager.init to pass in Spark context| +|[#8613](https://github.com/NVIDIA/spark-rapids/pull/8613)|Try print JVM core dump files if any test failures in CI| +|[#8616](https://github.com/NVIDIA/spark-rapids/pull/8616)|Wait for futures in multi-threaded writers even on exception| +|[#8578](https://github.com/NVIDIA/spark-rapids/pull/8578)|Add in metric to see how much computation time is lost due to retry| +|[#8590](https://github.com/NVIDIA/spark-rapids/pull/8590)|Drop ".dev0" suffix from Spark SNASHOT distro builds| +|[#8604](https://github.com/NVIDIA/spark-rapids/pull/8604)|Upgrade scalatest version to 3.2.16| +|[#8555](https://github.com/NVIDIA/spark-rapids/pull/8555)|Support `flatten` SQL function| +|[#8599](https://github.com/NVIDIA/spark-rapids/pull/8599)|Fix broken links in advanced_configs.md| +|[#8589](https://github.com/NVIDIA/spark-rapids/pull/8589)|Revert to the JVM-based Spark version extraction in pytests| +|[#8582](https://github.com/NVIDIA/spark-rapids/pull/8582)|Fix databricks shims build errors caused by DB updates| +|[#8564](https://github.com/NVIDIA/spark-rapids/pull/8564)|Fold `verify-all-modules-with-headSparkVersion` into `verify-all-modules` [skip ci]| +|[#8553](https://github.com/NVIDIA/spark-rapids/pull/8553)|Handle empty batch in ParquetCachedBatchSerializer| +|[#8575](https://github.com/NVIDIA/spark-rapids/pull/8575)|Corrected typos in CONTRIBUTING.md [skip ci]| +|[#8574](https://github.com/NVIDIA/spark-rapids/pull/8574)|Remove maxTaskFailures=4 for pre-3.1.1 Spark| +|[#8503](https://github.com/NVIDIA/spark-rapids/pull/8503)|Remove hard-coded version numbers for dependencies when building on| +|[#8544](https://github.com/NVIDIA/spark-rapids/pull/8544)|Fix auto merge conflict 8543 [skip ci]| +|[#8521](https://github.com/NVIDIA/spark-rapids/pull/8521)|List supported Spark versions when no shim found| +|[#8520](https://github.com/NVIDIA/spark-rapids/pull/8520)|Add support for first, last, nth, and collect_list aggregations for BinaryType| +|[#8509](https://github.com/NVIDIA/spark-rapids/pull/8509)|Remove legacy spark version check| +|[#8494](https://github.com/NVIDIA/spark-rapids/pull/8494)|Fix 23.08 build on Databricks 12.2| +|[#8487](https://github.com/NVIDIA/spark-rapids/pull/8487)|Move MockTaskContext to tests project| +|[#8426](https://github.com/NVIDIA/spark-rapids/pull/8426)|Pre-merge CI to support Databricks 12.2| +|[#8282](https://github.com/NVIDIA/spark-rapids/pull/8282)|Databricks 12.2 Support| +|[#8407](https://github.com/NVIDIA/spark-rapids/pull/8407)|Bump up dep version to 23.08.0-SNAPSHOT| +|[#8359](https://github.com/NVIDIA/spark-rapids/pull/8359)|Init version 23.08.0-SNAPSHOT| + ## Release 23.06 ### Features From 61e8e6c4cd1a44519fbe29b23c2ea82110bf6c4a Mon Sep 17 00:00:00 2001 From: Jim Brennan Date: Tue, 12 Dec 2023 09:11:22 -0600 Subject: [PATCH 13/19] Add cpu oom retry split handling to InternalRowToColumnarBatchIterator (#10011) Signed-off-by: Jim Brennan --- .../InternalRowToColumnarBatchIterator.java | 216 ++++++++++++------ .../rapids/AbstractGpuJoinIterator.scala | 4 +- .../spark/rapids/GpuRowToColumnarExec.scala | 5 +- .../com/nvidia/spark/rapids/HostAlloc.scala | 2 +- .../spark/rapids/RmmRapidsRetryIterator.scala | 81 +++++-- .../nvidia/spark/rapids/WithRetrySuite.scala | 10 +- 6 files changed, 208 insertions(+), 110 deletions(-) diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/InternalRowToColumnarBatchIterator.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/InternalRowToColumnarBatchIterator.java index e52b0b64634..a1f878cb078 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/InternalRowToColumnarBatchIterator.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/InternalRowToColumnarBatchIterator.java @@ -21,6 +21,7 @@ import java.util.NoSuchElementException; import java.util.Optional; +import com.nvidia.spark.Retryable; import scala.Option; import scala.Tuple2; import scala.collection.Iterator; @@ -54,8 +55,8 @@ public abstract class InternalRowToColumnarBatchIterator implements Iterator { protected final Iterator input; protected UnsafeRow pending = null; - protected final int numRowsEstimate; - protected final long dataLength; + protected int numRowsEstimate = 1; + protected final int sizePerRowEstimate; protected final DType[] rapidsTypes; protected final DataType[] outputTypes; protected final GpuMetric streamTime; @@ -74,10 +75,8 @@ protected InternalRowToColumnarBatchIterator( GpuMetric numOutputRows, GpuMetric numOutputBatches) { this.input = input; - int sizePerRowEstimate = CudfUnsafeRow.getRowSizeEstimate(schema); - numRowsEstimate = (int)Math.max(1, - Math.min(Integer.MAX_VALUE - 1, goal.targetSizeBytes() / sizePerRowEstimate)); - dataLength = ((long) sizePerRowEstimate) * numRowsEstimate; + sizePerRowEstimate = CudfUnsafeRow.getRowSizeEstimate(schema); + numRowsEstimate = calcNumRowsEstimate(goal.targetSizeBytes()); rapidsTypes = new DType[schema.length]; outputTypes = new DataType[schema.length]; @@ -92,6 +91,20 @@ protected InternalRowToColumnarBatchIterator( this.numOutputBatches = numOutputBatches; } + private int calcNumRowsEstimate(long targetBytes) { + return Math.max(1, + Math.min(Integer.MAX_VALUE - 1, (int) (targetBytes / sizePerRowEstimate))); + } + + private long calcDataLengthEstimate(int numRows) { + return ((long) sizePerRowEstimate) * numRows; + } + + private long calcOffsetLengthEstimate(int numRows) { + int BYTES_PER_OFFSET = DType.INT32.getSizeInBytes(); + return (long)(numRows + 1) * BYTES_PER_OFFSET; + } + @Override public boolean hasNext() { boolean ret = true; @@ -109,11 +122,13 @@ public ColumnarBatch next() { if (!hasNext()) { throw new NoSuchElementException(); } - final int BYTES_PER_OFFSET = DType.INT32.getSizeInBytes(); - long collectStart = System.nanoTime(); + long collectStart = System.nanoTime(); Tuple2 batchAndRange; + AutoCloseableTargetSize numRowsWrapper = + new AutoCloseableTargetSize(numRowsEstimate, 1); + Tuple2 bufsAndNumRows; // The row formatted data is stored as a column of lists of bytes. The current java CUDF APIs // don't do a great job from a performance standpoint with building this type of data structure @@ -121,75 +136,66 @@ public ColumnarBatch next() { // buffers. One will be for the byte data and the second will be for the offsets. We will then // write the data directly into those buffers using code generation in a child of this class. // that implements fillBatch. - HostMemoryBuffer db = - RmmRapidsRetryIterator.withRetryNoSplit( () -> { - return HostAlloc$.MODULE$.alloc(dataLength, true); - }); + bufsAndNumRows = + // Starting with initial num rows estimate, this retry block will recalculate the buffer + // sizes from the rows estimate, which is split in half if we get a split and retry oom, + // until we succeed or hit the min of 1 row. + RmmRapidsRetryIterator.withRetry(numRowsWrapper, + RmmRapidsRetryIterator.splitTargetSizeInHalfCpu(), (numRows) -> { + return allocBuffersWithRestore(numRows); + }).next(); + // Update our estimate for number of rows with the final size used to allocate the buffers. + numRowsEstimate = (int) bufsAndNumRows._2.targetSize(); + long dataLength = calcDataLengthEstimate(numRowsEstimate); try ( - SpillableHostBuffer sdb = SpillableHostBuffer$.MODULE$.apply(db, db.getLength(), - SpillPriorities$.MODULE$.ACTIVE_ON_DECK_PRIORITY(), - RapidsBufferCatalog$.MODULE$.singleton()); + SpillableHostBuffer sdb = bufsAndNumRows._1[0]; + SpillableHostBuffer sob = bufsAndNumRows._1[1]; ) { - HostMemoryBuffer ob = - RmmRapidsRetryIterator.withRetryNoSplit( () -> { - return HostAlloc$.MODULE$.alloc( - ((long) numRowsEstimate + 1) * BYTES_PER_OFFSET, true); - }); - try ( - SpillableHostBuffer sob = SpillableHostBuffer$.MODULE$.apply(ob, ob.getLength(), - SpillPriorities$.MODULE$.ACTIVE_ON_DECK_PRIORITY(), - RapidsBufferCatalog$.MODULE$.singleton()); - ) { - // Fill in buffer under write lock for host buffers - int[] used = sdb.withHostBufferWriteLock( (dataBuffer) -> { - return sob.withHostBufferWriteLock( (offsetsBuffer) -> { - return fillBatch(dataBuffer, offsetsBuffer); - }); - }); - batchAndRange = sdb.withHostBufferReadOnly( (dataBuffer) -> { - return sob.withHostBufferReadOnly( (offsetsBuffer) -> { - int dataOffset = used[0]; - int currentRow = used[1]; - // We don't want to loop forever trying to copy nothing - assert (currentRow > 0); - if (numInputRows != null) { - numInputRows.add(currentRow); - } - if (numOutputRows != null) { - numOutputRows.add(currentRow); - } - if (numOutputBatches != null) { - numOutputBatches.add(1); - } - // Now that we have filled the buffers with the data, we need to turn them into a - // HostColumnVector and copy them to the device so the GPU can turn it into a Table. - // To do this we first need to make a HostColumnCoreVector for the data, and then - // put that into a HostColumnVector as its child. This the basics of building up - // a column of lists of bytes in CUDF but it is typically hidden behind the higer level - // APIs. - dataBuffer.incRefCount(); - offsetsBuffer.incRefCount(); - try (HostColumnVectorCore dataCv = - new HostColumnVectorCore(DType.INT8, dataOffset, Optional.of(0L), - dataBuffer, null, null, new ArrayList<>()); - HostColumnVector hostColumn = new HostColumnVector(DType.LIST, - currentRow, Optional.of(0L), null, null, - offsetsBuffer, Collections.singletonList(dataCv))) { - - long ct = System.nanoTime() - collectStart; - streamTime.add(ct); - - // Grab the semaphore because we are about to put data onto the GPU. - GpuSemaphore$.MODULE$.acquireIfNecessary(TaskContext.get()); - NvtxRange range = NvtxWithMetrics.apply("RowToColumnar: build", NvtxColor.GREEN, - Option.apply(opTime)); - ColumnVector devColumn = - RmmRapidsRetryIterator.withRetryNoSplit(hostColumn::copyToDevice); - return Tuple2.apply(makeSpillableBatch(devColumn), range); - } - }); + // Fill in buffer under write lock for host buffers + batchAndRange = sdb.withHostBufferWriteLock( (dataBuffer) -> { + return sob.withHostBufferWriteLock( (offsetsBuffer) -> { + int[] used = fillBatch(dataBuffer, offsetsBuffer, dataLength, numRowsEstimate); + int dataOffset = used[0]; + int currentRow = used[1]; + // We don't want to loop forever trying to copy nothing + assert (currentRow > 0); + if (numInputRows != null) { + numInputRows.add(currentRow); + } + if (numOutputRows != null) { + numOutputRows.add(currentRow); + } + if (numOutputBatches != null) { + numOutputBatches.add(1); + } + // Now that we have filled the buffers with the data, we need to turn them into a + // HostColumnVector and copy them to the device so the GPU can turn it into a Table. + // To do this we first need to make a HostColumnCoreVector for the data, and then + // put that into a HostColumnVector as its child. This the basics of building up + // a column of lists of bytes in CUDF but it is typically hidden behind the higer level + // APIs. + dataBuffer.incRefCount(); + offsetsBuffer.incRefCount(); + try (HostColumnVectorCore dataCv = + new HostColumnVectorCore(DType.INT8, dataOffset, Optional.of(0L), + dataBuffer, null, null, new ArrayList<>()); + HostColumnVector hostColumn = new HostColumnVector(DType.LIST, + currentRow, Optional.of(0L), null, null, + offsetsBuffer, Collections.singletonList(dataCv))) { + + long ct = System.nanoTime() - collectStart; + streamTime.add(ct); + + // Grab the semaphore because we are about to put data onto the GPU. + GpuSemaphore$.MODULE$.acquireIfNecessary(TaskContext.get()); + NvtxRange range = NvtxWithMetrics.apply("RowToColumnar: build", NvtxColor.GREEN, + Option.apply(opTime)); + ColumnVector devColumn = + RmmRapidsRetryIterator.withRetryNoSplit(hostColumn::copyToDevice); + return Tuple2.apply(makeSpillableBatch(devColumn), range); + } }); - } + }); } try (NvtxRange ignored = batchAndRange._2; Table tab = @@ -202,6 +208,63 @@ public ColumnarBatch next() { } } + private Tuple2 + allocBuffers(SpillableHostBuffer[] sBufs, AutoCloseableTargetSize numRowsWrapper) { + HostMemoryBuffer[] hBufs = new HostMemoryBuffer[]{ null, null }; + try { + long dataBytes = calcDataLengthEstimate((int) numRowsWrapper.targetSize()); + long offsetBytes = calcOffsetLengthEstimate((int) numRowsWrapper.targetSize()); + hBufs[0] = HostAlloc$.MODULE$.alloc(dataBytes, true); + sBufs[0] = SpillableHostBuffer$.MODULE$.apply(hBufs[0], hBufs[0].getLength(), + SpillPriorities$.MODULE$.ACTIVE_ON_DECK_PRIORITY(), + RapidsBufferCatalog$.MODULE$.singleton()); + hBufs[0] = null; // Was closed by spillable + hBufs[1] = HostAlloc$.MODULE$.alloc(offsetBytes, true); + sBufs[1] = SpillableHostBuffer$.MODULE$.apply(hBufs[1], hBufs[1].getLength(), + SpillPriorities$.MODULE$.ACTIVE_ON_DECK_PRIORITY(), + RapidsBufferCatalog$.MODULE$.singleton()); + hBufs[1] = null; // Was closed by spillable + return Tuple2.apply(sBufs, numRowsWrapper); + } finally { + // Make sure host buffers are always closed + for (int i = 0; i < hBufs.length; i++) { + if (hBufs[i] != null) { + hBufs[i].close(); + hBufs[i] = null; + } + } + // If the second spillable buffer is null, we must have thrown, + // so we need to close the first one in case this is not a retry exception. + // Restore on retry is handled by the caller. + if ((sBufs[1] == null) && (sBufs[0] != null)) { + sBufs[0].close(); + sBufs[0] = null; + } + } + } + + private Tuple2 + allocBuffersWithRestore(AutoCloseableTargetSize numRows) { + SpillableHostBuffer[] spillableBufs = new SpillableHostBuffer[]{ null, null}; + Retryable retryBufs = new Retryable() { + @Override + public void checkpoint() {} + @Override + public void restore() { + for (int i = 0; i < spillableBufs.length; i++) { + if (spillableBufs[i] != null) { + spillableBufs[i].close(); + spillableBufs[i] = null; + } + } + } + }; + + return RmmRapidsRetryIterator.withRestoreOnRetry(retryBufs, () -> { + return allocBuffers(spillableBufs, numRows); + }); + } + /** * Take our device column of encoded rows and turn it into a spillable columnar batch. * This allows us to go into a retry block and be able to roll back our work. @@ -244,8 +307,11 @@ protected Table convertFromRowsUnderRetry(ColumnarBatch cb) { * virtual function call per batch instead of one per row. * @param dataBuffer the data buffer to populate * @param offsetsBuffer the offsets buffer to populate + * @param dataLength the data length corresponding to the current rows estimate. + * @param numRows the number of rows we can fill * @return an array of ints where the first index is the amount of data in bytes copied into * the data buffer and the second index is the number of rows copied into the buffers. */ - public abstract int[] fillBatch(HostMemoryBuffer dataBuffer, HostMemoryBuffer offsetsBuffer); + public abstract int[] fillBatch(HostMemoryBuffer dataBuffer, HostMemoryBuffer offsetsBuffer, + long dataLength, int numRows); } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AbstractGpuJoinIterator.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AbstractGpuJoinIterator.scala index cee705d8f8e..37d636bf82e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AbstractGpuJoinIterator.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AbstractGpuJoinIterator.scala @@ -19,7 +19,7 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.{GatherMap, NvtxColor, OutOfBoundsPolicy} import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.RapidsPluginImplicits._ -import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitTargetSizeInHalf, withRestoreOnRetry, withRetry} +import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitTargetSizeInHalfGpu, withRestoreOnRetry, withRetry} import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion import org.apache.spark.TaskContext @@ -148,7 +148,7 @@ abstract class AbstractGpuJoinIterator( // less from the gatherer, but because the gatherer tracks how much is used, the // next call to this function will start in the right place. gather.checkpoint() - withRetry(targetSizeWrapper, splitTargetSizeInHalf) { attempt => + withRetry(targetSizeWrapper, splitTargetSizeInHalfGpu) { attempt => withRestoreOnRetry(gather) { val nextRows = JoinGatherer.getRowsInNextBatch(gather, attempt.targetSize) gather.gatherNext(nextRows) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRowToColumnarExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRowToColumnarExec.scala index 8586e4c3dbe..f399bdeb20f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRowToColumnarExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRowToColumnarExec.scala @@ -787,7 +787,8 @@ object GeneratedInternalRowToCudfRowIterator extends Logging { | // of a row at a time. | @Override | public int[] fillBatch(ai.rapids.cudf.HostMemoryBuffer dataBuffer, - | ai.rapids.cudf.HostMemoryBuffer offsetsBuffer) { + | ai.rapids.cudf.HostMemoryBuffer offsetsBuffer, + | long dataLength, int numRows) { | final long dataBaseAddress = dataBuffer.getAddress(); | final long endDataAddress = dataBaseAddress + dataLength; | @@ -820,7 +821,7 @@ object GeneratedInternalRowToCudfRowIterator extends Logging { | } else { | currentRow += 1; | dataOffset += numBytesUsedByRow; - | done = !(currentRow < numRowsEstimate && + | done = !(currentRow < numRows && | dataOffset < dataLength && | input.hasNext()); | } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostAlloc.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostAlloc.scala index 7da626e21b0..cefdfa8766e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostAlloc.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostAlloc.scala @@ -144,7 +144,7 @@ private class HostAlloc(nonPinnedLimit: Long) extends HostMemoryAllocator with L logDebug(s"Targeting host store size of $targetSize bytes") // We could not make it work so try and spill enough to make it work val maybeAmountSpilled = - RapidsBufferCatalog.synchronousSpill(RapidsBufferCatalog.getHostStorage, allocSize) + RapidsBufferCatalog.synchronousSpill(RapidsBufferCatalog.getHostStorage, targetSize) maybeAmountSpilled.foreach { amountSpilled => logInfo(s"Spilled $amountSpilled bytes from the host store") } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RmmRapidsRetryIterator.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RmmRapidsRetryIterator.scala index 78fe8c7b31d..4973e276bb8 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RmmRapidsRetryIterator.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RmmRapidsRetryIterator.scala @@ -51,10 +51,10 @@ object RmmRapidsRetryIterator extends Logging { * `fn` must be idempotent: this is a requirement because we may call `fn` multiple times * while handling retries. * - * @param input an iterator of T + * @param input an iterator of T * @param splitPolicy a function that can split an item of type T into a Seq[T]. The split * function must close the item passed to it. - * @param fn the work to perform. Takes T and produces an output K + * @param fn the work to perform. Takes T and produces an output K * @tparam T element type that must be AutoCloseable (likely `SpillableColumnarBatch`) * @tparam K `fn` result type * @return an iterator of K @@ -84,10 +84,10 @@ object RmmRapidsRetryIterator extends Logging { * `fn` must be idempotent: this is a requirement because we may call `fn` multiple times * while handling retries. * - * @param input a single item T + * @param input a single item T * @param splitPolicy a function that can split an item of type T into a Seq[T]. The split * function must close the item passed to it. - * @param fn the work to perform. Takes T and produces an output K + * @param fn the work to perform. Takes T and produces an output K * @tparam T element type that must be AutoCloseable (likely `SpillableColumnarBatch`) * @tparam K `fn` result type * @return an iterator of K @@ -117,8 +117,8 @@ object RmmRapidsRetryIterator extends Logging { * `fn` must be idempotent: this is a requirement because we may call `fn` multiple times * while handling retries. * - * @param input a single item T - * @param fn the work to perform. Takes T and produces an output K + * @param input a single item T + * @param fn the work to perform. Takes T and produces an output K * @tparam T element type that must be AutoCloseable (likely `SpillableColumnarBatch`) * @tparam K `fn` result type * @return a single item of type K @@ -148,8 +148,8 @@ object RmmRapidsRetryIterator extends Logging { * `fn` must be idempotent: this is a requirement because we may call `fn` multiple times * while handling retries. * - * @param input a single item T - * @param fn the work to perform. Takes T and produces an output K + * @param input a single item T + * @param fn the work to perform. Takes T and produces an output K * @tparam T element type that must be AutoCloseable (likely `SpillableColumnarBatch`) * @tparam K `fn` result type * @return a single item of type K @@ -296,11 +296,12 @@ object RmmRapidsRetryIterator extends Logging { /** * AutoCloseable wrapper on Seq[T], returning a Seq[T] that can be closed. + * * @param ts the Seq to wrap * @tparam T the type of the items in `ts` */ private case class AutoCloseableSeqInternal[T <: AutoCloseable](ts: Seq[T]) - extends Seq[T] with AutoCloseable{ + extends Seq[T] with AutoCloseable { override def close(): Unit = { ts.foreach(_.safeClose()) } @@ -315,18 +316,22 @@ object RmmRapidsRetryIterator extends Logging { /** * An iterator of a single item that is able to close if .next * has not been called on it. + * * @param ts the AutoCloseable item to close if this iterator hasn't been drained * @tparam T the type of `ts`, must be AutoCloseable */ private case class SingleItemAutoCloseableIteratorInternal[T <: AutoCloseable](ts: T) - extends Iterator[T] with AutoCloseable { + extends Iterator[T] with AutoCloseable { private var wasCalledSuccessfully = false + override def hasNext: Boolean = !wasCalledSuccessfully + override def next(): T = { wasCalledSuccessfully = true ts } + override def close(): Unit = { if (!wasCalledSuccessfully) { ts.close() @@ -353,6 +358,7 @@ object RmmRapidsRetryIterator extends Logging { * using as an input), usually by splitting a batch in half by number of rows, or * splitting a collection of batches into smaller collections to be attempted separately, * likely reducing GPU memory that needs to be manifested while calling `.next`. + * * @param isFromGpuOom true if the split happened because of a GPU OOM. Otherwise it was a * CPU off heap OOM. */ @@ -366,6 +372,7 @@ object RmmRapidsRetryIterator extends Logging { /** * A spliterator that doesn't take any inputs, hence it is "empty", and it doesn't know * how to split. It allows the caller to call the function `fn` once on `next`. + * * @param fn the work to perform. It is a function that takes nothing and produces K * @tparam K the resulting type */ @@ -413,8 +420,8 @@ object RmmRapidsRetryIterator extends Logging { * * @tparam T element type that must be AutoCloseable * @tparam K `fn` result type - * @param input an iterator of T - * @param fn a function that takes T and produces K + * @param input an iterator of T + * @param fn a function that takes T and produces K * @param splitPolicy a function that can split an item of type T into a Seq[T]. The split * function must close the item passed to it. */ @@ -626,7 +633,7 @@ object RmmRapidsRetryIterator extends Logging { throw lastException } } - // else another exception wrapped a retry. So we are going to try again + // else another exception wrapped a retry. So we are going to try again } } if (result.isEmpty) { @@ -682,22 +689,46 @@ object RmmRapidsRetryIterator extends Logging { } } + private def splitTargetSizeInHalfInternal( + target: AutoCloseableTargetSize, isGpu: Boolean): Seq[AutoCloseableTargetSize] = { + withResource(target) { _ => + val newTarget = target.targetSize / 2 + if (newTarget < target.minSize) { + if (isGpu) { + throw new GpuSplitAndRetryOOM( + s"GPU OutOfMemory: targetSize: ${target.targetSize} cannot be split further!" + + s" minimum: ${target.minSize}") + } else { + throw new CpuSplitAndRetryOOM( + s"CPU OutOfMemory: targetSize: ${target.targetSize} cannot be split further!" + + s" minimum: ${target.minSize}") + } + } + Seq(AutoCloseableTargetSize(newTarget, target.minSize)) + } + } + /** * A common split function for an AutoCloseableTargetSize, which just divides the target size * in half, and creates a seq with just one element representing the new target size. * @return a Seq[AutoCloseableTargetSize] with 1 element. + * @throws GpuSplitAndRetryOOM if it reaches the split limit. */ - def splitTargetSizeInHalf: AutoCloseableTargetSize => Seq[AutoCloseableTargetSize] = + def splitTargetSizeInHalfGpu: AutoCloseableTargetSize => Seq[AutoCloseableTargetSize] = (target: AutoCloseableTargetSize) => { - withResource(target) { _ => - val newTarget = target.targetSize / 2 - if (newTarget < target.minSize) { - throw new GpuSplitAndRetryOOM( - s"GPU OutOfMemory: targetSize: ${target.targetSize} cannot be split further!" + - s" minimum: ${target.minSize}") - } - Seq(AutoCloseableTargetSize(newTarget, target.minSize)) - } + splitTargetSizeInHalfInternal(target, true) + } + + /** + * A common split function for an AutoCloseableTargetSize, which just divides the target size + * in half, and creates a seq with just one element representing the new target size. + * + * @return a Seq[AutoCloseableTargetSize] with 1 element. + * @throws CpuSplitAndRetryOOM if it reaches the split limit. + */ + def splitTargetSizeInHalfCpu: AutoCloseableTargetSize => Seq[AutoCloseableTargetSize] = + (target: AutoCloseableTargetSize) => { + splitTargetSizeInHalfInternal(target, false) } } @@ -705,8 +736,8 @@ object RmmRapidsRetryIterator extends Logging { * This is a wrapper that turns a target size into an autocloseable to allow it to be used * in withRetry blocks. It is intended to be used to help with cases where the split calculation * happens inside the retry block, and depends on the target size. On a `GpuSplitAndRetryOOM` or - * `CpuSplitAndRetryOOM`, a split policy like `splitTargetSizeInHalf` can be used to retry the - * block with a smaller target size. + * `CpuSplitAndRetryOOM`, a split policy like `splitTargetSizeInHalfGpu` or + * `splitTargetSizeInHalfCpu` can be used to retry the block with a smaller target size. */ case class AutoCloseableTargetSize(targetSize: Long, minSize: Long) extends AutoCloseable { override def close(): Unit = () diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/WithRetrySuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/WithRetrySuite.scala index 73618bd81ef..aa003c454f1 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/WithRetrySuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/WithRetrySuite.scala @@ -19,7 +19,7 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.{Rmm, RmmAllocationMode, RmmEventHandler, Table} import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.withResource -import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitTargetSizeInHalf, withRestoreOnRetry, withRetry, withRetryNoSplit} +import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitTargetSizeInHalfGpu, withRestoreOnRetry, withRetry, withRetryNoSplit} import com.nvidia.spark.rapids.jni.{GpuRetryOOM, GpuSplitAndRetryOOM, RmmSpark} import org.mockito.Mockito._ import org.scalatest.BeforeAndAfterEach @@ -245,7 +245,7 @@ class WithRetrySuite } } - test("splitTargetSizeInHalf splits for AutoCloseableTargetSize") { + test("splitTargetSizeInHalfGpu splits for AutoCloseableTargetSize") { val initialValue = 20L val minValue = 5L val numSplits = 2 @@ -253,7 +253,7 @@ class WithRetrySuite var lastSplitSize = 0L val myTarget = AutoCloseableTargetSize(initialValue, minValue) try { - withRetry(myTarget, splitTargetSizeInHalf) { attempt => + withRetry(myTarget, splitTargetSizeInHalfGpu) { attempt => lastSplitSize = attempt.targetSize if (doThrow > 0) { doThrow = doThrow - 1 @@ -266,7 +266,7 @@ class WithRetrySuite } } - test("splitTargetSizeInHalf on AutoCloseableTargetSize throws if limit reached") { + test("splitTargetSizeInHalfGpu on AutoCloseableTargetSize throws if limit reached") { val initialValue = 20L val minValue = 5L val numSplits = 3 @@ -275,7 +275,7 @@ class WithRetrySuite val myTarget = AutoCloseableTargetSize(initialValue, minValue) try { assertThrows[GpuSplitAndRetryOOM] { - withRetry(myTarget, splitTargetSizeInHalf) { attempt => + withRetry(myTarget, splitTargetSizeInHalfGpu) { attempt => lastSplitSize = attempt.targetSize if (doThrow > 0) { doThrow = doThrow - 1 From aa61be45ed60f4774e534354e514d2020c4e9479 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Tue, 12 Dec 2023 12:17:11 -0600 Subject: [PATCH 14/19] Fix array_transform to not recompute the argument (#10015) * Fix array_transform to not recompute the argument Signed-off-by: Robert (Bobby) Evans * Review comments and some code/test improvements --------- Signed-off-by: Robert (Bobby) Evans --- .../src/main/python/array_test.py | 10 +++++ .../nvidia/spark/rapids/GpuColumnVector.java | 16 ++++++++ .../spark/rapids/higherOrderFunctions.scala | 40 ++++++++++--------- 3 files changed, 48 insertions(+), 18 deletions(-) diff --git a/integration_tests/src/main/python/array_test.py b/integration_tests/src/main/python/array_test.py index ec29dce70d1..c9749865438 100644 --- a/integration_tests/src/main/python/array_test.py +++ b/integration_tests/src/main/python/array_test.py @@ -332,6 +332,16 @@ def do_it(spark): assert_gpu_and_cpu_are_equal_collect(do_it) +def test_array_transform_non_deterministic(): + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.range(1).selectExpr("transform(sequence(0, cast(rand(5)*10 as int) + 1), x -> x * 22) as t"), + conf={'spark.rapids.sql.castFloatToIntegralTypes.enabled': True}) + +def test_array_transform_non_deterministic_second_param(): + assert_gpu_and_cpu_are_equal_collect( + lambda spark : debug_df(spark.range(1).selectExpr("transform(sequence(0, cast(rand(5)*10 as int) + 1), (x, i) -> x + i) as t")), + conf={'spark.rapids.sql.castFloatToIntegralTypes.enabled': True}) + # TODO add back in string_gen when https://github.com/rapidsai/cudf/issues/9156 is fixed array_min_max_gens = [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, date_gen, timestamp_gen, null_gen] + decimal_gens 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 97402f5a58e..13cf88efea0 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 @@ -895,6 +895,22 @@ public static ColumnarBatch combineColumns(ColumnarBatch ... batches) { return incRefCounts(ret); } + public static ColumnarBatch appendColumns(ColumnarBatch cb, GpuColumnVector ... vectors) { + final int numRows = cb.numRows(); + final int numCbColumns = cb.numCols(); + ArrayList columns = new ArrayList<>(numCbColumns + vectors.length); + for (int i = 0; i < numCbColumns; i++) { + columns.add(cb.column(i)); + } + for (GpuColumnVector cv: vectors) { + assert cv.getBase().getRowCount() == numRows : "Rows do not match expected " + numRows + " found " + + cv.getBase().getRowCount(); + columns.add(cv); + } + ColumnarBatch ret = new ColumnarBatch(columns.toArray(new ColumnVector[columns.size()]), numRows); + return incRefCounts(ret); + } + /** * Remove columns from the batch. The order of the remaining columns is preserved. * dropList[] has an entry for each column in the batch which indicates whether the column should diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/higherOrderFunctions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/higherOrderFunctions.scala index dfb4272f23f..2093b52256e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/higherOrderFunctions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/higherOrderFunctions.scala @@ -19,7 +19,7 @@ package com.nvidia.spark.rapids import scala.collection.mutable import ai.rapids.cudf -import ai.rapids.cudf.DType +import ai.rapids.cudf.{DType, Table} import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.RapidsPluginImplicits.ReallyAGpuExpression import com.nvidia.spark.rapids.shims.ShimExpression @@ -214,7 +214,7 @@ trait GpuSimpleHigherOrderFunction extends GpuHigherOrderFunction with GpuBind { trait GpuArrayTransformBase extends GpuSimpleHigherOrderFunction { def isBound: Boolean - def boundIntermediate: Seq[GpuExpression] + def boundIntermediate: Seq[GpuExpression] protected lazy val inputToLambda: Seq[DataType] = { assert(isBound) @@ -223,18 +223,26 @@ trait GpuArrayTransformBase extends GpuSimpleHigherOrderFunction { private[this] def makeElementProjectBatch( inputBatch: ColumnarBatch, - listColumn: cudf.ColumnVector): ColumnarBatch = { - assert(listColumn.getType.equals(DType.LIST)) + argColumn: GpuColumnVector): ColumnarBatch = { + assert(argColumn.getBase.getType.equals(DType.LIST)) assert(isBound, "Trying to execute an un-bound transform expression") + def projectAndExplode(explodeOp: Table => Table): Table = { + withResource(GpuProjectExec.project(inputBatch, boundIntermediate)) { + intermediateBatch => + withResource(GpuColumnVector.appendColumns(intermediateBatch, argColumn)) { + projectedBatch => + withResource(GpuColumnVector.from(projectedBatch)) { projectedTable => + explodeOp(projectedTable) + } + } + } + } + if (function.asInstanceOf[GpuLambdaFunction].arguments.length >= 2) { // Need to do an explodePosition - val boundProject = boundIntermediate :+ argument - val explodedTable = withResource(GpuProjectExec.project(inputBatch, boundProject)) { - projectedBatch => - withResource(GpuColumnVector.from(projectedBatch)) { projectedTable => - projectedTable.explodePosition(boundIntermediate.length) - } + val explodedTable = projectAndExplode { projectedTable => + projectedTable.explodePosition(boundIntermediate.length) } val reorderedTable = withResource(explodedTable) { explodedTable => // The column order is wrong after an explodePosition. It is @@ -257,12 +265,8 @@ trait GpuArrayTransformBase extends GpuSimpleHigherOrderFunction { } } else { // Need to do an explode - val boundProject = boundIntermediate :+ argument - val explodedTable = withResource(GpuProjectExec.project(inputBatch, boundProject)) { - projectedBatch => - withResource(GpuColumnVector.from(projectedBatch)) { projectedTable => - projectedTable.explode(boundIntermediate.length) - } + val explodedTable = projectAndExplode { projectedTable => + projectedTable.explode(boundIntermediate.length) } withResource(explodedTable) { explodedTable => GpuColumnVector.from(explodedTable, inputToLambda.toArray) @@ -278,12 +282,12 @@ trait GpuArrayTransformBase extends GpuSimpleHigherOrderFunction { override def columnarEval(batch: ColumnarBatch): GpuColumnVector = { withResource(argument.columnarEval(batch)) { arg => - val dataCol = withResource(makeElementProjectBatch(batch, arg.getBase)) { cb => + val dataCol = withResource(makeElementProjectBatch(batch, arg)) { cb => function.columnarEval(cb) } withResource(dataCol) { _ => val cv = GpuListUtils.replaceListDataColumnAsView(arg.getBase, dataCol.getBase) - withResource(cv)(transformListColumnView(_)) + withResource(cv)(transformListColumnView) } } } From 5e789e8e9d03e505d8a7a7f87d3297344dd8a509 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Tue, 12 Dec 2023 14:41:41 -0600 Subject: [PATCH 15/19] Update links in shims.md (#10030) Signed-off-by: Robert (Bobby) Evans --- docs/dev/shims.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/dev/shims.md b/docs/dev/shims.md index e214d07862d..a1a5e5ab939 100644 --- a/docs/dev/shims.md +++ b/docs/dev/shims.md @@ -17,7 +17,7 @@ common code, maximize reuse, and minimize logic duplication. This is achieved by using a ServiceProvider pattern. All Shims implement the same API, the suitable Shim implementation is loaded after detecting the current Spark build version attempting to instantiate our plugin. We use the -[ShimLoader](https://github.com/NVIDIA/spark-rapids/blob/main/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala) +[ShimLoader](https://github.com/NVIDIA/spark-rapids/blob/main/sql-plugin-api/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala) class as a tight entry point for interacting with the host Spark runtime. In the following we provide recipes for typical scenarios addressed by the Shim layer. @@ -111,7 +111,7 @@ has not been loaded yet. More accurately, it may not be strictly needed until la query can be run when the Spark SQL session and its extensions are initialized. It improves the user experience if the first query is not penalized beyond necessary though. By design, Plugin guarantees that the classloader is -[set up at load time](https://github.com/NVIDIA/spark-rapids/blob/main/sql-plugin/src/main/scala/com/nvidia/spark/SQLPlugin.scala#L29) +[set up at load time](https://github.com/NVIDIA/spark-rapids/blob/main/sql-plugin-api/src/main/scala/com/nvidia/spark/SQLPlugin.scala#L29) before the DriverPlugin and ExecutorPlugin instances are called the `init` method on. By making a visible class merely a wrapper of the real implementation where the real implementation From c6dee9b89be7d38b56815588f3dfac76cbea10eb Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Wed, 13 Dec 2023 04:47:26 +0800 Subject: [PATCH 16/19] Use parse_url kernel for PROTOCOL parsing (#9481) Signed-off-by: Haoyang Li --- .../advanced_configs.md | 1 + docs/supported_ops.md | 1129 +++++++++-------- integration_tests/src/main/python/url_test.py | 167 +++ .../nvidia/spark/rapids/GpuOverrides.scala | 28 + .../apache/spark/sql/rapids/GpuParseUrl.scala | 106 ++ tools/generated_files/operatorsScore.csv | 1 + tools/generated_files/supportedExprs.csv | 4 + 7 files changed, 916 insertions(+), 520 deletions(-) create mode 100644 integration_tests/src/main/python/url_test.py create mode 100644 sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuParseUrl.scala diff --git a/docs/additional-functionality/advanced_configs.md b/docs/additional-functionality/advanced_configs.md index 089a9cb9b27..40acd48329c 100644 --- a/docs/additional-functionality/advanced_configs.md +++ b/docs/additional-functionality/advanced_configs.md @@ -301,6 +301,7 @@ Name | SQL Function(s) | Description | Default Value | Notes spark.rapids.sql.expression.NthValue|`nth_value`|nth window operator|true|None| spark.rapids.sql.expression.OctetLength|`octet_length`|The byte length of string data|true|None| spark.rapids.sql.expression.Or|`or`|Logical OR|true|None| +spark.rapids.sql.expression.ParseUrl|`parse_url`|Extracts a part from a URL|true|None| spark.rapids.sql.expression.PercentRank|`percent_rank`|Window function that returns the percent rank value within the aggregation window|true|None| spark.rapids.sql.expression.Pmod|`pmod`|Pmod|true|None| spark.rapids.sql.expression.PosExplode|`posexplode_outer`, `posexplode`|Given an input array produces a sequence of rows for each value in the array|true|None| diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 414a53c56ac..fab825a9c0f 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -10734,6 +10734,95 @@ are limited. +ParseUrl +`parse_url` +Extracts a part from a URL +None +project +url + + + + + + + + + +S + + + + + + + + + + +partToExtract + + + + + + + + + +PS
only support partToExtract=PROTOCOL;
Literal value only
+ + + + + + + + + + +key + + + + + + + + + +PS
Literal value only
+ + + + + + + + + + +result + + + + + + + + + +S + + + + + + + + + + PercentRank `percent_rank` Window function that returns the percent rank value within the aggregation window @@ -10849,6 +10938,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + PosExplode `posexplode_outer`, `posexplode` Given an input array produces a sequence of rows for each value in the array @@ -11028,32 +11143,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - PreciseTimestampConversion Expression used internally to convert the TimestampType to Long and back without losing precision, i.e. in microseconds. Used in time windowing @@ -11324,6 +11413,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + Quarter `quarter` Returns the quarter of the year for date, in the range 1 to 4 @@ -11439,32 +11554,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - RaiseError `raise_error` Throw an exception @@ -11695,6 +11784,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + RegExpExtractAll `regexp_extract_all` Extract all strings matching a regular expression corresponding to the regex group index @@ -11894,32 +12009,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - Remainder `%`, `mod` Remainder or modulo @@ -12082,6 +12171,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + Rint `rint` Rounds up a double value to the nearest double equal to an integer @@ -12266,32 +12381,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - ScalaUDF User Defined Function, the UDF can choose to implement a RAPIDS accelerated interface to get better performance. @@ -12522,6 +12611,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + ShiftLeft `shiftleft` Bitwise shift left (<<) @@ -12658,32 +12773,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - ShiftRightUnsigned `shiftrightunsigned` Bitwise unsigned shift right (>>>) @@ -12889,6 +12978,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + Sinh `sinh` Hyperbolic sine @@ -13026,32 +13141,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - SortArray `sort_array` Returns a sorted array with the input array and the ascending / descending order @@ -13261,6 +13350,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + Sqrt `sqrt` Square root @@ -13419,32 +13534,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - StartsWith Starts with @@ -13670,6 +13759,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + StringLocate `position`, `locate` Substring search operator @@ -13848,32 +13963,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - StringRepeat `repeat` StringRepeat operator that repeats the given strings with numbers of times given by repeatTimes @@ -14031,6 +14120,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + StringSplit `split` Splits `str` around occurrences that match `regex` @@ -14209,32 +14324,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - StringTranslate `translate` StringTranslate operator @@ -14392,6 +14481,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + StringTrimLeft `ltrim` StringTrimLeft operator @@ -14575,32 +14690,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - Substring `substr`, `substring` Substring operator @@ -14779,6 +14868,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + Subtract `-` Subtraction @@ -15001,32 +15116,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - Tanh `tanh` Hyperbolic tangent @@ -15185,6 +15274,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + ToDegrees `degrees` Converts radians to degrees @@ -15415,32 +15530,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - TransformValues `transform_values` Transform values in a map using a transform function @@ -15599,6 +15688,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + UnaryPositive `positive` A numeric value with a + in front of it @@ -15809,32 +15924,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - UnscaledValue Convert a Decimal to an unscaled long value for some aggregation optimizations @@ -15976,6 +16065,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + WindowExpression Calculates a return value for every input row of a table based on a group (or "window") of rows @@ -16206,32 +16321,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - AggregateExpression Aggregate expression @@ -16428,6 +16517,32 @@ are limited. S +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + ApproximatePercentile `percentile_approx`, `approx_percentile` Approximate percentile @@ -16602,32 +16717,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - Average `avg`, `mean` Average aggregate operator @@ -16894,6 +16983,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + CollectSet `collect_set` Collect a set of unique elements, not supported in reduction @@ -17027,32 +17142,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - Count `count` Count aggregate operator @@ -17319,6 +17408,32 @@ are limited. NS +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + Last `last`, `last_value` last aggregate operator @@ -17452,32 +17567,6 @@ are limited. NS -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - Max `max` Max aggregate operator @@ -17744,6 +17833,32 @@ are limited. NS +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + Percentile `percentile` Aggregation computing exact percentile @@ -17918,32 +18033,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - PivotFirst PivotFirst operator @@ -18209,6 +18298,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + StddevSamp `stddev_samp`, `std`, `stddev` Aggregation computing sample standard deviation @@ -18342,32 +18457,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - Sum `sum` Sum aggregate operator @@ -18634,6 +18723,32 @@ are limited. +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + VarianceSamp `var_samp`, `variance` Aggregation computing sample variance @@ -18767,32 +18882,6 @@ are limited. -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - NormalizeNaNAndZero Normalize NaN and zero diff --git a/integration_tests/src/main/python/url_test.py b/integration_tests/src/main/python/url_test.py new file mode 100644 index 00000000000..ba51170108d --- /dev/null +++ b/integration_tests/src/main/python/url_test.py @@ -0,0 +1,167 @@ +# Copyright (c) 2023, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import pytest + +from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error, assert_gpu_fallback_collect +from data_gen import * +from marks import * +from pyspark.sql.types import * +import pyspark.sql.functions as f +from spark_session import is_before_spark_340 + +# regex to generate limit length urls with HOST, PATH, QUERY, REF, PROTOCOL, FILE, AUTHORITY, USERINFO +url_pattern = r'((http|https|ftp)://)(([a-zA-Z][a-zA-Z0-9]{0,2}\.){0,3}([a-zA-Z][a-zA-Z0-9]{0,2})\.([a-zA-Z][a-zA-Z0-9]{0,2}))' \ + r'(:[0-9]{1,3}){0,1}(/[a-zA-Z0-9]{1,3}){0,3}(\?[a-zA-Z0-9]{1,3}=[a-zA-Z0-9]{1,3}){0,1}(#([a-zA-Z0-9]{1,3})){0,1}' + +url_pattern_with_key = r'((http|https|ftp|file)://)(([a-z]{1,3}\.){0,3}([a-z]{1,3})\.([a-z]{1,3}))' \ + r'(:[0-9]{1,3}){0,1}(/[a-z]{1,3}){0,3}(\?key=[a-z]{1,3}){0,1}(#([a-z]{1,3})){0,1}' + +edge_cases = [ + "http://foo.com/blah_blah", + "http://foo.com/blah_blah/", + "http://foo.com/blah_blah_(wikipedia)", + "http://foo.com/blah_blah_(wikipedia)_(again)", + "http://www.example.com/wpstyle/?p=364", + "https://www.example.com/foo/?bar=baz&inga=42&quux", + "http://✪df.ws/123", + "http://userid:password@example.com:8080", + "http://userid:password@example.com:8080/", + "http://userid:password@example.com", + "http://userid:password@example.com/", + "http://142.42.1.1/", + "http://142.42.1.1:8080/", + "http://➡.ws/䨹", + "http://⌘.ws", + "http://⌘.ws/", + "http://foo.com/blah_(wikipedia)#cite-1", + "http://foo.com/blah_(wikipedia)_blah#cite-1", + "http://foo.com/unicode_(✪)_in_parens", + "http://foo.com/(something)?after=parens", + "http://☺.damowmow.com/", + "http://code.google.com/events/#&product=browser", + "http://j.mp", + "ftp://foo.bar/baz", + r"http://foo.bar/?q=Test%20URL-encoded%20stuff", + "http://مثال.إختبار", + "http://例子.测试", + "http://उदाहरण.परीक्षा", + "http://-.~_!$&'()*+,;=:%40:80%2f::::::@example.com", + "http://1337.net", + "http://a.b-c.de", + "http://223.255.255.254", + "https://foo_bar.example.com/", + "http:# ", + "http://.", + "http://..", + "http://../", + "http://?", + "http://??", + "http://??/", + "http://#", + "http://##", + "http://##/", + "http://foo.bar?q=Spaces should be encoded", + "# ", + "//a", + "///a", + "/# ", + "http:///a", + "foo.com", + "rdar://1234", + "h://test", + "http:// shouldfail.com", + ":// should fail", + "http://foo.bar/foo(bar)baz quux", + "ftps://foo.bar/", + "http://-error-.invalid/", + "http://a.b--c.de/", + "http://-a.b.co", + "http://a.b-.co", + "http://0.0.0.0", + "http://10.1.1.0", + "http://10.1.1.255", + "http://224.1.1.1", + "http://1.1.1.1.1", + "http://123.123.123", + "http://3628126748", + "http://.www.foo.bar/", + "http://www.foo.bar./", + "http://.www.foo.bar./", + "http://10.1.1.1", + "http://10.1.1.254", + "http://userinfo@spark.apache.org/path?query=1#Ref", + r"https://use%20r:pas%20s@example.com/dir%20/pa%20th.HTML?query=x%20y&q2=2#Ref%20two", + "http://user:pass@host", + "http://user:pass@host/", + "http://user:pass@host/?#", + "http://user:pass@host/file;param?query;p2", + "inva lid://user:pass@host/file;param?query;p2", + "http://[1:2:3:4:5:6:7:8]", + "http://[1::]", + "http://[1:2:3:4:5:6:7::]", + "http://[1::8]", + "http://[1:2:3:4:5:6::8]", + "http://[1:2:3:4:5:6::8]", + "http://[1::7:8]", + "http://[1:2:3:4:5::7:8]", + "http://[1:2:3:4:5::8]", + "http://[1::6:7:8]", + "http://[1:2:3:4::6:7:8]", + "http://[1:2:3:4::8]", + "http://[1::5:6:7:8]", + "http://[1:2:3::5:6:7:8]", + "http://[1:2:3::8]", + "http://[1::4:5:6:7:8]", + "http://[1:2::4:5:6:7:8]", + "http://[1:2::8]", + "http://[1::3:4:5:6:7:8]", + "http://[1::3:4:5:6:7:8]", + "http://[1::8]", + "http://[::2:3:4:5:6:7:8]", + "http://[::2:3:4:5:6:7:8]", + "http://[::8]", + "http://[::]", + "http://[fe80::7:8%eth0]", + "http://[fe80::7:8%1]", + "http://[::255.255.255.255]", + "http://[::ffff:255.255.255.255]", + "http://[::ffff:0:255.255.255.255]", + "http://[2001:db8:3:4::192.0.2.33]", + "http://[64:ff9b::192.0.2.33]" +] + +edge_cases_gen = SetValuesGen(StringType(), edge_cases) + +url_gen = StringGen(url_pattern) + +@pytest.mark.parametrize('data_gen', [url_gen, edge_cases_gen], ids=idfn) +def test_parse_url_protocol(data_gen): + assert_gpu_and_cpu_are_equal_collect( + lambda spark : unary_op_df(spark, data_gen).selectExpr( + "a", + "parse_url(a, 'PROTOCOL')" + )) + +unsupported_parts = ['HOST', 'PATH', 'QUERY', 'REF', 'FILE', 'AUTHORITY', 'USERINFO'] + +@allow_non_gpu('ProjectExec', 'ParseUrl') +@pytest.mark.parametrize('part', unsupported_parts, ids=idfn) +def test_parse_url_host_fallback(part): + assert_gpu_fallback_collect( + lambda spark : unary_op_df(spark, url_gen).selectExpr( + "a", + "parse_url(a, '" + part + "')" + ), + 'ParseUrl') 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 74430ae8e90..6cb22f59885 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 @@ -3241,6 +3241,34 @@ object GpuOverrides extends Logging { ParamCheck("regexp", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING), ParamCheck("idx", TypeSig.lit(TypeEnum.INT), TypeSig.INT))), (a, conf, p, r) => new GpuRegExpExtractAllMeta(a, conf, p, r)), + expr[ParseUrl]( + "Extracts a part from a URL", + ExprChecks.projectOnly(TypeSig.STRING, TypeSig.STRING, + Seq(ParamCheck("url", TypeSig.STRING, TypeSig.STRING), + ParamCheck("partToExtract", TypeSig.lit(TypeEnum.STRING).withPsNote( + TypeEnum.STRING, "only support partToExtract=PROTOCOL"), TypeSig.STRING)), + // Should really be an OptionalParam + Some(RepeatingParamCheck("key", TypeSig.lit(TypeEnum.STRING), TypeSig.STRING))), + (a, conf, p, r) => new ExprMeta[ParseUrl](a, conf, p, r) { + override def tagExprForGpu(): Unit = { + if (a.failOnError) { + willNotWorkOnGpu("Fail on error is not supported on GPU when parsing urls.") + } + + extractStringLit(a.children(1)).map(_.toUpperCase) match { + case Some(GpuParseUrl.PROTOCOL) => + case Some(other) => + willNotWorkOnGpu(s"Part to extract $other is not supported on GPU") + case None => + // Should never get here, but just in case + willNotWorkOnGpu("GPU only supports a literal for the part to extract") + } + } + + override def convertToGpu(): GpuExpression = { + GpuParseUrl(childExprs.map(_.convertToGpu())) + } + }), expr[Length]( "String character length or binary byte length", ExprChecks.unaryProject(TypeSig.INT, TypeSig.INT, diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuParseUrl.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuParseUrl.scala new file mode 100644 index 00000000000..586814c38e7 --- /dev/null +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuParseUrl.scala @@ -0,0 +1,106 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.rapids + +import ai.rapids.cudf.ColumnVector +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.Arm._ +import com.nvidia.spark.rapids.RapidsPluginImplicits._ +import com.nvidia.spark.rapids.jni.ParseURI +import com.nvidia.spark.rapids.shims.ShimExpression + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types._ +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.unsafe.types.UTF8String + +object GpuParseUrl { + val HOST = "HOST" + val PATH = "PATH" + val QUERY = "QUERY" + val REF = "REF" + val PROTOCOL = "PROTOCOL" + val FILE = "FILE" + val AUTHORITY = "AUTHORITY" + val USERINFO = "USERINFO" +} + +case class GpuParseUrl(children: Seq[Expression]) + extends GpuExpression with ShimExpression with ExpectsInputTypes { + + override def nullable: Boolean = true + override def inputTypes: Seq[DataType] = Seq.fill(children.size)(StringType) + override def dataType: DataType = StringType + override def prettyName: String = "parse_url" + + import GpuParseUrl._ + + def doColumnar(url: GpuColumnVector, partToExtract: GpuScalar): ColumnVector = { + val part = partToExtract.getValue.asInstanceOf[UTF8String].toString + part match { + case PROTOCOL => + ParseURI.parseURIProtocol(url.getBase) + case HOST | PATH | QUERY | REF | FILE | AUTHORITY | USERINFO => + throw new UnsupportedOperationException(s"$this is not supported partToExtract=$part") + case _ => + throw new IllegalArgumentException(s"Invalid partToExtract: $partToExtract") + } + } + + def doColumnar(url: GpuColumnVector, partToExtract: GpuScalar, key: GpuScalar): ColumnVector = { + val part = partToExtract.getValue.asInstanceOf[UTF8String].toString + if (part != QUERY) { + // return a null columnvector + return ColumnVector.fromStrings(null, null) + } + throw new UnsupportedOperationException(s"$this only supports partToExtract = PROTOCOL") + } + + override def columnarEval(batch: ColumnarBatch): GpuColumnVector = { + if (children.size == 2) { + val Seq(url, partToExtract) = children + withResourceIfAllowed(url.columnarEval(batch)) { urls => + withResourceIfAllowed(partToExtract.columnarEvalAny(batch)) { parts => + parts match { + case partScalar: GpuScalar => + GpuColumnVector.from(doColumnar(urls, partScalar), dataType) + case _ => + throw new + UnsupportedOperationException(s"Cannot columnar evaluate expression: $this") + } + } + } + } else { + // 3-arg, i.e. QUERY with key + assert(children.size == 3) + val Seq(url, partToExtract, key) = children + withResourceIfAllowed(url.columnarEval(batch)) { urls => + withResourceIfAllowed(partToExtract.columnarEvalAny(batch)) { parts => + withResourceIfAllowed(key.columnarEvalAny(batch)) { keys => + (urls, parts, keys) match { + case (urlCv: GpuColumnVector, partScalar: GpuScalar, keyScalar: GpuScalar) => + GpuColumnVector.from(doColumnar(urlCv, partScalar, keyScalar), dataType) + case _ => + throw new + UnsupportedOperationException(s"Cannot columnar evaluate expression: $this") + } + } + } + } + } + } +} diff --git a/tools/generated_files/operatorsScore.csv b/tools/generated_files/operatorsScore.csv index 17c80f60bfb..4c7248bf975 100644 --- a/tools/generated_files/operatorsScore.csv +++ b/tools/generated_files/operatorsScore.csv @@ -181,6 +181,7 @@ Not,4 NthValue,4 OctetLength,4 Or,4 +ParseUrl,4 PercentRank,4 Percentile,4 PivotFirst,4 diff --git a/tools/generated_files/supportedExprs.csv b/tools/generated_files/supportedExprs.csv index bff8dc7359a..b55f893f40e 100644 --- a/tools/generated_files/supportedExprs.csv +++ b/tools/generated_files/supportedExprs.csv @@ -382,6 +382,10 @@ Or,S,`or`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA, Or,S,`or`,None,AST,lhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Or,S,`or`,None,AST,rhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Or,S,`or`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ParseUrl,S,`parse_url`,None,project,url,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +ParseUrl,S,`parse_url`,None,project,partToExtract,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA +ParseUrl,S,`parse_url`,None,project,key,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA +ParseUrl,S,`parse_url`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA PercentRank,S,`percent_rank`,None,window,ordering,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS PercentRank,S,`percent_rank`,None,window,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Pmod,S,`pmod`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA From 38eb267aceb8e2b06f0d5e36d97f14c20b7bd997 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Tue, 12 Dec 2023 15:15:05 -0600 Subject: [PATCH 17/19] Fix infinite loop in test_str_to_map_expr_random_delimiters (#10016) Signed-off-by: Jason Lowe --- integration_tests/src/main/python/map_test.py | 24 +++++++++++++++---- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/integration_tests/src/main/python/map_test.py b/integration_tests/src/main/python/map_test.py index d0e064535d5..d5e49d5eb65 100644 --- a/integration_tests/src/main/python/map_test.py +++ b/integration_tests/src/main/python/map_test.py @@ -369,13 +369,27 @@ def test_str_to_map_expr_fixed_delimiters(): def test_str_to_map_expr_random_delimiters(): + class UniqueStringGen(StringGen): + """Generate unique strings""" + def __init__(self, pattern, nullable): + super().__init__(pattern=pattern, nullable=nullable) + self.previous_values = set() + + def start(self, rand): + super().start(rand) + self.previous_values = set() + + def gen(self, force_no_nulls=False): + v = super().gen(force_no_nulls=force_no_nulls) + while v in self.previous_values: + v = super().gen(force_no_nulls=force_no_nulls) + self.previous_values.add(v) + return v data_gen = [('a', StringGen(pattern='[0-9a-z:,]{0,100}', nullable=True) .with_special_pattern('[abc]:.{0,20},[abc]:.{0,20}', weight=100))] - delim_gen = StringGen(pattern='[0-9a-z :,]', nullable=False) - (pair_delim, keyval_delim) = ('', '') - while pair_delim == keyval_delim: - (pair_delim, keyval_delim) = with_cpu_session( - lambda spark: gen_scalars_for_sql(delim_gen, 2, force_no_nulls=True)) + delim_gen = UniqueStringGen(pattern='[0-9a-z :,]', nullable=False) + (pair_delim, keyval_delim) = with_cpu_session( + lambda spark: gen_scalars_for_sql(delim_gen, 2, force_no_nulls=True)) assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, data_gen).selectExpr( 'str_to_map(a) as m1', From 2b4385147ba6f97e0e7fc1b2d1edaaee863000d5 Mon Sep 17 00:00:00 2001 From: Navin Kumar <97137715+NVnavkumar@users.noreply.github.com> Date: Tue, 12 Dec 2023 17:07:58 -0800 Subject: [PATCH 18/19] Remove GpuToTimestampImproved and spark.rapids.sql.improvedTimeOps.enabled (#10033) * Remove GpuToTimestampImproved and spark.rapids.sql.improvedTimeOps since no one is using and results are inconsistent with Spark Signed-off-by: Navin Kumar * Remove this config since it's not used Signed-off-by: Navin Kumar --------- Signed-off-by: Navin Kumar --- .../advanced_configs.md | 1 - .../src/main/python/date_time_test.py | 19 ----- .../src/main/python/spark_session.py | 1 - .../nvidia/spark/rapids/GpuOverrides.scala | 14 +--- .../com/nvidia/spark/rapids/RapidsConf.scala | 9 -- .../sql/rapids/datetimeExpressions.scala | 83 ------------------- .../spark/rapids/ParseDateTimeSuite.scala | 10 --- 7 files changed, 2 insertions(+), 135 deletions(-) diff --git a/docs/additional-functionality/advanced_configs.md b/docs/additional-functionality/advanced_configs.md index 40acd48329c..20a3691e269 100644 --- a/docs/additional-functionality/advanced_configs.md +++ b/docs/additional-functionality/advanced_configs.md @@ -114,7 +114,6 @@ Name | Description | Default Value | Applicable at spark.rapids.sql.hasExtendedYearValues|Spark 3.2.0+ extended parsing of years in dates and timestamps to support the full range of possible values. Prior to this it was limited to a positive 4 digit year. The Accelerator does not support the extended range yet. This config indicates if your data includes this extended range or not, or if you don't care about getting the correct values on values with the extended range.|true|Runtime spark.rapids.sql.hashOptimizeSort.enabled|Whether sorts should be inserted after some hashed operations to improve output ordering. This can improve output file sizes when saving to columnar formats.|false|Runtime spark.rapids.sql.improvedFloatOps.enabled|For some floating point operations spark uses one way to compute the value and the underlying cudf implementation can use an improved algorithm. In some cases this can result in cudf producing an answer when spark overflows.|true|Runtime -spark.rapids.sql.improvedTimeOps.enabled|When set to true, some operators will avoid overflowing by converting epoch days directly to seconds without first converting to microseconds|false|Runtime spark.rapids.sql.incompatibleDateFormats.enabled|When parsing strings as dates and timestamps in functions like unix_timestamp, some formats are fully supported on the GPU and some are unsupported and will fall back to the CPU. Some formats behave differently on the GPU than the CPU. Spark on the CPU interprets date formats with unsupported trailing characters as nulls, while Spark on the GPU will parse the date with invalid trailing characters. More detail can be found at [parsing strings as dates or timestamps](../compatibility.md#parsing-strings-as-dates-or-timestamps).|false|Runtime spark.rapids.sql.incompatibleOps.enabled|For operations that work, but are not 100% compatible with the Spark equivalent set if they should be enabled by default or disabled by default.|true|Runtime spark.rapids.sql.join.cross.enabled|When set to true cross joins are enabled on the GPU|true|Runtime diff --git a/integration_tests/src/main/python/date_time_test.py b/integration_tests/src/main/python/date_time_test.py index d44c261a9a5..1d4ce5e65d8 100644 --- a/integration_tests/src/main/python/date_time_test.py +++ b/integration_tests/src/main/python/date_time_test.py @@ -379,16 +379,6 @@ def fun(spark): assert_gpu_and_cpu_are_equal_collect(fun, conf=copy_and_update(parser_policy_dic, ansi_enabled_conf)) -@pytest.mark.parametrize('ansi_enabled', [True, False], ids=['ANSI_ON', 'ANSI_OFF']) -@pytest.mark.parametrize('data_gen', date_n_time_gens, ids=idfn) -@allow_non_gpu(*non_utc_allow) -def test_unix_timestamp_improved(data_gen, ansi_enabled): - conf = {"spark.rapids.sql.improvedTimeOps.enabled": "true", - "spark.sql.legacy.timeParserPolicy": "CORRECTED"} - assert_gpu_and_cpu_are_equal_collect( - lambda spark : unary_op_df(spark, data_gen).select(f.unix_timestamp(f.col('a'))), - copy_and_update({'spark.sql.ansi.enabled': ansi_enabled}, conf)) - @pytest.mark.parametrize('ansi_enabled', [True, False], ids=['ANSI_ON', 'ANSI_OFF']) @pytest.mark.parametrize('data_gen', date_n_time_gens, ids=idfn) @allow_non_gpu(*non_utc_allow) @@ -397,15 +387,6 @@ def test_unix_timestamp(data_gen, ansi_enabled): lambda spark : unary_op_df(spark, data_gen).select(f.unix_timestamp(f.col("a"))), {'spark.sql.ansi.enabled': ansi_enabled}) -@pytest.mark.parametrize('ansi_enabled', [True, False], ids=['ANSI_ON', 'ANSI_OFF']) -@pytest.mark.parametrize('data_gen', date_n_time_gens, ids=idfn) -@allow_non_gpu(*non_utc_allow) -def test_to_unix_timestamp_improved(data_gen, ansi_enabled): - conf = {"spark.rapids.sql.improvedTimeOps.enabled": "true"} - assert_gpu_and_cpu_are_equal_collect( - lambda spark : unary_op_df(spark, data_gen).selectExpr("to_unix_timestamp(a)"), - copy_and_update({'spark.sql.ansi.enabled': ansi_enabled}, conf)) - str_date_and_format_gen = [pytest.param(StringGen('[0-9]{4}/[01][0-9]'),'yyyy/MM', marks=pytest.mark.xfail(reason="cudf does no checks")), (StringGen('[0-9]{4}/[01][12]/[0-2][1-8]'),'yyyy/MM/dd'), (StringGen('[01][12]/[0-2][1-8]'), 'MM/dd'), diff --git a/integration_tests/src/main/python/spark_session.py b/integration_tests/src/main/python/spark_session.py index 606f9a31dc4..e65db7df3b3 100644 --- a/integration_tests/src/main/python/spark_session.py +++ b/integration_tests/src/main/python/spark_session.py @@ -47,7 +47,6 @@ def _from_scala_map(scala_map): 'spark.rapids.sql.hasExtendedYearValues': 'true', 'spark.rapids.sql.hashOptimizeSort.enabled': 'false', 'spark.rapids.sql.improvedFloatOps.enabled': 'false', - 'spark.rapids.sql.improvedTimeOps.enabled': 'false', 'spark.rapids.sql.incompatibleDateFormats.enabled': 'false', 'spark.rapids.sql.incompatibleOps.enabled': 'false', 'spark.rapids.sql.mode': 'executeongpu', 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 6cb22f59885..c63e615e98b 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 @@ -1696,12 +1696,7 @@ object GpuOverrides extends Logging { TypeSig.STRING)), (a, conf, p, r) => new UnixTimeExprMeta[ToUnixTimestamp](a, conf, p, r) { override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { - if (conf.isImprovedTimestampOpsEnabled) { - // passing the already converted strf string for a little optimization - GpuToUnixTimestampImproved(lhs, rhs, sparkFormat, strfFormat) - } else { - GpuToUnixTimestamp(lhs, rhs, sparkFormat, strfFormat) - } + GpuToUnixTimestamp(lhs, rhs, sparkFormat, strfFormat) } }), expr[UnixTimestamp]( @@ -1715,12 +1710,7 @@ object GpuOverrides extends Logging { TypeSig.STRING)), (a, conf, p, r) => new UnixTimeExprMeta[UnixTimestamp](a, conf, p, r) { override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { - if (conf.isImprovedTimestampOpsEnabled) { - // passing the already converted strf string for a little optimization - GpuUnixTimestampImproved(lhs, rhs, sparkFormat, strfFormat) - } else { - GpuUnixTimestamp(lhs, rhs, sparkFormat, strfFormat) - } + GpuUnixTimestamp(lhs, rhs, sparkFormat, strfFormat) } }), expr[Hour]( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index fa793ba84fa..631a935ede5 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -667,13 +667,6 @@ object RapidsConf { // ENABLE/DISABLE PROCESSING - val IMPROVED_TIMESTAMP_OPS = - conf("spark.rapids.sql.improvedTimeOps.enabled") - .doc("When set to true, some operators will avoid overflowing by converting epoch days " + - "directly to seconds without first converting to microseconds") - .booleanConf - .createWithDefault(false) - val SQL_ENABLED = conf("spark.rapids.sql.enabled") .doc("Enable (true) or disable (false) sql operations on the GPU") .commonlyUsed() @@ -2378,8 +2371,6 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val shouldExplainAll: Boolean = explain.equalsIgnoreCase("ALL") - lazy val isImprovedTimestampOpsEnabled: Boolean = get(IMPROVED_TIMESTAMP_OPS) - lazy val chunkedReaderEnabled: Boolean = get(CHUNKED_READER) lazy val maxReadBatchSizeRows: Int = get(MAX_READER_BATCH_SIZE_ROWS) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala index 913594e810c..7e4de33bb70 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala @@ -866,59 +866,6 @@ abstract class GpuToTimestamp } } -/** - * An improved version of GpuToTimestamp conversion which converts time to UNIX timestamp without - * first converting to microseconds - */ -abstract class GpuToTimestampImproved extends GpuToTimestamp { - import GpuToTimestamp._ - - override def doColumnar(lhs: GpuColumnVector, rhs: GpuScalar): ColumnVector = { - val tmp = if (lhs.dataType == StringType) { - // rhs is ignored we already parsed the format - if (getTimeParserPolicy == LegacyTimeParserPolicy) { - parseStringAsTimestampWithLegacyParserPolicy( - lhs, - sparkFormat, - strfFormat, - DType.TIMESTAMP_SECONDS, - (col, strfFormat) => col.asTimestampSeconds(strfFormat)) - } else { - parseStringAsTimestamp( - lhs, - sparkFormat, - strfFormat, - DType.TIMESTAMP_SECONDS, - failOnError) - } - } else if (lhs.dataType() == DateType){ - lhs.getBase.asTimestampSeconds() - } else { // Timestamp - // https://github.com/rapidsai/cudf/issues/5166 - // The time is off by 1 second if the result is < 0 - val longSecs = withResource(lhs.getBase.asTimestampSeconds()) { secs => - secs.asLongs() - } - withResource(longSecs) { secs => - val plusOne = withResource(Scalar.fromLong(1)) { one => - secs.add(one) - } - withResource(plusOne) { plusOne => - withResource(Scalar.fromLong(0)) { zero => - withResource(secs.lessThan(zero)) { neg => - neg.ifElse(plusOne, secs) - } - } - } - } - } - withResource(tmp) { r => - // The type we are returning is a long not an actual timestamp - r.asLongs() - } - } -} - case class GpuUnixTimestamp(strTs: Expression, format: Expression, sparkFormat: String, @@ -949,36 +896,6 @@ case class GpuToUnixTimestamp(strTs: Expression, } -case class GpuUnixTimestampImproved(strTs: Expression, - format: Expression, - sparkFormat: String, - strf: String, - timeZoneId: Option[String] = None) extends GpuToTimestampImproved { - override def strfFormat = strf - override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = { - copy(timeZoneId = Option(timeZoneId)) - } - - override def left: Expression = strTs - override def right: Expression = format - -} - -case class GpuToUnixTimestampImproved(strTs: Expression, - format: Expression, - sparkFormat: String, - strf: String, - timeZoneId: Option[String] = None) extends GpuToTimestampImproved { - override def strfFormat = strf - override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = { - copy(timeZoneId = Option(timeZoneId)) - } - - override def left: Expression = strTs - override def right: Expression = format - -} - case class GpuGetTimestamp( strTs: Expression, format: Expression, diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala index 74ef6e5cc73..e790be9cd8f 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala @@ -158,16 +158,6 @@ class ParseDateTimeSuite extends SparkQueryCompareTestSuite with BeforeAndAfterE } } - testSparkResultsAreEqual("to_unix_timestamp parse yyyy/MM (improvedTimeOps)", - timestampsAsStrings, - new SparkConf().set(SQLConf.LEGACY_TIME_PARSER_POLICY.key, "CORRECTED") - .set(RapidsConf.IMPROVED_TIMESTAMP_OPS.key, "true")) { - df => { - df.createOrReplaceTempView("df") - df.sqlContext.sql("SELECT c0, to_unix_timestamp(c0, 'yyyy/MM') FROM df") - } - } - testSparkResultsAreEqual("unix_timestamp parse timestamp", timestampsAsStrings, CORRECTED_TIME_PARSER_POLICY) { From b309f70a3d51a001ba5bd859dce89e693a6191c2 Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Wed, 13 Dec 2023 09:08:57 +0800 Subject: [PATCH 19/19] GpuFromUnixTime supports more formats by post process (#10023) Support format 'yyyyMMdd' for GpuFromUnixTime Signed-off-by: Firestarman --- .../src/main/python/date_time_test.py | 2 +- .../com/nvidia/spark/rapids/DateUtils.scala | 14 ++-- .../nvidia/spark/rapids/GpuOverrides.scala | 7 +- .../sql/rapids/datetimeExpressions.scala | 71 ++++++++++++++++++- 4 files changed, 78 insertions(+), 16 deletions(-) diff --git a/integration_tests/src/main/python/date_time_test.py b/integration_tests/src/main/python/date_time_test.py index 1d4ce5e65d8..4434655b8e3 100644 --- a/integration_tests/src/main/python/date_time_test.py +++ b/integration_tests/src/main/python/date_time_test.py @@ -455,7 +455,7 @@ def test_date_format(data_gen, date_format): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr("date_format(a, '{}')".format(date_format))) -@pytest.mark.parametrize('date_format', supported_date_formats, ids=idfn) +@pytest.mark.parametrize('date_format', supported_date_formats + ['yyyyMMdd'], ids=idfn) # from 0001-02-01 to 9999-12-30 to avoid 'year 0 is out of range' @pytest.mark.parametrize('data_gen', [LongGen(min_val=int(datetime(1, 2, 1).timestamp()), max_val=int(datetime(9999, 12, 30).timestamp()))], ids=idfn) @pytest.mark.skipif(not is_supported_time_zone(), reason="not all time zones are supported now, refer to https://github.com/NVIDIA/spark-rapids/issues/6839, please update after all time zones are supported") diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DateUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DateUtils.scala index ccb2e91f57a..771f8ecc695 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DateUtils.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DateUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2022, NVIDIA CORPORATION. + * Copyright (c) 2020-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -217,15 +217,17 @@ object DateUtils { def tagAndGetCudfFormat( meta: RapidsMeta[_, _, _], sparkFormat: String, - parseString: Boolean): String = { + parseString: Boolean, + inputFormat: Option[String] = None): String = { + val formatToConvert = inputFormat.getOrElse(sparkFormat) var strfFormat: String = null if (GpuOverrides.getTimeParserPolicy == LegacyTimeParserPolicy) { try { // try and convert the format to cuDF format - this will throw an exception if // the format contains unsupported characters or words - strfFormat = toStrf(sparkFormat, parseString) + strfFormat = toStrf(formatToConvert, parseString) // format parsed ok but we have no 100% compatible formats in LEGACY mode - if (GpuToTimestamp.LEGACY_COMPATIBLE_FORMATS.contains(sparkFormat)) { + if (GpuToTimestamp.LEGACY_COMPATIBLE_FORMATS.contains(formatToConvert)) { // LEGACY support has a number of issues that mean we cannot guarantee // compatibility with CPU // - we can only support 4 digit years but Spark supports a wider range @@ -249,9 +251,9 @@ object DateUtils { try { // try and convert the format to cuDF format - this will throw an exception if // the format contains unsupported characters or words - strfFormat = toStrf(sparkFormat, parseString) + strfFormat = toStrf(formatToConvert, parseString) // format parsed ok, so it is either compatible (tested/certified) or incompatible - if (!GpuToTimestamp.CORRECTED_COMPATIBLE_FORMATS.contains(sparkFormat) && + if (!GpuToTimestamp.CORRECTED_COMPATIBLE_FORMATS.contains(formatToConvert) && !meta.conf.incompatDateFormats) { meta.willNotWorkOnGpu(s"CORRECTED format '$sparkFormat' on the GPU is not guaranteed " + s"to produce the same results as Spark on CPU. Set " + 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 c63e615e98b..13369a25a15 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 @@ -1769,12 +1769,7 @@ object GpuOverrides extends Logging { ("format", TypeSig.lit(TypeEnum.STRING) .withPsNote(TypeEnum.STRING, "Only a limited number of formats are supported"), TypeSig.STRING)), - (a, conf, p, r) => new UnixTimeExprMeta[FromUnixTime](a, conf, p, r) { - override def isTimeZoneSupported = true - override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = - // passing the already converted strf string for a little optimization - GpuFromUnixTime(lhs, rhs, strfFormat, a.timeZoneId) - }), + (a, conf, p, r) => new FromUnixTimeMeta(a ,conf ,p ,r)), expr[FromUTCTimestamp]( "Render the input UTC timestamp in the input timezone", ExprChecks.binaryProject(TypeSig.TIMESTAMP, TypeSig.TIMESTAMP, diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala index 7e4de33bb70..238a65a3a65 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala @@ -27,7 +27,7 @@ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.jni.GpuTimeZoneDB import com.nvidia.spark.rapids.shims.ShimBinaryExpression -import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, ExpectsInputTypes, Expression, FromUTCTimestamp, ImplicitCastInputTypes, NullIntolerant, TimeZoneAwareExpression} +import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, ExpectsInputTypes, Expression, FromUnixTime, FromUTCTimestamp, ImplicitCastInputTypes, NullIntolerant, TimeZoneAwareExpression} import org.apache.spark.sql.catalyst.util.DateTimeConstants import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -914,19 +914,83 @@ case class GpuGetTimestamp( override def right: Expression = format } +class FromUnixTimeMeta(a: FromUnixTime, + override val conf: RapidsConf, + val p: Option[RapidsMeta[_, _, _]], + r: DataFromReplacementRule) extends UnixTimeExprMeta[FromUnixTime](a, conf, p, r) { + + private type FmtConverter = ColumnView => ColumnVector + + private var colConverter: Option[FmtConverter] = None + + /** + * More supported formats by post conversions. The idea is + * 1) Map the unsupported target format to a supported format as + * the intermediate format, + * 2) Call into cuDF with this intermediate format, + * 3) Run a post conversion to get the right output for the target format. + * + * NOTE: Need to remove the entry if the key format is supported by cuDF. + */ + private val FORMATS_BY_CONVERSION: Map[String, (String, FmtConverter)] = Map( + // spark format -> (intermediate format, converter) + "yyyyMMdd" -> (("yyyy-MM-dd", + col => { + withResource(Scalar.fromString("-")) { dashStr => + withResource(Scalar.fromString("")) { emptyStr => + col.stringReplace(dashStr, emptyStr) + } + } + } + )) + ) + + override def tagExprForGpu(): Unit = { + extractStringLit(a.right) match { + case Some(rightLit) => + sparkFormat = rightLit + var inputFormat: Option[String] = None + FORMATS_BY_CONVERSION.get(sparkFormat).foreach { case (tempFormat, converter) => + colConverter = Some(converter) + inputFormat = Some(tempFormat) + } + strfFormat = DateUtils.tagAndGetCudfFormat(this, sparkFormat, + a.left.dataType == DataTypes.StringType, inputFormat) + case None => + willNotWorkOnGpu("format has to be a string literal") + } + } + + override def isTimeZoneSupported = true + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = { + // passing the already converted strf string for a little optimization + GpuFromUnixTime(lhs, rhs, strfFormat, colConverter, a.timeZoneId) + } +} + case class GpuFromUnixTime( sec: Expression, format: Expression, strfFormat: String, - timeZoneId: Option[String] = None) + colConverter: Option[ColumnView => ColumnVector], + timeZoneId: Option[String]) extends GpuBinaryExpressionArgsAnyScalar with TimeZoneAwareExpression with ImplicitCastInputTypes { + // To avoid duplicated "if...else" for each input batch + private val convertFunc: ColumnVector => ColumnVector = { + if (colConverter.isDefined) { + col => withResource(col)(colConverter.get.apply) + } else { + identity[ColumnVector] + } + } + override def doColumnar(lhs: GpuColumnVector, rhs: GpuScalar): ColumnVector = { // we aren't using rhs as it was already converted in the GpuOverrides while creating the // expressions map and passed down here as strfFormat - withResource(lhs.getBase.asTimestampSeconds) { secondCV => + val ret = withResource(lhs.getBase.asTimestampSeconds) { secondCV => if (GpuOverrides.isUTCTimezone(zoneId)) { // UTC time zone secondCV.asStrings(strfFormat) @@ -937,6 +1001,7 @@ case class GpuFromUnixTime( } } } + convertFunc(ret) } override def doColumnar(numRows: Int, lhs: GpuScalar, rhs: GpuScalar): ColumnVector = {