From b471bd77226eb24d7125fc1de93e12131d9a28d6 Mon Sep 17 00:00:00 2001 From: Nghia Truong <7416935+ttnghia@users.noreply.github.com> Date: Wed, 11 Dec 2024 06:23:08 -0800 Subject: [PATCH 01/47] Enable tests (#11805) Signed-off-by: Nghia Truong --- integration_tests/src/main/python/json_test.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index b825975f398..726225346f5 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -1012,7 +1012,6 @@ def test_from_json_struct_of_list_with_mismatched_schema(): 'struct>>', 'struct>>']) @allow_non_gpu(*non_utc_allow) -@pytest.mark.xfail(reason='https://github.com/rapidsai/cudf/issues/17349') def test_from_json_struct_of_list_with_mixed_nested_types_input(schema): json_string_gen = StringGen(r'{"teacher": "[A-Z]{1}[a-z]{2,5}",' \ r'"student": \[{"name": "[A-Z]{1}[a-z]{2,5}", "class": "junior"},' \ @@ -1399,7 +1398,6 @@ def test_spark_from_json_empty_table(data): conf =_enable_all_types_conf) # SPARK-20549: from_json bad UTF-8 -@pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/10483') @allow_non_gpu(*non_utc_allow) # https://github.com/NVIDIA/spark-rapids/issues/10453 def test_spark_from_json_bad_json(): schema = StructType([StructField("a", IntegerType())]) From 38d66b0c58fa8c8296c216a5626a3233877a67fe Mon Sep 17 00:00:00 2001 From: MithunR Date: Wed, 11 Dec 2024 15:42:22 -0800 Subject: [PATCH 02/47] Document Hive text write serialization format checks (#11856) This commit documents the serialization format checks for writing Hive text, and why it differs from the read-side. `spark-rapids` supports only '^A'-separated Hive text files for read and write. This format tends to be denoted in a Hive table's Storage Properties with `serialization.format=1`. If a Hive table is written with a different/custom delimiter, it is denoted with a different value of `serialization.format`. For instance, a CSV table might be denoted by `serialization.format='', field.delim=','`. It was noticed in https://github.com/NVIDIA/spark-rapids/issues/11803 that: 1. On the [read side](https://github.com/NVIDIA/spark-rapids/blob/aa2da410511d8a737e207257769ec662a79174fe/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/HiveProviderImpl.scala#L155-L161), `spark-rapids` treats an empty `serialization.format` as `''`. 2. On the [write side](https://github.com/NVIDIA/spark-rapids/blob/aa2da410511d8a737e207257769ec662a79174fe/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala#L130-L136), an empty `serialization.format` is seen as `1`. The reason for the read side value is to be conservative. Since the table is pre-existing, its value should have been set already. The reason for the write side is that there are legitimate cases where a table might not have its `serialization.format` set. (CTAS, for one.) This commit documents all the scenarios that need to be considered on the write side. Signed-off-by: MithunR --- .../sql/hive/rapids/GpuHiveFileFormat.scala | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala index 3b5244e5c79..d39050a0c32 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala @@ -127,6 +127,38 @@ object GpuHiveFileFormat extends Logging { s"only $lazySimpleSerDe is currently supported for text") } + // The check for serialization key here differs slightly from the read-side check in + // HiveProviderImpl::getExecs(): + // 1. On the read-side, we do a strict check for `serialization.format == 1`, denoting + // '^A'-separated text. All other formatting is unsupported. + // 2. On the write-side too, we support only `serialization.format == 1`. But if + // `serialization.format` hasn't been set yet, it is still treated as `^A` separated. + // + // On the write side, there are a couple of scenarios to consider: + // 1. If the destination table exists beforehand, `serialization.format` should have been + // set already, to a non-empty value. This will look like: + // ```sql + // CREATE TABLE destination_table( col INT, ... ); --> serialization.format=1 + // INSERT INTO TABLE destination_table SELECT * FROM ... + // ``` + // 2. If the destination table is being created as part of a CTAS, without an explicit + // format specified, then Spark leaves `serialization.format` unpopulated, until *AFTER* + // the write operation is completed. Such a query might look like: + // ```sql + // CREATE TABLE destination_table AS SELECT * FROM ... + // --> serialization.format is absent from Storage Properties. "1" is inferred. + // ``` + // 3. If the destination table is being created as part of a CTAS, with a non-default + // text format specified explicitly, then the non-default `serialization.format` is made + // available as part of the destination table's storage properties. Such a table creation + // might look like: + // ```sql + // CREATE TABLE destination_table + // ROW FORMAT DELIMITED FIELDS TERMINATED BY `,` STORED AS TEXTFILE + // AS SELECT * FROM ... + // --> serialization.format="", field.delim=",". Unsupported case. + // ``` + // All these cases may be covered by explicitly checking for `serialization.format=1`. val serializationFormat = storage.properties.getOrElse(serializationKey, "1") if (serializationFormat != ctrlASeparatedFormat) { meta.willNotWorkOnGpu(s"unsupported serialization format found: " + From c0fe534aeb26c849aa9653211cfeefca3f56bfc2 Mon Sep 17 00:00:00 2001 From: "Hongbin Ma (Mahone)" Date: Fri, 13 Dec 2024 08:09:42 +0800 Subject: [PATCH 03/47] add a few more stage level metrics (#11821) * add a few more stage level metrics Signed-off-by: Hongbin Ma (Mahone) * address comments Signed-off-by: Hongbin Ma (Mahone) * address comments Signed-off-by: Hongbin Ma (Mahone) --------- Signed-off-by: Hongbin Ma (Mahone) --- .../nvidia/spark/rapids/GpuSemaphore.scala | 15 ++- .../spark/rapids/PrioritySemaphore.scala | 12 ++- .../spark/sql/rapids/GpuTaskMetrics.scala | 95 ++++++++++++++++++- .../spark/rapids/PrioritySemaphoreSuite.scala | 10 +- 4 files changed, 120 insertions(+), 12 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSemaphore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSemaphore.scala index 719c4525373..68912193920 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSemaphore.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSemaphore.scala @@ -133,6 +133,7 @@ object GpuSemaphore { } private val MAX_PERMITS = 1000 + val DEFAULT_PRIORITY = 0L def computeNumPermits(conf: SQLConf): Int = { val concurrentStr = conf.getConfString(RapidsConf.CONCURRENT_GPU_TASKS.key, null) @@ -184,7 +185,8 @@ private final class SemaphoreTaskInfo(val stageId: Int, val taskAttemptId: Long) * If this task holds the GPU semaphore or not. */ private var hasSemaphore = false - private var lastHeld: Long = 0 + private var lastAcquired: Long = GpuSemaphore.DEFAULT_PRIORITY + private var lastReleased: Long = GpuSemaphore.DEFAULT_PRIORITY type GpuBackingSemaphore = PrioritySemaphore[Long] @@ -256,11 +258,12 @@ private final class SemaphoreTaskInfo(val stageId: Int, val taskAttemptId: Long) if (!done && shouldBlockOnSemaphore) { // We cannot be in a synchronized block and wait on the semaphore // so we have to release it and grab it again afterwards. - semaphore.acquire(numPermits, lastHeld, taskAttemptId) + semaphore.acquire(numPermits, lastReleased, taskAttemptId) synchronized { // We now own the semaphore so we need to wake up all of the other tasks that are // waiting. hasSemaphore = true + lastAcquired = System.nanoTime() if (trackSemaphore) { nvtxRange = Some(new NvtxUniqueRange(s"Stage ${stageId} Task ${taskAttemptId} owning GPU", @@ -296,9 +299,10 @@ private final class SemaphoreTaskInfo(val stageId: Int, val taskAttemptId: Long) } else { if (blockedThreads.size() == 0) { // No other threads for this task are waiting, so we might be able to grab this directly - val ret = semaphore.tryAcquire(numPermits, lastHeld, taskAttemptId) + val ret = semaphore.tryAcquire(numPermits, lastReleased, taskAttemptId) if (ret) { hasSemaphore = true + lastAcquired = System.nanoTime() activeThreads.add(t) // no need to notify because there are no other threads and we are holding the lock // to ensure that. @@ -316,7 +320,8 @@ private final class SemaphoreTaskInfo(val stageId: Int, val taskAttemptId: Long) if (hasSemaphore) { semaphore.release(numPermits) hasSemaphore = false - lastHeld = System.currentTimeMillis() + lastReleased = System.nanoTime() + GpuTaskMetrics.get.addSemaphoreHoldingTime(lastReleased - lastAcquired) nvtxRange.foreach(_.close()) nvtxRange = None } @@ -333,7 +338,7 @@ private final class GpuSemaphore() extends Logging { import GpuSemaphore._ type GpuBackingSemaphore = PrioritySemaphore[Long] - private val semaphore = new GpuBackingSemaphore(MAX_PERMITS) + private val semaphore = new GpuBackingSemaphore(MAX_PERMITS, GpuSemaphore.DEFAULT_PRIORITY) // A map of taskAttemptId => semaphoreTaskInfo. // This map keeps track of all tasks that are both active on the GPU and blocked waiting // on the GPU. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/PrioritySemaphore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/PrioritySemaphore.scala index dc90382d3a0..594059d3c87 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/PrioritySemaphore.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/PrioritySemaphore.scala @@ -19,7 +19,12 @@ package com.nvidia.spark.rapids import java.util.PriorityQueue import java.util.concurrent.locks.{Condition, ReentrantLock} -class PrioritySemaphore[T](val maxPermits: Int)(implicit ordering: Ordering[T]) { +import scala.collection.JavaConverters.asScalaIteratorConverter + +import org.apache.spark.sql.rapids.GpuTaskMetrics + +class PrioritySemaphore[T](val maxPermits: Int, val priorityForNonStarted: T) + (implicit ordering: Ordering[T]) { // This lock is used to generate condition variables, which affords us the flexibility to notify // specific threads at a time. If we use the regular synchronized pattern, we have to either // notify randomly, or if we try creating condition variables not tied to a shared lock, they @@ -69,6 +74,11 @@ class PrioritySemaphore[T](val maxPermits: Int)(implicit ordering: Ordering[T]) val info = ThreadInfo(priority, condition, numPermits, taskAttemptId) try { waitingQueue.add(info) + // only count tasks that had held semaphore before, + // so they're very likely to have remaining data on GPU + GpuTaskMetrics.get.recordOnGpuTasksWaitingNumber( + waitingQueue.iterator().asScala.count(_.priority != priorityForNonStarted)) + while (!info.signaled) { info.condition.await() } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuTaskMetrics.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuTaskMetrics.scala index 84ca5e2ac51..e21a9b71cb3 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuTaskMetrics.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuTaskMetrics.scala @@ -107,12 +107,95 @@ class HighWatermarkAccumulator extends AccumulatorV2[jl.Long, Long] { override def value: Long = _value } +class MaxLongAccumulator extends AccumulatorV2[jl.Long, jl.Long] { + private var _v = 0L + + override def isZero: Boolean = _v == 0 + + override def copy(): MaxLongAccumulator = { + val newAcc = new MaxLongAccumulator + newAcc._v = this._v + newAcc + } + + override def reset(): Unit = { + _v = 0L + } + + override def add(v: jl.Long): Unit = { + if(v > _v) { + _v = v + } + } + + def add(v: Long): Unit = { + if(v > _v) { + _v = v + } + } + + override def merge(other: AccumulatorV2[jl.Long, jl.Long]): Unit = other match { + case o: MaxLongAccumulator => + add(o.value) + case _ => + throw new UnsupportedOperationException( + s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") + } + + override def value: jl.Long = _v +} + +class AvgLongAccumulator extends AccumulatorV2[jl.Long, jl.Double] { + private var _sum = 0L + private var _count = 0L + + override def isZero: Boolean = _count == 0L + + override def copy(): AvgLongAccumulator = { + val newAcc = new AvgLongAccumulator + newAcc._sum = this._sum + newAcc._count = this._count + newAcc + } + + override def reset(): Unit = { + _sum = 0L + _count = 0L + } + + override def add(v: jl.Long): Unit = { + _sum += v + _count += 1 + } + + override def merge(other: AccumulatorV2[jl.Long, jl.Double]): Unit = other match { + case o: AvgLongAccumulator => + _sum += o._sum + _count += o._count + case _ => + throw new UnsupportedOperationException( + s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") + } + + override def value: jl.Double = if (_count != 0) { + 1.0 * _sum / _count + } else 0; +} + class GpuTaskMetrics extends Serializable { + private val semaphoreHoldingTime = new NanoSecondAccumulator private val semWaitTimeNs = new NanoSecondAccumulator private val retryCount = new LongAccumulator private val splitAndRetryCount = new LongAccumulator private val retryBlockTime = new NanoSecondAccumulator private val retryComputationTime = new NanoSecondAccumulator + // onGpuTask means a task that has data in GPU memory. + // Since it's not easy to decided if a task has data in GPU memory, + // We only count the tasks that had held semaphore before, + // so it's very likely to have data in GPU memory + private val onGpuTasksInWaitingQueueAvgCount = new AvgLongAccumulator + private val onGpuTasksInWaitingQueueMaxCount = new MaxLongAccumulator + // Spill private val spillToHostTimeNs = new NanoSecondAccumulator @@ -156,6 +239,7 @@ class GpuTaskMetrics extends Serializable { } private val metrics = Map[String, AccumulatorV2[_, _]]( + "gpuTime" -> semaphoreHoldingTime, "gpuSemaphoreWait" -> semWaitTimeNs, "gpuRetryCount" -> retryCount, "gpuSplitAndRetryCount" -> splitAndRetryCount, @@ -167,7 +251,9 @@ class GpuTaskMetrics extends Serializable { "gpuReadSpillFromDiskTime" -> readSpillFromDiskTimeNs, "gpuMaxDeviceMemoryBytes" -> maxDeviceMemoryBytes, "gpuMaxHostMemoryBytes" -> maxHostMemoryBytes, - "gpuMaxDiskMemoryBytes" -> maxDiskMemoryBytes + "gpuMaxDiskMemoryBytes" -> maxDiskMemoryBytes, + "gpuOnGpuTasksWaitingGPUAvgCount" -> onGpuTasksInWaitingQueueAvgCount, + "gpuOnGpuTasksWaitingGPUMaxCount" -> onGpuTasksInWaitingQueueMaxCount ) def register(sc: SparkContext): Unit = { @@ -204,6 +290,8 @@ class GpuTaskMetrics extends Serializable { } } + def addSemaphoreHoldingTime(duration: Long): Unit = semaphoreHoldingTime.add(duration) + def getSemWaitTime(): Long = semWaitTimeNs.value.value def semWaitTime[A](f: => A): A = timeIt(semWaitTimeNs, "Acquire GPU", NvtxColor.RED, f) @@ -263,6 +351,11 @@ class GpuTaskMetrics extends Serializable { maxDiskMemoryBytes.add(maxDiskBytesAllocated) } } + + def recordOnGpuTasksWaitingNumber(num: Int): Unit = { + onGpuTasksInWaitingQueueAvgCount.add(num) + onGpuTasksInWaitingQueueMaxCount.add(num) + } } /** diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/PrioritySemaphoreSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/PrioritySemaphoreSuite.scala index 7199aa55df6..a2356a68c1e 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/PrioritySemaphoreSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/PrioritySemaphoreSuite.scala @@ -24,7 +24,7 @@ class PrioritySemaphoreSuite extends AnyFunSuite { type TestPrioritySemaphore = PrioritySemaphore[Long] test("tryAcquire should return true if permits are available") { - val semaphore = new TestPrioritySemaphore(10) + val semaphore = new TestPrioritySemaphore(10, GpuSemaphore.DEFAULT_PRIORITY ) assert(semaphore.tryAcquire(5, 0, 0)) assert(semaphore.tryAcquire(3, 0, 0)) @@ -33,7 +33,7 @@ class PrioritySemaphoreSuite extends AnyFunSuite { } test("acquire and release should work correctly") { - val semaphore = new TestPrioritySemaphore(1) + val semaphore = new TestPrioritySemaphore(1, GpuSemaphore.DEFAULT_PRIORITY) assert(semaphore.tryAcquire(1, 0, 0)) @@ -57,7 +57,7 @@ class PrioritySemaphoreSuite extends AnyFunSuite { } test("multiple threads should handle permits and priority correctly") { - val semaphore = new TestPrioritySemaphore(0) + val semaphore = new TestPrioritySemaphore(0, GpuSemaphore.DEFAULT_PRIORITY) val results = new java.util.ArrayList[Int]() def taskWithPriority(priority: Int) = new Runnable { @@ -83,7 +83,7 @@ class PrioritySemaphoreSuite extends AnyFunSuite { } test("low priority thread cannot surpass high priority thread") { - val semaphore = new TestPrioritySemaphore(10) + val semaphore = new TestPrioritySemaphore(10, GpuSemaphore.DEFAULT_PRIORITY) semaphore.acquire(5, 0, 0) val t = new Thread(() => { semaphore.acquire(10, 2, 0) @@ -103,7 +103,7 @@ class PrioritySemaphoreSuite extends AnyFunSuite { // this case is described at https://github.com/NVIDIA/spark-rapids/pull/11574/files#r1795652488 test("thread with larger task id should not surpass smaller task id in the waiting queue") { - val semaphore = new TestPrioritySemaphore(10) + val semaphore = new TestPrioritySemaphore(10, GpuSemaphore.DEFAULT_PRIORITY) semaphore.acquire(8, 0, 0) val t = new Thread(() => { semaphore.acquire(5, 0, 0) From 4b9bb2330ca646f44d2c0bc2845f197c2e34d83d Mon Sep 17 00:00:00 2001 From: liyuan <84758614+nvliyuan@users.noreply.github.com> Date: Fri, 13 Dec 2024 09:41:45 +0800 Subject: [PATCH 04/47] [DOC] update doc for 24.12 release [skip ci] (#11841) * update download page Signed-off-by: liyuan * update download page Signed-off-by: liyuan * update download page Signed-off-by: liyuan * update download page Signed-off-by: liyuan * update download page Signed-off-by: liyuan --------- Signed-off-by: liyuan --- docs/archive.md | 90 ++++++++++++++++++++++++++++++++++++++++++++++++ docs/download.md | 33 ++++++++++-------- 2 files changed, 108 insertions(+), 15 deletions(-) diff --git a/docs/archive.md b/docs/archive.md index 2d9a78ca3d4..4f9c5ed49fc 100644 --- a/docs/archive.md +++ b/docs/archive.md @@ -5,6 +5,96 @@ nav_order: 15 --- Below are archived releases for RAPIDS Accelerator for Apache Spark. +## Release v24.10.1 +### Hardware Requirements: + +The plugin is tested on the following architectures: + + GPU Models: NVIDIA V100, T4, A10/A100, L4 and H100 GPUs + +### Software Requirements: + + OS: Spark RAPIDS is compatible with any Linux distribution with glibc >= 2.28 (Please check ldd --version output). glibc 2.28 was released August 1, 2018. + Tested on Ubuntu 20.04, Ubuntu 22.04, Rocky Linux 8 and Rocky Linux 9 + + NVIDIA Driver*: R470+ + + Runtime: + 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. + + 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, 3.3.4 + Apache Spark 3.4.0, 3.4.1, 3.4.2, 3.4.3 + Apache Spark 3.5.0, 3.5.1, 3.5.2 + + Supported Databricks runtime versions for Azure and AWS: + 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) + Databricks 13.3 ML LTS (GPU, Scala 2.12, Spark 3.4.1) + + Supported Dataproc versions (Debian/Ubuntu/Rocky): + GCP Dataproc 2.1 + GCP Dataproc 2.2 + + Supported Dataproc Serverless versions: + Spark runtime 1.1 LTS + Spark runtime 2.0 + Spark runtime 2.1 + Spark runtime 2.2 + +*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 RAPIDS Accelerator for Apache Spark v24.10.1 + +| Processor | Scala Version | Download Jar | Download Signature | +|-----------|---------------|--------------|--------------------| +| x86_64 | Scala 2.12 | [RAPIDS Accelerator v24.10.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.10.1/rapids-4-spark_2.12-24.10.1.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.10.1/rapids-4-spark_2.12-24.10.1.jar.asc) | +| x86_64 | Scala 2.13 | [RAPIDS Accelerator v24.10.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.10.1/rapids-4-spark_2.13-24.10.1.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.10.1/rapids-4-spark_2.13-24.10.1.jar.asc) | +| arm64 | Scala 2.12 | [RAPIDS Accelerator v24.10.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.10.1/rapids-4-spark_2.12-24.10.1-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.10.1/rapids-4-spark_2.12-24.10.1-cuda11-arm64.jar.asc) | +| arm64 | Scala 2.13 | [RAPIDS Accelerator v24.10.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.10.1/rapids-4-spark_2.13-24.10.1-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.10.1/rapids-4-spark_2.13-24.10.1-cuda11-arm64.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 [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com). +* Import the public key: `gpg --import PUB_KEY` +* Verify the signature for Scala 2.12 jar: + `gpg --verify rapids-4-spark_2.12-24.10.1.jar.asc rapids-4-spark_2.12-24.10.1.jar` +* Verify the signature for Scala 2.13 jar: + `gpg --verify rapids-4-spark_2.13-24.10.1.jar.asc rapids-4-spark_2.13-24.10.1.jar` + +The output of signature verify: + + gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) " + +### Release Notes +* Optimize scheduling policy for GPU Semaphore +* Support distinct join for right outer joins +* Support MinBy and MaxBy for non-float ordering +* Support ArrayJoin expression +* Optimize Expand and Aggregate expression performance +* Improve JSON related expressions +* For updates on RAPIDS Accelerator Tools, please visit [this link](https://github.com/NVIDIA/spark-rapids-tools/releases) + +Note: There is a known issue in the 24.10.1 release when decompressing gzip files on H100 GPUs. +Please find more details in [issue-16661](https://github.com/rapidsai/cudf/issues/16661). + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + ## Release v24.10.0 ### Hardware Requirements: diff --git a/docs/download.md b/docs/download.md index 60c62071f8b..e16c94e5b90 100644 --- a/docs/download.md +++ b/docs/download.md @@ -18,7 +18,7 @@ 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 v24.10.1 +## Release v24.12.0 ### Hardware Requirements: The plugin is tested on the following architectures: @@ -69,14 +69,14 @@ for your hardware's minimum driver version. ### 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 RAPIDS Accelerator for Apache Spark v24.10.1 +### Download RAPIDS Accelerator for Apache Spark v24.12.0 | Processor | Scala Version | Download Jar | Download Signature | |-----------|---------------|--------------|--------------------| -| x86_64 | Scala 2.12 | [RAPIDS Accelerator v24.10.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.10.1/rapids-4-spark_2.12-24.10.1.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.10.1/rapids-4-spark_2.12-24.10.1.jar.asc) | -| x86_64 | Scala 2.13 | [RAPIDS Accelerator v24.10.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.10.1/rapids-4-spark_2.13-24.10.1.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.10.1/rapids-4-spark_2.13-24.10.1.jar.asc) | -| arm64 | Scala 2.12 | [RAPIDS Accelerator v24.10.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.10.1/rapids-4-spark_2.12-24.10.1-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.10.1/rapids-4-spark_2.12-24.10.1-cuda11-arm64.jar.asc) | -| arm64 | Scala 2.13 | [RAPIDS Accelerator v24.10.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.10.1/rapids-4-spark_2.13-24.10.1-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.10.1/rapids-4-spark_2.13-24.10.1-cuda11-arm64.jar.asc) | +| x86_64 | Scala 2.12 | [RAPIDS Accelerator v24.12.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.0/rapids-4-spark_2.12-24.12.0.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.0/rapids-4-spark_2.12-24.12.0.jar.asc) | +| x86_64 | Scala 2.13 | [RAPIDS Accelerator v24.12.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.0/rapids-4-spark_2.13-24.12.0.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.0/rapids-4-spark_2.13-24.12.0.jar.asc) | +| arm64 | Scala 2.12 | [RAPIDS Accelerator v24.12.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.0/rapids-4-spark_2.12-24.12.0-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.0/rapids-4-spark_2.12-24.12.0-cuda11-arm64.jar.asc) | +| arm64 | Scala 2.13 | [RAPIDS Accelerator v24.12.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.0/rapids-4-spark_2.13-24.12.0-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.0/rapids-4-spark_2.13-24.12.0-cuda11-arm64.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. @@ -85,24 +85,27 @@ CUDA 11.8 through CUDA 12.0. * 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 for Scala 2.12 jar: - `gpg --verify rapids-4-spark_2.12-24.10.1.jar.asc rapids-4-spark_2.12-24.10.1.jar` + `gpg --verify rapids-4-spark_2.12-24.12.0.jar.asc rapids-4-spark_2.12-24.12.0.jar` * Verify the signature for Scala 2.13 jar: - `gpg --verify rapids-4-spark_2.13-24.10.1.jar.asc rapids-4-spark_2.13-24.10.1.jar` + `gpg --verify rapids-4-spark_2.13-24.12.0.jar.asc rapids-4-spark_2.13-24.12.0.jar` The output of signature verify: gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) " ### Release Notes -* Optimize scheduling policy for GPU Semaphore -* Support distinct join for right outer joins -* Support MinBy and MaxBy for non-float ordering -* Support ArrayJoin expression -* Optimize Expand and Aggregate expression performance -* Improve JSON related expressions +* Add repartition-based algorithm fallback in hash aggregate +* Support Spark function months_between +* Support asynchronous writing for Parquet files +* Add retry support to improve sub hash-join stability +* Improve JSON scan and from_json +* Improved performance for CASE WHEN statements comparing a string column against multiple values +* Falling back to the CPU for ORC boolean writes by the GPU due to a bug in cudf's ORC writer +* Fix a device memory leak in timestamp operator in `incompatibleDateFormats` case +* Fix a host memory leak in GpuBroadcastNestedLoopJoinExecBase when `spillableBuiltBatch` is 0 * For updates on RAPIDS Accelerator Tools, please visit [this link](https://github.com/NVIDIA/spark-rapids-tools/releases) -Note: There is a known issue in the 24.10.1 release when decompressing gzip files on H100 GPUs. +Note: There is a known issue in the 24.12.0 release when decompressing gzip files on H100 GPUs. Please find more details in [issue-16661](https://github.com/rapidsai/cudf/issues/16661). For a detailed list of changes, please refer to the From edb7a678e01b4cdd401743dc15f50cd9b6af1683 Mon Sep 17 00:00:00 2001 From: Tim Liu Date: Fri, 13 Dec 2024 14:50:56 +0800 Subject: [PATCH 05/47] Generate the CHANGELOG based on the PR's target branch [skip ci] (#11867) * Generate the CHANGELOG based on the PR's target branch To fix: https://github.com/NVIDIA/spark-rapids/issues/11866 Generate the CHANGELOG based on the PR's target branch if the PR's project roadmap is empty. Signed-off-by: Tim Liu * Remove the unused variable 'no_project_prs.' We won't need the 'no_project_prs' variable because PRs will be added to the CHANGELOG. Signed-off-by: Tim Liu * Remove project cards checking it is no longer necessary to maintain compatibility with the old version of GitHub Projects Signed-off-by: Tim Liu --------- Signed-off-by: Tim Liu --- scripts/generate-changelog | 34 +++++++++++----------------------- 1 file changed, 11 insertions(+), 23 deletions(-) diff --git a/scripts/generate-changelog b/scripts/generate-changelog index d943160ebc6..8a56d82ccd7 100755 --- a/scripts/generate-changelog +++ b/scripts/generate-changelog @@ -1,6 +1,6 @@ #!/usr/bin/env python -# Copyright (c) 2020-2023, NVIDIA CORPORATION. +# Copyright (c) 2020-2024, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -164,8 +164,7 @@ query ($after: String, $since: DateTime) { """ -def process_changelog(resource_type: str, changelog: dict, releases: set, projects: set, no_project_prs: list, - token: str): +def process_changelog(resource_type: str, changelog: dict, releases: set, projects: set, token: str): if resource_type == PULL_REQUESTS: items = process_pr(releases=releases, token=token) time_field = 'mergedAt' @@ -178,14 +177,14 @@ def process_changelog(resource_type: str, changelog: dict, releases: set, projec for item in items: if len(item["projectItems"]["nodes"]) == 0 or not item["projectItems"]["nodes"][0]['roadmap']: - # compatibility support for project API V1 - if len(item['projectCards']['nodes']) == 0: - if resource_type == PULL_REQUESTS: - if '[bot]' in item['title']: - continue # skip auto-gen PR - no_project_prs.append(item) + if resource_type == PULL_REQUESTS: + if '[bot]' in item['title']: + continue # skip auto-gen PR + # Obtain the version from the PR's target branch, e.g. branch-x.y --> x.y + ver = item['baseRefName'].replace('branch-', '') + project = f"{RELEASE} {ver}" + else: continue - project = item['projectCards']['nodes'][0]['project']['name'] else: ver = item["projectItems"]["nodes"][0]['roadmap']['name'] project = f"{RELEASE} {ver}" @@ -309,12 +308,6 @@ def form_subsection(issues: dict, subtitle: str): return subsection -def print_no_project_pr(no_project_prs: list): - if len(no_project_prs) != 0: - print("\nNOTE: Merged Pull Requests w/o Project:") - for pr in no_project_prs: - print(f"{pr['baseRefName']} #{pr['number']} {pr['title']} {pr['url']}") - def main(rels: str, path: str, token: str): print('Generating changelog ...') @@ -323,16 +316,13 @@ def main(rels: str, path: str, token: str): changelog = {} # changelog dict releases = {x.strip() for x in rels.split(',')} projects = {f"{RELEASE} {rel}" for rel in releases} - no_project_prs = [] # list of merge pr w/o project print('Processing pull requests ...') process_changelog(resource_type=PULL_REQUESTS, changelog=changelog, - releases=releases, projects=projects, - no_project_prs=no_project_prs, token=token) + releases=releases, projects=projects, token=token) print('Processing issues ...') process_changelog(resource_type=ISSUES, changelog=changelog, - releases=releases, projects=projects, - no_project_prs=no_project_prs, token=token) + releases=releases, projects=projects, token=token) # form doc form_changelog(path=path, changelog=changelog) except Exception as e: # pylint: disable=broad-except @@ -340,8 +330,6 @@ def main(rels: str, path: str, token: str): sys.exit(1) print('Done.') - # post action - print_no_project_pr(no_project_prs=no_project_prs) if __name__ == '__main__': From 561068c50f973d5a3751bcc078d41075d13cbbdb Mon Sep 17 00:00:00 2001 From: Peixin Date: Fri, 13 Dec 2024 22:29:06 +0800 Subject: [PATCH 06/47] Workaround: Exclude cudf_log.txt in RAT check (#11870) Signed-off-by: Peixin Li --- .gitignore | 1 + pom.xml | 1 + scala2.13/pom.xml | 1 + 3 files changed, 3 insertions(+) diff --git a/.gitignore b/.gitignore index ed8a60ec20a..cf255a30e18 100644 --- a/.gitignore +++ b/.gitignore @@ -35,4 +35,5 @@ scalastyle-output.xml scalastyle.txt target/ cufile.log +cudf_log.txt build/*.class diff --git a/pom.xml b/pom.xml index bcfaf4f04af..dba06a7d08c 100644 --- a/pom.xml +++ b/pom.xml @@ -1645,6 +1645,7 @@ This will force full Scala code rebuild in downstream modules. default, but there are some projects that are conditionally included. --> **/target/**/* **/cufile.log + **/cudf_log.txt thirdparty/parquet-testing/** diff --git a/scala2.13/pom.xml b/scala2.13/pom.xml index e0aeb7af559..bd2c2156ca6 100644 --- a/scala2.13/pom.xml +++ b/scala2.13/pom.xml @@ -1645,6 +1645,7 @@ This will force full Scala code rebuild in downstream modules. default, but there are some projects that are conditionally included. --> **/target/**/* **/cufile.log + **/cudf_log.txt thirdparty/parquet-testing/** From e3798d279eaef3453f60cfd3870325a296fedf8b Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Fri, 13 Dec 2024 13:22:06 -0600 Subject: [PATCH 07/47] Spill framework refactor for better performance and extensibility [databricks] (#11747) * Spill framework refactor for better performance and extensibility Signed-off-by: Alessandro Bellina --------- Signed-off-by: Alessandro Bellina --- .../nvidia/spark/rapids/GpuColumnVector.java | 4 + .../rapids/GpuColumnVectorFromBuffer.java | 25 +- .../rapids/GpuCompressedColumnVector.java | 11 +- .../InternalRowToColumnarBatchIterator.java | 13 +- .../com/nvidia/spark/rapids/implicits.scala | 54 - .../com/nvidia/spark/rapids/implicits.scala | 56 +- .../scala/com/nvidia/spark/rapids/Arm.scala | 14 - .../rapids/DeviceMemoryEventHandler.scala | 32 +- .../spark/rapids/GpuDeviceManager.scala | 29 +- .../com/nvidia/spark/rapids/HostAlloc.scala | 32 +- .../com/nvidia/spark/rapids/MetaUtils.scala | 9 +- .../nvidia/spark/rapids/RapidsBuffer.scala | 485 ----- .../spark/rapids/RapidsBufferCatalog.scala | 1005 ---------- .../spark/rapids/RapidsBufferStore.scala | 640 ------ .../rapids/RapidsDeviceMemoryStore.scala | 518 ----- .../nvidia/spark/rapids/RapidsDiskStore.scala | 256 --- .../spark/rapids/RapidsHostMemoryStore.scala | 484 ----- .../rapids/RapidsSerializerManager.scala | 39 +- .../spark/rapids/ShuffleBufferCatalog.scala | 273 +-- .../rapids/ShuffleReceivedBufferCatalog.scala | 128 +- .../spark/rapids/SpillableColumnarBatch.scala | 294 +-- .../rapids/shuffle/BufferSendState.scala | 90 +- .../rapids/shuffle/RapidsShuffleClient.scala | 20 +- .../rapids/shuffle/RapidsShuffleServer.scala | 6 +- .../spark/rapids/spill/SpillFramework.scala | 1743 +++++++++++++++++ .../spark/sql/rapids/GpuShuffleEnv.scala | 10 +- .../spark/sql/rapids/GpuTaskMetrics.scala | 1 - .../RapidsShuffleInternalManagerBase.scala | 37 +- .../spark/sql/rapids/TempSpillBufferId.scala | 50 - .../execution/GpuBroadcastExchangeExec.scala | 2 +- .../rapids/execution/GpuBroadcastHelper.scala | 5 +- .../shuffle/RapidsShuffleIterator.scala | 30 +- .../sql/rapids/RapidsCachingReader.scala | 73 +- .../shuffle/RapidsShuffleIterator.scala | 24 +- .../sql/rapids/RapidsCachingReader.scala | 69 +- .../nvidia/spark/rapids/HostAllocSuite.scala | 22 +- .../DeviceMemoryEventHandlerSuite.scala | 38 +- ...ternalRowToCudfRowIteratorRetrySuite.scala | 65 +- .../rapids/GpuCoalesceBatchesRetrySuite.scala | 4 +- .../spark/rapids/GpuGenerateSuite.scala | 4 +- .../spark/rapids/GpuPartitioningSuite.scala | 115 +- .../rapids/GpuSinglePartitioningSuite.scala | 6 +- .../rapids/HashAggregateRetrySuite.scala | 247 +-- .../rapids/RapidsBufferCatalogSuite.scala | 368 ---- .../rapids/RapidsDeviceMemoryStoreSuite.scala | 489 ----- .../spark/rapids/RapidsDiskStoreSuite.scala | 607 ------ .../rapids/RapidsHostMemoryStoreSuite.scala | 614 ------ .../spark/rapids/RmmSparkRetrySuiteBase.scala | 23 +- .../spark/rapids/SerializationSuite.scala | 37 +- .../rapids/ShuffleBufferCatalogSuite.scala | 77 +- .../spark/rapids/WindowRetrySuite.scala | 10 +- .../nvidia/spark/rapids/WithRetrySuite.scala | 11 +- .../shuffle/RapidsShuffleClientSuite.scala | 11 +- .../shuffle/RapidsShuffleIteratorSuite.scala | 95 +- .../shuffle/RapidsShuffleServerSuite.scala | 363 ++-- .../rapids/spill/SpillFrameworkSuite.scala | 1105 +++++++++++ .../rapids/timezone/TimeZonePerfSuite.scala | 2 + .../spark/rapids/timezone/TimeZoneSuite.scala | 3 +- .../rapids/GpuFileFormatDataWriterSuite.scala | 18 +- .../rapids/SpillableColumnarBatchSuite.scala | 63 +- .../shuffle/RapidsShuffleTestHelper.scala | 49 +- .../shuffle/RapidsShuffleTestHelper.scala | 47 +- 62 files changed, 4146 insertions(+), 6908 deletions(-) delete mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala delete mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala delete mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala delete mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStore.scala delete mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDiskStore.scala delete mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsHostMemoryStore.scala create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/spill/SpillFramework.scala delete mode 100644 sql-plugin/src/main/scala/org/apache/spark/sql/rapids/TempSpillBufferId.scala delete mode 100644 tests/src/test/scala/com/nvidia/spark/rapids/RapidsBufferCatalogSuite.scala delete mode 100644 tests/src/test/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStoreSuite.scala delete mode 100644 tests/src/test/scala/com/nvidia/spark/rapids/RapidsDiskStoreSuite.scala delete mode 100644 tests/src/test/scala/com/nvidia/spark/rapids/RapidsHostMemoryStoreSuite.scala create mode 100644 tests/src/test/scala/com/nvidia/spark/rapids/spill/SpillFrameworkSuite.scala 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 f2be4264162..b72a389d2a0 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 @@ -1101,6 +1101,10 @@ public final int numNulls() { public static long getTotalDeviceMemoryUsed(ColumnarBatch batch) { long sum = 0; + if (batch.numCols() == 1 && batch.column(0) instanceof GpuPackedTableColumn) { + // this is a special case for a packed batch + return ((GpuPackedTableColumn) batch.column(0)).getTableBuffer().getLength(); + } if (batch.numCols() > 0) { if (batch.column(0) instanceof WithTableBuffer) { WithTableBuffer wtb = (WithTableBuffer) batch.column(0); diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuColumnVectorFromBuffer.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuColumnVectorFromBuffer.java index e23fa76c9f3..b5ed621821b 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuColumnVectorFromBuffer.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuColumnVectorFromBuffer.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2021, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -113,6 +113,29 @@ public GpuColumnVectorFromBuffer(DataType type, ColumnVector cudfColumn, this.tableMeta = meta; } + public static boolean isFromBuffer(ColumnarBatch cb) { + if (cb.numCols() > 0) { + long bufferAddr = 0L; + boolean isSet = false; + for (int i = 0; i < cb.numCols(); ++i) { + GpuColumnVectorFromBuffer gcvfb = null; + if (!(cb.column(i) instanceof GpuColumnVectorFromBuffer)) { + return false; + } else { + gcvfb = (GpuColumnVectorFromBuffer) cb.column(i); + if (!isSet) { + bufferAddr = gcvfb.buffer.getAddress(); + isSet = true; + } else if (bufferAddr != gcvfb.buffer.getAddress()) { + return false; + } + } + } + return true; + } + return false; + } + /** * Get the underlying contiguous buffer, shared between columns of the original * `ContiguousTable` diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuCompressedColumnVector.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuCompressedColumnVector.java index cd34f35ecab..1dc85cb2031 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuCompressedColumnVector.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/GpuCompressedColumnVector.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2021, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -47,6 +47,15 @@ public static boolean isBatchCompressed(ColumnarBatch batch) { return batch.numCols() == 1 && batch.column(0) instanceof GpuCompressedColumnVector; } + public static ColumnarBatch incRefCounts(ColumnarBatch batch) { + if (!isBatchCompressed(batch)) { + throw new IllegalStateException( + "Attempted to incRefCount for a compressed batch, but the batch was not compressed."); + } + ((GpuCompressedColumnVector)batch.column(0)).buffer.incRefCount(); + return batch; + } + /** * Build a columnar batch from a compressed data buffer and specified table metadata * NOTE: The data remains compressed and cannot be accessed directly from the columnar batch. 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 0aa3f0978e9..400b54626d8 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,19 +21,11 @@ import java.util.NoSuchElementException; import java.util.Optional; -import com.nvidia.spark.Retryable; import scala.Option; import scala.Tuple2; import scala.collection.Iterator; -import ai.rapids.cudf.ColumnVector; -import ai.rapids.cudf.DType; -import ai.rapids.cudf.HostColumnVector; -import ai.rapids.cudf.HostColumnVectorCore; -import ai.rapids.cudf.HostMemoryBuffer; -import ai.rapids.cudf.NvtxColor; -import ai.rapids.cudf.NvtxRange; -import ai.rapids.cudf.Table; +import ai.rapids.cudf.*; import com.nvidia.spark.rapids.jni.RowConversion; import com.nvidia.spark.rapids.shims.CudfUnsafeRow; @@ -236,8 +228,7 @@ private HostMemoryBuffer[] getHostBuffersWithRetry( try { hBuf = HostAlloc$.MODULE$.alloc((dataBytes + offsetBytes),true); SpillableHostBuffer sBuf = SpillableHostBuffer$.MODULE$.apply(hBuf, hBuf.getLength(), - SpillPriorities$.MODULE$.ACTIVE_ON_DECK_PRIORITY(), - RapidsBufferCatalog$.MODULE$.singleton()); + SpillPriorities$.MODULE$.ACTIVE_ON_DECK_PRIORITY()); hBuf = null; // taken over by spillable host buffer return Tuple2.apply(sBuf, numRowsWrapper); } finally { diff --git a/sql-plugin/src/main/scala-2.12/com/nvidia/spark/rapids/implicits.scala b/sql-plugin/src/main/scala-2.12/com/nvidia/spark/rapids/implicits.scala index eddad69ba97..89e717788f9 100644 --- a/sql-plugin/src/main/scala-2.12/com/nvidia/spark/rapids/implicits.scala +++ b/sql-plugin/src/main/scala-2.12/com/nvidia/spark/rapids/implicits.scala @@ -63,26 +63,6 @@ object RapidsPluginImplicits { } } - implicit class RapidsBufferColumn(rapidsBuffer: RapidsBuffer) { - - /** - * safeFree: Is an implicit on RapidsBuffer class that tries to free the resource, if an - * Exception was thrown prior to this free, it adds the new exception to the suppressed - * exceptions, otherwise just throws - * - * @param e Exception which we don't want to suppress - */ - def safeFree(e: Throwable = null): Unit = { - if (rapidsBuffer != null) { - try { - rapidsBuffer.free() - } catch { - case suppressed: Throwable if e != null => e.addSuppressed(suppressed) - } - } - } - } - implicit class AutoCloseableSeq[A <: AutoCloseable](val in: collection.SeqLike[A, _]) { /** * safeClose: Is an implicit on a sequence of AutoCloseable classes that tries to close each @@ -111,46 +91,12 @@ object RapidsPluginImplicits { } } - implicit class RapidsBufferSeq[A <: RapidsBuffer](val in: collection.SeqLike[A, _]) { - /** - * safeFree: Is an implicit on a sequence of RapidsBuffer classes that tries to free each - * element of the sequence, even if prior free calls fail. In case of failure in any of the - * free calls, an Exception is thrown containing the suppressed exceptions (getSuppressed), - * if any. - */ - def safeFree(error: Throwable = null): Unit = if (in != null) { - var freeException: Throwable = null - in.foreach { element => - if (element != null) { - try { - element.free() - } catch { - case e: Throwable if error != null => error.addSuppressed(e) - case e: Throwable if freeException == null => freeException = e - case e: Throwable => freeException.addSuppressed(e) - } - } - } - if (freeException != null) { - // an exception happened while we were trying to safely free - // resources, throw the exception to alert the caller - throw freeException - } - } - } - implicit class AutoCloseableArray[A <: AutoCloseable](val in: Array[A]) { def safeClose(e: Throwable = null): Unit = if (in != null) { in.toSeq.safeClose(e) } } - implicit class RapidsBufferArray[A <: RapidsBuffer](val in: Array[A]) { - def safeFree(e: Throwable = null): Unit = if (in != null) { - in.toSeq.safeFree(e) - } - } - class MapsSafely[A, Repr] { /** * safeMap: safeMap implementation that is leveraged by other type-specific implicits. diff --git a/sql-plugin/src/main/scala-2.13/com/nvidia/spark/rapids/implicits.scala b/sql-plugin/src/main/scala-2.13/com/nvidia/spark/rapids/implicits.scala index 5bdded6dbd4..1e4c5e39a19 100644 --- a/sql-plugin/src/main/scala-2.13/com/nvidia/spark/rapids/implicits.scala +++ b/sql-plugin/src/main/scala-2.13/com/nvidia/spark/rapids/implicits.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -63,26 +63,6 @@ object RapidsPluginImplicits { } } - implicit class RapidsBufferColumn(rapidsBuffer: RapidsBuffer) { - - /** - * safeFree: Is an implicit on RapidsBuffer class that tries to free the resource, if an - * Exception was thrown prior to this free, it adds the new exception to the suppressed - * exceptions, otherwise just throws - * - * @param e Exception which we don't want to suppress - */ - def safeFree(e: Throwable = null): Unit = { - if (rapidsBuffer != null) { - try { - rapidsBuffer.free() - } catch { - case suppressed: Throwable if e != null => e.addSuppressed(suppressed) - } - } - } - } - implicit class AutoCloseableSeq[A <: AutoCloseable](val in: collection.Iterable[A]) { /** * safeClose: Is an implicit on a sequence of AutoCloseable classes that tries to close each @@ -111,46 +91,12 @@ object RapidsPluginImplicits { } } - implicit class RapidsBufferSeq[A <: RapidsBuffer](val in: collection.SeqLike[A, _]) { - /** - * safeFree: Is an implicit on a sequence of RapidsBuffer classes that tries to free each - * element of the sequence, even if prior free calls fail. In case of failure in any of the - * free calls, an Exception is thrown containing the suppressed exceptions (getSuppressed), - * if any. - */ - def safeFree(error: Throwable = null): Unit = if (in != null) { - var freeException: Throwable = null - in.foreach { element => - if (element != null) { - try { - element.free() - } catch { - case e: Throwable if error != null => error.addSuppressed(e) - case e: Throwable if freeException == null => freeException = e - case e: Throwable => freeException.addSuppressed(e) - } - } - } - if (freeException != null) { - // an exception happened while we were trying to safely free - // resources, throw the exception to alert the caller - throw freeException - } - } - } - implicit class AutoCloseableArray[A <: AutoCloseable](val in: Array[A]) { def safeClose(e: Throwable = null): Unit = if (in != null) { in.toSeq.safeClose(e) } } - implicit class RapidsBufferArray[A <: RapidsBuffer](val in: Array[A]) { - def safeFree(e: Throwable = null): Unit = if (in != null) { - in.toSeq.safeFree(e) - } - } - class IterableMapsSafely[A, From[A] <: collection.Iterable[A] with collection.IterableOps[A, From, _]] { /** diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Arm.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Arm.scala index b0cd798c179..fcf65e1bc00 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Arm.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Arm.scala @@ -177,20 +177,6 @@ object Arm extends ArmScalaSpecificImpl { } } - /** Executes the provided code block, freeing the RapidsBuffer only if an exception occurs */ - def freeOnExcept[T <: RapidsBuffer, V](r: T)(block: T => V): V = { - try { - block(r) - } catch { - case t: ControlThrowable => - // Don't close for these cases.. - throw t - case t: Throwable => - r.safeFree(t) - throw t - } - } - /** Executes the provided code block and then closes the resource */ def withResource[T <: AutoCloseable, V](h: CloseableHolder[T]) (block: CloseableHolder[T] => V): V = { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandler.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandler.scala index 72808d1f376..9c867bb6a90 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandler.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandler.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,7 @@ import java.lang.management.ManagementFactory import java.util.concurrent.atomic.AtomicLong import ai.rapids.cudf.{Cuda, Rmm, RmmEventHandler} +import com.nvidia.spark.rapids.spill.SpillableDeviceStore import com.sun.management.HotSpotDiagnosticMXBean import org.apache.spark.internal.Logging @@ -34,8 +35,7 @@ import org.apache.spark.sql.rapids.execution.TrampolineUtil * depleting the device store */ class DeviceMemoryEventHandler( - catalog: RapidsBufferCatalog, - store: RapidsDeviceMemoryStore, + store: SpillableDeviceStore, oomDumpDir: Option[String], maxFailedOOMRetries: Int) extends RmmEventHandler with Logging { @@ -92,8 +92,8 @@ class DeviceMemoryEventHandler( * from cuDF. If we succeed, cuDF resets `retryCount`, and so the new count sent to us * must be <= than what we saw last, so we can reset our tracking. */ - def resetIfNeeded(retryCount: Int, storeSpillableSize: Long): Unit = { - if (storeSpillableSize != 0 || retryCount <= retryCountLastSynced) { + def resetIfNeeded(retryCount: Int, couldSpill: Boolean): Unit = { + if (couldSpill || retryCount <= retryCountLastSynced) { reset() } } @@ -114,9 +114,6 @@ class DeviceMemoryEventHandler( s"onAllocFailure invoked with invalid retryCount $retryCount") try { - val storeSize = store.currentSize - val storeSpillableSize = store.currentSpillableSize - val attemptMsg = if (retryCount > 0) { s"Attempt ${retryCount}. " } else { @@ -124,12 +121,13 @@ class DeviceMemoryEventHandler( } val retryState = oomRetryState.get() - retryState.resetIfNeeded(retryCount, storeSpillableSize) - logInfo(s"Device allocation of $allocSize bytes failed, device store has " + - s"$storeSize total and $storeSpillableSize spillable bytes. $attemptMsg" + - s"Total RMM allocated is ${Rmm.getTotalBytesAllocated} bytes. ") - if (storeSpillableSize == 0) { + val amountSpilled = store.spill(allocSize) + retryState.resetIfNeeded(retryCount, amountSpilled > 0) + logInfo(s"Device allocation of $allocSize bytes failed. " + + s"Device store spilled $amountSpilled bytes. $attemptMsg" + + s"Total RMM allocated is ${Rmm.getTotalBytesAllocated} bytes.") + if (amountSpilled == 0) { if (retryState.shouldTrySynchronizing(retryCount)) { Cuda.deviceSynchronize() logWarning(s"[RETRY ${retryState.getRetriesSoFar}] " + @@ -149,13 +147,7 @@ class DeviceMemoryEventHandler( false } } else { - val targetSize = Math.max(storeSpillableSize - allocSize, 0) - logDebug(s"Targeting device store size of $targetSize bytes") - val maybeAmountSpilled = catalog.synchronousSpill(store, targetSize, Cuda.DEFAULT_STREAM) - maybeAmountSpilled.foreach { amountSpilled => - logInfo(s"Spilled $amountSpilled bytes from the device store") - TrampolineUtil.incTaskMetricsMemoryBytesSpilled(amountSpilled) - } + TrampolineUtil.incTaskMetricsMemoryBytesSpilled(amountSpilled) true } } catch { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDeviceManager.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDeviceManager.scala index b0c86773166..42776a6cab0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDeviceManager.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDeviceManager.scala @@ -22,6 +22,8 @@ import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal import ai.rapids.cudf._ +import com.nvidia.spark.rapids.jni.RmmSpark +import com.nvidia.spark.rapids.spill.SpillFramework import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.internal.Logging @@ -169,7 +171,9 @@ object GpuDeviceManager extends Logging { chunkedPackMemoryResource = None poolSizeLimit = 0L - RapidsBufferCatalog.close() + SpillFramework.shutdown() + RmmSpark.clearEventHandler() + Rmm.clearEventHandler() GpuShuffleEnv.shutdown() // try to avoid segfault on RMM shutdown val timeout = System.nanoTime() + TimeUnit.SECONDS.toNanos(10) @@ -278,6 +282,8 @@ object GpuDeviceManager extends Logging { } } + private var memoryEventHandler: DeviceMemoryEventHandler = _ + private def initializeRmm(gpuId: Int, rapidsConf: Option[RapidsConf]): Unit = { if (!Rmm.isInitialized) { val conf = rapidsConf.getOrElse(new RapidsConf(SparkEnv.get.conf)) @@ -385,8 +391,25 @@ object GpuDeviceManager extends Logging { } } - RapidsBufferCatalog.init(conf) - GpuShuffleEnv.init(conf, RapidsBufferCatalog.getDiskBlockManager()) + SpillFramework.initialize(conf) + + memoryEventHandler = new DeviceMemoryEventHandler( + SpillFramework.stores.deviceStore, + conf.gpuOomDumpDir, + conf.gpuOomMaxRetries) + + if (conf.sparkRmmStateEnable) { + val debugLoc = if (conf.sparkRmmDebugLocation.isEmpty) { + null + } else { + conf.sparkRmmDebugLocation + } + RmmSpark.setEventHandler(memoryEventHandler, debugLoc) + } else { + logWarning("SparkRMM retry has been disabled") + Rmm.setEventHandler(memoryEventHandler) + } + GpuShuffleEnv.init(conf) } } 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 6a34d15dc6e..6079c0352df 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 @@ -18,6 +18,7 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.{DefaultHostMemoryAllocator, HostMemoryAllocator, HostMemoryBuffer, MemoryBuffer, PinnedMemoryPool} import com.nvidia.spark.rapids.jni.{CpuRetryOOM, RmmSpark} +import com.nvidia.spark.rapids.spill.SpillFramework import org.apache.spark.TaskContext import org.apache.spark.internal.Logging @@ -137,9 +138,7 @@ private class HostAlloc(nonPinnedLimit: Long) extends HostMemoryAllocator with L require(retryCount >= 0, s"spillAndCheckRetry invoked with invalid retryCount $retryCount") - val store = RapidsBufferCatalog.getHostStorage - val storeSize = store.currentSize - val storeSpillableSize = store.currentSpillableSize + val store = SpillFramework.stores.hostStore val totalSize: Long = synchronized { currentPinnedAllocated + currentNonPinnedAllocated } @@ -150,21 +149,20 @@ private class HostAlloc(nonPinnedLimit: Long) extends HostMemoryAllocator with L "First attempt" } - logInfo(s"Host allocation of $allocSize bytes failed, host store has " + - s"$storeSize total and $storeSpillableSize spillable bytes. $attemptMsg.") - if (storeSpillableSize == 0) { - logWarning(s"Host store exhausted, unable to allocate $allocSize bytes. " + - s"Total host allocated is $totalSize bytes.") - false - } else { - val targetSize = Math.max(storeSpillableSize - allocSize, 0) - 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, targetSize) - maybeAmountSpilled.foreach { amountSpilled => - logInfo(s"Spilled $amountSpilled bytes from the host store") + val amountSpilled = store.spill(allocSize) + + if (amountSpilled == 0) { + val shouldRetry = store.numHandles > 0 + val exhaustedMsg = s"Host store exhausted, unable to allocate $allocSize bytes. " + + s"Total host allocated is $totalSize bytes. $attemptMsg." + if (!shouldRetry) { + logWarning(exhaustedMsg) + } else { + logWarning(s"$exhaustedMsg Attempting a retry.") } + shouldRetry + } else { + logInfo(s"Spilled $amountSpilled bytes from the host store") true } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/MetaUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/MetaUtils.scala index 80acddcb257..f1561e2c251 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/MetaUtils.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/MetaUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -51,6 +51,13 @@ object MetaUtils { ct.getMetadataDirectBuffer, ct.getRowCount) + def buildTableMeta(tableId: Int, compressed: GpuCompressedColumnVector): TableMeta = + buildTableMeta( + tableId, + compressed.getTableBuffer.getLength, + compressed.getTableMeta.bufferMeta().getByteBuffer, + compressed.getTableMeta.rowCount()) + def buildTableMeta(tableId: Int, bufferSize: Long, packedMeta: ByteBuffer, rowCount: Long): TableMeta = { val fbb = new FlatBufferBuilder(1024) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala deleted file mode 100644 index a332755745f..00000000000 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala +++ /dev/null @@ -1,485 +0,0 @@ -/* - * 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. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids - -import java.io.File -import java.nio.channels.WritableByteChannel - -import scala.collection.mutable.ArrayBuffer - -import ai.rapids.cudf.{Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer, Table} -import com.nvidia.spark.rapids.Arm.withResource -import com.nvidia.spark.rapids.RapidsPluginImplicits._ -import com.nvidia.spark.rapids.StorageTier.StorageTier -import com.nvidia.spark.rapids.format.TableMeta - -import org.apache.spark.internal.Logging -import org.apache.spark.sql.rapids.RapidsDiskBlockManager -import org.apache.spark.sql.types.DataType -import org.apache.spark.sql.vectorized.ColumnarBatch - -/** - * An identifier for a RAPIDS buffer that can be automatically spilled between buffer stores. - * NOTE: Derived classes MUST implement proper hashCode and equals methods, as these objects are - * used as keys in hash maps. Scala case classes are recommended. - */ -trait RapidsBufferId { - val tableId: Int - - /** - * Indicates whether the buffer may share a spill file with other buffers. - * If false then the spill file will be automatically removed when the buffer is freed. - * If true then the spill file will not be automatically removed, and another subsystem needs - * to be responsible for cleaning up the spill files for those types of buffers. - */ - val canShareDiskPaths: Boolean = false - - /** - * Generate a path to a local file that can be used to spill the corresponding buffer to disk. - * The path must be unique across all buffers unless canShareDiskPaths is true. - */ - def getDiskPath(diskBlockManager: RapidsDiskBlockManager): File -} - -/** Enumeration of the storage tiers */ -object StorageTier extends Enumeration { - type StorageTier = Value - val DEVICE: StorageTier = Value(0, "device memory") - val HOST: StorageTier = Value(1, "host memory") - val DISK: StorageTier = Value(2, "local disk") -} - -/** - * ChunkedPacker is an Iterator that uses a cudf::chunked_pack to copy a cuDF `Table` - * to a target buffer in chunks. - * - * Each chunk is sized at most `bounceBuffer.getLength`, and the caller should cudaMemcpy - * bytes from `bounceBuffer` to a target buffer after each call to `next()`. - * - * @note `ChunkedPacker` must be closed by the caller as it has GPU and host resources - * associated with it. - * - * @param id The RapidsBufferId for this pack operation to be included in the metadata - * @param table cuDF Table to chunk_pack - * @param bounceBuffer GPU memory to be used for packing. The buffer should be at least 1MB - * in length. - */ -class ChunkedPacker( - id: RapidsBufferId, - table: Table, - bounceBuffer: DeviceMemoryBuffer) - extends Iterator[MemoryBuffer] - with Logging - with AutoCloseable { - - private var closed: Boolean = false - - // When creating cudf::chunked_pack use a pool if available, otherwise default to the - // per-device memory resource - private val chunkedPack = { - val pool = GpuDeviceManager.chunkedPackMemoryResource - val cudfChunkedPack = try { - pool.flatMap { chunkedPool => - Some(table.makeChunkedPack(bounceBuffer.getLength, chunkedPool)) - } - } catch { - case _: OutOfMemoryError => - if (!ChunkedPacker.warnedAboutPoolFallback) { - ChunkedPacker.warnedAboutPoolFallback = true - logWarning( - s"OOM while creating chunked_pack using pool sized ${pool.map(_.getMaxSize)}B. " + - "Falling back to the per-device memory resource.") - } - None - } - - // if the pool is not configured, or we got an OOM, try again with the per-device pool - cudfChunkedPack.getOrElse { - table.makeChunkedPack(bounceBuffer.getLength) - } - } - - private val tableMeta = withResource(chunkedPack.buildMetadata()) { packedMeta => - MetaUtils.buildTableMeta( - id.tableId, - chunkedPack.getTotalContiguousSize, - packedMeta.getMetadataDirectBuffer, - table.getRowCount) - } - - // take out a lease on the bounce buffer - bounceBuffer.incRefCount() - - def getTotalContiguousSize: Long = chunkedPack.getTotalContiguousSize - - def getMeta: TableMeta = { - tableMeta - } - - override def hasNext: Boolean = synchronized { - if (closed) { - throw new IllegalStateException(s"ChunkedPacker for $id is closed") - } - chunkedPack.hasNext - } - - def next(): MemoryBuffer = synchronized { - if (closed) { - throw new IllegalStateException(s"ChunkedPacker for $id is closed") - } - val bytesWritten = chunkedPack.next(bounceBuffer) - // we increment the refcount because the caller has no idea where - // this memory came from, so it should close it. - bounceBuffer.slice(0, bytesWritten) - } - - override def close(): Unit = synchronized { - if (!closed) { - closed = true - val toClose = new ArrayBuffer[AutoCloseable]() - toClose.append(chunkedPack, bounceBuffer) - toClose.safeClose() - } - } -} - -object ChunkedPacker { - private var warnedAboutPoolFallback: Boolean = false -} - -/** - * This iterator encapsulates a buffer's internal `MemoryBuffer` access - * for spill reasons. Internally, there are two known implementations: - * - either this is a "single shot" copy, where the entirety of the `RapidsBuffer` is - * already represented as a single contiguous blob of memory, then the expectation - * is that this iterator is exhausted with a single call to `next` - * - or, we have a `RapidsBuffer` that isn't contiguous. This iteration will then - * drive a `ChunkedPacker` to pack the `RapidsBuffer`'s table as needed. The - * iterator will likely need several calls to `next` to be exhausted. - * - * @param buffer `RapidsBuffer` to copy out of its tier. - */ -class RapidsBufferCopyIterator(buffer: RapidsBuffer) - extends Iterator[MemoryBuffer] with AutoCloseable with Logging { - - private val chunkedPacker: Option[ChunkedPacker] = if (buffer.supportsChunkedPacker) { - Some(buffer.makeChunkedPacker) - } else { - None - } - def isChunked: Boolean = chunkedPacker.isDefined - - // this is used for the single shot case to flag when `next` is call - // to satisfy the Iterator interface - private var singleShotCopyHasNext: Boolean = false - private var singleShotBuffer: MemoryBuffer = _ - - if (!isChunked) { - singleShotCopyHasNext = true - singleShotBuffer = buffer.getMemoryBuffer - } - - override def hasNext: Boolean = - chunkedPacker.map(_.hasNext).getOrElse(singleShotCopyHasNext) - - override def next(): MemoryBuffer = { - require(hasNext, - "next called on exhausted iterator") - chunkedPacker.map(_.next()).getOrElse { - singleShotCopyHasNext = false - singleShotBuffer.slice(0, singleShotBuffer.getLength) - } - } - - def getTotalCopySize: Long = { - chunkedPacker - .map(_.getTotalContiguousSize) - .getOrElse(singleShotBuffer.getLength) - } - - override def close(): Unit = { - val toClose = new ArrayBuffer[AutoCloseable]() - toClose.appendAll(chunkedPacker) - toClose.appendAll(Option(singleShotBuffer)) - - toClose.safeClose() - } -} - -/** Interface provided by all types of RAPIDS buffers */ -trait RapidsBuffer extends AutoCloseable { - /** The buffer identifier for this buffer. */ - val id: RapidsBufferId - - /** - * The size of this buffer in bytes in its _current_ store. As the buffer goes through - * contiguous split (either added as a contiguous table already, or spilled to host), - * its size changes because contiguous_split adds its own alignment padding. - * - * @note Do not use this size to allocate a target buffer to copy, always use `getPackedSize.` - */ - val memoryUsedBytes: Long - - /** - * The size of this buffer if it has already gone through contiguous_split. - * - * @note Use this function when allocating a target buffer for spill or shuffle purposes. - */ - def getPackedSizeBytes: Long = memoryUsedBytes - - /** - * At spill time, obtain an iterator used to copy this buffer to a different tier. - */ - def getCopyIterator: RapidsBufferCopyIterator = - new RapidsBufferCopyIterator(this) - - /** Descriptor for how the memory buffer is formatted */ - def meta: TableMeta - - /** The storage tier for this buffer */ - val storageTier: StorageTier - - /** - * Get the columnar batch within this buffer. The caller must have - * successfully acquired the buffer beforehand. - * @param sparkTypes the spark data types the batch should have - * @see [[addReference]] - * @note It is the responsibility of the caller to close the batch. - * @note If the buffer is compressed data then the resulting batch will be built using - * `GpuCompressedColumnVector`, and it is the responsibility of the caller to deal - * with decompressing the data if necessary. - */ - def getColumnarBatch(sparkTypes: Array[DataType]): ColumnarBatch - - /** - * Get the host-backed columnar batch from this buffer. The caller must have - * successfully acquired the buffer beforehand. - * - * If this `RapidsBuffer` was added originally to the device tier, or if this is - * a just a buffer (not a batch), this function will throw. - * - * @param sparkTypes the spark data types the batch should have - * @see [[addReference]] - * @note It is the responsibility of the caller to close the batch. - */ - def getHostColumnarBatch(sparkTypes: Array[DataType]): ColumnarBatch = { - throw new IllegalStateException(s"$this does not support host columnar batches.") - } - - /** - * Get the underlying memory buffer. This may be either a HostMemoryBuffer or a DeviceMemoryBuffer - * depending on where the buffer currently resides. - * The caller must have successfully acquired the buffer beforehand. - * @see [[addReference]] - * @note It is the responsibility of the caller to close the buffer. - */ - def getMemoryBuffer: MemoryBuffer - - val supportsChunkedPacker: Boolean = false - - /** - * Makes a new chunked packer. It is the responsibility of the caller to close this. - */ - def makeChunkedPacker: ChunkedPacker = { - throw new NotImplementedError("not implemented for this store") - } - - /** - * Copy the content of this buffer into the specified memory buffer, starting from the given - * offset. - * - * @param srcOffset offset to start copying from. - * @param dst the memory buffer to copy into. - * @param dstOffset offset to copy into. - * @param length number of bytes to copy. - * @param stream CUDA stream to use - */ - def copyToMemoryBuffer( - srcOffset: Long, dst: MemoryBuffer, dstOffset: Long, length: Long, stream: Cuda.Stream): Unit - - /** - * Get the device memory buffer from the underlying storage. If the buffer currently resides - * outside of device memory, a new DeviceMemoryBuffer is created with the data copied over. - * The caller must have successfully acquired the buffer beforehand. - * @see [[addReference]] - * @note It is the responsibility of the caller to close the buffer. - */ - def getDeviceMemoryBuffer: DeviceMemoryBuffer - - /** - * Get the host memory buffer from the underlying storage. If the buffer currently resides - * outside of host memory, a new HostMemoryBuffer is created with the data copied over. - * The caller must have successfully acquired the buffer beforehand. - * @see [[addReference]] - * @note It is the responsibility of the caller to close the buffer. - */ - def getHostMemoryBuffer: HostMemoryBuffer - - /** - * Try to add a reference to this buffer to acquire it. - * @note The close method must be called for every successfully obtained reference. - * @return true if the reference was added or false if this buffer is no longer valid - */ - def addReference(): Boolean - - /** - * Schedule the release of the buffer's underlying resources. - * Subsequent attempts to acquire the buffer will fail. As soon as the - * buffer has no outstanding references, the resources will be released. - *

- * This is separate from the close method which does not normally release - * resources. close will only release resources if called as the last - * outstanding reference and the buffer was previously marked as freed. - */ - def free(): Unit - - /** - * Get the spill priority value for this buffer. Lower values are higher - * priority for spilling, meaning buffers with lower values will be - * preferred for spilling over buffers with a higher value. - */ - def getSpillPriority: Long - - /** - * Set the spill priority for this buffer. Lower values are higher priority - * for spilling, meaning buffers with lower values will be preferred for - * spilling over buffers with a higher value. - * @note should only be called from the buffer catalog - * @param priority new priority value for this buffer - */ - def setSpillPriority(priority: Long): Unit - - /** - * Function invoked by the `RapidsBufferStore.addBuffer` method that prompts - * the specific `RapidsBuffer` to check its reference counting to make itself - * spillable or not. Only `RapidsTable` and `RapidsHostMemoryBuffer` implement - * this method. - */ - def updateSpillability(): Unit = {} - - /** - * Obtains a read lock on this instance of `RapidsBuffer` and calls the function - * in `body` while holding the lock. - * @param body function that takes a `MemoryBuffer` and produces `K` - * @tparam K any return type specified by `body` - * @return the result of body(memoryBuffer) - */ - def withMemoryBufferReadLock[K](body: MemoryBuffer => K): K - - /** - * Obtains a write lock on this instance of `RapidsBuffer` and calls the function - * in `body` while holding the lock. - * @param body function that takes a `MemoryBuffer` and produces `K` - * @tparam K any return type specified by `body` - * @return the result of body(memoryBuffer) - */ - def withMemoryBufferWriteLock[K](body: MemoryBuffer => K): K -} - -/** - * A buffer with no corresponding device data (zero rows or columns). - * These buffers are not tracked in buffer stores since they have no - * device memory. They are only tracked in the catalog and provide - * a representative `ColumnarBatch` but cannot provide a - * `MemoryBuffer`. - * @param id buffer ID to associate with the buffer - * @param meta schema metadata - */ -sealed class DegenerateRapidsBuffer( - override val id: RapidsBufferId, - override val meta: TableMeta) extends RapidsBuffer { - - override val memoryUsedBytes: Long = 0L - - override val storageTier: StorageTier = StorageTier.DEVICE - - override def getColumnarBatch(sparkTypes: Array[DataType]): ColumnarBatch = { - val rowCount = meta.rowCount - val packedMeta = meta.packedMetaAsByteBuffer() - if (packedMeta != null) { - withResource(DeviceMemoryBuffer.allocate(0)) { deviceBuffer => - withResource(Table.fromPackedTable(meta.packedMetaAsByteBuffer(), deviceBuffer)) { table => - GpuColumnVectorFromBuffer.from(table, deviceBuffer, meta, sparkTypes) - } - } - } else { - // no packed metadata, must be a table with zero columns - new ColumnarBatch(Array.empty, rowCount.toInt) - } - } - - override def free(): Unit = {} - - override def getMemoryBuffer: MemoryBuffer = - throw new UnsupportedOperationException("degenerate buffer has no memory buffer") - - override def copyToMemoryBuffer(srcOffset: Long, dst: MemoryBuffer, dstOffset: Long, length: Long, - stream: Cuda.Stream): Unit = - throw new UnsupportedOperationException("degenerate buffer cannot copy to memory buffer") - - override def getDeviceMemoryBuffer: DeviceMemoryBuffer = - throw new UnsupportedOperationException("degenerate buffer has no device memory buffer") - - override def getHostMemoryBuffer: HostMemoryBuffer = - throw new UnsupportedOperationException("degenerate buffer has no host memory buffer") - - override def addReference(): Boolean = true - - override def getSpillPriority: Long = Long.MaxValue - - override def setSpillPriority(priority: Long): Unit = {} - - override def withMemoryBufferReadLock[K](body: MemoryBuffer => K): K = { - throw new UnsupportedOperationException("degenerate buffer has no memory buffer") - } - - override def withMemoryBufferWriteLock[K](body: MemoryBuffer => K): K = { - throw new UnsupportedOperationException("degenerate buffer has no memory buffer") - } - - override def close(): Unit = {} -} - -trait RapidsHostBatchBuffer extends AutoCloseable { - /** - * Get the host-backed columnar batch from this buffer. The caller must have - * successfully acquired the buffer beforehand. - * - * If this `RapidsBuffer` was added originally to the device tier, or if this is - * a just a buffer (not a batch), this function will throw. - * - * @param sparkTypes the spark data types the batch should have - * @see [[addReference]] - * @note It is the responsibility of the caller to close the batch. - */ - def getHostColumnarBatch(sparkTypes: Array[DataType]): ColumnarBatch - - val memoryUsedBytes: Long -} - -trait RapidsBufferChannelWritable { - /** - * At spill time, write this buffer to an nio WritableByteChannel. - * @param writableChannel that this buffer can just write itself to, either byte-for-byte - * or via serialization if needed. - * @param stream the Cuda.Stream for the spilling thread. If the `RapidsBuffer` that - * implements this method is on the device, synchronization may be needed - * for staged copies. - * @return the amount of bytes written to the channel - */ - def writeToChannel(writableChannel: WritableByteChannel, stream: Cuda.Stream): Long -} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala deleted file mode 100644 index f61291a31ce..00000000000 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala +++ /dev/null @@ -1,1005 +0,0 @@ -/* - * Copyright (c) 2020-2024, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids - -import java.util.concurrent.ConcurrentHashMap -import java.util.function.BiFunction - -import scala.collection.JavaConverters.collectionAsScalaIterableConverter - -import ai.rapids.cudf.{ContiguousTable, Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer, Rmm, Table} -import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} -import com.nvidia.spark.rapids.RapidsBufferCatalog.getExistingRapidsBufferAndAcquire -import com.nvidia.spark.rapids.RapidsPluginImplicits._ -import com.nvidia.spark.rapids.StorageTier.StorageTier -import com.nvidia.spark.rapids.format.TableMeta -import com.nvidia.spark.rapids.jni.RmmSpark - -import org.apache.spark.{SparkConf, SparkEnv} -import org.apache.spark.internal.Logging -import org.apache.spark.sql.rapids.{RapidsDiskBlockManager, TempSpillBufferId} -import org.apache.spark.sql.vectorized.ColumnarBatch - -/** - * Exception thrown when inserting a buffer into the catalog with a duplicate buffer ID - * and storage tier combination. - */ -class DuplicateBufferException(s: String) extends RuntimeException(s) {} - -/** - * An object that client code uses to interact with an underlying RapidsBufferId. - * - * A handle is obtained when a buffer, batch, or table is added to the spill framework - * via the `RapidsBufferCatalog` api. - */ -trait RapidsBufferHandle extends AutoCloseable { - val id: RapidsBufferId - - /** - * Sets the spill priority for this handle and updates the maximum priority - * for the underlying `RapidsBuffer` if this new priority is the maximum. - * @param newPriority new priority for this handle - */ - def setSpillPriority(newPriority: Long): Unit -} - -/** - * Catalog for lookup of buffers by ID. The constructor is only visible for testing, generally - * `RapidsBufferCatalog.singleton` should be used instead. - */ -class RapidsBufferCatalog( - deviceStorage: RapidsDeviceMemoryStore = RapidsBufferCatalog.deviceStorage, - hostStorage: RapidsHostMemoryStore = RapidsBufferCatalog.hostStorage) - extends AutoCloseable with Logging { - - /** Map of buffer IDs to buffers sorted by storage tier */ - private[this] val bufferMap = new ConcurrentHashMap[RapidsBufferId, Seq[RapidsBuffer]] - - /** Map of buffer IDs to buffer handles in insertion order */ - private[this] val bufferIdToHandles = - new ConcurrentHashMap[RapidsBufferId, Seq[RapidsBufferHandleImpl]]() - - /** A counter used to skip a spill attempt if we detect a different thread has spilled */ - @volatile private[this] var spillCount: Integer = 0 - - class RapidsBufferHandleImpl( - override val id: RapidsBufferId, - var priority: Long) - extends RapidsBufferHandle { - - private var closed = false - - override def toString: String = - s"buffer handle $id at $priority" - - override def setSpillPriority(newPriority: Long): Unit = { - priority = newPriority - updateUnderlyingRapidsBuffer(this) - } - - /** - * Get the spill priority that was associated with this handle. Since there can - * be multiple handles associated with one `RapidsBuffer`, the priority returned - * here is only useful for code in the catalog that updates the maximum priority - * for the underlying `RapidsBuffer` as handles are added and removed. - * - * @return this handle's spill priority - */ - def getSpillPriority: Long = priority - - override def close(): Unit = synchronized { - // since the handle is stored in the catalog in addition to being - // handed out to potentially a `SpillableColumnarBatch` or `SpillableBuffer` - // there is a chance we may double close it. For example, a broadcast exec - // that is closing its spillable (and therefore the handle) + the handle being - // closed from the catalog's close method. - if (!closed) { - removeBuffer(this) - } - closed = true - } - } - - /** - * Makes a new `RapidsBufferHandle` associated with `id`, keeping track - * of the spill priority and callback within this handle. - * - * This function also adds the handle for internal tracking in the catalog. - * - * @param id the `RapidsBufferId` that this handle refers to - * @param spillPriority the spill priority specified on creation of the handle - * @note public for testing - * @return a new instance of `RapidsBufferHandle` - */ - def makeNewHandle( - id: RapidsBufferId, - spillPriority: Long): RapidsBufferHandle = { - val handle = new RapidsBufferHandleImpl(id, spillPriority) - trackNewHandle(handle) - handle - } - - /** - * Adds a handle to the internal `bufferIdToHandles` map. - * - * The priority and callback of the `RapidsBuffer` will also be updated. - * - * @param handle handle to start tracking - */ - private def trackNewHandle(handle: RapidsBufferHandleImpl): Unit = { - bufferIdToHandles.compute(handle.id, (_, h) => { - var handles = h - if (handles == null) { - handles = Seq.empty[RapidsBufferHandleImpl] - } - handles :+ handle - }) - updateUnderlyingRapidsBuffer(handle) - } - - /** - * Called when the `RapidsBufferHandle` is no longer needed by calling code - * - * If this is the last handle associated with a `RapidsBuffer`, `stopTrackingHandle` - * returns true, otherwise it returns false. - * - * @param handle handle to stop tracking - * @return true: if this was the last `RapidsBufferHandle` associated with the - * underlying buffer. - * false: if there are remaining live handles - */ - private def stopTrackingHandle(handle: RapidsBufferHandle): Boolean = { - withResource(acquireBuffer(handle)) { buffer => - val id = handle.id - var maxPriority = Long.MinValue - val newHandles = bufferIdToHandles.compute(id, (_, handles) => { - if (handles == null) { - throw new IllegalStateException( - s"$id not found and we attempted to remove handles!") - } - if (handles.size == 1) { - require(handles.head == handle, - "Tried to remove a single handle, and we couldn't match on it") - null - } else { - val newHandles = handles.filter(h => h != handle).map { h => - maxPriority = maxPriority.max(h.getSpillPriority) - h - } - if (newHandles.isEmpty) { - null // remove since no more handles exist, should not happen - } else { - newHandles - } - } - }) - - if (newHandles == null) { - // tell calling code that no more handles exist, - // for this RapidsBuffer - true - } else { - // more handles remain, our priority changed so we need to update things - buffer.setSpillPriority(maxPriority) - false // we have handles left - } - } - } - - /** - * Adds a buffer to the catalog and store. This does NOT take ownership of the - * buffer, so it is the responsibility of the caller to close it. - * - * This version of `addBuffer` should not be called from the shuffle catalogs - * since they provide their own ids. - * - * @param buffer buffer that will be owned by the store - * @param tableMeta metadata describing the buffer layout - * @param initialSpillPriority starting spill priority value for the buffer - * @param needsSync whether the spill framework should stream synchronize while adding - * this device buffer (defaults to true) - * @return RapidsBufferHandle handle for this buffer - */ - def addBuffer( - buffer: MemoryBuffer, - tableMeta: TableMeta, - initialSpillPriority: Long, - needsSync: Boolean = true): RapidsBufferHandle = synchronized { - // first time we see `buffer` - val existing = getExistingRapidsBufferAndAcquire(buffer) - existing match { - case None => - addBuffer( - TempSpillBufferId(), - buffer, - tableMeta, - initialSpillPriority, - needsSync) - case Some(rapidsBuffer) => - withResource(rapidsBuffer) { _ => - makeNewHandle(rapidsBuffer.id, initialSpillPriority) - } - } - } - - /** - * Adds a contiguous table to the device storage. This does NOT take ownership of the - * contiguous table, so it is the responsibility of the caller to close it. The refcount of the - * underlying device buffer will be incremented so the contiguous table can be closed before - * this buffer is destroyed. - * - * This version of `addContiguousTable` should not be called from the shuffle catalogs - * since they provide their own ids. - * - * @param contigTable contiguous table to track in storage - * @param initialSpillPriority starting spill priority value for the buffer - * @param needsSync whether the spill framework should stream synchronize while adding - * this device buffer (defaults to true) - * @return RapidsBufferHandle handle for this table - */ - def addContiguousTable( - contigTable: ContiguousTable, - initialSpillPriority: Long, - needsSync: Boolean = true): RapidsBufferHandle = synchronized { - val existing = getExistingRapidsBufferAndAcquire(contigTable.getBuffer) - existing match { - case None => - addContiguousTable( - TempSpillBufferId(), - contigTable, - initialSpillPriority, - needsSync) - case Some(rapidsBuffer) => - withResource(rapidsBuffer) { _ => - makeNewHandle(rapidsBuffer.id, initialSpillPriority) - } - } - } - - /** - * Adds a contiguous table to the device storage. This does NOT take ownership of the - * contiguous table, so it is the responsibility of the caller to close it. The refcount of the - * underlying device buffer will be incremented so the contiguous table can be closed before - * this buffer is destroyed. - * - * @param id the RapidsBufferId to use for this buffer - * @param contigTable contiguous table to track in storage - * @param initialSpillPriority starting spill priority value for the buffer - * @param needsSync whether the spill framework should stream synchronize while adding - * this device buffer (defaults to true) - * @return RapidsBufferHandle handle for this table - */ - def addContiguousTable( - id: RapidsBufferId, - contigTable: ContiguousTable, - initialSpillPriority: Long, - needsSync: Boolean): RapidsBufferHandle = synchronized { - addBuffer( - id, - contigTable.getBuffer, - MetaUtils.buildTableMeta(id.tableId, contigTable), - initialSpillPriority, - needsSync) - } - - /** - * Adds a buffer to either the device or host storage. This does NOT take - * ownership of the buffer, so it is the responsibility of the caller to close it. - * - * @param id the RapidsBufferId to use for this buffer - * @param buffer buffer that will be owned by the target store - * @param tableMeta metadata describing the buffer layout - * @param initialSpillPriority starting spill priority value for the buffer - * @param needsSync whether the spill framework should stream synchronize while adding - * this buffer (defaults to true) - * @return RapidsBufferHandle handle for this RapidsBuffer - */ - def addBuffer( - id: RapidsBufferId, - buffer: MemoryBuffer, - tableMeta: TableMeta, - initialSpillPriority: Long, - needsSync: Boolean): RapidsBufferHandle = synchronized { - val rapidsBuffer = buffer match { - case gpuBuffer: DeviceMemoryBuffer => - deviceStorage.addBuffer( - id, - gpuBuffer, - tableMeta, - initialSpillPriority, - needsSync) - case hostBuffer: HostMemoryBuffer => - hostStorage.addBuffer( - id, - hostBuffer, - tableMeta, - initialSpillPriority, - needsSync) - case _ => - throw new IllegalArgumentException( - s"Cannot call addBuffer with buffer $buffer") - } - registerNewBuffer(rapidsBuffer) - makeNewHandle(id, initialSpillPriority) - } - - /** - * Adds a batch to the device storage. This does NOT take ownership of the - * batch, so it is the responsibility of the caller to close it. - * - * @param batch batch that will be added to the store - * @param initialSpillPriority starting spill priority value for the batch - * @param needsSync whether the spill framework should stream synchronize while adding - * this batch (defaults to true) - * @return RapidsBufferHandle handle for this RapidsBuffer - */ - def addBatch( - batch: ColumnarBatch, - initialSpillPriority: Long, - needsSync: Boolean = true): RapidsBufferHandle = { - require(batch.numCols() > 0, - "Cannot call addBatch with a batch that doesn't have columns") - batch.column(0) match { - case _: RapidsHostColumnVector => - addHostBatch(batch, initialSpillPriority, needsSync) - case _ => - closeOnExcept(GpuColumnVector.from(batch)) { table => - addTable(table, initialSpillPriority, needsSync) - } - } - } - - /** - * Adds a table to the device storage. - * - * This takes ownership of the table. The reason for this is that tables - * don't have a reference count, so we cannot cleanly capture ownership by increasing - * ref count and decreasing from the caller. - * - * @param table table that will be owned by the store - * @param initialSpillPriority starting spill priority value - * @param needsSync whether the spill framework should stream synchronize while adding - * this table (defaults to true) - * @return RapidsBufferHandle handle for this RapidsBuffer - */ - def addTable( - table: Table, - initialSpillPriority: Long, - needsSync: Boolean = true): RapidsBufferHandle = { - addTable(TempSpillBufferId(), table, initialSpillPriority, needsSync) - } - - /** - * Adds a table to the device storage. - * - * This takes ownership of the table. The reason for this is that tables - * don't have a reference count, so we cannot cleanly capture ownership by increasing - * ref count and decreasing from the caller. - * - * @param id specific RapidsBufferId to use for this table - * @param table table that will be owned by the store - * @param initialSpillPriority starting spill priority value - * @param needsSync whether the spill framework should stream synchronize while adding - * this table (defaults to true) - * @return RapidsBufferHandle handle for this RapidsBuffer - */ - def addTable( - id: RapidsBufferId, - table: Table, - initialSpillPriority: Long, - needsSync: Boolean): RapidsBufferHandle = { - val rapidsBuffer = deviceStorage.addTable( - id, - table, - initialSpillPriority, - needsSync) - registerNewBuffer(rapidsBuffer) - makeNewHandle(id, initialSpillPriority) - } - - - /** - * Add a host-backed ColumnarBatch to the catalog. This is only called from addBatch - * after we detect that this is a host-backed batch. - */ - private def addHostBatch( - hostCb: ColumnarBatch, - initialSpillPriority: Long, - needsSync: Boolean): RapidsBufferHandle = { - val id = TempSpillBufferId() - val rapidsBuffer = hostStorage.addBatch( - id, - hostCb, - initialSpillPriority, - needsSync) - registerNewBuffer(rapidsBuffer) - makeNewHandle(id, initialSpillPriority) - } - - /** - * Register a degenerate RapidsBufferId given a TableMeta - * @note this is called from the shuffle catalogs only - */ - def registerDegenerateBuffer( - bufferId: RapidsBufferId, - meta: TableMeta): RapidsBufferHandle = synchronized { - val buffer = new DegenerateRapidsBuffer(bufferId, meta) - registerNewBuffer(buffer) - makeNewHandle(buffer.id, buffer.getSpillPriority) - } - - /** - * Called by the catalog when a handle is first added to the catalog, or to refresh - * the priority of the underlying buffer if a handle's priority changed. - */ - private def updateUnderlyingRapidsBuffer(handle: RapidsBufferHandle): Unit = { - withResource(acquireBuffer(handle)) { buffer => - val handles = bufferIdToHandles.get(buffer.id) - val maxPriority = handles.map(_.getSpillPriority).max - // update the priority of the underlying RapidsBuffer to be the - // maximum priority for all handles associated with it - buffer.setSpillPriority(maxPriority) - } - } - - /** - * Lookup the buffer that corresponds to the specified handle at the highest storage tier, - * and acquire it. - * NOTE: It is the responsibility of the caller to close the buffer. - * @param handle handle associated with this `RapidsBuffer` - * @return buffer that has been acquired - */ - def acquireBuffer(handle: RapidsBufferHandle): RapidsBuffer = { - val id = handle.id - def lookupAndReturn: Option[RapidsBuffer] = { - val buffers = bufferMap.get(id) - if (buffers == null || buffers.isEmpty) { - throw new NoSuchElementException( - s"Cannot locate buffers associated with ID: $id") - } - val buffer = buffers.head - if (buffer.addReference()) { - Some(buffer) - } else { - None - } - } - - // fast path - (0 until RapidsBufferCatalog.MAX_BUFFER_LOOKUP_ATTEMPTS).foreach { _ => - val mayBuffer = lookupAndReturn - if (mayBuffer.isDefined) { - return mayBuffer.get - } - } - - // try one last time after locking the catalog (slow path) - // if there is a lot of contention here, I would rather lock the world than - // have tasks error out with "Unable to acquire" - synchronized { - val mayBuffer = lookupAndReturn - if (mayBuffer.isDefined) { - return mayBuffer.get - } - } - throw new IllegalStateException(s"Unable to acquire buffer for ID: $id") - } - - /** - * Acquires a RapidsBuffer that the caller expects to be host-backed and not - * device bound. This ensures that the buffer acquired implements the correct - * trait, otherwise it throws and removes its buffer acquisition. - * - * @param handle handle associated with this `RapidsBuffer` - * @return host-backed RapidsBuffer that has been acquired - */ - def acquireHostBatchBuffer(handle: RapidsBufferHandle): RapidsHostBatchBuffer = { - closeOnExcept(acquireBuffer(handle)) { - case hrb: RapidsHostBatchBuffer => hrb - case other => - throw new IllegalStateException( - s"Attempted to acquire a RapidsHostBatchBuffer, but got $other instead") - } - } - - /** - * Lookup the buffer that corresponds to the specified buffer ID at the specified storage tier, - * and acquire it. - * NOTE: It is the responsibility of the caller to close the buffer. - * @param id buffer identifier - * @return buffer that has been acquired, None if not found - */ - def acquireBuffer(id: RapidsBufferId, tier: StorageTier): Option[RapidsBuffer] = { - val buffers = bufferMap.get(id) - if (buffers != null) { - buffers.find(_.storageTier == tier).foreach(buffer => - if (buffer.addReference()) { - return Some(buffer) - } - ) - } - None - } - - /** - * Check if the buffer that corresponds to the specified buffer ID is stored in a slower storage - * tier. - * - * @param id buffer identifier - * @param tier storage tier to check - * @note public for testing - * @return true if the buffer is stored in multiple tiers - */ - def isBufferSpilled(id: RapidsBufferId, tier: StorageTier): Boolean = { - val buffers = bufferMap.get(id) - buffers != null && buffers.exists(_.storageTier > tier) - } - - /** Get the table metadata corresponding to a buffer ID. */ - def getBufferMeta(id: RapidsBufferId): TableMeta = { - val buffers = bufferMap.get(id) - if (buffers == null || buffers.isEmpty) { - throw new NoSuchElementException(s"Cannot locate buffer associated with ID: $id") - } - buffers.head.meta - } - - /** - * Register a new buffer with the catalog. An exception will be thrown if an - * existing buffer was registered with the same buffer ID and storage tier. - * @note public for testing - */ - def registerNewBuffer(buffer: RapidsBuffer): Unit = { - val updater = new BiFunction[RapidsBufferId, Seq[RapidsBuffer], Seq[RapidsBuffer]] { - override def apply(key: RapidsBufferId, value: Seq[RapidsBuffer]): Seq[RapidsBuffer] = { - if (value == null) { - Seq(buffer) - } else { - val(first, second) = value.partition(_.storageTier < buffer.storageTier) - if (second.nonEmpty && second.head.storageTier == buffer.storageTier) { - throw new DuplicateBufferException( - s"Buffer ID ${buffer.id} at tier ${buffer.storageTier} already registered " + - s"${second.head}") - } - first ++ Seq(buffer) ++ second - } - } - } - - bufferMap.compute(buffer.id, updater) - } - - /** - * Free memory in `store` by spilling buffers to the spill store synchronously. - * @param store store to spill from - * @param targetTotalSize maximum total size of this store after spilling completes - * @param stream CUDA stream to use or omit for default stream - * @return optionally number of bytes that were spilled, or None if this call - * made no attempt to spill due to a detected spill race - */ - def synchronousSpill( - store: RapidsBufferStore, - targetTotalSize: Long, - stream: Cuda.Stream = Cuda.DEFAULT_STREAM): Option[Long] = { - if (store.spillStore == null) { - throw new OutOfMemoryError("Requested to spill without a spill store") - } - require(targetTotalSize >= 0, s"Negative spill target size: $targetTotalSize") - - val mySpillCount = spillCount - - // we have to hold this lock while freeing buffers, otherwise we could run - // into the case where a buffer is spilled yet it is aliased in addBuffer - // via an event handler that hasn't been reset (it resets during the free) - synchronized { - if (mySpillCount != spillCount) { - // a different thread already spilled, returning - // None which lets the calling code know that rmm should retry allocation - None - } else { - // this thread wins the race and should spill - spillCount += 1 - Some(store.synchronousSpill(targetTotalSize, this, stream)) - } - } - } - - def updateTiers(bufferSpill: SpillAction): Long = bufferSpill match { - case BufferSpill(spilledBuffer, maybeNewBuffer) => - logDebug(s"Spilled ${spilledBuffer.id} from tier ${spilledBuffer.storageTier}. " + - s"Removing. Registering ${maybeNewBuffer.map(_.id).getOrElse ("None")} " + - s"${maybeNewBuffer}") - maybeNewBuffer.foreach(registerNewBuffer) - removeBufferTier(spilledBuffer.id, spilledBuffer.storageTier) - spilledBuffer.memoryUsedBytes - - case BufferUnspill(unspilledBuffer, maybeNewBuffer) => - logDebug(s"Unspilled ${unspilledBuffer.id} from tier ${unspilledBuffer.storageTier}. " + - s"Removing. Registering ${maybeNewBuffer.map(_.id).getOrElse ("None")} " + - s"${maybeNewBuffer}") - maybeNewBuffer.foreach(registerNewBuffer) - removeBufferTier(unspilledBuffer.id, unspilledBuffer.storageTier) - unspilledBuffer.memoryUsedBytes - } - - /** - * Copies `buffer` to the `deviceStorage` store, registering a new `RapidsBuffer` in - * the process - * @param buffer - buffer to copy - * @param stream - Cuda.Stream to synchronize on - * @return - The `RapidsBuffer` instance that was added to the device store. - */ - def unspillBufferToDeviceStore( - buffer: RapidsBuffer, - stream: Cuda.Stream): RapidsBuffer = synchronized { - // try to acquire the buffer, if it's already in the store - // do not create a new one, else add a reference - acquireBuffer(buffer.id, StorageTier.DEVICE) match { - case None => - val maybeNewBuffer = deviceStorage.copyBuffer(buffer, this, stream) - maybeNewBuffer.map { newBuffer => - newBuffer.addReference() // add a reference since we are about to use it - registerNewBuffer(newBuffer) - newBuffer - }.get // the GPU store has to return a buffer here for now, or throw OOM - case Some(existingBuffer) => existingBuffer - } - } - - /** - * Copies `buffer` to the `hostStorage` store, registering a new `RapidsBuffer` in - * the process - * - * @param buffer - buffer to copy - * @param stream - Cuda.Stream to synchronize on - * @return - The `RapidsBuffer` instance that was added to the host store. - */ - def unspillBufferToHostStore( - buffer: RapidsBuffer, - stream: Cuda.Stream): RapidsBuffer = synchronized { - // try to acquire the buffer, if it's already in the store - // do not create a new one, else add a reference - acquireBuffer(buffer.id, StorageTier.HOST) match { - case Some(existingBuffer) => existingBuffer - case None => - val maybeNewBuffer = hostStorage.copyBuffer(buffer, this, stream) - maybeNewBuffer.map { newBuffer => - logDebug(s"got new RapidsHostMemoryStore buffer ${newBuffer.id}") - newBuffer.addReference() // add a reference since we are about to use it - updateTiers(BufferUnspill(buffer, Some(newBuffer))) - buffer.safeFree() - newBuffer - }.get // the host store has to return a buffer here for now, or throw OOM - } - } - - - /** - * Remove a buffer ID from the catalog at the specified storage tier. - * @note public for testing - */ - def removeBufferTier(id: RapidsBufferId, tier: StorageTier): Unit = synchronized { - val updater = new BiFunction[RapidsBufferId, Seq[RapidsBuffer], Seq[RapidsBuffer]] { - override def apply(key: RapidsBufferId, value: Seq[RapidsBuffer]): Seq[RapidsBuffer] = { - val updated = value.filter(_.storageTier != tier) - if (updated.isEmpty) { - null - } else { - updated - } - } - } - bufferMap.computeIfPresent(id, updater) - } - - /** - * Remove a buffer handle from the catalog and, if it this was the final handle, - * release the resources of the registered buffers. - * - * @return true: if the buffer for this handle was removed from the spill framework - * (`handle` was the last handle) - * false: if buffer was not removed due to other live handles. - */ - private def removeBuffer(handle: RapidsBufferHandle): Boolean = synchronized { - // if this is the last handle, remove the buffer - if (stopTrackingHandle(handle)) { - logDebug(s"Removing buffer ${handle.id}") - bufferMap.remove(handle.id).safeFree() - true - } else { - false - } - } - - /** Return the number of buffers currently in the catalog. */ - def numBuffers: Int = bufferMap.size() - - override def close(): Unit = { - bufferIdToHandles.values.asScala.toSeq.flatMap(_.seq).safeClose() - - bufferIdToHandles.clear() - } -} - -object RapidsBufferCatalog extends Logging { - private val MAX_BUFFER_LOOKUP_ATTEMPTS = 100 - - private var deviceStorage: RapidsDeviceMemoryStore = _ - private var hostStorage: RapidsHostMemoryStore = _ - private var diskBlockManager: RapidsDiskBlockManager = _ - private var diskStorage: RapidsDiskStore = _ - private var memoryEventHandler: DeviceMemoryEventHandler = _ - private var _shouldUnspill: Boolean = _ - private var _singleton: RapidsBufferCatalog = null - - def singleton: RapidsBufferCatalog = { - if (_singleton == null) { - synchronized { - if (_singleton == null) { - _singleton = new RapidsBufferCatalog(deviceStorage) - } - } - } - _singleton - } - - private lazy val conf: SparkConf = { - val env = SparkEnv.get - if (env != null) { - env.conf - } else { - // For some unit tests - new SparkConf() - } - } - - /** - * Set a `RapidsDeviceMemoryStore` instance to use when instantiating our - * catalog. - * @note This should only be called from tests! - */ - def setDeviceStorage(rdms: RapidsDeviceMemoryStore): Unit = { - deviceStorage = rdms - } - - /** - * Set a `RapidsDiskStore` instance to use when instantiating our - * catalog. - * - * @note This should only be called from tests! - */ - def setDiskStorage(rdms: RapidsDiskStore): Unit = { - diskStorage = rdms - } - - /** - * Set a `RapidsHostMemoryStore` instance to use when instantiating our - * catalog. - * - * @note This should only be called from tests! - */ - def setHostStorage(rhms: RapidsHostMemoryStore): Unit = { - hostStorage = rhms - } - - /** - * Set a `RapidsBufferCatalog` instance to use our singleton. - * @note This should only be called from tests! - */ - def setCatalog(catalog: RapidsBufferCatalog): Unit = synchronized { - if (_singleton != null) { - _singleton.close() - } - _singleton = catalog - } - - def init(rapidsConf: RapidsConf): Unit = { - // We are going to re-initialize so make sure all of the old things were closed... - closeImpl() - assert(memoryEventHandler == null) - deviceStorage = new RapidsDeviceMemoryStore( - rapidsConf.chunkedPackBounceBufferSize, - rapidsConf.spillToDiskBounceBufferSize) - diskBlockManager = new RapidsDiskBlockManager(conf) - val hostSpillStorageSize = if (rapidsConf.offHeapLimitEnabled) { - // Disable the limit because it is handled by the RapidsHostMemoryStore - None - } else if (rapidsConf.hostSpillStorageSize == -1) { - // + 1 GiB by default to match backwards compatibility - Some(rapidsConf.pinnedPoolSize + (1024 * 1024 * 1024)) - } else { - Some(rapidsConf.hostSpillStorageSize) - } - hostStorage = new RapidsHostMemoryStore(hostSpillStorageSize) - diskStorage = new RapidsDiskStore(diskBlockManager) - deviceStorage.setSpillStore(hostStorage) - hostStorage.setSpillStore(diskStorage) - - logInfo("Installing GPU memory handler for spill") - memoryEventHandler = new DeviceMemoryEventHandler( - singleton, - deviceStorage, - rapidsConf.gpuOomDumpDir, - rapidsConf.gpuOomMaxRetries) - - if (rapidsConf.sparkRmmStateEnable) { - val debugLoc = if (rapidsConf.sparkRmmDebugLocation.isEmpty) { - null - } else { - rapidsConf.sparkRmmDebugLocation - } - - RmmSpark.setEventHandler(memoryEventHandler, debugLoc) - } else { - logWarning("SparkRMM retry has been disabled") - Rmm.setEventHandler(memoryEventHandler) - } - - _shouldUnspill = rapidsConf.isUnspillEnabled - } - - def close(): Unit = { - logInfo("Closing storage") - closeImpl() - } - - /** - * Only used in unit tests, it returns the number of buffers in the catalog. - */ - def numBuffers: Int = { - _singleton.numBuffers - } - - private def closeImpl(): Unit = synchronized { - Seq(_singleton, deviceStorage, hostStorage, diskStorage).safeClose() - - _singleton = null - // Workaround for shutdown ordering problems where device buffers allocated - // with this handler are being freed after the handler is destroyed - //Rmm.clearEventHandler() - memoryEventHandler = null - deviceStorage = null - hostStorage = null - diskStorage = null - } - - def getDeviceStorage: RapidsDeviceMemoryStore = deviceStorage - - def getHostStorage: RapidsHostMemoryStore = hostStorage - - def shouldUnspill: Boolean = _shouldUnspill - - /** - * Adds a contiguous table to the device storage. This does NOT take ownership of the - * contiguous table, so it is the responsibility of the caller to close it. The refcount of the - * underlying device buffer will be incremented so the contiguous table can be closed before - * this buffer is destroyed. - * @param contigTable contiguous table to trackNewHandle in device storage - * @param initialSpillPriority starting spill priority value for the buffer - * @return RapidsBufferHandle associated with this buffer - */ - def addContiguousTable( - contigTable: ContiguousTable, - initialSpillPriority: Long): RapidsBufferHandle = { - singleton.addContiguousTable(contigTable, initialSpillPriority) - } - - /** - * Adds a buffer to the catalog and store. This does NOT take ownership of the - * buffer, so it is the responsibility of the caller to close it. - * @param buffer buffer that will be owned by the store - * @param tableMeta metadata describing the buffer layout - * @param initialSpillPriority starting spill priority value for the buffer - * @return RapidsBufferHandle associated with this buffer - */ - def addBuffer( - buffer: MemoryBuffer, - tableMeta: TableMeta, - initialSpillPriority: Long): RapidsBufferHandle = { - singleton.addBuffer(buffer, tableMeta, initialSpillPriority) - } - - def addBatch( - batch: ColumnarBatch, - initialSpillPriority: Long): RapidsBufferHandle = { - singleton.addBatch(batch, initialSpillPriority) - } - - /** - * Lookup the buffer that corresponds to the specified buffer handle and acquire it. - * NOTE: It is the responsibility of the caller to close the buffer. - * @param handle buffer handle - * @return buffer that has been acquired - */ - def acquireBuffer(handle: RapidsBufferHandle): RapidsBuffer = - singleton.acquireBuffer(handle) - - /** - * Acquires a RapidsBuffer that the caller expects to be host-backed and not - * device bound. This ensures that the buffer acquired implements the correct - * trait, otherwise it throws and removes its buffer acquisition. - * - * @param handle handle associated with this `RapidsBuffer` - * @return host-backed RapidsBuffer that has been acquired - */ - def acquireHostBatchBuffer(handle: RapidsBufferHandle): RapidsHostBatchBuffer = - singleton.acquireHostBatchBuffer(handle) - - def getDiskBlockManager(): RapidsDiskBlockManager = diskBlockManager - - /** - * Free memory in `store` by spilling buffers to its spill store synchronously. - * @param store store to spill from - * @param targetTotalSize maximum total size of this store after spilling completes - * @param stream CUDA stream to use or omit for default stream - * @return optionally number of bytes that were spilled, or None if this call - * made no attempt to spill due to a detected spill race - */ - def synchronousSpill( - store: RapidsBufferStore, - targetTotalSize: Long, - stream: Cuda.Stream = Cuda.DEFAULT_STREAM): Option[Long] = { - singleton.synchronousSpill(store, targetTotalSize, stream) - } - - /** - * Given a `MemoryBuffer` find out if a `MemoryBuffer.EventHandler` is associated - * with it. - * - * After getting the `RapidsBuffer` try to acquire it via `addReference`. - * If successful, we can point to this buffer with a new handle, otherwise the buffer is - * about to be removed/freed (unlikely, because we are holding onto the reference as we - * are adding it again). - * - * @note public for testing - * @param buffer - the `MemoryBuffer` to inspect - * @return - Some(RapidsBuffer): the handler is associated with a rapids buffer - * and the rapids buffer is currently valid, or - * - * - None: if no `RapidsBuffer` is associated with this buffer (it is - * brand new to the store, or the `RapidsBuffer` is invalid and - * about to be removed). - */ - private def getExistingRapidsBufferAndAcquire(buffer: MemoryBuffer): Option[RapidsBuffer] = { - buffer match { - case hb: HostMemoryBuffer => - HostAlloc.findEventHandler(hb) { - case rapidsBuffer: RapidsBuffer => - if (rapidsBuffer.addReference()) { - Some(rapidsBuffer) - } else { - None - } - }.flatten - case _ => - val eh = buffer.getEventHandler - eh match { - case null => - None - case rapidsBuffer: RapidsBuffer => - if (rapidsBuffer.addReference()) { - Some(rapidsBuffer) - } else { - None - } - case _ => - throw new IllegalStateException("Unknown event handler") - } - } - } -} - diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala deleted file mode 100644 index b1ee9e7a863..00000000000 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala +++ /dev/null @@ -1,640 +0,0 @@ -/* - * Copyright (c) 2020-2024, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids - -import java.util.Comparator -import java.util.concurrent.locks.ReentrantReadWriteLock - -import scala.collection.mutable - -import ai.rapids.cudf.{BaseDeviceMemoryBuffer, Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer, NvtxColor, NvtxRange} -import com.nvidia.spark.rapids.Arm._ -import com.nvidia.spark.rapids.RapidsPluginImplicits._ -import com.nvidia.spark.rapids.StorageTier.{DEVICE, HOST, StorageTier} -import com.nvidia.spark.rapids.format.TableMeta - -import org.apache.spark.internal.Logging -import org.apache.spark.sql.types.DataType -import org.apache.spark.sql.vectorized.ColumnarBatch - -/** - * Helper case classes that contain the buffer we spilled or unspilled from our current tier - * and likely a new buffer created in a target store tier, but it can be set to None. - * If the buffer already exists in the target store, `newBuffer` will be None. - * @param spillBuffer a `RapidsBuffer` we spilled or unspilled from this store - * @param newBuffer an optional `RapidsBuffer` in the target store. - */ -trait SpillAction { - val spillBuffer: RapidsBuffer - val newBuffer: Option[RapidsBuffer] -} - -case class BufferSpill(spillBuffer: RapidsBuffer, newBuffer: Option[RapidsBuffer]) - extends SpillAction - -case class BufferUnspill(spillBuffer: RapidsBuffer, newBuffer: Option[RapidsBuffer]) - extends SpillAction - -/** - * Base class for all buffer store types. - * - * @param tier storage tier of this store - * @param catalog catalog to register this store - */ -abstract class RapidsBufferStore(val tier: StorageTier) - extends AutoCloseable with Logging { - - val name: String = tier.toString - - private class BufferTracker { - private[this] val comparator: Comparator[RapidsBufferBase] = - (o1: RapidsBufferBase, o2: RapidsBufferBase) => - java.lang.Long.compare(o1.getSpillPriority, o2.getSpillPriority) - // buffers: contains all buffers in this store, whether spillable or not - private[this] val buffers = new java.util.HashMap[RapidsBufferId, RapidsBufferBase] - // spillable: contains only those buffers that are currently spillable - private[this] val spillable = new HashedPriorityQueue[RapidsBufferBase](comparator) - // spilling: contains only those buffers that are currently being spilled, but - // have not been removed from the store - private[this] val spilling = new mutable.HashSet[RapidsBufferId]() - // total bytes stored, regardless of spillable status - private[this] var totalBytesStored: Long = 0L - // total bytes that are currently eligible to be spilled - private[this] var totalBytesSpillable: Long = 0L - - def add(buffer: RapidsBufferBase): Unit = synchronized { - val old = buffers.put(buffer.id, buffer) - // it is unlikely that the buffer was in this collection, but removing - // anyway. We assume the buffer is safe in this tier, and is not spilling - spilling.remove(buffer.id) - if (old != null) { - throw new DuplicateBufferException(s"duplicate buffer registered: ${buffer.id}") - } - totalBytesStored += buffer.memoryUsedBytes - - // device buffers "spillability" is handled via DeviceMemoryBuffer ref counting - // so spillableOnAdd should be false, all other buffer tiers are spillable at - // all times. - if (spillableOnAdd && buffer.memoryUsedBytes > 0) { - if (spillable.offer(buffer)) { - totalBytesSpillable += buffer.memoryUsedBytes - } - } - } - - def remove(id: RapidsBufferId): Unit = synchronized { - // when removing a buffer we no longer need to know if it was spilling - spilling.remove(id) - val obj = buffers.remove(id) - if (obj != null) { - totalBytesStored -= obj.memoryUsedBytes - if (spillable.remove(obj)) { - totalBytesSpillable -= obj.memoryUsedBytes - } - } - } - - def freeAll(): Unit = { - val values = synchronized { - val buffs = buffers.values().toArray(new Array[RapidsBufferBase](0)) - buffers.clear() - spillable.clear() - spilling.clear() - buffs - } - // We need to release the `RapidsBufferStore` lock to prevent a lock order inversion - // deadlock: (1) `RapidsBufferBase.free` calls (2) `RapidsBufferStore.remove` and - // (1) `RapidsBufferStore.freeAll` calls (2) `RapidsBufferBase.free`. - values.safeFree() - } - - /** - * Sets a buffers state to spillable or non-spillable. - * - * If the buffer is currently being spilled or it is no longer in the `buffers` collection - * (e.g. it is not in this store), the action is skipped. - * - * @param buffer the buffer to mark as spillable or not - * @param isSpillable whether the buffer should now be spillable - */ - def setSpillable(buffer: RapidsBufferBase, isSpillable: Boolean): Unit = synchronized { - if (isSpillable && buffer.memoryUsedBytes > 0) { - // if this buffer is in the store and isn't currently spilling - if (!spilling.contains(buffer.id) && buffers.containsKey(buffer.id)) { - // try to add it to the spillable collection - if (spillable.offer(buffer)) { - totalBytesSpillable += buffer.memoryUsedBytes - logDebug(s"Buffer ${buffer.id} is spillable. " + - s"total=${totalBytesStored} spillable=${totalBytesSpillable}") - } // else it was already there (unlikely) - } - } else { - if (spillable.remove(buffer)) { - totalBytesSpillable -= buffer.memoryUsedBytes - logDebug(s"Buffer ${buffer.id} is not spillable. " + - s"total=${totalBytesStored}, spillable=${totalBytesSpillable}") - } // else it was already removed - } - } - - def nextSpillableBuffer(): RapidsBufferBase = synchronized { - val buffer = spillable.poll() - if (buffer != null) { - // mark the id as "spilling" (this buffer is in the middle of a spill operation) - spilling.add(buffer.id) - totalBytesSpillable -= buffer.memoryUsedBytes - logDebug(s"Spilling buffer ${buffer.id}. size=${buffer.memoryUsedBytes} " + - s"total=${totalBytesStored}, new spillable=${totalBytesSpillable}") - } - buffer - } - - def updateSpillPriority(buffer: RapidsBufferBase, priority:Long): Unit = synchronized { - buffer.updateSpillPriorityValue(priority) - spillable.priorityUpdated(buffer) - } - - def getTotalBytes: Long = synchronized { totalBytesStored } - - def getTotalSpillableBytes: Long = synchronized { totalBytesSpillable } - } - - /** - * Stores that need to stay within a specific byte limit of buffers stored override - * this function. Only the `HostMemoryBufferStore` requires such a limit. - * @return maximum amount of bytes that can be stored in the store, None for no - * limit - */ - def getMaxSize: Option[Long] = None - - private[this] val buffers = new BufferTracker - - /** A store that can be used for spilling. */ - var spillStore: RapidsBufferStore = _ - - /** Return the current byte total of buffers in this store. */ - def currentSize: Long = buffers.getTotalBytes - - def currentSpillableSize: Long = buffers.getTotalSpillableBytes - - /** - * A store that manages spillability of buffers should override this method - * to false, otherwise `BufferTracker` treats buffers as always spillable. - */ - protected def spillableOnAdd: Boolean = true - - /** - * Specify another store that can be used when this store needs to spill. - * @note Only one spill store can be registered. This will throw if a - * spill store has already been registered. - */ - def setSpillStore(store: RapidsBufferStore): Unit = { - require(spillStore == null, "spill store already registered") - spillStore = store - } - - /** - * Adds an existing buffer from another store to this store. The buffer must already - * have an active reference by the caller and needs to be eventually closed by the caller - * (i.e.: this method will not take ownership of the incoming buffer object). - * This does not need to update the catalog, the caller is responsible for that. - * @param buffer data from another store - * @param catalog RapidsBufferCatalog we may need to modify during this copy - * @param stream CUDA stream to use for copy or null - * @return the new buffer that was created - */ - def copyBuffer( - buffer: RapidsBuffer, - catalog: RapidsBufferCatalog, - stream: Cuda.Stream): Option[RapidsBufferBase] = { - createBuffer(buffer, catalog, stream).map { newBuffer => - freeOnExcept(newBuffer) { newBuffer => - addBuffer(newBuffer) - newBuffer - } - } - } - - protected def setSpillable(buffer: RapidsBufferBase, isSpillable: Boolean): Unit = { - buffers.setSpillable(buffer, isSpillable) - } - - /** - * Create a new buffer from an existing buffer in another store. - * If the data transfer will be performed asynchronously, this method is responsible for - * adding a reference to the existing buffer and later closing it when the transfer completes. - * - * @note DO NOT close the buffer unless adding a reference! - * @note `createBuffer` impls should synchronize against `stream` before returning, if needed. - * @param buffer data from another store - * @param catalog RapidsBufferCatalog we may need to modify during this create - * @param stream CUDA stream to use or null - * @return the new buffer that was created. - */ - protected def createBuffer( - buffer: RapidsBuffer, - catalog: RapidsBufferCatalog, - stream: Cuda.Stream): Option[RapidsBufferBase] - - /** Update bookkeeping for a new buffer */ - protected def addBuffer(buffer: RapidsBufferBase): Unit = { - buffers.add(buffer) - buffer.updateSpillability() - } - - /** - * Adds a buffer to the spill framework, stream synchronizing with the producer - * stream to ensure that the buffer is fully materialized, and can be safely copied - * as part of the spill. - * - * @param needsSync true if we should stream synchronize before adding the buffer - */ - protected def addBuffer(buffer: RapidsBufferBase, needsSync: Boolean): Unit = { - if (needsSync) { - Cuda.DEFAULT_STREAM.sync() - } - addBuffer(buffer) - } - - override def close(): Unit = { - buffers.freeAll() - } - - def nextSpillable(): RapidsBuffer = { - buffers.nextSpillableBuffer() - } - - def synchronousSpill( - targetTotalSize: Long, - catalog: RapidsBufferCatalog, - stream: Cuda.Stream = Cuda.DEFAULT_STREAM): Long = { - if (currentSpillableSize > targetTotalSize) { - logWarning(s"Targeting a ${name} size of $targetTotalSize. " + - s"Current total ${currentSize}. " + - s"Current spillable ${currentSpillableSize}") - val bufferSpills = new mutable.ArrayBuffer[BufferSpill]() - withResource(new NvtxRange(s"${name} sync spill", NvtxColor.ORANGE)) { _ => - logWarning(s"${name} store spilling to reduce usage from " + - s"${currentSize} total (${currentSpillableSize} spillable) " + - s"to $targetTotalSize bytes") - - // If the store has 0 spillable bytes left, it has exhausted. - try { - var exhausted = false - var totalSpilled = 0L - while (!exhausted && - currentSpillableSize > targetTotalSize) { - val nextSpillableBuffer = nextSpillable() - if (nextSpillableBuffer != null) { - if (nextSpillableBuffer.addReference()) { - withResource(nextSpillableBuffer) { _ => - val bufferHasSpilled = - catalog.isBufferSpilled( - nextSpillableBuffer.id, - nextSpillableBuffer.storageTier) - val bufferSpill = if (!bufferHasSpilled) { - spillBuffer( - nextSpillableBuffer, this, catalog, stream) - } else { - // if `nextSpillableBuffer` already spilled, we still need to - // remove it from our tier and call free on it, but set - // `newBuffer` to None because there's nothing to register - // as it has already spilled. - BufferSpill(nextSpillableBuffer, None) - } - totalSpilled += bufferSpill.spillBuffer.memoryUsedBytes - bufferSpills.append(bufferSpill) - catalog.updateTiers(bufferSpill) - } - } - } - } - if (totalSpilled <= 0) { - // we didn't spill in this iteration, exit loop - exhausted = true - logWarning("Unable to spill enough to meet request. " + - s"Total=${currentSize} " + - s"Spillable=${currentSpillableSize} " + - s"Target=$targetTotalSize") - } - totalSpilled - } finally { - if (bufferSpills.nonEmpty) { - // This is a hack in order to completely synchronize with the GPU before we free - // a buffer. It is necessary because of non-synchronous cuDF calls that could fall - // behind where the CPU is. Freeing a rapids buffer in these cases needs to wait for - // all launched GPU work, otherwise crashes or data corruption could occur. - // A more performant implementation would be to synchronize on the thread that read - // the buffer via events. - // https://github.com/NVIDIA/spark-rapids/issues/8610 - Cuda.deviceSynchronize() - bufferSpills.foreach(_.spillBuffer.safeFree()) - } - } - } - } else { - 0L // nothing spilled - } - } - - /** - * Given a specific `RapidsBuffer` spill it to `spillStore` - * - * @return a `BufferSpill` instance with the target buffer in this store, and an optional - * new `RapidsBuffer` in the target spill store if this rapids buffer hadn't already - * spilled. - * @note called with catalog lock held - */ - private def spillBuffer( - buffer: RapidsBuffer, - store: RapidsBufferStore, - catalog: RapidsBufferCatalog, - stream: Cuda.Stream): BufferSpill = { - // copy the buffer to spillStore - var maybeNewBuffer: Option[RapidsBuffer] = None - var lastTier: Option[StorageTier] = None - var nextSpillStore = store.spillStore - while (maybeNewBuffer.isEmpty && nextSpillStore != null) { - lastTier = Some(nextSpillStore.tier) - // copy buffer if it fits - maybeNewBuffer = nextSpillStore.copyBuffer(buffer, catalog, stream) - - // if it didn't fit, we can try a lower tier that has more space - if (maybeNewBuffer.isEmpty) { - nextSpillStore = nextSpillStore.spillStore - } - } - if (maybeNewBuffer.isEmpty) { - throw new IllegalStateException( - s"Unable to spill buffer ${buffer.id} of size ${buffer.memoryUsedBytes} " + - s"to tier ${lastTier}") - } - // return the buffer to free and the new buffer to register - BufferSpill(buffer, maybeNewBuffer) - } - - /** - * Tries to make room for `buffer` in the host store by spilling. - * - * @param buffer buffer that will be copied to the host store if it fits - * @param stream CUDA stream to synchronize for memory operations - * @return true if the buffer fits after a potential spill - */ - protected def trySpillToMaximumSize( - buffer: RapidsBuffer, - catalog: RapidsBufferCatalog, - stream: Cuda.Stream): Boolean = { - true // default to success, HostMemoryStore overrides this - } - - /** Base class for all buffers in this store. */ - abstract class RapidsBufferBase( - override val id: RapidsBufferId, - _meta: TableMeta, - initialSpillPriority: Long, - catalog: RapidsBufferCatalog = RapidsBufferCatalog.singleton) - extends RapidsBuffer { - private val MAX_UNSPILL_ATTEMPTS = 100 - - // isValid and refcount must be used with the `RapidsBufferBase` lock held - protected[this] var isValid = true - protected[this] var refcount = 0 - - private[this] var spillPriority: Long = initialSpillPriority - - private[this] val rwl: ReentrantReadWriteLock = new ReentrantReadWriteLock() - - - def meta: TableMeta = _meta - - /** Release the underlying resources for this buffer. */ - protected def releaseResources(): Unit - - /** - * Materialize the memory buffer from the underlying storage. - * - * If the buffer resides in device or host memory, only reference count is incremented. - * If the buffer resides in secondary storage, a new host or device memory buffer is created, - * with the data copied to the new buffer. - * The caller must have successfully acquired the buffer beforehand. - * @see [[addReference]] - * @note It is the responsibility of the caller to close the buffer. - * @note This is an internal API only used by Rapids buffer stores. - */ - protected def materializeMemoryBuffer: MemoryBuffer = getMemoryBuffer - - override def addReference(): Boolean = synchronized { - if (isValid) { - refcount += 1 - } - isValid - } - - override def getColumnarBatch(sparkTypes: Array[DataType]): ColumnarBatch = { - // NOTE: Cannot hold a lock on this buffer here because memory is being - // allocated. Allocations can trigger synchronous spills which can - // deadlock if another thread holds the device store lock and is trying - // to spill to this store. - withResource(getDeviceMemoryBuffer) { deviceBuffer => - columnarBatchFromDeviceBuffer(deviceBuffer, sparkTypes) - } - } - - protected def columnarBatchFromDeviceBuffer(devBuffer: DeviceMemoryBuffer, - sparkTypes: Array[DataType]): ColumnarBatch = { - val bufferMeta = meta.bufferMeta() - if (bufferMeta == null || bufferMeta.codecBufferDescrsLength == 0) { - MetaUtils.getBatchFromMeta(devBuffer, meta, sparkTypes) - } else { - GpuCompressedColumnVector.from(devBuffer, meta) - } - } - - override def copyToMemoryBuffer(srcOffset: Long, dst: MemoryBuffer, dstOffset: Long, - length: Long, stream: Cuda.Stream): Unit = { - withResource(getMemoryBuffer) { memBuff => - dst match { - case _: HostMemoryBuffer => - // TODO: consider moving to the async version. - dst.copyFromMemoryBuffer(dstOffset, memBuff, srcOffset, length, stream) - case _: BaseDeviceMemoryBuffer => - dst.copyFromMemoryBufferAsync(dstOffset, memBuff, srcOffset, length, stream) - case _ => - throw new IllegalStateException(s"Infeasible destination buffer type ${dst.getClass}") - } - } - } - - /** - * TODO: we want to remove this method from the buffer, instead we want the catalog - * to be responsible for producing the DeviceMemoryBuffer by asking the buffer. This - * hides the RapidsBuffer from clients and simplifies locking. - */ - override def getDeviceMemoryBuffer: DeviceMemoryBuffer = { - if (RapidsBufferCatalog.shouldUnspill) { - (0 until MAX_UNSPILL_ATTEMPTS).foreach { _ => - catalog.acquireBuffer(id, DEVICE) match { - case Some(buffer) => - withResource(buffer) { _ => - return buffer.getDeviceMemoryBuffer - } - case _ => - try { - logDebug(s"Unspilling $this $id to $DEVICE") - val newBuffer = catalog.unspillBufferToDeviceStore( - this, - Cuda.DEFAULT_STREAM) - withResource(newBuffer) { _ => - return newBuffer.getDeviceMemoryBuffer - } - } catch { - case _: DuplicateBufferException => - logDebug(s"Lost device buffer registration race for buffer $id, retrying...") - } - } - } - throw new IllegalStateException(s"Unable to get device memory buffer for ID: $id") - } else { - materializeMemoryBuffer match { - case h: HostMemoryBuffer => - withResource(h) { _ => - closeOnExcept(DeviceMemoryBuffer.allocate(h.getLength)) { deviceBuffer => - logDebug(s"copying ${h.getLength} from host $h to device $deviceBuffer " + - s"of size ${deviceBuffer.getLength}") - deviceBuffer.copyFromHostBuffer(h) - deviceBuffer - } - } - case d: DeviceMemoryBuffer => d - case b => throw new IllegalStateException(s"Unrecognized buffer: $b") - } - } - } - - override def getHostMemoryBuffer: HostMemoryBuffer = { - (0 until MAX_UNSPILL_ATTEMPTS).foreach { _ => - catalog.acquireBuffer(id, HOST) match { - case Some(buffer) => - withResource(buffer) { _ => - return buffer.getHostMemoryBuffer - } - case _ => - try { - logDebug(s"Unspilling $this $id to $HOST") - val newBuffer = catalog.unspillBufferToHostStore( - this, - Cuda.DEFAULT_STREAM) - withResource(newBuffer) { _ => - return newBuffer.getHostMemoryBuffer - } - } catch { - case _: DuplicateBufferException => - logDebug(s"Lost host buffer registration race for buffer $id, retrying...") - } - } - } - throw new IllegalStateException(s"Unable to get host memory buffer for ID: $id") - } - - /** - * close() is called by client code to decrease the ref count of this RapidsBufferBase. - * In the off chance that by the time close is invoked, the buffer was freed (not valid) - * then this close call winds up freeing the resources of the rapids buffer. - */ - override def close(): Unit = synchronized { - if (refcount == 0) { - throw new IllegalStateException("Buffer already closed") - } - refcount -= 1 - if (refcount == 0 && !isValid) { - freeBuffer() - } - } - - /** - * Mark the buffer as freed and no longer valid. This is called by the store when removing a - * buffer (it is no longer tracked). - * - * @note The resources may not be immediately released if the buffer has outstanding references. - * In that case the resources will be released when the reference count reaches zero. - */ - override def free(): Unit = synchronized { - if (isValid) { - isValid = false - buffers.remove(id) - if (refcount == 0) { - freeBuffer() - } - } else { - logWarning(s"Trying to free an invalid buffer => $id, size = ${memoryUsedBytes}, $this") - } - } - - override def getSpillPriority: Long = spillPriority - - override def setSpillPriority(priority: Long): Unit = - buffers.updateSpillPriority(this, priority) - - private[RapidsBufferStore] def updateSpillPriorityValue(priority: Long): Unit = { - spillPriority = priority - } - - override def withMemoryBufferReadLock[K](body: MemoryBuffer => K): K = { - withResource(getMemoryBuffer) { buff => - val lock = rwl.readLock() - try { - lock.lock() - body(buff) - } finally { - lock.unlock() - } - } - } - - override def withMemoryBufferWriteLock[K](body: MemoryBuffer => K): K = { - withResource(getMemoryBuffer) { buff => - val lock = rwl.writeLock() - try { - lock.lock() - body(buff) - } finally { - lock.unlock() - } - } - } - - /** Must be called with a lock on the buffer */ - private def freeBuffer(): Unit = { - releaseResources() - } - - override def toString: String = s"$name buffer size=$memoryUsedBytes" - } -} - -/** - * Buffers that inherit from this type do not support changing the spillable status - * of a `RapidsBuffer`. This is only used right now for disk. - * @param tier storage tier of this store - */ -abstract class RapidsBufferStoreWithoutSpill(override val tier: StorageTier) - extends RapidsBufferStore(tier) { - - override def setSpillable(rapidsBuffer: RapidsBufferBase, isSpillable: Boolean): Unit = { - throw new NotImplementedError(s"This store ${this} does not implement setSpillable") - } -} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStore.scala deleted file mode 100644 index c56806bc965..00000000000 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStore.scala +++ /dev/null @@ -1,518 +0,0 @@ -/* - * 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. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids - -import java.nio.channels.WritableByteChannel -import java.util.concurrent.ConcurrentHashMap - -import scala.collection.mutable - -import ai.rapids.cudf.{ColumnVector, Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer, Table} -import com.nvidia.spark.rapids.Arm._ -import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableSeq -import com.nvidia.spark.rapids.StorageTier.StorageTier -import com.nvidia.spark.rapids.format.TableMeta - -import org.apache.spark.sql.rapids.GpuTaskMetrics -import org.apache.spark.sql.rapids.storage.RapidsStorageUtils -import org.apache.spark.sql.types.DataType -import org.apache.spark.sql.vectorized.ColumnarBatch - -/** - * Buffer storage using device memory. - * @param chunkedPackBounceBufferSize this is the size of the bounce buffer to be used - * during spill in chunked_pack. The parameter defaults to 128MB, - * with a rule-of-thumb of 1MB per SM. - */ -class RapidsDeviceMemoryStore( - chunkedPackBounceBufferSize: Long = 128L*1024*1024, - hostBounceBufferSize: Long = 128L*1024*1024) - extends RapidsBufferStore(StorageTier.DEVICE) { - - // The RapidsDeviceMemoryStore handles spillability via ref counting - override protected def spillableOnAdd: Boolean = false - - // bounce buffer to be used during chunked pack in GPU to host memory spill - private var chunkedPackBounceBuffer: DeviceMemoryBuffer = - DeviceMemoryBuffer.allocate(chunkedPackBounceBufferSize) - - private var hostSpillBounceBuffer: HostMemoryBuffer = - HostMemoryBuffer.allocate(hostBounceBufferSize) - - override protected def createBuffer( - other: RapidsBuffer, - catalog: RapidsBufferCatalog, - stream: Cuda.Stream): Option[RapidsBufferBase] = { - val memoryBuffer = withResource(other.getCopyIterator) { copyIterator => - copyIterator.next() - } - withResource(memoryBuffer) { _ => - val deviceBuffer = { - memoryBuffer match { - case d: DeviceMemoryBuffer => d - case h: HostMemoryBuffer => - GpuTaskMetrics.get.readSpillFromHostTime { - closeOnExcept(DeviceMemoryBuffer.allocate(memoryBuffer.getLength)) { deviceBuffer => - logDebug(s"copying from host $h to device $deviceBuffer") - deviceBuffer.copyFromHostBuffer(h, stream) - deviceBuffer - } - } - case b => throw new IllegalStateException(s"Unrecognized buffer: $b") - } - } - Some(new RapidsDeviceMemoryBuffer( - other.id, - deviceBuffer.getLength, - other.meta, - deviceBuffer, - other.getSpillPriority)) - } - } - - /** - * Adds a buffer to the device storage. This does NOT take ownership of the - * buffer, so it is the responsibility of the caller to close it. - * - * This function is called only from the RapidsBufferCatalog, under the - * catalog lock. - * - * @param id the RapidsBufferId to use for this buffer - * @param buffer buffer that will be owned by the store - * @param tableMeta metadata describing the buffer layout - * @param initialSpillPriority starting spill priority value for the buffer - * @param needsSync whether the spill framework should stream synchronize while adding - * this device buffer (defaults to true) - * @return the RapidsBuffer instance that was added. - */ - def addBuffer( - id: RapidsBufferId, - buffer: DeviceMemoryBuffer, - tableMeta: TableMeta, - initialSpillPriority: Long, - needsSync: Boolean): RapidsBuffer = { - buffer.incRefCount() - val rapidsBuffer = new RapidsDeviceMemoryBuffer( - id, - buffer.getLength, - tableMeta, - buffer, - initialSpillPriority) - freeOnExcept(rapidsBuffer) { _ => - logDebug(s"Adding receive side table for: [id=$id, size=${buffer.getLength}, " + - s"uncompressed=${rapidsBuffer.meta.bufferMeta.uncompressedSize}, " + - s"meta_id=${tableMeta.bufferMeta.id}, " + - s"meta_size=${tableMeta.bufferMeta.size}]") - addBuffer(rapidsBuffer, needsSync) - rapidsBuffer - } - } - - /** - * Adds a table to the device storage. - * - * This takes ownership of the table. - * - * This function is called only from the RapidsBufferCatalog, under the - * catalog lock. - * - * @param id the RapidsBufferId to use for this table - * @param table table that will be owned by the store - * @param initialSpillPriority starting spill priority value - * @param needsSync whether the spill framework should stream synchronize while adding - * this table (defaults to true) - * @return the RapidsBuffer instance that was added. - */ - def addTable( - id: RapidsBufferId, - table: Table, - initialSpillPriority: Long, - needsSync: Boolean): RapidsBuffer = { - val rapidsTable = new RapidsTable( - id, - table, - initialSpillPriority) - freeOnExcept(rapidsTable) { _ => - addBuffer(rapidsTable, needsSync) - rapidsTable - } - } - - /** - * A per cuDF column event handler that handles calls to .close() - * inside of the `ColumnVector` lock. - */ - class RapidsDeviceColumnEventHandler - extends ColumnVector.EventHandler { - - // Every RapidsTable that references this column has an entry in this map. - // The value represents the number of times (normally 1) that a ColumnVector - // appears in the RapidsTable. This is also the ColumnVector refCount at which - // the column is considered spillable. - // The map is protected via the ColumnVector lock. - private val registration = new mutable.HashMap[RapidsTable, Int]() - - /** - * Every RapidsTable iterates through its columns and either creates - * a `ColumnTracking` object and associates it with the column's - * `eventHandler` or calls into the existing one, and registers itself. - * - * The registration has two goals: it accounts for repetition of a column - * in a `RapidsTable`. If a table has the same column repeated it must adjust - * the refCount at which this column is considered spillable. - * - * The second goal is to account for aliasing. If two tables alias this column - * we are going to mark it as non spillable. - * - * @param rapidsTable - the table that is registering itself with this tracker - */ - def register(rapidsTable: RapidsTable, repetition: Int): Unit = { - registration.put(rapidsTable, repetition) - } - - /** - * This is invoked during `RapidsTable.free` in order to remove the entry - * in `registration`. - * @param rapidsTable - the table that is de-registering itself - */ - def deregister(rapidsTable: RapidsTable): Unit = { - registration.remove(rapidsTable) - } - - // called with the cudfCv lock held from cuDF's side - override def onClosed(cudfCv: ColumnVector, refCount: Int): Unit = { - // we only handle spillability if there is a single table registered - // (no aliasing) - if (registration.size == 1) { - val (rapidsTable, spillableRefCount) = registration.head - if (spillableRefCount == refCount) { - rapidsTable.onColumnSpillable(cudfCv) - } - } - } - } - - /** - * A `RapidsTable` is the spill store holder of a cuDF `Table`. - * - * The table is not contiguous in GPU memory. Instead, this `RapidsBuffer` instance - * allows us to use the cuDF chunked_pack API to make the table contiguous as the spill - * is happening. - * - * This class owns the cuDF table and will close it when `close` is called. - * - * @param id the `RapidsBufferId` this table is associated with - * @param table the cuDF table that we are managing - * @param spillPriority a starting spill priority - */ - class RapidsTable( - id: RapidsBufferId, - table: Table, - spillPriority: Long) - extends RapidsBufferBase( - id, - null, - spillPriority) - with RapidsBufferChannelWritable { - - /** The storage tier for this buffer */ - override val storageTier: StorageTier = StorageTier.DEVICE - - override val supportsChunkedPacker: Boolean = true - - // This is the current size in batch form. It is to be used while this - // table hasn't migrated to another store. - private val unpackedSizeInBytes: Long = GpuColumnVector.getTotalDeviceMemoryUsed(table) - - // By default all columns are NOT spillable since we are not the only owners of - // the columns (the caller is holding onto a ColumnarBatch that will be closed - // after instantiation, triggering onClosed callbacks) - // This hash set contains the columns that are currently spillable. - private val columnSpillability = new ConcurrentHashMap[ColumnVector, Boolean]() - - private val numDistinctColumns = - (0 until table.getNumberOfColumns).map(table.getColumn).distinct.size - - // we register our event callbacks as the very first action to deal with - // spillability - registerOnCloseEventHandler() - - /** Release the underlying resources for this buffer. */ - override protected def releaseResources(): Unit = { - table.close() - } - - private lazy val (cachedMeta, cachedPackedSize) = { - withResource(makeChunkedPacker) { cp => - (cp.getMeta, cp.getTotalContiguousSize) - } - } - - override def meta: TableMeta = cachedMeta - - override val memoryUsedBytes: Long = unpackedSizeInBytes - - override def getPackedSizeBytes: Long = cachedPackedSize - - override def makeChunkedPacker: ChunkedPacker = - new ChunkedPacker(id, table, chunkedPackBounceBuffer) - - /** - * Mark a column as spillable - * - * @param column the ColumnVector to mark as spillable - */ - def onColumnSpillable(column: ColumnVector): Unit = { - columnSpillability.put(column, true) - updateSpillability() - } - - /** - * Update the spillability state of this RapidsTable. This is invoked from - * two places: - * - * - from the onColumnSpillable callback, which is invoked from a - * ColumnVector.EventHandler.onClosed callback. - * - * - after adding a table to the store to mark the table as spillable if - * all columns are spillable. - */ - override def updateSpillability(): Unit = { - setSpillable(this, columnSpillability.size == numDistinctColumns) - } - - /** - * Produce a `ColumnarBatch` from our table, and in the process make ourselves - * not spillable. - * - * @param sparkTypes the spark data types the batch should have - */ - override def getColumnarBatch(sparkTypes: Array[DataType]): ColumnarBatch = { - columnSpillability.clear() - setSpillable(this, false) - GpuColumnVector.from(table, sparkTypes) - } - - /** - * Get the underlying memory buffer. This may be either a HostMemoryBuffer or a - * DeviceMemoryBuffer depending on where the buffer currently resides. - * The caller must have successfully acquired the buffer beforehand. - * - * @see [[addReference]] - * @note It is the responsibility of the caller to close the buffer. - */ - override def getMemoryBuffer: MemoryBuffer = { - throw new UnsupportedOperationException( - "RapidsDeviceMemoryBatch doesn't support getMemoryBuffer") - } - - override def free(): Unit = { - // lets remove our handler from the chain of handlers for each column - removeOnCloseEventHandler() - super.free() - } - - private def registerOnCloseEventHandler(): Unit = { - val columns = (0 until table.getNumberOfColumns).map(table.getColumn) - // cudfColumns could contain duplicates. We need to take this into account when we are - // deciding the floor refCount for a duplicated column - val repetitionPerColumn = new mutable.HashMap[ColumnVector, Int]() - columns.foreach { col => - val repetitionCount = repetitionPerColumn.getOrElse(col, 0) - repetitionPerColumn(col) = repetitionCount + 1 - } - repetitionPerColumn.foreach { case (distinctCv, repetition) => - // lock the column because we are setting its event handler, and we are inspecting - // its refCount. - distinctCv.synchronized { - val eventHandler = distinctCv.getEventHandler match { - case null => - val eventHandler = new RapidsDeviceColumnEventHandler - distinctCv.setEventHandler(eventHandler) - eventHandler - case existing: RapidsDeviceColumnEventHandler => - existing - case other => - throw new IllegalStateException( - s"Invalid column event handler $other") - } - eventHandler.register(this, repetition) - if (repetition == distinctCv.getRefCount) { - onColumnSpillable(distinctCv) - } - } - } - } - - // this method is called from free() - private def removeOnCloseEventHandler(): Unit = { - val distinctColumns = - (0 until table.getNumberOfColumns).map(table.getColumn).distinct - distinctColumns.foreach { distinctCv => - distinctCv.synchronized { - distinctCv.getEventHandler match { - case eventHandler: RapidsDeviceColumnEventHandler => - eventHandler.deregister(this) - case t => - throw new IllegalStateException( - s"Invalid column event handler $t") - } - } - } - } - - override def writeToChannel(outputChannel: WritableByteChannel, stream: Cuda.Stream): Long = { - var written: Long = 0L - withResource(getCopyIterator) { copyIter => - while(copyIter.hasNext) { - withResource(copyIter.next()) { slice => - val iter = - new MemoryBufferToHostByteBufferIterator( - slice, - hostSpillBounceBuffer, - stream) - iter.foreach { bb => - try { - while (bb.hasRemaining) { - written += outputChannel.write(bb) - } - } finally { - RapidsStorageUtils.dispose(bb) - } - } - } - } - written - } - } - - } - - class RapidsDeviceMemoryBuffer( - id: RapidsBufferId, - size: Long, - meta: TableMeta, - contigBuffer: DeviceMemoryBuffer, - spillPriority: Long) - extends RapidsBufferBase(id, meta, spillPriority) - with MemoryBuffer.EventHandler - with RapidsBufferChannelWritable { - - override val memoryUsedBytes: Long = size - - override val storageTier: StorageTier = StorageTier.DEVICE - - // If this require triggers, we are re-adding a `DeviceMemoryBuffer` outside of - // the catalog lock, which should not possible. The event handler is set to null - // when we free the `RapidsDeviceMemoryBuffer` and if the buffer is not free, we - // take out another handle (in the catalog). - // TODO: This is not robust (to rely on outside locking and addReference/free) - // and should be revisited. - require(contigBuffer.setEventHandler(this) == null, - "DeviceMemoryBuffer with non-null event handler failed to add!!") - - /** - * Override from the MemoryBuffer.EventHandler interface. - * - * If we are being invoked we have the `contigBuffer` lock, as this callback - * is being invoked from `MemoryBuffer.close` - * - * @param refCount - contigBuffer's current refCount - */ - override def onClosed(refCount: Int): Unit = { - // refCount == 1 means only 1 reference exists to `contigBuffer` in the - // RapidsDeviceMemoryBuffer (we own it) - if (refCount == 1) { - // setSpillable is being called here as an extension of `MemoryBuffer.close()` - // we hold the MemoryBuffer lock and we could be called from a Spark task thread - // Since we hold the MemoryBuffer lock, `incRefCount` waits for us. The only other - // call to `setSpillable` is also under this same MemoryBuffer lock (see: - // `getDeviceMemoryBuffer`) - setSpillable(this, true) - } - } - - override protected def releaseResources(): Unit = synchronized { - // we need to disassociate this RapidsBuffer from the underlying buffer - contigBuffer.close() - } - - /** - * Get and increase the reference count of the device memory buffer - * in this RapidsBuffer, while making the RapidsBuffer non-spillable. - * - * @note It is the responsibility of the caller to close the DeviceMemoryBuffer - */ - override def getDeviceMemoryBuffer: DeviceMemoryBuffer = synchronized { - contigBuffer.synchronized { - setSpillable(this, false) - contigBuffer.incRefCount() - contigBuffer - } - } - - override def getMemoryBuffer: MemoryBuffer = getDeviceMemoryBuffer - - override def getColumnarBatch(sparkTypes: Array[DataType]): ColumnarBatch = { - // calling `getDeviceMemoryBuffer` guarantees that we have marked this RapidsBuffer - // as not spillable and increased its refCount atomically - withResource(getDeviceMemoryBuffer) { buff => - columnarBatchFromDeviceBuffer(buff, sparkTypes) - } - } - - /** - * We overwrite free to make sure we don't have a handler for the underlying - * contigBuffer, since this `RapidsBuffer` is no longer tracked. - */ - override def free(): Unit = synchronized { - if (isValid) { - // it is going to be invalid when calling super.free() - contigBuffer.setEventHandler(null) - } - super.free() - } - - override def writeToChannel(outputChannel: WritableByteChannel, stream: Cuda.Stream): Long = { - var written: Long = 0L - val iter = new MemoryBufferToHostByteBufferIterator( - contigBuffer, - hostSpillBounceBuffer, - stream) - iter.foreach { bb => - try { - while (bb.hasRemaining) { - written += outputChannel.write(bb) - } - } finally { - RapidsStorageUtils.dispose(bb) - } - } - written - } - - } - override def close(): Unit = { - try { - super.close() - } finally { - Seq(chunkedPackBounceBuffer, hostSpillBounceBuffer).safeClose() - chunkedPackBounceBuffer = null - hostSpillBounceBuffer = null - } - } -} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDiskStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDiskStore.scala deleted file mode 100644 index eb3692d434a..00000000000 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDiskStore.scala +++ /dev/null @@ -1,256 +0,0 @@ -/* - * Copyright (c) 2020-2024, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids - -import java.io.{File, FileInputStream} -import java.nio.channels.{Channels, FileChannel} -import java.nio.channels.FileChannel.MapMode -import java.nio.file.StandardOpenOption -import java.util.concurrent.ConcurrentHashMap - -import ai.rapids.cudf.{Cuda, HostMemoryBuffer, MemoryBuffer} -import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} -import com.nvidia.spark.rapids.StorageTier.StorageTier -import com.nvidia.spark.rapids.format.TableMeta -import org.apache.commons.io.IOUtils - -import org.apache.spark.TaskContext -import org.apache.spark.sql.rapids.{GpuTaskMetrics, RapidsDiskBlockManager} -import org.apache.spark.sql.rapids.execution.{SerializedHostTableUtils, TrampolineUtil} -import org.apache.spark.sql.types.DataType -import org.apache.spark.sql.vectorized.ColumnarBatch - -/** A buffer store using files on the local disks. */ -class RapidsDiskStore(diskBlockManager: RapidsDiskBlockManager) - extends RapidsBufferStoreWithoutSpill(StorageTier.DISK) { - private[this] val sharedBufferFiles = new ConcurrentHashMap[RapidsBufferId, File] - - private def reportDiskAllocMetrics(metrics: GpuTaskMetrics): String = { - val taskId = TaskContext.get().taskAttemptId() - val totalSize = metrics.getDiskBytesAllocated - val maxSize = metrics.getMaxDiskBytesAllocated - s"total size for task $taskId is $totalSize, max size is $maxSize" - } - - override protected def createBuffer( - incoming: RapidsBuffer, - catalog: RapidsBufferCatalog, - stream: Cuda.Stream): Option[RapidsBufferBase] = { - // assuming that the disk store gets contiguous buffers - val id = incoming.id - val path = if (id.canShareDiskPaths) { - sharedBufferFiles.computeIfAbsent(id, _ => id.getDiskPath(diskBlockManager)) - } else { - id.getDiskPath(diskBlockManager) - } - - val (fileOffset, uncompressedSize, diskLength) = if (id.canShareDiskPaths) { - // only one writer at a time for now when using shared files - path.synchronized { - writeToFile(incoming, path, append = true, stream) - } - } else { - writeToFile(incoming, path, append = false, stream) - } - logDebug(s"Spilled to $path $fileOffset:$diskLength") - val buff = incoming match { - case _: RapidsHostBatchBuffer => - new RapidsDiskColumnarBatch( - id, - fileOffset, - uncompressedSize, - diskLength, - incoming.meta, - incoming.getSpillPriority) - - case _ => - new RapidsDiskBuffer( - id, - fileOffset, - uncompressedSize, - diskLength, - incoming.meta, - incoming.getSpillPriority) - } - TrampolineUtil.incTaskMetricsDiskBytesSpilled(uncompressedSize) - - val metrics = GpuTaskMetrics.get - metrics.incDiskBytesAllocated(uncompressedSize) - logDebug(s"acquiring resources for disk buffer $id of size $uncompressedSize bytes") - logDebug(reportDiskAllocMetrics(metrics)) - Some(buff) - } - - /** - * Copy a host buffer to a file. It leverages [[RapidsSerializerManager]] from - * [[RapidsDiskBlockManager]] to do compression or encryption if needed. - * - * @param incoming the rapid buffer to be written into a file - * @param path file path - * @param append whether to append or written into the beginning of the file - * @param stream cuda stream - * @return a tuple of file offset, memory byte size and written size on disk. File offset is where - * buffer starts in the targeted file path. Memory byte size is the size of byte buffer - * occupied in memory before writing to disk. Written size on disk is actual byte size - * written to disk. - */ - private def writeToFile( - incoming: RapidsBuffer, - path: File, - append: Boolean, - stream: Cuda.Stream): (Long, Long, Long) = { - incoming match { - case fileWritable: RapidsBufferChannelWritable => - val option = if (append) { - Array(StandardOpenOption.CREATE, StandardOpenOption.APPEND) - } else { - Array(StandardOpenOption.CREATE, StandardOpenOption.WRITE) - } - var currentPos, writtenBytes = 0L - - GpuTaskMetrics.get.spillToDiskTime { - withResource(FileChannel.open(path.toPath, option: _*)) { fc => - currentPos = fc.position() - withResource(Channels.newOutputStream(fc)) { os => - withResource(diskBlockManager.getSerializerManager() - .wrapStream(incoming.id, os)) { cos => - val outputChannel = Channels.newChannel(cos) - writtenBytes = fileWritable.writeToChannel(outputChannel, stream) - } - } - (currentPos, writtenBytes, path.length() - currentPos) - } - } - case other => - throw new IllegalStateException( - s"Unable to write $other to file") - } - } - - /** - * A RapidsDiskBuffer that is mean to represent device-bound memory. This - * buffer can produce a device-backed ColumnarBatch. - */ - class RapidsDiskBuffer( - id: RapidsBufferId, - fileOffset: Long, - uncompressedSize: Long, - onDiskSizeInBytes: Long, - meta: TableMeta, - spillPriority: Long) - extends RapidsBufferBase(id, meta, spillPriority) { - - // FIXME: Need to be clean up. Tracked in https://github.com/NVIDIA/spark-rapids/issues/9496 - override val memoryUsedBytes: Long = uncompressedSize - - override val storageTier: StorageTier = StorageTier.DISK - - override def getMemoryBuffer: MemoryBuffer = synchronized { - require(onDiskSizeInBytes > 0, - s"$this attempted an invalid 0-byte mmap of a file") - val path = id.getDiskPath(diskBlockManager) - val serializerManager = diskBlockManager.getSerializerManager() - val memBuffer = if (serializerManager.isRapidsSpill(id)) { - // Only go through serializerManager's stream wrapper for spill case - closeOnExcept(HostAlloc.alloc(uncompressedSize)) { - decompressed => GpuTaskMetrics.get.readSpillFromDiskTime { - withResource(FileChannel.open(path.toPath, StandardOpenOption.READ)) { c => - c.position(fileOffset) - withResource(Channels.newInputStream(c)) { compressed => - withResource(serializerManager.wrapStream(id, compressed)) { in => - withResource(new HostMemoryOutputStream(decompressed)) { out => - IOUtils.copy(in, out) - } - decompressed - } - } - } - } - } - } else { - // Reserved mmap read fashion for UCX shuffle path. Also it's skipping encryption and - // compression. - HostMemoryBuffer.mapFile(path, MapMode.READ_WRITE, fileOffset, onDiskSizeInBytes) - } - memBuffer - } - - override def close(): Unit = synchronized { - super.close() - } - - override protected def releaseResources(): Unit = { - logDebug(s"releasing resources for disk buffer $id of size $memoryUsedBytes bytes") - val metrics = GpuTaskMetrics.get - metrics.decDiskBytesAllocated(memoryUsedBytes) - logDebug(reportDiskAllocMetrics(metrics)) - - // Buffers that share paths must be cleaned up elsewhere - if (id.canShareDiskPaths) { - sharedBufferFiles.remove(id) - } else { - val path = id.getDiskPath(diskBlockManager) - if (!path.delete() && path.exists()) { - logWarning(s"Unable to delete spill path $path") - } - } - } - } - - /** - * A RapidsDiskBuffer that should remain in the host, producing host-backed - * ColumnarBatch if the caller invokes getHostColumnarBatch, but not producing - * anything on the device. - */ - class RapidsDiskColumnarBatch( - id: RapidsBufferId, - fileOffset: Long, - size: Long, - uncompressedSize: Long, - // TODO: remove meta - meta: TableMeta, - spillPriority: Long) - extends RapidsDiskBuffer( - id, fileOffset, size, uncompressedSize, meta, spillPriority) - with RapidsHostBatchBuffer { - - override def getMemoryBuffer: MemoryBuffer = - throw new IllegalStateException( - "Called getMemoryBuffer on a disk buffer that needs deserialization") - - override def getColumnarBatch(sparkTypes: Array[DataType]): ColumnarBatch = - throw new IllegalStateException( - "Called getColumnarBatch on a disk buffer that needs deserialization") - - override def getHostColumnarBatch(sparkTypes: Array[DataType]): ColumnarBatch = { - require(fileOffset == 0, - "Attempted to obtain a HostColumnarBatch from a spilled RapidsBuffer that is sharing " + - "paths on disk") - val path = id.getDiskPath(diskBlockManager) - withResource(new FileInputStream(path)) { fis => - withResource(diskBlockManager.getSerializerManager() - .wrapStream(id, fis)) { fs => - val (header, hostBuffer) = SerializedHostTableUtils.readTableHeaderAndBuffer(fs) - val hostCols = withResource(hostBuffer) { _ => - SerializedHostTableUtils.buildHostColumns(header, hostBuffer, sparkTypes) - } - new ColumnarBatch(hostCols.toArray, header.getNumRows) - } - } - } - } -} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsHostMemoryStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsHostMemoryStore.scala deleted file mode 100644 index 235ed9ddb45..00000000000 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsHostMemoryStore.scala +++ /dev/null @@ -1,484 +0,0 @@ -/* - * Copyright (c) 2020-2024, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids - -import java.io.DataOutputStream -import java.nio.channels.{Channels, WritableByteChannel} -import java.util.concurrent.ConcurrentHashMap - -import scala.collection.mutable - -import ai.rapids.cudf.{Cuda, DeviceMemoryBuffer, HostColumnVector, HostMemoryBuffer, JCudfSerialization, MemoryBuffer} -import com.nvidia.spark.rapids.Arm.{closeOnExcept, freeOnExcept, withResource} -import com.nvidia.spark.rapids.SpillPriorities.{applyPriorityOffset, HOST_MEMORY_BUFFER_SPILL_OFFSET} -import com.nvidia.spark.rapids.StorageTier.StorageTier -import com.nvidia.spark.rapids.format.TableMeta - -import org.apache.spark.TaskContext -import org.apache.spark.sql.rapids.GpuTaskMetrics -import org.apache.spark.sql.rapids.storage.RapidsStorageUtils -import org.apache.spark.sql.types.DataType -import org.apache.spark.sql.vectorized.ColumnarBatch - -/** - * A buffer store using host memory. - * @param maxSize maximum size in bytes for all buffers in this store - */ -class RapidsHostMemoryStore( - maxSize: Option[Long]) - extends RapidsBufferStore(StorageTier.HOST) { - - override protected def spillableOnAdd: Boolean = false - - override def getMaxSize: Option[Long] = maxSize - - def addBuffer( - id: RapidsBufferId, - buffer: HostMemoryBuffer, - tableMeta: TableMeta, - initialSpillPriority: Long, - needsSync: Boolean): RapidsBuffer = { - buffer.incRefCount() - val rapidsBuffer = new RapidsHostMemoryBuffer( - id, - buffer.getLength, - tableMeta, - initialSpillPriority, - buffer) - freeOnExcept(rapidsBuffer) { _ => - logDebug(s"Adding host buffer for: [id=$id, size=${buffer.getLength}, " + - s"uncompressed=${rapidsBuffer.meta.bufferMeta.uncompressedSize}, " + - s"meta_id=${tableMeta.bufferMeta.id}, " + - s"meta_size=${tableMeta.bufferMeta.size}]") - addBuffer(rapidsBuffer, needsSync) - rapidsBuffer - } - } - - def addBatch(id: RapidsBufferId, - hostCb: ColumnarBatch, - initialSpillPriority: Long, - needsSync: Boolean): RapidsBuffer = { - RapidsHostColumnVector.incRefCounts(hostCb) - val rapidsBuffer = new RapidsHostColumnarBatch( - id, - hostCb, - initialSpillPriority) - freeOnExcept(rapidsBuffer) { _ => - addBuffer(rapidsBuffer, needsSync) - rapidsBuffer - } - } - - override protected def trySpillToMaximumSize( - buffer: RapidsBuffer, - catalog: RapidsBufferCatalog, - stream: Cuda.Stream): Boolean = { - maxSize.forall { ms => - // this spillStore has a maximum size requirement (host only). We need to spill from it - // in order to make room for `buffer`. - val targetTotalSize = ms - buffer.memoryUsedBytes - if (targetTotalSize < 0) { - // lets not spill to host when the buffer we are about - // to spill is larger than our limit - false - } else { - val amountSpilled = synchronousSpill(targetTotalSize, catalog, stream) - if (amountSpilled != 0) { - logDebug(s"Task ${TaskContext.get.taskAttemptId()} spilled $amountSpilled bytes from" + - s"${name} to make room for ${buffer.id}") - } - // if after spill we can fit the new buffer, return true - buffer.memoryUsedBytes <= (ms - currentSize) - } - } - } - - override protected def createBuffer( - other: RapidsBuffer, - catalog: RapidsBufferCatalog, - stream: Cuda.Stream): Option[RapidsBufferBase] = { - val wouldFit = trySpillToMaximumSize(other, catalog, stream) - if (!wouldFit) { - // skip host - logWarning(s"Buffer $other with size ${other.memoryUsedBytes} does not fit " + - s"in the host store, skipping tier.") - None - } else { - // If the other is from the local disk store, we are unspilling to host memory. - if (other.storageTier == StorageTier.DISK) { - logDebug(s"copying RapidsDiskStore buffer ${other.id} to a HostMemoryBuffer") - val hostBuffer = other.getMemoryBuffer.asInstanceOf[HostMemoryBuffer] - Some(new RapidsHostMemoryBuffer( - other.id, - hostBuffer.getLength(), - other.meta, - applyPriorityOffset(other.getSpillPriority, HOST_MEMORY_BUFFER_SPILL_OFFSET), - hostBuffer)) - } else { - withResource(other.getCopyIterator) { otherBufferIterator => - val isChunked = otherBufferIterator.isChunked - val totalCopySize = otherBufferIterator.getTotalCopySize - closeOnExcept(HostAlloc.tryAlloc(totalCopySize)) { hb => - hb.map { hostBuffer => - val spillNs = GpuTaskMetrics.get.spillToHostTime { - var hostOffset = 0L - val start = System.nanoTime() - while (otherBufferIterator.hasNext) { - val otherBuffer = otherBufferIterator.next() - withResource(otherBuffer) { _ => - otherBuffer match { - case devBuffer: DeviceMemoryBuffer => - hostBuffer.copyFromMemoryBufferAsync( - hostOffset, devBuffer, 0, otherBuffer.getLength, stream) - hostOffset += otherBuffer.getLength - case _ => - throw new IllegalStateException("copying from buffer without device memory") - } - } - } - stream.sync() - System.nanoTime() - start - } - val szMB = (totalCopySize.toDouble / 1024.0 / 1024.0).toLong - val bw = (szMB.toDouble / (spillNs.toDouble / 1000000000.0)).toLong - logDebug(s"Spill to host (chunked=$isChunked) " + - s"size=$szMB MiB bandwidth=$bw MiB/sec") - new RapidsHostMemoryBuffer( - other.id, - totalCopySize, - other.meta, - applyPriorityOffset(other.getSpillPriority, HOST_MEMORY_BUFFER_SPILL_OFFSET), - hostBuffer) - }.orElse { - // skip host - logWarning(s"Buffer $other with size ${other.memoryUsedBytes} does not fit " + - s"in the host store, skipping tier.") - None - } - } - } - } - } - } - - def numBytesFree: Option[Long] = maxSize.map(_ - currentSize) - - class RapidsHostMemoryBuffer( - id: RapidsBufferId, - size: Long, - meta: TableMeta, - spillPriority: Long, - buffer: HostMemoryBuffer) - extends RapidsBufferBase(id, meta, spillPriority) - with RapidsBufferChannelWritable - with MemoryBuffer.EventHandler { - override val storageTier: StorageTier = StorageTier.HOST - - override def getMemoryBuffer: MemoryBuffer = getHostMemoryBuffer - - override def getHostMemoryBuffer: HostMemoryBuffer = synchronized { - buffer.synchronized { - setSpillable(this, false) - buffer.incRefCount() - buffer - } - } - - override def writeToChannel(outputChannel: WritableByteChannel, ignored: Cuda.Stream): Long = { - var written: Long = 0L - val iter = new HostByteBufferIterator(buffer) - iter.foreach { bb => - try { - while (bb.hasRemaining) { - written += outputChannel.write(bb) - } - } finally { - RapidsStorageUtils.dispose(bb) - } - } - written - } - - override def updateSpillability(): Unit = { - if (buffer.getRefCount == 1) { - setSpillable(this, true) - } - } - - override protected def releaseResources(): Unit = { - buffer.close() - } - - /** The size of this buffer in bytes. */ - override val memoryUsedBytes: Long = size - - // If this require triggers, we are re-adding a `HostMemoryBuffer` outside of - // the catalog lock, which should not possible. The event handler is set to null - // when we free the `RapidsHostMemoryBuffer` and if the buffer is not free, we - // take out another handle (in the catalog). - HostAlloc.addEventHandler(buffer, this) - - /** - * Override from the MemoryBuffer.EventHandler interface. - * - * If we are being invoked we have the `buffer` lock, as this callback - * is being invoked from `MemoryBuffer.close` - * - * @param refCount - buffer's current refCount - */ - override def onClosed(refCount: Int): Unit = { - // refCount == 1 means only 1 reference exists to `buffer` in the - // RapidsHostMemoryBuffer (we own it) - if (refCount == 1) { - // setSpillable is being called here as an extension of `MemoryBuffer.close()` - // we hold the MemoryBuffer lock and we could be called from a Spark task thread - // Since we hold the MemoryBuffer lock, `incRefCount` waits for us. The only other - // call to `setSpillable` is also under this same MemoryBuffer lock (see: - // `getMemoryBuffer`) - setSpillable(this, true) - } - } - - /** - * We overwrite free to make sure we don't have a handler for the underlying - * buffer, since this `RapidsBuffer` is no longer tracked. - */ - override def free(): Unit = synchronized { - if (isValid) { - // it is going to be invalid when calling super.free() - HostAlloc.removeEventHandler(buffer, this) - } - super.free() - } - } - - /** - * A per cuDF host column event handler that handles calls to .close() - * inside of the `HostColumnVector` lock. - */ - class RapidsHostColumnEventHandler - extends HostColumnVector.EventHandler { - - // Every RapidsHostColumnarBatch that references this column has an entry in this map. - // The value represents the number of times (normally 1) that a ColumnVector - // appears in the RapidsHostColumnarBatch. This is also the HosColumnVector refCount at which - // the column is considered spillable. - // The map is protected via the ColumnVector lock. - private val registration = new mutable.HashMap[RapidsHostColumnarBatch, Int]() - - /** - * Every RapidsHostColumnarBatch iterates through its columns and either creates - * a `RapidsHostColumnEventHandler` object and associates it with the column's - * `eventHandler` or calls into the existing one, and registers itself. - * - * The registration has two goals: it accounts for repetition of a column - * in a `RapidsHostColumnarBatch`. If a batch has the same column repeated it must adjust - * the refCount at which this column is considered spillable. - * - * The second goal is to account for aliasing. If two host batches alias this column - * we are going to mark it as non spillable. - * - * @param rapidsHostCb - the host batch that is registering itself with this tracker - */ - def register(rapidsHostCb: RapidsHostColumnarBatch, repetition: Int): Unit = { - registration.put(rapidsHostCb, repetition) - } - - /** - * This is invoked during `RapidsHostColumnarBatch.free` in order to remove the entry - * in `registration`. - * - * @param rapidsHostCb - the batch that is de-registering itself - */ - def deregister(rapidsHostCb: RapidsHostColumnarBatch): Unit = { - registration.remove(rapidsHostCb) - } - - // called with the cudf HostColumnVector lock held from cuDF's side - override def onClosed(cudfCv: HostColumnVector, refCount: Int): Unit = { - // we only handle spillability if there is a single batch registered - // (no aliasing) - if (registration.size == 1) { - val (rapidsHostCb, spillableRefCount) = registration.head - if (spillableRefCount == refCount) { - rapidsHostCb.onColumnSpillable(cudfCv) - } - } - } - } - - /** - * A `RapidsHostColumnarBatch` is the spill store holder of ColumnarBatch backed by - * HostColumnVector. - * - * This class owns the host batch and will close it when `close` is called. - * - * @param id the `RapidsBufferId` this batch is associated with - * @param batch the host ColumnarBatch we are managing - * @param spillPriority a starting spill priority - */ - class RapidsHostColumnarBatch( - id: RapidsBufferId, - hostCb: ColumnarBatch, - spillPriority: Long) - extends RapidsBufferBase( - id, - null, - spillPriority) - with RapidsBufferChannelWritable - with RapidsHostBatchBuffer { - - override val storageTier: StorageTier = StorageTier.HOST - - // By default all columns are NOT spillable since we are not the only owners of - // the columns (the caller is holding onto a ColumnarBatch that will be closed - // after instantiation, triggering onClosed callbacks) - // This hash set contains the columns that are currently spillable. - private val columnSpillability = new ConcurrentHashMap[HostColumnVector, Boolean]() - - private val numDistinctColumns = RapidsHostColumnVector.extractBases(hostCb).distinct.size - - // we register our event callbacks as the very first action to deal with - // spillability - registerOnCloseEventHandler() - - /** Release the underlying resources for this buffer. */ - override protected def releaseResources(): Unit = { - hostCb.close() - } - - override def meta: TableMeta = { - null - } - - // This is the current size in batch form. It is to be used while this - // batch hasn't migrated to another store. - override val memoryUsedBytes: Long = RapidsHostColumnVector.getTotalHostMemoryUsed(hostCb) - - /** - * Mark a column as spillable - * - * @param column the ColumnVector to mark as spillable - */ - def onColumnSpillable(column: HostColumnVector): Unit = { - columnSpillability.put(column, true) - updateSpillability() - } - - /** - * Update the spillability state of this RapidsHostColumnarBatch. This is invoked from - * two places: - * - * - from the onColumnSpillable callback, which is invoked from a - * HostColumnVector.EventHandler.onClosed callback. - * - * - after adding a batch to the store to mark the batch as spillable if - * all columns are spillable. - */ - override def updateSpillability(): Unit = { - setSpillable(this, columnSpillability.size == numDistinctColumns) - } - - override def getColumnarBatch(sparkTypes: Array[DataType]): ColumnarBatch = { - throw new UnsupportedOperationException( - "RapidsHostColumnarBatch does not support getColumnarBatch") - } - - override def getHostColumnarBatch(sparkTypes: Array[DataType]): ColumnarBatch = { - columnSpillability.clear() - setSpillable(this, false) - RapidsHostColumnVector.incRefCounts(hostCb) - } - - override def getMemoryBuffer: MemoryBuffer = { - throw new UnsupportedOperationException( - "RapidsHostColumnarBatch does not support getMemoryBuffer") - } - - override def getCopyIterator: RapidsBufferCopyIterator = { - throw new UnsupportedOperationException( - "RapidsHostColumnarBatch does not support getCopyIterator") - } - - override def writeToChannel(outputChannel: WritableByteChannel, ignored: Cuda.Stream): Long = { - withResource(Channels.newOutputStream(outputChannel)) { outputStream => - withResource(new DataOutputStream(outputStream)) { dos => - val columns = RapidsHostColumnVector.extractBases(hostCb) - JCudfSerialization.writeToStream(columns, dos, 0, hostCb.numRows()) - dos.size() - } - } - } - - override def free(): Unit = { - // lets remove our handler from the chain of handlers for each column - removeOnCloseEventHandler() - super.free() - } - - private def registerOnCloseEventHandler(): Unit = { - val columns = RapidsHostColumnVector.extractBases(hostCb) - // cudfColumns could contain duplicates. We need to take this into account when we are - // deciding the floor refCount for a duplicated column - val repetitionPerColumn = new mutable.HashMap[HostColumnVector, Int]() - columns.foreach { col => - val repetitionCount = repetitionPerColumn.getOrElse(col, 0) - repetitionPerColumn(col) = repetitionCount + 1 - } - repetitionPerColumn.foreach { case (distinctCv, repetition) => - // lock the column because we are setting its event handler, and we are inspecting - // its refCount. - distinctCv.synchronized { - val eventHandler = distinctCv.getEventHandler match { - case null => - val eventHandler = new RapidsHostColumnEventHandler - distinctCv.setEventHandler(eventHandler) - eventHandler - case existing: RapidsHostColumnEventHandler => - existing - case other => - throw new IllegalStateException( - s"Invalid column event handler $other") - } - eventHandler.register(this, repetition) - if (repetition == distinctCv.getRefCount) { - onColumnSpillable(distinctCv) - } - } - } - } - - // this method is called from free() - private def removeOnCloseEventHandler(): Unit = { - val distinctColumns = RapidsHostColumnVector.extractBases(hostCb).distinct - distinctColumns.foreach { distinctCv => - distinctCv.synchronized { - distinctCv.getEventHandler match { - case eventHandler: RapidsHostColumnEventHandler => - eventHandler.deregister(this) - case t => - throw new IllegalStateException( - s"Invalid column event handler $t") - } - } - } - } - } -} - - diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsSerializerManager.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsSerializerManager.scala index ab4a6398d32..74aed062142 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsSerializerManager.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsSerializerManager.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,8 +20,8 @@ import java.io.{InputStream, OutputStream} import org.apache.spark.SparkConf import org.apache.spark.io.CompressionCodec -import org.apache.spark.sql.rapids.TempSpillBufferId import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.storage.BlockId /** @@ -44,22 +44,20 @@ class RapidsSerializerManager (conf: SparkConf) { private lazy val compressionCodec: CompressionCodec = TrampolineUtil.createCodec(conf) - // Whether it really goes through crypto streams replies on Spark configuration - // (e.g., `` `spark.io.encryption.enabled` ``) and the existence of crypto keys. - def wrapStream(bufferId: RapidsBufferId, s: OutputStream): OutputStream = { - if(isRapidsSpill(bufferId)) wrapForCompression(bufferId, wrapForEncryption(s)) else s + def wrapStream(blockId: BlockId, s: OutputStream): OutputStream = { + if(isRapidsSpill(blockId)) wrapForCompression(blockId, wrapForEncryption(s)) else s } - def wrapStream(bufferId: RapidsBufferId, s: InputStream): InputStream = { - if(isRapidsSpill(bufferId)) wrapForCompression(bufferId, wrapForEncryption(s)) else s + def wrapStream(blockId: BlockId, s: InputStream): InputStream = { + if(isRapidsSpill(blockId)) wrapForCompression(blockId, wrapForEncryption(s)) else s } - private[this] def wrapForCompression(bufferId: RapidsBufferId, s: InputStream): InputStream = { - if (shouldCompress(bufferId)) compressionCodec.compressedInputStream(s) else s + private[this] def wrapForCompression(blockId: BlockId, s: InputStream): InputStream = { + if (shouldCompress(blockId)) compressionCodec.compressedInputStream(s) else s } - private[this] def wrapForCompression(bufferId: RapidsBufferId, s: OutputStream): OutputStream = { - if (shouldCompress(bufferId)) compressionCodec.compressedOutputStream(s) else s + private[this] def wrapForCompression(blockId: BlockId, s: OutputStream): OutputStream = { + if (shouldCompress(blockId)) compressionCodec.compressedOutputStream(s) else s } private[this] def wrapForEncryption(s: InputStream): InputStream = { @@ -70,18 +68,15 @@ class RapidsSerializerManager (conf: SparkConf) { if (serializerManager != null) serializerManager.wrapForEncryption(s) else s } - def isRapidsSpill(bufferId: RapidsBufferId): Boolean = { - bufferId match { - case _: TempSpillBufferId => true - case _ => false - } + def isRapidsSpill(blockId: BlockId): Boolean = { + !blockId.isShuffle } - private[this] def shouldCompress(bufferId: RapidsBufferId): Boolean = { - bufferId match { - case _: TempSpillBufferId => compressSpill - case _: ShuffleBufferId | _: ShuffleReceivedBufferId => false - case _ => false + private[this] def shouldCompress(blockId: BlockId): Boolean = { + if (!blockId.isShuffle) { + compressSpill + } else { + false } } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShuffleBufferCatalog.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShuffleBufferCatalog.scala index a587c5cd7ae..542fb2deb2d 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShuffleBufferCatalog.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShuffleBufferCatalog.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,52 +16,76 @@ package com.nvidia.spark.rapids -import java.io.File import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicInteger import java.util.function.{Consumer, IntUnaryOperator} import scala.collection.mutable.ArrayBuffer -import ai.rapids.cudf.{ContiguousTable, Cuda, DeviceMemoryBuffer} +import ai.rapids.cudf.{ContiguousTable, Cuda, DeviceMemoryBuffer, Table} import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.format.TableMeta +import com.nvidia.spark.rapids.spill.{SpillableDeviceBufferHandle, SpillableHandle} -import org.apache.spark.SparkEnv +import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.internal.Logging -import org.apache.spark.sql.rapids.RapidsDiskBlockManager import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.storage.ShuffleBlockId /** Identifier for a shuffle buffer that holds the data for a table */ case class ShuffleBufferId( blockId: ShuffleBlockId, - override val tableId: Int) extends RapidsBufferId { + tableId: Int) { val shuffleId: Int = blockId.shuffleId val mapId: Long = blockId.mapId - - override val canShareDiskPaths: Boolean = true - - override def getDiskPath(diskBlockManager: RapidsDiskBlockManager): File = { - diskBlockManager.getFile(blockId) - } } /** Catalog for lookup of shuffle buffers by block ID */ -class ShuffleBufferCatalog( - catalog: RapidsBufferCatalog, - diskBlockManager: RapidsDiskBlockManager) extends Logging { +class ShuffleBufferCatalog extends Logging { + /** + * Information stored for each active shuffle. + * A shuffle block can be comprised of multiple batches. Each batch + * is given a `ShuffleBufferId`. + */ + private type ShuffleInfo = + ConcurrentHashMap[ShuffleBlockId, ArrayBuffer[ShuffleBufferId]] + + private val bufferIdToHandle = + new ConcurrentHashMap[ + ShuffleBufferId, + (Option[SpillableDeviceBufferHandle], TableMeta)]() - private val bufferIdToHandle = new ConcurrentHashMap[RapidsBufferId, RapidsBufferHandle]() + /** shuffle information for each active shuffle */ + private[this] val activeShuffles = new ConcurrentHashMap[Int, ShuffleInfo] + + /** Mapping of table ID to shuffle buffer ID */ + private[this] val tableMap = new ConcurrentHashMap[Int, ShuffleBufferId] + + /** Tracks the next table identifier */ + private[this] val tableIdCounter = new AtomicInteger(0) private def trackCachedHandle( bufferId: ShuffleBufferId, - bufferHandle: RapidsBufferHandle): Unit = { - bufferIdToHandle.put(bufferId, bufferHandle) + handle: SpillableDeviceBufferHandle, + meta: TableMeta): Unit = { + bufferIdToHandle.put(bufferId, (Some(handle), meta)) + } + + private def trackDegenerate(bufferId: ShuffleBufferId, + meta: TableMeta): Unit = { + bufferIdToHandle.put(bufferId, (None, meta)) } def removeCachedHandles(): Unit = { - bufferIdToHandle.forEach { (_, handle) => removeBuffer(handle) } + val bufferIt = bufferIdToHandle.keySet().iterator() + while (bufferIt.hasNext) { + val buffer = bufferIt.next() + val (maybeHandle, _) = bufferIdToHandle.remove(buffer) + tableMap.remove(buffer.tableId) + maybeHandle.foreach(_.close()) + } } /** @@ -70,56 +94,47 @@ class ShuffleBufferCatalog( * The refcount of the underlying device buffer will be incremented so the contiguous table * can be closed before this buffer is destroyed. * - * @param blockId Spark's `ShuffleBlockId` that identifies this buffer - * @param contigTable contiguous table to track in storage + * @param blockId Spark's `ShuffleBlockId` that identifies this buffer + * @param contigTable contiguous table to track in storage * @param initialSpillPriority starting spill priority value for the buffer - * @param needsSync whether the spill framework should stream synchronize while adding - * this device buffer (defaults to true) * @return RapidsBufferHandle identifying this table */ - def addContiguousTable( - blockId: ShuffleBlockId, - contigTable: ContiguousTable, - initialSpillPriority: Long, - needsSync: Boolean): RapidsBufferHandle = { - val bufferId = nextShuffleBufferId(blockId) + def addContiguousTable(blockId: ShuffleBlockId, + contigTable: ContiguousTable, + initialSpillPriority: Long): Unit = { withResource(contigTable) { _ => - val handle = catalog.addContiguousTable( - bufferId, - contigTable, - initialSpillPriority, - needsSync) - trackCachedHandle(bufferId, handle) - handle + val bufferId = nextShuffleBufferId(blockId) + val tableMeta = MetaUtils.buildTableMeta(bufferId.tableId, contigTable) + val buff = contigTable.getBuffer + buff.incRefCount() + val handle = SpillableDeviceBufferHandle(buff) + trackCachedHandle(bufferId, handle, tableMeta) } } /** * Adds a buffer to the device storage, taking ownership of the buffer. * - * @param blockId Spark's `ShuffleBlockId` that identifies this buffer - * @param buffer buffer that will be owned by the store - * @param tableMeta metadata describing the buffer layout + * @param blockId Spark's `ShuffleBlockId` that identifies this buffer + * @param compressedBatch Compressed ColumnarBatch * @param initialSpillPriority starting spill priority value for the buffer * @return RapidsBufferHandle associated with this buffer */ - def addBuffer( - blockId: ShuffleBlockId, - buffer: DeviceMemoryBuffer, - tableMeta: TableMeta, - initialSpillPriority: Long, - needsSync: Boolean): RapidsBufferHandle = { - val bufferId = nextShuffleBufferId(blockId) - // update the table metadata for the buffer ID generated above - tableMeta.bufferMeta.mutateId(bufferId.tableId) - val handle = catalog.addBuffer( - bufferId, - buffer, - tableMeta, - initialSpillPriority, - needsSync) - trackCachedHandle(bufferId, handle) - handle + def addCompressedBatch( + blockId: ShuffleBlockId, + compressedBatch: ColumnarBatch, + initialSpillPriority: Long): Unit = { + withResource(compressedBatch) { _ => + val bufferId = nextShuffleBufferId(blockId) + val compressed = compressedBatch.column(0).asInstanceOf[GpuCompressedColumnVector] + val tableMeta = compressed.getTableMeta + // update the table metadata for the buffer ID generated above + tableMeta.bufferMeta().mutateId(bufferId.tableId) + val buff = compressed.getTableBuffer + buff.incRefCount() + val handle = SpillableDeviceBufferHandle(buff) + trackCachedHandle(bufferId, handle, tableMeta) + } } /** @@ -128,39 +143,18 @@ class ShuffleBufferCatalog( */ def addDegenerateRapidsBuffer( blockId: ShuffleBlockId, - meta: TableMeta): RapidsBufferHandle = { + meta: TableMeta): Unit = { val bufferId = nextShuffleBufferId(blockId) - val handle = catalog.registerDegenerateBuffer(bufferId, meta) - trackCachedHandle(bufferId, handle) - handle + trackDegenerate(bufferId, meta) } - /** - * Information stored for each active shuffle. - * NOTE: ArrayBuffer in blockMap must be explicitly locked when using it! - * - * @param blockMap mapping of block ID to array of buffers for the block - */ - private case class ShuffleInfo( - blockMap: ConcurrentHashMap[ShuffleBlockId, ArrayBuffer[ShuffleBufferId]]) - - /** shuffle information for each active shuffle */ - private[this] val activeShuffles = new ConcurrentHashMap[Int, ShuffleInfo] - - /** Mapping of table ID to shuffle buffer ID */ - private[this] val tableMap = new ConcurrentHashMap[Int, ShuffleBufferId] - - /** Tracks the next table identifier */ - private[this] val tableIdCounter = new AtomicInteger(0) - /** * Register a new shuffle. * This must be called before any buffer identifiers associated with this shuffle can be tracked. * @param shuffleId shuffle identifier */ def registerShuffle(shuffleId: Int): Unit = { - activeShuffles.computeIfAbsent(shuffleId, _ => ShuffleInfo( - new ConcurrentHashMap[ShuffleBlockId, ArrayBuffer[ShuffleBufferId]])) + activeShuffles.computeIfAbsent(shuffleId, _ => new ShuffleInfo) } /** Frees all buffers that correspond to the specified shuffle. */ @@ -174,22 +168,11 @@ class ShuffleBufferCatalog( // NOTE: Not synchronizing array buffer because this shuffle should be inactive. bufferIds.foreach { id => tableMap.remove(id.tableId) - val handle = bufferIdToHandle.remove(id) - if (handle != null) { - handle.close() - } - } - } - info.blockMap.forEachValue(Long.MaxValue, bufferRemover) - - val fileRemover: Consumer[ShuffleBlockId] = { blockId => - val file = diskBlockManager.getFile(blockId) - logDebug(s"Deleting file $file") - if (!file.delete() && file.exists()) { - logWarning(s"Unable to delete $file") + val handleAndMeta = bufferIdToHandle.remove(id) + handleAndMeta._1.foreach(_.close()) } } - info.blockMap.forEachKey(Long.MaxValue, fileRemover) + info.forEachValue(Long.MaxValue, bufferRemover) } else { // currently shuffle unregister can get called on the driver which never saw a register if (!TrampolineUtil.isDriver(SparkEnv.get)) { @@ -201,12 +184,12 @@ class ShuffleBufferCatalog( def hasActiveShuffle(shuffleId: Int): Boolean = activeShuffles.containsKey(shuffleId) /** Get all the buffer IDs that correspond to a shuffle block identifier. */ - def blockIdToBuffersIds(blockId: ShuffleBlockId): Array[ShuffleBufferId] = { + private def blockIdToBuffersIds(blockId: ShuffleBlockId): Array[ShuffleBufferId] = { val info = activeShuffles.get(blockId.shuffleId) if (info == null) { - throw new NoSuchElementException(s"unknown shuffle $blockId.shuffleId") + throw new NoSuchElementException(s"unknown shuffle ${blockId.shuffleId}") } - val entries = info.blockMap.get(blockId) + val entries = info.get(blockId) if (entries == null) { throw new NoSuchElementException(s"unknown shuffle block $blockId") } @@ -215,27 +198,61 @@ class ShuffleBufferCatalog( } } - def blockIdToBufferHandles(blockId: ShuffleBlockId): Array[RapidsBufferHandle] = { + def getColumnarBatchIterator( + blockId: ShuffleBlockId, + sparkTypes: Array[DataType]): Iterator[ColumnarBatch] = { + val bufferIDs = blockIdToBuffersIds(blockId) + bufferIDs.iterator.map { bId => + GpuSemaphore.acquireIfNecessary(TaskContext.get) + val (maybeHandle, meta) = bufferIdToHandle.get(bId) + maybeHandle.map { handle => + withResource(handle.materialize()) { buff => + val bufferMeta = meta.bufferMeta() + if (bufferMeta == null || bufferMeta.codecBufferDescrsLength == 0) { + MetaUtils.getBatchFromMeta(buff, meta, sparkTypes) + } else { + GpuCompressedColumnVector.from(buff, meta) + } + } + }.getOrElse { + // degenerate table (handle is None) + // make a batch out of denegerate meta + val rowCount = meta.rowCount + val packedMeta = meta.packedMetaAsByteBuffer() + if (packedMeta != null) { + withResource(DeviceMemoryBuffer.allocate(0)) { deviceBuffer => + withResource(Table.fromPackedTable( + meta.packedMetaAsByteBuffer(), deviceBuffer)) { table => + GpuColumnVectorFromBuffer.from(table, deviceBuffer, meta, sparkTypes) + } + } + } else { + // no packed metadata, must be a table with zero columns + new ColumnarBatch(Array.empty, rowCount.toInt) + } + } + } + } + + /** Get all the buffer metadata that correspond to a shuffle block identifier. */ + def blockIdToMetas(blockId: ShuffleBlockId): Seq[TableMeta] = { val info = activeShuffles.get(blockId.shuffleId) if (info == null) { - throw new NoSuchElementException(s"unknown shuffle $blockId.shuffleId") + throw new NoSuchElementException(s"unknown shuffle ${blockId.shuffleId}") } - val entries = info.blockMap.get(blockId) + val entries = info.get(blockId) if (entries == null) { throw new NoSuchElementException(s"unknown shuffle block $blockId") } - entries.synchronized { - entries.map(bufferIdToHandle.get).toArray - } - } - - /** Get all the buffer metadata that correspond to a shuffle block identifier. */ - def blockIdToMetas(blockId: ShuffleBlockId): Seq[TableMeta] = { - blockIdToBuffersIds(blockId).map(catalog.getBufferMeta) + entries.synchronized { + entries.map(bufferIdToHandle.get).map { case (_, meta) => + meta + } + }.toSeq } /** Allocate a new shuffle buffer identifier and update the shuffle block mapping. */ - def nextShuffleBufferId(blockId: ShuffleBlockId): ShuffleBufferId = { + private def nextShuffleBufferId(blockId: ShuffleBlockId): ShuffleBufferId = { val info = activeShuffles.get(blockId.shuffleId) if (info == null) { throw new IllegalStateException(s"unknown shuffle ${blockId.shuffleId}") @@ -249,7 +266,7 @@ class ShuffleBufferCatalog( } // associate this new buffer with the shuffle block - val blockBufferIds = info.blockMap.computeIfAbsent(blockId, _ => + val blockBufferIds = info.computeIfAbsent(blockId, _ => new ArrayBuffer[ShuffleBufferId]) blockBufferIds.synchronized { blockBufferIds.append(id) @@ -258,35 +275,29 @@ class ShuffleBufferCatalog( } /** Lookup the shuffle buffer handle that corresponds to the specified table identifier. */ - def getShuffleBufferHandle(tableId: Int): RapidsBufferHandle = { + def getShuffleBufferHandle(tableId: Int): RapidsShuffleHandle = { val shuffleBufferId = tableMap.get(tableId) if (shuffleBufferId == null) { throw new NoSuchElementException(s"unknown table ID $tableId") } - bufferIdToHandle.get(shuffleBufferId) + val (maybeHandle, meta) = bufferIdToHandle.get(shuffleBufferId) + maybeHandle match { + case Some(spillable) => + RapidsShuffleHandle(spillable, meta) + case None => + throw new IllegalStateException( + "a buffer handle could not be obtained for a degenerate buffer") + } } /** * Update the spill priority of a shuffle buffer that soon will be read locally. * @param handle shuffle buffer handle of buffer to update */ - def updateSpillPriorityForLocalRead(handle: RapidsBufferHandle): Unit = { - handle.setSpillPriority(SpillPriorities.INPUT_FROM_SHUFFLE_PRIORITY) - } - - /** - * Lookup the shuffle buffer that corresponds to the specified buffer handle and acquire it. - * NOTE: It is the responsibility of the caller to close the buffer. - * @param handle shuffle buffer handle - * @return shuffle buffer that has been acquired - */ - def acquireBuffer(handle: RapidsBufferHandle): RapidsBuffer = { - val buffer = catalog.acquireBuffer(handle) - // Shuffle buffers that have been read are less likely to be read again, - // so update the spill priority based on this access - handle.setSpillPriority(SpillPriorities.getShuffleOutputBufferReadPriority) - buffer - } + // TODO: AB: priorities + //def updateSpillPriorityForLocalRead(handle: RapidsBufferHandle): Unit = { + // handle.setSpillPriority(SpillPriorities.INPUT_FROM_SHUFFLE_PRIORITY) + //} /** * Remove a buffer and table given a buffer handle @@ -294,9 +305,7 @@ class ShuffleBufferCatalog( * the handle being removed is not being utilized by another thread. * @param handle buffer handle */ - def removeBuffer(handle: RapidsBufferHandle): Unit = { - val id = handle.id - tableMap.remove(id.tableId) + def removeBuffer(handle: SpillableHandle): Unit = { handle.close() } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShuffleReceivedBufferCatalog.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShuffleReceivedBufferCatalog.scala index 0ff4f9278be..450622ef3ba 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShuffleReceivedBufferCatalog.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ShuffleReceivedBufferCatalog.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,49 +16,25 @@ package com.nvidia.spark.rapids -import java.io.File -import java.util.concurrent.ConcurrentHashMap -import java.util.concurrent.atomic.AtomicInteger -import java.util.function.IntUnaryOperator - -import ai.rapids.cudf.DeviceMemoryBuffer +import ai.rapids.cudf.{DeviceMemoryBuffer, Table} import com.nvidia.spark.rapids.Arm.withResource +import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableColumn import com.nvidia.spark.rapids.format.TableMeta +import com.nvidia.spark.rapids.spill.SpillableDeviceBufferHandle import org.apache.spark.internal.Logging -import org.apache.spark.sql.rapids.RapidsDiskBlockManager - -/** Identifier for a shuffle buffer that holds the data for a table on the read side */ - -case class ShuffleReceivedBufferId( - override val tableId: Int) extends RapidsBufferId { - override val canShareDiskPaths: Boolean = false +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.vectorized.ColumnarBatch - override def getDiskPath(diskBlockManager: RapidsDiskBlockManager): File = { - diskBlockManager.getFile(s"temp_shuffle_${tableId}") +case class RapidsShuffleHandle( + spillable: SpillableDeviceBufferHandle, tableMeta: TableMeta) extends AutoCloseable { + override def close(): Unit = { + spillable.safeClose() } } /** Catalog for lookup of shuffle buffers by block ID */ -class ShuffleReceivedBufferCatalog( - catalog: RapidsBufferCatalog) extends Logging { - - /** Mapping of table ID to shuffle buffer ID */ - private[this] val tableMap = new ConcurrentHashMap[Int, ShuffleReceivedBufferId] - - /** Tracks the next table identifier */ - private[this] val tableIdCounter = new AtomicInteger(0) - - /** Allocate a new shuffle buffer identifier and update the shuffle block mapping. */ - private def nextShuffleReceivedBufferId(): ShuffleReceivedBufferId = { - val tableId = tableIdCounter.getAndUpdate(ShuffleReceivedBufferCatalog.TABLE_ID_UPDATER) - val id = ShuffleReceivedBufferId(tableId) - val prev = tableMap.put(tableId, id) - if (prev != null) { - throw new IllegalStateException(s"table ID $tableId is already in use") - } - id - } +class ShuffleReceivedBufferCatalog() extends Logging { /** * Adds a buffer to the device storage, taking ownership of the buffer. @@ -70,64 +46,52 @@ class ShuffleReceivedBufferCatalog( * @param initialSpillPriority starting spill priority value for the buffer * @param needsSync tells the store a synchronize in the current stream is required * before storing this buffer - * @return RapidsBufferHandle associated with this buffer + * @return RapidsShuffleHandle associated with this buffer */ def addBuffer( buffer: DeviceMemoryBuffer, tableMeta: TableMeta, - initialSpillPriority: Long, - needsSync: Boolean): RapidsBufferHandle = { - val bufferId = nextShuffleReceivedBufferId() - tableMeta.bufferMeta.mutateId(bufferId.tableId) - // when we call `addBuffer` the store will incRefCount - withResource(buffer) { _ => - catalog.addBuffer( - bufferId, - buffer, - tableMeta, - initialSpillPriority, - needsSync) - } + initialSpillPriority: Long): RapidsShuffleHandle = { + RapidsShuffleHandle(SpillableDeviceBufferHandle(buffer), tableMeta) } /** - * Adds a degenerate buffer (zero rows or columns) + * Adds a degenerate batch (zero rows or columns), described only by metadata. * * @param meta metadata describing the buffer layout - * @return RapidsBufferHandle associated with this buffer + * @return RapidsShuffleHandle associated with this buffer */ - def addDegenerateRapidsBuffer( - meta: TableMeta): RapidsBufferHandle = { - val bufferId = nextShuffleReceivedBufferId() - catalog.registerDegenerateBuffer(bufferId, meta) + def addDegenerateBatch(meta: TableMeta): RapidsShuffleHandle = { + RapidsShuffleHandle(null, meta) } - /** - * Lookup the shuffle buffer that corresponds to the specified shuffle buffer - * handle and acquire it. - * NOTE: It is the responsibility of the caller to close the buffer. - * - * @param handle shuffle buffer handle - * @return shuffle buffer that has been acquired - */ - def acquireBuffer(handle: RapidsBufferHandle): RapidsBuffer = catalog.acquireBuffer(handle) - - /** - * Remove a buffer and table given a buffer handle - * NOTE: This function is not thread safe! The caller should only invoke if - * the handle being removed is not being utilized by another thread. - * @param handle buffer handle - */ - def removeBuffer(handle: RapidsBufferHandle): Unit = { - val id = handle.id - tableMap.remove(id.tableId) - handle.close() - } -} - -object ShuffleReceivedBufferCatalog{ - private val MAX_TABLE_ID = Integer.MAX_VALUE - private val TABLE_ID_UPDATER = new IntUnaryOperator { - override def applyAsInt(i: Int): Int = if (i < MAX_TABLE_ID) i + 1 else 0 + def getColumnarBatchAndRemove(handle: RapidsShuffleHandle, + sparkTypes: Array[DataType]): (ColumnarBatch, Long) = { + withResource(handle) { _ => + val spillable = handle.spillable + var memoryUsedBytes = 0L + val cb = if (spillable != null) { + memoryUsedBytes = spillable.sizeInBytes + withResource(spillable.materialize()) { buff => + MetaUtils.getBatchFromMeta(buff, handle.tableMeta, sparkTypes) + } + } else { + val rowCount = handle.tableMeta.rowCount + val packedMeta = handle.tableMeta.packedMetaAsByteBuffer() + if (packedMeta != null) { + withResource(DeviceMemoryBuffer.allocate(0)) { deviceBuffer => + withResource(Table.fromPackedTable( + handle.tableMeta.packedMetaAsByteBuffer(), deviceBuffer)) { table => + GpuColumnVectorFromBuffer.from( + table, deviceBuffer, handle.tableMeta, sparkTypes) + } + } + } else { + // no packed metadata, must be a table with zero columns + new ColumnarBatch(Array.empty, rowCount.toInt) + } + } + (cb, memoryUsedBytes) + } } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SpillableColumnarBatch.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SpillableColumnarBatch.scala index e1f45c34180..7b247af42d3 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SpillableColumnarBatch.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SpillableColumnarBatch.scala @@ -16,8 +16,9 @@ package com.nvidia.spark.rapids -import ai.rapids.cudf.{ContiguousTable, DeviceMemoryBuffer, HostMemoryBuffer} -import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} +import ai.rapids.cudf.{ContiguousTable, Cuda, DeviceMemoryBuffer, HostMemoryBuffer} +import com.nvidia.spark.rapids.Arm.closeOnExcept +import com.nvidia.spark.rapids.spill.{SpillableColumnarBatchFromBufferHandle, SpillableColumnarBatchHandle, SpillableCompressedColumnarBatchHandle, SpillableDeviceBufferHandle, SpillableHostBufferHandle, SpillableHostColumnarBatchHandle} import org.apache.spark.TaskContext import org.apache.spark.sql.types.DataType @@ -93,7 +94,7 @@ class JustRowsColumnarBatch(numRows: Int) * use `SpillableColumnarBatch.apply` instead. */ class SpillableColumnarBatchImpl ( - handle: RapidsBufferHandle, + handle: SpillableColumnarBatchHandle, rowCount: Int, sparkTypes: Array[DataType]) extends SpillableColumnarBatch { @@ -105,27 +106,128 @@ class SpillableColumnarBatchImpl ( */ override def numRows(): Int = rowCount - private def withRapidsBuffer[T](fn: RapidsBuffer => T): T = { - withResource(RapidsBufferCatalog.acquireBuffer(handle)) { rapidsBuffer => - fn(rapidsBuffer) + override lazy val sizeInBytes: Long = handle.approxSizeInBytes + + /** + * Set a new spill priority. + */ + override def setSpillPriority(priority: Long): Unit = { + // TODO: handle.setSpillPriority(priority) + } + + override def getColumnarBatch(): ColumnarBatch = { + GpuSemaphore.acquireIfNecessary(TaskContext.get()) + handle.materialize(sparkTypes) + } + + override def incRefCount(): SpillableColumnarBatch = { + if (refCount <= 0) { + throw new IllegalStateException("Use after free on SpillableColumnarBatchImpl") + } + refCount += 1 + this + } + + /** + * Remove the `ColumnarBatch` from the cache. + */ + override def close(): Unit = { + refCount -= 1 + if (refCount == 0) { + // closing my reference + handle.close() } + // TODO this is causing problems so we need to look into this + // https://github.com/NVIDIA/spark-rapids/issues/10161 + //else if (refCount < 0) { + // throw new IllegalStateException("Double free on SpillableColumnarBatchImpl") + //} } - override lazy val sizeInBytes: Long = - withRapidsBuffer(_.memoryUsedBytes) + override def toString: String = + s"SCB $handle $rowCount ${sparkTypes.toList} $refCount" +} + +class SpillableCompressedColumnarBatchImpl( + handle: SpillableCompressedColumnarBatchHandle, rowCount: Int) + extends SpillableColumnarBatch { + + private var refCount = 1 + + /** + * The number of rows stored in this batch. + */ + override def numRows(): Int = rowCount + + override lazy val sizeInBytes: Long = handle.compressedSizeInBytes /** * Set a new spill priority. */ override def setSpillPriority(priority: Long): Unit = { - handle.setSpillPriority(priority) + // TODO: handle.setSpillPriority(priority) } override def getColumnarBatch(): ColumnarBatch = { - withRapidsBuffer { rapidsBuffer => - GpuSemaphore.acquireIfNecessary(TaskContext.get()) - rapidsBuffer.getColumnarBatch(sparkTypes) + GpuSemaphore.acquireIfNecessary(TaskContext.get()) + handle.materialize() + } + + override def incRefCount(): SpillableColumnarBatch = { + if (refCount <= 0) { + throw new IllegalStateException("Use after free on SpillableColumnarBatchImpl") } + refCount += 1 + this + } + + /** + * Remove the `ColumnarBatch` from the cache. + */ + override def close(): Unit = { + refCount -= 1 + if (refCount == 0) { + // closing my reference + handle.close() + } + // TODO this is causing problems so we need to look into this + // https://github.com/NVIDIA/spark-rapids/issues/10161 + //else if (refCount < 0) { + // throw new IllegalStateException("Double free on SpillableColumnarBatchImpl") + //} + } + + override def toString: String = + s"SCCB $handle $rowCount $refCount" + + override def dataTypes: Array[DataType] = null +} + +class SpillableColumnarBatchFromBufferImpl( + handle: SpillableColumnarBatchFromBufferHandle, + rowCount: Int, + sparkTypes: Array[DataType]) + extends SpillableColumnarBatch { + private var refCount = 1 + + override def dataTypes: Array[DataType] = sparkTypes + /** + * The number of rows stored in this batch. + */ + override def numRows(): Int = rowCount + + override lazy val sizeInBytes: Long = handle.sizeInBytes + + /** + * Set a new spill priority. + */ + override def setSpillPriority(priority: Long): Unit = { + // TODO: handle.setSpillPriority(priority) + } + + override def getColumnarBatch(): ColumnarBatch = { + GpuSemaphore.acquireIfNecessary(TaskContext.get()) + handle.materialize(dataTypes) } override def incRefCount(): SpillableColumnarBatch = { @@ -147,9 +249,9 @@ class SpillableColumnarBatchImpl ( } // TODO this is causing problems so we need to look into this // https://github.com/NVIDIA/spark-rapids/issues/10161 -// else if (refCount < 0) { -// throw new IllegalStateException("Double free on SpillableColumnarBatchImpl") -// } + //else if (refCount < 0) { + // throw new IllegalStateException("Double free on SpillableColumnarBatchImpl") + //} } override def toString: String = @@ -184,10 +286,9 @@ class JustRowsHostColumnarBatch(numRows: Int) * use `SpillableHostColumnarBatch.apply` instead. */ class SpillableHostColumnarBatchImpl ( - handle: RapidsBufferHandle, + handle: SpillableHostColumnarBatchHandle, rowCount: Int, - sparkTypes: Array[DataType], - catalog: RapidsBufferCatalog) + sparkTypes: Array[DataType]) extends SpillableColumnarBatch { private var refCount = 1 @@ -198,27 +299,17 @@ class SpillableHostColumnarBatchImpl ( */ override def numRows(): Int = rowCount - private def withRapidsHostBatchBuffer[T](fn: RapidsHostBatchBuffer => T): T = { - withResource(catalog.acquireHostBatchBuffer(handle)) { rapidsBuffer => - fn(rapidsBuffer) - } - } - - override lazy val sizeInBytes: Long = { - withRapidsHostBatchBuffer(_.memoryUsedBytes) - } + override lazy val sizeInBytes: Long = handle.approxSizeInBytes /** * Set a new spill priority. */ override def setSpillPriority(priority: Long): Unit = { - handle.setSpillPriority(priority) + // TODO: handle.setSpillPriority(priority) } override def getColumnarBatch(): ColumnarBatch = { - withRapidsHostBatchBuffer { hostBatchBuffer => - hostBatchBuffer.getHostColumnarBatch(sparkTypes) - } + handle.materialize(sparkTypes) } override def incRefCount(): SpillableColumnarBatch = { @@ -257,18 +348,29 @@ object SpillableColumnarBatch { */ def apply(batch: ColumnarBatch, priority: Long): SpillableColumnarBatch = { + Cuda.DEFAULT_STREAM.sync() val numRows = batch.numRows() if (batch.numCols() <= 0) { // We consumed it batch.close() new JustRowsColumnarBatch(numRows) } else { - val types = GpuColumnVector.extractTypes(batch) - val handle = addBatch(batch, priority) - new SpillableColumnarBatchImpl( - handle, - numRows, - types) + if (GpuCompressedColumnVector.isBatchCompressed(batch)) { + new SpillableCompressedColumnarBatchImpl( + SpillableCompressedColumnarBatchHandle(batch), + numRows) + } else if (GpuColumnVectorFromBuffer.isFromBuffer(batch)) { + new SpillableColumnarBatchFromBufferImpl( + SpillableColumnarBatchFromBufferHandle(batch), + numRows, + GpuColumnVector.extractTypes(batch) + ) + } else { + new SpillableColumnarBatchImpl( + SpillableColumnarBatchHandle(batch), + numRows, + GpuColumnVector.extractTypes(batch)) + } } } @@ -283,54 +385,11 @@ object SpillableColumnarBatch { ct: ContiguousTable, sparkTypes: Array[DataType], priority: Long): SpillableColumnarBatch = { - withResource(ct) { _ => - val handle = RapidsBufferCatalog.addContiguousTable(ct, priority) - new SpillableColumnarBatchImpl(handle, ct.getRowCount.toInt, sparkTypes) - } - } - - private[this] def allFromSameBuffer(batch: ColumnarBatch): Boolean = { - var bufferAddr = 0L - var isSet = false - val numColumns = batch.numCols() - (0 until numColumns).forall { i => - batch.column(i) match { - case fb: GpuColumnVectorFromBuffer => - if (!isSet) { - bufferAddr = fb.getBuffer.getAddress - isSet = true - true - } else { - bufferAddr == fb.getBuffer.getAddress - } - case _ => false - } - } - } - - private[this] def addBatch( - batch: ColumnarBatch, - initialSpillPriority: Long): RapidsBufferHandle = { - withResource(batch) { batch => - val numColumns = batch.numCols() - if (GpuCompressedColumnVector.isBatchCompressed(batch)) { - val cv = batch.column(0).asInstanceOf[GpuCompressedColumnVector] - val buff = cv.getTableBuffer - RapidsBufferCatalog.addBuffer(buff, cv.getTableMeta, initialSpillPriority) - } else if (GpuPackedTableColumn.isBatchPacked(batch)) { - val cv = batch.column(0).asInstanceOf[GpuPackedTableColumn] - RapidsBufferCatalog.addContiguousTable( - cv.getContiguousTable, - initialSpillPriority) - } else if (numColumns > 0 && - allFromSameBuffer(batch)) { - val cv = batch.column(0).asInstanceOf[GpuColumnVectorFromBuffer] - val buff = cv.getBuffer - RapidsBufferCatalog.addBuffer(buff, cv.getTableMeta, initialSpillPriority) - } else { - RapidsBufferCatalog.addBatch(batch, initialSpillPriority) - } - } + Cuda.DEFAULT_STREAM.sync() + new SpillableColumnarBatchFromBufferImpl( + SpillableColumnarBatchFromBufferHandle(ct, sparkTypes), + ct.getRowCount.toInt, + sparkTypes) } } @@ -342,10 +401,7 @@ object SpillableHostColumnarBatch { * @param batch the batch to make spillable * @param priority the initial spill priority of this batch */ - def apply( - batch: ColumnarBatch, - priority: Long, - catalog: RapidsBufferCatalog = RapidsBufferCatalog.singleton): SpillableColumnarBatch = { + def apply(batch: ColumnarBatch, priority: Long): SpillableColumnarBatch = { val numRows = batch.numRows() if (batch.numCols() <= 0) { // We consumed it @@ -353,45 +409,30 @@ object SpillableHostColumnarBatch { new JustRowsHostColumnarBatch(numRows) } else { val types = RapidsHostColumnVector.extractColumns(batch).map(_.dataType()) - val handle = addHostBatch(batch, priority, catalog) - new SpillableHostColumnarBatchImpl( - handle, - numRows, - types, - catalog) - } - } - - private[this] def addHostBatch( - batch: ColumnarBatch, - initialSpillPriority: Long, - catalog: RapidsBufferCatalog): RapidsBufferHandle = { - withResource(batch) { batch => - catalog.addBatch(batch, initialSpillPriority) + val handle = SpillableHostColumnarBatchHandle(batch) + new SpillableHostColumnarBatchImpl(handle, numRows, types) } } - } + /** * Just like a SpillableColumnarBatch but for buffers. */ class SpillableBuffer( - handle: RapidsBufferHandle) extends AutoCloseable { + handle: SpillableDeviceBufferHandle) extends AutoCloseable { /** * Set a new spill priority. */ def setSpillPriority(priority: Long): Unit = { - handle.setSpillPriority(priority) + // TODO: handle.setSpillPriority(priority) } /** * Use the device buffer. */ def getDeviceBuffer(): DeviceMemoryBuffer = { - withResource(RapidsBufferCatalog.acquireBuffer(handle)) { rapidsBuffer => - rapidsBuffer.getDeviceMemoryBuffer - } + handle.materialize() } /** @@ -402,9 +443,7 @@ class SpillableBuffer( } override def toString: String = { - val size = withResource(RapidsBufferCatalog.acquireBuffer(handle)) { rapidsBuffer => - rapidsBuffer.memoryUsedBytes - } + val size = handle.sizeInBytes s"SpillableBuffer size:$size, handle:$handle" } } @@ -416,17 +455,15 @@ class SpillableBuffer( * @param length a metadata-only length that is kept in the `SpillableHostBuffer` * instance. Used in cases where the backing host buffer is larger * than the number of usable bytes. - * @param catalog this was added for tests, it defaults to - * `RapidsBufferCatalog.singleton` in the companion object. */ -class SpillableHostBuffer(handle: RapidsBufferHandle, - val length: Long, - catalog: RapidsBufferCatalog) extends AutoCloseable { +class SpillableHostBuffer(handle: SpillableHostBufferHandle, + val length: Long) + extends AutoCloseable { /** * Set a new spill priority. */ def setSpillPriority(priority: Long): Unit = { - handle.setSpillPriority(priority) + // TODO: handle.setSpillPriority(priority) } /** @@ -437,9 +474,7 @@ class SpillableHostBuffer(handle: RapidsBufferHandle, } def getHostBuffer(): HostMemoryBuffer = { - withResource(catalog.acquireBuffer(handle)) { rapidsBuffer => - rapidsBuffer.getHostMemoryBuffer - } + handle.materialize() } override def toString: String = @@ -457,10 +492,8 @@ object SpillableBuffer { def apply( buffer: DeviceMemoryBuffer, priority: Long): SpillableBuffer = { - val meta = MetaUtils.getTableMetaNoTable(buffer.getLength) - val handle = withResource(buffer) { _ => - RapidsBufferCatalog.addBuffer(buffer, meta, priority) - } + Cuda.DEFAULT_STREAM.sync() + val handle = SpillableDeviceBufferHandle(buffer) // TODO: AB: priority new SpillableBuffer(handle) } } @@ -478,17 +511,12 @@ object SpillableHostBuffer { */ def apply(buffer: HostMemoryBuffer, length: Long, - priority: Long, - catalog: RapidsBufferCatalog = RapidsBufferCatalog.singleton): SpillableHostBuffer = { + priority: Long): SpillableHostBuffer = { closeOnExcept(buffer) { _ => require(length <= buffer.getLength, s"Attempted to add a host spillable with a length ${length} B which is " + s"greater than the backing host buffer length ${buffer.getLength} B") } - val meta = MetaUtils.getTableMetaNoTable(buffer.getLength) - val handle = withResource(buffer) { _ => - catalog.addBuffer(buffer, meta, priority) - } - new SpillableHostBuffer(handle, length, catalog) + new SpillableHostBuffer(SpillableHostBufferHandle(buffer), length) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/BufferSendState.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/BufferSendState.scala index 08a1ae22f5e..0a7942bd581 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/BufferSendState.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/BufferSendState.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,11 +16,9 @@ package com.nvidia.spark.rapids.shuffle -import java.io.IOException - -import ai.rapids.cudf.{Cuda, MemoryBuffer} -import com.nvidia.spark.rapids.{RapidsBuffer, ShuffleMetadata, StorageTier} -import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} +import ai.rapids.cudf.{Cuda, DeviceMemoryBuffer, MemoryBuffer} +import com.nvidia.spark.rapids.{RapidsShuffleHandle, ShuffleMetadata} +import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.format.{BufferMeta, BufferTransferRequest} @@ -60,8 +58,17 @@ class BufferSendState( serverStream: Cuda.Stream = Cuda.DEFAULT_STREAM) extends AutoCloseable with Logging { - class SendBlock(val bufferId: Int, tableSize: Long) extends BlockWithSize { - override def size: Long = tableSize + class SendBlock(val bufferHandle: RapidsShuffleHandle) extends BlockWithSize { + // we assume that the size of the buffer won't change as it goes to host/disk + // we also are likely to assume this is just a device buffer, and so we should + // copy to device and then send. + override def size: Long = { + if (bufferHandle.spillable != null) { + bufferHandle.spillable.sizeInBytes + } else { + 0L // degenerate + } + } } val peerExecutorId: Long = transaction.peerExecutorId() @@ -80,13 +87,10 @@ class BufferSendState( val btr = new BufferTransferRequest() // for reuse val blocksToSend = (0 until transferRequest.requestsLength()).map { ix => val bufferTransferRequest = transferRequest.requests(btr, ix) - withResource(requestHandler.acquireShuffleBuffer( - bufferTransferRequest.bufferId())) { table => - bufferMetas(ix) = table.meta.bufferMeta() - new SendBlock(bufferTransferRequest.bufferId(), table.getPackedSizeBytes) - } + val handle = requestHandler.getShuffleHandle(bufferTransferRequest.bufferId()) + bufferMetas(ix) = handle.tableMeta.bufferMeta() + new SendBlock(handle) } - (peerBufferReceiveHeader, bufferMetas, blocksToSend) } } @@ -145,7 +149,7 @@ class BufferSendState( } case class RangeBuffer( - range: BlockRange[SendBlock], rapidsBuffer: RapidsBuffer) + range: BlockRange[SendBlock], rapidsBuffer: MemoryBuffer) extends AutoCloseable { override def close(): Unit = { rapidsBuffer.close() @@ -170,50 +174,50 @@ class BufferSendState( if (hasMoreBlocks) { var deviceBuffs = 0L var hostBuffs = 0L - acquiredBuffs = blockRanges.safeMap { blockRange => - val bufferId = blockRange.block.bufferId - // we acquire these buffers now, and keep them until the caller releases them - // using `releaseAcquiredToCatalog` - closeOnExcept( - requestHandler.acquireShuffleBuffer(bufferId)) { rapidsBuffer => + var needsCleanup = false + try { + acquiredBuffs = blockRanges.safeMap { blockRange => + // we acquire these buffers now, and keep them until the caller releases them + // using `releaseAcquiredToCatalog` //these are closed later, after we synchronize streams - rapidsBuffer.storageTier match { - case StorageTier.DEVICE => + val spillable = blockRange.block.bufferHandle.spillable + val buff = spillable.materialize() + buff match { + case _: DeviceMemoryBuffer => deviceBuffs += blockRange.rangeSize() - case _ => // host/disk + case _ => hostBuffs += blockRange.rangeSize() } - RangeBuffer(blockRange, rapidsBuffer) + RangeBuffer(blockRange, buff) } - } - logDebug(s"Occupancy for bounce buffer is [device=${deviceBuffs}, host=${hostBuffs}] Bytes") + logDebug(s"Occupancy for bounce buffer is " + + s"[device=${deviceBuffs}, host=${hostBuffs}] Bytes") - bounceBuffToUse = if (deviceBuffs >= hostBuffs || hostBounceBuffer == null) { - deviceBounceBuffer.buffer - } else { - hostBounceBuffer.buffer - } + bounceBuffToUse = if (deviceBuffs >= hostBuffs || hostBounceBuffer == null) { + deviceBounceBuffer.buffer + } else { + hostBounceBuffer.buffer + } - // `copyToMemoryBuffer` can throw if the `RapidsBuffer` is in the DISK tier and - // the file fails to mmap. We catch the `IOException` and attempt a retry - // in the server. - var needsCleanup = false - try { - acquiredBuffs.foreach { case RangeBuffer(blockRange, rapidsBuffer) => + acquiredBuffs.foreach { case RangeBuffer(blockRange, memoryBuffer) => needsCleanup = true require(blockRange.rangeSize() <= bounceBuffToUse.getLength - buffOffset) - rapidsBuffer.copyToMemoryBuffer(blockRange.rangeStart, bounceBuffToUse, buffOffset, - blockRange.rangeSize(), serverStream) + bounceBuffToUse.copyFromMemoryBufferAsync( + buffOffset, + memoryBuffer, + blockRange.rangeStart, + blockRange.rangeSize(), + serverStream) buffOffset += blockRange.rangeSize() } needsCleanup = false } catch { - case ioe: IOException => + case ex: Exception => throw new RapidsShuffleSendPrepareException( s"Error while copying to bounce buffer for executor ${peerExecutorId} and " + - s"header ${TransportUtils.toHex(peerBufferReceiveHeader)}", ioe) + s"header ${TransportUtils.toHex(peerBufferReceiveHeader)}", ex) } finally { if (needsCleanup) { // we likely failed in `copyToMemoryBuffer` @@ -251,4 +255,4 @@ class BufferSendState( acquiredBuffs.foreach(_.close()) acquiredBuffs = Seq.empty } -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleClient.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleClient.scala index b73f9820bad..2723e24b0f0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleClient.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleClient.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -47,7 +47,7 @@ trait RapidsShuffleFetchHandler { * @return a boolean that lets the caller know the batch was accepted (true), or * rejected (false), in which case the caller should dispose of the batch. */ - def batchReceived(handle: RapidsBufferHandle): Boolean + def batchReceived(handle: RapidsShuffleHandle): Boolean /** * Called when the transport layer is not able to handle a fetch error for metadata @@ -390,7 +390,7 @@ class RapidsShuffleClient( buffMetas.foreach { consumed: ConsumedBatchFromBounceBuffer => val handle = track(consumed.contigBuffer, consumed.meta) if (!consumed.handler.batchReceived(handle)) { - catalog.removeBuffer(handle) + handle.close() numBatchesRejected += 1 } transport.doneBytesInFlight(consumed.contigBuffer.getLength) @@ -431,25 +431,19 @@ class RapidsShuffleClient( * used to look up the buffer from the catalog going (e.g. from the iterator) * @param buffer contiguous [[DeviceMemoryBuffer]] with the tables' data * @param meta [[TableMeta]] describing [[buffer]] - * @return the [[RapidsBufferId]] to be used to look up the buffer from catalog + * @return a [[RapidsShuffleHandle]] with a spillable and metadata */ private[shuffle] def track( - buffer: DeviceMemoryBuffer, meta: TableMeta): RapidsBufferHandle = { + buffer: DeviceMemoryBuffer, meta: TableMeta): RapidsShuffleHandle = { if (buffer != null) { // add the buffer to the catalog so it is available for spill catalog.addBuffer( buffer, meta, - SpillPriorities.INPUT_FROM_SHUFFLE_PRIORITY, - // set needsSync to false because we already have stream synchronized after - // consuming the bounce buffer, so we know these buffers are synchronized - // w.r.t. the CPU - needsSync = false) + SpillPriorities.INPUT_FROM_SHUFFLE_PRIORITY) } else { // no device data, just tracking metadata - catalog.addDegenerateRapidsBuffer( - meta) - + catalog.addDegenerateBatch(meta) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleServer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleServer.scala index 126b9200c90..95c411c64f2 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleServer.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleServer.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,7 +21,7 @@ import java.util.concurrent.{ConcurrentLinkedQueue, Executor} import scala.collection.mutable.ArrayBuffer import ai.rapids.cudf.{Cuda, MemoryBuffer, NvtxColor, NvtxRange} -import com.nvidia.spark.rapids.{RapidsBuffer, RapidsConf, ShuffleMetadata} +import com.nvidia.spark.rapids.{RapidsConf, RapidsShuffleHandle, ShuffleMetadata} import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.format.TableMeta @@ -49,7 +49,7 @@ trait RapidsShuffleRequestHandler { * @param tableId the unique id for a table in the catalog * @return a [[RapidsBuffer]] which is reference counted, and should be closed by the acquirer */ - def acquireShuffleBuffer(tableId: Int): RapidsBuffer + def getShuffleHandle(tableId: Int): RapidsShuffleHandle } /** diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/spill/SpillFramework.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/spill/SpillFramework.scala new file mode 100644 index 00000000000..57f2a823432 --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/spill/SpillFramework.scala @@ -0,0 +1,1743 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.spill + +import java.io._ +import java.nio.ByteBuffer +import java.nio.channels.{Channels, FileChannel, WritableByteChannel} +import java.nio.file.StandardOpenOption +import java.util +import java.util.UUID +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.mutable + +import ai.rapids.cudf._ +import com.nvidia.spark.rapids.{GpuColumnVector, GpuColumnVectorFromBuffer, GpuCompressedColumnVector, GpuDeviceManager, HostAlloc, HostMemoryOutputStream, MemoryBufferToHostByteBufferIterator, RapidsConf, RapidsHostColumnVector} +import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} +import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableSeq +import com.nvidia.spark.rapids.format.TableMeta +import com.nvidia.spark.rapids.internal.HostByteBufferIterator +import org.apache.commons.io.IOUtils + +import org.apache.spark.{SparkConf, SparkEnv, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.rapids.{GpuTaskMetrics, RapidsDiskBlockManager} +import org.apache.spark.sql.rapids.execution.SerializedHostTableUtils +import org.apache.spark.sql.rapids.storage.RapidsStorageUtils +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.storage.BlockId + +/** + * Spark-RAPIDS Spill Framework + * + * The spill framework tracks device/host/disk object lifecycle in the RAPIDS Accelerator + * for Apache Spark. A set of stores is used to track these objects, which are wrapped in + * "handles" that describe the state of each to the user and to the framework. + * + * This file comment covers some pieces of the framework that are worth knowing up front. + * + * Ownership: + * + * Any object handed to the framework via the factory methods for each of the handles + * should not be used directly by the user. The framework takes ownership of all objects. + * To get a reference back, call the `materialize` method, and always close what the framework + * returns. + * + * CUDA/Host synchronization: + * + * We assume all device backed handles are completely materialized on the device (before adding + * to the store, the CUDA stream has been synchronized with the CPU thread creating the handle), + * and that all host memory backed handles are completely materialized and not mutated by + * other CPU threads, because the contents of the handle may spill at any time, using any CUDA + * stream or thread, without synchronization. If handles added to the store are not synchronized + * we could write incomplete data to host memory or to disk. + * + * Spillability: + * + * An object is spillable (it will be copied to host or disk during OOM) if: + * - it has a approxSizeInBytes > 0 + * - it is not actively being referenced by the user (call to `materialize`, or aliased) + * - it hasn't already spilled + * - it hasn't been closed + * + * Aliasing: + * + * We handle aliasing of objects, either in the spill framework or outside, by looking at the + * reference count. All objects added to the store should support a ref count. + * If the ref count is greater than the expected value, we assume it is being aliased, + * and therefore we don't waste time spilling the aliased object. Please take a look at the + * `spillable` method in each of the handles on how this is implemented. + * + * Materialization: + * + * Every store handle supports a `materialize` method that isn't part of the interface. + * The reason is that to materialize certain objects, you may need some data (for example, + * Spark schema descriptors). `materialize` incRefCounts the object if it's resident in the + * intended store (`DeviceSpillableHandle` incRefCounts an object if it is in the device store), + * and otherwise it will create a new copy from the spilled version and hand it to the user. + * Any time a user calls `materialize`, they are responsible for closing the returned object. + * + * Spilling: + * + * A `SpillableHandle` will track an object in a specific store (`DeviceSpillable` tracks + * device "intended" objects) for example. If the handle is asked to spill, it is the handle's + * responsibility to initiate that spill, and to track the spilled handle (a device spillable + * would have a `host` handle, which tracks the host spilled object). + * + * Spill is broken down into two methods: `spill` and `releaseSpilled`. This is a two stage + * process because we need to make sure that there is no code running kernels on the spilled + * data before we actually free it. See method documentations for `spill` and `releasedSpilled` + * for more info. + * + * A cascade of spills can occur device -> host -> disk, given that host allocations can fail, or + * could not fit in the SpillableHostStore's limit (if defined). In this case, the call to spill + * will either create a host handle tracking an object on the host store (if we made room), or it + * will create a host handle that points to a disk handle, tracking a file on disk. + * + * Host handles created directly, via the factory methods `SpillableHostBufferHandle(...)` or + * `SpillableHostColumnarBatchHandle(...)`, do not trigger immediate spills. For example: + * if the host store limit is set to 1GB, and we add a 1.5GB host buffer via + * its factory method, we are going to have 2.5GB worth of host memory in the host store. + * That said, if we run out of device memory and we need to spill to host, 1.5GB will be spilled + * to disk, as device OOM triggers the pipeline spill. + * + * If we don't have a host store limit, spilling from the host store is done entirely via + * host memory allocation failure callbacks. All objects added to the host store are tracked + * immediately, since they were successfully allocated. If we fail to allocate host memory + * during a device->host spill, however, the spill framework will bypass host memory and + * go straight to disk (this last part works the same whether there are host limits or not). + * + * If the disk is full, we do not handle this in any special way. We expect this to be a + * terminal state to the executor. Every handle spills to its own file on disk, identified + * as a "temporary block" `BlockId` from Spark. + * + * Notes on locking: + * + * All stores use a concurrent hash map to store instances of `StoreHandle`. The only store + * with extra locking is the `SpillableHostStore`, to maintain a `totalSize` number that is + * used to figure out cheaply when it is full. + * + * All handles, except for disk handles, hold a reference to an object in their respective store: + * `SpillableDeviceBufferHandle` has a `dev` reference that holds a `DeviceMemoryBuffer`, and a + * `host` reference to `SpillableHostBufferHandle` that is only set if spilled. Disk handles are + * different because they don't spill, as disk is considered the final store. When a user calls + * `materialize` on a handle, the handle must guarantee that it can satisfy that, even if the caller + * should wait until a spill happens. This is currently implemented using the handle lock. + * + * Note that we hold the handle lock while we are spilling (performing IO). That means that no other + * consumer can access this spillable device handle while it is being spilled, including a second + * thread that is trying to spill and is generating a spill plan, as the handle lock is likely held + * up with IO. We will relax this likely in follow on work. + * + * We never hold a store-wide coarse grain lock in the stores when we do IO. + */ + +/** + * Common interface for all handles in the spill framework. + */ +trait StoreHandle extends AutoCloseable { + /** + * Approximate size of this handle, used in three scenarios: + * - Used by callers when accumulating up to a batch size for size goals. + * - Used from the host store to figure out how much host memory total it is tracking. + * - If approxSizeInBytes is 0, the object is tracked by the stores so it can be + * removed on shutdown, or by handle.close, but 0-byte handles are not spillable. + */ + val approxSizeInBytes: Long +} + +trait SpillableHandle extends StoreHandle { + /** + * Method called to spill this handle. It can be triggered from the spill store, + * or directly against the handle. + * + * This will not free the spilled data. If you would like to free the spill + * call `releaseSpilled` + * + * @note The size returned from this method is only used by the spill framework + * to track the approximate size. It should just return `approxSizeInBytes`, as + * that's the size that it used when it first started tracking the object. + * @return approxSizeInBytes if spilled, 0 for any other reason (not spillable, closed) + */ + def spill(): Long + + /** + * Method used to determine whether a handle tracks an object that could be spilled + * @note At the level of `SpillableHandle`, the only requirement of spillability + * is that the size of the handle is > 0. `approxSizeInBytes` is known at + * construction, and is immutable. + * @return true if currently spillable, false otherwise + */ + private[spill] def spillable: Boolean = approxSizeInBytes > 0 +} + +/** + * Spillable handles that can be materialized on the device. + * @tparam T an auto closeable subclass. `dev` tracks an instance of this object, + * on the device. + */ +trait DeviceSpillableHandle[T <: AutoCloseable] extends SpillableHandle { + private[spill] var dev: Option[T] + + private[spill] override def spillable: Boolean = synchronized { + super.spillable && dev.isDefined + } + + protected def releaseDeviceResource(): Unit = { + SpillFramework.removeFromDeviceStore(this) + synchronized { + dev.foreach(_.close()) + dev = None + } + } + + /** + * Part two of the two-stage process for spilling device buffers. We call `releaseSpilled` after + * a handle has spilled, and after a device synchronize. This prevents a race + * between threads working on cuDF kernels, that did not synchronize while holding the + * materialized handle's refCount, and the spiller thread (the spiller thread cannot + * free a device buffer that the worker thread isn't done with). + * See https://github.com/NVIDIA/spark-rapids/issues/8610 for more info. + */ + def releaseSpilled(): Unit = { + releaseDeviceResource() + } +} + +/** + * Spillable handles that can be materialized on the host. + * @tparam T an auto closeable subclass. `host` tracks an instance of this object, + * on the host. + */ +trait HostSpillableHandle[T <: AutoCloseable] extends SpillableHandle { + private[spill] var host: Option[T] + + private[spill] override def spillable: Boolean = synchronized { + super.spillable && host.isDefined + } + + protected def releaseHostResource(): Unit = { + SpillFramework.removeFromHostStore(this) + synchronized { + host.foreach(_.close()) + host = None + } + } +} + +object SpillableHostBufferHandle extends Logging { + def apply(hmb: HostMemoryBuffer): SpillableHostBufferHandle = { + val handle = new SpillableHostBufferHandle(hmb.getLength, host = Some(hmb)) + SpillFramework.stores.hostStore.trackNoSpill(handle) + handle + } + + private[spill] def createHostHandleWithPacker( + chunkedPacker: ChunkedPacker): SpillableHostBufferHandle = { + val handle = new SpillableHostBufferHandle(chunkedPacker.getTotalContiguousSize) + withResource( + SpillFramework.stores.hostStore.makeBuilder(handle)) { builder => + while (chunkedPacker.hasNext) { + val (bb, len) = chunkedPacker.next() + withResource(bb) { _ => + builder.copyNext(bb.dmb, len, Cuda.DEFAULT_STREAM) + // copyNext is synchronous w.r.t. the cuda stream passed, + // no need to synchronize here. + } + } + builder.build + } + } + + private[spill] def createHostHandleFromDeviceBuff( + buff: DeviceMemoryBuffer): SpillableHostBufferHandle = { + val handle = new SpillableHostBufferHandle(buff.getLength) + withResource( + SpillFramework.stores.hostStore.makeBuilder(handle)) { builder => + builder.copyNext(buff, buff.getLength, Cuda.DEFAULT_STREAM) + builder.build + } + } +} + +class SpillableHostBufferHandle private ( + val sizeInBytes: Long, + private[spill] override var host: Option[HostMemoryBuffer] = None, + private[spill] var disk: Option[DiskHandle] = None) + extends HostSpillableHandle[HostMemoryBuffer] { + + override val approxSizeInBytes: Long = sizeInBytes + + private[spill] override def spillable: Boolean = synchronized { + if (super.spillable) { + host.getOrElse { + throw new IllegalStateException( + s"$this is spillable but it doesn't have a materialized host buffer!") + }.getRefCount == 1 + } else { + false + } + } + + def materialize(): HostMemoryBuffer = { + var materialized: HostMemoryBuffer = null + var diskHandle: DiskHandle = null + synchronized { + if (host.isDefined) { + materialized = host.get + materialized.incRefCount() + } else if (disk.isDefined) { + diskHandle = disk.get + } else { + throw new IllegalStateException( + "attempting to materialize a closed handle") + } + } + if (materialized == null) { + materialized = closeOnExcept(HostMemoryBuffer.allocate(sizeInBytes)) { hmb => + diskHandle.materializeToHostMemoryBuffer(hmb) + hmb + } + } + materialized + } + + override def spill(): Long = { + if (!spillable) { + 0L + } else { + val spilled = synchronized { + if (disk.isEmpty && host.isDefined) { + withResource(DiskHandleStore.makeBuilder) { diskHandleBuilder => + val outputChannel = diskHandleBuilder.getChannel + GpuTaskMetrics.get.spillToDiskTime { + val iter = new HostByteBufferIterator(host.get) + iter.foreach { bb => + try { + while (bb.hasRemaining) { + outputChannel.write(bb) + } + } finally { + RapidsStorageUtils.dispose(bb) + } + } + } + disk = Some(diskHandleBuilder.build) + sizeInBytes + } + } else { + 0L + } + } + releaseHostResource() + spilled + } + } + + override def close(): Unit = { + releaseHostResource() + synchronized { + disk.foreach(_.close()) + disk = None + } + } + + private[spill] def materializeToDeviceMemoryBuffer(dmb: DeviceMemoryBuffer): Unit = { + var hostBuffer: HostMemoryBuffer = null + var diskHandle: DiskHandle = null + synchronized { + if (host.isDefined) { + hostBuffer = host.get + hostBuffer.incRefCount() + } else if (disk.isDefined) { + diskHandle = disk.get + } else { + throw new IllegalStateException( + "attempting to materialize a closed handle") + } + } + if (hostBuffer != null) { + GpuTaskMetrics.get.readSpillFromHostTime { + withResource(hostBuffer) { _ => + dmb.copyFromHostBuffer( + /*dstOffset*/ 0, + /*src*/ hostBuffer, + /*srcOffset*/ 0, + /*length*/ hostBuffer.getLength) + } + } + } else { + // cannot find a full host buffer, get chunked api + // from disk + diskHandle.materializeToDeviceMemoryBuffer(dmb) + } + } + + private[spill] def setHost(singleShotBuffer: HostMemoryBuffer): Unit = synchronized { + host = Some(singleShotBuffer) + } + + private[spill] def setDisk(handle: DiskHandle): Unit = synchronized { + disk = Some(handle) + } +} + +object SpillableDeviceBufferHandle { + def apply(dmb: DeviceMemoryBuffer): SpillableDeviceBufferHandle = { + val handle = new SpillableDeviceBufferHandle(dmb.getLength, dev = Some(dmb)) + SpillFramework.stores.deviceStore.track(handle) + handle + } +} + +class SpillableDeviceBufferHandle private ( + val sizeInBytes: Long, + private[spill] override var dev: Option[DeviceMemoryBuffer], + private[spill] var host: Option[SpillableHostBufferHandle] = None) + extends DeviceSpillableHandle[DeviceMemoryBuffer] { + + override val approxSizeInBytes: Long = sizeInBytes + + private[spill] override def spillable: Boolean = synchronized { + if (super.spillable) { + dev.getOrElse { + throw new IllegalStateException( + s"$this is spillable but it doesn't have a dev buffer!") + }.getRefCount == 1 + } else { + false + } + } + + def materialize(): DeviceMemoryBuffer = { + var materialized: DeviceMemoryBuffer = null + var hostHandle: SpillableHostBufferHandle = null + synchronized { + if (host.isDefined) { + // since we spilled, host must be set. + hostHandle = host.get + } else if (dev.isDefined) { + materialized = dev.get + materialized.incRefCount() + } else { + throw new IllegalStateException( + "attempting to materialize a closed handle") + } + } + // if `materialized` is null, we spilled. This is a terminal + // state, as we are not allowing unspill, and we don't need + // to hold locks while we copy back from here. + if (materialized == null) { + materialized = closeOnExcept(DeviceMemoryBuffer.allocate(sizeInBytes)) { dmb => + hostHandle.materializeToDeviceMemoryBuffer(dmb) + dmb + } + } + materialized + } + + override def spill(): Long = { + if (!spillable) { + 0L + } else { + synchronized { + if (host.isEmpty && dev.isDefined) { + host = Some(SpillableHostBufferHandle.createHostHandleFromDeviceBuff(dev.get)) + sizeInBytes + } else { + 0L + } + } + } + } + + override def close(): Unit = { + releaseDeviceResource() + synchronized { + host.foreach(_.close()) + host = None + } + } +} + +class SpillableColumnarBatchHandle private ( + override val approxSizeInBytes: Long, + private[spill] override var dev: Option[ColumnarBatch], + private[spill] var host: Option[SpillableHostBufferHandle] = None) + extends DeviceSpillableHandle[ColumnarBatch] with Logging { + + override def spillable: Boolean = synchronized { + if (super.spillable) { + val dcvs = GpuColumnVector.extractBases(dev.get) + val colRepetition = mutable.HashMap[ColumnVector, Int]() + dcvs.foreach { hcv => + colRepetition.put(hcv, colRepetition.getOrElse(hcv, 0) + 1) + } + dcvs.forall(dcv => { + colRepetition(dcv) == dcv.getRefCount + }) + } else { + false + } + } + + private var meta: Option[ByteBuffer] = None + + def materialize(dt: Array[DataType]): ColumnarBatch = { + var materialized: ColumnarBatch = null + var hostHandle: SpillableHostBufferHandle = null + synchronized { + if (host.isDefined) { + hostHandle = host.get + } else if (dev.isDefined) { + materialized = GpuColumnVector.incRefCounts(dev.get) + } else { + throw new IllegalStateException( + "attempting to materialize a closed handle") + } + } + if (materialized == null) { + val devBuffer = closeOnExcept(DeviceMemoryBuffer.allocate(hostHandle.sizeInBytes)) { dmb => + hostHandle.materializeToDeviceMemoryBuffer(dmb) + dmb + } + val cb = withResource(devBuffer) { _ => + withResource(Table.fromPackedTable(meta.get, devBuffer)) { tbl => + GpuColumnVector.from(tbl, dt) + } + } + materialized = cb + } + materialized + } + + override def spill(): Long = { + if (!spillable) { + 0L + } else { + synchronized { + if (host.isEmpty && dev.isDefined) { + withChunkedPacker { chunkedPacker => + meta = Some(chunkedPacker.getPackedMeta) + host = Some(SpillableHostBufferHandle.createHostHandleWithPacker(chunkedPacker)) + } + // We return the size we were created with. This is not the actual size + // of this batch when it is packed, and it is used by the calling code + // to figure out more or less how much did we free in the device. + approxSizeInBytes + } else { + 0L + } + } + } + } + + private def withChunkedPacker[T](body: ChunkedPacker => T): T = { + val tbl = synchronized { + if (dev.isEmpty) { + throw new IllegalStateException("cannot get copier without a batch") + } + GpuColumnVector.from(dev.get) + } + withResource(tbl) { _ => + withResource(new ChunkedPacker(tbl, SpillFramework.chunkedPackBounceBufferPool)) { packer => + body(packer) + } + } + } + + override def close(): Unit = { + releaseDeviceResource() + synchronized { + host.foreach(_.close()) + host = None + } + } +} + +object SpillableColumnarBatchFromBufferHandle { + def apply( + ct: ContiguousTable, + dataTypes: Array[DataType]): SpillableColumnarBatchFromBufferHandle = { + withResource(ct) { _ => + val sizeInBytes = ct.getBuffer.getLength + val cb = GpuColumnVectorFromBuffer.from(ct, dataTypes) + val handle = new SpillableColumnarBatchFromBufferHandle( + sizeInBytes, dev = Some(cb)) + SpillFramework.stores.deviceStore.track(handle) + handle + } + } + + def apply(cb: ColumnarBatch): SpillableColumnarBatchFromBufferHandle = { + require(GpuColumnVectorFromBuffer.isFromBuffer(cb), + "Columnar batch isn't a batch from buffer") + val sizeInBytes = + cb.column(0).asInstanceOf[GpuColumnVectorFromBuffer].getBuffer.getLength + val handle = new SpillableColumnarBatchFromBufferHandle( + sizeInBytes, dev = Some(cb)) + SpillFramework.stores.deviceStore.track(handle) + handle + } +} + +class SpillableColumnarBatchFromBufferHandle private ( + val sizeInBytes: Long, + private[spill] override var dev: Option[ColumnarBatch], + private[spill] var host: Option[SpillableHostBufferHandle] = None) + extends DeviceSpillableHandle[ColumnarBatch] { + + override val approxSizeInBytes: Long = sizeInBytes + + private var meta: Option[TableMeta] = None + + private[spill] override def spillable: Boolean = synchronized { + if (super.spillable) { + val dcvs = GpuColumnVector.extractBases(dev.get) + val colRepetition = mutable.HashMap[ColumnVector, Int]() + dcvs.foreach { hcv => + colRepetition.put(hcv, colRepetition.getOrElse(hcv, 0) + 1) + } + dcvs.forall(dcv => { + colRepetition(dcv) == dcv.getRefCount + }) + } else { + false + } + } + + def materialize(dt: Array[DataType]): ColumnarBatch = { + var materialized: ColumnarBatch = null + var hostHandle: SpillableHostBufferHandle = null + synchronized { + if (host.isDefined) { + hostHandle = host.get + } else if (dev.isDefined) { + materialized = GpuColumnVector.incRefCounts(dev.get) + } else { + throw new IllegalStateException( + "attempting to materialize a closed handle") + } + } + if (materialized == null) { + val devBuffer = closeOnExcept(DeviceMemoryBuffer.allocate(hostHandle.sizeInBytes)) { dmb => + hostHandle.materializeToDeviceMemoryBuffer(dmb) + dmb + } + val cb = withResource(devBuffer) { _ => + withResource(Table.fromPackedTable(meta.get.packedMetaAsByteBuffer(), devBuffer)) { tbl => + GpuColumnVector.from(tbl, dt) + } + } + materialized = cb + } + materialized + } + + override def spill(): Long = { + if (!spillable) { + 0 + } else { + synchronized { + if (host.isEmpty && dev.isDefined) { + val cvFromBuffer = dev.get.column(0).asInstanceOf[GpuColumnVectorFromBuffer] + meta = Some(cvFromBuffer.getTableMeta) + host = Some(SpillableHostBufferHandle.createHostHandleFromDeviceBuff( + cvFromBuffer.getBuffer)) + sizeInBytes + } else { + 0L + } + } + } + } + + override def close(): Unit = { + releaseDeviceResource() + synchronized { + host.foreach(_.close()) + host = None + } + } +} + +object SpillableCompressedColumnarBatchHandle { + def apply(cb: ColumnarBatch): SpillableCompressedColumnarBatchHandle = { + require(GpuCompressedColumnVector.isBatchCompressed(cb), + "Tried to track a compressed batch, but the batch wasn't compressed") + val compressedSize = + cb.column(0).asInstanceOf[GpuCompressedColumnVector].getTableBuffer.getLength + val handle = new SpillableCompressedColumnarBatchHandle(compressedSize, dev = Some(cb)) + SpillFramework.stores.deviceStore.track(handle) + handle + } +} + +class SpillableCompressedColumnarBatchHandle private ( + val compressedSizeInBytes: Long, + private[spill] override var dev: Option[ColumnarBatch], + private[spill] var host: Option[SpillableHostBufferHandle] = None) + extends DeviceSpillableHandle[ColumnarBatch] { + + override val approxSizeInBytes: Long = compressedSizeInBytes + + protected var meta: Option[TableMeta] = None + + override def spillable: Boolean = synchronized { + if (super.spillable) { + val cb = dev.get + val buff = cb.column(0).asInstanceOf[GpuCompressedColumnVector].getTableBuffer + buff.getRefCount == 1 + } else { + false + } + } + + def materialize(): ColumnarBatch = { + var materialized: ColumnarBatch = null + var hostHandle: SpillableHostBufferHandle = null + synchronized { + if (host.isDefined) { + hostHandle = host.get + } else if (dev.isDefined) { + materialized = GpuCompressedColumnVector.incRefCounts(dev.get) + } else { + throw new IllegalStateException( + "attempting to materialize a closed handle") + } + } + if (materialized == null) { + val devBuffer = closeOnExcept(DeviceMemoryBuffer.allocate(hostHandle.sizeInBytes)) { dmb => + hostHandle.materializeToDeviceMemoryBuffer(dmb) + dmb + } + materialized = withResource(devBuffer) { _ => + GpuCompressedColumnVector.from(devBuffer, meta.get) + } + } + materialized + } + + override def spill(): Long = { + if (!spillable) { + 0L + } else { + synchronized { + if (host.isEmpty && dev.isDefined) { + val cvFromBuffer = dev.get.column(0).asInstanceOf[GpuCompressedColumnVector] + meta = Some(cvFromBuffer.getTableMeta) + host = Some(SpillableHostBufferHandle.createHostHandleFromDeviceBuff( + cvFromBuffer.getTableBuffer)) + compressedSizeInBytes + } else { + 0L + } + } + } + } + + override def close(): Unit = { + releaseDeviceResource() + synchronized { + host.foreach(_.close()) + host = None + meta = None + } + } +} + +object SpillableHostColumnarBatchHandle { + def apply(cb: ColumnarBatch): SpillableHostColumnarBatchHandle = { + val sizeInBytes = RapidsHostColumnVector.getTotalHostMemoryUsed(cb) + val handle = new SpillableHostColumnarBatchHandle(sizeInBytes, cb.numRows(), host = Some(cb)) + SpillFramework.stores.hostStore.trackNoSpill(handle) + handle + } +} + +class SpillableHostColumnarBatchHandle private ( + override val approxSizeInBytes: Long, + val numRows: Int, + private[spill] override var host: Option[ColumnarBatch], + private[spill] var disk: Option[DiskHandle] = None) + extends HostSpillableHandle[ColumnarBatch] { + + override def spillable: Boolean = synchronized { + if (super.spillable) { + val hcvs = RapidsHostColumnVector.extractBases(host.get) + val colRepetition = mutable.HashMap[HostColumnVector, Int]() + hcvs.foreach { hcv => + colRepetition.put(hcv, colRepetition.getOrElse(hcv, 0) + 1) + } + hcvs.forall(hcv => { + colRepetition(hcv) == hcv.getRefCount + }) + } else { + false + } + } + + def materialize(sparkTypes: Array[DataType]): ColumnarBatch = { + var materialized: ColumnarBatch = null + var diskHandle: DiskHandle = null + synchronized { + if (host.isDefined) { + materialized = RapidsHostColumnVector.incRefCounts(host.get) + } else if (disk.isDefined) { + diskHandle = disk.get + } else { + throw new IllegalStateException( + "attempting to materialize a closed handle") + } + } + if (materialized == null) { + materialized = diskHandle.withInputWrappedStream { inputStream => + val (header, hostBuffer) = SerializedHostTableUtils.readTableHeaderAndBuffer(inputStream) + val hostCols = withResource(hostBuffer) { _ => + SerializedHostTableUtils.buildHostColumns(header, hostBuffer, sparkTypes) + } + new ColumnarBatch(hostCols.toArray, numRows) + } + } + materialized + } + + override def spill(): Long = { + if (!spillable) { + 0L + } else { + val spilled = synchronized { + if (disk.isEmpty && host.isDefined) { + withResource(DiskHandleStore.makeBuilder) { diskHandleBuilder => + GpuTaskMetrics.get.spillToDiskTime { + val dos = diskHandleBuilder.getDataOutputStream + val columns = RapidsHostColumnVector.extractBases(host.get) + JCudfSerialization.writeToStream(columns, dos, 0, host.get.numRows()) + } + disk = Some(diskHandleBuilder.build) + approxSizeInBytes + } + } else { + 0L + } + } + releaseHostResource() + spilled + } + } + + override def close(): Unit = { + releaseHostResource() + synchronized { + disk.foreach(_.close()) + disk = None + } + } +} + +object DiskHandle { + def apply(blockId: BlockId, + offset: Long, + diskSizeInBytes: Long): DiskHandle = { + val handle = new DiskHandle( + blockId, offset, diskSizeInBytes) + SpillFramework.stores.diskStore.track(handle) + handle + } +} + +/** + * A disk buffer handle helps us track spill-framework originated data on disk. + * This type of handle isn't spillable, and therefore it just implements `StoreHandle` + * @param blockId - a spark `BlockId` obtained from the configured `BlockManager` + * @param offset - starting offset for the data within the file backing `blockId` + * @param sizeInBytes - amount of bytes on disk (usually compressed and could also be encrypted). + */ +class DiskHandle private( + val blockId: BlockId, + val offset: Long, + val sizeInBytes: Long) + extends StoreHandle { + + override val approxSizeInBytes: Long = sizeInBytes + + private def withInputChannel[T](body: FileChannel => T): T = synchronized { + val file = SpillFramework.stores.diskStore.diskBlockManager.getFile(blockId) + GpuTaskMetrics.get.readSpillFromDiskTime { + withResource(new FileInputStream(file)) { fs => + withResource(fs.getChannel) { channel => + body(channel) + } + } + } + } + + def withInputWrappedStream[T](body: InputStream => T): T = synchronized { + val diskBlockManager = SpillFramework.stores.diskStore.diskBlockManager + val serializerManager = diskBlockManager.getSerializerManager() + GpuTaskMetrics.get.readSpillFromDiskTime { + withInputChannel { inputChannel => + inputChannel.position(offset) + withResource(Channels.newInputStream(inputChannel)) { compressed => + withResource(serializerManager.wrapStream(blockId, compressed)) { in => + body(in) + } + } + } + } + } + + override def close(): Unit = { + SpillFramework.removeFromDiskStore(this) + SpillFramework.stores.diskStore.deleteFile(blockId) + } + + def materializeToHostMemoryBuffer(mb: HostMemoryBuffer): Unit = { + withInputWrappedStream { in => + withResource(new HostMemoryOutputStream(mb)) { out => + IOUtils.copy(in, out) + } + } + } + + def materializeToDeviceMemoryBuffer(dmb: DeviceMemoryBuffer): Unit = { + var copyOffset = 0L + withInputWrappedStream { in => + SpillFramework.withHostSpillBounceBuffer { hmb => + val bbLength = hmb.getLength.toInt + withResource(new HostMemoryOutputStream(hmb)) { out => + var sizeRead = IOUtils.copyLarge(in, out, 0, bbLength) + while (sizeRead > 0) { + // this syncs at every copy, since for now we are + // reusing a single host spill bounce buffer + dmb.copyFromHostBuffer( + /*dstOffset*/ copyOffset, + /*src*/ hmb, + /*srcOffset*/ 0, + /*length*/ sizeRead) + out.seek(0) // start over + copyOffset += sizeRead + sizeRead = IOUtils.copyLarge(in, out, 0, bbLength) + } + } + } + } + } +} + +trait HandleStore[T <: StoreHandle] extends AutoCloseable with Logging { + protected val handles = new ConcurrentHashMap[T, java.lang.Boolean]() + + def numHandles: Int = { + handles.size() + } + + def track(handle: T): Unit = { + doTrack(handle) + } + + def remove(handle: T): Unit = { + doRemove(handle) + } + + protected def doTrack(handle: T): Boolean = { + handles.put(handle, true) == null + } + + protected def doRemove(handle: T): Boolean = { + handles.remove(handle) != null + } + + override def close(): Unit = { + handles.forEach((handle, _ )=> { + handle.close() + }) + handles.clear() + } +} + +trait SpillableStore[T <: SpillableHandle] + extends HandleStore[T] with Logging { + protected def spillNvtxRange: NvtxRange + + /** + * Internal class to provide an interface to our plan for this spill. + * + * We will build up this SpillPlan by adding spillables: handles + * that are marked spillable given the `spillable` method returning true. + * The spill store will call `trySpill`, which moves handles from the + * `spillableHandles` array to the `spilledHandles` array. + * + * At any point in time, a spill framework can call `getSpilled` + * to obtain the list of spilled handles. The device store does this + * to inject CUDA synchronization before actually releasing device handles. + */ + class SpillPlan { + private val spillableHandles = new util.ArrayList[T]() + private val spilledHandles = new util.ArrayList[T]() + + def add(spillable: T): Unit = { + spillableHandles.add(spillable) + } + + def trySpill(): Long = { + var amountSpilled = 0L + val it = spillableHandles.iterator() + while (it.hasNext) { + val handle = it.next() + val spilled = handle.spill() + if (spilled > 0) { + // this thread was successful at spilling handle. + amountSpilled += spilled + spilledHandles.add(handle) + } else { + // else, either: + // - this thread lost the race and the handle was closed + // - another thread spilled it + // - the handle isn't spillable anymore, due to ref count. + it.remove() + } + } + amountSpilled + } + + def getSpilled: util.ArrayList[T] = { + spilledHandles + } + } + + private def makeSpillPlan(spillNeeded: Long): SpillPlan = { + val plan = new SpillPlan() + var amountToSpill = 0L + val allHandles = handles.keySet().iterator() + // two threads could be here trying to spill and creating a list of spillables + while (allHandles.hasNext && amountToSpill < spillNeeded) { + val handle = allHandles.next() + if (handle.spillable) { + amountToSpill += handle.approxSizeInBytes + plan.add(handle) + } + } + plan + } + + protected def postSpill(plan: SpillPlan): Unit = {} + + def spill(spillNeeded: Long): Long = { + if (spillNeeded == 0) { + 0L + } else { + withResource(spillNvtxRange) { _ => + val plan = makeSpillPlan(spillNeeded) + val amountSpilled = plan.trySpill() + postSpill(plan) + amountSpilled + } + } + } +} + +class SpillableHostStore(val maxSize: Option[Long] = None) + extends SpillableStore[HostSpillableHandle[_]] + with Logging { + + private[spill] var totalSize: Long = 0L + + private def tryTrack(handle: HostSpillableHandle[_]): Boolean = { + if (maxSize.isEmpty || handle.approxSizeInBytes == 0) { + super.doTrack(handle) + // for now, keep this totalSize part, we technically + // do not need to track `totalSize` if we don't have a limit + synchronized { + totalSize += handle.approxSizeInBytes + } + true + } else { + synchronized { + val storeMaxSize = maxSize.get + if (totalSize > 0 && totalSize + handle.approxSizeInBytes > storeMaxSize) { + // we want to try to make room for this buffer + false + } else { + // it fits + if (super.doTrack(handle)) { + totalSize += handle.approxSizeInBytes + } + true + } + } + } + } + + override def track(handle: HostSpillableHandle[_]): Unit = { + trackInternal(handle) + } + + private def trackInternal(handle: HostSpillableHandle[_]): Boolean = { + // try to track the handle: in the case of no limits + // this should just be add to the store + var tracked = false + tracked = tryTrack(handle) + if (!tracked) { + // we only end up here if we have host store limits. + var numRetries = 0 + // we are going to try to track again, in a loop, + // since we want to release + var canFit = true + val handleSize = handle.approxSizeInBytes + var amountSpilled = 0L + val hadHandlesToSpill = !handles.isEmpty + while (canFit && !tracked && numRetries < 5) { + // if we are trying to add a handle larger than our limit + if (maxSize.get < handleSize) { + // no point in checking how much is free, just spill all + // we have + amountSpilled += spill(maxSize.get) + } else { + // handleSize is within the limits + val freeAmount = synchronized { + maxSize.get - totalSize + } + val spillNeeded = handleSize - freeAmount + if (spillNeeded > 0) { + amountSpilled += spill(spillNeeded) + } + } + tracked = tryTrack(handle) + if (!tracked) { + // we tried to spill, and we still couldn't fit this buffer + // if we have a totalSize > 0, we could try some more + // the disk api + synchronized { + canFit = totalSize > 0 + } + } + numRetries += 1 + } + val taskId = Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(0) + if (hadHandlesToSpill) { + logInfo(s"Task $taskId spilled $amountSpilled bytes while trying to " + + s"track $handleSize bytes.") + } + } + tracked + } + + /** + * This is a special method in the host store where spillable handles can be added + * but they will not trigger the cascade host->disk spill logic. This is to replicate + * how the stores used to work in the past, and is only called from factory + * methods that are used by client code. + */ + def trackNoSpill(handle: HostSpillableHandle[_]): Unit = { + synchronized { + if (doTrack(handle)) { + totalSize += handle.approxSizeInBytes + } + } + } + + override def remove(handle: HostSpillableHandle[_]): Unit = { + synchronized { + if (doRemove(handle)) { + totalSize -= handle.approxSizeInBytes + } + } + } + + /** + * Makes a builder object for `SpillableHostBufferHandle`. The builder will + * either copy ot host or disk, if the host buffer fits in the host store (if tracking + * is enabled). + * + * Host store locks and disk store locks will be taken/released during this call, but + * after the builder is created, no locks are held in the store. + * + * @note When creating the host buffer handle, never call the factory Spillable* methods, + * instead, construct the handles directly. This is because the factory methods + * trigger a spill to disk, and that standard behavior of the spill framework so far. + * @param handle a host handle that only has a size set, and no backing store. + * @return the builder to be closed by caller + */ + def makeBuilder(handle: SpillableHostBufferHandle): SpillableHostBufferHandleBuilder = { + var builder: Option[SpillableHostBufferHandleBuilder] = None + if (handle.sizeInBytes <= maxSize.getOrElse(Long.MaxValue)) { + HostAlloc.tryAlloc(handle.sizeInBytes).foreach { hmb => + withResource(hmb) { _ => + if (trackInternal(handle)) { + hmb.incRefCount() + // the host store made room or fit this buffer + builder = Some(new SpillableHostBufferHandleBuilderForHost(handle, hmb)) + } + } + } + } + builder.getOrElse { + // the disk store will track this when we call .build + new SpillableHostBufferHandleBuilderForDisk(handle) + } + } + + trait SpillableHostBufferHandleBuilder extends AutoCloseable { + /** + * Copy `mb` from offset 0 to len to host or disk. + * + * We synchronize after each copy since we do not manage the lifetime + * of `mb`. + * + * @param mb buffer to copy from + * @param len the amount of bytes that should be copied from `mb` + * @param stream CUDA stream to use, and synchronize against + */ + def copyNext(mb: DeviceMemoryBuffer, len: Long, stream: Cuda.Stream): Unit + + /** + * Returns a usable `SpillableHostBufferHandle` with either the + * `host` or `disk` set with the appropriate object. + * + * Note that if we are writing to disk, we are going to add a + * new `DiskHandle` in the disk store's concurrent collection. + * + * @return host handle with data in host or disk + */ + def build: SpillableHostBufferHandle + } + + private class SpillableHostBufferHandleBuilderForHost( + var handle: SpillableHostBufferHandle, + var singleShotBuffer: HostMemoryBuffer) + extends SpillableHostBufferHandleBuilder with Logging { + private var copied = 0L + + override def copyNext(mb: DeviceMemoryBuffer, len: Long, stream: Cuda.Stream): Unit = { + GpuTaskMetrics.get.spillToHostTime { + singleShotBuffer.copyFromMemoryBuffer( + copied, + mb, + 0, + len, + stream) + copied += len + } + } + + override def build: SpillableHostBufferHandle = { + // add some sort of setter method to Host Handle + require(handle != null, "Called build too many times") + require(copied == handle.sizeInBytes, + s"Expected ${handle.sizeInBytes} B but copied $copied B instead") + handle.setHost(singleShotBuffer) + singleShotBuffer = null + val res = handle + handle = null + res + } + + override def close(): Unit = { + if (handle != null) { + handle.close() + handle = null + } + if (singleShotBuffer != null) { + singleShotBuffer.close() + singleShotBuffer = null + } + } + } + + private class SpillableHostBufferHandleBuilderForDisk( + var handle: SpillableHostBufferHandle) + extends SpillableHostBufferHandleBuilder { + private var copied = 0L + private var diskHandleBuilder = DiskHandleStore.makeBuilder + + override def copyNext(mb: DeviceMemoryBuffer, len: Long, stream: Cuda.Stream): Unit = { + SpillFramework.withHostSpillBounceBuffer { hostSpillBounceBuffer => + GpuTaskMetrics.get.spillToDiskTime { + val outputChannel = diskHandleBuilder.getChannel + withResource(mb.slice(0, len)) { slice => + val iter = new MemoryBufferToHostByteBufferIterator( + slice, + hostSpillBounceBuffer, + Cuda.DEFAULT_STREAM) + iter.foreach { byteBuff => + try { + while (byteBuff.hasRemaining) { + outputChannel.write(byteBuff) + } + copied += byteBuff.capacity() + } finally { + RapidsStorageUtils.dispose(byteBuff) + } + } + } + } + } + } + + override def build: SpillableHostBufferHandle = { + // add some sort of setter method to Host Handle + require(handle != null, "Called build too many times") + require(copied == handle.sizeInBytes, + s"Expected ${handle.sizeInBytes} B but copied $copied B instead") + handle.setDisk(diskHandleBuilder.build) + val res = handle + handle = null + res + } + + override def close(): Unit = { + if (handle != null) { + handle.close() + handle = null + } + if (diskHandleBuilder!= null) { + diskHandleBuilder.close() + diskHandleBuilder = null + } + } + } + + override protected def spillNvtxRange: NvtxRange = + new NvtxRange("disk spill", NvtxColor.RED) +} + +class SpillableDeviceStore extends SpillableStore[DeviceSpillableHandle[_]] { + override protected def spillNvtxRange: NvtxRange = + new NvtxRange("device spill", NvtxColor.ORANGE) + + override def postSpill(plan: SpillPlan): Unit = { + // spillables is the list of handles that have to be closed + // we synchronize every thread before we release what was spilled + Cuda.deviceSynchronize() + // this is safe to be called unconditionally if another thread spilled + plan.getSpilled.forEach(_.releaseSpilled()) + } +} + +class DiskHandleStore(conf: SparkConf) + extends HandleStore[DiskHandle] with Logging { + val diskBlockManager: RapidsDiskBlockManager = new RapidsDiskBlockManager(conf) + + def getFile(blockId: BlockId): File = { + diskBlockManager.getFile(blockId) + } + + def deleteFile(blockId: BlockId): Unit = { + val file = getFile(blockId) + file.delete() + if (file.exists()) { + logWarning(s"Unable to delete $file") + } + } + + override def track(handle: DiskHandle): Unit = { + // protects the off chance that someone adds this handle twice.. + if (doTrack(handle)) { + GpuTaskMetrics.get.incDiskBytesAllocated(handle.sizeInBytes) + } + } + + override def remove(handle: DiskHandle): Unit = { + // protects the off chance that someone removes this handle twice.. + if (doRemove(handle)) { + GpuTaskMetrics.get.decDiskBytesAllocated(handle.sizeInBytes) + } + } +} + +object DiskHandleStore { + /** + * An object that knows how to write a block to disk in Spark. + * It supports + * @param blockId the BlockManager `BlockId` to use. + * @param startPos the position to start writing from, useful if we can + * share files + */ + class DiskHandleBuilder(val blockId: BlockId, + val startPos: Long = 0L) extends AutoCloseable { + private val file = SpillFramework.stores.diskStore.getFile(blockId) + + private val serializerManager = + SpillFramework.stores.diskStore.diskBlockManager.getSerializerManager() + + // this is just to make sure we use DiskWriter once and we are not leaking + // as it is, we could use `DiskWriter` to start writing at other offsets + private var closed = false + + private var fc: FileChannel = _ + + private def getFileChannel: FileChannel = { + val options = Seq(StandardOpenOption.CREATE, StandardOpenOption.WRITE) + fc = FileChannel.open(file.toPath, options:_*) + // seek to the starting pos + fc.position(startPos) + fc + } + + private def wrapChannel(channel: FileChannel): OutputStream = { + val os = Channels.newOutputStream(channel) + serializerManager.wrapStream(blockId, os) + } + + private var outputChannel: WritableByteChannel = _ + private var outputStream: DataOutputStream = _ + + def getChannel: WritableByteChannel = { + require(!closed, "Cannot write to closed DiskWriter") + require(outputStream == null, + "either channel or data output stream supported, but not both") + if (outputChannel != null) { + outputChannel + } else { + val fc = getFileChannel + val wrappedStream = closeOnExcept(fc)(wrapChannel) + outputChannel = closeOnExcept(wrappedStream)(Channels.newChannel) + outputChannel + } + } + + def getDataOutputStream: DataOutputStream = { + require(!closed, "Cannot write to closed DiskWriter") + require(outputStream == null, + "either channel or data output stream supported, but not both") + if (outputStream != null) { + outputStream + } else { + val fc = getFileChannel + val wrappedStream = closeOnExcept(fc)(wrapChannel) + outputStream = new DataOutputStream(wrappedStream) + outputStream + } + } + + override def close(): Unit = { + if (closed) { + throw new IllegalStateException("already closed DiskWriter") + } + if (outputStream != null) { + outputStream.close() + outputStream = null + } + if (outputChannel != null) { + outputChannel.close() + outputChannel = null + } + closed = true + } + + def build: DiskHandle = + DiskHandle( + blockId, + startPos, + fc.position() - startPos) + } + + def makeBuilder: DiskHandleBuilder = { + val blockId = BlockId(s"temp_local_${UUID.randomUUID().toString}") + new DiskHandleBuilder(blockId) + } +} + +trait SpillableStores extends AutoCloseable { + var deviceStore: SpillableDeviceStore + var hostStore: SpillableHostStore + var diskStore: DiskHandleStore + override def close(): Unit = { + Seq(deviceStore, hostStore, diskStore).safeClose() + } +} + +/** + * A spillable that is meant to be interacted with from the device. + */ +object SpillableColumnarBatchHandle { + def apply(tbl: Table, dataTypes: Array[DataType]): SpillableColumnarBatchHandle = { + withResource(tbl) { _ => + SpillableColumnarBatchHandle(GpuColumnVector.from(tbl, dataTypes)) + } + } + + def apply(cb: ColumnarBatch): SpillableColumnarBatchHandle = { + require(!GpuColumnVectorFromBuffer.isFromBuffer(cb), + "A SpillableColumnarBatchHandle doesn't support cuDF packed batches") + require(!GpuCompressedColumnVector.isBatchCompressed(cb), + "A SpillableColumnarBatchHandle doesn't support comprssed batches") + val sizeInBytes = GpuColumnVector.getTotalDeviceMemoryUsed(cb) + val handle = new SpillableColumnarBatchHandle(sizeInBytes, dev = Some(cb)) + SpillFramework.stores.deviceStore.track(handle) + handle + } +} + +object SpillFramework extends Logging { + // public for tests. Some tests not in the `spill` package require setting this + // because they need fine control over allocations. + var storesInternal: SpillableStores = _ + + def stores: SpillableStores = { + if (storesInternal == null) { + throw new IllegalStateException( + "Cannot use SpillFramework without calling SpillFramework.initialize first") + } + storesInternal + } + + // TODO: these should be pools, instead of individual buffers + private var hostSpillBounceBuffer: HostMemoryBuffer = _ + + private lazy val conf: SparkConf = { + val env = SparkEnv.get + if (env != null) { + env.conf + } else { + // For some unit tests + new SparkConf() + } + } + + def initialize(rapidsConf: RapidsConf): Unit = synchronized { + require(storesInternal == null, + s"cannot initialize SpillFramework multiple times.") + + val hostSpillStorageSize = if (rapidsConf.offHeapLimitEnabled) { + // Disable the limit because it is handled by the RapidsHostMemoryStore + None + } else if (rapidsConf.hostSpillStorageSize == -1) { + // + 1 GiB by default to match backwards compatibility + Some(rapidsConf.pinnedPoolSize + (1024L * 1024 * 1024)) + } else { + Some(rapidsConf.hostSpillStorageSize) + } + // this should hopefully be pinned, but it would work without + hostSpillBounceBuffer = HostMemoryBuffer.allocate(rapidsConf.spillToDiskBounceBufferSize) + + chunkedPackBounceBufferPool = new DeviceBounceBufferPool { + private val bounceBuffer: DeviceBounceBuffer = + DeviceBounceBuffer(DeviceMemoryBuffer.allocate(rapidsConf.chunkedPackBounceBufferSize)) + override def bufferSize: Long = rapidsConf.chunkedPackBounceBufferSize + override def nextBuffer(): DeviceBounceBuffer = { + // can block waiting for bounceBuffer to be released + bounceBuffer.acquire() + } + override def close(): Unit = { + // this closes the DeviceMemoryBuffer wrapped by the bounce buffer class + bounceBuffer.release() + } + } + storesInternal = new SpillableStores { + override var deviceStore: SpillableDeviceStore = new SpillableDeviceStore + override var hostStore: SpillableHostStore = new SpillableHostStore(hostSpillStorageSize) + override var diskStore: DiskHandleStore = new DiskHandleStore(conf) + } + val hostSpillStorageSizeStr = hostSpillStorageSize.map(sz => s"$sz B").getOrElse("unlimited") + logInfo(s"Initialized SpillFramework. Host spill store max size is: $hostSpillStorageSizeStr.") + } + + def shutdown(): Unit = { + if (hostSpillBounceBuffer != null) { + hostSpillBounceBuffer.close() + hostSpillBounceBuffer = null + } + if (chunkedPackBounceBufferPool != null) { + chunkedPackBounceBufferPool.close() + chunkedPackBounceBufferPool = null + } + if (storesInternal != null) { + storesInternal.close() + storesInternal = null + } + } + + def withHostSpillBounceBuffer[T](body: HostMemoryBuffer => T): T = + hostSpillBounceBuffer.synchronized { + body(hostSpillBounceBuffer) + } + + var chunkedPackBounceBufferPool: DeviceBounceBufferPool = _ + + // if the stores have already shut down, we don't want to create them here + // so we use `storesInternal` directly in these remove functions. + + private[spill] def removeFromDeviceStore(handle: DeviceSpillableHandle[_]): Unit = { + synchronized { + Option(storesInternal).map(_.deviceStore) + }.foreach(_.remove(handle)) + } + + private[spill] def removeFromHostStore(handle: HostSpillableHandle[_]): Unit = { + synchronized { + Option(storesInternal).map(_.hostStore) + }.foreach(_.remove(handle)) + } + + private[spill] def removeFromDiskStore(handle: DiskHandle): Unit = { + synchronized { + Option(storesInternal).map(_.diskStore) + }.foreach(_.remove(handle)) + } +} + +/** + * A bounce buffer wrapper class that supports the concept of acquisition. + * + * The bounce buffer is acquired exclusively. So any calls to acquire while the + * buffer is in use will block at `acquire`. Calls to `release` notify the blocked + * threads, and they will check to see if they can acquire. + * + * `close` is the interface to unacquire the bounce buffer. + * + * `release` actually closes the underlying DeviceMemoryBuffer, and should be called + * once at the end of the lifetime of the executor. + * + * @param dmb - actual cudf DeviceMemoryBuffer that this class is protecting. + */ +private[spill] case class DeviceBounceBuffer(var dmb: DeviceMemoryBuffer) extends AutoCloseable { + private var acquired: Boolean = false + def acquire(): DeviceBounceBuffer = synchronized { + while (acquired) { + wait() + } + acquired = true + this + } + + private def unaquire(): Unit = synchronized { + acquired = false + notifyAll() + } + + override def close(): Unit = { + unaquire() + } + + def release(): Unit = synchronized { + if (acquired) { + throw new IllegalStateException( + "closing device buffer pool, but some bounce buffers are in use.") + } + if (dmb != null) { + dmb.close() + dmb = null + } + } +} + +/** + * A bounce buffer pool with buffers of size `bufferSize` + * + * This pool returns instances of `DeviceBounceBuffer`, that should + * be closed in order to be reused. + * + * Callers should synchronize before calling close on their `DeviceMemoryBuffer`s. + */ +trait DeviceBounceBufferPool extends AutoCloseable { + def bufferSize: Long + def nextBuffer(): DeviceBounceBuffer +} + +/** + * ChunkedPacker is an Iterator-like class that uses a cudf::chunked_pack to copy a cuDF `Table` + * to a target buffer in chunks. It implements a next method that takes a DeviceMemoryBuffer + * as an argument to be used for the copy. + * + * Each chunk is sized at most `bounceBuffer.getLength`, and the caller should cudaMemcpy + * bytes from `bounceBuffer` to a target buffer after each call to `next()`. + * + * @note `ChunkedPacker` must be closed by the caller as it has GPU and host resources + * associated with it. + * + * @param table cuDF Table to chunk_pack + * @param bounceBufferPool bounce buffer pool to use during the lifetime of this packer. + */ +class ChunkedPacker(table: Table, + bounceBufferPool: DeviceBounceBufferPool) + extends Iterator[(DeviceBounceBuffer, Long)] with Logging with AutoCloseable { + + private var closed: Boolean = false + + // When creating cudf::chunked_pack use a pool if available, otherwise default to the + // per-device memory resource + private val chunkedPack = { + val pool = GpuDeviceManager.chunkedPackMemoryResource + val cudfChunkedPack = try { + pool.flatMap { chunkedPool => + Some(table.makeChunkedPack(bounceBufferPool.bufferSize, chunkedPool)) + } + } catch { + case _: OutOfMemoryError => + if (!ChunkedPacker.warnedAboutPoolFallback) { + ChunkedPacker.warnedAboutPoolFallback = true + logWarning( + s"OOM while creating chunked_pack using pool sized ${pool.map(_.getMaxSize)}B. " + + "Falling back to the per-device memory resource.") + } + None + } + + // if the pool is not configured, or we got an OOM, try again with the per-device pool + cudfChunkedPack.getOrElse { + table.makeChunkedPack(bounceBufferPool.bufferSize) + } + } + + private val packedMeta = withResource(chunkedPack.buildMetadata()) { packedMeta => + val tmpBB = packedMeta.getMetadataDirectBuffer + val metaCopy = ByteBuffer.allocateDirect(tmpBB.capacity()) + metaCopy.put(tmpBB) + metaCopy.flip() + metaCopy + } + + def getTotalContiguousSize: Long = chunkedPack.getTotalContiguousSize + + def getPackedMeta: ByteBuffer = { + packedMeta + } + + override def hasNext: Boolean = { + if (closed) { + throw new IllegalStateException(s"ChunkedPacker is closed") + } + chunkedPack.hasNext + } + + override def next(): (DeviceBounceBuffer, Long) = { + withResource(bounceBufferPool.nextBuffer()) { bounceBuffer => + if (closed) { + throw new IllegalStateException(s"ChunkedPacker is closed") + } + val bytesWritten = chunkedPack.next(bounceBuffer.dmb) + // we increment the refcount because the caller has no idea where + // this memory came from, so it should close it. + (bounceBuffer, bytesWritten) + } + } + + override def close(): Unit = { + if (!closed) { + closed = true + chunkedPack.close() + } + } +} + +private object ChunkedPacker { + private var warnedAboutPoolFallback: Boolean = false +} \ No newline at end of file diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala index 1b0ee21d494..7f8733b9e00 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala @@ -44,12 +44,12 @@ class GpuShuffleEnv(rapidsConf: RapidsConf) extends Logging { } } - def init(diskBlockManager: RapidsDiskBlockManager): Unit = { + def init(): Unit = { if (isRapidsShuffleConfigured) { shuffleCatalog = - new ShuffleBufferCatalog(RapidsBufferCatalog.singleton, diskBlockManager) + new ShuffleBufferCatalog() shuffleReceivedBufferCatalog = - new ShuffleReceivedBufferCatalog(RapidsBufferCatalog.singleton) + new ShuffleReceivedBufferCatalog() } } @@ -172,9 +172,9 @@ object GpuShuffleEnv extends Logging { // Functions below only get called from the executor // - def init(conf: RapidsConf, diskBlockManager: RapidsDiskBlockManager): Unit = { + def init(conf: RapidsConf): Unit = { val shuffleEnv = new GpuShuffleEnv(conf) - shuffleEnv.init(diskBlockManager) + shuffleEnv.init() env = shuffleEnv } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuTaskMetrics.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuTaskMetrics.scala index e21a9b71cb3..03193aa0152 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuTaskMetrics.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuTaskMetrics.scala @@ -228,7 +228,6 @@ class GpuTaskMetrics extends Serializable { GpuTaskMetrics.decHostBytesAllocated(bytes) } - def incDiskBytesAllocated(bytes: Long): Unit = { GpuTaskMetrics.incDiskBytesAllocated(bytes) maxDiskBytesAllocated = maxDiskBytesAllocated.max(GpuTaskMetrics.diskBytesAllocated) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala index fc255a6bfd0..6f4065ac309 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsShuffleInternalManagerBase.scala @@ -1074,7 +1074,7 @@ class RapidsCachingWriter[K, V]( val blockId = ShuffleBlockId(handle.shuffleId, mapId, partId) if (batch.numRows > 0 && batch.numCols > 0) { // Add the table to the shuffle store - val handle = batch.column(0) match { + batch.column(0) match { case c: GpuPackedTableColumn => val contigTable = c.getContiguousTable partSize = c.getTableBuffer.getLength @@ -1082,23 +1082,14 @@ class RapidsCachingWriter[K, V]( catalog.addContiguousTable( blockId, contigTable, - SpillPriorities.OUTPUT_FOR_SHUFFLE_INITIAL_PRIORITY, - // we don't need to sync here, because we sync on the cuda - // stream after sliceInternalOnGpu (contiguous_split) - needsSync = false) + SpillPriorities.OUTPUT_FOR_SHUFFLE_INITIAL_PRIORITY) case c: GpuCompressedColumnVector => - val buffer = c.getTableBuffer - partSize = buffer.getLength - val tableMeta = c.getTableMeta - uncompressedMetric += tableMeta.bufferMeta().uncompressedSize() - catalog.addBuffer( + partSize = c.getTableBuffer.getLength + uncompressedMetric += c.getTableMeta.bufferMeta().uncompressedSize() + catalog.addCompressedBatch( blockId, - buffer, - tableMeta, - SpillPriorities.OUTPUT_FOR_SHUFFLE_INITIAL_PRIORITY, - // we don't need to sync here, because we sync on the cuda - // stream after compression. - needsSync = false) + batch, + SpillPriorities.OUTPUT_FOR_SHUFFLE_INITIAL_PRIORITY) case c => throw new IllegalStateException(s"Unexpected column type: ${c.getClass}") } @@ -1112,21 +1103,18 @@ class RapidsCachingWriter[K, V]( } else { sizes(partId) += partSize } - handle } else { // no device data, tracking only metadata val tableMeta = MetaUtils.buildDegenerateTableMeta(batch) - val handle = - catalog.addDegenerateRapidsBuffer( - blockId, - tableMeta) + catalog.addDegenerateRapidsBuffer( + blockId, + tableMeta) // ensure that we set the partition size to the default in this case if // we have non-zero rows, so this degenerate batch is shuffled. if (batch.numRows > 0) { sizes(partId) += DEGENERATE_PARTITION_BYTE_SIZE_DEFAULT } - handle } } metricsReporter.incBytesWritten(bytesWritten) @@ -1280,9 +1268,8 @@ class RapidsShuffleInternalManagerBase(conf: SparkConf, val isDriver: Boolean) if (rapidsConf.isGPUShuffle && !isDriver) { val catalog = getCatalogOrThrow val requestHandler = new RapidsShuffleRequestHandler() { - override def acquireShuffleBuffer(tableId: Int): RapidsBuffer = { - val handle = catalog.getShuffleBufferHandle(tableId) - catalog.acquireBuffer(handle) + override def getShuffleHandle(tableId: Int): RapidsShuffleHandle = { + catalog.getShuffleBufferHandle(tableId) } override def getShuffleBufferMetas(sbbId: ShuffleBlockBatchId): Seq[TableMeta] = { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/TempSpillBufferId.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/TempSpillBufferId.scala deleted file mode 100644 index 0f9510a28ba..00000000000 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/TempSpillBufferId.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Copyright (c) 2020, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.rapids - -import java.io.File -import java.util.UUID -import java.util.concurrent.atomic.AtomicInteger -import java.util.function.IntUnaryOperator - -import com.nvidia.spark.rapids.RapidsBufferId - -import org.apache.spark.storage.TempLocalBlockId - -object TempSpillBufferId { - private val MAX_TABLE_ID = Integer.MAX_VALUE - private val TABLE_ID_UPDATER = new IntUnaryOperator { - override def applyAsInt(i: Int): Int = if (i < MAX_TABLE_ID) i + 1 else 0 - } - - /** Tracks the next table identifier */ - private[this] val tableIdCounter = new AtomicInteger(0) - - def apply(): TempSpillBufferId = { - val tableId = tableIdCounter.getAndUpdate(TABLE_ID_UPDATER) - val tempBlockId = TempLocalBlockId(UUID.randomUUID()) - new TempSpillBufferId(tableId, tempBlockId) - } -} - -case class TempSpillBufferId private( - override val tableId: Int, - bufferId: TempLocalBlockId) extends RapidsBufferId { - - override def getDiskPath(diskBlockManager: RapidsDiskBlockManager): File = - diskBlockManager.getFile(bufferId) -} \ No newline at end of file diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala index bd30459d63e..9290a8a9482 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala @@ -167,7 +167,7 @@ class SerializeConcatHostBuffersDeserializeBatch( * This will populate `data` before any task has had a chance to call `.batch` on this class. * * If `batchInternal` is defined we are in the executor, and there is no work to be done. - * This broadcast has been materialized on the GPU/RapidsBufferCatalog, and it is completely + * This broadcast has been materialized on the GPU/spill store, and it is completely * managed by the plugin. * * Public for unit tests. diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala index b2bb5461a40..cfd7e4e60b8 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHelper.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2023, NVIDIA CORPORATION. + * Copyright (c) 2021-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -47,7 +47,8 @@ object GpuBroadcastHelper { case broadcastBatch: SerializeConcatHostBuffersDeserializeBatch => RmmRapidsRetryIterator.withRetryNoSplit { withResource(new NvtxRange("getBroadcastBatch", NvtxColor.YELLOW)) { _ => - broadcastBatch.batch.getColumnarBatch() + val spillable = broadcastBatch.batch + spillable.getColumnarBatch() } } case v if SparkShimImpl.isEmptyRelation(v) => diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala index 70942001cae..f2f655efef5 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala @@ -36,10 +36,11 @@ package com.nvidia.spark.rapids.shuffle import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} +import scala.collection import scala.collection.mutable import ai.rapids.cudf.{NvtxColor, NvtxRange} -import com.nvidia.spark.rapids.{GpuSemaphore, RapidsBuffer, RapidsBufferHandle, RapidsConf, ShuffleReceivedBufferCatalog} +import com.nvidia.spark.rapids.{GpuSemaphore, RapidsConf, RapidsShuffleHandle, ShuffleReceivedBufferCatalog} import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion import com.nvidia.spark.rapids.jni.RmmSpark @@ -71,7 +72,7 @@ class RapidsShuffleIterator( localBlockManagerId: BlockManagerId, rapidsConf: RapidsConf, transport: RapidsShuffleTransport, - blocksByAddress: Array[(BlockManagerId, Seq[(BlockId, Long, Int)])], + blocksByAddress: Array[(BlockManagerId, collection.Seq[(BlockId, Long, Int)])], metricsUpdater: ShuffleMetricsUpdater, sparkTypes: Array[DataType], taskAttemptId: Long, @@ -90,7 +91,7 @@ class RapidsShuffleIterator( * A result for a successful buffer received * @param handle - the shuffle received buffer handle as tracked in the catalog */ - case class BufferReceived(handle: RapidsBufferHandle) extends ShuffleClientResult + case class BufferReceived(handle: RapidsShuffleHandle) extends ShuffleClientResult /** * A result for a failed attempt at receiving block metadata, or corresponding batches. @@ -180,7 +181,7 @@ class RapidsShuffleIterator( val (local, remote) = blocksByAddress.partition(ba => ba._1.host == localHost) (local ++ remote).foreach { - case (blockManagerId: BlockManagerId, blockIds: Seq[(BlockId, Long, Int)]) => { + case (blockManagerId: BlockManagerId, blockIds: collection.Seq[(BlockId, Long, Int)]) => { val shuffleRequestsMapIndex: Seq[BlockIdMapIndex] = blockIds.map { case (blockId, _, mapIndex) => /** @@ -200,7 +201,7 @@ class RapidsShuffleIterator( throw new IllegalArgumentException( s"${blockId.getClass} $blockId is not currently supported") } - } + }.toSeq val client = try { transport.makeClient(blockManagerId) @@ -245,7 +246,7 @@ class RapidsShuffleIterator( def clientDone: Boolean = clientExpectedBatches > 0 && clientExpectedBatches == clientResolvedBatches - override def batchReceived(handle: RapidsBufferHandle): Boolean = { + override def batchReceived(handle: RapidsShuffleHandle): Boolean = { resolvedBatches.synchronized { if (taskComplete) { false @@ -310,8 +311,7 @@ class RapidsShuffleIterator( logWarning(s"Iterator for task ${taskAttemptIdStr} closing, " + s"but it is not done. Closing ${resolvedBatches.size()} resolved batches!!") resolvedBatches.forEach { - case BufferReceived(handle) => - GpuShuffleEnv.getReceivedCatalog.removeBuffer(handle) + case BufferReceived(handle) => handle.close() case _ => } // tell the client to cancel pending requests @@ -337,8 +337,6 @@ class RapidsShuffleIterator( } override def next(): ColumnarBatch = { - var cb: ColumnarBatch = null - var sb: RapidsBuffer = null val range = new NvtxRange(s"RapidshuffleIterator.next", NvtxColor.RED) // If N tasks downstream are accumulating memory we run the risk OOM @@ -356,6 +354,7 @@ class RapidsShuffleIterator( // fetches and so it could produce device memory. Note this is not allowing for some external // thread to schedule the fetches for us, it may be something we consider in the future, given // memory pressure. + // No good way to get a metric in here for semaphore time. taskContext.foreach(GpuSemaphore.acquireIfNecessary) if (!started) { @@ -379,16 +378,12 @@ class RapidsShuffleIterator( val nvtxRangeAfterGettingBatch = new NvtxRange("RapidsShuffleIterator.gotBatch", NvtxColor.PURPLE) try { - sb = catalog.acquireBuffer(handle) - cb = sb.getColumnarBatch(sparkTypes) - metricsUpdater.update(blockedTime, 1, sb.memoryUsedBytes, cb.numRows()) + val (cb, memoryUsedBytes) = catalog.getColumnarBatchAndRemove(handle, sparkTypes) + metricsUpdater.update(blockedTime, 1, memoryUsedBytes, cb.numRows()) + cb } finally { nvtxRangeAfterGettingBatch.close() range.close() - if (sb != null) { - sb.close() - } - catalog.removeBuffer(handle) } case Some( TransferError(blockManagerId, shuffleBlockBatchId, mapIndex, errorMessage, throwable)) => @@ -414,6 +409,5 @@ class RapidsShuffleIterator( case _ => throw new IllegalStateException(s"Invalid result type $result") } - cb } } diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala index 3334187bb16..1905190f30e 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala @@ -38,13 +38,13 @@ import scala.collection.mutable.ArrayBuffer import ai.rapids.cudf.{NvtxColor, NvtxRange} import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.shuffle.{RapidsShuffleIterator, RapidsShuffleTransport} import org.apache.spark.{InterruptibleIterator, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.shuffle.{ShuffleReader, ShuffleReadMetricsReporter} import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockBatchId, ShuffleBlockId} import org.apache.spark.util.CompletionIterator @@ -79,10 +79,12 @@ class RapidsCachingReader[K, C]( override def read(): Iterator[Product2[K, C]] = { val readRange = new NvtxRange(s"RapidsCachingReader.read", NvtxColor.DARK_GREEN) try { - val blocksForRapidsTransport = new ArrayBuffer[(BlockManagerId, Seq[(BlockId, Long, Int)])]() - val cachedBlocks = new ArrayBuffer[BlockId]() - val cachedBufferHandles = new ArrayBuffer[RapidsBufferHandle]() - val blocksByAddressMap: Map[BlockManagerId, Seq[(BlockId, Long, Int)]] = blocksByAddress.toMap + val blocksForRapidsTransport = + new ArrayBuffer[(BlockManagerId, Seq[(BlockId, Long, Int)])]() + var cachedBatchIterator: Iterator[ColumnarBatch] = Iterator.empty + val blocksByAddressMap: Map[BlockManagerId, Seq[(BlockId, Long, Int)]] = + blocksByAddress.toMap + var numCachedBlocks: Int = 0 blocksByAddressMap.keys.foreach(blockManagerId => { val blockInfos: Seq[(BlockId, Long, Int)] = blocksByAddressMap(blockManagerId) @@ -91,33 +93,29 @@ class RapidsCachingReader[K, C]( if (blockManagerId.executorId == localId.executorId) { val readLocalRange = new NvtxRange("Read Local", NvtxColor.GREEN) try { - blockInfos.foreach( - blockInfo => { - val blockId = blockInfo._1 - val shuffleBufferHandles: IndexedSeq[RapidsBufferHandle] = blockId match { - case sbbid: ShuffleBlockBatchId => - (sbbid.startReduceId to sbbid.endReduceId).flatMap { reduceId => - cachedBlocks.append(blockId) - val sBlockId = ShuffleBlockId(sbbid.shuffleId, sbbid.mapId, reduceId) - catalog.blockIdToBufferHandles(sBlockId) - } - case sbid: ShuffleBlockId => - cachedBlocks.append(blockId) - catalog.blockIdToBufferHandles(sbid) - case _ => throw new IllegalArgumentException( - s"${blockId.getClass} $blockId is not currently supported") - } - - cachedBufferHandles ++= shuffleBufferHandles - - // Update the spill priorities of these buffers to indicate they are about - // to be read and therefore should not be spilled if possible. - shuffleBufferHandles.foreach(catalog.updateSpillPriorityForLocalRead) - - if (shuffleBufferHandles.nonEmpty) { - metrics.incLocalBlocksFetched(1) - } - }) + cachedBatchIterator = blockInfos.iterator.flatMap { blockInfo => + val blockId = blockInfo._1 + val shuffleBufferHandles = blockId match { + case sbbid: ShuffleBlockBatchId => + (sbbid.startReduceId to sbbid.endReduceId).iterator.flatMap { reduceId => + val sbid = ShuffleBlockId(sbbid.shuffleId, sbbid.mapId, reduceId) + numCachedBlocks += 1 + catalog.getColumnarBatchIterator(sbid, sparkTypes) + } + case sbid: ShuffleBlockId => + numCachedBlocks += 1 + catalog.getColumnarBatchIterator(sbid, sparkTypes) + case _ => throw new IllegalArgumentException( + s"${blockId.getClass} $blockId is not currently supported") + } + + shuffleBufferHandles + } + + // Update the spill priorities of these buffers to indicate they are about + // to be read and therefore should not be spilled if possible. + // TODO: AB: shuffleBufferHandles.foreach(catalog.updateSpillPriorityForLocalRead) + metrics.incLocalBlocksFetched(numCachedBlocks) } finally { readLocalRange.close() } @@ -139,7 +137,7 @@ class RapidsCachingReader[K, C]( } }) - logInfo(s"Will read ${cachedBlocks.size} cached blocks, " + + logInfo(s"Will read ${numCachedBlocks} cached blocks, " + s"${blocksForRapidsTransport.size} remote blocks from the RapidsShuffleTransport. ") if (transport.isEmpty && blocksForRapidsTransport.nonEmpty) { @@ -159,17 +157,12 @@ class RapidsCachingReader[K, C]( val itRange = new NvtxRange("Shuffle Iterator prep", NvtxColor.BLUE) try { - val cachedIt = cachedBufferHandles.iterator.map(bufferHandle => { - // No good way to get a metric in here for semaphore wait time - GpuSemaphore.acquireIfNecessary(context) - val cb = withResource(catalog.acquireBuffer(bufferHandle)) { buffer => - buffer.getColumnarBatch(sparkTypes) - } + val cachedIt = cachedBatchIterator.map { cb => val cachedBytesRead = GpuColumnVector.getTotalDeviceMemoryUsed(cb) metrics.incLocalBytesRead(cachedBytesRead) metrics.incRecordsRead(cb.numRows()) (0, cb) - }).asInstanceOf[Iterator[(K, C)]] + }.asInstanceOf[Iterator[(K, C)]] val cbArrayFromUcx: Iterator[(K, C)] = if (blocksForRapidsTransport.nonEmpty) { val rapidsShuffleIterator = new RapidsShuffleIterator(localId, rapidsConf, transport.get, diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala index 56552dac7b7..868e5492e2b 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala @@ -36,7 +36,7 @@ import scala.collection import scala.collection.mutable import ai.rapids.cudf.{NvtxColor, NvtxRange} -import com.nvidia.spark.rapids.{GpuSemaphore, RapidsBuffer, RapidsBufferHandle, RapidsConf, ShuffleReceivedBufferCatalog} +import com.nvidia.spark.rapids.{GpuSemaphore, RapidsConf, RapidsShuffleHandle, ShuffleReceivedBufferCatalog} import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion import com.nvidia.spark.rapids.jni.RmmSpark @@ -87,7 +87,7 @@ class RapidsShuffleIterator( * A result for a successful buffer received * @param handle - the shuffle received buffer handle as tracked in the catalog */ - case class BufferReceived(handle: RapidsBufferHandle) extends ShuffleClientResult + case class BufferReceived(handle: RapidsShuffleHandle) extends ShuffleClientResult /** * A result for a failed attempt at receiving block metadata, or corresponding batches. @@ -223,7 +223,7 @@ class RapidsShuffleIterator( override def getTaskIds: Array[Long] = taskIds - def start(expectedBatches: Int): Unit = resolvedBatches.synchronized { + override def start(expectedBatches: Int): Unit = resolvedBatches.synchronized { if (expectedBatches == 0) { throw new IllegalStateException( s"Received an invalid response from shuffle server: " + @@ -242,7 +242,7 @@ class RapidsShuffleIterator( def clientDone: Boolean = clientExpectedBatches > 0 && clientExpectedBatches == clientResolvedBatches - def batchReceived(handle: RapidsBufferHandle): Boolean = { + override def batchReceived(handle: RapidsShuffleHandle): Boolean = { resolvedBatches.synchronized { if (taskComplete) { false @@ -307,8 +307,7 @@ class RapidsShuffleIterator( logWarning(s"Iterator for task ${taskAttemptIdStr} closing, " + s"but it is not done. Closing ${resolvedBatches.size()} resolved batches!!") resolvedBatches.forEach { - case BufferReceived(handle) => - GpuShuffleEnv.getReceivedCatalog.removeBuffer(handle) + case BufferReceived(handle) => handle.close() case _ => } // tell the client to cancel pending requests @@ -334,8 +333,6 @@ class RapidsShuffleIterator( } override def next(): ColumnarBatch = { - var cb: ColumnarBatch = null - var sb: RapidsBuffer = null val range = new NvtxRange(s"RapidshuffleIterator.next", NvtxColor.RED) // If N tasks downstream are accumulating memory we run the risk OOM @@ -377,16 +374,12 @@ class RapidsShuffleIterator( val nvtxRangeAfterGettingBatch = new NvtxRange("RapidsShuffleIterator.gotBatch", NvtxColor.PURPLE) try { - sb = catalog.acquireBuffer(handle) - cb = sb.getColumnarBatch(sparkTypes) - metricsUpdater.update(blockedTime, 1, sb.memoryUsedBytes, cb.numRows()) + val (cb, memoryUsedBytes) = catalog.getColumnarBatchAndRemove(handle, sparkTypes) + metricsUpdater.update(blockedTime, 1, memoryUsedBytes, cb.numRows()) + cb } finally { nvtxRangeAfterGettingBatch.close() range.close() - if (sb != null) { - sb.close() - } - catalog.removeBuffer(handle) } case Some( TransferError(blockManagerId, shuffleBlockBatchId, mapIndex, errorMessage, throwable)) => @@ -412,6 +405,5 @@ class RapidsShuffleIterator( case _ => throw new IllegalStateException(s"Invalid result type $result") } - cb } } diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala index f7afe6aeba4..bc962e1bf5c 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala @@ -35,13 +35,13 @@ import scala.collection.mutable.ArrayBuffer import ai.rapids.cudf.{NvtxColor, NvtxRange} import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.shuffle.{RapidsShuffleIterator, RapidsShuffleTransport} import org.apache.spark.{InterruptibleIterator, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.shuffle.{ShuffleReader, ShuffleReadMetricsReporter} import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockBatchId, ShuffleBlockId} import org.apache.spark.util.CompletionIterator @@ -78,10 +78,10 @@ class RapidsCachingReader[K, C]( try { val blocksForRapidsTransport = new ArrayBuffer[(BlockManagerId, collection.Seq[(BlockId, Long, Int)])]() - val cachedBlocks = new ArrayBuffer[BlockId]() - val cachedBufferHandles = new ArrayBuffer[RapidsBufferHandle]() - val blocksByAddressMap: Map[BlockManagerId, collection.Seq[(BlockId, Long, Int)]] = + var cachedBatchIterator: Iterator[ColumnarBatch] = Iterator.empty + val blocksByAddressMap: Map[BlockManagerId, collection.Seq[(BlockId, Long, Int)]] = blocksByAddress.toMap + var numCachedBlocks: Int = 0 blocksByAddressMap.keys.foreach(blockManagerId => { val blockInfos: collection.Seq[(BlockId, Long, Int)] = blocksByAddressMap(blockManagerId) @@ -90,33 +90,29 @@ class RapidsCachingReader[K, C]( if (blockManagerId.executorId == localId.executorId) { val readLocalRange = new NvtxRange("Read Local", NvtxColor.GREEN) try { - blockInfos.foreach( - blockInfo => { - val blockId = blockInfo._1 - val shuffleBufferHandles: IndexedSeq[RapidsBufferHandle] = blockId match { - case sbbid: ShuffleBlockBatchId => - (sbbid.startReduceId to sbbid.endReduceId).flatMap { reduceId => - cachedBlocks.append(blockId) - val sBlockId = ShuffleBlockId(sbbid.shuffleId, sbbid.mapId, reduceId) - catalog.blockIdToBufferHandles(sBlockId) - } - case sbid: ShuffleBlockId => - cachedBlocks.append(blockId) - catalog.blockIdToBufferHandles(sbid) - case _ => throw new IllegalArgumentException( - s"${blockId.getClass} $blockId is not currently supported") - } - - cachedBufferHandles ++= shuffleBufferHandles - - // Update the spill priorities of these buffers to indicate they are about - // to be read and therefore should not be spilled if possible. - shuffleBufferHandles.foreach(catalog.updateSpillPriorityForLocalRead) - - if (shuffleBufferHandles.nonEmpty) { - metrics.incLocalBlocksFetched(1) - } - }) + cachedBatchIterator = blockInfos.iterator.flatMap { blockInfo => + val blockId = blockInfo._1 + val shuffleBufferHandles = blockId match { + case sbbid: ShuffleBlockBatchId => + (sbbid.startReduceId to sbbid.endReduceId).iterator.flatMap { reduceId => + val sbid = ShuffleBlockId(sbbid.shuffleId, sbbid.mapId, reduceId) + numCachedBlocks += 1 + catalog.getColumnarBatchIterator(sbid, sparkTypes) + } + case sbid: ShuffleBlockId => + numCachedBlocks += 1 + catalog.getColumnarBatchIterator(sbid, sparkTypes) + case _ => throw new IllegalArgumentException( + s"${blockId.getClass} $blockId is not currently supported") + } + + shuffleBufferHandles + } + + // Update the spill priorities of these buffers to indicate they are about + // to be read and therefore should not be spilled if possible. + // TODO: AB: shuffleBufferHandles.foreach(catalog.updateSpillPriorityForLocalRead) + metrics.incLocalBlocksFetched(numCachedBlocks) } finally { readLocalRange.close() } @@ -138,7 +134,7 @@ class RapidsCachingReader[K, C]( } }) - logInfo(s"Will read ${cachedBlocks.size} cached blocks, " + + logInfo(s"Will read ${numCachedBlocks} cached blocks, " + s"${blocksForRapidsTransport.size} remote blocks from the RapidsShuffleTransport. ") if (transport.isEmpty && blocksForRapidsTransport.nonEmpty) { @@ -158,17 +154,12 @@ class RapidsCachingReader[K, C]( val itRange = new NvtxRange("Shuffle Iterator prep", NvtxColor.BLUE) try { - val cachedIt = cachedBufferHandles.iterator.map(bufferHandle => { - // No good way to get a metric in here for semaphore wait time - GpuSemaphore.acquireIfNecessary(context) - val cb = withResource(catalog.acquireBuffer(bufferHandle)) { buffer => - buffer.getColumnarBatch(sparkTypes) - } + val cachedIt = cachedBatchIterator.map { cb => val cachedBytesRead = GpuColumnVector.getTotalDeviceMemoryUsed(cb) metrics.incLocalBytesRead(cachedBytesRead) metrics.incRecordsRead(cb.numRows()) (0, cb) - }).asInstanceOf[Iterator[(K, C)]] + }.asInstanceOf[Iterator[(K, C)]] val cbArrayFromUcx: Iterator[(K, C)] = if (blocksForRapidsTransport.nonEmpty) { val rapidsShuffleIterator = new RapidsShuffleIterator(localId, rapidsConf, transport.get, diff --git a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/HostAllocSuite.scala b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/HostAllocSuite.scala index 24755c2c0a1..fbd61b9a7fb 100644 --- a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/HostAllocSuite.scala +++ b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/HostAllocSuite.scala @@ -21,6 +21,7 @@ import java.util.concurrent.{ExecutionException, Future, LinkedBlockingQueue, Ti import ai.rapids.cudf.{HostMemoryBuffer, PinnedMemoryPool, Rmm, RmmAllocationMode} import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.jni.{RmmSpark, RmmSparkThreadState} +import com.nvidia.spark.rapids.spill._ import org.mockito.Mockito.when import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} import org.scalatest.concurrent.{Signaler, TimeLimits} @@ -28,7 +29,7 @@ import org.scalatest.funsuite.AnyFunSuite import org.scalatest.time._ import org.scalatestplus.mockito.MockitoSugar.mock -import org.apache.spark.TaskContext +import org.apache.spark.{SparkConf, TaskContext} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.execution.TrampolineUtil @@ -36,7 +37,7 @@ import org.apache.spark.sql.rapids.execution.TrampolineUtil class HostAllocSuite extends AnyFunSuite with BeforeAndAfterEach with BeforeAndAfterAll with TimeLimits { private val sqlConf = new SQLConf() - private val rc = new RapidsConf(sqlConf) + Rmm.shutdown() private val timeoutMs = 10000 def setMockContext(taskAttemptId: Long): Unit = { @@ -316,23 +317,34 @@ class HostAllocSuite extends AnyFunSuite with BeforeAndAfterEach with private var rmmWasInitialized = false override def beforeEach(): Unit = { - RapidsBufferCatalog.close() + val sc = new SparkConf SparkSession.getActiveSession.foreach(_.stop()) SparkSession.clearActiveSession() + SpillFramework.shutdown() if (Rmm.isInitialized) { rmmWasInitialized = true Rmm.shutdown() } Rmm.initialize(RmmAllocationMode.CUDA_DEFAULT, null, 512 * 1024 * 1024) + // this doesn't allocate memory for bounce buffers, as HostAllocSuite + // is playing games with the pools. + SpillFramework.storesInternal = new SpillableStores { + override var deviceStore: SpillableDeviceStore = new SpillableDeviceStore + override var hostStore: SpillableHostStore = new SpillableHostStore(None) + override var diskStore: DiskHandleStore = new DiskHandleStore(sc) + } + // some tests need an event handler + RmmSpark.setEventHandler( + new DeviceMemoryEventHandler(SpillFramework.stores.deviceStore, None, 0)) PinnedMemoryPool.shutdown() HostAlloc.initialize(-1) - RapidsBufferCatalog.init(rc) } override def afterAll(): Unit = { - RapidsBufferCatalog.close() + SpillFramework.shutdown() PinnedMemoryPool.shutdown() Rmm.shutdown() + RmmSpark.clearEventHandler() if (rmmWasInitialized) { // put RMM back for other tests to use Rmm.initialize(RmmAllocationMode.CUDA_DEFAULT, null, 512 * 1024 * 1024) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandlerSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandlerSuite.scala index 0b531adabb7..9ba0147d878 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandlerSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandlerSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * Copyright (c) 2022-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,6 +16,7 @@ package com.nvidia.spark.rapids +import com.nvidia.spark.rapids.spill.SpillableDeviceStore import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.when import org.scalatestplus.mockito.MockitoSugar @@ -23,12 +24,9 @@ import org.scalatestplus.mockito.MockitoSugar class DeviceMemoryEventHandlerSuite extends RmmSparkRetrySuiteBase with MockitoSugar { test("a failed allocation should be retried if we spilled enough") { - val mockCatalog = mock[RapidsBufferCatalog] - val mockStore = mock[RapidsDeviceMemoryStore] - when(mockStore.currentSpillableSize).thenReturn(1024) - when(mockCatalog.synchronousSpill(any(), any(), any())).thenAnswer(_ => Some(1024L)) + val mockStore = mock[SpillableDeviceStore] + when(mockStore.spill(any())).thenAnswer(_ => 1024L) val handler = new DeviceMemoryEventHandler( - mockCatalog, mockStore, None, 2) @@ -36,12 +34,9 @@ class DeviceMemoryEventHandlerSuite extends RmmSparkRetrySuiteBase with MockitoS } test("when we deplete the store, retry up to max failed OOM retries") { - val mockCatalog = mock[RapidsBufferCatalog] - val mockStore = mock[RapidsDeviceMemoryStore] - when(mockStore.currentSpillableSize).thenReturn(0) - when(mockCatalog.synchronousSpill(any(), any(), any())).thenAnswer(_ => Some(0L)) + val mockStore = mock[SpillableDeviceStore] + when(mockStore.spill(any())).thenAnswer(_ => 0L) val handler = new DeviceMemoryEventHandler( - mockCatalog, mockStore, None, 2) @@ -51,12 +46,9 @@ class DeviceMemoryEventHandlerSuite extends RmmSparkRetrySuiteBase with MockitoS } test("we reset our OOM state after a successful retry") { - val mockCatalog = mock[RapidsBufferCatalog] - val mockStore = mock[RapidsDeviceMemoryStore] - when(mockStore.currentSpillableSize).thenReturn(0) - when(mockCatalog.synchronousSpill(any(), any(), any())).thenAnswer(_ => Some(0L)) + val mockStore = mock[SpillableDeviceStore] + when(mockStore.spill(any())).thenAnswer(_ => 0L) val handler = new DeviceMemoryEventHandler( - mockCatalog, mockStore, None, 2) @@ -69,12 +61,9 @@ class DeviceMemoryEventHandlerSuite extends RmmSparkRetrySuiteBase with MockitoS } test("a negative allocation cannot be retried and handler throws") { - val mockCatalog = mock[RapidsBufferCatalog] - val mockStore = mock[RapidsDeviceMemoryStore] - when(mockStore.currentSpillableSize).thenReturn(1024) - when(mockCatalog.synchronousSpill(any(), any(), any())).thenAnswer(_ => Some(1024L)) + val mockStore = mock[SpillableDeviceStore] + when(mockStore.spill(any())).thenAnswer(_ => 1024L) val handler = new DeviceMemoryEventHandler( - mockCatalog, mockStore, None, 2) @@ -82,12 +71,9 @@ class DeviceMemoryEventHandlerSuite extends RmmSparkRetrySuiteBase with MockitoS } test("a negative retry count is invalid") { - val mockCatalog = mock[RapidsBufferCatalog] - val mockStore = mock[RapidsDeviceMemoryStore] - when(mockStore.currentSpillableSize).thenReturn(1024) - when(mockCatalog.synchronousSpill(any(), any(), any())).thenAnswer(_ => Some(1024L)) + val mockStore = mock[SpillableDeviceStore] + when(mockStore.spill(any())).thenAnswer(_ => 1024L) val handler = new DeviceMemoryEventHandler( - mockCatalog, mockStore, None, 2) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GeneratedInternalRowToCudfRowIteratorRetrySuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GeneratedInternalRowToCudfRowIteratorRetrySuite.scala index 725a2e37032..da03992f72f 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GeneratedInternalRowToCudfRowIteratorRetrySuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GeneratedInternalRowToCudfRowIteratorRetrySuite.scala @@ -19,8 +19,9 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.Table import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.jni.{GpuSplitAndRetryOOM, RmmSpark} +import com.nvidia.spark.rapids.spill.{SpillableColumnarBatchHandle, SpillableDeviceStore, SpillFramework} import org.mockito.ArgumentMatchers.any -import org.mockito.Mockito.{doAnswer, spy, times, verify} +import org.mockito.Mockito.{doAnswer, spy} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatestplus.mockito.MockitoSugar @@ -41,6 +42,14 @@ class GeneratedInternalRowToCudfRowIteratorRetrySuite } } + override def beforeEach(): Unit = { + // some tests in this suite will want to perform `verify` calls on the device store + // so we close it and create a spy around one. + super.beforeEach() + SpillFramework.storesInternal.deviceStore.close() + SpillFramework.storesInternal.deviceStore = spy(new SpillableDeviceStore) + } + private def getAndResetNumRetryThrowCurrentTask: Int = { // taskId 1 was associated with the current thread in RmmSparkRetrySuiteBase RmmSpark.getAndResetNumRetryThrow(/*taskId*/ 1) @@ -65,26 +74,24 @@ class GeneratedInternalRowToCudfRowIteratorRetrySuite } assert(!GpuColumnVector.extractBases(batch).exists(_.getRefCount > 0)) assert(!myIter.hasNext) - assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) + assertResult(0)(SpillFramework.stores.deviceStore.spill(1)) } } test("a retry when converting to a table is handled") { val batch = buildBatch() val batchIter = Seq(batch).iterator - var rapidsBufferSpy: RapidsBuffer = null - doAnswer(new Answer[AnyRef]() { - override def answer(invocation: InvocationOnMock): AnyRef = { - val res = invocation.callRealMethod() + doAnswer(new Answer[Boolean]() { + override def answer(invocation: InvocationOnMock): Boolean = { + invocation.callRealMethod() // we mock things this way due to code generation issues with mockito. // when we add a table we have RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 3, RmmSpark.OomInjectionType.GPU.ordinal, 0) - rapidsBufferSpy = spy(res.asInstanceOf[RapidsBuffer]) - rapidsBufferSpy + true } - }).when(deviceStorage) - .addTable(any(), any(), any(), any()) + }).when(SpillFramework.stores.deviceStore) + .track(any()) withResource(new ColumnarToRowIterator(batchIter, NoopMetric, NoopMetric, NoopMetric, NoopMetric)) { ctriter => @@ -102,35 +109,27 @@ class GeneratedInternalRowToCudfRowIteratorRetrySuite } assertResult(6)(getAndResetNumRetryThrowCurrentTask) assert(!myIter.hasNext) - assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) - // This is my wrap around of checking that we did retry the last part - // where we are converting the device column of rows into an actual column. - // Because we asked for 3 retries, we would ask the spill framework 4 times to materialize - // a batch. - verify(rapidsBufferSpy, times(4)) - .getColumnarBatch(any()) + assertResult(0)(SpillFramework.stores.deviceStore.spill(1)) } } test("spilling the device column of rows works") { val batch = buildBatch() val batchIter = Seq(batch).iterator - var rapidsBufferSpy: RapidsBuffer = null - doAnswer(new Answer[AnyRef]() { - override def answer(invocation: InvocationOnMock): AnyRef = { - val res = invocation.callRealMethod() + doAnswer(new Answer[Boolean]() { + override def answer(invocation: InvocationOnMock): Boolean = { + val handle = invocation.getArgument(0).asInstanceOf[SpillableColumnarBatchHandle] // we mock things this way due to code generation issues with mockito. // when we add a table we have RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 3, RmmSpark.OomInjectionType.GPU.ordinal, 0) - rapidsBufferSpy = spy(res.asInstanceOf[RapidsBuffer]) // at this point we have created a buffer in the Spill Framework // lets spill it - RapidsBufferCatalog.singleton.synchronousSpill(deviceStorage, 0) - rapidsBufferSpy + SpillFramework.stores.deviceStore.spill(handle.approxSizeInBytes) + true } - }).when(deviceStorage) - .addTable(any(), any(), any(), any()) + }).when(SpillFramework.stores.deviceStore) + .track(any()) withResource(new ColumnarToRowIterator(batchIter, NoopMetric, NoopMetric, NoopMetric, NoopMetric)) { ctriter => @@ -148,13 +147,7 @@ class GeneratedInternalRowToCudfRowIteratorRetrySuite } assertResult(6)(getAndResetNumRetryThrowCurrentTask) assert(!myIter.hasNext) - assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) - // This is my wrap around of checking that we did retry the last part - // where we are converting the device column of rows into an actual column. - // Because we asked for 3 retries, we would ask the spill framework 4 times to materialize - // a batch. - verify(rapidsBufferSpy, times(4)) - .getColumnarBatch(any()) + assertResult(0)(SpillFramework.stores.deviceStore.spill(1)) } } @@ -173,7 +166,7 @@ class GeneratedInternalRowToCudfRowIteratorRetrySuite assertThrows[GpuSplitAndRetryOOM] { myIter.next() } - assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) + assertResult(0)(SpillFramework.stores.deviceStore.spill(1)) } } @@ -199,7 +192,7 @@ class GeneratedInternalRowToCudfRowIteratorRetrySuite } assert(!GpuColumnVector.extractBases(batch).exists(_.getRefCount > 0)) assert(!myIter.hasNext) - assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) + assertResult(0)(SpillFramework.stores.deviceStore.spill(1)) } } @@ -225,7 +218,7 @@ class GeneratedInternalRowToCudfRowIteratorRetrySuite } assert(!GpuColumnVector.extractBases(batch).exists(_.getRefCount > 0)) assert(!myIter.hasNext) - assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) + assertResult(0)(SpillFramework.stores.deviceStore.spill(1)) } } } \ No newline at end of file diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesRetrySuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesRetrySuite.scala index 34a9fa984d5..4f0027d4c2a 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesRetrySuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesRetrySuite.scala @@ -199,7 +199,7 @@ class GpuCoalesceBatchesRetrySuite val batches = iter.asInstanceOf[CoalesceIteratorMocks].getBatches() assertResult(10)(batches.length) batches.foreach(b => - verify(b, times(1)).close() + GpuColumnVector.extractBases(b).forall(_.getRefCount == 0) ) } } @@ -209,7 +209,7 @@ class GpuCoalesceBatchesRetrySuite var refCount = 1 override def numRows(): Int = 0 override def setSpillPriority(priority: Long): Unit = {} - override def getColumnarBatch(): ColumnarBatch = { + override def getColumnarBatch: ColumnarBatch = { throw new GpuSplitAndRetryOOM() } override def sizeInBytes: Long = 0 diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuGenerateSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuGenerateSuite.scala index e69f7c75118..fbbf0acc20d 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuGenerateSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuGenerateSuite.scala @@ -268,12 +268,12 @@ class GpuGenerateSuite var forceOOM: Boolean) extends SpillableColumnarBatch { override def numRows(): Int = spillable.numRows() override def setSpillPriority(priority: Long): Unit = spillable.setSpillPriority(priority) - override def getColumnarBatch(): ColumnarBatch = { + override def getColumnarBatch: ColumnarBatch = { if (forceOOM) { forceOOM = false throw new GpuSplitAndRetryOOM(s"mock split and retry") } - spillable.getColumnarBatch() + spillable.getColumnarBatch } override def sizeInBytes: Long = spillable.sizeInBytes override def dataTypes: Array[DataType] = spillable.dataTypes diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuPartitioningSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuPartitioningSuite.scala index 1e3c0f699da..fdbe316a394 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuPartitioningSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuPartitioningSuite.scala @@ -16,20 +16,20 @@ package com.nvidia.spark.rapids -import java.io.File import java.math.RoundingMode import ai.rapids.cudf.{ColumnVector, Cuda, DType, Table} import com.nvidia.spark.rapids.Arm.withResource +import org.scalatest.BeforeAndAfterEach import org.scalatest.funsuite.AnyFunSuite import org.apache.spark.SparkConf -import org.apache.spark.sql.rapids.{GpuShuffleEnv, RapidsDiskBlockManager} +import org.apache.spark.sql.rapids.GpuShuffleEnv import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.types.{DecimalType, DoubleType, IntegerType, StringType} import org.apache.spark.sql.vectorized.ColumnarBatch -class GpuPartitioningSuite extends AnyFunSuite { +class GpuPartitioningSuite extends AnyFunSuite with BeforeAndAfterEach { var rapidsConf = new RapidsConf(Map[String, String]()) private def buildBatch(): ColumnarBatch = { @@ -113,7 +113,7 @@ class GpuPartitioningSuite extends AnyFunSuite { TrampolineUtil.cleanupAnyExistingSession() val conf = new SparkConf().set(RapidsConf.SHUFFLE_COMPRESSION_CODEC.key, "none") TestUtils.withGpuSparkSession(conf) { _ => - GpuShuffleEnv.init(new RapidsConf(conf), new RapidsDiskBlockManager(conf)) + GpuShuffleEnv.init(new RapidsConf(conf)) val partitionIndices = Array(0, 2, 2) val gp = new GpuPartitioning { override val numPartitions: Int = partitionIndices.length @@ -157,61 +157,53 @@ class GpuPartitioningSuite extends AnyFunSuite { val conf = new SparkConf() .set(RapidsConf.SHUFFLE_COMPRESSION_CODEC.key, codecName) TestUtils.withGpuSparkSession(conf) { _ => - GpuShuffleEnv.init(new RapidsConf(conf), new RapidsDiskBlockManager(conf)) - val spillPriority = 7L - - withResource(new RapidsDeviceMemoryStore) { store => - val catalog = new RapidsBufferCatalog(store) - val partitionIndices = Array(0, 2, 2) - val gp = new GpuPartitioning { - override val numPartitions: Int = partitionIndices.length - } - withResource(buildBatch()) { batch => - // `sliceInternalOnGpuAndClose` will close the batch, but in this test we want to - // reuse it - GpuColumnVector.incRefCounts(batch) - val columns = GpuColumnVector.extractColumns(batch) - val sparkTypes = GpuColumnVector.extractTypes(batch) - val numRows = batch.numRows - withResource( - gp.sliceInternalOnGpuAndClose(numRows, partitionIndices, columns)) { partitions => - partitions.zipWithIndex.foreach { case (partBatch, partIndex) => - val startRow = partitionIndices(partIndex) - val endRow = if (partIndex < partitionIndices.length - 1) { - partitionIndices(partIndex + 1) - } else { - batch.numRows - } - val expectedRows = endRow - startRow - assertResult(expectedRows)(partBatch.numRows) - val columns = (0 until partBatch.numCols).map(i => partBatch.column(i)) - columns.foreach { column => - // batches with any rows should be compressed, and - // batches with no rows should not be compressed. - val actualRows = column match { - case c: GpuCompressedColumnVector => - val rows = c.getTableMeta.rowCount - assert(rows != 0) - rows - case c: GpuPackedTableColumn => - val rows = c.getContiguousTable.getRowCount - assert(rows == 0) - rows - case _ => - throw new IllegalStateException("column should either be compressed or packed") - } - assertResult(expectedRows)(actualRows) + GpuShuffleEnv.init(new RapidsConf(conf)) + val partitionIndices = Array(0, 2, 2) + val gp = new GpuPartitioning { + override val numPartitions: Int = partitionIndices.length + } + withResource(buildBatch()) { batch => + // `sliceInternalOnGpuAndClose` will close the batch, but in this test we want to + // reuse it + GpuColumnVector.incRefCounts(batch) + val columns = GpuColumnVector.extractColumns(batch) + val numRows = batch.numRows + withResource( + gp.sliceInternalOnGpuAndClose(numRows, partitionIndices, columns)) { partitions => + partitions.zipWithIndex.foreach { case (partBatch, partIndex) => + val startRow = partitionIndices(partIndex) + val endRow = if (partIndex < partitionIndices.length - 1) { + partitionIndices(partIndex + 1) + } else { + batch.numRows + } + val expectedRows = endRow - startRow + assertResult(expectedRows)(partBatch.numRows) + val columns = (0 until partBatch.numCols).map(i => partBatch.column(i)) + columns.foreach { column => + // batches with any rows should be compressed, and + // batches with no rows should not be compressed. + val actualRows = column match { + case c: GpuCompressedColumnVector => + val rows = c.getTableMeta.rowCount + assert(rows != 0) + rows + case c: GpuPackedTableColumn => + val rows = c.getContiguousTable.getRowCount + assert(rows == 0) + rows + case _ => + throw new IllegalStateException( + "column should either be compressed or packed") } - if (GpuCompressedColumnVector.isBatchCompressed(partBatch)) { - val gccv = columns.head.asInstanceOf[GpuCompressedColumnVector] - val devBuffer = gccv.getTableBuffer - val handle = catalog.addBuffer(devBuffer, gccv.getTableMeta, spillPriority) - withResource(buildSubBatch(batch, startRow, endRow)) { expectedBatch => - withResource(catalog.acquireBuffer(handle)) { buffer => - withResource(buffer.getColumnarBatch(sparkTypes)) { batch => - compareBatches(expectedBatch, batch) - } - } + assertResult(expectedRows)(actualRows) + } + if (GpuCompressedColumnVector.isBatchCompressed(partBatch)) { + GpuCompressedColumnVector.incRefCounts(partBatch) + val handle = SpillableColumnarBatch(partBatch, -1) + withResource(buildSubBatch(batch, startRow, endRow)) { expectedBatch => + withResource(handle.getColumnarBatch()) { cb => + compareBatches(expectedBatch, cb) } } } @@ -220,9 +212,4 @@ class GpuPartitioningSuite extends AnyFunSuite { } } } -} - -case class MockRapidsBufferId(tableId: Int) extends RapidsBufferId { - override def getDiskPath(diskBlockManager: RapidsDiskBlockManager): File = - throw new UnsupportedOperationException -} +} \ No newline at end of file diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuSinglePartitioningSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuSinglePartitioningSuite.scala index 9211c32e142..6c5ff16ffcb 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuSinglePartitioningSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuSinglePartitioningSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -23,7 +23,7 @@ import com.nvidia.spark.rapids.Arm.withResource import org.scalatest.funsuite.AnyFunSuite import org.apache.spark.SparkConf -import org.apache.spark.sql.rapids.{GpuShuffleEnv, RapidsDiskBlockManager} +import org.apache.spark.sql.rapids.GpuShuffleEnv import org.apache.spark.sql.types.{DecimalType, DoubleType, IntegerType, StringType} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -48,7 +48,7 @@ class GpuSinglePartitioningSuite extends AnyFunSuite { .set("spark.rapids.shuffle.mode", RapidsConf.RapidsShuffleManagerMode.UCX.toString) .set(RapidsConf.SHUFFLE_COMPRESSION_CODEC.key, "none") TestUtils.withGpuSparkSession(conf) { _ => - GpuShuffleEnv.init(new RapidsConf(conf), new RapidsDiskBlockManager(conf)) + GpuShuffleEnv.init(new RapidsConf(conf)) val partitioner = GpuSinglePartitioning withResource(buildBatch()) { batch => withResource(GpuColumnVector.from(batch)) { table => diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregateRetrySuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregateRetrySuite.scala index 58608ed132c..82fb1dd4154 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregateRetrySuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregateRetrySuite.scala @@ -70,10 +70,8 @@ class HashAggregateRetrySuite when(aggHelper.aggOrdinals).thenReturn(aggOrdinals) // attempt a cuDF reduction - withResource(input) { _ => - GpuAggregateIterator.aggregate( - aggHelper, input, mockMetrics) - } + GpuAggregateIterator.aggregate( + aggHelper, input, mockMetrics) } def makeGroupByAggHelper(forceMerge: Boolean): AggHelper = { @@ -118,172 +116,187 @@ class HashAggregateRetrySuite test("computeAndAggregate reduction with retry") { val reductionBatch = buildReductionBatch() - RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 1, - RmmSpark.OomInjectionType.GPU.ordinal, 0) - val result = doReduction(reductionBatch) - withResource(result) { spillable => - withResource(spillable.getColumnarBatch) { cb => - assertResult(1)(cb.numRows) - val gcv = cb.column(0).asInstanceOf[GpuColumnVector] - withResource(gcv.getBase.copyToHost()) { hcv => - assertResult(9)(hcv.getLong(0)) + withResource(reductionBatch.incRefCount()) { _ => + RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 1, + RmmSpark.OomInjectionType.GPU.ordinal, 0) + val result = doReduction(reductionBatch) + withResource(result) { spillable => + withResource(spillable.getColumnarBatch) { cb => + assertResult(1)(cb.numRows) + val gcv = cb.column(0).asInstanceOf[GpuColumnVector] + withResource(gcv.getBase.copyToHost()) { hcv => + assertResult(9)(hcv.getLong(0)) + } } } + // we need to request a ColumnarBatch twice here for the retry + // why is this invoking the underlying method + verify(reductionBatch, times(2)).getColumnarBatch } - // we need to request a ColumnarBatch twice here for the retry - verify(reductionBatch, times(2)).getColumnarBatch() } test("computeAndAggregate reduction with two retries") { val reductionBatch = buildReductionBatch() - RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 2, - RmmSpark.OomInjectionType.GPU.ordinal, 0) - val result = doReduction(reductionBatch) - withResource(result) { spillable => - withResource(spillable.getColumnarBatch) { cb => - assertResult(1)(cb.numRows) - val gcv = cb.column(0).asInstanceOf[GpuColumnVector] - withResource(gcv.getBase.copyToHost()) { hcv => - assertResult(9)(hcv.getLong(0)) + withResource(reductionBatch.incRefCount()) { _ => + RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 2, + RmmSpark.OomInjectionType.GPU.ordinal, 0) + val result = doReduction(reductionBatch) + withResource(result) { spillable => + withResource(spillable.getColumnarBatch) { cb => + assertResult(1)(cb.numRows) + val gcv = cb.column(0).asInstanceOf[GpuColumnVector] + withResource(gcv.getBase.copyToHost()) { hcv => + assertResult(9)(hcv.getLong(0)) + } } } + // we need to request a ColumnarBatch three times, because of 1 regular attempt, + // and two retries + verify(reductionBatch, times(3)).getColumnarBatch } - // we need to request a ColumnarBatch three times, because of 1 regular attempt, - // and two retries - verify(reductionBatch, times(3)).getColumnarBatch() } test("computeAndAggregate reduction with cudf exception") { val reductionBatch = buildReductionBatch() - RmmSpark.forceCudfException(RmmSpark.getCurrentThreadId) - assertThrows[CudfException] { - doReduction(reductionBatch) + withResource(reductionBatch.incRefCount()) { _ => + RmmSpark.forceCudfException(RmmSpark.getCurrentThreadId) + assertThrows[CudfException] { + doReduction(reductionBatch) + } + // columnar batch was obtained once, but since this was not a retriable exception + // we don't retry it + verify(reductionBatch, times(1)).getColumnarBatch } - // columnar batch was obtained once, but since this was not a retriable exception - // we don't retry it - verify(reductionBatch, times(1)).getColumnarBatch() } test("computeAndAggregate group by with retry") { val groupByBatch = buildGroupByBatch() - RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 1, - RmmSpark.OomInjectionType.GPU.ordinal, 0) - val result = doGroupBy(groupByBatch) - withResource(result) { spillable => - withResource(spillable.getColumnarBatch) { cb => - assertResult(3)(cb.numRows) - val gcv = cb.column(0).asInstanceOf[GpuColumnVector] - val aggv = cb.column(1).asInstanceOf[GpuColumnVector] - var rowsLeftToMatch = 3 - withResource(aggv.getBase.copyToHost()) { aggvh => - withResource(gcv.getBase.copyToHost()) { grph => - (0 until 3).foreach { row => - if (grph.isNull(row)) { - assertResult(2L)(aggvh.getLong(row)) - rowsLeftToMatch -= 1 - } else if (grph.getInt(row) == 5) { - assertResult(1L)(aggvh.getLong(row)) - rowsLeftToMatch -= 1 - } else if (grph.getInt(row) == 1) { - assertResult(7L)(aggvh.getLong(row)) - rowsLeftToMatch -= 1 + withResource(groupByBatch.incRefCount()) { _ => + RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 1, + RmmSpark.OomInjectionType.GPU.ordinal, 0) + val result = doGroupBy(groupByBatch) + withResource(result) { spillable => + withResource(spillable.getColumnarBatch) { cb => + assertResult(3)(cb.numRows) + val gcv = cb.column(0).asInstanceOf[GpuColumnVector] + val aggv = cb.column(1).asInstanceOf[GpuColumnVector] + var rowsLeftToMatch = 3 + withResource(aggv.getBase.copyToHost()) { aggvh => + withResource(gcv.getBase.copyToHost()) { grph => + (0 until 3).foreach { row => + if (grph.isNull(row)) { + assertResult(2L)(aggvh.getLong(row)) + rowsLeftToMatch -= 1 + } else if (grph.getInt(row) == 5) { + assertResult(1L)(aggvh.getLong(row)) + rowsLeftToMatch -= 1 + } else if (grph.getInt(row) == 1) { + assertResult(7L)(aggvh.getLong(row)) + rowsLeftToMatch -= 1 + } } } } + assertResult(0)(rowsLeftToMatch) } - assertResult(0)(rowsLeftToMatch) } + // we need to request a ColumnarBatch twice here for the retry + verify(groupByBatch, times(2)).getColumnarBatch } - // we need to request a ColumnarBatch twice here for the retry - verify(groupByBatch, times(2)).getColumnarBatch() } test("computeAndAggregate reduction with split and retry") { val reductionBatch = buildReductionBatch() - RmmSpark.forceSplitAndRetryOOM(RmmSpark.getCurrentThreadId, 1, - RmmSpark.OomInjectionType.GPU.ordinal, 0) - val result = doReduction(reductionBatch) - withResource(result) { spillable => - withResource(spillable.getColumnarBatch) { cb => - assertResult(1)(cb.numRows) - val gcv = cb.column(0).asInstanceOf[GpuColumnVector] + withResource(reductionBatch.incRefCount()) { _ => + RmmSpark.forceSplitAndRetryOOM(RmmSpark.getCurrentThreadId, 1, + RmmSpark.OomInjectionType.GPU.ordinal, 0) + val result = doReduction(reductionBatch) + withResource(result) { spillable => + withResource(spillable.getColumnarBatch) { cb => + assertResult(1)(cb.numRows) + val gcv = cb.column(0).asInstanceOf[GpuColumnVector] - withResource(gcv.getBase.copyToHost()) { hcv => - assertResult(9L)(hcv.getLong(0)) + withResource(gcv.getBase.copyToHost()) { hcv => + assertResult(9L)(hcv.getLong(0)) + } } } + // the second time we access this batch is to split it + verify(reductionBatch, times(2)).getColumnarBatch } - // the second time we access this batch is to split it - verify(reductionBatch, times(2)).getColumnarBatch() } test("computeAndAggregate group by with split retry") { val groupByBatch = buildGroupByBatch() - RmmSpark.forceSplitAndRetryOOM(RmmSpark.getCurrentThreadId, 1, - RmmSpark.OomInjectionType.GPU.ordinal, 0) - val result = doGroupBy(groupByBatch) - withResource(result) { spillable => - withResource(spillable.getColumnarBatch) { cb => - assertResult(3)(cb.numRows) - val gcv = cb.column(0).asInstanceOf[GpuColumnVector] - val aggv = cb.column(1).asInstanceOf[GpuColumnVector] - var rowsLeftToMatch = 3 - withResource(aggv.getBase.copyToHost()) { aggvh => - withResource(gcv.getBase.copyToHost()) { grph => - (0 until 3).foreach { row => - if (grph.isNull(row)) { - assertResult(2L)(aggvh.getLong(row)) - rowsLeftToMatch -= 1 - } else if (grph.getInt(row) == 5) { - assertResult(1L)(aggvh.getLong(row)) - rowsLeftToMatch -= 1 - } else if (grph.getInt(row) == 1) { - assertResult(7L)(aggvh.getLong(row)) - rowsLeftToMatch -= 1 + withResource(groupByBatch.incRefCount()) { _ => + RmmSpark.forceSplitAndRetryOOM(RmmSpark.getCurrentThreadId, 1, + RmmSpark.OomInjectionType.GPU.ordinal, 0) + val result = doGroupBy(groupByBatch) + withResource(result) { spillable => + withResource(spillable.getColumnarBatch) { cb => + assertResult(3)(cb.numRows) + val gcv = cb.column(0).asInstanceOf[GpuColumnVector] + val aggv = cb.column(1).asInstanceOf[GpuColumnVector] + var rowsLeftToMatch = 3 + withResource(aggv.getBase.copyToHost()) { aggvh => + withResource(gcv.getBase.copyToHost()) { grph => + (0 until 3).foreach { row => + if (grph.isNull(row)) { + assertResult(2L)(aggvh.getLong(row)) + rowsLeftToMatch -= 1 + } else if (grph.getInt(row) == 5) { + assertResult(1L)(aggvh.getLong(row)) + rowsLeftToMatch -= 1 + } else if (grph.getInt(row) == 1) { + assertResult(7L)(aggvh.getLong(row)) + rowsLeftToMatch -= 1 + } } } } + assertResult(0)(rowsLeftToMatch) } - assertResult(0)(rowsLeftToMatch) } + // the second time we access this batch is to split it + verify(groupByBatch, times(2)).getColumnarBatch } - // the second time we access this batch is to split it - verify(groupByBatch, times(2)).getColumnarBatch() } test("computeAndAggregate group by with retry and forceMerge") { // with forceMerge we expect 1 batch to be returned at all costs val groupByBatch = buildGroupByBatch() - // we force a split because that would cause us to compute two aggs - RmmSpark.forceSplitAndRetryOOM(RmmSpark.getCurrentThreadId, 1, - RmmSpark.OomInjectionType.GPU.ordinal, 0) - val result = doGroupBy(groupByBatch, forceMerge = true) - withResource(result) { spillable => - withResource(spillable.getColumnarBatch) { cb => - assertResult(3)(cb.numRows) - val gcv = cb.column(0).asInstanceOf[GpuColumnVector] - val aggv = cb.column(1).asInstanceOf[GpuColumnVector] - var rowsLeftToMatch = 3 - withResource(aggv.getBase.copyToHost()) { aggvh => - withResource(gcv.getBase.copyToHost()) { grph => - (0 until 3).foreach { row => - if (grph.isNull(row)) { - assertResult(2L)(aggvh.getLong(row)) - rowsLeftToMatch -= 1 - } else if (grph.getInt(row) == 5) { - assertResult(1L)(aggvh.getLong(row)) - rowsLeftToMatch -= 1 - } else if (grph.getInt(row) == 1) { - assertResult(7L)(aggvh.getLong(row)) - rowsLeftToMatch -= 1 + withResource(groupByBatch.incRefCount()) { _ => + // we force a split because that would cause us to compute two aggs + RmmSpark.forceSplitAndRetryOOM(RmmSpark.getCurrentThreadId, 1, + RmmSpark.OomInjectionType.GPU.ordinal, 0) + val result = doGroupBy(groupByBatch, forceMerge = true) + withResource(result) { spillable => + withResource(spillable.getColumnarBatch) { cb => + assertResult(3)(cb.numRows) + val gcv = cb.column(0).asInstanceOf[GpuColumnVector] + val aggv = cb.column(1).asInstanceOf[GpuColumnVector] + var rowsLeftToMatch = 3 + withResource(aggv.getBase.copyToHost()) { aggvh => + withResource(gcv.getBase.copyToHost()) { grph => + (0 until 3).foreach { row => + if (grph.isNull(row)) { + assertResult(2L)(aggvh.getLong(row)) + rowsLeftToMatch -= 1 + } else if (grph.getInt(row) == 5) { + assertResult(1L)(aggvh.getLong(row)) + rowsLeftToMatch -= 1 + } else if (grph.getInt(row) == 1) { + assertResult(7L)(aggvh.getLong(row)) + rowsLeftToMatch -= 1 + } } } } + assertResult(0)(rowsLeftToMatch) } - assertResult(0)(rowsLeftToMatch) } + // we need to request a ColumnarBatch twice here for the retry + verify(groupByBatch, times(2)).getColumnarBatch } - // we need to request a ColumnarBatch twice here for the retry - verify(groupByBatch, times(2)).getColumnarBatch() } } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsBufferCatalogSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsBufferCatalogSuite.scala deleted file mode 100644 index 9b5b37af480..00000000000 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsBufferCatalogSuite.scala +++ /dev/null @@ -1,368 +0,0 @@ -/* - * 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. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids - -import java.io.File - -import ai.rapids.cudf.{Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer} -import com.nvidia.spark.rapids.Arm.withResource -import com.nvidia.spark.rapids.StorageTier.{DEVICE, DISK, HOST, StorageTier} -import com.nvidia.spark.rapids.format.TableMeta -import org.mockito.ArgumentMatchers.any -import org.mockito.Mockito._ -import org.scalatest.funsuite.AnyFunSuite -import org.scalatestplus.mockito.MockitoSugar - -import org.apache.spark.sql.rapids.RapidsDiskBlockManager -import org.apache.spark.sql.types.DataType -import org.apache.spark.sql.vectorized.ColumnarBatch - -class RapidsBufferCatalogSuite extends AnyFunSuite with MockitoSugar { - test("lookup unknown buffer") { - val catalog = new RapidsBufferCatalog - val bufferId = new RapidsBufferId { - override val tableId: Int = 10 - override def getDiskPath(m: RapidsDiskBlockManager): File = null - } - val bufferHandle = new RapidsBufferHandle { - override val id: RapidsBufferId = bufferId - override def setSpillPriority(newPriority: Long): Unit = {} - override def close(): Unit = {} - } - - assertThrows[NoSuchElementException](catalog.acquireBuffer(bufferHandle)) - assertThrows[NoSuchElementException](catalog.getBufferMeta(bufferId)) - } - - test("buffer double register throws") { - val catalog = new RapidsBufferCatalog - val bufferId = MockBufferId(5) - val buffer = mockBuffer(bufferId) - catalog.registerNewBuffer(buffer) - val buffer2 = mockBuffer(bufferId) - assertThrows[DuplicateBufferException](catalog.registerNewBuffer(buffer2)) - } - - test("a second handle prevents buffer to be removed") { - val catalog = new RapidsBufferCatalog - val bufferId = MockBufferId(5) - val buffer = mockBuffer(bufferId) - catalog.registerNewBuffer(buffer) - val handle1 = - catalog.makeNewHandle(bufferId, -1) - val handle2 = - catalog.makeNewHandle(bufferId, -1) - - handle1.close() - - // this does not throw - catalog.acquireBuffer(handle2).close() - // actually this doesn't throw either - catalog.acquireBuffer(handle1).close() - - handle2.close() - - assertThrows[NoSuchElementException](catalog.acquireBuffer(handle1)) - assertThrows[NoSuchElementException](catalog.acquireBuffer(handle2)) - } - - test("spill priorities are updated as handles are registered and unregistered") { - val catalog = new RapidsBufferCatalog - val bufferId = MockBufferId(5) - val buffer = mockBuffer(bufferId, initialPriority = -1) - catalog.registerNewBuffer(buffer) - val handle1 = - catalog.makeNewHandle(bufferId, -1) - withResource(catalog.acquireBuffer(handle1)) { buff => - assertResult(-1)(buff.getSpillPriority) - } - val handle2 = - catalog.makeNewHandle(bufferId, 0) - withResource(catalog.acquireBuffer(handle2)) { buff => - assertResult(0)(buff.getSpillPriority) - } - - // removing the lower priority handle, keeps the high priority spill - handle1.close() - withResource(catalog.acquireBuffer(handle2)) { buff => - assertResult(0)(buff.getSpillPriority) - } - - // adding a lower priority -1000 handle keeps the high priority (0) spill - val handle3 = - catalog.makeNewHandle(bufferId, -1000) - withResource(catalog.acquireBuffer(handle3)) { buff => - assertResult(0)(buff.getSpillPriority) - } - - // removing the high priority spill (0) brings us down to the - // low priority that is remaining - handle2.close() - withResource(catalog.acquireBuffer(handle2)) { buff => - assertResult(-1000)(buff.getSpillPriority) - } - - handle3.close() - } - - test("buffer registering slower tier does not hide faster tier") { - val catalog = new RapidsBufferCatalog - val bufferId = MockBufferId(5) - val buffer = mockBuffer(bufferId, tier = DEVICE) - catalog.registerNewBuffer(buffer) - val handle = catalog.makeNewHandle(bufferId, 0) - val buffer2 = mockBuffer(bufferId, tier = HOST) - catalog.registerNewBuffer(buffer2) - val buffer3 = mockBuffer(bufferId, tier = DISK) - catalog.registerNewBuffer(buffer3) - val acquired = catalog.acquireBuffer(handle) - assertResult(5)(acquired.id.tableId) - assertResult(buffer)(acquired) - - // registering the handle acquires the buffer - verify(buffer, times(2)).addReference() - } - - test("acquire buffer") { - val catalog = new RapidsBufferCatalog - val bufferId = MockBufferId(5) - val buffer = mockBuffer(bufferId) - catalog.registerNewBuffer(buffer) - val handle = catalog.makeNewHandle(bufferId, 0) - val acquired = catalog.acquireBuffer(handle) - assertResult(5)(acquired.id.tableId) - assertResult(buffer)(acquired) - - // registering the handle acquires the buffer - verify(buffer, times(2)).addReference() - } - - test("acquire buffer retries automatically") { - val catalog = new RapidsBufferCatalog - val bufferId = MockBufferId(5) - val buffer = mockBuffer(bufferId, acquireAttempts = 9) - catalog.registerNewBuffer(buffer) - val handle = catalog.makeNewHandle(bufferId, 0) - val acquired = catalog.acquireBuffer(handle) - assertResult(5)(acquired.id.tableId) - assertResult(buffer)(acquired) - - // registering the handle acquires the buffer - verify(buffer, times(10)).addReference() - } - - test("acquire buffer at specific tier") { - val catalog = new RapidsBufferCatalog - val bufferId = MockBufferId(5) - val buffer = mockBuffer(bufferId, tier = DEVICE) - catalog.registerNewBuffer(buffer) - val buffer2 = mockBuffer(bufferId, tier = HOST) - catalog.registerNewBuffer(buffer2) - val acquired = catalog.acquireBuffer(MockBufferId(5), HOST).get - assertResult(5)(acquired.id.tableId) - assertResult(buffer2)(acquired) - verify(buffer2).addReference() - } - - test("acquire buffer at nonexistent tier") { - val catalog = new RapidsBufferCatalog - val bufferId = MockBufferId(5) - val buffer = mockBuffer(bufferId, tier = HOST) - catalog.registerNewBuffer(buffer) - assert(catalog.acquireBuffer(MockBufferId(5), DEVICE).isEmpty) - assert(catalog.acquireBuffer(MockBufferId(5), DISK).isEmpty) - } - - test("get buffer meta") { - val catalog = new RapidsBufferCatalog - val bufferId = MockBufferId(5) - val expectedMeta = new TableMeta - val buffer = mockBuffer(bufferId, tableMeta = expectedMeta) - catalog.registerNewBuffer(buffer) - val meta = catalog.getBufferMeta(bufferId) - assertResult(expectedMeta)(meta) - } - - test("buffer is spilled to slower tier only") { - val catalog = new RapidsBufferCatalog - val bufferId = MockBufferId(5) - val buffer = mockBuffer(bufferId, tier = DEVICE) - catalog.registerNewBuffer(buffer) - val buffer2 = mockBuffer(bufferId, tier = HOST) - catalog.registerNewBuffer(buffer2) - val buffer3 = mockBuffer(bufferId, tier = DISK) - catalog.registerNewBuffer(buffer3) - assert(catalog.isBufferSpilled(bufferId, DEVICE)) - assert(catalog.isBufferSpilled(bufferId, HOST)) - assert(!catalog.isBufferSpilled(bufferId, DISK)) - } - - test("multiple calls to unspill return existing DEVICE buffer") { - withResource(spy(new RapidsDeviceMemoryStore)) { deviceStore => - val mockStore = mock[RapidsBufferStore] - withResource( - new RapidsHostMemoryStore(Some(10000))) { hostStore => - deviceStore.setSpillStore(hostStore) - hostStore.setSpillStore(mockStore) - val catalog = new RapidsBufferCatalog(deviceStore) - val handle = withResource(DeviceMemoryBuffer.allocate(1024)) { buff => - val meta = MetaUtils.getTableMetaNoTable(buff.getLength) - catalog.addBuffer( - buff, meta, -1) - } - withResource(handle) { _ => - catalog.synchronousSpill(deviceStore, 0) - val acquiredHostBuffer = catalog.acquireBuffer(handle) - val unspilled = withResource(acquiredHostBuffer) { _ => - assertResult(HOST)(acquiredHostBuffer.storageTier) - val unspilled = - catalog.unspillBufferToDeviceStore( - acquiredHostBuffer, - Cuda.DEFAULT_STREAM) - withResource(unspilled) { _ => - assertResult(DEVICE)(unspilled.storageTier) - } - val unspilledSame = catalog.unspillBufferToDeviceStore( - acquiredHostBuffer, - Cuda.DEFAULT_STREAM) - withResource(unspilledSame) { _ => - assertResult(unspilled)(unspilledSame) - } - // verify that we invoked the copy function exactly once - verify(deviceStore, times(1)).copyBuffer(any(), any(), any()) - unspilled - } - val unspilledSame = catalog.unspillBufferToDeviceStore( - acquiredHostBuffer, - Cuda.DEFAULT_STREAM) - withResource(unspilledSame) { _ => - assertResult(unspilled)(unspilledSame) - } - // verify that we invoked the copy function exactly once - verify(deviceStore, times(1)).copyBuffer(any(), any(), any()) - } - } - } - } - - test("remove buffer tier") { - val catalog = new RapidsBufferCatalog - val bufferId = MockBufferId(5) - val buffer = mockBuffer(bufferId, tier = DEVICE) - catalog.registerNewBuffer(buffer) - val buffer2 = mockBuffer(bufferId, tier = HOST) - catalog.registerNewBuffer(buffer2) - val buffer3 = mockBuffer(bufferId, tier = DISK) - catalog.registerNewBuffer(buffer3) - catalog.removeBufferTier(bufferId, DEVICE) - catalog.removeBufferTier(bufferId, DISK) - assert(catalog.acquireBuffer(MockBufferId(5), DEVICE).isEmpty) - assert(catalog.acquireBuffer(MockBufferId(5), HOST).isDefined) - assert(catalog.acquireBuffer(MockBufferId(5), DISK).isEmpty) - } - - test("remove nonexistent buffer tier") { - val catalog = new RapidsBufferCatalog - val bufferId = MockBufferId(5) - val buffer = mockBuffer(bufferId, tier = DEVICE) - catalog.registerNewBuffer(buffer) - catalog.removeBufferTier(bufferId, HOST) - catalog.removeBufferTier(bufferId, DISK) - assert(catalog.acquireBuffer(MockBufferId(5), DEVICE).isDefined) - assert(catalog.acquireBuffer(MockBufferId(5), HOST).isEmpty) - assert(catalog.acquireBuffer(MockBufferId(5), DISK).isEmpty) - } - - test("remove buffer releases buffer resources") { - val catalog = new RapidsBufferCatalog - val bufferId = MockBufferId(5) - val buffer = mockBuffer(bufferId) - catalog.registerNewBuffer(buffer) - val handle = catalog.makeNewHandle( - bufferId, -1) - handle.close() - verify(buffer).free() - } - - test("remove buffer releases buffer resources at all tiers") { - val catalog = new RapidsBufferCatalog - val bufferId = MockBufferId(5) - val buffer = mockBuffer(bufferId, tier = DEVICE) - catalog.registerNewBuffer(buffer) - val handle = catalog.makeNewHandle( - bufferId, -1) - - // these next registrations don't get their own handle. This is an internal - // operation from the store where it has spilled to host and disk the RapidsBuffer - val buffer2 = mockBuffer(bufferId, tier = HOST) - catalog.registerNewBuffer(buffer2) - val buffer3 = mockBuffer(bufferId, tier = DISK) - catalog.registerNewBuffer(buffer3) - - // removing the original handle removes all buffers from all tiers. - handle.close() - verify(buffer).free() - verify(buffer2).free() - verify(buffer3).free() - } - - private def mockBuffer( - bufferId: RapidsBufferId, - tableMeta: TableMeta = null, - tier: StorageTier = StorageTier.DEVICE, - acquireAttempts: Int = 1, - initialPriority: Long = -1): RapidsBuffer = { - spy(new RapidsBuffer { - var _acquireAttempts: Int = acquireAttempts - var currentPriority: Long = initialPriority - override val id: RapidsBufferId = bufferId - override val memoryUsedBytes: Long = 0 - override def meta: TableMeta = tableMeta - override val storageTier: StorageTier = tier - override def getColumnarBatch(sparkTypes: Array[DataType]): ColumnarBatch = null - override def getMemoryBuffer: MemoryBuffer = null - override def copyToMemoryBuffer( - srcOffset: Long, - dst: MemoryBuffer, - dstOffset: Long, - length: Long, - stream: Cuda.Stream): Unit = {} - override def getDeviceMemoryBuffer: DeviceMemoryBuffer = null - override def getHostMemoryBuffer: HostMemoryBuffer = null - override def addReference(): Boolean = { - if (_acquireAttempts > 0) { - _acquireAttempts -= 1 - } - _acquireAttempts == 0 - } - override def free(): Unit = {} - override def getSpillPriority: Long = currentPriority - override def setSpillPriority(priority: Long): Unit = { - currentPriority = priority - } - - override def withMemoryBufferReadLock[K](body: MemoryBuffer => K): K = { body(null) } - override def withMemoryBufferWriteLock[K](body: MemoryBuffer => K): K = { body(null) } - override def close(): Unit = {} - }) - } -} - -case class MockBufferId(override val tableId: Int) extends RapidsBufferId { - override def getDiskPath(dbm: RapidsDiskBlockManager): File = - throw new UnsupportedOperationException -} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStoreSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStoreSuite.scala deleted file mode 100644 index 45d96be4cb6..00000000000 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStoreSuite.scala +++ /dev/null @@ -1,489 +0,0 @@ -/* - * 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. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids - -import java.io.File -import java.math.RoundingMode - -import scala.collection.mutable.ArrayBuffer - -import ai.rapids.cudf.{ColumnVector, ContiguousTable, Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer, Table} -import com.nvidia.spark.rapids.Arm.withResource -import com.nvidia.spark.rapids.StorageTier.StorageTier -import com.nvidia.spark.rapids.format.TableMeta -import org.mockito.ArgumentCaptor -import org.mockito.Mockito.{spy, verify} -import org.scalatest.funsuite.AnyFunSuite -import org.scalatestplus.mockito.MockitoSugar - -import org.apache.spark.sql.rapids.RapidsDiskBlockManager -import org.apache.spark.sql.types.{DataType, DecimalType, DoubleType, IntegerType, StringType} - -class RapidsDeviceMemoryStoreSuite extends AnyFunSuite with MockitoSugar { - private def buildTable(): Table = { - new Table.TestBuilder() - .column(5, null.asInstanceOf[java.lang.Integer], 3, 1) - .column("five", "two", null, null) - .column(5.0D, 2.0D, 3.0D, 1.0D) - .decimal64Column(-5, RoundingMode.UNNECESSARY, 0, null, -1.4, 10.123) - .build() - } - - private def buildTableWithDuplicate(): Table = { - withResource(ColumnVector.fromInts(5, null.asInstanceOf[java.lang.Integer], 3, 1)) { intCol => - withResource(ColumnVector.fromStrings("five", "two", null, null)) { stringCol => - withResource(ColumnVector.fromDoubles(5.0, 2.0, 3.0, 1.0)) { doubleCol => - // add intCol twice - new Table(intCol, intCol, stringCol, doubleCol) - } - } - } - } - - private def buildContiguousTable(): ContiguousTable = { - withResource(buildTable()) { table => - table.contiguousSplit()(0) - } - } - - test("add table registers with catalog") { - withResource(new RapidsDeviceMemoryStore) { store => - val catalog = spy(new RapidsBufferCatalog(store)) - val spillPriority = 3 - val bufferId = MockRapidsBufferId(7) - withResource(buildContiguousTable()) { ct => - catalog.addContiguousTable( - bufferId, ct, spillPriority, false) - } - val captor: ArgumentCaptor[RapidsBuffer] = ArgumentCaptor.forClass(classOf[RapidsBuffer]) - verify(catalog).registerNewBuffer(captor.capture()) - val resultBuffer = captor.getValue - assertResult(bufferId)(resultBuffer.id) - assertResult(spillPriority)(resultBuffer.getSpillPriority) - } - } - - test("a non-contiguous table is spillable and it is handed over to the store") { - withResource(new RapidsDeviceMemoryStore) { store => - val catalog = spy(new RapidsBufferCatalog(store)) - val spillPriority = 3 - val table = buildTable() - catalog.addTable(table, spillPriority) - val buffSize = GpuColumnVector.getTotalDeviceMemoryUsed(table) - assertResult(buffSize)(store.currentSize) - assertResult(buffSize)(store.currentSpillableSize) - } - } - - test("a non-contiguous table becomes non-spillable when batch is obtained") { - withResource(new RapidsDeviceMemoryStore) { store => - val catalog = spy(new RapidsBufferCatalog(store)) - val spillPriority = 3 - val table = buildTable() - val handle = catalog.addTable(table, spillPriority) - val types: Array[DataType] = - Seq(IntegerType, StringType, DoubleType, DecimalType(10, 5)).toArray - val buffSize = GpuColumnVector.getTotalDeviceMemoryUsed(table) - assertResult(buffSize)(store.currentSize) - assertResult(buffSize)(store.currentSpillableSize) - val batch = withResource(catalog.acquireBuffer(handle)) { rapidsBuffer => - rapidsBuffer.getColumnarBatch(types) - } - withResource(batch) { _ => - assertResult(buffSize)(store.currentSize) - assertResult(0)(store.currentSpillableSize) - } - assertResult(buffSize)(store.currentSpillableSize) - } - } - - test("a non-contiguous table is non-spillable until all columns are returned") { - withResource(new RapidsDeviceMemoryStore) { store => - val catalog = spy(new RapidsBufferCatalog(store)) - val spillPriority = 3 - val table = buildTable() - val handle = catalog.addTable(table, spillPriority) - val types: Array[DataType] = - Seq(IntegerType, StringType, DoubleType, DecimalType(10, 5)).toArray - val buffSize = GpuColumnVector.getTotalDeviceMemoryUsed(table) - assertResult(buffSize)(store.currentSize) - assertResult(buffSize)(store.currentSpillableSize) - // incRefCount all the columns via `batch` - val batch = withResource(catalog.acquireBuffer(handle)) { rapidsBuffer => - rapidsBuffer.getColumnarBatch(types) - } - val columns = GpuColumnVector.extractBases(batch) - withResource(columns.head) { _ => - columns.head.incRefCount() - withResource(batch) { _ => - assertResult(buffSize)(store.currentSize) - assertResult(0)(store.currentSpillableSize) - } - // still 0 after the batch is closed, because of the extra incRefCount - // for columns.head - assertResult(0)(store.currentSpillableSize) - } - // columns.head is closed, so now our RapidsTable is spillable again - assertResult(buffSize)(store.currentSpillableSize) - } - } - - test("an aliased non-contiguous table is not spillable (until closing the alias) ") { - withResource(new RapidsDeviceMemoryStore) { store => - val catalog = spy(new RapidsBufferCatalog(store)) - val spillPriority = 3 - val table = buildTable() - val handle = catalog.addTable(table, spillPriority) - val types: Array[DataType] = - Seq(IntegerType, StringType, DoubleType, DecimalType(10, 5)).toArray - val buffSize = GpuColumnVector.getTotalDeviceMemoryUsed(table) - assertResult(buffSize)(store.currentSize) - assertResult(buffSize)(store.currentSpillableSize) - val aliasHandle = withResource(catalog.acquireBuffer(handle)) { rapidsBuffer => - // extract the batch from the table we added, and add it back as a batch - withResource(rapidsBuffer.getColumnarBatch(types)) { batch => - catalog.addBatch(batch, spillPriority) - } - } // we now have two copies in the store - assertResult(buffSize*2)(store.currentSize) - assertResult(0)(store.currentSpillableSize) - - aliasHandle.close() // remove the alias - - assertResult(buffSize)(store.currentSize) - assertResult(buffSize)(store.currentSpillableSize) - } - } - - test("an aliased non-contiguous table is not spillable (until closing the original) ") { - withResource(new RapidsDeviceMemoryStore) { store => - val catalog = spy(new RapidsBufferCatalog(store)) - val spillPriority = 3 - val table = buildTable() - val handle = catalog.addTable(table, spillPriority) - val types: Array[DataType] = - Seq(IntegerType, StringType, DoubleType, DecimalType(10, 5)).toArray - val buffSize = GpuColumnVector.getTotalDeviceMemoryUsed(table) - assertResult(buffSize)(store.currentSize) - assertResult(buffSize)(store.currentSpillableSize) - withResource(catalog.acquireBuffer(handle)) { rapidsBuffer => - // extract the batch from the table we added, and add it back as a batch - withResource(rapidsBuffer.getColumnarBatch(types)) { batch => - catalog.addBatch(batch, spillPriority) - } - } // we now have two copies in the store - assertResult(buffSize * 2)(store.currentSize) - assertResult(0)(store.currentSpillableSize) - - handle.close() // remove the original - - assertResult(buffSize)(store.currentSize) - assertResult(buffSize)(store.currentSpillableSize) - } - } - - test("an non-contiguous table supports duplicated columns") { - withResource(new RapidsDeviceMemoryStore) { store => - val catalog = spy(new RapidsBufferCatalog(store)) - val spillPriority = 3 - val table = buildTableWithDuplicate() - val handle = catalog.addTable(table, spillPriority) - val types: Array[DataType] = - Seq(IntegerType, IntegerType, StringType, DoubleType).toArray - val buffSize = GpuColumnVector.getTotalDeviceMemoryUsed(table) - assertResult(buffSize)(store.currentSize) - assertResult(buffSize)(store.currentSpillableSize) - withResource(catalog.acquireBuffer(handle)) { rapidsBuffer => - // extract the batch from the table we added, and add it back as a batch - withResource(rapidsBuffer.getColumnarBatch(types)) { batch => - catalog.addBatch(batch, spillPriority) - } - } // we now have two copies in the store - assertResult(buffSize * 2)(store.currentSize) - assertResult(0)(store.currentSpillableSize) - - handle.close() // remove the original - - assertResult(buffSize)(store.currentSize) - assertResult(buffSize)(store.currentSpillableSize) - } - } - - test("a contiguous table is not spillable until the owner closes it") { - withResource(new RapidsDeviceMemoryStore) { store => - val catalog = spy(new RapidsBufferCatalog(store)) - val spillPriority = 3 - val bufferId = MockRapidsBufferId(7) - val ct = buildContiguousTable() - val buffSize = ct.getBuffer.getLength - withResource(ct) { _ => - catalog.addContiguousTable( - bufferId, - ct, - spillPriority, - false) - assertResult(buffSize)(store.currentSize) - assertResult(0)(store.currentSpillableSize) - } - // after closing the original table, the RapidsBuffer should be spillable - assertResult(buffSize)(store.currentSize) - assertResult(buffSize)(store.currentSpillableSize) - } - } - - test("a buffer is not spillable until the owner closes columns referencing it") { - withResource(new RapidsDeviceMemoryStore) { store => - val spillPriority = 3 - val bufferId = MockRapidsBufferId(7) - val ct = buildContiguousTable() - val buffSize = ct.getBuffer.getLength - withResource(ct) { _ => - val meta = MetaUtils.buildTableMeta(bufferId.tableId, ct) - withResource(ct) { _ => - store.addBuffer( - bufferId, - ct.getBuffer, - meta, - spillPriority, - false) - assertResult(buffSize)(store.currentSize) - assertResult(0)(store.currentSpillableSize) - } - } - // after closing the original table, the RapidsBuffer should be spillable - assertResult(buffSize)(store.currentSize) - assertResult(buffSize)(store.currentSpillableSize) - } - } - - test("a buffer is not spillable when the underlying device buffer is obtained from it") { - withResource(new RapidsDeviceMemoryStore) { store => - val spillPriority = 3 - val bufferId = MockRapidsBufferId(7) - val ct = buildContiguousTable() - val buffSize = ct.getBuffer.getLength - val buffer = withResource(ct) { _ => - val meta = MetaUtils.buildTableMeta(bufferId.tableId, ct) - val buffer = store.addBuffer( - bufferId, - ct.getBuffer, - meta, - spillPriority, - false) - assertResult(buffSize)(store.currentSize) - assertResult(0)(store.currentSpillableSize) - buffer - } - - // after closing the original table, the RapidsBuffer should be spillable - assertResult(buffSize)(store.currentSize) - assertResult(buffSize)(store.currentSpillableSize) - - // if a device memory buffer is obtained from the buffer, it is no longer spillable - withResource(buffer.getDeviceMemoryBuffer) { deviceBuffer => - assertResult(buffSize)(store.currentSize) - assertResult(0)(store.currentSpillableSize) - } - - // once the DeviceMemoryBuffer is closed, the RapidsBuffer should be spillable again - assertResult(buffSize)(store.currentSpillableSize) - } - } - - test("add buffer registers with catalog") { - withResource(new RapidsDeviceMemoryStore) { store => - val catalog = spy(new RapidsBufferCatalog(store)) - val spillPriority = 3 - val bufferId = MockRapidsBufferId(7) - val meta = withResource(buildContiguousTable()) { ct => - val meta = MetaUtils.buildTableMeta(bufferId.tableId, ct) - withResource(ct) { _ => - catalog.addBuffer( - bufferId, - ct.getBuffer, - meta, - spillPriority, - false) - } - meta - } - val captor: ArgumentCaptor[RapidsBuffer] = ArgumentCaptor.forClass(classOf[RapidsBuffer]) - verify(catalog).registerNewBuffer(captor.capture()) - val resultBuffer = captor.getValue - assertResult(bufferId)(resultBuffer.id) - assertResult(spillPriority)(resultBuffer.getSpillPriority) - assertResult(meta)(resultBuffer.meta) - } - } - - test("get memory buffer") { - withResource(new RapidsDeviceMemoryStore) { store => - val catalog = spy(new RapidsBufferCatalog(store)) - val bufferId = MockRapidsBufferId(7) - withResource(buildContiguousTable()) { ct => - withResource(HostMemoryBuffer.allocate(ct.getBuffer.getLength)) { expectedHostBuffer => - expectedHostBuffer.copyFromDeviceBuffer(ct.getBuffer) - val meta = MetaUtils.buildTableMeta(bufferId.tableId, ct) - val handle = withResource(ct) { _ => - catalog.addBuffer( - bufferId, - ct.getBuffer, - meta, - initialSpillPriority = 3, - needsSync = false) - } - withResource(catalog.acquireBuffer(handle)) { buffer => - withResource(buffer.getMemoryBuffer.asInstanceOf[DeviceMemoryBuffer]) { devbuf => - withResource(HostMemoryBuffer.allocate(devbuf.getLength)) { actualHostBuffer => - actualHostBuffer.copyFromDeviceBuffer(devbuf) - assertResult(expectedHostBuffer.asByteBuffer())(actualHostBuffer.asByteBuffer()) - } - } - } - } - } - } - } - - test("get column batch") { - withResource(new RapidsDeviceMemoryStore) { store => - val catalog = new RapidsBufferCatalog(store) - val sparkTypes = Array[DataType](IntegerType, StringType, DoubleType, - DecimalType(ai.rapids.cudf.DType.DECIMAL64_MAX_PRECISION, 5)) - val bufferId = MockRapidsBufferId(7) - withResource(buildContiguousTable()) { ct => - withResource(GpuColumnVector.from(ct.getTable, sparkTypes)) { - expectedBatch => - val meta = MetaUtils.buildTableMeta(bufferId.tableId, ct) - val handle = withResource(ct) { _ => - catalog.addBuffer( - bufferId, - ct.getBuffer, - meta, - initialSpillPriority = 3, - false) - } - withResource(catalog.acquireBuffer(handle)) { buffer => - withResource(buffer.getColumnarBatch(sparkTypes)) { actualBatch => - TestUtils.compareBatches(expectedBatch, actualBatch) - } - } - } - } - } - } - - test("size statistics") { - - withResource(new RapidsDeviceMemoryStore) { store => - val catalog = new RapidsBufferCatalog(store) - assertResult(0)(store.currentSize) - val bufferSizes = new Array[Long](2) - val bufferHandles = new Array[RapidsBufferHandle](2) - bufferSizes.indices.foreach { i => - withResource(buildContiguousTable()) { ct => - bufferSizes(i) = ct.getBuffer.getLength - // store takes ownership of the table - bufferHandles(i) = - catalog.addContiguousTable( - MockRapidsBufferId(i), - ct, - initialSpillPriority = 0, - false) - } - assertResult(bufferSizes.take(i+1).sum)(store.currentSize) - } - bufferHandles(0).close() - assertResult(bufferSizes(1))(store.currentSize) - bufferHandles(1).close() - assertResult(0)(store.currentSize) - } - } - - test("spill") { - val spillStore = new MockSpillStore - val spillPriorities = Array(0, -1, 2) - val bufferSizes = new Array[Long](spillPriorities.length) - withResource(new RapidsDeviceMemoryStore) { store => - val catalog = new RapidsBufferCatalog(store) - store.setSpillStore(spillStore) - spillPriorities.indices.foreach { i => - withResource(buildContiguousTable()) { ct => - bufferSizes(i) = ct.getBuffer.getLength - // store takes ownership of the table - catalog.addContiguousTable( - MockRapidsBufferId(i), ct, spillPriorities(i), - false) - } - } - assert(spillStore.spilledBuffers.isEmpty) - - // asking to spill 0 bytes should not spill - val sizeBeforeSpill = store.currentSize - catalog.synchronousSpill(store, sizeBeforeSpill) - assert(spillStore.spilledBuffers.isEmpty) - assertResult(sizeBeforeSpill)(store.currentSize) - catalog.synchronousSpill(store, sizeBeforeSpill + 1) - assert(spillStore.spilledBuffers.isEmpty) - assertResult(sizeBeforeSpill)(store.currentSize) - - // spilling 1 byte should force one buffer to spill in priority order - catalog.synchronousSpill(store, sizeBeforeSpill - 1) - assertResult(1)(spillStore.spilledBuffers.length) - assertResult(bufferSizes.drop(1).sum)(store.currentSize) - assertResult(1)(spillStore.spilledBuffers(0).tableId) - - // spilling to zero should force all buffers to spill in priority order - catalog.synchronousSpill(store, 0) - assertResult(3)(spillStore.spilledBuffers.length) - assertResult(0)(store.currentSize) - assertResult(0)(spillStore.spilledBuffers(1).tableId) - assertResult(2)(spillStore.spilledBuffers(2).tableId) - } - } - - case class MockRapidsBufferId(tableId: Int) extends RapidsBufferId { - override def getDiskPath(diskBlockManager: RapidsDiskBlockManager): File = - throw new UnsupportedOperationException - } - - class MockSpillStore extends RapidsBufferStore(StorageTier.HOST) { - val spilledBuffers = new ArrayBuffer[RapidsBufferId] - - override protected def createBuffer( - b: RapidsBuffer, - c: RapidsBufferCatalog, - s: Cuda.Stream): Option[RapidsBufferBase] = { - spilledBuffers += b.id - Some(new MockRapidsBuffer( - b.id, b.getPackedSizeBytes, b.meta, b.getSpillPriority)) - } - - class MockRapidsBuffer(id: RapidsBufferId, size: Long, meta: TableMeta, spillPriority: Long) - extends RapidsBufferBase(id, meta, spillPriority) { - override protected def releaseResources(): Unit = {} - - override val storageTier: StorageTier = StorageTier.HOST - - override def getMemoryBuffer: MemoryBuffer = - throw new UnsupportedOperationException - - /** The size of this buffer in bytes. */ - override val memoryUsedBytes: Long = size - } - } -} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDiskStoreSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDiskStoreSuite.scala deleted file mode 100644 index fce88f116b3..00000000000 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDiskStoreSuite.scala +++ /dev/null @@ -1,607 +0,0 @@ -/* - * 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. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids - -import java.io.File -import java.math.RoundingMode - -import ai.rapids.cudf.{ColumnVector, ContiguousTable, Cuda, DeviceMemoryBuffer, HostMemoryBuffer, Table} -import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} -import org.mockito.ArgumentMatchers -import org.mockito.ArgumentMatchers.any -import org.mockito.Mockito.{spy, times, verify, when} -import org.scalatestplus.mockito.MockitoSugar - -import org.apache.spark.SparkConf -import org.apache.spark.sql.rapids.{RapidsDiskBlockManager, TempSpillBufferId} -import org.apache.spark.sql.types.{DataType, DecimalType, DoubleType, IntegerType, StringType} -import org.apache.spark.storage.{BlockId, ShuffleBlockId} - - -class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { - - private def buildContiguousTable(): ContiguousTable = { - withResource(buildTable()) { table => - table.contiguousSplit()(0) - } - } - - private def buildTable(): Table = { - new Table.TestBuilder() - .column(5, null.asInstanceOf[java.lang.Integer], 3, 1) - .column("five", "two", null, null) - .column(5.0, 2.0, 3.0, 1.0) - .decimal64Column(-5, RoundingMode.UNNECESSARY, 0, null, -1.4, 10.123) - .build() - } - - private def buildEmptyTable(): Table = { - withResource(buildTable()) { tbl => - withResource(ColumnVector.fromBooleans(false, false, false, false)) { mask => - tbl.filter(mask) // filter all out - } - } - } - - private val mockTableDataTypes: Array[DataType] = - Array(IntegerType, StringType, DoubleType, DecimalType(10, 5)) - - test("spill updates catalog") { - val bufferId = MockRapidsBufferId(7, canShareDiskPaths = false) - val mockDiskBlockManager = mock[RapidsDiskBlockManager] - when(mockDiskBlockManager.getSerializerManager()) - .thenReturn(new RapidsSerializerManager(new SparkConf())) - val spillPriority = -7 - val hostStoreMaxSize = 1L * 1024 * 1024 - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = spy(new RapidsBufferCatalog(devStore)) - withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { - hostStore => - devStore.setSpillStore(hostStore) - withResource(new RapidsDiskStore(mockDiskBlockManager)) { diskStore => - assertResult(0)(diskStore.currentSize) - hostStore.setSpillStore(diskStore) - val (bufferSize, handle) = - addContiguousTableToCatalog(catalog, bufferId, spillPriority) - val path = handle.id.getDiskPath(null) - assert(!path.exists()) - catalog.synchronousSpill(devStore, 0) - catalog.synchronousSpill(hostStore, 0) - assertResult(0)(hostStore.currentSize) - assertResult(bufferSize)(diskStore.currentSize) - assert(path.exists) - assertResult(bufferSize)(path.length) - verify(catalog, times(3)).registerNewBuffer(ArgumentMatchers.any[RapidsBuffer]) - verify(catalog).removeBufferTier( - ArgumentMatchers.eq(handle.id), ArgumentMatchers.eq(StorageTier.DEVICE)) - withResource(catalog.acquireBuffer(handle)) { buffer => - assertResult(StorageTier.DISK)(buffer.storageTier) - assertResult(bufferSize)(buffer.memoryUsedBytes) - assertResult(handle.id)(buffer.id) - assertResult(spillPriority)(buffer.getSpillPriority) - } - } - } - } - } - - test("Get columnar batch") { - val bufferId = MockRapidsBufferId(1, canShareDiskPaths = false) - val bufferPath = bufferId.getDiskPath(null) - val mockDiskBlockManager = mock[RapidsDiskBlockManager] - when(mockDiskBlockManager.getSerializerManager()) - .thenReturn(new RapidsSerializerManager(new SparkConf())) - assert(!bufferPath.exists) - val sparkTypes = Array[DataType](IntegerType, StringType, DoubleType, - DecimalType(ai.rapids.cudf.DType.DECIMAL64_MAX_PRECISION, 5)) - val spillPriority = -7 - val hostStoreMaxSize = 1L * 1024 * 1024 - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { - hostStore => - devStore.setSpillStore(hostStore) - withResource(new RapidsDiskStore(mockDiskBlockManager)) { - diskStore => - hostStore.setSpillStore(diskStore) - val (_, handle) = addContiguousTableToCatalog(catalog, bufferId, spillPriority) - assert(!handle.id.getDiskPath(null).exists()) - val expectedTable = withResource(catalog.acquireBuffer(handle)) { buffer => - assertResult(StorageTier.DEVICE)(buffer.storageTier) - withResource(buffer.getColumnarBatch(sparkTypes)) { beforeSpill => - withResource(GpuColumnVector.from(beforeSpill)) { table => - table.contiguousSplit()(0) - } - } // closing the batch from the store so that we can spill it - } - withResource(expectedTable) { _ => - withResource( - GpuColumnVector.from(expectedTable.getTable, sparkTypes)) { expectedBatch => - catalog.synchronousSpill(devStore, 0) - catalog.synchronousSpill(hostStore, 0) - withResource(catalog.acquireBuffer(handle)) { buffer => - assertResult(StorageTier.DISK)(buffer.storageTier) - withResource(buffer.getColumnarBatch(sparkTypes)) { actualBatch => - TestUtils.compareBatches(expectedBatch, actualBatch) - } - } - } - } - } - } - } - } - - test("get memory buffer") { - val bufferId = MockRapidsBufferId(1, canShareDiskPaths = false) - val bufferPath = bufferId.getDiskPath(null) - val mockDiskBlockManager = mock[RapidsDiskBlockManager] - when(mockDiskBlockManager.getSerializerManager()) - .thenReturn(new RapidsSerializerManager(new SparkConf())) - assert(!bufferPath.exists) - val spillPriority = -7 - val hostStoreMaxSize = 1L * 1024 * 1024 - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { - hostStore => - devStore.setSpillStore(hostStore) - withResource(new RapidsDiskStore(mockDiskBlockManager)) { diskStore => - hostStore.setSpillStore(diskStore) - val (_, handle) = addContiguousTableToCatalog(catalog, bufferId, spillPriority) - assert(!handle.id.getDiskPath(mockDiskBlockManager).exists()) - val expectedBuffer = withResource(catalog.acquireBuffer(handle)) { buffer => - assertResult(StorageTier.DEVICE)(buffer.storageTier) - withResource(buffer.getMemoryBuffer) { devbuf => - closeOnExcept(HostMemoryBuffer.allocate(devbuf.getLength)) { hostbuf => - hostbuf.copyFromDeviceBuffer(devbuf.asInstanceOf[DeviceMemoryBuffer]) - hostbuf - } - } - } - withResource(expectedBuffer) { expectedBuffer => - catalog.synchronousSpill(devStore, 0) - catalog.synchronousSpill(hostStore, 0) - withResource(catalog.acquireBuffer(handle)) { buffer => - assertResult(StorageTier.DISK)(buffer.storageTier) - withResource(buffer.getMemoryBuffer) { actualBuffer => - assert(actualBuffer.isInstanceOf[HostMemoryBuffer]) - val actualHostBuffer = actualBuffer.asInstanceOf[HostMemoryBuffer] - assertResult(expectedBuffer. - asByteBuffer.limit())(actualHostBuffer.asByteBuffer.limit()) - } - } - } - } - } - } - } - - test("Compression on with or without encryption for spill block using single batch") { - Seq("true", "false").foreach { encryptionEnabled => - val conf = new SparkConf() - conf.set(RapidsConf.TEST_IO_ENCRYPTION.key, encryptionEnabled) - conf.set("spark.io.compression.codec", "zstd") - conf.set("spark.shuffle.spill.compress", "true") - conf.set("spark.shuffle.compress", "true") - readWriteTestWithBatches(conf, TempSpillBufferId.apply()) - } - } - - test("Compression off with or without encryption for spill block using single batch") { - Seq("true", "false").foreach { encryptionEnabled => - val conf = new SparkConf() - conf.set(RapidsConf.TEST_IO_ENCRYPTION.key, encryptionEnabled) - conf.set("spark.shuffle.spill.compress", "false") - conf.set("spark.shuffle.compress", "false") - readWriteTestWithBatches(conf, TempSpillBufferId.apply()) - } - } - - test("Compression on with or without encryption for spill block using multiple batches") { - Seq("true", "false").foreach { encryptionEnabled => - val conf = new SparkConf() - conf.set(RapidsConf.TEST_IO_ENCRYPTION.key, encryptionEnabled) - conf.set("spark.io.compression.codec", "zstd") - conf.set("spark.shuffle.spill.compress", "true") - conf.set("spark.shuffle.compress", "true") - readWriteTestWithBatches(conf, TempSpillBufferId.apply(), TempSpillBufferId.apply()) - } - } - - test("Compression off with or without encryption for spill block using multiple batches") { - Seq("true", "false").foreach { encryptionEnabled => - val conf = new SparkConf() - conf.set(RapidsConf.TEST_IO_ENCRYPTION.key, encryptionEnabled) - conf.set("spark.shuffle.spill.compress", "false") - conf.set("spark.shuffle.compress", "false") - readWriteTestWithBatches(conf, TempSpillBufferId.apply(), TempSpillBufferId.apply()) - } - } - - // ===== Tests for shuffle block ===== - - test("Compression on with or without encryption for shuffle block using single batch") { - Seq("true", "false").foreach { encryptionEnabled => - val conf = new SparkConf() - conf.set(RapidsConf.TEST_IO_ENCRYPTION.key, encryptionEnabled) - conf.set("spark.io.compression.codec", "zstd") - conf.set("spark.shuffle.spill.compress", "true") - conf.set("spark.shuffle.compress", "true") - readWriteTestWithBatches(conf, ShuffleBufferId(ShuffleBlockId(1, 1, 1), 1)) - } - } - - test("Compression off with or without encryption for shuffle block using single batch") { - Seq("true", "false").foreach { encryptionEnabled => - val conf = new SparkConf() - conf.set(RapidsConf.TEST_IO_ENCRYPTION.key, encryptionEnabled) - conf.set("spark.shuffle.spill.compress", "false") - conf.set("spark.shuffle.compress", "false") - readWriteTestWithBatches(conf, ShuffleBufferId(ShuffleBlockId(1, 1, 1), 1)) - } - } - - test("Compression on with or without encryption for shuffle block using multiple batches") { - Seq("true", "false").foreach { encryptionEnabled => - val conf = new SparkConf() - conf.set(RapidsConf.TEST_IO_ENCRYPTION.key, encryptionEnabled) - conf.set("spark.io.compression.codec", "zstd") - conf.set("spark.shuffle.spill.compress", "true") - conf.set("spark.shuffle.compress", "true") - readWriteTestWithBatches(conf, - ShuffleBufferId(ShuffleBlockId(1, 1, 1), 1), ShuffleBufferId(ShuffleBlockId(2, 2, 2), 2)) - } - } - - test("Compression off with or without encryption for shuffle block using multiple batches") { - Seq("true", "false").foreach { encryptionEnabled => - val conf = new SparkConf() - conf.set(RapidsConf.TEST_IO_ENCRYPTION.key, encryptionEnabled) - conf.set("spark.shuffle.spill.compress", "false") - conf.set("spark.shuffle.compress", "false") - readWriteTestWithBatches(conf, - ShuffleBufferId(ShuffleBlockId(1, 1, 1), 1), ShuffleBufferId(ShuffleBlockId(2, 2, 2), 2)) - } - } - - test("No encryption and compression for shuffle block using multiple batches") { - readWriteTestWithBatches(new SparkConf(), - ShuffleBufferId(ShuffleBlockId(1, 1, 1), 1), ShuffleBufferId(ShuffleBlockId(2, 2, 2), 2)) - } - - private def readWriteTestWithBatches(conf: SparkConf, bufferIds: RapidsBufferId*) = { - assert(bufferIds.size != 0) - val mockDiskBlockManager = mock[RapidsDiskBlockManager] - when(mockDiskBlockManager.getSerializerManager()) - .thenReturn(new RapidsSerializerManager(conf)) - - if (bufferIds(0).canShareDiskPaths) { - // Return the same path - val bufferPath = new File(TEST_FILES_ROOT, s"diskbuffer-${bufferIds(0).tableId}") - when(mockDiskBlockManager.getFile(any[BlockId]())).thenReturn(bufferPath) - if (bufferPath.exists) bufferPath.delete() - } else { - when(mockDiskBlockManager.getFile(any[BlockId]())) - .thenAnswer { invocation => - new File(TEST_FILES_ROOT, s"diskbuffer-${invocation.getArgument[BlockId](0).name}") - } - } - - val spillPriority = -7 - val hostStoreMaxSize = 1L * 1024 * 1024 - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { - hostStore => - devStore.setSpillStore(hostStore) - withResource(new RapidsDiskStore(mockDiskBlockManager)) { diskStore => - hostStore.setSpillStore(diskStore) - bufferIds.foreach { bufferId => - val (_, handle) = addContiguousTableToCatalog(catalog, bufferId, spillPriority) - val expectedBuffer = withResource(catalog.acquireBuffer(handle)) { buffer => - assertResult(StorageTier.DEVICE)(buffer.storageTier) - withResource(buffer.getMemoryBuffer) { devbuf => - closeOnExcept(HostMemoryBuffer.allocate(devbuf.getLength)) { hostbuf => - hostbuf.copyFromDeviceBuffer(devbuf.asInstanceOf[DeviceMemoryBuffer]) - hostbuf - } - } - } - withResource(expectedBuffer) { expectedBuffer => - catalog.synchronousSpill(devStore, 0) - catalog.synchronousSpill(hostStore, 0) - withResource(catalog.acquireBuffer(handle)) { buffer => - assertResult(StorageTier.DISK)(buffer.storageTier) - withResource(buffer.getMemoryBuffer) { actualBuffer => - assert(actualBuffer.isInstanceOf[HostMemoryBuffer]) - val actualHostBuffer = actualBuffer.asInstanceOf[HostMemoryBuffer] - assertResult(expectedBuffer.asByteBuffer)(actualHostBuffer.asByteBuffer) - } - } - } - } - } - } - } - } - - test("skip host: spill device memory buffer to disk") { - val bufferId = MockRapidsBufferId(1, canShareDiskPaths = false) - val bufferPath = bufferId.getDiskPath(null) - assert(!bufferPath.exists) - val mockDiskBlockManager = mock[RapidsDiskBlockManager] - when(mockDiskBlockManager.getSerializerManager()) - .thenReturn(new RapidsSerializerManager(new SparkConf())) - val spillPriority = -7 - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = new RapidsBufferCatalog(devStore) - withResource(new AlwaysFailingRapidsHostMemoryStore) { - hostStore => - devStore.setSpillStore(hostStore) - withResource(new RapidsDiskStore(mockDiskBlockManager)) { diskStore => - hostStore.setSpillStore(diskStore) - val (_, handle) = addContiguousTableToCatalog(catalog, bufferId, spillPriority) - assert(!handle.id.getDiskPath(null).exists()) - val expectedBuffer = withResource(catalog.acquireBuffer(handle)) { buffer => - assertResult(StorageTier.DEVICE)(buffer.storageTier) - withResource(buffer.getMemoryBuffer) { devbuf => - closeOnExcept(HostMemoryBuffer.allocate(devbuf.getLength)) { hostbuf => - hostbuf.copyFromDeviceBuffer(devbuf.asInstanceOf[DeviceMemoryBuffer]) - hostbuf - } - } - } - withResource(expectedBuffer) { expectedBuffer => - catalog.synchronousSpill(devStore, 0) - withResource(catalog.acquireBuffer(handle)) { buffer => - assertResult(StorageTier.DISK)(buffer.storageTier) - withResource(buffer.getMemoryBuffer) { actualBuffer => - assert(actualBuffer.isInstanceOf[HostMemoryBuffer]) - val actualHostBuffer = actualBuffer.asInstanceOf[HostMemoryBuffer] - assertResult(expectedBuffer.asByteBuffer)(actualHostBuffer.asByteBuffer) - } - } - } - } - } - } - } - - test("skip host: spill table to disk") { - val bufferId = MockRapidsBufferId(1, canShareDiskPaths = false) - val bufferPath = bufferId.getDiskPath(null) - assert(!bufferPath.exists) - val mockDiskBlockManager = mock[RapidsDiskBlockManager] - when(mockDiskBlockManager.getSerializerManager()) - .thenReturn(new RapidsSerializerManager(new SparkConf())) - val spillPriority = -7 - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = new RapidsBufferCatalog(devStore) - withResource(new AlwaysFailingRapidsHostMemoryStore) { - hostStore => - devStore.setSpillStore(hostStore) - withResource(new RapidsDiskStore(mockDiskBlockManager)) { diskStore => - hostStore.setSpillStore(diskStore) - val handle = addTableToCatalog(catalog, bufferId, spillPriority) - withResource(buildTable()) { expectedTable => - withResource( - GpuColumnVector.from(expectedTable, mockTableDataTypes)) { expectedBatch => - catalog.synchronousSpill(devStore, 0) - withResource(catalog.acquireBuffer(handle)) { buffer => - assert(handle.id.getDiskPath(null).exists()) - assertResult(StorageTier.DISK)(buffer.storageTier) - withResource(buffer.getColumnarBatch(mockTableDataTypes)) { fromDiskBatch => - TestUtils.compareBatches(expectedBatch, fromDiskBatch) - } - } - } - } - } - } - } - } - - test("skip host: spill table to disk with small host bounce buffer") { - val bufferId = MockRapidsBufferId(1, canShareDiskPaths = false) - val bufferPath = bufferId.getDiskPath(null) - assert(!bufferPath.exists) - val mockDiskBlockManager = mock[RapidsDiskBlockManager] - when(mockDiskBlockManager.getSerializerManager()) - .thenReturn(new RapidsSerializerManager(new SparkConf())) - val spillPriority = -7 - withResource(new RapidsDeviceMemoryStore(1L*1024*1024, 10)) { devStore => - val catalog = new RapidsBufferCatalog(devStore) - withResource(new AlwaysFailingRapidsHostMemoryStore) { - hostStore => - devStore.setSpillStore(hostStore) - withResource(new RapidsDiskStore(mockDiskBlockManager)) { diskStore => - hostStore.setSpillStore(diskStore) - val handle = addTableToCatalog(catalog, bufferId, spillPriority) - withResource(buildTable()) { expectedTable => - withResource( - GpuColumnVector.from(expectedTable, mockTableDataTypes)) { expectedBatch => - catalog.synchronousSpill(devStore, 0) - withResource(catalog.acquireBuffer(handle)) { buffer => - assert(handle.id.getDiskPath(null).exists()) - assertResult(StorageTier.DISK)(buffer.storageTier) - withResource(buffer.getColumnarBatch(mockTableDataTypes)) { fromDiskBatch => - TestUtils.compareBatches(expectedBatch, fromDiskBatch) - } - } - } - } - } - } - } - } - - - test("0-byte table is never spillable as we would fail to mmap") { - val bufferId = MockRapidsBufferId(1, canShareDiskPaths = false) - val bufferPath = bufferId.getDiskPath(null) - val bufferId2 = MockRapidsBufferId(2, canShareDiskPaths = false) - assert(!bufferPath.exists) - val mockDiskBlockManager = mock[RapidsDiskBlockManager] - when(mockDiskBlockManager.getSerializerManager()) - .thenReturn(new RapidsSerializerManager(new SparkConf())) - val spillPriority = -7 - val hostStoreMaxSize = 1L * 1024 * 1024 - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => - devStore.setSpillStore(hostStore) - withResource(new RapidsDiskStore(mockDiskBlockManager)) { diskStore => - hostStore.setSpillStore(diskStore) - val handle = addZeroRowsTableToCatalog(catalog, bufferId, spillPriority - 1) - val handle2 = addTableToCatalog(catalog, bufferId2, spillPriority) - withResource(handle2) { _ => - assert(!handle.id.getDiskPath(null).exists()) - withResource(buildTable()) { expectedTable => - withResource(buildEmptyTable()) { expectedEmptyTable => - withResource( - GpuColumnVector.from( - expectedTable, mockTableDataTypes)) { expectedCb => - withResource( - GpuColumnVector.from( - expectedEmptyTable, mockTableDataTypes)) { expectedEmptyCb => - catalog.synchronousSpill(devStore, 0) - catalog.synchronousSpill(hostStore, 0) - withResource(catalog.acquireBuffer(handle2)) { buffer => - withResource(catalog.acquireBuffer(handle)) { emptyBuffer => - // the 0-byte table never moved from device. It is not spillable - assertResult(StorageTier.DEVICE)(emptyBuffer.storageTier) - withResource(emptyBuffer.getColumnarBatch(mockTableDataTypes)) { cb => - TestUtils.compareBatches(expectedEmptyCb, cb) - } - // the second table (with rows) did spill - assertResult(StorageTier.DISK)(buffer.storageTier) - withResource(buffer.getColumnarBatch(mockTableDataTypes)) { cb => - TestUtils.compareBatches(expectedCb, cb) - } - } - } - assertResult(0)(devStore.currentSize) - assertResult(0)(hostStore.currentSize) - } - } - } - } - } - } - } - } - } - - test("exclusive spill files are deleted when buffer deleted") { - testBufferFileDeletion(canShareDiskPaths = false) - } - - test("shared spill files are not deleted when a buffer is deleted") { - testBufferFileDeletion(canShareDiskPaths = true) - } - - class AlwaysFailingRapidsHostMemoryStore extends RapidsHostMemoryStore(Some(0L)){ - override def createBuffer( - other: RapidsBuffer, - catalog: RapidsBufferCatalog, - stream: Cuda.Stream): Option[RapidsBufferBase] = { - None - } - } - - private def testBufferFileDeletion(canShareDiskPaths: Boolean): Unit = { - val bufferId = MockRapidsBufferId(1, canShareDiskPaths) - val bufferPath = bufferId.getDiskPath(null) - assert(!bufferPath.exists) - val mockDiskBlockManager = mock[RapidsDiskBlockManager] - when(mockDiskBlockManager.getSerializerManager()) - .thenReturn(new RapidsSerializerManager(new SparkConf())) - val spillPriority = -7 - val hostStoreMaxSize = 1L * 1024 * 1024 - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { - hostStore => - devStore.setSpillStore(hostStore) - withResource(new RapidsDiskStore(mockDiskBlockManager)) { diskStore => - hostStore.setSpillStore(diskStore) - val (_, handle) = addContiguousTableToCatalog(catalog, bufferId, spillPriority) - val bufferPath = handle.id.getDiskPath(null) - assert(!bufferPath.exists()) - catalog.synchronousSpill(devStore, 0) - catalog.synchronousSpill(hostStore, 0) - assert(bufferPath.exists) - handle.close() - if (canShareDiskPaths) { - assert(bufferPath.exists()) - } else { - assert(!bufferPath.exists) - } - } - } - } - } - - private def addContiguousTableToCatalog( - catalog: RapidsBufferCatalog, - bufferId: RapidsBufferId, - spillPriority: Long): (Long, RapidsBufferHandle) = { - withResource(buildContiguousTable()) { ct => - val bufferSize = ct.getBuffer.getLength - // store takes ownership of the table - val handle = catalog.addContiguousTable( - bufferId, - ct, - spillPriority, - false) - (bufferSize, handle) - } - } - - private def addTableToCatalog( - catalog: RapidsBufferCatalog, - bufferId: RapidsBufferId, - spillPriority: Long): RapidsBufferHandle = { - // store takes ownership of the table - catalog.addTable( - bufferId, - buildTable(), - spillPriority, - false) - } - - private def addZeroRowsTableToCatalog( - catalog: RapidsBufferCatalog, - bufferId: RapidsBufferId, - spillPriority: Long): RapidsBufferHandle = { - val table = buildEmptyTable() - // store takes ownership of the table - catalog.addTable( - bufferId, - table, - spillPriority, - false) - } - - case class MockRapidsBufferId( - tableId: Int, - override val canShareDiskPaths: Boolean) extends RapidsBufferId { - override def getDiskPath(diskBlockManager: RapidsDiskBlockManager): File = - new File(TEST_FILES_ROOT, s"diskbuffer-$tableId") - } -} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsHostMemoryStoreSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsHostMemoryStoreSuite.scala deleted file mode 100644 index 1ffad031451..00000000000 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsHostMemoryStoreSuite.scala +++ /dev/null @@ -1,614 +0,0 @@ -/* - * 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. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids - -import java.io.File -import java.math.RoundingMode - -import ai.rapids.cudf.{ContiguousTable, Cuda, HostColumnVector, HostMemoryBuffer, Table} -import com.nvidia.spark.rapids.Arm._ -import org.mockito.{ArgumentCaptor, ArgumentMatchers} -import org.mockito.Mockito.{spy, times, verify} -import org.scalatest.funsuite.AnyFunSuite -import org.scalatestplus.mockito.MockitoSugar - -import org.apache.spark.SparkConf -import org.apache.spark.sql.rapids.RapidsDiskBlockManager -import org.apache.spark.sql.types.{DataType, DecimalType, DoubleType, IntegerType, LongType, StringType} -import org.apache.spark.sql.vectorized.ColumnarBatch - -class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { - private def buildContiguousTable(): ContiguousTable = { - withResource(new Table.TestBuilder() - .column(5, null.asInstanceOf[java.lang.Integer], 3, 1) - .column("five", "two", null, null) - .column(5.0, 2.0, 3.0, 1.0) - .decimal64Column(-5, RoundingMode.UNNECESSARY, 0, null, -1.4, 10.123) - .build()) { table => - table.contiguousSplit()(0) - } - } - - private def buildContiguousTable(numRows: Int): ContiguousTable = { - val vals = (0 until numRows).map(_.toLong) - withResource(HostColumnVector.fromLongs(vals: _*)) { hcv => - withResource(hcv.copyToDevice()) { cv => - withResource(new Table(cv)) { table => - table.contiguousSplit()(0) - } - } - } - } - - private def buildHostBatch(): ColumnarBatch = { - val sparkTypes = Array[DataType](IntegerType, StringType, DoubleType, - DecimalType(ai.rapids.cudf.DType.DECIMAL64_MAX_PRECISION, 5)) - val hostCols = withResource(buildContiguousTable()) { ct => - withResource(ct.getTable) { tbl => - (0 until tbl.getNumberOfColumns) - .map(c => tbl.getColumn(c).copyToHost()) - } - }.toArray - new ColumnarBatch( - hostCols.zip(sparkTypes).map { case (hostCol, dataType) => - new RapidsHostColumnVector(dataType, hostCol) - }, hostCols.head.getRowCount.toInt) - } - - private def buildHostBatchWithDuplicate(): ColumnarBatch = { - val sparkTypes = Array[DataType](IntegerType, StringType, DoubleType, - DecimalType(ai.rapids.cudf.DType.DECIMAL64_MAX_PRECISION, 5)) - val hostCols = withResource(buildContiguousTable()) { ct => - withResource(ct.getTable) { tbl => - (0 until tbl.getNumberOfColumns) - .map(c => tbl.getColumn(c).copyToHost()) - } - }.toArray - hostCols.foreach(_.incRefCount()) - new ColumnarBatch( - (hostCols ++ hostCols).zip(sparkTypes ++ sparkTypes).map { case (hostCol, dataType) => - new RapidsHostColumnVector(dataType, hostCol) - }, hostCols.head.getRowCount.toInt) - } - - test("spill updates catalog") { - val spillPriority = -7 - val hostStoreMaxSize = 1L * 1024 * 1024 - val mockStore = mock[RapidsHostMemoryStore] - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = spy(new RapidsBufferCatalog(devStore)) - withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { - hostStore => - assertResult(0)(hostStore.currentSize) - assertResult(hostStoreMaxSize)(hostStore.numBytesFree.get) - devStore.setSpillStore(hostStore) - hostStore.setSpillStore(mockStore) - - val (bufferSize, handle) = withResource(buildContiguousTable()) { ct => - val len = ct.getBuffer.getLength - // store takes ownership of the table - val handle = catalog.addContiguousTable( - ct, - spillPriority) - (len, handle) - } - - catalog.synchronousSpill(devStore, 0) - assertResult(bufferSize)(hostStore.currentSize) - assertResult(hostStoreMaxSize - bufferSize)(hostStore.numBytesFree.get) - verify(catalog, times(2)).registerNewBuffer(ArgumentMatchers.any[RapidsBuffer]) - verify(catalog).removeBufferTier( - ArgumentMatchers.eq(handle.id), ArgumentMatchers.eq(StorageTier.DEVICE)) - withResource(catalog.acquireBuffer(handle)) { buffer => - assertResult(StorageTier.HOST)(buffer.storageTier) - assertResult(bufferSize)(buffer.memoryUsedBytes) - assertResult(handle.id)(buffer.id) - assertResult(spillPriority)(buffer.getSpillPriority) - } - } - } - } - - test("get columnar batch") { - val spillPriority = -10 - val hostStoreMaxSize = 1L * 1024 * 1024 - val mockStore = mock[RapidsHostMemoryStore] - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { - hostStore => - devStore.setSpillStore(hostStore) - hostStore.setSpillStore(mockStore) - var expectedBuffer: HostMemoryBuffer = null - val handle = withResource(buildContiguousTable()) { ct => - expectedBuffer = HostMemoryBuffer.allocate(ct.getBuffer.getLength) - expectedBuffer.copyFromDeviceBuffer(ct.getBuffer) - catalog.addContiguousTable( - ct, - spillPriority) - } - withResource(expectedBuffer) { _ => - catalog.synchronousSpill(devStore, 0) - withResource(catalog.acquireBuffer(handle)) { buffer => - withResource(buffer.getMemoryBuffer) { actualBuffer => - assert(actualBuffer.isInstanceOf[HostMemoryBuffer]) - assertResult(expectedBuffer.asByteBuffer) { - actualBuffer.asInstanceOf[HostMemoryBuffer].asByteBuffer - } - } - } - } - } - } - } - - test("get memory buffer") { - val sparkTypes = Array[DataType](IntegerType, StringType, DoubleType, - DecimalType(ai.rapids.cudf.DType.DECIMAL64_MAX_PRECISION, 5)) - val spillPriority = -10 - val hostStoreMaxSize = 1L * 1024 * 1024 - val mockStore = mock[RapidsHostMemoryStore] - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { - hostStore => - devStore.setSpillStore(hostStore) - hostStore.setSpillStore(mockStore) - var expectedBatch: ColumnarBatch = null - val handle = withResource(buildContiguousTable()) { ct => - // make a copy of the table so we can compare it later to the - // one reconstituted after the spill - withResource(ct.getTable.contiguousSplit()) { copied => - expectedBatch = GpuColumnVector.from(copied(0).getTable, sparkTypes) - } - catalog.addContiguousTable( - ct, - spillPriority) - } - withResource(expectedBatch) { _ => - catalog.synchronousSpill(devStore, 0) - withResource(catalog.acquireBuffer(handle)) { buffer => - assertResult(StorageTier.HOST)(buffer.storageTier) - withResource(buffer.getColumnarBatch(sparkTypes)) { actualBatch => - TestUtils.compareBatches(expectedBatch, actualBatch) - } - } - } - } - } - } - - test("get memory buffer after host spill") { - RapidsBufferCatalog.close() - val sparkTypes = Array[DataType](IntegerType, StringType, DoubleType, - DecimalType(ai.rapids.cudf.DType.DECIMAL64_MAX_PRECISION, 5)) - val spillPriority = -10 - val hostStoreMaxSize = 1L * 1024 * 1024 - try { - val bm = new RapidsDiskBlockManager(new SparkConf()) - val (catalog, devStore, hostStore, diskStore) = - closeOnExcept(new RapidsDiskStore(bm)) { diskStore => - closeOnExcept(new RapidsDeviceMemoryStore()) { devStore => - closeOnExcept(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => - devStore.setSpillStore(hostStore) - hostStore.setSpillStore(diskStore) - val catalog = closeOnExcept( - new RapidsBufferCatalog(devStore, hostStore)) { catalog => catalog } - (catalog, devStore, hostStore, diskStore) - } - } - } - - RapidsBufferCatalog.setDeviceStorage(devStore) - RapidsBufferCatalog.setHostStorage(hostStore) - RapidsBufferCatalog.setDiskStorage(diskStore) - RapidsBufferCatalog.setCatalog(catalog) - - var expectedBatch: ColumnarBatch = null - val handle = withResource(buildContiguousTable()) { ct => - // make a copy of the table so we can compare it later to the - // one reconstituted after the spill - withResource(ct.getTable.contiguousSplit()) { copied => - expectedBatch = GpuColumnVector.from(copied(0).getTable, sparkTypes) - } - RapidsBufferCatalog.addContiguousTable( - ct, - spillPriority) - } - withResource(expectedBatch) { _ => - val spilledToHost = - RapidsBufferCatalog.synchronousSpill( - RapidsBufferCatalog.getDeviceStorage, 0) - assert(spilledToHost.isDefined && spilledToHost.get > 0) - - val spilledToDisk = - RapidsBufferCatalog.synchronousSpill( - RapidsBufferCatalog.getHostStorage, 0) - assert(spilledToDisk.isDefined && spilledToDisk.get > 0) - - withResource(RapidsBufferCatalog.acquireBuffer(handle)) { buffer => - assertResult(StorageTier.DISK)(buffer.storageTier) - withResource(buffer.getColumnarBatch(sparkTypes)) { actualBatch => - TestUtils.compareBatches(expectedBatch, actualBatch) - } - } - } - } finally { - RapidsBufferCatalog.close() - } - } - - test("host buffer originated: get host memory buffer") { - val spillPriority = -10 - val hostStoreMaxSize = 1L * 1024 * 1024 - val mockStore = mock[RapidsDiskStore] - withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = new RapidsBufferCatalog(devStore, hostStore) - devStore.setSpillStore(hostStore) - hostStore.setSpillStore(mockStore) - val hmb = HostMemoryBuffer.allocate(1L * 1024) - val spillableBuffer = - SpillableHostBuffer(hmb, hmb.getLength, spillPriority, catalog) - withResource(spillableBuffer) { _ => - // the refcount of 1 is the store - assertResult(1)(hmb.getRefCount) - withResource(spillableBuffer.getHostBuffer()) { memoryBuffer => - assertResult(hmb)(memoryBuffer) - assertResult(2)(memoryBuffer.getRefCount) - } - } - assertResult(0)(hmb.getRefCount) - } - } - } - - test("host buffer originated: get host memory buffer after spill") { - RapidsBufferCatalog.close() - val spillPriority = -10 - val hostStoreMaxSize = 1L * 1024 * 1024 - try { - val bm = new RapidsDiskBlockManager(new SparkConf()) - val (catalog, devStore, hostStore, diskStore) = - closeOnExcept(new RapidsDiskStore(bm)) { diskStore => - closeOnExcept(new RapidsDeviceMemoryStore()) { devStore => - closeOnExcept(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => - devStore.setSpillStore(hostStore) - hostStore.setSpillStore(diskStore) - val catalog = closeOnExcept( - new RapidsBufferCatalog(devStore, hostStore)) { catalog => catalog } - (catalog, devStore, hostStore, diskStore) - } - } - } - - RapidsBufferCatalog.setDeviceStorage(devStore) - RapidsBufferCatalog.setHostStorage(hostStore) - RapidsBufferCatalog.setDiskStorage(diskStore) - RapidsBufferCatalog.setCatalog(catalog) - - val hmb = HostMemoryBuffer.allocate(1L * 1024) - val spillableBuffer = SpillableHostBuffer( - hmb, - hmb.getLength, - spillPriority) - assertResult(1)(hmb.getRefCount) - // we spill it - RapidsBufferCatalog.synchronousSpill(RapidsBufferCatalog.getHostStorage, 0) - withResource(spillableBuffer) { _ => - // the refcount of the original buffer is 0 because it spilled - assertResult(0)(hmb.getRefCount) - withResource(spillableBuffer.getHostBuffer()) { memoryBuffer => - assertResult(memoryBuffer.getLength)(hmb.getLength) - } - } - } finally { - RapidsBufferCatalog.close() - } - } - - test("host buffer originated: get host memory buffer OOM when unable to spill") { - val spillPriority = -10 - val hostStoreMaxSize = 1L * 1024 * 1024 - val bm = new RapidsDiskBlockManager(new SparkConf()) - withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = new RapidsBufferCatalog(devStore, hostStore) - devStore.setSpillStore(hostStore) - hostStore.setSpillStore(diskStore) - val hmb = HostMemoryBuffer.allocate(1L * 1024) - val spillableBuffer = SpillableHostBuffer( - hmb, - hmb.getLength, - spillPriority, - catalog) - // spillable is 1K - assertResult(hmb.getLength)(hostStore.currentSpillableSize) - withResource(spillableBuffer.getHostBuffer()) { memoryBuffer => - // 0 because we have a reference to the memoryBuffer - assertResult(0)(hostStore.currentSpillableSize) - val spilled = catalog.synchronousSpill(hostStore, 0) - assertResult(0)(spilled.get) - } - assertResult(hmb.getLength)(hostStore.currentSpillableSize) - val spilled = catalog.synchronousSpill(hostStore, 0) - assertResult(1L * 1024)(spilled.get) - spillableBuffer.close() - } - } - } - } - - test("host batch originated: get host memory batch") { - val spillPriority = -10 - val hostStoreMaxSize = 1L * 1024 * 1024 - val bm = new RapidsDiskBlockManager(new SparkConf()) - withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = new RapidsBufferCatalog(devStore, hostStore) - devStore.setSpillStore(hostStore) - hostStore.setSpillStore(diskStore) - - val hostCb = buildHostBatch() - - val sizeOnHost = RapidsHostColumnVector.getTotalHostMemoryUsed(hostCb) - - withResource( - SpillableHostColumnarBatch(hostCb, spillPriority, catalog)) { spillableBuffer => - assertResult(sizeOnHost)(hostStore.currentSpillableSize) - - withResource(spillableBuffer.getColumnarBatch()) { hostCb => - // 0 because we have a reference to the memoryBuffer - assertResult(0)(hostStore.currentSpillableSize) - val spilled = catalog.synchronousSpill(hostStore, 0) - assertResult(0)(spilled.get) - } - - assertResult(sizeOnHost)(hostStore.currentSpillableSize) - val spilled = catalog.synchronousSpill(hostStore, 0) - assertResult(sizeOnHost)(spilled.get) - - val sizeOnDisk = diskStore.currentSpillableSize - - // reconstitute batch from disk - withResource(spillableBuffer.getColumnarBatch()) { hostCbFromDisk => - // disk has a different size, so this spillable batch has a different sizeInBytes - // right now, because this is the serialized represenation size - assertResult(sizeOnDisk)(spillableBuffer.sizeInBytes) - // lets recreate our original batch and compare to make sure contents match - withResource(buildHostBatch()) { expectedHostCb => - TestUtils.compareBatches(expectedHostCb, hostCbFromDisk) - } - } - } - } - } - } - } - - test("a host batch is not spillable when we leak it") { - val spillPriority = -10 - val hostStoreMaxSize = 1L * 1024 * 1024 - val bm = new RapidsDiskBlockManager(new SparkConf()) - withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = new RapidsBufferCatalog(devStore, hostStore) - devStore.setSpillStore(hostStore) - hostStore.setSpillStore(diskStore) - - val hostCb = buildHostBatch() - - val sizeOnHost = RapidsHostColumnVector.getTotalHostMemoryUsed(hostCb) - - val leakedBatch = withResource( - SpillableHostColumnarBatch(hostCb, spillPriority, catalog)) { spillableBuffer => - assertResult(sizeOnHost)(hostStore.currentSpillableSize) - - val leakedBatch = spillableBuffer.getColumnarBatch() - // 0 because we have a reference to the host batch - assertResult(0)(hostStore.currentSpillableSize) - val spilled = catalog.synchronousSpill(hostStore, 0) - assertResult(0)(spilled.get) - leakedBatch - } - - withResource(leakedBatch) { _ => - // 0 because we have leaked that the host batch - assertResult(0)(hostStore.currentSize) - assertResult(0)(hostStore.currentSpillableSize) - val spilled = catalog.synchronousSpill(hostStore, 0) - assertResult(0)(spilled.get) - } - // after closing we still have 0 bytes in the store or available to spill - assertResult(0)(hostStore.currentSize) - assertResult(0)(hostStore.currentSpillableSize) - } - } - } - } - - test("a host batch is not spillable when columns are incRefCounted") { - val spillPriority = -10 - val hostStoreMaxSize = 1L * 1024 * 1024 - val bm = new RapidsDiskBlockManager(new SparkConf()) - withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = new RapidsBufferCatalog(devStore, hostStore) - devStore.setSpillStore(hostStore) - hostStore.setSpillStore(diskStore) - - val hostCb = buildHostBatch() - - val sizeOnHost = RapidsHostColumnVector.getTotalHostMemoryUsed(hostCb) - - withResource( - SpillableHostColumnarBatch(hostCb, spillPriority, catalog)) { spillableBuffer => - assertResult(sizeOnHost)(hostStore.currentSpillableSize) - - val leakedFirstColumn = withResource(spillableBuffer.getColumnarBatch()) { hostCb => - // 0 because we have a reference to the host batch - assertResult(0)(hostStore.currentSpillableSize) - val spilled = catalog.synchronousSpill(hostStore, 0) - assertResult(0)(spilled.get) - // leak it by increasing the ref count of the underlying cuDF column - RapidsHostColumnVector.extractBases(hostCb).head.incRefCount() - } - withResource(leakedFirstColumn) { _ => - // 0 because we have a reference to the first column - assertResult(0)(hostStore.currentSpillableSize) - val spilled = catalog.synchronousSpill(hostStore, 0) - assertResult(0)(spilled.get) - } - // batch is now spillable because we close our reference to the column - assertResult(sizeOnHost)(hostStore.currentSpillableSize) - val spilled = catalog.synchronousSpill(hostStore, 0) - assertResult(sizeOnHost)(spilled.get) - } - } - } - } - } - - test("an aliased host batch is not spillable (until closing the original) ") { - val spillPriority = -10 - val hostStoreMaxSize = 1L * 1024 * 1024 - val bm = new RapidsDiskBlockManager(new SparkConf()) - withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = new RapidsBufferCatalog(devStore, hostStore) - val hostBatch = buildHostBatch() - val sizeOnHost = RapidsHostColumnVector.getTotalHostMemoryUsed(hostBatch) - val handle = withResource(hostBatch) { _ => - catalog.addBatch(hostBatch, spillPriority) - } - withResource(handle) { _ => - val types: Array[DataType] = - Seq(IntegerType, StringType, DoubleType, DecimalType(10, 5)).toArray - assertResult(sizeOnHost)(hostStore.currentSize) - assertResult(sizeOnHost)(hostStore.currentSpillableSize) - withResource(catalog.acquireBuffer(handle)) { rapidsBuffer => - // extract the batch from the table we added, and add it back as a batch - withResource(rapidsBuffer.getHostColumnarBatch(types)) { batch => - catalog.addBatch(batch, spillPriority) - } - } // we now have two copies in the store - assertResult(sizeOnHost * 2)(hostStore.currentSize) - assertResult(0)(hostStore.currentSpillableSize) - } // remove the original - assertResult(sizeOnHost)(hostStore.currentSize) - assertResult(sizeOnHost)(hostStore.currentSpillableSize) - } - } - } - } - - test("an aliased host batch supports duplicated columns") { - val spillPriority = -10 - val hostStoreMaxSize = 1L * 1024 * 1024 - val bm = new RapidsDiskBlockManager(new SparkConf()) - withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = new RapidsBufferCatalog(devStore, hostStore) - val hostBatch = buildHostBatchWithDuplicate() - val sizeOnHost = RapidsHostColumnVector.getTotalHostMemoryUsed(hostBatch) - val handle = withResource(hostBatch) { _ => - catalog.addBatch(hostBatch, spillPriority) - } - withResource(handle) { _ => - val types: Array[DataType] = - Seq(IntegerType, StringType, DoubleType, DecimalType(10, 5)).toArray - assertResult(sizeOnHost)(hostStore.currentSize) - assertResult(sizeOnHost)(hostStore.currentSpillableSize) - withResource(catalog.acquireBuffer(handle)) { rapidsBuffer => - // extract the batch from the table we added, and add it back as a batch - withResource(rapidsBuffer.getHostColumnarBatch(types)) { batch => - catalog.addBatch(batch, spillPriority) - } - } // we now have two copies in the store - assertResult(sizeOnHost * 2)(hostStore.currentSize) - assertResult(0)(hostStore.currentSpillableSize) - } // remove the original - assertResult(sizeOnHost)(hostStore.currentSize) - assertResult(sizeOnHost)(hostStore.currentSpillableSize) - } - } - } - } - - test("buffer exceeds maximum size") { - val sparkTypes = Array[DataType](LongType) - val spillPriority = -10 - val hostStoreMaxSize = 256 - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = new RapidsBufferCatalog(devStore) - val spyStore = spy(new RapidsDiskStore(new RapidsDiskBlockManager(new SparkConf()))) - withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => - devStore.setSpillStore(hostStore) - hostStore.setSpillStore(spyStore) - var bigHandle: RapidsBufferHandle = null - var bigTable = buildContiguousTable(1024 * 1024) - closeOnExcept(bigTable) { _ => - // make a copy of the table so we can compare it later to the - // one reconstituted after the spill - val expectedBatch = - withResource(bigTable.getTable.contiguousSplit()) { expectedTable => - GpuColumnVector.from(expectedTable(0).getTable, sparkTypes) - } - withResource(expectedBatch) { _ => - bigHandle = withResource(bigTable) { _ => - catalog.addContiguousTable( - bigTable, - spillPriority) - } // close the bigTable so it can be spilled - bigTable = null - withResource(catalog.acquireBuffer(bigHandle)) { buffer => - assertResult(StorageTier.DEVICE)(buffer.storageTier) - withResource(buffer.getColumnarBatch(sparkTypes)) { actualBatch => - TestUtils.compareBatches(expectedBatch, actualBatch) - } - } - catalog.synchronousSpill(devStore, 0) - val rapidsBufferCaptor: ArgumentCaptor[RapidsBuffer] = - ArgumentCaptor.forClass(classOf[RapidsBuffer]) - verify(spyStore).copyBuffer( - rapidsBufferCaptor.capture(), - ArgumentMatchers.any[RapidsBufferCatalog], - ArgumentMatchers.any[Cuda.Stream]) - assertResult(bigHandle.id)(rapidsBufferCaptor.getValue.id) - withResource(catalog.acquireBuffer(bigHandle)) { buffer => - assertResult(StorageTier.DISK)(buffer.storageTier) - withResource(buffer.getColumnarBatch(sparkTypes)) { actualBatch => - TestUtils.compareBatches(expectedBatch, actualBatch) - } - } - } - } - } - } - } - - case class MockRapidsBufferId(tableId: Int) extends RapidsBufferId { - override def getDiskPath(diskBlockManager: RapidsDiskBlockManager): File = - throw new UnsupportedOperationException - } -} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RmmSparkRetrySuiteBase.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RmmSparkRetrySuiteBase.scala index 5776b2f99a8..24d5984f749 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RmmSparkRetrySuiteBase.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RmmSparkRetrySuiteBase.scala @@ -18,16 +18,15 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.{Rmm, RmmAllocationMode, RmmEventHandler} import com.nvidia.spark.rapids.jni.RmmSpark -import org.mockito.Mockito.spy +import com.nvidia.spark.rapids.spill.SpillFramework import org.scalatest.BeforeAndAfterEach import org.scalatest.funsuite.AnyFunSuite +import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession trait RmmSparkRetrySuiteBase extends AnyFunSuite with BeforeAndAfterEach { private var rmmWasInitialized = false - protected var deviceStorage: RapidsDeviceMemoryStore = _ - override def beforeEach(): Unit = { super.beforeEach() SparkSession.getActiveSession.foreach(_.stop()) @@ -37,14 +36,14 @@ trait RmmSparkRetrySuiteBase extends AnyFunSuite with BeforeAndAfterEach { rmmWasInitialized = true Rmm.initialize(RmmAllocationMode.CUDA_DEFAULT, null, 512 * 1024 * 1024) } - deviceStorage = spy(new RapidsDeviceMemoryStore()) - val hostStore = new RapidsHostMemoryStore(Some(1L * 1024 * 1024)) - deviceStorage.setSpillStore(hostStore) - val catalog = new RapidsBufferCatalog(deviceStorage, hostStore) - // set these against the singleton so we close them later - RapidsBufferCatalog.setDeviceStorage(deviceStorage) - RapidsBufferCatalog.setHostStorage(hostStore) - RapidsBufferCatalog.setCatalog(catalog) + val sc = new SparkConf + sc.set(RapidsConf.HOST_SPILL_STORAGE_SIZE.key, "1MB") + val conf = new RapidsConf(sc) + SpillFramework.shutdown() + SpillFramework.initialize(conf) + + RmmSpark.clearEventHandler() + val mockEventHandler = new BaseRmmEventHandler() RmmSpark.setEventHandler(mockEventHandler) RmmSpark.currentThreadIsDedicatedToTask(1) @@ -53,11 +52,11 @@ trait RmmSparkRetrySuiteBase extends AnyFunSuite with BeforeAndAfterEach { override def afterEach(): Unit = { super.afterEach() + SpillFramework.shutdown() SparkSession.getActiveSession.foreach(_.stop()) SparkSession.clearActiveSession() RmmSpark.removeAllCurrentThreadAssociation() RmmSpark.clearEventHandler() - RapidsBufferCatalog.close() GpuSemaphore.shutdown() if (rmmWasInitialized) { Rmm.shutdown() diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/SerializationSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/SerializationSuite.scala index 56ecb1a8c57..c6c251aeabb 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/SerializationSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/SerializationSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2023, NVIDIA CORPORATION. + * Copyright (c) 2021-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,13 +18,15 @@ package com.nvidia.spark.rapids import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream} -import ai.rapids.cudf.Table +import ai.rapids.cudf.{Rmm, RmmAllocationMode, Table} import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.RapidsPluginImplicits._ +import com.nvidia.spark.rapids.spill.SpillFramework import org.apache.commons.lang3.SerializationUtils import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite +import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExecBase, SerializeBatchDeserializeHostBuffer, SerializeConcatHostBuffersDeserializeBatch} import org.apache.spark.sql.types.{DoubleType, FloatType, IntegerType, StringType} @@ -33,12 +35,17 @@ import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} class SerializationSuite extends AnyFunSuite with BeforeAndAfterAll { + override def beforeAll(): Unit = { - RapidsBufferCatalog.setDeviceStorage(new RapidsDeviceMemoryStore()) + super.beforeAll() + Rmm.initialize(RmmAllocationMode.CUDA_DEFAULT, null, 512 * 1024 * 1024) + SpillFramework.initialize(new RapidsConf(new SparkConf)) } override def afterAll(): Unit = { - RapidsBufferCatalog.close() + super.afterAll() + SpillFramework.shutdown() + Rmm.shutdown() } private def buildBatch(): ColumnarBatch = { @@ -170,7 +177,7 @@ class SerializationSuite extends AnyFunSuite withResource(toHostBatch(gpuBatch)) { expectedHostBatch => val broadcast = makeBroadcastBatch(gpuBatch) withBroadcast(broadcast) { _ => - withResource(broadcast.batch.getColumnarBatch()) { materialized => + withResource(broadcast.batch.getColumnarBatch) { materialized => TestUtils.compareBatches(gpuBatch, materialized) } // the host batch here is obtained from the GPU batch since @@ -193,12 +200,12 @@ class SerializationSuite extends AnyFunSuite batches.foreach { gpuExpected => val broadcast = makeBroadcastBatch(gpuExpected) withBroadcast(broadcast) { _ => - withResource(broadcast.batch.getColumnarBatch()) { gpuBatch => + withResource(broadcast.batch.getColumnarBatch) { gpuBatch => TestUtils.compareBatches(gpuExpected, gpuBatch) } // clone via serialization after manifesting the GPU batch withBroadcast(SerializationUtils.clone(broadcast)) { clonedObj => - withResource(clonedObj.batch.getColumnarBatch()) { gpuClonedBatch => + withResource(clonedObj.batch.getColumnarBatch) { gpuClonedBatch => TestUtils.compareBatches(gpuExpected, gpuClonedBatch) } // try to clone it again from the cloned object @@ -214,12 +221,12 @@ class SerializationSuite extends AnyFunSuite batches.foreach { gpuExpected => val broadcast = makeBroadcastBatch(gpuExpected) withBroadcast(broadcast) { _ => - withResource(broadcast.batch.getColumnarBatch()) { gpuBatch => + withResource(broadcast.batch.getColumnarBatch) { gpuBatch => TestUtils.compareBatches(gpuExpected, gpuBatch) } // clone via serialization after manifesting the GPU batch withBroadcast(SerializationUtils.clone(broadcast)) { clonedObj => - withResource(clonedObj.batch.getColumnarBatch()) { gpuClonedBatch => + withResource(clonedObj.batch.getColumnarBatch) { gpuClonedBatch => TestUtils.compareBatches(gpuExpected, gpuClonedBatch) } // try to clone it again from the cloned object @@ -234,12 +241,12 @@ class SerializationSuite extends AnyFunSuite withResource(buildBatch()) { gpuExpected => val broadcast = makeBroadcastBatch(gpuExpected) withBroadcast(broadcast) { _ => - withResource(broadcast.batch.getColumnarBatch()) { gpuBatch => + withResource(broadcast.batch.getColumnarBatch) { gpuBatch => TestUtils.compareBatches(gpuExpected, gpuBatch) } // clone via serialization after manifesting the GPU batch withBroadcast(SerializationUtils.clone(broadcast)) { clonedObj => - withResource(clonedObj.batch.getColumnarBatch()) { gpuClonedBatch => + withResource(clonedObj.batch.getColumnarBatch) { gpuClonedBatch => TestUtils.compareBatches(gpuExpected, gpuClonedBatch) } // try to clone it again from the cloned object @@ -263,7 +270,7 @@ class SerializationSuite extends AnyFunSuite broadcast.doReadObject(inputStream) // use it now - withResource(broadcast.batch.getColumnarBatch()) { gpuBatch => + withResource(broadcast.batch.getColumnarBatch) { gpuBatch => TestUtils.compareBatches(gpuExpected, gpuBatch) } } @@ -275,7 +282,7 @@ class SerializationSuite extends AnyFunSuite val broadcast = makeBroadcastBatch(gpuExpected) withBroadcast(broadcast) { _ => // materialize - withResource(broadcast.batch.getColumnarBatch()) { cb => + withResource(broadcast.batch.getColumnarBatch) { cb => TestUtils.compareBatches(gpuExpected, cb) } @@ -292,7 +299,7 @@ class SerializationSuite extends AnyFunSuite assertResult(before)(broadcast.batch) // it is the same as before // use it now - withResource(broadcast.batch.getColumnarBatch()) { gpuBatch => + withResource(broadcast.batch.getColumnarBatch) { gpuBatch => TestUtils.compareBatches(gpuExpected, gpuBatch) } } @@ -311,7 +318,7 @@ class SerializationSuite extends AnyFunSuite .deserialize[SerializeConcatHostBuffersDeserializeBatch]( inputStream)) { materialized => // this materializes a new batch from what was deserialized - withResource(materialized.batch.getColumnarBatch()) { gpuBatch => + withResource(materialized.batch.getColumnarBatch) { gpuBatch => TestUtils.compareBatches(gpuExpected, gpuBatch) } } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/ShuffleBufferCatalogSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/ShuffleBufferCatalogSuite.scala index 00209461d3c..a7bea7a5b37 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/ShuffleBufferCatalogSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/ShuffleBufferCatalogSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,21 +16,84 @@ package com.nvidia.spark.rapids +import com.nvidia.spark.rapids.Arm.withResource +import com.nvidia.spark.rapids.format.TableMeta +import com.nvidia.spark.rapids.shuffle.RapidsShuffleTestHelper +import com.nvidia.spark.rapids.spill.SpillFramework +import org.scalatest.BeforeAndAfterEach import org.scalatest.funsuite.AnyFunSuite import org.scalatestplus.mockito.MockitoSugar -import org.apache.spark.sql.rapids.RapidsDiskBlockManager +import org.apache.spark.SparkConf +import org.apache.spark.sql.types.{DataType, IntegerType} +import org.apache.spark.storage.ShuffleBlockId -class ShuffleBufferCatalogSuite extends AnyFunSuite with MockitoSugar { - test("registered shuffles should be active") { - val catalog = mock[RapidsBufferCatalog] - val rapidsDiskBlockManager = mock[RapidsDiskBlockManager] - val shuffleCatalog = new ShuffleBufferCatalog(catalog, rapidsDiskBlockManager) +class ShuffleBufferCatalogSuite + extends AnyFunSuite with MockitoSugar with BeforeAndAfterEach { + + override def beforeEach(): Unit = { + super.beforeEach() + SpillFramework.initialize(new RapidsConf(new SparkConf)) + } + override def afterEach(): Unit = { + super.afterEach() + SpillFramework.shutdown() + } + + test("registered shuffles should be active") { + val shuffleCatalog = new ShuffleBufferCatalog() assertResult(false)(shuffleCatalog.hasActiveShuffle(123)) shuffleCatalog.registerShuffle(123) assertResult(true)(shuffleCatalog.hasActiveShuffle(123)) shuffleCatalog.unregisterShuffle(123) assertResult(false)(shuffleCatalog.hasActiveShuffle(123)) } + + test("adding a degenerate batch") { + val shuffleCatalog = new ShuffleBufferCatalog() + val tableMeta = mock[TableMeta] + // need to register the shuffle id first + assertThrows[IllegalStateException] { + shuffleCatalog.addDegenerateRapidsBuffer(ShuffleBlockId(1, 1L, 1), tableMeta) + } + shuffleCatalog.registerShuffle(1) + shuffleCatalog.addDegenerateRapidsBuffer(ShuffleBlockId(1,1L,1), tableMeta) + val storedMetas = shuffleCatalog.blockIdToMetas(ShuffleBlockId(1, 1L, 1)) + assertResult(1)(storedMetas.size) + assertResult(tableMeta)(storedMetas.head) + } + + test("adding a contiguous batch adds it to the spill store") { + val shuffleCatalog = new ShuffleBufferCatalog() + val ct = RapidsShuffleTestHelper.buildContiguousTable(1000) + shuffleCatalog.registerShuffle(1) + assertResult(0)(SpillFramework.stores.deviceStore.numHandles) + shuffleCatalog.addContiguousTable(ShuffleBlockId(1, 1L, 1), ct, -1) + assertResult(1)(SpillFramework.stores.deviceStore.numHandles) + val storedMetas = shuffleCatalog.blockIdToMetas(ShuffleBlockId(1, 1L, 1)) + assertResult(1)(storedMetas.size) + shuffleCatalog.unregisterShuffle(1) + } + + test("get a columnar batch iterator from catalog") { + val shuffleCatalog = new ShuffleBufferCatalog() + shuffleCatalog.registerShuffle(1) + // add metadata only table + val tableMeta = RapidsShuffleTestHelper.mockTableMeta(0) + shuffleCatalog.addDegenerateRapidsBuffer(ShuffleBlockId(1, 1L, 1), tableMeta) + val ct = RapidsShuffleTestHelper.buildContiguousTable(1000) + shuffleCatalog.addContiguousTable(ShuffleBlockId(1, 1L, 1), ct, -1) + val iter = + shuffleCatalog.getColumnarBatchIterator( + ShuffleBlockId(1, 1L, 1), Array[DataType](IntegerType)) + withResource(iter.toArray) { cbs => + assertResult(2)(cbs.length) + assertResult(0)(cbs.head.numRows()) + assertResult(1)(cbs.head.numCols()) + assertResult(1000)(cbs.last.numRows()) + assertResult(1)(cbs.last.numCols()) + shuffleCatalog.unregisterShuffle(1) + } + } } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/WindowRetrySuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/WindowRetrySuite.scala index 0ecc5faf1a3..bf61e495c06 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/WindowRetrySuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/WindowRetrySuite.scala @@ -80,7 +80,7 @@ class WindowRetrySuite assertResult(4)(hostCol.getLong(row)) } } - verify(inputBatch, times(2)).getColumnarBatch() + verify(inputBatch, times(2)).getColumnarBatch verify(inputBatch, times(1)).close() } } @@ -102,7 +102,7 @@ class WindowRetrySuite assertResult(row + 1)(hostCol.getLong(row)) } } - verify(inputBatch, times(2)).getColumnarBatch() + verify(inputBatch, times(2)).getColumnarBatch verify(inputBatch, times(1)).close() } } @@ -126,7 +126,7 @@ class WindowRetrySuite assertResult(4)(hostCol.getLong(row)) } } - verify(inputBatch, times(2)).getColumnarBatch() + verify(inputBatch, times(2)).getColumnarBatch verify(inputBatch, times(1)).close() } } @@ -143,7 +143,7 @@ class WindowRetrySuite assertThrows[GpuSplitAndRetryOOM] { it.next() } - verify(inputBatch, times(1)).getColumnarBatch() + verify(inputBatch, times(1)).getColumnarBatch verify(inputBatch, times(1)).close() } @@ -173,7 +173,7 @@ class WindowRetrySuite } } } - verify(inputBatch, times(2)).getColumnarBatch() + verify(inputBatch, times(2)).getColumnarBatch verify(inputBatch, times(1)).close() } } 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 aa003c454f1..0f37b7566d9 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/WithRetrySuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/WithRetrySuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,11 +21,13 @@ import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitTargetSizeInHalfGpu, withRestoreOnRetry, withRetry, withRetryNoSplit} import com.nvidia.spark.rapids.jni.{GpuRetryOOM, GpuSplitAndRetryOOM, RmmSpark} +import com.nvidia.spark.rapids.spill.SpillFramework import org.mockito.Mockito._ import org.scalatest.BeforeAndAfterEach import org.scalatest.funsuite.AnyFunSuite import org.scalatestplus.mockito.MockitoSugar +import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession import org.apache.spark.sql.types.{DataType, LongType} @@ -52,19 +54,16 @@ class WithRetrySuite rmmWasInitialized = true Rmm.initialize(RmmAllocationMode.CUDA_DEFAULT, null, 512 * 1024 * 1024) } - val deviceStorage = new RapidsDeviceMemoryStore() - val catalog = new RapidsBufferCatalog(deviceStorage) - RapidsBufferCatalog.setDeviceStorage(deviceStorage) - RapidsBufferCatalog.setCatalog(catalog) + SpillFramework.initialize(new RapidsConf(new SparkConf)) val mockEventHandler = new BaseRmmEventHandler() RmmSpark.setEventHandler(mockEventHandler) RmmSpark.currentThreadIsDedicatedToTask(1) } override def afterEach(): Unit = { + SpillFramework.shutdown() RmmSpark.removeAllCurrentThreadAssociation() RmmSpark.clearEventHandler() - RapidsBufferCatalog.close() if (rmmWasInitialized) { Rmm.shutdown() } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleClientSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleClientSuite.scala index e9873b0bc5f..515ff75ffaa 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleClientSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleClientSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -255,7 +255,7 @@ class RapidsShuffleClientSuite extends RapidsShuffleTestHelper { verify(client, times(1)).track(any[DeviceMemoryBuffer](), tmCaptor.capture()) verifyTableMeta(tableMeta, tmCaptor.getValue.asInstanceOf[TableMeta]) verify(mockCatalog, times(1)) - .addBuffer(dmbCaptor.capture(), any(), any(), any()) + .addBuffer(dmbCaptor.capture(), any(), any()) val receivedBuff = dmbCaptor.getValue.asInstanceOf[DeviceMemoryBuffer] assertResult(tableMeta.bufferMeta().size())(receivedBuff.getLength) @@ -310,8 +310,7 @@ class RapidsShuffleClientSuite extends RapidsShuffleTestHelper { verify(client, times(1)).track(any[DeviceMemoryBuffer](), tmCaptor.capture()) verifyTableMeta(tableMeta, tmCaptor.getValue.asInstanceOf[TableMeta]) verify(mockCatalog, times(1)) - .addBuffer(dmbCaptor.capture(), any(), any(), any()) - verify(mockCatalog, times(1)).removeBuffer(any()) + .addBuffer(dmbCaptor.capture(), any(), any()) val receivedBuff = dmbCaptor.getValue.asInstanceOf[DeviceMemoryBuffer] assertResult(tableMeta.bufferMeta().size())(receivedBuff.getLength) @@ -367,7 +366,7 @@ class RapidsShuffleClientSuite extends RapidsShuffleTestHelper { } verify(mockCatalog, times(5)) - .addBuffer(dmbCaptor.capture(), any(), any(), any()) + .addBuffer(dmbCaptor.capture(), any(), any()) assertResult(totalExpectedSize)( dmbCaptor.getAllValues().toArray().map(_.asInstanceOf[DeviceMemoryBuffer].getLength).sum) @@ -424,7 +423,7 @@ class RapidsShuffleClientSuite extends RapidsShuffleTestHelper { } verify(mockCatalog, times(20)) - .addBuffer(dmbCaptor.capture(), any(), any(), any()) + .addBuffer(dmbCaptor.capture(), any(), any()) assertResult(totalExpectedSize)( dmbCaptor.getAllValues().toArray().map(_.asInstanceOf[DeviceMemoryBuffer].getLength).sum) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIteratorSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIteratorSuite.scala index 70064682ed0..af24b332c83 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIteratorSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIteratorSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,8 +16,9 @@ package com.nvidia.spark.rapids.shuffle -import com.nvidia.spark.rapids.{RapidsBuffer, RapidsBufferHandle} +import com.nvidia.spark.rapids.RapidsShuffleHandle import com.nvidia.spark.rapids.jni.RmmSpark +import com.nvidia.spark.rapids.spill.SpillableDeviceBufferHandle import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers._ import org.mockito.Mockito._ @@ -30,18 +31,9 @@ class RapidsShuffleIteratorSuite extends RapidsShuffleTestHelper { val taskId = 1 try { RmmSpark.currentThreadIsDedicatedToTask(taskId) - val blocksByAddress = RapidsShuffleTestHelper.getBlocksByAddress - - val cl = new RapidsShuffleIterator( - RapidsShuffleTestHelper.makeMockBlockManager("1", "1"), - mockConf, - mockTransport, - blocksByAddress, - testMetricsUpdater, - Array.empty, - taskId, - mockCatalog, - 123) + val cl = + RapidsShuffleTestHelper.makeIterator( + mockConf, mockTransport, testMetricsUpdater, taskId, mockCatalog) when(mockTransaction.getStatus).thenReturn(TransactionStatus.Error) @@ -64,18 +56,9 @@ class RapidsShuffleIteratorSuite extends RapidsShuffleTestHelper { RmmSpark.currentThreadIsDedicatedToTask(taskId) when(mockTransaction.getStatus).thenReturn(status) - val blocksByAddress = RapidsShuffleTestHelper.getBlocksByAddress - - val cl = spy(new RapidsShuffleIterator( - RapidsShuffleTestHelper.makeMockBlockManager("1", "1"), - mockConf, - mockTransport, - blocksByAddress, - testMetricsUpdater, - Array.empty, - taskId, - mockCatalog, - 123)) + val cl = + RapidsShuffleTestHelper.makeIterator( + mockConf, mockTransport, testMetricsUpdater, taskId, mockCatalog) val ac = ArgumentCaptor.forClass(classOf[RapidsShuffleFetchHandler]) when(mockTransport.makeClient(any())).thenReturn(client) @@ -112,18 +95,10 @@ class RapidsShuffleIteratorSuite extends RapidsShuffleTestHelper { val taskId = 1 try { RmmSpark.currentThreadIsDedicatedToTask(taskId) - val blocksByAddress = RapidsShuffleTestHelper.getBlocksByAddress - - val cl = spy(new RapidsShuffleIterator( - RapidsShuffleTestHelper.makeMockBlockManager("1", "1"), - mockConf, - mockTransport, - blocksByAddress, - testMetricsUpdater, - Array.empty, - taskId, - mockCatalog, - 123)) + + val cl = + RapidsShuffleTestHelper.makeIterator( + mockConf, mockTransport, testMetricsUpdater, taskId, mockCatalog) val ac = ArgumentCaptor.forClass(classOf[RapidsShuffleFetchHandler]) when(mockTransport.makeClient(any())).thenReturn(client) @@ -162,18 +137,10 @@ class RapidsShuffleIteratorSuite extends RapidsShuffleTestHelper { val taskId = 1 try { RmmSpark.currentThreadIsDedicatedToTask(taskId) - val blocksByAddress = RapidsShuffleTestHelper.getBlocksByAddress - - val cl = spy(new RapidsShuffleIterator( - RapidsShuffleTestHelper.makeMockBlockManager("1", "1"), - mockConf, - mockTransport, - blocksByAddress, - testMetricsUpdater, - Array.empty, - taskId, - mockCatalog, - 123)) + + val cl = + RapidsShuffleTestHelper.makeIterator( + mockConf, mockTransport, testMetricsUpdater, taskId, mockCatalog) when(mockTransport.makeClient(any())).thenReturn(client) doNothing().when(client).doFetch(any(), any()) @@ -198,29 +165,21 @@ class RapidsShuffleIteratorSuite extends RapidsShuffleTestHelper { val taskId = 1 try { RmmSpark.currentThreadIsDedicatedToTask(taskId) - val blocksByAddress = RapidsShuffleTestHelper.getBlocksByAddress - - val cl = new RapidsShuffleIterator( - RapidsShuffleTestHelper.makeMockBlockManager("1", "1"), - mockConf, - mockTransport, - blocksByAddress, - testMetricsUpdater, - Array.empty, - taskId, - mockCatalog, - 123) + val cl = + RapidsShuffleTestHelper.makeIterator( + mockConf, mockTransport, testMetricsUpdater, taskId, mockCatalog) val ac = ArgumentCaptor.forClass(classOf[RapidsShuffleFetchHandler]) when(mockTransport.makeClient(any())).thenReturn(client) doNothing().when(client).doFetch(any(), ac.capture()) - val mockBuffer = mock[RapidsBuffer] + val mockBuffer = RapidsShuffleHandle(mock[SpillableDeviceBufferHandle], null) + when(mockBuffer.spillable.sizeInBytes).thenReturn(123L) val cb = new ColumnarBatch(Array.empty, 10) - val handle = mock[RapidsBufferHandle] - when(mockBuffer.getColumnarBatch(Array.empty)).thenReturn(cb) - when(mockCatalog.acquireBuffer(any[RapidsBufferHandle]())).thenReturn(mockBuffer) - doNothing().when(mockCatalog).removeBuffer(any()) + val handle = mock[RapidsShuffleHandle] + doAnswer(_ => (cb, 123L)).when(mockCatalog) + .getColumnarBatchAndRemove(any[RapidsShuffleHandle](), any()) + cl.start() val handler = ac.getValue.asInstanceOf[RapidsShuffleFetchHandler] @@ -232,7 +191,7 @@ class RapidsShuffleIteratorSuite extends RapidsShuffleTestHelper { assert(cl.hasNext) assertResult(cb)(cl.next()) assertResult(1)(testMetricsUpdater.totalRemoteBlocksFetched) - assertResult(mockBuffer.memoryUsedBytes)(testMetricsUpdater.totalRemoteBytesRead) + assertResult(123L)(testMetricsUpdater.totalRemoteBytesRead) assertResult(10)(testMetricsUpdater.totalRowsFetched) } finally { RmmSpark.taskDone(taskId) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleServerSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleServerSuite.scala index 3eb73ef0f13..8d7415fba04 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleServerSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleServerSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,73 +18,58 @@ package com.nvidia.spark.rapids.shuffle import java.io.IOException import java.nio.ByteBuffer -import java.util -import ai.rapids.cudf.{Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer} -import com.nvidia.spark.rapids.{MetaUtils, RapidsBuffer, ShuffleMetadata} +import ai.rapids.cudf.{DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer} +import com.nvidia.spark.rapids.{MetaUtils, RapidsShuffleHandle, ShuffleMetadata} import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.format.TableMeta +import com.nvidia.spark.rapids.spill.SpillableDeviceBufferHandle import org.mockito.{ArgumentCaptor, ArgumentMatchers} -import org.mockito.ArgumentMatchers.{any, anyLong} +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.apache.spark.storage.ShuffleBlockBatchId -class RapidsShuffleServerSuite extends RapidsShuffleTestHelper { +class MockRapidsShuffleRequestHandler(mockBuffers: Seq[RapidsShuffleHandle]) + extends RapidsShuffleRequestHandler with AutoCloseable { + var acquiredTables = Seq[Int]() + override def getShuffleBufferMetas( + shuffleBlockBatchId: ShuffleBlockBatchId): Seq[TableMeta] = { + throw new NotImplementedError("getShuffleBufferMetas") + } - def setupMocks(deviceBuffers: Seq[DeviceMemoryBuffer]): (RapidsShuffleRequestHandler, - Seq[RapidsBuffer], util.HashMap[RapidsBuffer, Int]) = { + override def getShuffleHandle(tableId: Int): RapidsShuffleHandle = { + acquiredTables = acquiredTables :+ tableId + mockBuffers(tableId) + } - val numCloses = new util.HashMap[RapidsBuffer, Int]() + override def close(): Unit = { + // a removeShuffle action would likewise remove handles + mockBuffers.foreach(_.close()) + } +} + +class RapidsShuffleServerSuite extends RapidsShuffleTestHelper { + + def setupMocks(deviceBuffers: Seq[DeviceMemoryBuffer]): MockRapidsShuffleRequestHandler = { val mockBuffers = deviceBuffers.map { deviceBuffer => withResource(HostMemoryBuffer.allocate(deviceBuffer.getLength)) { hostBuff => fillBuffer(hostBuff) deviceBuffer.copyFromHostBuffer(hostBuff) - val mockBuffer = mock[RapidsBuffer] val mockMeta = RapidsShuffleTestHelper.mockTableMeta(100000) - when(mockBuffer.copyToMemoryBuffer(anyLong(), any[MemoryBuffer](), anyLong(), anyLong(), - any[Cuda.Stream]())).thenAnswer { invocation => - // start at 1 close, since we'll need to close at refcount 0 too - val newNumCloses = numCloses.getOrDefault(mockBuffer, 1) + 1 - numCloses.put(mockBuffer, newNumCloses) - val srcOffset = invocation.getArgument[Long](0) - val dst = invocation.getArgument[MemoryBuffer](1) - val dstOffset = invocation.getArgument[Long](2) - val length = invocation.getArgument[Long](3) - val stream = invocation.getArgument[Cuda.Stream](4) - dst.copyFromMemoryBuffer(dstOffset, deviceBuffer, srcOffset, length, stream) - } - when(mockBuffer.getPackedSizeBytes).thenReturn(deviceBuffer.getLength) - when(mockBuffer.meta).thenReturn(mockMeta) - mockBuffer + RapidsShuffleHandle(SpillableDeviceBufferHandle(deviceBuffer), mockMeta) } } - - val handler = new RapidsShuffleRequestHandler { - var acquiredTables = Seq[Int]() - override def getShuffleBufferMetas( - shuffleBlockBatchId: ShuffleBlockBatchId): Seq[TableMeta] = { - throw new NotImplementedError("getShuffleBufferMetas") - } - - override def acquireShuffleBuffer(tableId: Int): RapidsBuffer = { - acquiredTables = acquiredTables :+ tableId - mockBuffers(tableId) - } - } - (handler, mockBuffers, numCloses) + new MockRapidsShuffleRequestHandler(mockBuffers) } - class MockBlockWithSize(val b: DeviceMemoryBuffer) extends BlockWithSize { - override def size: Long = b.getLength - } + class MockBlockWithSize(override val size: Long) extends BlockWithSize {} def compareRanges( bounceBuffer: SendBounceBuffers, - receiveBlocks: Seq[BlockRange[MockBlockWithSize]]): Unit = { + receiveBlocks: Seq[(BlockRange[MockBlockWithSize], DeviceMemoryBuffer)]): Unit = { var bounceBuffOffset = 0L - receiveBlocks.foreach { range => - val deviceBuff = range.block.b + receiveBlocks.foreach { case (range, deviceBuff) => val deviceBounceBuff = bounceBuffer.deviceBounceBuffer.buffer withResource(deviceBounceBuff.slice(bounceBuffOffset, range.rangeSize())) { bbSlice => bounceBuffOffset = bounceBuffOffset + range.rangeSize() @@ -104,26 +89,24 @@ class RapidsShuffleServerSuite extends RapidsShuffleTestHelper { when(mockTx.releaseMessage()).thenReturn(transferRequest) val bb = closeOnExcept(getSendBounceBuffer(10000)) { bounceBuffer => - withResource((0 until 10).map(_ => DeviceMemoryBuffer.allocate(1000))) { deviceBuffers => - val receiveSide = deviceBuffers.map(b => new MockBlockWithSize(b)) + val deviceBuffers = (0 until 10).map(_ => DeviceMemoryBuffer.allocate(1000)) + val receiveSide = deviceBuffers.map(_ => new MockBlockWithSize(1000)) + withResource(setupMocks(deviceBuffers)) { handler => val receiveWindow = new WindowedBlockIterator[MockBlockWithSize](receiveSide, 10000) - val (handler, mockBuffers, numCloses) = setupMocks(deviceBuffers) withResource(new BufferSendState(mockTx, bounceBuffer, handler)) { bss => assert(bss.hasMoreSends) withResource(bss.getBufferToSend()) { mb => val receiveBlocks = receiveWindow.next() - compareRanges(bounceBuffer, receiveBlocks) + compareRanges(bounceBuffer, receiveBlocks.zip(deviceBuffers)) assertResult(10000)(mb.getLength) assert(!bss.hasMoreSends) bss.releaseAcquiredToCatalog() - mockBuffers.foreach { b: RapidsBuffer => - // should have seen 2 closes, one for BufferSendState acquiring for metadata - // and the second acquisition for copying - verify(b, times(numCloses.get(b))).close() - } } } } + deviceBuffers.foreach { b => + assertResult(0)(b.getRefCount) + } bounceBuffer } assert(bb.deviceBounceBuffer.isClosed) @@ -136,32 +119,29 @@ class RapidsShuffleServerSuite extends RapidsShuffleTestHelper { when(mockTx.releaseMessage()).thenReturn(transferRequest) val bb = closeOnExcept(getSendBounceBuffer(10000)) { bounceBuffer => - withResource((0 until 20).map(_ => DeviceMemoryBuffer.allocate(1000))) { deviceBuffers => - val receiveSide = deviceBuffers.map(b => new MockBlockWithSize(b)) - val receiveWindow = new WindowedBlockIterator[MockBlockWithSize](receiveSide, 10000) - val (handler, mockBuffers, numCloses) = setupMocks(deviceBuffers) + val deviceBuffers = (0 until 20).map(_ => DeviceMemoryBuffer.allocate(1000)) + val receiveSide = deviceBuffers.map(_ => new MockBlockWithSize(1000)) + val receiveWindow = new WindowedBlockIterator[MockBlockWithSize](receiveSide, 10000) + withResource(setupMocks(deviceBuffers)) { handler => withResource(new BufferSendState(mockTx, bounceBuffer, handler)) { bss => withResource(bss.getBufferToSend()) { _ => val receiveBlocks = receiveWindow.next() - compareRanges(bounceBuffer, receiveBlocks) + compareRanges(bounceBuffer, receiveBlocks.zip(deviceBuffers)) assert(bss.hasMoreSends) bss.releaseAcquiredToCatalog() } withResource(bss.getBufferToSend()) { _ => val receiveBlocks = receiveWindow.next() - compareRanges(bounceBuffer, receiveBlocks) + compareRanges(bounceBuffer, receiveBlocks.zip(deviceBuffers)) assert(!bss.hasMoreSends) bss.releaseAcquiredToCatalog() } - - mockBuffers.foreach { b: RapidsBuffer => - // should have seen 2 closes, one for BufferSendState acquiring for metadata - // and the second acquisition for copying - verify(b, times(numCloses.get(b))).close() - } } } + deviceBuffers.foreach { b => + assertResult(0)(b.getRefCount) + } bounceBuffer } assert(bb.deviceBounceBuffer.isClosed) @@ -174,24 +154,23 @@ class RapidsShuffleServerSuite extends RapidsShuffleTestHelper { when(mockTx.releaseMessage()).thenReturn(transferRequest) val bb = closeOnExcept(getSendBounceBuffer(10000)) { bounceBuffer => - withResource((0 until 20).map(_ => DeviceMemoryBuffer.allocate(123000))) { deviceBuffers => - val (handler, mockBuffers, numCloses) = setupMocks(deviceBuffers) - - val receiveSide = deviceBuffers.map(b => new MockBlockWithSize(b)) + val deviceBuffers = (0 until 20).map(_ => DeviceMemoryBuffer.allocate(123000)) + withResource(setupMocks(deviceBuffers)) { handler => + val receiveSide = deviceBuffers.map(_ => new MockBlockWithSize(123000)) val receiveWindow = new WindowedBlockIterator[MockBlockWithSize](receiveSide, 10000) withResource(new BufferSendState(mockTx, bounceBuffer, handler)) { bss => (0 until 246).foreach { _ => withResource(bss.getBufferToSend()) { _ => val receiveBlocks = receiveWindow.next() - compareRanges(bounceBuffer, receiveBlocks) + compareRanges(bounceBuffer, receiveBlocks.zip(deviceBuffers)) bss.releaseAcquiredToCatalog() } } assert(!bss.hasMoreSends) } - mockBuffers.foreach { b: RapidsBuffer => - verify(b, times(numCloses.get(b))).close() - } + } + deviceBuffers.foreach { b => + assertResult(0)(b.getRefCount) } bounceBuffer } @@ -224,14 +203,13 @@ class RapidsShuffleServerSuite extends RapidsShuffleTestHelper { any(), any(), any(), any[MemoryBuffer](), ac.capture())).thenReturn(mockTransaction) val mockRequestHandler = mock[RapidsShuffleRequestHandler] - val rapidsBuffer = mock[RapidsBuffer] val bb = ByteBuffer.allocateDirect(123) withResource(new RefCountedDirectByteBuffer(bb)) { _ => val tableMeta = MetaUtils.buildTableMeta(1, 456, bb, 100) - when(rapidsBuffer.meta).thenReturn(tableMeta) - when(rapidsBuffer.getPackedSizeBytes).thenReturn(tableMeta.bufferMeta().size()) - when(mockRequestHandler.acquireShuffleBuffer(ArgumentMatchers.eq(1))) + val testHandle = SpillableDeviceBufferHandle(DeviceMemoryBuffer.allocate(456)) + val rapidsBuffer = RapidsShuffleHandle(testHandle, tableMeta) + when(mockRequestHandler.getShuffleHandle(ArgumentMatchers.eq(1))) .thenReturn(rapidsBuffer) val server = new RapidsShuffleServer( @@ -245,7 +223,7 @@ class RapidsShuffleServerSuite extends RapidsShuffleTestHelper { server.start() - val bss = new BufferSendState(mockTransaction, mockSendBuffer, mockRequestHandler, null) + val bss = new BufferSendState(mockTransaction, mockSendBuffer, mockRequestHandler) server.doHandleTransferRequest(Seq(bss)) val cb = ac.getValue.asInstanceOf[TransactionCallback] cb(mockTransaction) @@ -253,10 +231,14 @@ class RapidsShuffleServerSuite extends RapidsShuffleTestHelper { cb(mockTransaction) // bounce buffers are freed verify(mockSendBuffer, times(1)).close() - // acquire 3 times, and close 3 times - verify(mockRequestHandler, times(3)) - .acquireShuffleBuffer(ArgumentMatchers.eq(1)) - verify(rapidsBuffer, times(3)).close() + // acquire once at the beginning, and closed at the end + verify(mockRequestHandler, times(1)) + .getShuffleHandle(ArgumentMatchers.eq(1)) + withResource(rapidsBuffer.spillable.materialize()) { dmb => + // refcount=2 because it was on the device, and we +1 to materialize. + // but it shows no leaks. + assertResult(2)(dmb.getRefCount) + } } } } @@ -264,74 +246,80 @@ class RapidsShuffleServerSuite extends RapidsShuffleTestHelper { test("when we fail to prepare a send, throw if nothing can be handled") { val mockSendBuffer = mock[SendBounceBuffers] val mockDeviceBounceBuffer = mock[BounceBuffer] - withResource(DeviceMemoryBuffer.allocate(123)) { buff => - when(mockDeviceBounceBuffer.buffer).thenReturn(buff) - when(mockSendBuffer.bounceBufferSize).thenReturn(buff.getLength) - when(mockSendBuffer.hostBounceBuffer).thenReturn(None) - when(mockSendBuffer.deviceBounceBuffer).thenReturn(mockDeviceBounceBuffer) - - when(mockTransport.tryGetSendBounceBuffers(any(), any())) - .thenReturn(Seq(mockSendBuffer)) - - val tr = ShuffleMetadata.buildTransferRequest(0, Seq(1)) - when(mockTransaction.getStatus) - .thenReturn(TransactionStatus.Success) - when(mockTransaction.releaseMessage()).thenReturn( - new MetadataTransportBuffer(new RefCountedDirectByteBuffer(tr))) - - val mockServerConnection = mock[ServerConnection] - val mockRequestHandler = mock[RapidsShuffleRequestHandler] - val rapidsBuffer = mock[RapidsBuffer] - - val bb = ByteBuffer.allocateDirect(123) - withResource(new RefCountedDirectByteBuffer(bb)) { _ => - val tableMeta = MetaUtils.buildTableMeta(1, 456, bb, 100) - when(rapidsBuffer.meta).thenReturn(tableMeta) - when(rapidsBuffer.getPackedSizeBytes).thenReturn(tableMeta.bufferMeta().size()) - when(mockRequestHandler.acquireShuffleBuffer(ArgumentMatchers.eq(1))) - .thenReturn(rapidsBuffer) - - val server = spy(new RapidsShuffleServer( - mockTransport, - mockServerConnection, - RapidsShuffleTestHelper.makeMockBlockManager("1", "foo"), - mockRequestHandler, - mockExecutor, - mockBssExecutor, - mockConf)) - - server.start() - - val ioe = new IOException("mmap failed in test") - - when(rapidsBuffer.copyToMemoryBuffer(any(), any(), any(), any(), any())) - .thenAnswer(_ => throw ioe) - - val bss = new BufferSendState(mockTransaction, mockSendBuffer, mockRequestHandler, null) - // if nothing else can be handled, we throw - assertThrows[IllegalStateException] { - try { - server.doHandleTransferRequest(Seq(bss)) - } catch { - case e: Throwable => - assertResult(1)(e.getSuppressed.length) - assertResult(ioe)(e.getSuppressed()(0).getCause) - throw e - } + val mockDeviceMemoryBuffer = mock[DeviceMemoryBuffer] + when(mockDeviceBounceBuffer.buffer).thenReturn(mockDeviceMemoryBuffer) + when(mockSendBuffer.bounceBufferSize).thenReturn(1024) + when(mockSendBuffer.hostBounceBuffer).thenReturn(None) + when(mockSendBuffer.deviceBounceBuffer).thenReturn(mockDeviceBounceBuffer) + + when(mockTransport.tryGetSendBounceBuffers(any(), any())) + .thenReturn(Seq(mockSendBuffer)) + + val tr = ShuffleMetadata.buildTransferRequest(0, Seq(1, 2)) + when(mockTransaction.getStatus) + .thenReturn(TransactionStatus.Success) + when(mockTransaction.releaseMessage()).thenReturn( + new MetadataTransportBuffer(new RefCountedDirectByteBuffer(tr))) + + val mockServerConnection = mock[ServerConnection] + val mockRequestHandler = mock[RapidsShuffleRequestHandler] + + val bb = ByteBuffer.allocateDirect(123) + withResource(new RefCountedDirectByteBuffer(bb)) { _ => + val tableMeta = MetaUtils.buildTableMeta(1, 456, bb, 100) + val mockHandle = mock[SpillableDeviceBufferHandle] + val mockHandleThatThrows = mock[SpillableDeviceBufferHandle] + val mockMaterialized = mock[DeviceMemoryBuffer] + when(mockHandle.sizeInBytes).thenReturn(tableMeta.bufferMeta().size()) + when(mockHandle.materialize()).thenAnswer(_ => mockMaterialized) + + when(mockHandleThatThrows.sizeInBytes).thenReturn(tableMeta.bufferMeta().size()) + val ex = new IllegalStateException("something happened") + when(mockHandleThatThrows.materialize()).thenThrow(ex) + + val rapidsBuffer = RapidsShuffleHandle(mockHandle, tableMeta) + val rapidsBufferThatThrows = RapidsShuffleHandle(mockHandleThatThrows, tableMeta) + + when(mockRequestHandler.getShuffleHandle(ArgumentMatchers.eq(1))) + .thenReturn(rapidsBuffer) + when(mockRequestHandler.getShuffleHandle(ArgumentMatchers.eq(2))) + .thenReturn(rapidsBufferThatThrows) + + val server = spy(new RapidsShuffleServer( + mockTransport, + mockServerConnection, + RapidsShuffleTestHelper.makeMockBlockManager("1", "foo"), + mockRequestHandler, + mockExecutor, + mockBssExecutor, + mockConf)) + + server.start() + + val bss = new BufferSendState(mockTransaction, mockSendBuffer, mockRequestHandler, null) + // if nothing else can be handled, we throw + assertThrows[IllegalStateException] { + try { + server.doHandleTransferRequest(Seq(bss)) + } catch { + case e: Throwable => + assertResult(1)(e.getSuppressed.length) + assertResult(ex)(e.getSuppressed()(0).getCause) + throw e } + } - // since nothing could be handled, we don't try again - verify(server, times(0)).addToContinueQueue(any()) + // since nothing could be handled, we don't try again + verify(server, times(0)).addToContinueQueue(any()) - // bounce buffers are freed - verify(mockSendBuffer, times(1)).close() + // bounce buffers are freed + verify(mockSendBuffer, times(1)).close() - // acquire 2 times, 1 to make the ranges, and the 2 before the copy - // close 2 times corresponding to each open - verify(mockRequestHandler, times(2)) - .acquireShuffleBuffer(ArgumentMatchers.eq(1)) - verify(rapidsBuffer, times(2)).close() - } + verify(mockRequestHandler, times(1)) + .getShuffleHandle(ArgumentMatchers.eq(1)) + + // the spillable that materialized we need to close + verify(mockMaterialized, times(1)).close() } } @@ -367,13 +355,24 @@ class RapidsShuffleServerSuite extends RapidsShuffleTestHelper { val mockRequestHandler = mock[RapidsShuffleRequestHandler] - def makeMockBuffer(tableId: Int, bb: ByteBuffer): RapidsBuffer = { - val rapidsBuffer = mock[RapidsBuffer] + def makeMockBuffer(tableId: Int, bb: ByteBuffer, error: Boolean): RapidsShuffleHandle = { val tableMeta = MetaUtils.buildTableMeta(tableId, 456, bb, 100) - when(rapidsBuffer.meta).thenReturn(tableMeta) - when(rapidsBuffer.getPackedSizeBytes).thenReturn(tableMeta.bufferMeta().size()) - when(mockRequestHandler.acquireShuffleBuffer(ArgumentMatchers.eq(tableId))) - .thenReturn(rapidsBuffer) + val rapidsBuffer = if (error) { + val mockHandle = mock[SpillableDeviceBufferHandle] + val rapidsBuffer = RapidsShuffleHandle(mockHandle, tableMeta) + when(mockHandle.sizeInBytes).thenReturn(tableMeta.bufferMeta().size()) + // mock an error with the copy + when(rapidsBuffer.spillable.materialize()) + .thenAnswer(_ => { + throw new IOException("mmap failed in test") + }) + rapidsBuffer + } else { + val testHandle = spy(SpillableDeviceBufferHandle(spy(DeviceMemoryBuffer.allocate(456)))) + RapidsShuffleHandle(testHandle, tableMeta) + } + when(mockRequestHandler.getShuffleHandle(ArgumentMatchers.eq(tableId))) + .thenAnswer(_ => rapidsBuffer) rapidsBuffer } @@ -381,19 +380,8 @@ class RapidsShuffleServerSuite extends RapidsShuffleTestHelper { val bb2 = ByteBuffer.allocateDirect(123) withResource(new RefCountedDirectByteBuffer(bb)) { _ => withResource(new RefCountedDirectByteBuffer(bb2)) { _ => - val rapidsBuffer = makeMockBuffer(1, bb) - val rapidsBuffer2 = makeMockBuffer(2, bb2) - - // error with copy - when(rapidsBuffer.copyToMemoryBuffer(any(), any(), any(), any(), any())) - .thenAnswer(_ => { - throw new IOException("mmap failed in test") - }) - - // successful copy - doNothing() - .when(rapidsBuffer2) - .copyToMemoryBuffer(any(), any(), any(), any(), any()) + val rapidsHandle = makeMockBuffer(1, bb, error = true) + val rapidsHandle2 = makeMockBuffer(2, bb2, error = false) val server = spy(new RapidsShuffleServer( mockTransport, @@ -407,14 +395,21 @@ class RapidsShuffleServerSuite extends RapidsShuffleTestHelper { server.start() val bssFailed = new BufferSendState( - mockTransaction, mockSendBuffer, mockRequestHandler, null) + mockTransaction, mockSendBuffer, mockRequestHandler) val bssSuccess = spy(new BufferSendState( - mockTransaction2, mockSendBuffer, mockRequestHandler, null)) - - when(bssSuccess.hasMoreSends) - .thenReturn(true) // send 1 bounce buffer length - .thenReturn(false) + mockTransaction2, mockSendBuffer, mockRequestHandler)) + + var callCount = 0 + doAnswer { _ => + callCount += 1 + // send 1 buffer length + if (callCount > 1){ + false + } else { + true + } + }.when(bssSuccess).hasMoreSends // if something else can be handled we don't throw, and re-queue server.doHandleTransferRequest(Seq(bssFailed, bssSuccess)) @@ -427,15 +422,21 @@ class RapidsShuffleServerSuite extends RapidsShuffleTestHelper { // the bounce buffer is freed 1 time for `bssSuccess`, but not for `bssFailed` verify(mockSendBuffer, times(1)).close() - // acquire/close 4 times => - // we had two requests for 1 buffer, and each request acquires 2 times and closes - // 2 times. - verify(mockRequestHandler, times(2)) - .acquireShuffleBuffer(ArgumentMatchers.eq(1)) - verify(mockRequestHandler, times(2)) - .acquireShuffleBuffer(ArgumentMatchers.eq(2)) - verify(rapidsBuffer, times(2)).close() - verify(rapidsBuffer2, times(2)).close() + // we obtained the handles once, we don't need to get them again + verify(mockRequestHandler, times(1)) + .getShuffleHandle(ArgumentMatchers.eq(1)) + verify(mockRequestHandler, times(1)) + .getShuffleHandle(ArgumentMatchers.eq(2)) + // this handle fails to materialize + verify(rapidsHandle.spillable, times(1)).materialize() + + // this handle materializes, so make sure we close it + verify(rapidsHandle2.spillable, times(1)).materialize() + withResource(rapidsHandle2.spillable.materialize()) { dmb => + // refcount=2 because it was on the device, and we +1 to materialize. + // but it shows no leaks. + assertResult(2)(dmb.getRefCount) + } } } } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/spill/SpillFrameworkSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/spill/SpillFrameworkSuite.scala new file mode 100644 index 00000000000..31377695fe4 --- /dev/null +++ b/tests/src/test/scala/com/nvidia/spark/rapids/spill/SpillFrameworkSuite.scala @@ -0,0 +1,1105 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.spill + +import java.io.File +import java.math.RoundingMode + +import scala.collection.mutable.ArrayBuffer + +import ai.rapids.cudf._ +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} +import com.nvidia.spark.rapids.format.CodecType +import org.mockito.Mockito.when +import org.scalatest.BeforeAndAfterAll +import org.scalatestplus.mockito.MockitoSugar + +import org.apache.spark.SparkConf +import org.apache.spark.sql.rapids.RapidsDiskBlockManager +import org.apache.spark.sql.types._ +import org.apache.spark.sql.vectorized.ColumnarBatch + +class SpillFrameworkSuite + extends FunSuiteWithTempDir + with MockitoSugar + with BeforeAndAfterAll { + + override def beforeEach(): Unit = { + super.beforeEach() + val sc = new SparkConf + sc.set(RapidsConf.HOST_SPILL_STORAGE_SIZE.key, "1024") + SpillFramework.initialize(new RapidsConf(sc)) + } + + override def afterEach(): Unit = { + super.afterEach() + SpillFramework.shutdown() + } + + private def buildContiguousTable(): (ContiguousTable, Array[DataType]) = { + val (tbl, dataTypes) = buildTable() + withResource(tbl) { _ => + (tbl.contiguousSplit()(0), dataTypes) + } + } + + private def buildTableOfLongs(numRows: Int): (ContiguousTable, Array[DataType])= { + val vals = (0 until numRows).map(_.toLong) + withResource(HostColumnVector.fromLongs(vals: _*)) { hcv => + withResource(hcv.copyToDevice()) { cv => + withResource(new Table(cv)) { table => + (table.contiguousSplit()(0), Array[DataType](LongType)) + } + } + } + } + + private def buildNonContiguousTableOfLongs( + numRows: Int): (Table, Array[DataType])= { + val vals = (0 until numRows).map(_.toLong) + withResource(HostColumnVector.fromLongs(vals: _*)) { hcv => + withResource(hcv.copyToDevice()) { cv => + (new Table(cv), Array[DataType](LongType)) + } + } + } + + private def buildTable(): (Table, Array[DataType]) = { + val tbl = new Table.TestBuilder() + .column(5, null.asInstanceOf[java.lang.Integer], 3, 1) + .column("five", "two", null, null) + .column(5.0, 2.0, 3.0, 1.0) + .decimal64Column(-5, RoundingMode.UNNECESSARY, 0, null, -1.4, 10.123) + .build() + val types: Array[DataType] = + Seq(IntegerType, StringType, DoubleType, DecimalType(10, 5)).toArray + (tbl, types) + } + + private def buildTableWithDuplicate(): (Table, Array[DataType]) = { + withResource(ColumnVector.fromInts(5, null.asInstanceOf[java.lang.Integer], 3, 1)) { intCol => + withResource(ColumnVector.fromStrings("five", "two", null, null)) { stringCol => + withResource(ColumnVector.fromDoubles(5.0, 2.0, 3.0, 1.0)) { doubleCol => + // add intCol twice + (new Table(intCol, intCol, stringCol, doubleCol), + Array(IntegerType, IntegerType, StringType, DoubleType)) + } + } + } + } + + private def buildEmptyTable(): (Table, Array[DataType]) = { + val (tbl, types) = buildTable() + val emptyTbl = withResource(tbl) { _ => + withResource(ColumnVector.fromBooleans(false, false, false, false)) { mask => + tbl.filter(mask) // filter all out + } + } + (emptyTbl, types) + } + + private def testBufferFileDeletion(canShareDiskPaths: Boolean): Unit = { + val (_, handle, _) = addContiguousTableToFramework() + var path: File = null + withResource(handle) { _ => + SpillFramework.stores.deviceStore.spill(handle.approxSizeInBytes) + SpillFramework.stores.hostStore.spill(handle.approxSizeInBytes) + assert(handle.host.isDefined) + assert(handle.host.map(_.disk.isDefined).get) + path = SpillFramework.stores.diskStore.getFile(handle.host.flatMap(_.disk).get.blockId) + assert(path.exists) + } + assert(!path.exists) + } + + private def addContiguousTableToFramework(): ( + Long, SpillableColumnarBatchFromBufferHandle, Array[DataType]) = { + val (ct, dataTypes) = buildContiguousTable() + val bufferSize = ct.getBuffer.getLength + val handle = SpillableColumnarBatchFromBufferHandle(ct, dataTypes) + (bufferSize, handle, dataTypes) + } + + private def addTableToFramework(): (SpillableColumnarBatchHandle, Array[DataType]) = { + // store takes ownership of the table + val (tbl, dataTypes) = buildTable() + val cb = withResource(tbl) { _ => GpuColumnVector.from(tbl, dataTypes) } + val handle = SpillableColumnarBatchHandle(cb) + (handle, dataTypes) + } + + private def addZeroRowsTableToFramework(): (SpillableColumnarBatchHandle, Array[DataType]) = { + val (table, dataTypes) = buildEmptyTable() + val cb = withResource(table) { _ => GpuColumnVector.from(table, dataTypes) } + val handle = SpillableColumnarBatchHandle(cb) + (handle, dataTypes) + } + + private def buildHostBatch(): (ColumnarBatch, Array[DataType]) = { + val (ct, dataTypes) = buildContiguousTable() + val hostCols = withResource(ct) { _ => + withResource(ct.getTable) { tbl => + (0 until tbl.getNumberOfColumns) + .map(c => tbl.getColumn(c).copyToHost()) + } + }.toArray + (new ColumnarBatch( + hostCols.zip(dataTypes).map { case (hostCol, dataType) => + new RapidsHostColumnVector(dataType, hostCol) + }, hostCols.head.getRowCount.toInt), dataTypes) + } + + private def buildHostBatchWithDuplicate(): (ColumnarBatch, Array[DataType]) = { + val (ct, dataTypes) = buildContiguousTable() + val hostCols = withResource(ct) { _ => + withResource(ct.getTable) { tbl => + (0 until tbl.getNumberOfColumns) + .map(c => tbl.getColumn(c).copyToHost()) + } + }.toArray + hostCols.foreach(_.incRefCount()) + (new ColumnarBatch( + (hostCols ++ hostCols).zip(dataTypes ++ dataTypes).map { case (hostCol, dataType) => + new RapidsHostColumnVector(dataType, hostCol) + }, hostCols.head.getRowCount.toInt), dataTypes) + } + + test("add table registers with device store") { + val (ct, dataTypes) = buildContiguousTable() + withResource(SpillableColumnarBatchFromBufferHandle(ct, dataTypes)) { _ => + assertResult(1)(SpillFramework.stores.deviceStore.numHandles) + } + } + + test("a non-contiguous table is spillable and it is handed over to the store") { + val (tbl, dataTypes) = buildTable() + withResource(SpillableColumnarBatchHandle(tbl, dataTypes)) { handle => + assertResult(1)(SpillFramework.stores.deviceStore.numHandles) + assert(handle.spillable) + } + } + + test("a non-contiguous table becomes non-spillable when batch is obtained") { + val (tbl, dataTypes) = buildTable() + withResource(SpillableColumnarBatchHandle(tbl, dataTypes)) { handle => + assertResult(1)(SpillFramework.stores.deviceStore.numHandles) + assert(handle.spillable) + withResource(handle.materialize(dataTypes)) { _ => + assertResult(1)(SpillFramework.stores.deviceStore.numHandles) + assert(!handle.spillable) + assertResult(0)(SpillFramework.stores.deviceStore.spill(handle.approxSizeInBytes)) + } + assert(handle.spillable) + assertResult(1)(SpillFramework.stores.deviceStore.numHandles) + assertResult(handle.approxSizeInBytes)( + SpillFramework.stores.deviceStore.spill(handle.approxSizeInBytes)) + } + } + + test("a non-contiguous table is non-spillable until all columns are returned") { + val (table, dataTypes) = buildTable() + withResource(SpillableColumnarBatchHandle(table, dataTypes)) { handle => + assert(handle.spillable) + val cb = handle.materialize(dataTypes) + assert(!handle.spillable) + val columns = GpuColumnVector.extractBases(cb) + withResource(columns.head) { _ => + columns.head.incRefCount() + withResource(cb) { _ => + assert(!handle.spillable) + } + // still 0 after the batch is closed, because of the extra incRefCount + // for columns.head + assert(!handle.spillable) + } + // columns.head is closed, so now our RapidsTable is spillable again + assert(handle.spillable) + } + } + + test("an aliased non-contiguous table is not spillable (until closing the alias) ") { + val (table, dataTypes) = buildTable() + withResource(SpillableColumnarBatchHandle(table, dataTypes)) { handle => + assertResult(1)(SpillFramework.stores.deviceStore.numHandles) + assert(handle.spillable) + withResource(SpillableColumnarBatchHandle(handle.materialize(dataTypes))) { aliasHandle => + assertResult(2)(SpillFramework.stores.deviceStore.numHandles) + assert(!handle.spillable) + assert(!aliasHandle.spillable) + } // we now have two copies in the store + assert(handle.spillable) + assertResult(1)(SpillFramework.stores.deviceStore.numHandles) + } + } + + test("an aliased contiguous table is not spillable (until closing the alias) ") { + val (table, dataTypes) = buildContiguousTable() + withResource(SpillableColumnarBatchFromBufferHandle(table, dataTypes)) { handle => + assertResult(1)(SpillFramework.stores.deviceStore.numHandles) + assert(handle.spillable) + val materialized = handle.materialize(dataTypes) + // note that materialized is a batch "from buffer", it is not a regular batch + withResource(SpillableColumnarBatchFromBufferHandle(materialized)) { aliasHandle => + // we now have two copies in the store + assertResult(2)(SpillFramework.stores.deviceStore.numHandles) + assert(!handle.spillable) + assert(!aliasHandle.spillable) + } + assert(handle.spillable) + assertResult(1)(SpillFramework.stores.deviceStore.numHandles) + } + } + + test("an non-contiguous table supports duplicated columns") { + val (table, dataTypes) = buildTableWithDuplicate() + withResource(SpillableColumnarBatchHandle(table, dataTypes)) { handle => + assertResult(1)(SpillFramework.stores.deviceStore.numHandles) + assert(handle.spillable) + withResource(SpillableColumnarBatchHandle(handle.materialize(dataTypes))) { aliasHandle => + assertResult(2)(SpillFramework.stores.deviceStore.numHandles) + assert(!handle.spillable) + assert(!aliasHandle.spillable) + } // we now have two copies in the store + assert(handle.spillable) + assertResult(1)(SpillFramework.stores.deviceStore.numHandles) + } + } + + test("a buffer is not spillable until the owner closes columns referencing it") { + val (ct, _) = buildContiguousTable() + // the contract for spillable handles is that they take ownership + // incRefCount to follow that pattern + val buff = ct.getBuffer + buff.incRefCount() + withResource(SpillableDeviceBufferHandle(buff)) { handle => + withResource(ct) { _ => + assert(!handle.spillable) + } + assert(handle.spillable) + } + } + + private def buildContiguousTable(start: Int, numRows: Int): ContiguousTable = { + val vals = (0 until numRows).map(_.toLong + start) + withResource(HostColumnVector.fromLongs(vals: _*)) { hcv => + withResource(hcv.copyToDevice()) { cv => + withResource(HostColumnVector.decimalFromLongs(-3, vals: _*)) { decHcv => + withResource(decHcv.copyToDevice()) { decCv => + withResource(new Table(cv, decCv)) { table => + table.contiguousSplit()(0) + } + } + } + } + } + } + + private def buildCompressedBatch(start: Int, numRows: Int): ColumnarBatch = { + val codec = TableCompressionCodec.getCodec( + CodecType.NVCOMP_LZ4, TableCompressionCodec.makeCodecConfig(new RapidsConf(new SparkConf))) + withResource(codec.createBatchCompressor(0, Cuda.DEFAULT_STREAM)) { compressor => + compressor.addTableToCompress(buildContiguousTable(start, numRows)) + withResource(compressor.finish()) { compressed => + GpuCompressedColumnVector.from(compressed.head) + } + } + } + + private def decompressBatch(cb: ColumnarBatch): ColumnarBatch = { + val schema = new StructType().add("i", LongType) + .add("j", DecimalType(ai.rapids.cudf.DType.DECIMAL64_MAX_PRECISION, 3)) + val sparkTypes = GpuColumnVector.extractTypes(schema) + val codec = TableCompressionCodec.getCodec( + CodecType.NVCOMP_LZ4, TableCompressionCodec.makeCodecConfig(new RapidsConf(new SparkConf))) + withResource(codec.createBatchDecompressor(0, Cuda.DEFAULT_STREAM)) { decompressor => + val gcv = cb.column(0).asInstanceOf[GpuCompressedColumnVector] + // we need to incRefCount since the decompressor closes its inputs + gcv.getTableBuffer.incRefCount() + decompressor.addBufferToDecompress(gcv.getTableBuffer, gcv.getTableMeta.bufferMeta()) + withResource(decompressor.finishAsync()) { decompressed => + MetaUtils.getBatchFromMeta( + decompressed.head, + MetaUtils.dropCodecs(gcv.getTableMeta), + sparkTypes) + } + } + } + + test("a compressed batch can be added and recovered") { + val ct = buildCompressedBatch(0, 1000) + withResource(SpillableCompressedColumnarBatchHandle(ct)) { handle => + assert(handle.spillable) + withResource(handle.materialize()) { materialized => + assert(!handle.spillable) + // since we didn't spill, these buffers are exactly the same + assert( + ct.column(0).asInstanceOf[GpuCompressedColumnVector].getTableBuffer == + materialized.column(0).asInstanceOf[GpuCompressedColumnVector].getTableBuffer) + } + assert(handle.spillable) + } + } + + test("a compressed batch can be added and recovered after being spilled to host") { + val ct = buildCompressedBatch(0, 1000) + withResource(decompressBatch(ct)) { decompressedExpected => + withResource(SpillableCompressedColumnarBatchHandle(ct)) { handle => + assert(handle.spillable) + SpillFramework.stores.deviceStore.spill(handle.approxSizeInBytes) + assert(!handle.spillable) + assert(handle.dev.isEmpty) + assert(handle.host.isDefined) + withResource(handle.materialize()) { materialized => + withResource(decompressBatch(materialized)) { decompressed => + TestUtils.compareBatches(decompressedExpected, decompressed) + } + } + } + } + } + + test("a compressed batch can be added and recovered after being spilled to disk") { + val ct = buildCompressedBatch(0, 1000) + withResource(decompressBatch(ct)) { decompressedExpected => + withResource(SpillableCompressedColumnarBatchHandle(ct)) { handle => + assert(handle.spillable) + SpillFramework.stores.deviceStore.spill(handle.approxSizeInBytes) + assert(!handle.spillable) + SpillFramework.stores.hostStore.spill(handle.approxSizeInBytes) + assert(handle.dev.isEmpty) + assert(handle.host.isDefined) + assert(handle.host.get.host.isEmpty) + assert(handle.host.get.disk.isDefined) + withResource(handle.materialize()) { materialized => + withResource(decompressBatch(materialized)) { decompressed => + TestUtils.compareBatches(decompressedExpected, decompressed) + } + } + } + } + } + + + test("a second handle prevents buffer to be spilled") { + val buffer = DeviceMemoryBuffer.allocate(123) + val handle1 = SpillableDeviceBufferHandle(buffer) + // materialize will incRefCount `buffer`. This looks a little weird + // but it simulates aliasing as it happens in real code + val handle2 = SpillableDeviceBufferHandle(handle1.materialize()) + + withResource(handle1) { _ => + withResource(handle2) { _ => + assertResult(2)(handle1.dev.get.getRefCount) + assertResult(2)(handle2.dev.get.getRefCount) + assertResult(false)(handle1.spillable) + assertResult(false)(handle2.spillable) + } + assertResult(1)(handle1.dev.get.getRefCount) + assertResult(true)(handle1.spillable) + } + } + + test("removing handle releases buffer resources in all stores") { + val handle = SpillableDeviceBufferHandle(DeviceMemoryBuffer.allocate(123)) + withResource(handle) { _ => + assertResult(1)(SpillFramework.stores.deviceStore.numHandles) + assertResult(0)(SpillFramework.stores.hostStore.numHandles) + assertResult(0)(SpillFramework.stores.diskStore.numHandles) + + assertResult(123)(SpillFramework.stores.deviceStore.spill(123)) // spill to host memory + assertResult(0)(SpillFramework.stores.deviceStore.numHandles) + assertResult(1)(SpillFramework.stores.hostStore.numHandles) + assertResult(0)(SpillFramework.stores.diskStore.numHandles) + assert(handle.dev.isEmpty) + assert(handle.host.isDefined) + assert(handle.host.get.host.isDefined) + + assertResult(123)(SpillFramework.stores.hostStore.spill(123)) // spill to disk + assertResult(0)(SpillFramework.stores.deviceStore.numHandles) + assertResult(0)(SpillFramework.stores.hostStore.numHandles) + assertResult(1)(SpillFramework.stores.diskStore.numHandles) + assert(handle.dev.isEmpty) + assert(handle.host.isDefined) + assert(handle.host.get.host.isEmpty) + assert(handle.host.get.disk.isDefined) + } + assert(handle.host.isEmpty) + assert(handle.dev.isEmpty) + assertResult(0)(SpillFramework.stores.deviceStore.numHandles) + assertResult(0)(SpillFramework.stores.hostStore.numHandles) + assertResult(0)(SpillFramework.stores.diskStore.numHandles) + } + + test("spill updates store state") { + val diskStore = SpillFramework.stores.diskStore + val hostStore = SpillFramework.stores.hostStore + val deviceStore = SpillFramework.stores.deviceStore + + val (bufferSize, handle, _) = + addContiguousTableToFramework() + + withResource(handle) { _ => + assertResult(1)(deviceStore.numHandles) + assertResult(0)(diskStore.numHandles) + assertResult(0)(hostStore.numHandles) + + assertResult(bufferSize)(SpillFramework.stores.deviceStore.spill(bufferSize)) + assertResult(bufferSize)(SpillFramework.stores.hostStore.spill(bufferSize)) + + assertResult(0)(deviceStore.numHandles) + assertResult(0)(hostStore.numHandles) + assertResult(1)(diskStore.numHandles) + + val diskHandle = handle.host.flatMap(_.disk).get + val path = diskStore.getFile(diskHandle.blockId) + assert(path.exists) + } + } + + test("get columnar batch after host spill") { + val (ct, dataTypes) = buildContiguousTable() + val expectedBatch = GpuColumnVector.from(ct.getTable, dataTypes) + withResource(SpillableColumnarBatchFromBufferHandle( + ct, dataTypes)) { handle => + withResource(expectedBatch) { _ => + SpillFramework.stores.deviceStore.spill(handle.approxSizeInBytes) + withResource(handle.materialize(dataTypes)) { cb => + TestUtils.compareBatches(expectedBatch, cb) + } + } + } + } + + test("get memory buffer after host spill") { + val (ct, dataTypes) = buildContiguousTable() + val expectedBatch = closeOnExcept(ct) { _ => + // make a copy of the table so we can compare it later to the + // one reconstituted after the spill + withResource(ct.getTable.contiguousSplit()) { copied => + GpuColumnVector.from(copied(0).getTable, dataTypes) + } + } + val handle = SpillableColumnarBatchFromBufferHandle(ct, dataTypes) + withResource(handle) { _ => + withResource(expectedBatch) { _ => + assertResult(SpillFramework.stores.deviceStore.spill(handle.approxSizeInBytes))( + handle.approxSizeInBytes) + val hostSize = handle.host.get.approxSizeInBytes + assertResult(SpillFramework.stores.hostStore.spill(hostSize))(hostSize) + withResource(handle.materialize(dataTypes)) { actualBatch => + TestUtils.compareBatches(expectedBatch, actualBatch) + } + } + } + } + + test("host originated: get host memory buffer") { + val spillPriority = -10 + val hmb = HostMemoryBuffer.allocate(1L * 1024) + val spillableBuffer = SpillableHostBuffer(hmb, hmb.getLength, spillPriority) + withResource(spillableBuffer) { _ => + // the refcount of 1 is the store + assertResult(1)(hmb.getRefCount) + withResource(spillableBuffer.getHostBuffer()) { memoryBuffer => + assertResult(hmb)(memoryBuffer) + assertResult(2)(memoryBuffer.getRefCount) + } + } + assertResult(0)(hmb.getRefCount) + } + + test("host originated: get host memory buffer after spill to disk") { + val spillPriority = -10 + val hmb = HostMemoryBuffer.allocate(1L * 1024) + val spillableBuffer = SpillableHostBuffer( + hmb, + hmb.getLength, + spillPriority) + assertResult(1)(hmb.getRefCount) + // we spill it + SpillFramework.stores.hostStore.spill(hmb.getLength) + withResource(spillableBuffer) { _ => + // the refcount of the original buffer is 0 because it spilled + assertResult(0)(hmb.getRefCount) + withResource(spillableBuffer.getHostBuffer()) { memoryBuffer => + assertResult(memoryBuffer.getLength)(hmb.getLength) + } + } + } + + test("host originated: a buffer is not spillable when we leak it") { + val spillPriority = -10 + val hmb = HostMemoryBuffer.allocate(1L * 1024) + withResource(SpillableHostBuffer(hmb, hmb.getLength, spillPriority)) { spillableBuffer => + withResource(spillableBuffer.getHostBuffer()) { _ => + assertResult(0)(SpillFramework.stores.hostStore.spill(hmb.getLength)) + } + assertResult(hmb.getLength)(SpillFramework.stores.hostStore.spill(hmb.getLength)) + } + } + + test("host originated: a host batch is not spillable when we leak it") { + val (hostCb, sparkTypes) = buildHostBatch() + val sizeOnHost = RapidsHostColumnVector.getTotalHostMemoryUsed(hostCb) + withResource(SpillableHostColumnarBatchHandle(hostCb)) { handle => + assertResult(true)(handle.spillable) + + withResource(handle.materialize(sparkTypes)) { _ => + // 0 because we have a reference to the host batch + assertResult(false)(handle.spillable) + assertResult(0)(SpillFramework.stores.hostStore.spill(sizeOnHost)) + } + + // after closing we still have 0 bytes in the store or available to spill + assertResult(true)(handle.spillable) + } + } + + test("host originated: a host batch is not spillable when columns are incRefCounted") { + val (hostCb, sparkTypes) = buildHostBatch() + val sizeOnHost = RapidsHostColumnVector.getTotalHostMemoryUsed(hostCb) + withResource(SpillableHostColumnarBatchHandle(hostCb)) { handle => + assertResult(true)(handle.spillable) + val leakedFirstColumn = withResource(handle.materialize(sparkTypes)) { cb => + // 0 because we have a reference to the host batch + assertResult(false)(handle.spillable) + assertResult(0)(SpillFramework.stores.hostStore.spill(sizeOnHost)) + // leak it by increasing the ref count of the underlying cuDF column + RapidsHostColumnVector.extractBases(cb).head.incRefCount() + } + withResource(leakedFirstColumn) { _ => + // 0 because we have a reference to the first column + assertResult(false)(handle.spillable) + assertResult(0)(SpillFramework.stores.hostStore.spill(sizeOnHost)) + } + // batch is now spillable because we close our reference to the column + assertResult(true)(handle.spillable) + assertResult(sizeOnHost)(SpillFramework.stores.hostStore.spill(sizeOnHost)) + } + } + + test("host originated: an aliased host batch is not spillable (until closing the original) ") { + val (hostBatch, sparkTypes) = buildHostBatch() + val handle = SpillableHostColumnarBatchHandle(hostBatch) + withResource(handle) { _ => + assertResult(1)(SpillFramework.stores.hostStore.numHandles) + assertResult(true)(handle.spillable) + withResource(handle.materialize(sparkTypes)) { _ => + assertResult(false)(handle.spillable) + } // we now have two copies in the store + assertResult(true)(handle.spillable) + } + } + + test("host originated: a host batch supports duplicated columns") { + val (hostBatch, sparkTypes) = buildHostBatchWithDuplicate() + val handle = SpillableHostColumnarBatchHandle(hostBatch) + withResource(handle) { _ => + assertResult(1)(SpillFramework.stores.hostStore.numHandles) + assertResult(true)(handle.spillable) + withResource(handle.materialize(sparkTypes)) { _ => + assertResult(false)(handle.spillable) + } // we now have two copies in the store + assertResult(true)(handle.spillable) + } + } + + test("host originated: a host batch supports aliasing and duplicated columns") { + SpillFramework.shutdown() + val sc = new SparkConf + // disables the host store limit by enabling off heap limits + sc.set(RapidsConf.OFF_HEAP_LIMIT_ENABLED.key, "true") + SpillFramework.initialize(new RapidsConf(sc)) + + try { + val (hostBatch, sparkTypes) = buildHostBatchWithDuplicate() + withResource(SpillableHostColumnarBatchHandle(hostBatch)) { handle => + withResource(SpillableHostColumnarBatchHandle(handle.materialize(sparkTypes))) { handle2 => + assertResult(2)(SpillFramework.stores.hostStore.numHandles) + assertResult(false)(handle.spillable) + assertResult(false)(handle2.spillable) + } + assertResult(true)(handle.spillable) + } + } finally { + SpillFramework.shutdown() + } + } + + // this is a key behavior that we wanted to keep during the spill refactor + // where host objects that are added directly to the store do not cause a + // host->disk spill on their own, instead they will get spilled later + // due to device->host spills. + test("host factory methods do not spill on addition") { + SpillFramework.shutdown() + val sc = new SparkConf + // set a very small store size + sc.set(RapidsConf.HOST_SPILL_STORAGE_SIZE.key, "1KB") + SpillFramework.initialize(new RapidsConf(sc)) + + try { + // add a lot of batches, surpassing the limits of the store + val handles = new ArrayBuffer[SpillableHostColumnarBatchHandle]() + var dataTypes: Array[DataType] = null + (0 until 100).foreach { _ => + val (hostBatch, dt) = buildHostBatch() + if (dataTypes == null) { + dataTypes = dt + } + handles.append(SpillableHostColumnarBatchHandle(hostBatch)) + } + // no spill to disk + assertResult(100)(SpillFramework.stores.hostStore.numHandles) + + val dmb = DeviceMemoryBuffer.allocate(1024) + withResource(SpillableDeviceBufferHandle(dmb)) { _ => + // simulate an OOM by spilling device memory + SpillFramework.stores.deviceStore.spill(1024) + assertResult(0)(SpillFramework.stores.deviceStore.numHandles) + } + + val buffersSpilledToDisk = SpillFramework.stores.diskStore.numHandles + // we spilled to disk + assert(SpillFramework.stores.diskStore.numHandles > 0) + // and the remaining objects that didn't spill, are still in the host store + assertResult(100 - buffersSpilledToDisk)(SpillFramework.stores.hostStore.numHandles) + assert(SpillFramework.stores.hostStore.totalSize <= 1024) + } finally { + SpillFramework.shutdown() + } + } + + test("direct spill to disk: when buffer exceeds maximum size") { + var (bigTable, sparkTypes) = buildTableOfLongs(2 * 1024 * 1024) + closeOnExcept(bigTable) { _ => + // make a copy of the table so we can compare it later to the + // one reconstituted after the spill + val expectedBatch = + withResource(bigTable.getTable.contiguousSplit()) { expectedTable => + GpuColumnVector.from(expectedTable(0).getTable, sparkTypes) + } + withResource(expectedBatch) { _ => + withResource(SpillableColumnarBatchFromBufferHandle( + bigTable, sparkTypes)) { bigHandle => + bigTable = null + withResource(bigHandle.materialize(sparkTypes)) { actualBatch => + TestUtils.compareBatches(expectedBatch, actualBatch) + } + SpillFramework.stores.deviceStore.spill(bigHandle.approxSizeInBytes) + assertResult(true)(bigHandle.dev.isEmpty) + assertResult(true)(bigHandle.host.get.host.isEmpty) + assertResult(false)(bigHandle.host.get.disk.isEmpty) + + withResource(bigHandle.materialize(sparkTypes)) { actualBatch => + TestUtils.compareBatches(expectedBatch, actualBatch) + } + } + } + } + } + + test("get columnar batch after spilling to disk") { + val (size, handle, dataTypes) = addContiguousTableToFramework() + val diskStore = SpillFramework.stores.diskStore + val hostStore = SpillFramework.stores.hostStore + val deviceStore = SpillFramework.stores.deviceStore + withResource(handle) { _ => + assertResult(1)(deviceStore.numHandles) + assertResult(0)(diskStore.numHandles) + assertResult(0)(hostStore.numHandles) + + val expectedTable = + withResource(handle.materialize(dataTypes)) { beforeSpill => + withResource(GpuColumnVector.from(beforeSpill)) { table => + table.contiguousSplit()(0) + } + } // closing the batch from the store so that we can spill it + + withResource(expectedTable) { _ => + withResource( + GpuColumnVector.from(expectedTable.getTable, dataTypes)) { expectedBatch => + deviceStore.spill(size) + hostStore.spill(size) + + assertResult(0)(deviceStore.numHandles) + assertResult(0)(hostStore.numHandles) + assertResult(1)(diskStore.numHandles) + + val diskHandle = handle.host.flatMap(_.disk).get + val path = diskStore.getFile(diskHandle.blockId) + assert(path.exists) + withResource(handle.materialize(dataTypes)) { actualBatch => + TestUtils.compareBatches(expectedBatch, actualBatch) + } + } + } + } + } + + // -1 disables the host store limit + val hostSpillStorageSizes = Seq("-1", "1MB", "16MB") + val spillToDiskBounceBuffers = Seq("128KB", "2MB", "128MB") + val chunkedPackBounceBuffers = Seq("1MB", "8MB", "128MB") + hostSpillStorageSizes.foreach { hostSpillStorageSize => + spillToDiskBounceBuffers.foreach { spillToDiskBounceBufferSize => + chunkedPackBounceBuffers.foreach { chunkedPackBounceBufferSize => + test("materialize non-contiguous batch after " + + s"host_storage_size=$hostSpillStorageSize " + + s"spilling chunked_pack_bb=$chunkedPackBounceBufferSize " + + s"spill_to_disk_bb=$spillToDiskBounceBufferSize") { + SpillFramework.shutdown() + try { + val sc = new SparkConf + sc.set(RapidsConf.HOST_SPILL_STORAGE_SIZE.key, hostSpillStorageSize) + sc.set(RapidsConf.CHUNKED_PACK_BOUNCE_BUFFER_SIZE.key, chunkedPackBounceBufferSize) + sc.set(RapidsConf.SPILL_TO_DISK_BOUNCE_BUFFER_SIZE.key, spillToDiskBounceBufferSize) + SpillFramework.initialize(new RapidsConf(sc)) + val (largeTable, dataTypes) = buildNonContiguousTableOfLongs(numRows = 1000000) + val handle = SpillableColumnarBatchHandle(largeTable, dataTypes) + val diskStore = SpillFramework.stores.diskStore + val hostStore = SpillFramework.stores.hostStore + val deviceStore = SpillFramework.stores.deviceStore + withResource(handle) { _ => + assertResult(1)(deviceStore.numHandles) + assertResult(0)(diskStore.numHandles) + assertResult(0)(hostStore.numHandles) + + val expectedTable = + withResource(handle.materialize(dataTypes)) { beforeSpill => + withResource(GpuColumnVector.from(beforeSpill)) { table => + table.contiguousSplit()(0) + } + } // closing the batch from the store so that we can spill it + + withResource(expectedTable) { _ => + withResource( + GpuColumnVector.from(expectedTable.getTable, dataTypes)) { expectedBatch => + deviceStore.spill(handle.approxSizeInBytes) + hostStore.spill(handle.approxSizeInBytes) + + assertResult(0)(deviceStore.numHandles) + assertResult(0)(hostStore.numHandles) + assertResult(1)(diskStore.numHandles) + + val diskHandle = handle.host.flatMap(_.disk).get + val path = diskStore.getFile(diskHandle.blockId) + assert(path.exists) + withResource(handle.materialize(dataTypes)) { actualBatch => + TestUtils.compareBatches(expectedBatch, actualBatch) + } + } + } + } + } finally { + SpillFramework.shutdown() + } + } + } + } + } + + test("get memory buffer after spilling to disk") { + val handle = SpillableDeviceBufferHandle(DeviceMemoryBuffer.allocate(123)) + val diskStore = SpillFramework.stores.diskStore + val hostStore = SpillFramework.stores.hostStore + val deviceStore = SpillFramework.stores.deviceStore + withResource(handle) { _ => + assertResult(1)(deviceStore.numHandles) + assertResult(0)(diskStore.numHandles) + assertResult(0)(hostStore.numHandles) + val expectedBuffer = + withResource(handle.materialize()) { devbuf => + closeOnExcept(HostMemoryBuffer.allocate(devbuf.getLength)) { hostbuf => + hostbuf.copyFromDeviceBuffer(devbuf) + hostbuf + } + } + withResource(expectedBuffer) { expectedBuffer => + deviceStore.spill(handle.approxSizeInBytes) + hostStore.spill(handle.approxSizeInBytes) + withResource(handle.host.map(_.materialize()).get) { actualHostBuffer => + assertResult(expectedBuffer. + asByteBuffer.limit())(actualHostBuffer.asByteBuffer.limit()) + } + } + } + } + + test("Compression on with or without encryption for spill block using single batch") { + Seq("true", "false").foreach { encryptionEnabled => + val conf = new SparkConf() + conf.set(RapidsConf.TEST_IO_ENCRYPTION.key, encryptionEnabled) + conf.set("spark.io.compression.codec", "zstd") + conf.set("spark.shuffle.spill.compress", "true") + conf.set("spark.shuffle.compress", "true") + readWriteTestWithBatches(conf, false) + } + } + + test("Compression off with or without encryption for spill block using single batch") { + Seq("true", "false").foreach { encryptionEnabled => + val conf = new SparkConf() + conf.set(RapidsConf.TEST_IO_ENCRYPTION.key, encryptionEnabled) + conf.set("spark.shuffle.spill.compress", "false") + conf.set("spark.shuffle.compress", "false") + readWriteTestWithBatches(conf, false) + } + } + + test("Compression on with or without encryption for spill block using multiple batches") { + Seq("true", "false").foreach { encryptionEnabled => + val conf = new SparkConf() + conf.set(RapidsConf.TEST_IO_ENCRYPTION.key, encryptionEnabled) + conf.set("spark.io.compression.codec", "zstd") + conf.set("spark.shuffle.spill.compress", "true") + conf.set("spark.shuffle.compress", "true") + readWriteTestWithBatches(conf, false) + } + } + + test("Compression off with or without encryption for spill block using multiple batches") { + Seq("true", "false").foreach { encryptionEnabled => + val conf = new SparkConf() + conf.set(RapidsConf.TEST_IO_ENCRYPTION.key, encryptionEnabled) + conf.set("spark.shuffle.spill.compress", "false") + conf.set("spark.shuffle.compress", "false") + readWriteTestWithBatches(conf, false) + } + } + + // ===== Tests for shuffle block ===== + + test("Compression on with or without encryption for shuffle block using single batch") { + Seq("true", "false").foreach { encryptionEnabled => + val conf = new SparkConf() + conf.set(RapidsConf.TEST_IO_ENCRYPTION.key, encryptionEnabled) + conf.set("spark.io.compression.codec", "zstd") + conf.set("spark.shuffle.spill.compress", "true") + conf.set("spark.shuffle.compress", "true") + readWriteTestWithBatches(conf, true) + } + } + + test("Compression off with or without encryption for shuffle block using single batch") { + Seq("true", "false").foreach { encryptionEnabled => + val conf = new SparkConf() + conf.set(RapidsConf.TEST_IO_ENCRYPTION.key, encryptionEnabled) + conf.set("spark.shuffle.spill.compress", "false") + conf.set("spark.shuffle.compress", "false") + readWriteTestWithBatches(conf, true) + } + } + + test("Compression on with or without encryption for shuffle block using multiple batches") { + Seq("true", "false").foreach { encryptionEnabled => + val conf = new SparkConf() + conf.set(RapidsConf.TEST_IO_ENCRYPTION.key, encryptionEnabled) + conf.set("spark.io.compression.codec", "zstd") + conf.set("spark.shuffle.spill.compress", "true") + conf.set("spark.shuffle.compress", "true") + readWriteTestWithBatches(conf, true, true) + } + } + + test("Compression off with or without encryption for shuffle block using multiple batches") { + Seq("true", "false").foreach { encryptionEnabled => + val conf = new SparkConf() + conf.set(RapidsConf.TEST_IO_ENCRYPTION.key, encryptionEnabled) + conf.set("spark.shuffle.spill.compress", "false") + conf.set("spark.shuffle.compress", "false") + readWriteTestWithBatches(conf, true, true) + } + } + + test("No encryption and compression for shuffle block using multiple batches") { + readWriteTestWithBatches(new SparkConf(), true, true) + } + + private def readWriteTestWithBatches(conf: SparkConf, shareDiskPaths: Boolean*) = { + assert(shareDiskPaths.nonEmpty) + val mockDiskBlockManager = mock[RapidsDiskBlockManager] + when(mockDiskBlockManager.getSerializerManager()) + .thenReturn(new RapidsSerializerManager(conf)) + + shareDiskPaths.foreach { _ => + val (_, handle, dataTypes) = addContiguousTableToFramework() + withResource(handle) { _ => + val expectedCt = withResource(handle.materialize(dataTypes)) { devbatch => + withResource(GpuColumnVector.from(devbatch)) { tmpTbl => + tmpTbl.contiguousSplit()(0) + } + } + withResource(expectedCt) { _ => + val expectedBatch = withResource(expectedCt.getTable) { expectedTbl => + GpuColumnVector.from(expectedTbl, dataTypes) + } + withResource(expectedBatch) { _ => + assertResult(true)( + SpillFramework.stores.deviceStore.spill(handle.approxSizeInBytes) > 0) + assertResult(true)( + SpillFramework.stores.hostStore.spill(handle.approxSizeInBytes) > 0) + withResource(handle.materialize(dataTypes)) { actualBatch => + TestUtils.compareBatches(expectedBatch, actualBatch) + } + } + } + } + } + } + + test("skip host: spill device memory buffer to disk") { + SpillFramework.shutdown() + try { + val sc = new SparkConf + // disables the host store limit + sc.set(RapidsConf.HOST_SPILL_STORAGE_SIZE.key, "1KB") + SpillFramework.initialize(new RapidsConf(sc)) + // buffer is too big for host store limit, so we will skip host + val handle = SpillableDeviceBufferHandle(DeviceMemoryBuffer.allocate(1025)) + val deviceStore = SpillFramework.stores.deviceStore + withResource(handle) { _ => + val expectedBuffer = + withResource(handle.materialize()) { devbuf => + closeOnExcept(HostMemoryBuffer.allocate(devbuf.getLength)) { hostbuf => + hostbuf.copyFromDeviceBuffer(devbuf) + hostbuf + } + } + + withResource(expectedBuffer) { _ => + // host store will fail to spill + deviceStore.spill(handle.approxSizeInBytes) + assert(handle.host.map(_.host.isEmpty).get) + assert(handle.host.map(_.disk.isDefined).get) + withResource(handle.host.map(_.materialize()).get) { buffer => + assertResult(expectedBuffer.asByteBuffer)(buffer.asByteBuffer) + } + } + } + } finally { + SpillFramework.shutdown() + } + } + + test("skip host: spill table to disk") { + SpillFramework.shutdown() + try { + val sc = new SparkConf + sc.set(RapidsConf.HOST_SPILL_STORAGE_SIZE.key, "1KB") + SpillFramework.initialize(new RapidsConf(sc)) + // fill up the host store + withResource(SpillableHostBufferHandle(HostMemoryBuffer.allocate(1024))) { hostHandle => + // make sure the host handle isn't spillable + withResource(hostHandle.materialize()) { _ => + val (handle, _) = addTableToFramework() + withResource(handle) { _ => + val (expectedTable, dataTypes) = buildTable() + withResource(expectedTable) { _ => + withResource( + GpuColumnVector.from(expectedTable, dataTypes)) { expectedBatch => + SpillFramework.stores.deviceStore.spill(handle.approxSizeInBytes) + assert(handle.host.map(_.host.isEmpty).get) + assert(handle.host.map(_.disk.isDefined).get) + withResource(handle.materialize(dataTypes)) { fromDiskBatch => + TestUtils.compareBatches(expectedBatch, fromDiskBatch) + assert(handle.dev.isEmpty) + assert(handle.host.map(_.host.isEmpty).get) + assert(handle.host.map(_.disk.isDefined).get) + } + } + } + } + } + } + } finally { + SpillFramework.shutdown() + } + } + + test("skip host: spill table to disk with small host bounce buffer") { + try { + SpillFramework.shutdown() + val sc = new SparkConf + // make this super small so we skip the host + sc.set(RapidsConf.HOST_SPILL_STORAGE_SIZE.key, "1") + sc.set(RapidsConf.SPILL_TO_DISK_BOUNCE_BUFFER_SIZE.key, "10") + sc.set(RapidsConf.CHUNKED_PACK_BOUNCE_BUFFER_SIZE.key, "1MB") + val rapidsConf = new RapidsConf(sc) + SpillFramework.initialize(rapidsConf) + val (handle, _) = addTableToFramework() + withResource(handle) { _ => + val (expectedTable, dataTypes) = buildTable() + withResource(expectedTable) { _ => + withResource( + GpuColumnVector.from(expectedTable, dataTypes)) { expectedBatch => + SpillFramework.stores.deviceStore.spill(handle.approxSizeInBytes) + assert(handle.dev.isEmpty) + assert(handle.host.map(_.host.isEmpty).get) + assert(handle.host.map(_.disk.isDefined).get) + withResource(handle.materialize(dataTypes)) { fromDiskBatch => + TestUtils.compareBatches(expectedBatch, fromDiskBatch) + } + } + } + } + } finally { + SpillFramework.shutdown() + } + } + + test("0-byte table is never spillable") { + val (handle, _) = addZeroRowsTableToFramework() + val (handle2, _) = addTableToFramework() + + withResource(handle) { _ => + withResource(handle2) { _ => + assert(handle2.host.isEmpty) + val (expectedTable, expectedTypes) = buildTable() + withResource(expectedTable) { _ => + withResource( + GpuColumnVector.from(expectedTable, expectedTypes)) { expectedCb => + SpillFramework.stores.deviceStore.spill( + handle.approxSizeInBytes + handle2.approxSizeInBytes) + SpillFramework.stores.hostStore.spill( + handle.approxSizeInBytes + handle2.approxSizeInBytes) + // the 0-byte table never moved from device. It is not spillable + assert(handle.host.isEmpty) + assert(!handle.spillable) + // the second table (with rows) did spill + assert(handle2.host.isDefined) + assert(handle2.host.map(_.host.isEmpty).get) + assert(handle2.host.map(_.disk.isDefined).get) + + withResource(handle2.materialize(expectedTypes)) { spilledBatch => + TestUtils.compareBatches(expectedCb, spilledBatch) + } + } + } + } + } + } + + test("exclusive spill files are deleted when buffer deleted") { + testBufferFileDeletion(canShareDiskPaths = false) + } + + test("shared spill files are not deleted when a buffer is deleted") { + testBufferFileDeletion(canShareDiskPaths = true) + } + +} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/timezone/TimeZonePerfSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/timezone/TimeZonePerfSuite.scala index a9618a448cf..26ac7a177a3 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/timezone/TimeZonePerfSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/timezone/TimeZonePerfSuite.scala @@ -65,11 +65,13 @@ class TimeZonePerfSuite extends SparkQueryCompareTestSuite with BeforeAndAfterAl * Create a Parquet file to test */ override def beforeAll(): Unit = { + super.beforeAll() withCpuSparkSession( spark => createDF(spark).write.mode("overwrite").parquet(path)) } override def afterAll(): Unit = { + super.afterAll() FileUtils.deleteRecursively(new File(path)) } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/timezone/TimeZoneSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/timezone/TimeZoneSuite.scala index dcfbc508034..b22f827916b 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/timezone/TimeZoneSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/timezone/TimeZoneSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -319,6 +319,7 @@ class TimeZoneSuite extends SparkQueryCompareTestSuite with BeforeAndAfterAll { } override def afterAll(): Unit = { + super.afterAll() if (useGPU) { GpuTimeZoneDB.shutdown() } diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriterSuite.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriterSuite.scala index d52c8b47ae7..033173468fd 100644 --- a/tests/src/test/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriterSuite.scala +++ b/tests/src/test/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriterSuite.scala @@ -15,10 +15,11 @@ */ package org.apache.spark.sql.rapids -import ai.rapids.cudf.TableWriter -import com.nvidia.spark.rapids.{ColumnarOutputWriter, ColumnarOutputWriterFactory, GpuColumnVector, GpuLiteral, RapidsBufferCatalog, RapidsDeviceMemoryStore, ScalableTaskCompletion} +import ai.rapids.cudf.{Rmm, RmmAllocationMode, TableWriter} +import com.nvidia.spark.rapids.{ColumnarOutputWriter, ColumnarOutputWriterFactory, GpuColumnVector, GpuLiteral, RapidsConf, ScalableTaskCompletion} import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.jni.{GpuRetryOOM, GpuSplitAndRetryOOM} +import com.nvidia.spark.rapids.spill.SpillFramework import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FSDataOutputStream import org.apache.hadoop.mapred.TaskAttemptContext @@ -28,6 +29,7 @@ import org.scalatest.BeforeAndAfterEach import org.scalatest.funsuite.AnyFunSuite import org.scalatestplus.mockito.MockitoSugar.mock +import org.apache.spark.SparkConf import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Ascending, AttributeReference, ExprId, SortOrder} @@ -42,7 +44,6 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { private var mockCommitter: FileCommitProtocol = _ private var mockOutputWriterFactory: ColumnarOutputWriterFactory = _ private var mockOutputWriter: NoTransformColumnarOutputWriter = _ - private var devStore: RapidsDeviceMemoryStore = _ private var allCols: Seq[AttributeReference] = _ private var partSpec: Seq[AttributeReference] = _ private var dataSpec: Seq[AttributeReference] = _ @@ -175,16 +176,13 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { } override def beforeEach(): Unit = { - devStore = new RapidsDeviceMemoryStore() - val catalog = new RapidsBufferCatalog(devStore) - RapidsBufferCatalog.setCatalog(catalog) + Rmm.initialize(RmmAllocationMode.CUDA_DEFAULT, null, 512 * 1024 * 1024) + SpillFramework.initialize(new RapidsConf(new SparkConf)) } override def afterEach(): Unit = { - // test that no buffers we left in the spill framework - assertResult(0)(RapidsBufferCatalog.numBuffers) - RapidsBufferCatalog.close() - devStore.close() + SpillFramework.shutdown() + Rmm.shutdown() } def buildEmptyBatch: ColumnarBatch = diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/SpillableColumnarBatchSuite.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/SpillableColumnarBatchSuite.scala index 001f82ab3a0..9b7a1dbdddd 100644 --- a/tests/src/test/scala/org/apache/spark/sql/rapids/SpillableColumnarBatchSuite.scala +++ b/tests/src/test/scala/org/apache/spark/sql/rapids/SpillableColumnarBatchSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,53 +16,30 @@ package org.apache.spark.sql.rapids -import java.util.UUID - -import ai.rapids.cudf.{Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer} -import com.nvidia.spark.rapids.{RapidsBuffer, RapidsBufferCatalog, RapidsBufferId, SpillableColumnarBatchImpl, StorageTier} -import com.nvidia.spark.rapids.StorageTier.StorageTier -import com.nvidia.spark.rapids.format.TableMeta +import ai.rapids.cudf.DeviceMemoryBuffer +import com.nvidia.spark.rapids.{RapidsConf, SpillableBuffer} +import com.nvidia.spark.rapids.spill.SpillFramework +import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite -import org.apache.spark.sql.types.{DataType, IntegerType} -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.storage.TempLocalBlockId +import org.apache.spark.SparkConf -class SpillableColumnarBatchSuite extends AnyFunSuite { +class SpillableColumnarBatchSuite extends AnyFunSuite with BeforeAndAfterAll { + override def beforeAll(): Unit = { + super.beforeAll() + SpillFramework.initialize(new RapidsConf(new SparkConf())) + } - test("close updates catalog") { - val id = TempSpillBufferId(0, TempLocalBlockId(new UUID(1, 2))) - val mockBuffer = new MockBuffer(id) - val catalog = RapidsBufferCatalog.singleton - val oldBufferCount = catalog.numBuffers - catalog.registerNewBuffer(mockBuffer) - val handle = catalog.makeNewHandle(id, -1) - assertResult(oldBufferCount + 1)(catalog.numBuffers) - val spillableBatch = new SpillableColumnarBatchImpl( - handle, - 5, - Array[DataType](IntegerType)) - spillableBatch.close() - assertResult(oldBufferCount)(catalog.numBuffers) + override def afterAll(): Unit = { + super.afterAll() + SpillFramework.shutdown() } - class MockBuffer(override val id: RapidsBufferId) extends RapidsBuffer { - override val memoryUsedBytes: Long = 123 - override def meta: TableMeta = null - override val storageTier: StorageTier = StorageTier.DEVICE - override def getMemoryBuffer: MemoryBuffer = null - override def copyToMemoryBuffer(srcOffset: Long, dst: MemoryBuffer, dstOffset: Long, - length: Long, stream: Cuda.Stream): Unit = {} - override def getDeviceMemoryBuffer: DeviceMemoryBuffer = null - override def getHostMemoryBuffer: HostMemoryBuffer = null - override def addReference(): Boolean = true - override def free(): Unit = {} - override def getSpillPriority: Long = 0 - override def setSpillPriority(priority: Long): Unit = {} - override def close(): Unit = {} - override def getColumnarBatch( - sparkTypes: Array[DataType]): ColumnarBatch = null - override def withMemoryBufferReadLock[K](body: MemoryBuffer => K): K = { body(null) } - override def withMemoryBufferWriteLock[K](body: MemoryBuffer => K): K = { body(null) } + test("close updates catalog") { + assertResult(0)(SpillFramework.stores.deviceStore.numHandles) + val deviceHandle = SpillableBuffer(DeviceMemoryBuffer.allocate(1234), -1) + assertResult(1)(SpillFramework.stores.deviceStore.numHandles) + deviceHandle.close() + assertResult(0)(SpillFramework.stores.deviceStore.numHandles) } } diff --git a/tests/src/test/spark320/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala b/tests/src/test/spark320/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala index ab303d8098e..525f30c7a87 100644 --- a/tests/src/test/spark320/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala +++ b/tests/src/test/spark320/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala @@ -36,10 +36,11 @@ package com.nvidia.spark.rapids.shuffle import java.nio.ByteBuffer import java.util.concurrent.Executor +import scala.collection.immutable import scala.collection.mutable.ArrayBuffer import ai.rapids.cudf.{ColumnVector, ContiguousTable, DeviceMemoryBuffer, HostMemoryBuffer} -import com.nvidia.spark.rapids.{GpuColumnVector, MetaUtils, RapidsBufferHandle, RapidsConf, RapidsDeviceMemoryStore, RmmSparkRetrySuiteBase, ShuffleMetadata, ShuffleReceivedBufferCatalog} +import com.nvidia.spark.rapids.{GpuColumnVector, MetaUtils, RapidsConf, RapidsShuffleHandle, RmmSparkRetrySuiteBase, ShuffleMetadata, ShuffleReceivedBufferCatalog} import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.format.TableMeta import org.mockito.ArgumentCaptor @@ -79,7 +80,6 @@ abstract class RapidsShuffleTestHelper var mockCopyExecutor: Executor = _ var mockBssExecutor: Executor = _ var mockHandler: RapidsShuffleFetchHandler = _ - var mockStorage: RapidsDeviceMemoryStore = _ var mockCatalog: ShuffleReceivedBufferCatalog = _ var mockConf: RapidsConf = _ var testMetricsUpdater: TestShuffleMetricsUpdater = _ @@ -160,11 +160,11 @@ abstract class RapidsShuffleTestHelper testMetricsUpdater = spy(new TestShuffleMetricsUpdater) val dmbCaptor = ArgumentCaptor.forClass(classOf[DeviceMemoryBuffer]) - when(mockCatalog.addBuffer(dmbCaptor.capture(), any(), any(), any())) + when(mockCatalog.addBuffer(dmbCaptor.capture(), any(), any())) .thenAnswer(_ => { val buffer = dmbCaptor.getValue.asInstanceOf[DeviceMemoryBuffer] buffersToClose.append(buffer) - mock[RapidsBufferHandle] + mock[RapidsShuffleHandle] }) client = spy(new RapidsShuffleClient( @@ -185,25 +185,30 @@ object RapidsShuffleTestHelper extends MockitoSugar { MetaUtils.buildDegenerateTableMeta(new ColumnarBatch(Array.empty, 123)) } - def withMockContiguousTable[T](numRows: Long)(body: ContiguousTable => T): T = { + def buildContiguousTable(numRows: Long): ContiguousTable = { val rows: Seq[Integer] = (0 until numRows.toInt).map(Int.box) withResource(ColumnVector.fromBoxedInts(rows:_*)) { cvBase => cvBase.incRefCount() val gpuCv = GpuColumnVector.from(cvBase, IntegerType) withResource(new ColumnarBatch(Array(gpuCv))) { cb => withResource(GpuColumnVector.from(cb)) { table => - withResource(table.contiguousSplit(0, numRows.toInt)) { ct => - body(ct(1)) // we get a degenerate table at 0 and another at 2 - } + val cts = table.contiguousSplit() + cts(0) } } } } + def withMockContiguousTable[T](numRows: Long)(body: ContiguousTable => T): T = { + withResource(buildContiguousTable(numRows)) { ct => + body(ct) + } + } + def mockMetaResponse( mockTransaction: Transaction, numRows: Long, - numBatches: Int): (Seq[TableMeta], MetadataTransportBuffer) = + numBatches: Int): (immutable.Seq[TableMeta], MetadataTransportBuffer) = withMockContiguousTable(numRows) { ct => val tableMetas = (0 until numBatches).map(b => buildMockTableMeta(b, ct)) val res = ShuffleMetadata.buildMetaResponse(tableMetas) @@ -214,7 +219,7 @@ object RapidsShuffleTestHelper extends MockitoSugar { def mockDegenerateMetaResponse( mockTransaction: Transaction, - numBatches: Int): (Seq[TableMeta], MetadataTransportBuffer) = { + numBatches: Int): (immutable.Seq[TableMeta], MetadataTransportBuffer) = { val tableMetas = (0 until numBatches).map(b => buildDegenerateMockTableMeta()) val res = ShuffleMetadata.buildMetaResponse(tableMetas) val refCountedRes = new MetadataTransportBuffer(new RefCountedDirectByteBuffer(res)) @@ -246,8 +251,8 @@ object RapidsShuffleTestHelper extends MockitoSugar { tableMeta } - def getShuffleBlocks: Seq[(ShuffleBlockBatchId, Long, Int)] = { - Seq( + def getShuffleBlocks: Array[(ShuffleBlockBatchId, Long, Int)] = { + Array( (ShuffleBlockBatchId(1,1,1,1), 123L, 1), (ShuffleBlockBatchId(2,2,2,2), 456L, 2), (ShuffleBlockBatchId(3,3,3,3), 456L, 3) @@ -261,11 +266,24 @@ object RapidsShuffleTestHelper extends MockitoSugar { bmId } - def getBlocksByAddress: Array[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { - val blocksByAddress = new ArrayBuffer[(BlockManagerId, Seq[(BlockId, Long, Int)])]() + def makeIterator(conf: RapidsConf, + transport: RapidsShuffleTransport, + testMetricsUpdater: TestShuffleMetricsUpdater, + taskId: Long, + catalog: ShuffleReceivedBufferCatalog): RapidsShuffleIterator = { + val blocksByAddress = new ArrayBuffer[(BlockManagerId, collection.Seq[(BlockId, Long, Int)])]() val blocks = getShuffleBlocks blocksByAddress.append((makeMockBlockManager("2", "2"), blocks)) - blocksByAddress.toArray + spy(new RapidsShuffleIterator( + RapidsShuffleTestHelper.makeMockBlockManager("1", "1"), + conf, + transport, + blocksByAddress.toArray, + testMetricsUpdater, + Array.empty, + taskId, + catalog, + 123)) } } @@ -289,4 +307,3 @@ class MockClientConnection(mockTransaction: Transaction) extends ClientConnectio override def registerReceiveHandler(messageType: MessageType.Value): Unit = {} } - diff --git a/tests/src/test/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala b/tests/src/test/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala index 89c317f7620..0efcb4f1d7d 100644 --- a/tests/src/test/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala +++ b/tests/src/test/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala @@ -37,7 +37,7 @@ import scala.collection import scala.collection.mutable.ArrayBuffer import ai.rapids.cudf.{ColumnVector, ContiguousTable, DeviceMemoryBuffer, HostMemoryBuffer} -import com.nvidia.spark.rapids.{GpuColumnVector, MetaUtils, RapidsBufferHandle, RapidsConf, RapidsDeviceMemoryStore, RmmSparkRetrySuiteBase, ShuffleMetadata, ShuffleReceivedBufferCatalog} +import com.nvidia.spark.rapids.{GpuColumnVector, MetaUtils, RapidsConf, RapidsShuffleHandle, RmmSparkRetrySuiteBase, ShuffleMetadata, ShuffleReceivedBufferCatalog} import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.format.TableMeta import org.mockito.ArgumentCaptor @@ -77,7 +77,6 @@ abstract class RapidsShuffleTestHelper var mockCopyExecutor: Executor = _ var mockBssExecutor: Executor = _ var mockHandler: RapidsShuffleFetchHandler = _ - var mockStorage: RapidsDeviceMemoryStore = _ var mockCatalog: ShuffleReceivedBufferCatalog = _ var mockConf: RapidsConf = _ var testMetricsUpdater: TestShuffleMetricsUpdater = _ @@ -158,11 +157,11 @@ abstract class RapidsShuffleTestHelper testMetricsUpdater = spy(new TestShuffleMetricsUpdater) val dmbCaptor = ArgumentCaptor.forClass(classOf[DeviceMemoryBuffer]) - when(mockCatalog.addBuffer(dmbCaptor.capture(), any(), any(), any())) + when(mockCatalog.addBuffer(dmbCaptor.capture(), any(), any())) .thenAnswer(_ => { val buffer = dmbCaptor.getValue.asInstanceOf[DeviceMemoryBuffer] buffersToClose.append(buffer) - mock[RapidsBufferHandle] + mock[RapidsShuffleHandle] }) client = spy(new RapidsShuffleClient( @@ -183,25 +182,30 @@ object RapidsShuffleTestHelper extends MockitoSugar { MetaUtils.buildDegenerateTableMeta(new ColumnarBatch(Array.empty, 123)) } - def withMockContiguousTable[T](numRows: Long)(body: ContiguousTable => T): T = { + def buildContiguousTable(numRows: Long): ContiguousTable = { val rows: Seq[Integer] = (0 until numRows.toInt).map(Int.box) withResource(ColumnVector.fromBoxedInts(rows:_*)) { cvBase => cvBase.incRefCount() val gpuCv = GpuColumnVector.from(cvBase, IntegerType) withResource(new ColumnarBatch(Array(gpuCv))) { cb => withResource(GpuColumnVector.from(cb)) { table => - withResource(table.contiguousSplit(0, numRows.toInt)) { ct => - body(ct(1)) // we get a degenerate table at 0 and another at 2 - } + val cts = table.contiguousSplit() + cts(0) } } } } + def withMockContiguousTable[T](numRows: Long)(body: ContiguousTable => T): T = { + withResource(buildContiguousTable(numRows)) { ct => + body(ct) + } + } + def mockMetaResponse( mockTransaction: Transaction, numRows: Long, - numBatches: Int): (Seq[TableMeta], MetadataTransportBuffer) = + numBatches: Int): (collection.Seq[TableMeta], MetadataTransportBuffer) = withMockContiguousTable(numRows) { ct => val tableMetas = (0 until numBatches).map(b => buildMockTableMeta(b, ct)) val res = ShuffleMetadata.buildMetaResponse(tableMetas) @@ -212,7 +216,7 @@ object RapidsShuffleTestHelper extends MockitoSugar { def mockDegenerateMetaResponse( mockTransaction: Transaction, - numBatches: Int): (Seq[TableMeta], MetadataTransportBuffer) = { + numBatches: Int): (collection.Seq[TableMeta], MetadataTransportBuffer) = { val tableMetas = (0 until numBatches).map(b => buildDegenerateMockTableMeta()) val res = ShuffleMetadata.buildMetaResponse(tableMetas) val refCountedRes = new MetadataTransportBuffer(new RefCountedDirectByteBuffer(res)) @@ -244,8 +248,8 @@ object RapidsShuffleTestHelper extends MockitoSugar { tableMeta } - def getShuffleBlocks: collection.Seq[(ShuffleBlockBatchId, Long, Int)] = { - collection.Seq( + def getShuffleBlocks: Array[(ShuffleBlockBatchId, Long, Int)] = { + Array( (ShuffleBlockBatchId(1,1,1,1), 123L, 1), (ShuffleBlockBatchId(2,2,2,2), 456L, 2), (ShuffleBlockBatchId(3,3,3,3), 456L, 3) @@ -259,11 +263,25 @@ object RapidsShuffleTestHelper extends MockitoSugar { bmId } - def getBlocksByAddress: Array[(BlockManagerId, collection.Seq[(BlockId, Long, Int)])] = { + def makeIterator( + conf: RapidsConf, + transport: RapidsShuffleTransport, + testMetricsUpdater: TestShuffleMetricsUpdater, + taskId: Long, + catalog: ShuffleReceivedBufferCatalog): RapidsShuffleIterator = { val blocksByAddress = new ArrayBuffer[(BlockManagerId, collection.Seq[(BlockId, Long, Int)])]() val blocks = getShuffleBlocks blocksByAddress.append((makeMockBlockManager("2", "2"), blocks)) - blocksByAddress.toArray + spy(new RapidsShuffleIterator( + RapidsShuffleTestHelper.makeMockBlockManager("1", "1"), + conf, + transport, + blocksByAddress.toArray, + testMetricsUpdater, + Array.empty, + taskId, + catalog, + 123)) } } @@ -287,4 +305,3 @@ class MockClientConnection(mockTransaction: Transaction) extends ClientConnectio override def registerReceiveHandler(messageType: MessageType.Value): Unit = {} } - From 0764c186c67304ac7d180437eca4e7d7c67ebb7d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 13 Dec 2024 17:18:04 -0800 Subject: [PATCH 08/47] Add integration test for async writer (#11855) Signed-off-by: Jihoon Son --- .../src/main/python/parquet_write_test.py | 21 +++++++++++++++++++ .../spark/rapids/ColumnarOutputWriter.scala | 13 +++++++----- .../spark/rapids/GpuParquetFileFormat.scala | 8 ++++--- .../rapids/io/async/TrafficController.scala | 6 +++--- 4 files changed, 37 insertions(+), 11 deletions(-) diff --git a/integration_tests/src/main/python/parquet_write_test.py b/integration_tests/src/main/python/parquet_write_test.py index 775b4a9d1cb..1d395d0e29a 100644 --- a/integration_tests/src/main/python/parquet_write_test.py +++ b/integration_tests/src/main/python/parquet_write_test.py @@ -676,6 +676,27 @@ def test_write_daytime_interval(spark_tmp_path): data_path, conf=writer_confs) + +hold_gpu_configs = [True, False] +@pytest.mark.parametrize('hold_gpu', hold_gpu_configs, ids=idfn) +def test_async_writer(spark_tmp_path, hold_gpu): + data_path = spark_tmp_path + '/PARQUET_DATA' + num_rows = 2048 + num_cols = 10 + parquet_gen = [int_gen for _ in range(num_cols)] + gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gen)] + assert_gpu_and_cpu_writes_are_equal_collect( + lambda spark, path: gen_df(spark, gen_list, length=num_rows).coalesce(1).write.parquet(path), + lambda spark, path: spark.read.parquet(path), + data_path, + copy_and_update( + writer_confs, + {"spark.rapids.sql.asyncWrite.queryOutput.enabled": "true", + "spark.rapids.sql.batchSizeBytes": 4 * num_cols * 100, # 100 rows per batch + "spark.rapids.sql.queryOutput.holdGpuInTask": hold_gpu} + )) + + @ignore_order @pytest.mark.skipif(is_before_spark_320(), reason="is only supported in Spark 320+") def test_concurrent_writer(spark_tmp_path): diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala index df62683d346..8d89583d9df 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala @@ -73,13 +73,14 @@ abstract class ColumnarOutputWriter(context: TaskAttemptContext, dataSchema: StructType, rangeName: String, includeRetry: Boolean, - holdGpuBetweenBatches: Boolean = false) extends HostBufferConsumer with Logging { + holdGpuBetweenBatches: Boolean = false, + useAsyncWrite: Boolean = false) extends HostBufferConsumer with Logging { protected val tableWriter: TableWriter protected val conf: Configuration = context.getConfiguration - private val trafficController: Option[TrafficController] = TrafficController.getInstance + private val trafficController: TrafficController = TrafficController.getInstance private def openOutputStream(): OutputStream = { val hadoopPath = new Path(path) @@ -90,10 +91,12 @@ abstract class ColumnarOutputWriter(context: TaskAttemptContext, // This is implemented as a method to make it easier to subclass // ColumnarOutputWriter in the tests, and override this behavior. protected def getOutputStream: OutputStream = { - trafficController.map(controller => { + if (useAsyncWrite) { logWarning("Async output write enabled") - new AsyncOutputStream(() => openOutputStream(), controller) - }).getOrElse(openOutputStream()) + new AsyncOutputStream(() => openOutputStream(), trafficController) + } else { + openOutputStream() + } } protected val outputStream: OutputStream = getOutputStream diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala index 2b5f246e56a..e5aa52c727d 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala @@ -283,7 +283,7 @@ class GpuParquetFileFormat extends ColumnarFileFormat with Logging { context: TaskAttemptContext): ColumnarOutputWriter = { new GpuParquetWriter(path, dataSchema, compressionType, outputTimestampType.toString, dateTimeRebaseMode, timestampRebaseMode, context, parquetFieldIdWriteEnabled, - holdGpuBetweenBatches) + holdGpuBetweenBatches, asyncOutputWriteEnabled) } override def getFileExtension(context: TaskAttemptContext): String = { @@ -306,8 +306,10 @@ class GpuParquetWriter( timestampRebaseMode: DateTimeRebaseMode, context: TaskAttemptContext, parquetFieldIdEnabled: Boolean, - holdGpuBetweenBatches: Boolean) - extends ColumnarOutputWriter(context, dataSchema, "Parquet", true, holdGpuBetweenBatches) { + holdGpuBetweenBatches: Boolean, + useAsyncWrite: Boolean) + extends ColumnarOutputWriter(context, dataSchema, "Parquet", true, holdGpuBetweenBatches, + useAsyncWrite) { override def throwIfRebaseNeededInExceptionMode(batch: ColumnarBatch): Unit = { val cols = GpuColumnVector.extractBases(batch) cols.foreach { col => diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/TrafficController.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/TrafficController.scala index e69af5bf258..f417fcb57cb 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/TrafficController.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/TrafficController.scala @@ -142,14 +142,14 @@ object TrafficController { * This is called once per executor. */ def initialize(conf: RapidsConf): Unit = synchronized { - if (conf.isAsyncOutputWriteEnabled && instance == null) { + if (instance == null) { instance = new TrafficController( new HostMemoryThrottle(conf.asyncWriteMaxInFlightHostMemoryBytes)) } } - def getInstance: Option[TrafficController] = synchronized { - Option(instance) + def getInstance: TrafficController = synchronized { + instance } def shutdown(): Unit = synchronized { From ab3111af7ae2487e5c4419c82492f8f76ae4c77d Mon Sep 17 00:00:00 2001 From: ustcfy <96854327+ustcfy@users.noreply.github.com> Date: Sat, 14 Dec 2024 12:20:13 +0800 Subject: [PATCH 09/47] Add `HiveHash` support for nested types (#11660) * Add HiveHash support for nested types Signed-off-by: ustcfy * Primitive types do not enter the stack Signed-off-by: Yan Feng * Generate docs Signed-off-by: Yan Feng * Generate docs Signed-off-by: Yan Feng * Add fallback test case for exceeding nesting depth limit Signed-off-by: Yan Feng * Retrieve supportedDepth from JNI Signed-off-by: Yan Feng --------- Signed-off-by: ustcfy Signed-off-by: Yan Feng --- docs/supported_ops.md | 4 +-- .../main/python/datasourcev2_write_test.py | 34 +++++++++++++++++-- .../main/python/hive_parquet_write_test.py | 23 ++++++++++--- .../nvidia/spark/rapids/GpuOverrides.scala | 23 ++++++++++++- tools/generated_files/320/supportedExprs.csv | 2 +- tools/generated_files/321/supportedExprs.csv | 2 +- .../generated_files/321cdh/supportedExprs.csv | 2 +- tools/generated_files/322/supportedExprs.csv | 2 +- tools/generated_files/323/supportedExprs.csv | 2 +- tools/generated_files/324/supportedExprs.csv | 2 +- tools/generated_files/330/supportedExprs.csv | 2 +- .../generated_files/330cdh/supportedExprs.csv | 2 +- tools/generated_files/331/supportedExprs.csv | 2 +- tools/generated_files/332/supportedExprs.csv | 2 +- .../generated_files/332cdh/supportedExprs.csv | 2 +- tools/generated_files/333/supportedExprs.csv | 2 +- tools/generated_files/334/supportedExprs.csv | 2 +- tools/generated_files/340/supportedExprs.csv | 2 +- tools/generated_files/341/supportedExprs.csv | 2 +- tools/generated_files/342/supportedExprs.csv | 2 +- tools/generated_files/343/supportedExprs.csv | 2 +- tools/generated_files/344/supportedExprs.csv | 2 +- tools/generated_files/350/supportedExprs.csv | 2 +- tools/generated_files/351/supportedExprs.csv | 2 +- tools/generated_files/352/supportedExprs.csv | 2 +- tools/generated_files/353/supportedExprs.csv | 2 +- tools/generated_files/400/supportedExprs.csv | 2 +- tools/generated_files/supportedExprs.csv | 2 +- 28 files changed, 98 insertions(+), 34 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index acf7133af40..be52afd9799 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -8480,9 +8480,9 @@ are limited. S NS NS +PS
The nesting depth has a certain limit;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types DECIMAL, BINARY, CALENDAR, MAP, UDT, DAYTIME, YEARMONTH
NS -NS -NS +PS
The nesting depth has a certain limit;
UTC is only supported TZ for child TIMESTAMP;
unsupported child types DECIMAL, BINARY, CALENDAR, MAP, UDT, DAYTIME, YEARMONTH
NS NS NS diff --git a/integration_tests/src/main/python/datasourcev2_write_test.py b/integration_tests/src/main/python/datasourcev2_write_test.py index 4fffd10ab44..35860453b2d 100644 --- a/integration_tests/src/main/python/datasourcev2_write_test.py +++ b/integration_tests/src/main/python/datasourcev2_write_test.py @@ -15,10 +15,10 @@ import pytest from asserts import assert_gpu_fallback_collect, assert_equal_with_local_sort -from data_gen import gen_df, decimal_gens, non_utc_allow +from data_gen import gen_df, decimal_gens, non_utc_allow, StructGen, ArrayGen, string_gen from marks import * from spark_session import is_hive_available, is_spark_330_or_later, with_cpu_session, with_gpu_session -from hive_parquet_write_test import _hive_bucket_gens_sans_bools, _hive_array_gens, _hive_struct_gens +from hive_parquet_write_test import _hive_bucket_gens_sans_bools from hive_parquet_write_test import read_single_bucket _hive_write_conf = { @@ -77,7 +77,7 @@ def write_hive_table(spark, out_table): @pytest.mark.skipif(not (is_hive_available() and is_spark_330_or_later()), reason="Must have Hive on Spark 3.3+") @pytest.mark.parametrize('file_format', ['parquet', 'orc']) -@pytest.mark.parametrize('gen', decimal_gens + _hive_array_gens + _hive_struct_gens) +@pytest.mark.parametrize('gen', decimal_gens) def test_write_hive_bucketed_unsupported_types_fallback(spark_tmp_table_factory, file_format, gen): out_table = spark_tmp_table_factory.get() @@ -94,3 +94,31 @@ def create_hive_table(spark): "insert into {0} select * from {1}".format(out_table, input_table)), 'DataWritingCommandExec', _hive_write_conf) + +# The calculation of nesting depth here does not take basic types into account. +nested_gen_depth_9 = [StructGen([('depth_8', StructGen([('depth_7', StructGen([('depth_6', + StructGen([('depth_5', ArrayGen(StructGen([('depth_3', StructGen([('depth_2', + ArrayGen(StructGen([('depth_0', string_gen)]), max_length=5))]))]), max_length=5))]))]))]))])] + +@ignore_order +@allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,SortExec,WriteFilesExec') +@pytest.mark.skipif(not (is_hive_available() and is_spark_330_or_later()), + reason="Must have Hive on Spark 3.3+") +@pytest.mark.parametrize('file_format', ['parquet', 'orc']) +@pytest.mark.parametrize('gen', nested_gen_depth_9) +def test_write_hive_bucketed_nesting_depth_exceed_limit_fallback(spark_tmp_table_factory, file_format, gen): + out_table = spark_tmp_table_factory.get() + + def create_hive_table(spark): + spark.sql("create table {0} (a {1}) stored as {2} clustered by (a) into 3 buckets".format( + out_table, gen.data_type.simpleString(), file_format)) + data_table = spark_tmp_table_factory.get() + gen_df(spark, [('a', gen)], length=1).createOrReplaceTempView(data_table) + return data_table + + input_table = with_cpu_session(create_hive_table, _hive_write_conf) + assert_gpu_fallback_collect( + lambda spark: spark.sql( + "insert into {0} select * from {1}".format(out_table, input_table)), + 'DataWritingCommandExec', + _hive_write_conf) diff --git a/integration_tests/src/main/python/hive_parquet_write_test.py b/integration_tests/src/main/python/hive_parquet_write_test.py index 540db74a1ad..684b35047a0 100644 --- a/integration_tests/src/main/python/hive_parquet_write_test.py +++ b/integration_tests/src/main/python/hive_parquet_write_test.py @@ -25,12 +25,27 @@ # "GpuInsertIntoHiveTable" for Parquet write. _write_to_hive_conf = {"spark.sql.hive.convertMetastoreParquet": False} -_hive_bucket_gens_sans_bools = [ +_hive_bucket_basic_gens_sans_bools = [ byte_gen, short_gen, int_gen, long_gen, string_gen, float_gen, double_gen, DateGen(start=date(1590, 1, 1)), _restricted_timestamp()] -_hive_bucket_gens = [boolean_gen] + _hive_bucket_gens_sans_bools +_hive_bucket_basic_gens = [boolean_gen] + _hive_bucket_basic_gens_sans_bools -_hive_basic_gens = _hive_bucket_gens + [ +_hive_bucket_basic_struct_gen = StructGen( + [['c'+str(ind), c_gen] for ind, c_gen in enumerate(_hive_bucket_basic_gens_sans_bools)]) + +_hive_bucket_struct_gens = [ + _hive_bucket_basic_struct_gen, + StructGen([['child0', byte_gen], ['child1', _hive_bucket_basic_struct_gen]]), + StructGen([['child0', ArrayGen(short_gen)], ['child1', double_gen]])] + +_hive_bucket_array_gens = [ArrayGen(sub_gen) for sub_gen in _hive_bucket_basic_gens_sans_bools] + [ + ArrayGen(ArrayGen(short_gen, max_length=10), max_length=10), + ArrayGen(ArrayGen(string_gen, max_length=10), max_length=10), + ArrayGen(StructGen([['child0', byte_gen], ['child1', string_gen], ['child2', float_gen]]))] + +_hive_bucket_gens_sans_bools = _hive_bucket_basic_gens_sans_bools + _hive_bucket_struct_gens + _hive_bucket_array_gens + +_hive_basic_gens = _hive_bucket_basic_gens + [ DecimalGen(precision=19, scale=1, nullable=True), DecimalGen(precision=23, scale=5, nullable=True), DecimalGen(precision=36, scale=3, nullable=True)] @@ -197,7 +212,7 @@ def test_insert_hive_bucketed_table(spark_tmp_table_factory): num_rows = 2048 def gen_table(spark): - gen_list = [('_c' + str(i), gen) for i, gen in enumerate(_hive_bucket_gens)] + gen_list = [('_c' + str(i), gen) for i, gen in enumerate(_hive_bucket_gens_sans_bools)] types_sql_str = ','.join('{} {}'.format( name, gen.data_type.simpleString()) for name, gen in gen_list) col_names_str = ','.join(name for name, gen in gen_list) 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 07b2d022f67..bd9aad97772 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 @@ -25,6 +25,7 @@ import scala.util.control.NonFatal import ai.rapids.cudf.DType import com.nvidia.spark.rapids.RapidsConf.{SUPPRESS_PLANNING_FAILURE, TEST_CONF} import com.nvidia.spark.rapids.jni.GpuTimeZoneDB +import com.nvidia.spark.rapids.jni.Hash import com.nvidia.spark.rapids.lore.GpuLore import com.nvidia.spark.rapids.shims._ import com.nvidia.spark.rapids.window.{GpuDenseRank, GpuLag, GpuLead, GpuPercentRank, GpuRank, GpuRowNumber, GpuSpecialFrameBoundary, GpuWindowExecMeta, GpuWindowSpecDefinitionMeta} @@ -3320,8 +3321,28 @@ object GpuOverrides extends Logging { "hive hash operator", ExprChecks.projectOnly(TypeSig.INT, TypeSig.INT, repeatingParamCheck = Some(RepeatingParamCheck("input", - TypeSig.commonCudfTypes + TypeSig.NULL, TypeSig.all))), + (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.STRUCT + TypeSig.ARRAY).nested() + + TypeSig.psNote(TypeEnum.ARRAY, "The nesting depth has a certain limit") + + TypeSig.psNote(TypeEnum.STRUCT, "The nesting depth has a certain limit"), + TypeSig.all))), (a, conf, p, r) => new ExprMeta[HiveHash](a, conf, p, r) { + override def tagExprForGpu(): Unit = { + def getMaxStackDepth(inputType: DataType): Int = { + inputType match { + case at: ArrayType => 1 + getMaxStackDepth(at.elementType) + case st: StructType => + 1 + st.map(f => getMaxStackDepth(f.dataType)).max + case _ => 0 // primitive types + } + } + val maxDepth = a.children.map(c => getMaxStackDepth(c.dataType)).max + val supportedDepth = Hash.MAX_STACK_DEPTH + if (maxDepth > supportedDepth) { + willNotWorkOnGpu(s"the data type requires a stack size of $maxDepth, " + + s"which exceeds the GPU limit of $supportedDepth") + } + } + def convertToGpu(): GpuExpression = GpuHiveHash(childExprs.map(_.convertToGpu())) }), diff --git a/tools/generated_files/320/supportedExprs.csv b/tools/generated_files/320/supportedExprs.csv index e4a4db760b0..573367d1d70 100644 --- a/tools/generated_files/320/supportedExprs.csv +++ b/tools/generated_files/320/supportedExprs.csv @@ -258,7 +258,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/321/supportedExprs.csv b/tools/generated_files/321/supportedExprs.csv index e4a4db760b0..573367d1d70 100644 --- a/tools/generated_files/321/supportedExprs.csv +++ b/tools/generated_files/321/supportedExprs.csv @@ -258,7 +258,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/321cdh/supportedExprs.csv b/tools/generated_files/321cdh/supportedExprs.csv index e4a4db760b0..573367d1d70 100644 --- a/tools/generated_files/321cdh/supportedExprs.csv +++ b/tools/generated_files/321cdh/supportedExprs.csv @@ -258,7 +258,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/322/supportedExprs.csv b/tools/generated_files/322/supportedExprs.csv index e4a4db760b0..573367d1d70 100644 --- a/tools/generated_files/322/supportedExprs.csv +++ b/tools/generated_files/322/supportedExprs.csv @@ -258,7 +258,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/323/supportedExprs.csv b/tools/generated_files/323/supportedExprs.csv index e4a4db760b0..573367d1d70 100644 --- a/tools/generated_files/323/supportedExprs.csv +++ b/tools/generated_files/323/supportedExprs.csv @@ -258,7 +258,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/324/supportedExprs.csv b/tools/generated_files/324/supportedExprs.csv index e4a4db760b0..573367d1d70 100644 --- a/tools/generated_files/324/supportedExprs.csv +++ b/tools/generated_files/324/supportedExprs.csv @@ -258,7 +258,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/330/supportedExprs.csv b/tools/generated_files/330/supportedExprs.csv index 0073281cb32..e1a4492676c 100644 --- a/tools/generated_files/330/supportedExprs.csv +++ b/tools/generated_files/330/supportedExprs.csv @@ -267,7 +267,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/330cdh/supportedExprs.csv b/tools/generated_files/330cdh/supportedExprs.csv index 0073281cb32..e1a4492676c 100644 --- a/tools/generated_files/330cdh/supportedExprs.csv +++ b/tools/generated_files/330cdh/supportedExprs.csv @@ -267,7 +267,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/331/supportedExprs.csv b/tools/generated_files/331/supportedExprs.csv index f62af4c9513..7329b2c4756 100644 --- a/tools/generated_files/331/supportedExprs.csv +++ b/tools/generated_files/331/supportedExprs.csv @@ -269,7 +269,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/332/supportedExprs.csv b/tools/generated_files/332/supportedExprs.csv index f62af4c9513..7329b2c4756 100644 --- a/tools/generated_files/332/supportedExprs.csv +++ b/tools/generated_files/332/supportedExprs.csv @@ -269,7 +269,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/332cdh/supportedExprs.csv b/tools/generated_files/332cdh/supportedExprs.csv index f62af4c9513..7329b2c4756 100644 --- a/tools/generated_files/332cdh/supportedExprs.csv +++ b/tools/generated_files/332cdh/supportedExprs.csv @@ -269,7 +269,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/333/supportedExprs.csv b/tools/generated_files/333/supportedExprs.csv index f62af4c9513..7329b2c4756 100644 --- a/tools/generated_files/333/supportedExprs.csv +++ b/tools/generated_files/333/supportedExprs.csv @@ -269,7 +269,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/334/supportedExprs.csv b/tools/generated_files/334/supportedExprs.csv index f62af4c9513..7329b2c4756 100644 --- a/tools/generated_files/334/supportedExprs.csv +++ b/tools/generated_files/334/supportedExprs.csv @@ -269,7 +269,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/340/supportedExprs.csv b/tools/generated_files/340/supportedExprs.csv index 01a48b40249..22b1f73d68b 100644 --- a/tools/generated_files/340/supportedExprs.csv +++ b/tools/generated_files/340/supportedExprs.csv @@ -269,7 +269,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/341/supportedExprs.csv b/tools/generated_files/341/supportedExprs.csv index 01a48b40249..22b1f73d68b 100644 --- a/tools/generated_files/341/supportedExprs.csv +++ b/tools/generated_files/341/supportedExprs.csv @@ -269,7 +269,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/342/supportedExprs.csv b/tools/generated_files/342/supportedExprs.csv index 01a48b40249..22b1f73d68b 100644 --- a/tools/generated_files/342/supportedExprs.csv +++ b/tools/generated_files/342/supportedExprs.csv @@ -269,7 +269,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/343/supportedExprs.csv b/tools/generated_files/343/supportedExprs.csv index 01a48b40249..22b1f73d68b 100644 --- a/tools/generated_files/343/supportedExprs.csv +++ b/tools/generated_files/343/supportedExprs.csv @@ -269,7 +269,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/344/supportedExprs.csv b/tools/generated_files/344/supportedExprs.csv index 01a48b40249..22b1f73d68b 100644 --- a/tools/generated_files/344/supportedExprs.csv +++ b/tools/generated_files/344/supportedExprs.csv @@ -269,7 +269,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/350/supportedExprs.csv b/tools/generated_files/350/supportedExprs.csv index 4cbfc7c1c27..3c7ee2a51eb 100644 --- a/tools/generated_files/350/supportedExprs.csv +++ b/tools/generated_files/350/supportedExprs.csv @@ -269,7 +269,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/351/supportedExprs.csv b/tools/generated_files/351/supportedExprs.csv index 4cbfc7c1c27..3c7ee2a51eb 100644 --- a/tools/generated_files/351/supportedExprs.csv +++ b/tools/generated_files/351/supportedExprs.csv @@ -269,7 +269,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/352/supportedExprs.csv b/tools/generated_files/352/supportedExprs.csv index 4cbfc7c1c27..3c7ee2a51eb 100644 --- a/tools/generated_files/352/supportedExprs.csv +++ b/tools/generated_files/352/supportedExprs.csv @@ -269,7 +269,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/353/supportedExprs.csv b/tools/generated_files/353/supportedExprs.csv index 4cbfc7c1c27..3c7ee2a51eb 100644 --- a/tools/generated_files/353/supportedExprs.csv +++ b/tools/generated_files/353/supportedExprs.csv @@ -269,7 +269,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/400/supportedExprs.csv b/tools/generated_files/400/supportedExprs.csv index 4cfa1020889..0623b25bce0 100644 --- a/tools/generated_files/400/supportedExprs.csv +++ b/tools/generated_files/400/supportedExprs.csv @@ -269,7 +269,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/supportedExprs.csv b/tools/generated_files/supportedExprs.csv index e4a4db760b0..573367d1d70 100644 --- a/tools/generated_files/supportedExprs.csv +++ b/tools/generated_files/supportedExprs.csv @@ -258,7 +258,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA From 33d97e85c8241c1c67146cb3e19aff0bb00becde Mon Sep 17 00:00:00 2001 From: Nghia Truong <7416935+ttnghia@users.noreply.github.com> Date: Sat, 14 Dec 2024 13:09:31 -0800 Subject: [PATCH 10/47] Support `trunc` and `date_trunc` SQL functions (#11833) * Implement GpuOverride Signed-off-by: Nghia Truong * Implement `GpuTruncDateTime` Signed-off-by: Nghia Truong * Do not fallback Signed-off-by: Nghia Truong * All variant input will be converted to just one Signed-off-by: Nghia Truong * Add generated docs Signed-off-by: Nghia Truong * Add tests Signed-off-by: Nghia Truong * Fix parameter types Signed-off-by: Nghia Truong * Fix test Signed-off-by: Nghia Truong * Change abstract class to trait Signed-off-by: Nghia Truong * Fix expression order and implement `TimeZoneAwareExpression` Signed-off-by: Nghia Truong * Update generated docs Signed-off-by: Nghia Truong * Update generated docs Signed-off-by: Nghia Truong * Add generated docs Signed-off-by: Nghia Truong * Add generated docs Signed-off-by: Nghia Truong * Allow non-utc timezone for timestamp tests Signed-off-by: Nghia Truong * Adopt to JNI changes Signed-off-by: Nghia Truong * Rewrite all classes Signed-off-by: Nghia Truong * Rename variable Signed-off-by: Nghia Truong --------- Signed-off-by: Nghia Truong --- .../advanced_configs.md | 2 + docs/supported_ops.md | 632 +++++++++++------- .../src/main/python/date_time_test.py | 75 ++- .../nvidia/spark/rapids/GpuOverrides.scala | 14 + .../sql/rapids/datetimeExpressions.scala | 157 ++++- tools/generated_files/320/operatorsScore.csv | 2 + tools/generated_files/320/supportedExprs.csv | 6 + tools/generated_files/321/operatorsScore.csv | 2 + tools/generated_files/321/supportedExprs.csv | 6 + tools/generated_files/330/operatorsScore.csv | 2 + tools/generated_files/330/supportedExprs.csv | 6 + tools/generated_files/331/operatorsScore.csv | 2 + tools/generated_files/331/supportedExprs.csv | 6 + tools/generated_files/340/operatorsScore.csv | 2 + tools/generated_files/340/supportedExprs.csv | 6 + tools/generated_files/342/operatorsScore.csv | 2 + tools/generated_files/342/supportedExprs.csv | 6 + tools/generated_files/400/operatorsScore.csv | 2 + tools/generated_files/400/supportedExprs.csv | 6 + tools/generated_files/operatorsScore.csv | 2 + tools/generated_files/supportedExprs.csv | 6 + 21 files changed, 702 insertions(+), 242 deletions(-) diff --git a/docs/additional-functionality/advanced_configs.md b/docs/additional-functionality/advanced_configs.md index a4427d9495a..5519e56b419 100644 --- a/docs/additional-functionality/advanced_configs.md +++ b/docs/additional-functionality/advanced_configs.md @@ -386,6 +386,8 @@ Name | SQL Function(s) | Description | Default Value | Notes spark.rapids.sql.expression.ToUnixTimestamp|`to_unix_timestamp`|Returns the UNIX timestamp of the given time|true|None| spark.rapids.sql.expression.TransformKeys|`transform_keys`|Transform keys in a map using a transform function|true|None| spark.rapids.sql.expression.TransformValues|`transform_values`|Transform values in a map using a transform function|true|None| +spark.rapids.sql.expression.TruncDate|`trunc`|Truncate the date to the unit specified by the given string format|true|None| +spark.rapids.sql.expression.TruncTimestamp|`date_trunc`|Truncate the timestamp to the unit specified by the given string format|true|None| spark.rapids.sql.expression.UnaryMinus|`negative`|Negate a numeric value|true|None| spark.rapids.sql.expression.UnaryPositive|`positive`|A numeric value with a + in front of it|true|None| spark.rapids.sql.expression.UnboundedFollowing$| |Special boundary for a window frame, indicating all rows preceding the current row|true|None| diff --git a/docs/supported_ops.md b/docs/supported_ops.md index be52afd9799..2ac5a27daa7 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -17702,6 +17702,154 @@ are limited. YEARMONTH +TruncDate +`trunc` +Truncate the date to the unit specified by the given string format +None +project +date + + + + + + + +S + + + + + + + + + + + + + + +format + + + + + + + + + +S + + + + + + + + + + + + +result + + + + + + + +S + + + + + + + + + + + + + + +TruncTimestamp +`date_trunc` +Truncate the timestamp to the unit specified by the given string format +None +project +format + + + + + + + + + +S + + + + + + + + + + + + +date + + + + + + + + +PS
UTC is only supported TZ for TIMESTAMP
+ + + + + + + + + + + + + +result + + + + + + + + +PS
UTC is only supported TZ for TIMESTAMP
+ + + + + + + + + + + + + UnaryMinus `negative` Negate a numeric value @@ -17926,6 +18074,34 @@ 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 +DAYTIME +YEARMONTH + + UnboundedPreceding$ Special boundary for a window frame, indicating all rows preceding the current row @@ -18079,34 +18255,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 -DAYTIME -YEARMONTH - - Upper `ucase`, `upper` String uppercase operator @@ -18357,6 +18505,34 @@ 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 +DAYTIME +YEARMONTH + + XxHash64 `xxhash64` xxhash64 hash operator @@ -18673,34 +18849,6 @@ 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 -DAYTIME -YEARMONTH - - ApproximatePercentile `approx_percentile`, `percentile_approx` Approximate percentile @@ -18891,6 +19039,34 @@ 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 +DAYTIME +YEARMONTH + + Average `avg`, `mean` Average aggregate operator @@ -19181,34 +19357,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 -DAYTIME -YEARMONTH - - CollectSet `collect_set` Collect a set of unique elements, not supported in reduction @@ -19354,6 +19502,34 @@ 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 +DAYTIME +YEARMONTH + + Count `count` Count aggregate operator @@ -19638,38 +19814,10 @@ are limited. NS PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT, DAYTIME, YEARMONTH
PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT, DAYTIME, YEARMONTH
-PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT, DAYTIME, YEARMONTH
-NS -NS -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 -DAYTIME -YEARMONTH +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types CALENDAR, UDT, DAYTIME, YEARMONTH
+NS +NS +NS Last @@ -19817,6 +19965,34 @@ 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 +DAYTIME +YEARMONTH + + Max `max` Max aggregate operator @@ -20106,34 +20282,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 -DAYTIME -YEARMONTH - - Min `min` Min aggregate operator @@ -20279,6 +20427,34 @@ 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 +DAYTIME +YEARMONTH + + MinBy `min_by` MinBy aggregate operator. It may produce different results than CPU when multiple rows in a group have same minimum value in the ordering column and different associated values in the value column. @@ -20613,34 +20789,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 -DAYTIME -YEARMONTH - - PivotFirst PivotFirst operator @@ -20785,6 +20933,34 @@ 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 +DAYTIME +YEARMONTH + + StddevPop `stddev_pop` Aggregation computing population standard deviation @@ -21075,34 +21251,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 -DAYTIME -YEARMONTH - - Sum `sum` Sum aggregate operator @@ -21248,6 +21396,34 @@ 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 +DAYTIME +YEARMONTH + + VariancePop `var_pop` Aggregation computing population variance @@ -21538,34 +21714,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 -DAYTIME -YEARMONTH - - NormalizeNaNAndZero Normalize NaN and zero @@ -21645,6 +21793,34 @@ 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 +DAYTIME +YEARMONTH + + HiveGenericUDF Hive Generic UDF, the UDF can choose to implement a RAPIDS accelerated interface to get better performance diff --git a/integration_tests/src/main/python/date_time_test.py b/integration_tests/src/main/python/date_time_test.py index 1a7024dac85..d29e6a5b5fa 100644 --- a/integration_tests/src/main/python/date_time_test.py +++ b/integration_tests/src/main/python/date_time_test.py @@ -321,7 +321,7 @@ def test_unsupported_fallback_to_unix_timestamp(data_gen): spark, [("a", data_gen), ("b", string_gen)], length=10).selectExpr( "to_unix_timestamp(a, b)"), "ToUnixTimestamp") - + supported_timezones = ["Asia/Shanghai", "UTC", "UTC+0", "UTC-0", "GMT", "GMT+0", "GMT-0", "EST", "MST", "VST"] unsupported_timezones = ["PST", "NST", "AST", "America/Los_Angeles", "America/New_York", "America/Chicago"] @@ -681,7 +681,7 @@ def test_unsupported_fallback_to_date(): conf) -# (-62135510400, 253402214400) is the range of seconds that can be represented by timestamp_seconds +# (-62135510400, 253402214400) is the range of seconds that can be represented by timestamp_seconds # considering the influence of time zone. ts_float_gen = SetValuesGen(FloatType(), [0.0, -0.0, 1.0, -1.0, 1.234567, -1.234567, 16777215.0, float('inf'), float('-inf'), float('nan')]) seconds_gens = [LongGen(min_val=-62135510400, max_val=253402214400), IntegerGen(), ShortGen(), ByteGen(), @@ -710,7 +710,7 @@ def test_timestamp_seconds_rounding_necessary(data_gen): lambda spark : unary_op_df(spark, data_gen).selectExpr("timestamp_seconds(a)").collect(), conf={}, error_message='Rounding necessary') - + @pytest.mark.parametrize('data_gen', [DecimalGen(19, 6), DecimalGen(20, 6)], ids=idfn) @allow_non_gpu(*non_utc_allow) def test_timestamp_seconds_decimal_overflow(data_gen): @@ -725,7 +725,7 @@ def test_timestamp_seconds_decimal_overflow(data_gen): def test_timestamp_millis(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr("timestamp_millis(a)")) - + @allow_non_gpu(*non_utc_allow) def test_timestamp_millis_long_overflow(): assert_gpu_and_cpu_error( @@ -751,3 +751,70 @@ def test_date_to_timestamp(parser_policy): conf = { "spark.sql.legacy.timeParserPolicy": parser_policy, "spark.rapids.sql.incompatibleDateFormats.enabled": True}) + +# Generate format strings, which are case insensitive and have some garbage rows. +trunc_date_format_gen = StringGen('(?i:YEAR|YYYY|YY|QUARTER|MONTH|MM|MON|WEEK)') \ + .with_special_pattern('invalid', weight=50) +trunc_timestamp_format_gen = StringGen('(?i:YEAR|YYYY|YY|QUARTER|MONTH|MM|MON|WEEK|DAY|DD|HOUR|MINUTE|SECOND|MILLISECOND|MICROSECOND)') \ + .with_special_pattern('invalid', weight=50) + +@pytest.mark.parametrize('data_gen', [date_gen], ids=idfn) +@pytest.mark.parametrize('format_gen', [trunc_date_format_gen], ids=idfn) +def test_trunc_date_full_input(data_gen, format_gen): + assert_gpu_and_cpu_are_equal_collect( + lambda spark : two_col_df(spark, data_gen, format_gen).selectExpr('trunc(a, b)')) + +@allow_non_gpu(*non_utc_tz_allow) +@pytest.mark.parametrize('format_gen', [trunc_timestamp_format_gen], ids=idfn) +@pytest.mark.parametrize('data_gen', [timestamp_gen], ids=idfn) +def test_trunc_timestamp_full_input(format_gen, data_gen): + assert_gpu_and_cpu_are_equal_collect( + lambda spark : two_col_df(spark, format_gen, data_gen).selectExpr('date_trunc(a, b)')) + +@pytest.mark.parametrize('format_gen', [trunc_date_format_gen], ids=idfn) +def test_trunc_date_single_value(format_gen): + assert_gpu_and_cpu_are_equal_collect( + lambda spark : unary_op_df(spark, format_gen).selectExpr('trunc("1980-05-18", a)')) + +@allow_non_gpu(*non_utc_tz_allow) +@pytest.mark.parametrize('format_gen', [trunc_timestamp_format_gen], ids=idfn) +def test_trunc_timestamp_single_value(format_gen): + assert_gpu_and_cpu_are_equal_collect( + lambda spark : unary_op_df(spark, format_gen).selectExpr( + 'date_trunc(a, "1980-05-18T09:32:05.359")')) + +@pytest.mark.parametrize('data_gen', [date_gen], ids=idfn) +def test_trunc_date_single_format(data_gen): + assert_gpu_and_cpu_are_equal_collect( + lambda spark : unary_op_df(spark, data_gen).selectExpr( + 'trunc(a, "YEAR")', + 'trunc(a, "YYYY")', + 'trunc(a, "YY")', + 'trunc(a, "QUARTER")', + 'trunc(a, "MONTH")', + 'trunc(a, "MM")', + 'trunc(a, "MON")', + 'trunc(a, "WEEK")', + 'trunc(a, "invalid")')) + +@allow_non_gpu(*non_utc_tz_allow) +@pytest.mark.parametrize('data_gen', [timestamp_gen], ids=idfn) +def test_trunc_timestamp_single_format(data_gen): + assert_gpu_and_cpu_are_equal_collect( + lambda spark : unary_op_df(spark, data_gen).selectExpr( + 'date_trunc("YEAR", a)', + 'date_trunc("YYYY", a)', + 'date_trunc("YY", a)', + 'date_trunc("QUARTER", a)', + 'date_trunc("MONTH", a)', + 'date_trunc("MM", a)', + 'date_trunc("MON", a)', + 'date_trunc("WEEK", a)', + 'date_trunc("DAY", a)', + 'date_trunc("DD", a)', + 'date_trunc("HOUR", a)', + 'date_trunc("MINUTE", a)', + 'date_trunc("SECOND", a)', + 'date_trunc("MILLISECOND", a)', + 'date_trunc("MICROSECOND", a)', + 'date_trunc("invalid", a)')) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index bd9aad97772..12ebfc01862 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 @@ -1822,6 +1822,20 @@ object GpuOverrides extends Logging { ParamCheck("round", TypeSig.lit(TypeEnum.BOOLEAN), TypeSig.BOOLEAN))), (a, conf, p, r) => new MonthsBetweenExprMeta(a, conf, p, r) ), + expr[TruncDate]( + "Truncate the date to the unit specified by the given string format", + ExprChecks.binaryProject(TypeSig.DATE, TypeSig.DATE, + ("date", TypeSig.DATE, TypeSig.DATE), + ("format", TypeSig.STRING, TypeSig.STRING)), + (a, conf, p, r) => new TruncDateExprMeta(a, conf, p, r) + ), + expr[TruncTimestamp]( + "Truncate the timestamp to the unit specified by the given string format", + ExprChecks.binaryProject(TypeSig.TIMESTAMP, TypeSig.TIMESTAMP, + ("format", TypeSig.STRING, TypeSig.STRING), + ("date", TypeSig.TIMESTAMP, TypeSig.TIMESTAMP)), + (a, conf, p, r) => new TruncTimestampExprMeta(a, conf, p, r) + ), expr[Pmod]( "Pmod", // Decimal support disabled https://github.com/NVIDIA/spark-rapids/issues/7553 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 d08c598cba4..ad14ab400dd 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,10 +27,10 @@ import com.nvidia.spark.rapids.Arm._ import com.nvidia.spark.rapids.ExprMeta import com.nvidia.spark.rapids.GpuOverrides.{extractStringLit, getTimeParserPolicy} import com.nvidia.spark.rapids.RapidsPluginImplicits._ -import com.nvidia.spark.rapids.jni.GpuTimeZoneDB +import com.nvidia.spark.rapids.jni.{DateTimeUtils, GpuTimeZoneDB} import com.nvidia.spark.rapids.shims.{NullIntolerantShim, ShimBinaryExpression, ShimExpression} -import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, ExpectsInputTypes, Expression, FromUnixTime, FromUTCTimestamp, ImplicitCastInputTypes, MonthsBetween, TimeZoneAwareExpression, ToUTCTimestamp} +import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, ExpectsInputTypes, Expression, FromUnixTime, FromUTCTimestamp, ImplicitCastInputTypes, MonthsBetween, TimeZoneAwareExpression, ToUTCTimestamp, TruncDate, TruncTimestamp} import org.apache.spark.sql.catalyst.util.DateTimeConstants import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -417,12 +417,11 @@ abstract class UnixTimeExprMeta[A <: BinaryExpression with TimeZoneAwareExpressi } trait GpuNumberToTimestampUnaryExpression extends GpuUnaryExpression { - override def dataType: DataType = TimestampType override def outputTypeOverride: DType = DType.TIMESTAMP_MICROSECONDS /** - * Test whether if input * multiplier will cause Long-overflow. In Math.multiplyExact, + * Test whether if input * multiplier will cause Long-overflow. In Math.multiplyExact, * if there is an integer-overflow, then it will throw an ArithmeticException "long overflow" */ def checkLongMultiplicationOverflow(input: ColumnVector, multiplier: Long): Unit = { @@ -439,7 +438,7 @@ trait GpuNumberToTimestampUnaryExpression extends GpuUnaryExpression { } protected val convertTo : GpuColumnVector => ColumnVector - + override def doColumnar(input: GpuColumnVector): ColumnVector = { convertTo(input) } @@ -543,14 +542,14 @@ case class GpuSecondsToTimestamp(child: Expression) extends GpuNumberToTimestamp longs.asTimestampSeconds() } case _ => - throw new UnsupportedOperationException(s"Unsupport type ${child.dataType} " + + throw new UnsupportedOperationException(s"Unsupport type ${child.dataType} " + s"for SecondsToTimestamp ") } } case class GpuMillisToTimestamp(child: Expression) extends GpuNumberToTimestampUnaryExpression { protected lazy val convertTo: GpuColumnVector => ColumnVector = child.dataType match { - case LongType => + case LongType => (input: GpuColumnVector) => { checkLongMultiplicationOverflow(input.getBase, DateTimeConstants.MICROS_PER_MILLIS) input.getBase.asTimestampMilliseconds() @@ -563,7 +562,7 @@ case class GpuMillisToTimestamp(child: Expression) extends GpuNumberToTimestampU } } case _ => - throw new UnsupportedOperationException(s"Unsupport type ${child.dataType} " + + throw new UnsupportedOperationException(s"Unsupport type ${child.dataType} " + s"for MillisToTimestamp ") } } @@ -581,7 +580,7 @@ case class GpuMicrosToTimestamp(child: Expression) extends GpuNumberToTimestampU } } case _ => - throw new UnsupportedOperationException(s"Unsupport type ${child.dataType} " + + throw new UnsupportedOperationException(s"Unsupport type ${child.dataType} " + s"for MicrosToTimestamp ") } } @@ -1110,7 +1109,7 @@ abstract class ConvertUTCTimestampExprMetaBase[INPUT <: BinaryExpression]( rule: DataFromReplacementRule) extends BinaryExprMeta[INPUT](expr, conf, parent, rule) { - protected[this] var timezoneId: ZoneId = null + protected[this] var timezoneId: ZoneId = null override def tagExprForGpu(): Unit = { extractStringLit(expr.right) match { @@ -1527,3 +1526,141 @@ case class GpuLastDay(startDate: Expression) override protected def doColumnar(input: GpuColumnVector): ColumnVector = input.getBase.lastDayOfMonth() } + +abstract class GpuTruncDateTime(fmtStr: Option[String]) extends GpuBinaryExpression + with ImplicitCastInputTypes with Serializable { + override def nullable: Boolean = true + + protected def truncate(datetimeCol: GpuColumnVector, fmtCol: GpuColumnVector): ColumnVector = { + DateTimeUtils.truncate(datetimeCol.getBase, fmtCol.getBase) + } + + protected def truncate(datetimeVal: GpuScalar, fmtCol: GpuColumnVector): ColumnVector = { + withResource(ColumnVector.fromScalar(datetimeVal.getBase, 1)) { datetimeCol => + DateTimeUtils.truncate(datetimeCol, fmtCol.getBase) + } + } + + protected def truncate(datetimeCol: GpuColumnVector, fmtVal: GpuScalar): ColumnVector = { + // fmtVal is unused, as it was extracted to `fmtStr` before. + fmtStr match { + case Some(fmt) => DateTimeUtils.truncate(datetimeCol.getBase, fmt) + case None => throw new IllegalArgumentException("Invalid format string.") + } + } + + protected def truncate(numRows: Int, datetimeVal: GpuScalar, fmtVal: GpuScalar): ColumnVector = { + // fmtVal is unused, as it was extracted to `fmtStr` before. + fmtStr match { + case Some(fmt) => + withResource(ColumnVector.fromScalar(datetimeVal.getBase, 1)) { datetimeCol => + val truncated = DateTimeUtils.truncate(datetimeCol, fmt) + if (numRows == 1) { + truncated + } else { + withResource(truncated) { _ => + withResource(truncated.getScalarElement(0)) { truncatedScalar => + ColumnVector.fromScalar(truncatedScalar, numRows) + } + } + } + } + case None => throw new IllegalArgumentException("Invalid format string.") + } + } +} + +case class GpuTruncDate(date: Expression, fmt: Expression, fmtStr: Option[String]) + extends GpuTruncDateTime(fmtStr) { + override def left: Expression = date + + override def right: Expression = fmt + + override def inputTypes: Seq[AbstractDataType] = Seq(DateType, StringType) + + override def dataType: DataType = DateType + + override def prettyName: String = "trunc" + + override def doColumnar(lhs: GpuColumnVector, rhs: GpuColumnVector): ColumnVector = { + truncate(lhs, rhs) + } + + override def doColumnar(lhs: GpuScalar, rhs: GpuColumnVector): ColumnVector = { + truncate(lhs, rhs) + } + + override def doColumnar(lhs: GpuColumnVector, rhs: GpuScalar): ColumnVector = { + truncate(lhs, rhs) + } + + override def doColumnar(numRows: Int, lhs: GpuScalar, rhs: GpuScalar): ColumnVector = { + truncate(numRows, lhs, rhs) + } +} + +case class GpuTruncTimestamp(fmt: Expression, timestamp: Expression, timeZoneId: Option[String], + fmtStr: Option[String]) + extends GpuTruncDateTime(fmtStr) with TimeZoneAwareExpression { + + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = { + copy(timeZoneId = Option(timeZoneId)) + } + + override def left: Expression = fmt + + override def right: Expression = timestamp + + override def inputTypes: Seq[AbstractDataType] = Seq(StringType, TimestampType) + + override def dataType: DataType = TimestampType + + override def prettyName: String = "date_trunc" + + // Since the input order of this class is opposite compared to the `GpuTruncDate` class, + // we need to switch `lhs` and `rhs` in the `doColumnar` methods below. + + override def doColumnar(lhs: GpuColumnVector, rhs: GpuColumnVector): ColumnVector = { + truncate(rhs, lhs) + } + + override def doColumnar(lhs: GpuScalar, rhs: GpuColumnVector): ColumnVector = { + truncate(rhs, lhs) + } + + override def doColumnar(lhs: GpuColumnVector, rhs: GpuScalar): ColumnVector = { + truncate(rhs, lhs) + } + + override def doColumnar(numRows: Int, lhs: GpuScalar, rhs: GpuScalar): ColumnVector = { + truncate(numRows, rhs, lhs) + } +} + +class TruncDateExprMeta(expr: TruncDate, + override val conf: RapidsConf, + override val parent: Option[RapidsMeta[_, _, _]], + rule: DataFromReplacementRule) + extends BinaryExprMeta[TruncDate](expr, conf, parent, rule) { + + // Store the format string as we need to process it on the CPU later on. + private val fmtStr = extractStringLit(expr.format) + + override def convertToGpu(date: Expression, format: Expression): GpuExpression = { + GpuTruncDate(date, format, fmtStr) + } +} + +class TruncTimestampExprMeta(expr: TruncTimestamp, + override val conf: RapidsConf, + override val parent: Option[RapidsMeta[_, _, _]], + rule: DataFromReplacementRule) + extends BinaryExprMeta[TruncTimestamp](expr, conf, parent, rule) { + + // Store the format string as we need to process it on the CPU later on. + private val fmtStr = extractStringLit(expr.format) + + override def convertToGpu(format: Expression, timestamp: Expression): GpuExpression = { + GpuTruncTimestamp(format, timestamp, expr.timeZoneId, fmtStr) + } +} diff --git a/tools/generated_files/320/operatorsScore.csv b/tools/generated_files/320/operatorsScore.csv index 19c999aa796..d8c4ca63adc 100644 --- a/tools/generated_files/320/operatorsScore.csv +++ b/tools/generated_files/320/operatorsScore.csv @@ -265,6 +265,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/320/supportedExprs.csv b/tools/generated_files/320/supportedExprs.csv index 573367d1d70..39d69c0836b 100644 --- a/tools/generated_files/320/supportedExprs.csv +++ b/tools/generated_files/320/supportedExprs.csv @@ -606,6 +606,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NS,NS UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NS,NS UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/321/operatorsScore.csv b/tools/generated_files/321/operatorsScore.csv index 19c999aa796..d8c4ca63adc 100644 --- a/tools/generated_files/321/operatorsScore.csv +++ b/tools/generated_files/321/operatorsScore.csv @@ -265,6 +265,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/321/supportedExprs.csv b/tools/generated_files/321/supportedExprs.csv index 573367d1d70..39d69c0836b 100644 --- a/tools/generated_files/321/supportedExprs.csv +++ b/tools/generated_files/321/supportedExprs.csv @@ -606,6 +606,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NS,NS UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NS,NS UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/330/operatorsScore.csv b/tools/generated_files/330/operatorsScore.csv index e5978fb9f1a..e86e30e606c 100644 --- a/tools/generated_files/330/operatorsScore.csv +++ b/tools/generated_files/330/operatorsScore.csv @@ -275,6 +275,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/330/supportedExprs.csv b/tools/generated_files/330/supportedExprs.csv index e1a4492676c..c8df29cfca5 100644 --- a/tools/generated_files/330/supportedExprs.csv +++ b/tools/generated_files/330/supportedExprs.csv @@ -627,6 +627,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/331/operatorsScore.csv b/tools/generated_files/331/operatorsScore.csv index b988344e702..229201ba885 100644 --- a/tools/generated_files/331/operatorsScore.csv +++ b/tools/generated_files/331/operatorsScore.csv @@ -276,6 +276,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/331/supportedExprs.csv b/tools/generated_files/331/supportedExprs.csv index 7329b2c4756..e217b6ce31d 100644 --- a/tools/generated_files/331/supportedExprs.csv +++ b/tools/generated_files/331/supportedExprs.csv @@ -629,6 +629,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/340/operatorsScore.csv b/tools/generated_files/340/operatorsScore.csv index b1e9198e58b..16ac93a02ba 100644 --- a/tools/generated_files/340/operatorsScore.csv +++ b/tools/generated_files/340/operatorsScore.csv @@ -277,6 +277,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/340/supportedExprs.csv b/tools/generated_files/340/supportedExprs.csv index 22b1f73d68b..0c2d6a74bc1 100644 --- a/tools/generated_files/340/supportedExprs.csv +++ b/tools/generated_files/340/supportedExprs.csv @@ -629,6 +629,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/342/operatorsScore.csv b/tools/generated_files/342/operatorsScore.csv index b1e9198e58b..16ac93a02ba 100644 --- a/tools/generated_files/342/operatorsScore.csv +++ b/tools/generated_files/342/operatorsScore.csv @@ -277,6 +277,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/342/supportedExprs.csv b/tools/generated_files/342/supportedExprs.csv index 22b1f73d68b..0c2d6a74bc1 100644 --- a/tools/generated_files/342/supportedExprs.csv +++ b/tools/generated_files/342/supportedExprs.csv @@ -629,6 +629,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/400/operatorsScore.csv b/tools/generated_files/400/operatorsScore.csv index 53791a06705..0a099fc2233 100644 --- a/tools/generated_files/400/operatorsScore.csv +++ b/tools/generated_files/400/operatorsScore.csv @@ -278,6 +278,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/400/supportedExprs.csv b/tools/generated_files/400/supportedExprs.csv index 0623b25bce0..d3f1f1b851a 100644 --- a/tools/generated_files/400/supportedExprs.csv +++ b/tools/generated_files/400/supportedExprs.csv @@ -635,6 +635,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/operatorsScore.csv b/tools/generated_files/operatorsScore.csv index 19c999aa796..d8c4ca63adc 100644 --- a/tools/generated_files/operatorsScore.csv +++ b/tools/generated_files/operatorsScore.csv @@ -265,6 +265,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/supportedExprs.csv b/tools/generated_files/supportedExprs.csv index 573367d1d70..39d69c0836b 100644 --- a/tools/generated_files/supportedExprs.csv +++ b/tools/generated_files/supportedExprs.csv @@ -606,6 +606,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NS,NS UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NS,NS UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS From 4d7373b2c28f34eba3044b1ccc28aca91b968905 Mon Sep 17 00:00:00 2001 From: Jenkins Automation <70000568+nvauto@users.noreply.github.com> Date: Mon, 16 Dec 2024 09:05:08 +0800 Subject: [PATCH 11/47] Update rapids JNI and private dependency to 24.12.0 (#11849) \nWait for the pre-merge CI job to SUCCEED Signed-off-by: nvauto <70000568+nvauto@users.noreply.github.com> --- pom.xml | 4 ++-- scala2.13/pom.xml | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index 12828404031..00d2777a58e 100644 --- a/pom.xml +++ b/pom.xml @@ -829,8 +829,8 @@ spark${buildver} cuda11 ${cuda.version} - 24.12.0-SNAPSHOT - 24.12.0-SNAPSHOT + 24.12.0 + 24.12.0 2.12 2.8.0 incremental diff --git a/scala2.13/pom.xml b/scala2.13/pom.xml index d52c8658423..b9b97830d8b 100644 --- a/scala2.13/pom.xml +++ b/scala2.13/pom.xml @@ -829,8 +829,8 @@ spark${buildver} cuda11 ${cuda.version} - 24.12.0-SNAPSHOT - 24.12.0-SNAPSHOT + 24.12.0 + 24.12.0 2.13 2.8.0 incremental From 22680f57119012a2fcc125cc93b71dcd1f6473a3 Mon Sep 17 00:00:00 2001 From: Jenkins Automation <70000568+nvauto@users.noreply.github.com> Date: Mon, 16 Dec 2024 09:54:46 +0800 Subject: [PATCH 12/47] Update latest changelog [skip ci] (#11851) * Update latest changelog [skip ci] Update change log with CLI: \n\n scripts/generate-changelog --token= --releases=24.10,24.12 Signed-off-by: nvauto <70000568+nvauto@users.noreply.github.com> * Update changelog Signed-off-by: Tim Liu * update changelog to involve new changes. Signed-off-by: Yanxuan Liu --------- Signed-off-by: nvauto <70000568+nvauto@users.noreply.github.com> Signed-off-by: Tim Liu Signed-off-by: Yanxuan Liu Co-authored-by: Tim Liu Co-authored-by: Yanxuan Liu --- CHANGELOG.md | 387 ++++++++---------- ...o-24.06.md => CHANGELOG_24.02-to-24.08.md} | 209 +++++++++- 2 files changed, 385 insertions(+), 211 deletions(-) rename docs/archives/{CHANGELOG_24.02-to-24.06.md => CHANGELOG_24.02-to-24.08.md} (74%) diff --git a/CHANGELOG.md b/CHANGELOG.md index 2510eba5dfe..5dd6c7b19ba 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,175 @@ # Change log -Generated on 2024-10-31 +Generated on 2024-12-16 + +## Release 24.12 + +### Features +||| +|:---|:---| +|[#11630](https://github.com/NVIDIA/spark-rapids/issues/11630)|[FEA] enable from_json and json scan by default| +|[#11709](https://github.com/NVIDIA/spark-rapids/issues/11709)|[FEA] Add support for `MonthsBetween`| +|[#11666](https://github.com/NVIDIA/spark-rapids/issues/11666)|[FEA] support task limit profiling for specified stages| +|[#11662](https://github.com/NVIDIA/spark-rapids/issues/11662)|[FEA] Support Apache Spark 3.4.4| +|[#11657](https://github.com/NVIDIA/spark-rapids/issues/11657)|[FEA] Support format 'yyyyMMdd HH:mm:ss' for legacy mode| +|[#11419](https://github.com/NVIDIA/spark-rapids/issues/11419)|[FEA] Support Spark 3.5.3 release| +|[#11505](https://github.com/NVIDIA/spark-rapids/issues/11505)|[FEA] Support yyyymmdd format for GetTimestamp for LEGACY mode.| + +### Performance +||| +|:---|:---| +|[#8391](https://github.com/NVIDIA/spark-rapids/issues/8391)|[FEA] Do a hash based re-partition instead of a sort based fallback for hash aggregate| +|[#11560](https://github.com/NVIDIA/spark-rapids/issues/11560)|[FEA] Improve `GpuJsonToStructs` performance| +|[#11458](https://github.com/NVIDIA/spark-rapids/issues/11458)|[FEA] enable prune_columns for from_json| + +### Bugs Fixed +||| +|:---|:---| +|[#10907](https://github.com/NVIDIA/spark-rapids/issues/10907)|from_json function parses a column containing an empty array, throws an exception.| +|[#11793](https://github.com/NVIDIA/spark-rapids/issues/11793)|[BUG] "Time in Heuristic" should not include previous operator's compute time| +|[#11798](https://github.com/NVIDIA/spark-rapids/issues/11798)|[BUG] mismatch CPU and GPU result in test_months_between_first_day[DATAGEN_SEED=1733006411, TZ=Africa/Casablanca]| +|[#11790](https://github.com/NVIDIA/spark-rapids/issues/11790)|[BUG] test_hash_* failed "java.util.NoSuchElementException: head of empty list" or "Too many times of repartition, may hit a bug?"| +|[#11643](https://github.com/NVIDIA/spark-rapids/issues/11643)|[BUG] Support AQE with Broadcast Hash Join and DPP on Databricks 14.3| +|[#10910](https://github.com/NVIDIA/spark-rapids/issues/10910)|from_json, when input = empty object, rapids throws an exception.| +|[#10891](https://github.com/NVIDIA/spark-rapids/issues/10891)|Parsing a column containing invalid json into StructureType with schema throws an Exception.| +|[#11741](https://github.com/NVIDIA/spark-rapids/issues/11741)|[BUG] Fix spark400 build due to writeWithV1 return value change| +|[#11533](https://github.com/NVIDIA/spark-rapids/issues/11533)|Fix JSON Matrix tests on Databricks 14.3| +|[#11722](https://github.com/NVIDIA/spark-rapids/issues/11722)|[BUG] Spark 4.0.0 has moved `NullIntolerant` and builds are breaking because they are unable to find it.| +|[#11726](https://github.com/NVIDIA/spark-rapids/issues/11726)|[BUG] Databricks 14.3 nightly deploy fails due to incorrect DB_SHIM_NAME| +|[#11293](https://github.com/NVIDIA/spark-rapids/issues/11293)|[BUG] A user query with from_json failed with "JSON Parser encountered an invalid format at location"| +|[#9592](https://github.com/NVIDIA/spark-rapids/issues/9592)|[BUG][JSON] `from_json` to Map type should produce null for invalid entries| +|[#11715](https://github.com/NVIDIA/spark-rapids/issues/11715)|[BUG] parquet_testing_test.py failed on "AssertionError: GPU and CPU boolean values are different"| +|[#11716](https://github.com/NVIDIA/spark-rapids/issues/11716)|[BUG] delta_lake_write_test.py failed on "AssertionError: GPU and CPU boolean values are different"| +|[#11684](https://github.com/NVIDIA/spark-rapids/issues/11684)|[BUG] 24.12 Precommit fails with wrong number of arguments in `GpuDataSource`| +|[#11168](https://github.com/NVIDIA/spark-rapids/issues/11168)|[BUG] reserve allocation should be displayed when erroring due to lack of memory on startup| +|[#7585](https://github.com/NVIDIA/spark-rapids/issues/7585)|[BUG] [Regexp] Line anchor '$' incorrect matching of unicode line terminators| +|[#11622](https://github.com/NVIDIA/spark-rapids/issues/11622)|[BUG] GPU Parquet scan filter pushdown fails with timestamp/INT96 column| +|[#11646](https://github.com/NVIDIA/spark-rapids/issues/11646)|[BUG] NullPointerException in GpuRand| +|[#10498](https://github.com/NVIDIA/spark-rapids/issues/10498)|[BUG] Unit tests failed: [INTERVAL_ARITHMETIC_OVERFLOW] integer overflow. Use 'try_add' to tolerate overflow and return NULL instead| +|[#11659](https://github.com/NVIDIA/spark-rapids/issues/11659)|[BUG] parse_url throws exception if partToExtract is invalid while Spark returns null| +|[#10894](https://github.com/NVIDIA/spark-rapids/issues/10894)|Parsing a column containing a nested structure to json thows an exception| +|[#10895](https://github.com/NVIDIA/spark-rapids/issues/10895)|Converting a column containing a map into json throws an exception| +|[#10896](https://github.com/NVIDIA/spark-rapids/issues/10896)|Converting an column containing an array into json throws an exception| +|[#10915](https://github.com/NVIDIA/spark-rapids/issues/10915)|to_json when converts an array will throw an exception:| +|[#10916](https://github.com/NVIDIA/spark-rapids/issues/10916)|to_json function doesn't support map[string, struct] to json conversion.| +|[#10919](https://github.com/NVIDIA/spark-rapids/issues/10919)|to_json converting map[string, integer] to json, throws an exception| +|[#10920](https://github.com/NVIDIA/spark-rapids/issues/10920)|to_json converting an array with maps throws an exception.| +|[#10921](https://github.com/NVIDIA/spark-rapids/issues/10921)|to_json - array with single map| +|[#10923](https://github.com/NVIDIA/spark-rapids/issues/10923)|[BUG] Spark UT framework: to_json function to convert the array with a single empty row to a JSON string throws an exception.| +|[#10924](https://github.com/NVIDIA/spark-rapids/issues/10924)|[BUG] Spark UT framework: to_json when converts an empty array into json throws an exception. | +|[#11024](https://github.com/NVIDIA/spark-rapids/issues/11024)|Fix tests failures in parquet_write_test.py| +|[#11174](https://github.com/NVIDIA/spark-rapids/issues/11174)|Opcode Suite fails for Scala 2.13.8+ | +|[#10483](https://github.com/NVIDIA/spark-rapids/issues/10483)|[BUG] JsonToStructs fails to parse all empty dicts and invalid lines| +|[#10489](https://github.com/NVIDIA/spark-rapids/issues/10489)|[BUG] from_json does not support input with \n in it.| +|[#10347](https://github.com/NVIDIA/spark-rapids/issues/10347)|[BUG] Failures in Integration Tests on Dataproc Serverless| +|[#11021](https://github.com/NVIDIA/spark-rapids/issues/11021)|Fix tests failures in orc_cast_test.py| +|[#11609](https://github.com/NVIDIA/spark-rapids/issues/11609)|[BUG] test_hash_repartition_long_overflow_ansi_exception failed on 341DB| +|[#11600](https://github.com/NVIDIA/spark-rapids/issues/11600)|[BUG] regex_test failed mismatched cpu and gpu values in UT and IT| +|[#11611](https://github.com/NVIDIA/spark-rapids/issues/11611)|[BUG] Spark 4.0 build failure - value cannotSaveIntervalIntoExternalStorageError is not a member of object org.apache.spark.sql.errors.QueryCompilationErrors| +|[#10922](https://github.com/NVIDIA/spark-rapids/issues/10922)|from_json cannot support line separator in the input string.| +|[#11009](https://github.com/NVIDIA/spark-rapids/issues/11009)|Fix tests failures in cast_test.py| +|[#11572](https://github.com/NVIDIA/spark-rapids/issues/11572)|[BUG] MultiFileReaderThreadPool may flood the console with log messages| + +### PRs +||| +|:---|:---| +|[#11849](https://github.com/NVIDIA/spark-rapids/pull/11849)|Update rapids JNI and private dependency to 24.12.0| +|[#11857](https://github.com/NVIDIA/spark-rapids/pull/11857)|Increase the pre-merge CI timeout to 6 hours| +|[#11845](https://github.com/NVIDIA/spark-rapids/pull/11845)|Fix leak in isTimeStamp| +|[#11823](https://github.com/NVIDIA/spark-rapids/pull/11823)|Fix for `LEAD/LAG` window function test failures.| +|[#11832](https://github.com/NVIDIA/spark-rapids/pull/11832)|Fix leak in GpuBroadcastNestedLoopJoinExecBase| +|[#11763](https://github.com/NVIDIA/spark-rapids/pull/11763)|Orc writes don't fully support Booleans with nulls | +|[#11794](https://github.com/NVIDIA/spark-rapids/pull/11794)|exclude previous operator's time out of firstBatchHeuristic| +|[#11802](https://github.com/NVIDIA/spark-rapids/pull/11802)|Fall back to CPU for non-UTC months_between| +|[#11792](https://github.com/NVIDIA/spark-rapids/pull/11792)|[BUG] Fix issue 11790| +|[#11768](https://github.com/NVIDIA/spark-rapids/pull/11768)|Fix `dpp_test.py` failures on 14.3| +|[#11752](https://github.com/NVIDIA/spark-rapids/pull/11752)|Ability to decompress snappy and zstd Parquet files via CPU| +|[#11777](https://github.com/NVIDIA/spark-rapids/pull/11777)|Append knoguchi22 to blossom-ci whitelist [skip ci]| +|[#11712](https://github.com/NVIDIA/spark-rapids/pull/11712)|repartition-based fallback for hash aggregate v3| +|[#11771](https://github.com/NVIDIA/spark-rapids/pull/11771)|Fix query hang when using rapids multithread shuffle manager with kudo| +|[#11759](https://github.com/NVIDIA/spark-rapids/pull/11759)|Avoid using StringBuffer in single-threaded methods.| +|[#11766](https://github.com/NVIDIA/spark-rapids/pull/11766)|Fix Kudo batch serializer to only read header in hasNext| +|[#11730](https://github.com/NVIDIA/spark-rapids/pull/11730)|Add support for asynchronous writing for parquet| +|[#11750](https://github.com/NVIDIA/spark-rapids/pull/11750)|Fix aqe_test failures on 14.3.| +|[#11753](https://github.com/NVIDIA/spark-rapids/pull/11753)|Enable JSON Scan and from_json by default| +|[#11733](https://github.com/NVIDIA/spark-rapids/pull/11733)|Print out the current attempt object when OOM inside a retry block| +|[#11618](https://github.com/NVIDIA/spark-rapids/pull/11618)|Execute `from_json` with struct schema using `JSONUtils.fromJSONToStructs`| +|[#11725](https://github.com/NVIDIA/spark-rapids/pull/11725)|host watermark metric| +|[#11746](https://github.com/NVIDIA/spark-rapids/pull/11746)|Remove batch size bytes limits| +|[#11723](https://github.com/NVIDIA/spark-rapids/pull/11723)|Add NVIDIA Copyright| +|[#11721](https://github.com/NVIDIA/spark-rapids/pull/11721)|Add a few more JSON tests for MAP| +|[#11744](https://github.com/NVIDIA/spark-rapids/pull/11744)|Do not package the Databricks 14.3 shim into the dist jar [skip ci]| +|[#11724](https://github.com/NVIDIA/spark-rapids/pull/11724)|Integrate with kudo| +|[#11739](https://github.com/NVIDIA/spark-rapids/pull/11739)|Update to Spark 4.0 changing signature of SupportsV1Write.writeWithV1| +|[#11737](https://github.com/NVIDIA/spark-rapids/pull/11737)|Add in support for months_between| +|[#11700](https://github.com/NVIDIA/spark-rapids/pull/11700)|Fix leak with RapidsHostColumnBuilder in GpuUserDefinedFunction| +|[#11727](https://github.com/NVIDIA/spark-rapids/pull/11727)|Widen type promotion for decimals with larger scale in Parquet Read| +|[#11719](https://github.com/NVIDIA/spark-rapids/pull/11719)|Skip `from_json` overflow tests for 14.3| +|[#11708](https://github.com/NVIDIA/spark-rapids/pull/11708)|Support profiling for specific stages on a limited number of tasks| +|[#11731](https://github.com/NVIDIA/spark-rapids/pull/11731)|Add NullIntolerantShim to adapt to Spark 4.0 removing NullIntolerant| +|[#11413](https://github.com/NVIDIA/spark-rapids/pull/11413)|Support multi string contains| +|[#11728](https://github.com/NVIDIA/spark-rapids/pull/11728)|Change Databricks 14.3 shim name to spark350db143 [skip ci]| +|[#11702](https://github.com/NVIDIA/spark-rapids/pull/11702)|Improve JSON scan and `from_json`| +|[#11635](https://github.com/NVIDIA/spark-rapids/pull/11635)|Added Shims for adding Databricks 14.3 Support| +|[#11714](https://github.com/NVIDIA/spark-rapids/pull/11714)|Let AWS Databricks automatically choose an Availability Zone| +|[#11703](https://github.com/NVIDIA/spark-rapids/pull/11703)|Simplify $ transpiling and fix newline character bug| +|[#11707](https://github.com/NVIDIA/spark-rapids/pull/11707)|impalaFile cannot be found by UT framework. | +|[#11697](https://github.com/NVIDIA/spark-rapids/pull/11697)|Make delta-lake shim dependencies parametrizable| +|[#11710](https://github.com/NVIDIA/spark-rapids/pull/11710)|Add shim version 344 to LogicalPlanShims.scala| +|[#11706](https://github.com/NVIDIA/spark-rapids/pull/11706)|Add retry support in sub hash join| +|[#11673](https://github.com/NVIDIA/spark-rapids/pull/11673)|Fix Parquet Writer tests on 14.3| +|[#11669](https://github.com/NVIDIA/spark-rapids/pull/11669)|Fix `string_test` for 14.3| +|[#11692](https://github.com/NVIDIA/spark-rapids/pull/11692)|Add Spark 3.4.4 Shim | +|[#11695](https://github.com/NVIDIA/spark-rapids/pull/11695)|Fix spark400 build due to LogicalRelation signature changes| +|[#11689](https://github.com/NVIDIA/spark-rapids/pull/11689)|Update the Maven repository to download Spark JAR files [skip ci]| +|[#11670](https://github.com/NVIDIA/spark-rapids/pull/11670)|Fix `misc_expr_test` for 14.3| +|[#11652](https://github.com/NVIDIA/spark-rapids/pull/11652)|Fix skipping fixed_length_char ORC tests on > 13.3| +|[#11644](https://github.com/NVIDIA/spark-rapids/pull/11644)|Skip AQE-join-DPP tests for 14.3| +|[#11667](https://github.com/NVIDIA/spark-rapids/pull/11667)|Preparation for the coming Kudo support| +|[#11685](https://github.com/NVIDIA/spark-rapids/pull/11685)|Exclude shimplify-generated files from scalastyle| +|[#11282](https://github.com/NVIDIA/spark-rapids/pull/11282)|Reserve allocation should be displayed when erroring due to lack of memory on startup| +|[#11671](https://github.com/NVIDIA/spark-rapids/pull/11671)|Use the new host memory allocation API| +|[#11682](https://github.com/NVIDIA/spark-rapids/pull/11682)|Fix auto merge conflict 11679 [skip ci]| +|[#11663](https://github.com/NVIDIA/spark-rapids/pull/11663)|Simplify Transpilation of $ with Extended Line Separator Support in cuDF Regex| +|[#11672](https://github.com/NVIDIA/spark-rapids/pull/11672)|Fix race condition with Parquet filter pushdown modifying shared hadoop Configuration| +|[#11596](https://github.com/NVIDIA/spark-rapids/pull/11596)|Add a new NVTX range for task GPU ownership| +|[#11664](https://github.com/NVIDIA/spark-rapids/pull/11664)|Fix `orc_write_test.py` for 14.3| +|[#11656](https://github.com/NVIDIA/spark-rapids/pull/11656)|[DOC] update the supported OS in download page [skip ci]| +|[#11665](https://github.com/NVIDIA/spark-rapids/pull/11665)|Generate classes identical up to the shim package name| +|[#11647](https://github.com/NVIDIA/spark-rapids/pull/11647)|Fix a NPE issue in GpuRand| +|[#11658](https://github.com/NVIDIA/spark-rapids/pull/11658)|Support format 'yyyyMMdd HH:mm:ss' for legacy mode| +|[#11661](https://github.com/NVIDIA/spark-rapids/pull/11661)|Support invalid partToExtract for parse_url| +|[#11520](https://github.com/NVIDIA/spark-rapids/pull/11520)|UT adjust override checkScanSchemata & enabling ut of exclude_by_suffix fea.| +|[#11634](https://github.com/NVIDIA/spark-rapids/pull/11634)|Put DF_UDF plugin code into the main uber jar.| +|[#11522](https://github.com/NVIDIA/spark-rapids/pull/11522)|UT adjust test SPARK-26677: negated null-safe equality comparison| +|[#11521](https://github.com/NVIDIA/spark-rapids/pull/11521)|Datetime rebasing issue fixed| +|[#11642](https://github.com/NVIDIA/spark-rapids/pull/11642)|Update to_json to be more generic and fix some bugs| +|[#11615](https://github.com/NVIDIA/spark-rapids/pull/11615)|Spark 4 parquet_writer_test.py fixes| +|[#11623](https://github.com/NVIDIA/spark-rapids/pull/11623)|Fix `collection_ops_test` for 14.3| +|[#11553](https://github.com/NVIDIA/spark-rapids/pull/11553)|Fix udf-compiler scala2.13 internal return statements| +|[#11640](https://github.com/NVIDIA/spark-rapids/pull/11640)|Disable date/timestamp types by default when parsing JSON| +|[#11570](https://github.com/NVIDIA/spark-rapids/pull/11570)|Add support for Spark 3.5.3| +|[#11591](https://github.com/NVIDIA/spark-rapids/pull/11591)|Spark UT framework: Read Parquet file generated by parquet-thrift Rapids, UT case adjust.| +|[#11631](https://github.com/NVIDIA/spark-rapids/pull/11631)|Update JSON tests based on a closed/fixed issues| +|[#11617](https://github.com/NVIDIA/spark-rapids/pull/11617)|Quick fix for the build script failure of Scala 2.13 jars [skip ci]| +|[#11614](https://github.com/NVIDIA/spark-rapids/pull/11614)|Ensure repartition overflow test always overflows| +|[#11612](https://github.com/NVIDIA/spark-rapids/pull/11612)|Revert "Disable regex tests to unblock CI (#11606)"| +|[#11597](https://github.com/NVIDIA/spark-rapids/pull/11597)|`install_deps` changes for Databricks 14.3| +|[#11608](https://github.com/NVIDIA/spark-rapids/pull/11608)|Use mvn -f scala2.13/ in the build scripts to build the 2.13 jars| +|[#11610](https://github.com/NVIDIA/spark-rapids/pull/11610)|Change DataSource calendar interval error to fix spark400 build| +|[#11549](https://github.com/NVIDIA/spark-rapids/pull/11549)|Adopt `JSONUtils.concatenateJsonStrings` for concatenating JSON strings| +|[#11595](https://github.com/NVIDIA/spark-rapids/pull/11595)|Remove an unused config shuffle.spillThreads| +|[#11606](https://github.com/NVIDIA/spark-rapids/pull/11606)|Disable regex tests to unblock CI| +|[#11605](https://github.com/NVIDIA/spark-rapids/pull/11605)|Fix auto merge conflict 11604 [skip ci]| +|[#11587](https://github.com/NVIDIA/spark-rapids/pull/11587)|avoid long tail tasks due to PrioritySemaphore, remaing part| +|[#11574](https://github.com/NVIDIA/spark-rapids/pull/11574)|avoid long tail tasks due to PrioritySemaphore| +|[#11559](https://github.com/NVIDIA/spark-rapids/pull/11559)|[Spark 4.0] Address test failures in cast_test.py| +|[#11579](https://github.com/NVIDIA/spark-rapids/pull/11579)|Fix merge conflict with branch-24.10| +|[#11571](https://github.com/NVIDIA/spark-rapids/pull/11571)|Log reconfigure multi-file thread pool only once| +|[#11564](https://github.com/NVIDIA/spark-rapids/pull/11564)|Disk spill metric| +|[#11561](https://github.com/NVIDIA/spark-rapids/pull/11561)|Add in a basic plugin for dataframe UDF support in Apache Spark| +|[#11563](https://github.com/NVIDIA/spark-rapids/pull/11563)|Fix the latest merge conflict in integration tests| +|[#11542](https://github.com/NVIDIA/spark-rapids/pull/11542)|Update rapids JNI and private dependency to 24.12.0-SNAPSHOT [skip ci]| +|[#11493](https://github.com/NVIDIA/spark-rapids/pull/11493)|Support legacy mode for yyyymmdd format| ## Release 24.10 @@ -69,15 +239,21 @@ Generated on 2024-10-31 ### PRs ||| |:---|:---| +|[#11683](https://github.com/NVIDIA/spark-rapids/pull/11683)|[DOC] update download page for 2410 hot fix release [skip ci]| +|[#11680](https://github.com/NVIDIA/spark-rapids/pull/11680)|Update latest changelog [skip ci]| +|[#11678](https://github.com/NVIDIA/spark-rapids/pull/11678)|Update version to 24.10.1-SNAPSHOT [skip ci]| |[#11676](https://github.com/NVIDIA/spark-rapids/pull/11676)| Fix race condition with Parquet filter pushdown modifying shared hadoop Configuration| |[#11626](https://github.com/NVIDIA/spark-rapids/pull/11626)|Update latest changelog [skip ci]| |[#11624](https://github.com/NVIDIA/spark-rapids/pull/11624)|Update the download link [skip ci]| |[#11577](https://github.com/NVIDIA/spark-rapids/pull/11577)|Update latest changelog [skip ci]| |[#11576](https://github.com/NVIDIA/spark-rapids/pull/11576)|Update rapids JNI and private dependency to 24.10.0| |[#11582](https://github.com/NVIDIA/spark-rapids/pull/11582)|[DOC] update doc for 24.10 release [skip ci]| +|[#11414](https://github.com/NVIDIA/spark-rapids/pull/11414)|Fix `collection_ops_tests` for Spark 4.0| |[#11588](https://github.com/NVIDIA/spark-rapids/pull/11588)|backport fixes of #11573 to branch 24.10| |[#11569](https://github.com/NVIDIA/spark-rapids/pull/11569)|Have "dump always" dump input files before trying to decode them| +|[#11544](https://github.com/NVIDIA/spark-rapids/pull/11544)|Update test case related to LEACY datetime format to unblock nightly CI| |[#11567](https://github.com/NVIDIA/spark-rapids/pull/11567)|Fix test case unix_timestamp(col, 'yyyyMMdd') failed for Africa/Casablanca timezone and LEGACY mode| +|[#11519](https://github.com/NVIDIA/spark-rapids/pull/11519)|Spark 4: Fix parquet_test.py| |[#11496](https://github.com/NVIDIA/spark-rapids/pull/11496)|Update test now that code is fixed| |[#11548](https://github.com/NVIDIA/spark-rapids/pull/11548)|Fix negative rs. shuffle write time| |[#11545](https://github.com/NVIDIA/spark-rapids/pull/11545)|Update test case related to LEACY datetime format to unblock nightly CI| @@ -157,215 +333,6 @@ Generated on 2024-10-31 |[#11280](https://github.com/NVIDIA/spark-rapids/pull/11280)|Asynchronously copy table data to the host during shuffle| |[#11258](https://github.com/NVIDIA/spark-rapids/pull/11258)|Explicitly disable ANSI mode for ast_test.py| |[#11267](https://github.com/NVIDIA/spark-rapids/pull/11267)|Update the rapids JNI and private dependency version to 24.10.0-SNAPSHOT| -|[#11241](https://github.com/NVIDIA/spark-rapids/pull/11241)|Auto merge PRs to branch-24.10 from branch-24.08 [skip ci]| -|[#11231](https://github.com/NVIDIA/spark-rapids/pull/11231)|Cache dependencies for scala 2.13 [skip ci]| - -## Release 24.08 - -### Features -||| -|:---|:---| -|[#9259](https://github.com/NVIDIA/spark-rapids/issues/9259)|[FEA] Create Spark 4.0.0 shim and build env| -|[#10366](https://github.com/NVIDIA/spark-rapids/issues/10366)|[FEA] It would be nice if we could support Hive-style write bucketing table| -|[#10987](https://github.com/NVIDIA/spark-rapids/issues/10987)|[FEA] Implement lore framework to support all operators.| -|[#11087](https://github.com/NVIDIA/spark-rapids/issues/11087)|[FEA] Support regex pattern with brackets when rewrite to PrefixRange patten in rlike| -|[#22](https://github.com/NVIDIA/spark-rapids/issues/22)|[FEA] Add support for bucketed writes| -|[#9939](https://github.com/NVIDIA/spark-rapids/issues/9939)|[FEA] `GpuInsertIntoHiveTable` supports parquet format| - -### Performance -||| -|:---|:---| -|[#8750](https://github.com/NVIDIA/spark-rapids/issues/8750)|[FEA] Rework GpuSubstringIndex to use cudf::slice_strings| -|[#7404](https://github.com/NVIDIA/spark-rapids/issues/7404)|[FEA] explore a hash agg passthrough on partial aggregates| -|[#10976](https://github.com/NVIDIA/spark-rapids/issues/10976)|Rewrite `pattern1|pattern2|pattern3` to multiple contains in `rlike`| - -### Bugs Fixed -||| -|:---|:---| -|[#11287](https://github.com/NVIDIA/spark-rapids/issues/11287)|[BUG] String split APIs on empty string produce incorrect result| -|[#11270](https://github.com/NVIDIA/spark-rapids/issues/11270)|[BUG] test_regexp_replace[DATAGEN_SEED=1722297411, TZ=UTC] hanging there forever in pre-merge CI intermittently| -|[#9682](https://github.com/NVIDIA/spark-rapids/issues/9682)|[BUG] Casting FLOAT64 to DECIMAL(12,7) produces different rows from Apache Spark CPU| -|[#10809](https://github.com/NVIDIA/spark-rapids/issues/10809)|[BUG] cast(9.95 as decimal(3,1)), actual: 9.9, expected: 10.0| -|[#11266](https://github.com/NVIDIA/spark-rapids/issues/11266)|[BUG] test_broadcast_hash_join_constant_keys failed in databricks runtimes| -|[#11243](https://github.com/NVIDIA/spark-rapids/issues/11243)|[BUG] ArrayIndexOutOfBoundsException on a left outer join| -|[#11030](https://github.com/NVIDIA/spark-rapids/issues/11030)|Fix tests failures in string_test.py| -|[#11245](https://github.com/NVIDIA/spark-rapids/issues/11245)|[BUG] mvn verify for the source-javadoc fails and no pre-merge check catches it| -|[#11223](https://github.com/NVIDIA/spark-rapids/issues/11223)|[BUG] Remove unreferenced `CUDF_VER=xxx` in the CI script| -|[#11114](https://github.com/NVIDIA/spark-rapids/issues/11114)|[BUG] Update nightly tests for Scala 2.13 to use JDK 17 only| -|[#11229](https://github.com/NVIDIA/spark-rapids/issues/11229)|[BUG] test_delta_name_column_mapping_no_field_ids fails on Spark | -|[#11031](https://github.com/NVIDIA/spark-rapids/issues/11031)|Fix tests failures in multiple files | -|[#10948](https://github.com/NVIDIA/spark-rapids/issues/10948)|Figure out why `MapFromArrays ` appears in the tests for hive parquet write| -|[#11018](https://github.com/NVIDIA/spark-rapids/issues/11018)|Fix tests failures in hash_aggregate_test.py| -|[#11173](https://github.com/NVIDIA/spark-rapids/issues/11173)|[BUG] The `rs. serialization time` metric is misleading| -|[#11017](https://github.com/NVIDIA/spark-rapids/issues/11017)|Fix tests failures in url_test.py| -|[#11201](https://github.com/NVIDIA/spark-rapids/issues/11201)|[BUG] Delta Lake tables with name mapping can throw exceptions on read| -|[#11175](https://github.com/NVIDIA/spark-rapids/issues/11175)|[BUG] Clean up unused and duplicated 'org/roaringbitmap' folder in the spark3xx shims| -|[#11196](https://github.com/NVIDIA/spark-rapids/issues/11196)|[BUG] pipeline failed due to class not found exception: NoClassDefFoundError: com/nvidia/spark/rapids/GpuScalar| -|[#11189](https://github.com/NVIDIA/spark-rapids/issues/11189)|[BUG] regression in NDS after PR #11170| -|[#11167](https://github.com/NVIDIA/spark-rapids/issues/11167)|[BUG] UnsupportedOperationException during delta write with `optimize()`| -|[#11172](https://github.com/NVIDIA/spark-rapids/issues/11172)|[BUG] `get_json_object` returns wrong output with wildcard path| -|[#11148](https://github.com/NVIDIA/spark-rapids/issues/11148)|[BUG] Integration test `test_write_hive_bucketed_table` fails| -|[#11155](https://github.com/NVIDIA/spark-rapids/issues/11155)|[BUG] ArrayIndexOutOfBoundsException in BatchWithPartitionData.splitColumnarBatch| -|[#11152](https://github.com/NVIDIA/spark-rapids/issues/11152)|[BUG] LORE dumping consumes too much memory.| -|[#11029](https://github.com/NVIDIA/spark-rapids/issues/11029)|Fix tests failures in subquery_test.py| -|[#11150](https://github.com/NVIDIA/spark-rapids/issues/11150)|[BUG] hive_parquet_write_test.py::test_insert_hive_bucketed_table failure| -|[#11070](https://github.com/NVIDIA/spark-rapids/issues/11070)|[BUG] numpy2 fail fastparquet cases: numpy.dtype size changed| -|[#11136](https://github.com/NVIDIA/spark-rapids/issues/11136)|UnaryPositive expression doesn't extend UnaryExpression| -|[#11122](https://github.com/NVIDIA/spark-rapids/issues/11122)|[BUG] UT MetricRange failed 651070526 was not less than 1.5E8 in spark313| -|[#11119](https://github.com/NVIDIA/spark-rapids/issues/11119)|[BUG] window_function_test.py::test_window_group_limits_fallback_for_row_number fails in a distributed environment| -|[#11023](https://github.com/NVIDIA/spark-rapids/issues/11023)|Fix tests failures in dpp_test.py| -|[#11026](https://github.com/NVIDIA/spark-rapids/issues/11026)|Fix tests failures in map_test.py| -|[#11020](https://github.com/NVIDIA/spark-rapids/issues/11020)|Fix tests failures in grouping_sets_test.py| -|[#11113](https://github.com/NVIDIA/spark-rapids/issues/11113)|[BUG] Update premerge tests for Scala 2.13 to use JDK 17 only| -|[#11027](https://github.com/NVIDIA/spark-rapids/issues/11027)|Fix tests failures in sort_test.py| -|[#10775](https://github.com/NVIDIA/spark-rapids/issues/10775)|[BUG] Issues found by Spark UT Framework on RapidsStringExpressionsSuite| -|[#11033](https://github.com/NVIDIA/spark-rapids/issues/11033)|[BUG] CICD failed a case: cmp_test.py::test_empty_filter[>]| -|[#11103](https://github.com/NVIDIA/spark-rapids/issues/11103)|[BUG] UCX Shuffle With scala.MatchError | -|[#11007](https://github.com/NVIDIA/spark-rapids/issues/11007)|Fix tests failures in array_test.py| -|[#10801](https://github.com/NVIDIA/spark-rapids/issues/10801)|[BUG] JDK17 nightly build after Spark UT Framework is merged| -|[#11019](https://github.com/NVIDIA/spark-rapids/issues/11019)|Fix tests failures in window_function_test.py| -|[#11063](https://github.com/NVIDIA/spark-rapids/issues/11063)|[BUG] op time for GpuCoalesceBatches is more than actual| -|[#11006](https://github.com/NVIDIA/spark-rapids/issues/11006)|Fix test failures in arithmetic_ops_test.py| -|[#10995](https://github.com/NVIDIA/spark-rapids/issues/10995)|Fallback TimeZoneAwareExpression that only support UTC with zoneId instead of timeZone config| -|[#8652](https://github.com/NVIDIA/spark-rapids/issues/8652)|[BUG] array_item test failures on Spark 3.3.x| -|[#11053](https://github.com/NVIDIA/spark-rapids/issues/11053)|[BUG] Build on Databricks 330 fails| -|[#10925](https://github.com/NVIDIA/spark-rapids/issues/10925)| Concat cannot accept no parameter| -|[#10975](https://github.com/NVIDIA/spark-rapids/issues/10975)|[BUG] regex `^.*literal` cannot be rewritten as `contains(literal)` for multiline strings| -|[#10956](https://github.com/NVIDIA/spark-rapids/issues/10956)|[BUG] hive_parquet_write_test.py: test_write_compressed_parquet_into_hive_table integration test failures| -|[#10772](https://github.com/NVIDIA/spark-rapids/issues/10772)|[BUG] Issues found by Spark UT Framework on RapidsDataFrameAggregateSuite| -|[#10986](https://github.com/NVIDIA/spark-rapids/issues/10986)|[BUG]Cast from string to float using hand-picked values failed in CastOpSuite| -|[#10972](https://github.com/NVIDIA/spark-rapids/issues/10972)|Spark 4.0 compile errors | -|[#10794](https://github.com/NVIDIA/spark-rapids/issues/10794)|[BUG] Incorrect cast of string columns containing various infinity notations with trailing spaces | -|[#10964](https://github.com/NVIDIA/spark-rapids/issues/10964)|[BUG] Improve stability of pre-merge jenkinsfile| -|[#10714](https://github.com/NVIDIA/spark-rapids/issues/10714)|Signature changed for `PythonUDFRunner.writeUDFs` | -|[#10712](https://github.com/NVIDIA/spark-rapids/issues/10712)|[AUDIT] BatchScanExec/DataSourceV2Relation to group splits by join keys if they differ from partition keys| -|[#10673](https://github.com/NVIDIA/spark-rapids/issues/10673)|[AUDIT] Rename plan nodes for PythonMapInArrowExec| -|[#10710](https://github.com/NVIDIA/spark-rapids/issues/10710)|[AUDIT] `uncacheTableOrView` changed in CommandUtils | -|[#10711](https://github.com/NVIDIA/spark-rapids/issues/10711)|[AUDIT] Match DataSourceV2ScanExecBase changes to groupPartitions method | -|[#10669](https://github.com/NVIDIA/spark-rapids/issues/10669)|Supporting broadcast of multiple filtering keys in DynamicPruning | - -### PRs -||| -|:---|:---| -|[#11400](https://github.com/NVIDIA/spark-rapids/pull/11400)|[DOC] update notes in download page for the decompressing gzip issue [skip ci]| -|[#11355](https://github.com/NVIDIA/spark-rapids/pull/11355)|Update changelog for the v24.08 release [skip ci]| -|[#11353](https://github.com/NVIDIA/spark-rapids/pull/11353)|Update download doc for v24.08.1 [skip ci]| -|[#11352](https://github.com/NVIDIA/spark-rapids/pull/11352)|Update version to 24.08.1-SNAPSHOT [skip ci]| -|[#11337](https://github.com/NVIDIA/spark-rapids/pull/11337)|Update changelog for the v24.08 release [skip ci]| -|[#11335](https://github.com/NVIDIA/spark-rapids/pull/11335)|Fix Delta Lake truncation of min/max string values| -|[#11304](https://github.com/NVIDIA/spark-rapids/pull/11304)|Update changelog for v24.08.0 release [skip ci]| -|[#11303](https://github.com/NVIDIA/spark-rapids/pull/11303)|Update rapids JNI and private dependency to 24.08.0| -|[#11296](https://github.com/NVIDIA/spark-rapids/pull/11296)|[DOC] update doc for 2408 release [skip CI]| -|[#11309](https://github.com/NVIDIA/spark-rapids/pull/11309)|[Doc ]Update lore doc about the range [skip ci]| -|[#11292](https://github.com/NVIDIA/spark-rapids/pull/11292)|Add work around for string split with empty input.| -|[#11278](https://github.com/NVIDIA/spark-rapids/pull/11278)|Fix formatting of advanced configs doc| -|[#10917](https://github.com/NVIDIA/spark-rapids/pull/10917)|Adopt changes from JNI for casting from float to decimal| -|[#11269](https://github.com/NVIDIA/spark-rapids/pull/11269)|Revert "upgrade ucx to 1.17.0"| -|[#11260](https://github.com/NVIDIA/spark-rapids/pull/11260)|Mitigate intermittent test_buckets and shuffle_smoke_test OOM issue| -|[#11268](https://github.com/NVIDIA/spark-rapids/pull/11268)|Fix degenerate conditional nested loop join detection| -|[#11244](https://github.com/NVIDIA/spark-rapids/pull/11244)|Fix ArrayIndexOutOfBoundsException on join counts with constant join keys| -|[#11259](https://github.com/NVIDIA/spark-rapids/pull/11259)|CI Docker to support integration tests with Rocky OS + jdk17 [skip ci]| -|[#11247](https://github.com/NVIDIA/spark-rapids/pull/11247)|Fix `string_test.py` errors on Spark 4.0| -|[#11246](https://github.com/NVIDIA/spark-rapids/pull/11246)|Rework Maven Source Plugin Skip| -|[#11149](https://github.com/NVIDIA/spark-rapids/pull/11149)|Rework on substring index| -|[#11236](https://github.com/NVIDIA/spark-rapids/pull/11236)|Remove the unused vars from the version-def CI script| -|[#11237](https://github.com/NVIDIA/spark-rapids/pull/11237)|Fork jvm for maven-source-plugin| -|[#11200](https://github.com/NVIDIA/spark-rapids/pull/11200)|Multi-get_json_object| -|[#11230](https://github.com/NVIDIA/spark-rapids/pull/11230)|Skip test where Delta Lake may not be fully compatible with Spark| -|[#11220](https://github.com/NVIDIA/spark-rapids/pull/11220)|Avoid failing spark bug SPARK-44242 while generate run_dir| -|[#11226](https://github.com/NVIDIA/spark-rapids/pull/11226)|Fix auto merge conflict 11212| -|[#11129](https://github.com/NVIDIA/spark-rapids/pull/11129)|Spark 4: Fix miscellaneous tests including logic, repart, hive_delimited.| -|[#11163](https://github.com/NVIDIA/spark-rapids/pull/11163)|Support `MapFromArrays` on GPU| -|[#11219](https://github.com/NVIDIA/spark-rapids/pull/11219)|Fix hash_aggregate_test.py to run with ANSI enabled| -|[#11186](https://github.com/NVIDIA/spark-rapids/pull/11186)|from_json Json to Struct Exception Logging| -|[#11180](https://github.com/NVIDIA/spark-rapids/pull/11180)|More accurate estimation for the result serialization time in RapidsShuffleThreadedWriterBase| -|[#11194](https://github.com/NVIDIA/spark-rapids/pull/11194)|Fix ANSI mode test failures in url_test.py| -|[#11202](https://github.com/NVIDIA/spark-rapids/pull/11202)|Fix read from Delta Lake table with name column mapping and missing Parquet IDs| -|[#11185](https://github.com/NVIDIA/spark-rapids/pull/11185)|Fix multi-release jar problem| -|[#11144](https://github.com/NVIDIA/spark-rapids/pull/11144)|Build the Scala2.13 dist jar with JDK17| -|[#11197](https://github.com/NVIDIA/spark-rapids/pull/11197)|Fix class not found error: com/nvidia/spark/rapids/GpuScalar| -|[#11191](https://github.com/NVIDIA/spark-rapids/pull/11191)|Fix dynamic pruning regression in GpuFileSourceScanExec| -|[#10994](https://github.com/NVIDIA/spark-rapids/pull/10994)|Add Spark 4.0.0 Build Profile and Other Supporting Changes| -|[#11192](https://github.com/NVIDIA/spark-rapids/pull/11192)|Append new authorized user to blossom-ci whitelist [skip ci]| -|[#11179](https://github.com/NVIDIA/spark-rapids/pull/11179)|Allow more expressions to be tiered| -|[#11141](https://github.com/NVIDIA/spark-rapids/pull/11141)|Enable some Rapids config in RapidsSQLTestsBaseTrait for Spark UT| -|[#11170](https://github.com/NVIDIA/spark-rapids/pull/11170)|Avoid listFiles or inputFiles on relations with static partitioning| -|[#11159](https://github.com/NVIDIA/spark-rapids/pull/11159)|Drop spark31x shims| -|[#10951](https://github.com/NVIDIA/spark-rapids/pull/10951)|Case when performance improvement: reduce the `copy_if_else`| -|[#11165](https://github.com/NVIDIA/spark-rapids/pull/11165)|Fix some GpuBroadcastToRowExec by not dropping columns| -|[#11126](https://github.com/NVIDIA/spark-rapids/pull/11126)|Coalesce batches after a logical coalesce operation| -|[#11164](https://github.com/NVIDIA/spark-rapids/pull/11164)|fix the bucketed write error for non-utc cases| -|[#11132](https://github.com/NVIDIA/spark-rapids/pull/11132)|Add deletion vector metrics for low shuffle merge.| -|[#11156](https://github.com/NVIDIA/spark-rapids/pull/11156)|Fix batch splitting for partition column size on row-count-only batches| -|[#11153](https://github.com/NVIDIA/spark-rapids/pull/11153)|Fix LORE dump oom.| -|[#11102](https://github.com/NVIDIA/spark-rapids/pull/11102)|Fix ANSI mode failures in subquery_test.py| -|[#11151](https://github.com/NVIDIA/spark-rapids/pull/11151)|Fix the test error of the bucketed write for the non-utc case| -|[#11147](https://github.com/NVIDIA/spark-rapids/pull/11147)|upgrade ucx to 1.17.0| -|[#11138](https://github.com/NVIDIA/spark-rapids/pull/11138)|Update fastparquet to 2024.5.0 for numpy2 compatibility| -|[#11137](https://github.com/NVIDIA/spark-rapids/pull/11137)|Handle the change for UnaryPositive now extending RuntimeReplaceable| -|[#11094](https://github.com/NVIDIA/spark-rapids/pull/11094)|Add `HiveHash` support on GPU| -|[#11139](https://github.com/NVIDIA/spark-rapids/pull/11139)|Improve MetricsSuite to allow more gc jitter| -|[#11133](https://github.com/NVIDIA/spark-rapids/pull/11133)|Fix `test_window_group_limits_fallback`| -|[#11097](https://github.com/NVIDIA/spark-rapids/pull/11097)|Fix miscellaneous integ tests for Spark 4| -|[#11118](https://github.com/NVIDIA/spark-rapids/pull/11118)|Fix issue with DPP and AQE on reused broadcast exchanges| -|[#11043](https://github.com/NVIDIA/spark-rapids/pull/11043)|Dataproc serverless test fixes| -|[#10965](https://github.com/NVIDIA/spark-rapids/pull/10965)|Profiler: Disable collecting async allocation events by default| -|[#11117](https://github.com/NVIDIA/spark-rapids/pull/11117)|Update Scala2.13 premerge CI against JDK17| -|[#11084](https://github.com/NVIDIA/spark-rapids/pull/11084)|Introduce LORE framework.| -|[#11099](https://github.com/NVIDIA/spark-rapids/pull/11099)|Spark 4: Handle ANSI mode in sort_test.py| -|[#11115](https://github.com/NVIDIA/spark-rapids/pull/11115)|Fix match error in RapidsShuffleIterator.scala [scala2.13]| -|[#11088](https://github.com/NVIDIA/spark-rapids/pull/11088)|Support regex patterns with brackets when rewriting to PrefixRange pattern in rlike.| -|[#10950](https://github.com/NVIDIA/spark-rapids/pull/10950)|Add a heuristic to skip second or third agg pass| -|[#11048](https://github.com/NVIDIA/spark-rapids/pull/11048)|Fixed array_tests for Spark 4.0.0| -|[#11049](https://github.com/NVIDIA/spark-rapids/pull/11049)|Fix some cast_tests for Spark 4.0.0| -|[#11066](https://github.com/NVIDIA/spark-rapids/pull/11066)|Replaced spark3xx-common references to spark-shared| -|[#11083](https://github.com/NVIDIA/spark-rapids/pull/11083)|Exclude a case based on JDK version in Spark UT| -|[#10997](https://github.com/NVIDIA/spark-rapids/pull/10997)|Fix some test issues in Spark UT and keep RapidsTestSettings update-to-date| -|[#11073](https://github.com/NVIDIA/spark-rapids/pull/11073)|Disable ANSI mode for window function tests| -|[#11076](https://github.com/NVIDIA/spark-rapids/pull/11076)|Improve the diagnostics for 'conv' fallback explain| -|[#11092](https://github.com/NVIDIA/spark-rapids/pull/11092)|Add GpuBucketingUtils shim to Spark 4.0.0| -|[#11062](https://github.com/NVIDIA/spark-rapids/pull/11062)|fix duplicate counted metrics like op time for GpuCoalesceBatches| -|[#11044](https://github.com/NVIDIA/spark-rapids/pull/11044)|Fixed Failing tests in arithmetic_ops_tests for Spark 4.0.0| -|[#11086](https://github.com/NVIDIA/spark-rapids/pull/11086)|upgrade blossom-ci actions version [skip ci]| -|[#10957](https://github.com/NVIDIA/spark-rapids/pull/10957)|Support bucketing write for GPU| -|[#10979](https://github.com/NVIDIA/spark-rapids/pull/10979)|[FEA] Introduce low shuffle merge.| -|[#10996](https://github.com/NVIDIA/spark-rapids/pull/10996)|Fallback non-UTC TimeZoneAwareExpression with zoneId| -|[#11072](https://github.com/NVIDIA/spark-rapids/pull/11072)|Workaround numpy2 failed fastparquet compatibility tests| -|[#11046](https://github.com/NVIDIA/spark-rapids/pull/11046)|Calculate parallelism to speed up pre-merge CI| -|[#11054](https://github.com/NVIDIA/spark-rapids/pull/11054)|fix flaky array_item test failures| -|[#11051](https://github.com/NVIDIA/spark-rapids/pull/11051)|[FEA] Increase parallelism of deltalake test on databricks| -|[#10993](https://github.com/NVIDIA/spark-rapids/pull/10993)|`binary-dedupe` changes for Spark 4.0.0| -|[#11060](https://github.com/NVIDIA/spark-rapids/pull/11060)|Add in the ability to fingerprint JSON columns| -|[#11059](https://github.com/NVIDIA/spark-rapids/pull/11059)|Revert "Add in the ability to fingerprint JSON columns (#11002)" [skip ci]| -|[#11039](https://github.com/NVIDIA/spark-rapids/pull/11039)|Concat() Exception bug fix| -|[#11002](https://github.com/NVIDIA/spark-rapids/pull/11002)|Add in the ability to fingerprint JSON columns| -|[#10977](https://github.com/NVIDIA/spark-rapids/pull/10977)|Rewrite multiple literal choice regex to multiple contains in rlike| -|[#11035](https://github.com/NVIDIA/spark-rapids/pull/11035)|Fix auto merge conflict 11034 [skip ci]| -|[#11040](https://github.com/NVIDIA/spark-rapids/pull/11040)|Append new authorized user to blossom-ci whitelist [skip ci]| -|[#11036](https://github.com/NVIDIA/spark-rapids/pull/11036)|Update blossom-ci ACL to secure format [skip ci]| -|[#11032](https://github.com/NVIDIA/spark-rapids/pull/11032)|Fix a hive write test failure for Spark 350| -|[#10998](https://github.com/NVIDIA/spark-rapids/pull/10998)|Improve log to print more lines in build [skip ci]| -|[#10992](https://github.com/NVIDIA/spark-rapids/pull/10992)|Addressing the Named Parameter change in Spark 4.0.0| -|[#10943](https://github.com/NVIDIA/spark-rapids/pull/10943)|Fix Spark UT issues in RapidsDataFrameAggregateSuite| -|[#10963](https://github.com/NVIDIA/spark-rapids/pull/10963)|Add rapids configs to enable GPU running in Spark UT| -|[#10978](https://github.com/NVIDIA/spark-rapids/pull/10978)|More compilation fixes for Spark 4.0.0| -|[#10953](https://github.com/NVIDIA/spark-rapids/pull/10953)|Speed up the integration tests by running them in parallel on the Databricks cluster| -|[#10958](https://github.com/NVIDIA/spark-rapids/pull/10958)|Fix a hive write test failure| -|[#10970](https://github.com/NVIDIA/spark-rapids/pull/10970)|Move Support for `RaiseError` to a Shim Excluding Spark 4.0.0| -|[#10966](https://github.com/NVIDIA/spark-rapids/pull/10966)|Add default value for REF of premerge jenkinsfile to avoid bad overwritten [skip ci]| -|[#10959](https://github.com/NVIDIA/spark-rapids/pull/10959)|Add new ID to blossom-ci allow list [skip ci]| -|[#10952](https://github.com/NVIDIA/spark-rapids/pull/10952)|Add shims to take care of the signature change for writeUDFs in PythonUDFRunner| -|[#10931](https://github.com/NVIDIA/spark-rapids/pull/10931)|Add Support for Renaming of PythonMapInArrow| -|[#10949](https://github.com/NVIDIA/spark-rapids/pull/10949)|Change dependency version to 24.08.0-SNAPSHOT| -|[#10857](https://github.com/NVIDIA/spark-rapids/pull/10857)|[Spark 4.0] Account for `PartitionedFileUtil.splitFiles` signature change.| -|[#10912](https://github.com/NVIDIA/spark-rapids/pull/10912)|GpuInsertIntoHiveTable supports parquet format| -|[#10863](https://github.com/NVIDIA/spark-rapids/pull/10863)|[Spark 4.0] Account for `CommandUtils.uncacheTableOrView` signature change.| -|[#10944](https://github.com/NVIDIA/spark-rapids/pull/10944)|Added Shim for BatchScanExec to Support Spark 4.0| -|[#10946](https://github.com/NVIDIA/spark-rapids/pull/10946)|Unarchive Spark test jar for spark.read(ability)| -|[#10945](https://github.com/NVIDIA/spark-rapids/pull/10945)|Add Support for Multiple Filtering Keys for Subquery Broadcast| -|[#10871](https://github.com/NVIDIA/spark-rapids/pull/10871)|Add classloader diagnostics to initShuffleManager error message| -|[#10933](https://github.com/NVIDIA/spark-rapids/pull/10933)|Fixed Databricks build| -|[#10929](https://github.com/NVIDIA/spark-rapids/pull/10929)|Append new authorized user to blossom-ci whitelist [skip ci]| ## Older Releases Changelog of older releases can be found at [docs/archives](/docs/archives) diff --git a/docs/archives/CHANGELOG_24.02-to-24.06.md b/docs/archives/CHANGELOG_24.02-to-24.08.md similarity index 74% rename from docs/archives/CHANGELOG_24.02-to-24.06.md rename to docs/archives/CHANGELOG_24.02-to-24.08.md index d95307a1efe..075b42966d4 100644 --- a/docs/archives/CHANGELOG_24.02-to-24.06.md +++ b/docs/archives/CHANGELOG_24.02-to-24.08.md @@ -1,5 +1,212 @@ # Change log -Generated on 2024-10-09 +Generated on 2024-12-10 +## Release 24.08 + +### Features +||| +|:---|:---| +|[#9259](https://github.com/NVIDIA/spark-rapids/issues/9259)|[FEA] Create Spark 4.0.0 shim and build env| +|[#10366](https://github.com/NVIDIA/spark-rapids/issues/10366)|[FEA] It would be nice if we could support Hive-style write bucketing table| +|[#10987](https://github.com/NVIDIA/spark-rapids/issues/10987)|[FEA] Implement lore framework to support all operators.| +|[#11087](https://github.com/NVIDIA/spark-rapids/issues/11087)|[FEA] Support regex pattern with brackets when rewrite to PrefixRange patten in rlike| +|[#22](https://github.com/NVIDIA/spark-rapids/issues/22)|[FEA] Add support for bucketed writes| +|[#9939](https://github.com/NVIDIA/spark-rapids/issues/9939)|[FEA] `GpuInsertIntoHiveTable` supports parquet format| + +### Performance +||| +|:---|:---| +|[#8750](https://github.com/NVIDIA/spark-rapids/issues/8750)|[FEA] Rework GpuSubstringIndex to use cudf::slice_strings| +|[#7404](https://github.com/NVIDIA/spark-rapids/issues/7404)|[FEA] explore a hash agg passthrough on partial aggregates| +|[#10976](https://github.com/NVIDIA/spark-rapids/issues/10976)|Rewrite `pattern1|pattern2|pattern3` to multiple contains in `rlike`| + +### Bugs Fixed +||| +|:---|:---| +|[#11287](https://github.com/NVIDIA/spark-rapids/issues/11287)|[BUG] String split APIs on empty string produce incorrect result| +|[#11270](https://github.com/NVIDIA/spark-rapids/issues/11270)|[BUG] test_regexp_replace[DATAGEN_SEED=1722297411, TZ=UTC] hanging there forever in pre-merge CI intermittently| +|[#9682](https://github.com/NVIDIA/spark-rapids/issues/9682)|[BUG] Casting FLOAT64 to DECIMAL(12,7) produces different rows from Apache Spark CPU| +|[#10809](https://github.com/NVIDIA/spark-rapids/issues/10809)|[BUG] cast(9.95 as decimal(3,1)), actual: 9.9, expected: 10.0| +|[#11266](https://github.com/NVIDIA/spark-rapids/issues/11266)|[BUG] test_broadcast_hash_join_constant_keys failed in databricks runtimes| +|[#11243](https://github.com/NVIDIA/spark-rapids/issues/11243)|[BUG] ArrayIndexOutOfBoundsException on a left outer join| +|[#11030](https://github.com/NVIDIA/spark-rapids/issues/11030)|Fix tests failures in string_test.py| +|[#11245](https://github.com/NVIDIA/spark-rapids/issues/11245)|[BUG] mvn verify for the source-javadoc fails and no pre-merge check catches it| +|[#11223](https://github.com/NVIDIA/spark-rapids/issues/11223)|[BUG] Remove unreferenced `CUDF_VER=xxx` in the CI script| +|[#11114](https://github.com/NVIDIA/spark-rapids/issues/11114)|[BUG] Update nightly tests for Scala 2.13 to use JDK 17 only| +|[#11229](https://github.com/NVIDIA/spark-rapids/issues/11229)|[BUG] test_delta_name_column_mapping_no_field_ids fails on Spark | +|[#11031](https://github.com/NVIDIA/spark-rapids/issues/11031)|Fix tests failures in multiple files | +|[#10948](https://github.com/NVIDIA/spark-rapids/issues/10948)|Figure out why `MapFromArrays ` appears in the tests for hive parquet write| +|[#11018](https://github.com/NVIDIA/spark-rapids/issues/11018)|Fix tests failures in hash_aggregate_test.py| +|[#11173](https://github.com/NVIDIA/spark-rapids/issues/11173)|[BUG] The `rs. serialization time` metric is misleading| +|[#11017](https://github.com/NVIDIA/spark-rapids/issues/11017)|Fix tests failures in url_test.py| +|[#11201](https://github.com/NVIDIA/spark-rapids/issues/11201)|[BUG] Delta Lake tables with name mapping can throw exceptions on read| +|[#11175](https://github.com/NVIDIA/spark-rapids/issues/11175)|[BUG] Clean up unused and duplicated 'org/roaringbitmap' folder in the spark3xx shims| +|[#11196](https://github.com/NVIDIA/spark-rapids/issues/11196)|[BUG] pipeline failed due to class not found exception: NoClassDefFoundError: com/nvidia/spark/rapids/GpuScalar| +|[#11189](https://github.com/NVIDIA/spark-rapids/issues/11189)|[BUG] regression in NDS after PR #11170| +|[#11167](https://github.com/NVIDIA/spark-rapids/issues/11167)|[BUG] UnsupportedOperationException during delta write with `optimize()`| +|[#11172](https://github.com/NVIDIA/spark-rapids/issues/11172)|[BUG] `get_json_object` returns wrong output with wildcard path| +|[#11148](https://github.com/NVIDIA/spark-rapids/issues/11148)|[BUG] Integration test `test_write_hive_bucketed_table` fails| +|[#11155](https://github.com/NVIDIA/spark-rapids/issues/11155)|[BUG] ArrayIndexOutOfBoundsException in BatchWithPartitionData.splitColumnarBatch| +|[#11152](https://github.com/NVIDIA/spark-rapids/issues/11152)|[BUG] LORE dumping consumes too much memory.| +|[#11029](https://github.com/NVIDIA/spark-rapids/issues/11029)|Fix tests failures in subquery_test.py| +|[#11150](https://github.com/NVIDIA/spark-rapids/issues/11150)|[BUG] hive_parquet_write_test.py::test_insert_hive_bucketed_table failure| +|[#11070](https://github.com/NVIDIA/spark-rapids/issues/11070)|[BUG] numpy2 fail fastparquet cases: numpy.dtype size changed| +|[#11136](https://github.com/NVIDIA/spark-rapids/issues/11136)|UnaryPositive expression doesn't extend UnaryExpression| +|[#11122](https://github.com/NVIDIA/spark-rapids/issues/11122)|[BUG] UT MetricRange failed 651070526 was not less than 1.5E8 in spark313| +|[#11119](https://github.com/NVIDIA/spark-rapids/issues/11119)|[BUG] window_function_test.py::test_window_group_limits_fallback_for_row_number fails in a distributed environment| +|[#11023](https://github.com/NVIDIA/spark-rapids/issues/11023)|Fix tests failures in dpp_test.py| +|[#11026](https://github.com/NVIDIA/spark-rapids/issues/11026)|Fix tests failures in map_test.py| +|[#11020](https://github.com/NVIDIA/spark-rapids/issues/11020)|Fix tests failures in grouping_sets_test.py| +|[#11113](https://github.com/NVIDIA/spark-rapids/issues/11113)|[BUG] Update premerge tests for Scala 2.13 to use JDK 17 only| +|[#11027](https://github.com/NVIDIA/spark-rapids/issues/11027)|Fix tests failures in sort_test.py| +|[#10775](https://github.com/NVIDIA/spark-rapids/issues/10775)|[BUG] Issues found by Spark UT Framework on RapidsStringExpressionsSuite| +|[#11033](https://github.com/NVIDIA/spark-rapids/issues/11033)|[BUG] CICD failed a case: cmp_test.py::test_empty_filter[>]| +|[#11103](https://github.com/NVIDIA/spark-rapids/issues/11103)|[BUG] UCX Shuffle With scala.MatchError | +|[#11007](https://github.com/NVIDIA/spark-rapids/issues/11007)|Fix tests failures in array_test.py| +|[#10801](https://github.com/NVIDIA/spark-rapids/issues/10801)|[BUG] JDK17 nightly build after Spark UT Framework is merged| +|[#11019](https://github.com/NVIDIA/spark-rapids/issues/11019)|Fix tests failures in window_function_test.py| +|[#11063](https://github.com/NVIDIA/spark-rapids/issues/11063)|[BUG] op time for GpuCoalesceBatches is more than actual| +|[#11006](https://github.com/NVIDIA/spark-rapids/issues/11006)|Fix test failures in arithmetic_ops_test.py| +|[#10995](https://github.com/NVIDIA/spark-rapids/issues/10995)|Fallback TimeZoneAwareExpression that only support UTC with zoneId instead of timeZone config| +|[#8652](https://github.com/NVIDIA/spark-rapids/issues/8652)|[BUG] array_item test failures on Spark 3.3.x| +|[#11053](https://github.com/NVIDIA/spark-rapids/issues/11053)|[BUG] Build on Databricks 330 fails| +|[#10925](https://github.com/NVIDIA/spark-rapids/issues/10925)| Concat cannot accept no parameter| +|[#10975](https://github.com/NVIDIA/spark-rapids/issues/10975)|[BUG] regex `^.*literal` cannot be rewritten as `contains(literal)` for multiline strings| +|[#10956](https://github.com/NVIDIA/spark-rapids/issues/10956)|[BUG] hive_parquet_write_test.py: test_write_compressed_parquet_into_hive_table integration test failures| +|[#10772](https://github.com/NVIDIA/spark-rapids/issues/10772)|[BUG] Issues found by Spark UT Framework on RapidsDataFrameAggregateSuite| +|[#10986](https://github.com/NVIDIA/spark-rapids/issues/10986)|[BUG]Cast from string to float using hand-picked values failed in CastOpSuite| +|[#10972](https://github.com/NVIDIA/spark-rapids/issues/10972)|Spark 4.0 compile errors | +|[#10794](https://github.com/NVIDIA/spark-rapids/issues/10794)|[BUG] Incorrect cast of string columns containing various infinity notations with trailing spaces | +|[#10964](https://github.com/NVIDIA/spark-rapids/issues/10964)|[BUG] Improve stability of pre-merge jenkinsfile| +|[#10714](https://github.com/NVIDIA/spark-rapids/issues/10714)|Signature changed for `PythonUDFRunner.writeUDFs` | +|[#10712](https://github.com/NVIDIA/spark-rapids/issues/10712)|[AUDIT] BatchScanExec/DataSourceV2Relation to group splits by join keys if they differ from partition keys| +|[#10673](https://github.com/NVIDIA/spark-rapids/issues/10673)|[AUDIT] Rename plan nodes for PythonMapInArrowExec| +|[#10710](https://github.com/NVIDIA/spark-rapids/issues/10710)|[AUDIT] `uncacheTableOrView` changed in CommandUtils | +|[#10711](https://github.com/NVIDIA/spark-rapids/issues/10711)|[AUDIT] Match DataSourceV2ScanExecBase changes to groupPartitions method | +|[#10669](https://github.com/NVIDIA/spark-rapids/issues/10669)|Supporting broadcast of multiple filtering keys in DynamicPruning | + +### PRs +||| +|:---|:---| +|[#11400](https://github.com/NVIDIA/spark-rapids/pull/11400)|[DOC] update notes in download page for the decompressing gzip issue [skip ci]| +|[#11355](https://github.com/NVIDIA/spark-rapids/pull/11355)|Update changelog for the v24.08 release [skip ci]| +|[#11353](https://github.com/NVIDIA/spark-rapids/pull/11353)|Update download doc for v24.08.1 [skip ci]| +|[#11352](https://github.com/NVIDIA/spark-rapids/pull/11352)|Update version to 24.08.1-SNAPSHOT [skip ci]| +|[#11337](https://github.com/NVIDIA/spark-rapids/pull/11337)|Update changelog for the v24.08 release [skip ci]| +|[#11335](https://github.com/NVIDIA/spark-rapids/pull/11335)|Fix Delta Lake truncation of min/max string values| +|[#11304](https://github.com/NVIDIA/spark-rapids/pull/11304)|Update changelog for v24.08.0 release [skip ci]| +|[#11303](https://github.com/NVIDIA/spark-rapids/pull/11303)|Update rapids JNI and private dependency to 24.08.0| +|[#11296](https://github.com/NVIDIA/spark-rapids/pull/11296)|[DOC] update doc for 2408 release [skip CI]| +|[#11309](https://github.com/NVIDIA/spark-rapids/pull/11309)|[Doc ]Update lore doc about the range [skip ci]| +|[#11292](https://github.com/NVIDIA/spark-rapids/pull/11292)|Add work around for string split with empty input.| +|[#11278](https://github.com/NVIDIA/spark-rapids/pull/11278)|Fix formatting of advanced configs doc| +|[#10917](https://github.com/NVIDIA/spark-rapids/pull/10917)|Adopt changes from JNI for casting from float to decimal| +|[#11269](https://github.com/NVIDIA/spark-rapids/pull/11269)|Revert "upgrade ucx to 1.17.0"| +|[#11260](https://github.com/NVIDIA/spark-rapids/pull/11260)|Mitigate intermittent test_buckets and shuffle_smoke_test OOM issue| +|[#11268](https://github.com/NVIDIA/spark-rapids/pull/11268)|Fix degenerate conditional nested loop join detection| +|[#11244](https://github.com/NVIDIA/spark-rapids/pull/11244)|Fix ArrayIndexOutOfBoundsException on join counts with constant join keys| +|[#11259](https://github.com/NVIDIA/spark-rapids/pull/11259)|CI Docker to support integration tests with Rocky OS + jdk17 [skip ci]| +|[#11247](https://github.com/NVIDIA/spark-rapids/pull/11247)|Fix `string_test.py` errors on Spark 4.0| +|[#11246](https://github.com/NVIDIA/spark-rapids/pull/11246)|Rework Maven Source Plugin Skip| +|[#11149](https://github.com/NVIDIA/spark-rapids/pull/11149)|Rework on substring index| +|[#11236](https://github.com/NVIDIA/spark-rapids/pull/11236)|Remove the unused vars from the version-def CI script| +|[#11237](https://github.com/NVIDIA/spark-rapids/pull/11237)|Fork jvm for maven-source-plugin| +|[#11200](https://github.com/NVIDIA/spark-rapids/pull/11200)|Multi-get_json_object| +|[#11230](https://github.com/NVIDIA/spark-rapids/pull/11230)|Skip test where Delta Lake may not be fully compatible with Spark| +|[#11220](https://github.com/NVIDIA/spark-rapids/pull/11220)|Avoid failing spark bug SPARK-44242 while generate run_dir| +|[#11226](https://github.com/NVIDIA/spark-rapids/pull/11226)|Fix auto merge conflict 11212| +|[#11129](https://github.com/NVIDIA/spark-rapids/pull/11129)|Spark 4: Fix miscellaneous tests including logic, repart, hive_delimited.| +|[#11163](https://github.com/NVIDIA/spark-rapids/pull/11163)|Support `MapFromArrays` on GPU| +|[#11219](https://github.com/NVIDIA/spark-rapids/pull/11219)|Fix hash_aggregate_test.py to run with ANSI enabled| +|[#11186](https://github.com/NVIDIA/spark-rapids/pull/11186)|from_json Json to Struct Exception Logging| +|[#11180](https://github.com/NVIDIA/spark-rapids/pull/11180)|More accurate estimation for the result serialization time in RapidsShuffleThreadedWriterBase| +|[#11194](https://github.com/NVIDIA/spark-rapids/pull/11194)|Fix ANSI mode test failures in url_test.py| +|[#11202](https://github.com/NVIDIA/spark-rapids/pull/11202)|Fix read from Delta Lake table with name column mapping and missing Parquet IDs| +|[#11185](https://github.com/NVIDIA/spark-rapids/pull/11185)|Fix multi-release jar problem| +|[#11144](https://github.com/NVIDIA/spark-rapids/pull/11144)|Build the Scala2.13 dist jar with JDK17| +|[#11197](https://github.com/NVIDIA/spark-rapids/pull/11197)|Fix class not found error: com/nvidia/spark/rapids/GpuScalar| +|[#11191](https://github.com/NVIDIA/spark-rapids/pull/11191)|Fix dynamic pruning regression in GpuFileSourceScanExec| +|[#10994](https://github.com/NVIDIA/spark-rapids/pull/10994)|Add Spark 4.0.0 Build Profile and Other Supporting Changes| +|[#11192](https://github.com/NVIDIA/spark-rapids/pull/11192)|Append new authorized user to blossom-ci whitelist [skip ci]| +|[#11179](https://github.com/NVIDIA/spark-rapids/pull/11179)|Allow more expressions to be tiered| +|[#11141](https://github.com/NVIDIA/spark-rapids/pull/11141)|Enable some Rapids config in RapidsSQLTestsBaseTrait for Spark UT| +|[#11170](https://github.com/NVIDIA/spark-rapids/pull/11170)|Avoid listFiles or inputFiles on relations with static partitioning| +|[#11159](https://github.com/NVIDIA/spark-rapids/pull/11159)|Drop spark31x shims| +|[#10951](https://github.com/NVIDIA/spark-rapids/pull/10951)|Case when performance improvement: reduce the `copy_if_else`| +|[#11165](https://github.com/NVIDIA/spark-rapids/pull/11165)|Fix some GpuBroadcastToRowExec by not dropping columns| +|[#11126](https://github.com/NVIDIA/spark-rapids/pull/11126)|Coalesce batches after a logical coalesce operation| +|[#11164](https://github.com/NVIDIA/spark-rapids/pull/11164)|fix the bucketed write error for non-utc cases| +|[#11132](https://github.com/NVIDIA/spark-rapids/pull/11132)|Add deletion vector metrics for low shuffle merge.| +|[#11156](https://github.com/NVIDIA/spark-rapids/pull/11156)|Fix batch splitting for partition column size on row-count-only batches| +|[#11153](https://github.com/NVIDIA/spark-rapids/pull/11153)|Fix LORE dump oom.| +|[#11102](https://github.com/NVIDIA/spark-rapids/pull/11102)|Fix ANSI mode failures in subquery_test.py| +|[#11151](https://github.com/NVIDIA/spark-rapids/pull/11151)|Fix the test error of the bucketed write for the non-utc case| +|[#11147](https://github.com/NVIDIA/spark-rapids/pull/11147)|upgrade ucx to 1.17.0| +|[#11138](https://github.com/NVIDIA/spark-rapids/pull/11138)|Update fastparquet to 2024.5.0 for numpy2 compatibility| +|[#11137](https://github.com/NVIDIA/spark-rapids/pull/11137)|Handle the change for UnaryPositive now extending RuntimeReplaceable| +|[#11094](https://github.com/NVIDIA/spark-rapids/pull/11094)|Add `HiveHash` support on GPU| +|[#11139](https://github.com/NVIDIA/spark-rapids/pull/11139)|Improve MetricsSuite to allow more gc jitter| +|[#11133](https://github.com/NVIDIA/spark-rapids/pull/11133)|Fix `test_window_group_limits_fallback`| +|[#11097](https://github.com/NVIDIA/spark-rapids/pull/11097)|Fix miscellaneous integ tests for Spark 4| +|[#11118](https://github.com/NVIDIA/spark-rapids/pull/11118)|Fix issue with DPP and AQE on reused broadcast exchanges| +|[#11043](https://github.com/NVIDIA/spark-rapids/pull/11043)|Dataproc serverless test fixes| +|[#10965](https://github.com/NVIDIA/spark-rapids/pull/10965)|Profiler: Disable collecting async allocation events by default| +|[#11117](https://github.com/NVIDIA/spark-rapids/pull/11117)|Update Scala2.13 premerge CI against JDK17| +|[#11084](https://github.com/NVIDIA/spark-rapids/pull/11084)|Introduce LORE framework.| +|[#11099](https://github.com/NVIDIA/spark-rapids/pull/11099)|Spark 4: Handle ANSI mode in sort_test.py| +|[#11115](https://github.com/NVIDIA/spark-rapids/pull/11115)|Fix match error in RapidsShuffleIterator.scala [scala2.13]| +|[#11088](https://github.com/NVIDIA/spark-rapids/pull/11088)|Support regex patterns with brackets when rewriting to PrefixRange pattern in rlike.| +|[#10950](https://github.com/NVIDIA/spark-rapids/pull/10950)|Add a heuristic to skip second or third agg pass| +|[#11048](https://github.com/NVIDIA/spark-rapids/pull/11048)|Fixed array_tests for Spark 4.0.0| +|[#11049](https://github.com/NVIDIA/spark-rapids/pull/11049)|Fix some cast_tests for Spark 4.0.0| +|[#11066](https://github.com/NVIDIA/spark-rapids/pull/11066)|Replaced spark3xx-common references to spark-shared| +|[#11083](https://github.com/NVIDIA/spark-rapids/pull/11083)|Exclude a case based on JDK version in Spark UT| +|[#10997](https://github.com/NVIDIA/spark-rapids/pull/10997)|Fix some test issues in Spark UT and keep RapidsTestSettings update-to-date| +|[#11073](https://github.com/NVIDIA/spark-rapids/pull/11073)|Disable ANSI mode for window function tests| +|[#11076](https://github.com/NVIDIA/spark-rapids/pull/11076)|Improve the diagnostics for 'conv' fallback explain| +|[#11092](https://github.com/NVIDIA/spark-rapids/pull/11092)|Add GpuBucketingUtils shim to Spark 4.0.0| +|[#11062](https://github.com/NVIDIA/spark-rapids/pull/11062)|fix duplicate counted metrics like op time for GpuCoalesceBatches| +|[#11044](https://github.com/NVIDIA/spark-rapids/pull/11044)|Fixed Failing tests in arithmetic_ops_tests for Spark 4.0.0| +|[#11086](https://github.com/NVIDIA/spark-rapids/pull/11086)|upgrade blossom-ci actions version [skip ci]| +|[#10957](https://github.com/NVIDIA/spark-rapids/pull/10957)|Support bucketing write for GPU| +|[#10979](https://github.com/NVIDIA/spark-rapids/pull/10979)|[FEA] Introduce low shuffle merge.| +|[#10996](https://github.com/NVIDIA/spark-rapids/pull/10996)|Fallback non-UTC TimeZoneAwareExpression with zoneId| +|[#11072](https://github.com/NVIDIA/spark-rapids/pull/11072)|Workaround numpy2 failed fastparquet compatibility tests| +|[#11046](https://github.com/NVIDIA/spark-rapids/pull/11046)|Calculate parallelism to speed up pre-merge CI| +|[#11054](https://github.com/NVIDIA/spark-rapids/pull/11054)|fix flaky array_item test failures| +|[#11051](https://github.com/NVIDIA/spark-rapids/pull/11051)|[FEA] Increase parallelism of deltalake test on databricks| +|[#10993](https://github.com/NVIDIA/spark-rapids/pull/10993)|`binary-dedupe` changes for Spark 4.0.0| +|[#11060](https://github.com/NVIDIA/spark-rapids/pull/11060)|Add in the ability to fingerprint JSON columns| +|[#11059](https://github.com/NVIDIA/spark-rapids/pull/11059)|Revert "Add in the ability to fingerprint JSON columns (#11002)" [skip ci]| +|[#11039](https://github.com/NVIDIA/spark-rapids/pull/11039)|Concat() Exception bug fix| +|[#11002](https://github.com/NVIDIA/spark-rapids/pull/11002)|Add in the ability to fingerprint JSON columns| +|[#10977](https://github.com/NVIDIA/spark-rapids/pull/10977)|Rewrite multiple literal choice regex to multiple contains in rlike| +|[#11035](https://github.com/NVIDIA/spark-rapids/pull/11035)|Fix auto merge conflict 11034 [skip ci]| +|[#11040](https://github.com/NVIDIA/spark-rapids/pull/11040)|Append new authorized user to blossom-ci whitelist [skip ci]| +|[#11036](https://github.com/NVIDIA/spark-rapids/pull/11036)|Update blossom-ci ACL to secure format [skip ci]| +|[#11032](https://github.com/NVIDIA/spark-rapids/pull/11032)|Fix a hive write test failure for Spark 350| +|[#10998](https://github.com/NVIDIA/spark-rapids/pull/10998)|Improve log to print more lines in build [skip ci]| +|[#10992](https://github.com/NVIDIA/spark-rapids/pull/10992)|Addressing the Named Parameter change in Spark 4.0.0| +|[#10943](https://github.com/NVIDIA/spark-rapids/pull/10943)|Fix Spark UT issues in RapidsDataFrameAggregateSuite| +|[#10963](https://github.com/NVIDIA/spark-rapids/pull/10963)|Add rapids configs to enable GPU running in Spark UT| +|[#10978](https://github.com/NVIDIA/spark-rapids/pull/10978)|More compilation fixes for Spark 4.0.0| +|[#10953](https://github.com/NVIDIA/spark-rapids/pull/10953)|Speed up the integration tests by running them in parallel on the Databricks cluster| +|[#10958](https://github.com/NVIDIA/spark-rapids/pull/10958)|Fix a hive write test failure| +|[#10970](https://github.com/NVIDIA/spark-rapids/pull/10970)|Move Support for `RaiseError` to a Shim Excluding Spark 4.0.0| +|[#10966](https://github.com/NVIDIA/spark-rapids/pull/10966)|Add default value for REF of premerge jenkinsfile to avoid bad overwritten [skip ci]| +|[#10959](https://github.com/NVIDIA/spark-rapids/pull/10959)|Add new ID to blossom-ci allow list [skip ci]| +|[#10952](https://github.com/NVIDIA/spark-rapids/pull/10952)|Add shims to take care of the signature change for writeUDFs in PythonUDFRunner| +|[#10931](https://github.com/NVIDIA/spark-rapids/pull/10931)|Add Support for Renaming of PythonMapInArrow| +|[#10949](https://github.com/NVIDIA/spark-rapids/pull/10949)|Change dependency version to 24.08.0-SNAPSHOT| +|[#10857](https://github.com/NVIDIA/spark-rapids/pull/10857)|[Spark 4.0] Account for `PartitionedFileUtil.splitFiles` signature change.| +|[#10912](https://github.com/NVIDIA/spark-rapids/pull/10912)|GpuInsertIntoHiveTable supports parquet format| +|[#10863](https://github.com/NVIDIA/spark-rapids/pull/10863)|[Spark 4.0] Account for `CommandUtils.uncacheTableOrView` signature change.| +|[#10944](https://github.com/NVIDIA/spark-rapids/pull/10944)|Added Shim for BatchScanExec to Support Spark 4.0| +|[#10946](https://github.com/NVIDIA/spark-rapids/pull/10946)|Unarchive Spark test jar for spark.read(ability)| +|[#10945](https://github.com/NVIDIA/spark-rapids/pull/10945)|Add Support for Multiple Filtering Keys for Subquery Broadcast| +|[#10871](https://github.com/NVIDIA/spark-rapids/pull/10871)|Add classloader diagnostics to initShuffleManager error message| +|[#10933](https://github.com/NVIDIA/spark-rapids/pull/10933)|Fixed Databricks build| +|[#10929](https://github.com/NVIDIA/spark-rapids/pull/10929)|Append new authorized user to blossom-ci whitelist [skip ci]| + ## Release 24.06 ### Features From 795aef8dade72d76b42a247a3b7bd8eee03332b7 Mon Sep 17 00:00:00 2001 From: Tim Liu Date: Mon, 16 Dec 2024 11:22:35 +0800 Subject: [PATCH 13/47] Remove 350db143 shim's build (#11874) Skip the build of the 350db143 shim, as v24.12.0 will not contain the 350db143 shim Moreover, the v24.12.0 private dependency jar is not released. To fix below error: [ERROR] Failed to execute goal on project rapids-4-spark-sql_2.12: Could not resolve dependencies for project com.nvidia:rapids-4-spark-sql_2.12:jar:24.12.0: Failure to find com.nvidia:rapids-4-spark-private_2.12:jar:spark350db143:24.12.0 in https://repo1.maven.org/maven2 was cached in the local repository, resolution will not be reattempted until the update interval of central has elapsed or updates are forced Signed-off-by: Tim Liu --- jenkins/Jenkinsfile-blossom.premerge-databricks | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/jenkins/Jenkinsfile-blossom.premerge-databricks b/jenkins/Jenkinsfile-blossom.premerge-databricks index 2f8b926898a..cfe3cf6940e 100644 --- a/jenkins/Jenkinsfile-blossom.premerge-databricks +++ b/jenkins/Jenkinsfile-blossom.premerge-databricks @@ -91,7 +91,7 @@ pipeline { // 'name' and 'value' only supprt literal string in the declarative Jenkins // Refer to Jenkins issue https://issues.jenkins.io/browse/JENKINS-62127 name 'DB_RUNTIME' - values '11.3', '12.2', '13.3', '14.3' + values '11.3', '12.2', '13.3' } } stages { From d1ea935626797435fbf948abafd6280d12847307 Mon Sep 17 00:00:00 2001 From: Jenkins Automation <70000568+nvauto@users.noreply.github.com> Date: Mon, 16 Dec 2024 11:33:30 +0800 Subject: [PATCH 14/47] Update latest changelog [skip ci] (#11876) Update change log with CLI: \n\n scripts/generate-changelog --token= --releases=24.10,24.12 Signed-off-by: nvauto <70000568+nvauto@users.noreply.github.com> --- CHANGELOG.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5dd6c7b19ba..d78202be823 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -72,7 +72,10 @@ Generated on 2024-12-16 ### PRs ||| |:---|:---| +|[#11874](https://github.com/NVIDIA/spark-rapids/pull/11874)|Remove 350db143 shim's build [skip ci]| +|[#11851](https://github.com/NVIDIA/spark-rapids/pull/11851)|Update latest changelog [skip ci]| |[#11849](https://github.com/NVIDIA/spark-rapids/pull/11849)|Update rapids JNI and private dependency to 24.12.0| +|[#11841](https://github.com/NVIDIA/spark-rapids/pull/11841)|[DOC] update doc for 24.12 release [skip ci]| |[#11857](https://github.com/NVIDIA/spark-rapids/pull/11857)|Increase the pre-merge CI timeout to 6 hours| |[#11845](https://github.com/NVIDIA/spark-rapids/pull/11845)|Fix leak in isTimeStamp| |[#11823](https://github.com/NVIDIA/spark-rapids/pull/11823)|Fix for `LEAD/LAG` window function test failures.| From 94653288f40a5d600b986913f5b04b91038310a8 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Tue, 17 Dec 2024 11:54:35 -0600 Subject: [PATCH 15/47] Fix non-nullable under nullable struct write (#11781) Signed-off-by: Robert (Bobby) Evans --- .../src/main/python/orc_write_test.py | 24 +++++++++++++++++++ .../src/main/python/parquet_write_test.py | 19 +++++++++++++++ .../spark/rapids/GpuParquetFileFormat.scala | 1 + .../rapids/ParquetCachedBatchSerializer.scala | 5 ++-- .../com/nvidia/spark/rapids/SchemaUtils.scala | 19 +++++++++------ .../sql/hive/rapids/GpuHiveFileFormat.scala | 1 + .../spark/sql/rapids/GpuOrcFileFormat.scala | 2 +- 7 files changed, 61 insertions(+), 10 deletions(-) diff --git a/integration_tests/src/main/python/orc_write_test.py b/integration_tests/src/main/python/orc_write_test.py index 7e415c79a46..103cae474a3 100644 --- a/integration_tests/src/main/python/orc_write_test.py +++ b/integration_tests/src/main/python/orc_write_test.py @@ -133,6 +133,30 @@ def test_write_round_trip_corner(spark_tmp_path, orc_gen, orc_impl): data_path, conf={'spark.sql.orc.impl': orc_impl, 'spark.rapids.sql.format.orc.write.enabled': True}) +@pytest.mark.parametrize('gen', [ByteGen(nullable=False), + ShortGen(nullable=False), + IntegerGen(nullable=False), + LongGen(nullable=False), + FloatGen(nullable=False), + DoubleGen(nullable=False), + BooleanGen(nullable=False), + StringGen(nullable=False), + StructGen([('b', LongGen(nullable=False))], nullable=False)], ids=idfn) +@pytest.mark.parametrize('orc_impl', ["native", "hive"]) +@allow_non_gpu(*non_utc_allow) +def test_write_round_trip_nullable_struct(spark_tmp_path, gen, orc_impl): + gen_for_struct = StructGen([('c', gen)], nullable=True) + data_path = spark_tmp_path + '/ORC_DATA' + assert_gpu_and_cpu_writes_are_equal_collect( + lambda spark, path: unary_op_df(spark, gen_for_struct, num_slices=1).write.orc(path), + lambda spark, path: spark.read.orc(path), + data_path, + conf={'spark.sql.orc.impl': orc_impl, + 'spark.rapids.sql.format.orc.write.enabled': True, + # https://github.com/NVIDIA/spark-rapids/issues/11736, so verify that we still do it correctly + # once this is fixed + 'spark.rapids.sql.format.orc.write.boolType.enabled' : True}) + orc_part_write_gens = [ # Add back boolean_gen when https://github.com/rapidsai/cudf/issues/6763 is fixed byte_gen, short_gen, int_gen, long_gen, diff --git a/integration_tests/src/main/python/parquet_write_test.py b/integration_tests/src/main/python/parquet_write_test.py index 1d395d0e29a..e5719d267b4 100644 --- a/integration_tests/src/main/python/parquet_write_test.py +++ b/integration_tests/src/main/python/parquet_write_test.py @@ -105,6 +105,25 @@ def test_write_round_trip(spark_tmp_path, parquet_gens): data_path, conf=writer_confs) +@pytest.mark.parametrize('gen', [ByteGen(nullable=False), + ShortGen(nullable=False), + IntegerGen(nullable=False), + LongGen(nullable=False), + FloatGen(nullable=False), + DoubleGen(nullable=False), + BooleanGen(nullable=False), + StringGen(nullable=False), + StructGen([('b', LongGen(nullable=False))], nullable=False)], ids=idfn) +@allow_non_gpu(*non_utc_allow) +def test_write_round_trip_nullable_struct(spark_tmp_path, gen): + gen_for_struct = StructGen([('c', gen)], nullable=True) + data_path = spark_tmp_path + '/PARQUET_DATA' + assert_gpu_and_cpu_writes_are_equal_collect( + lambda spark, path: unary_op_df(spark, gen_for_struct, num_slices=1).write.parquet(path), + lambda spark, path: spark.read.parquet(path), + data_path, + conf=writer_confs) + all_nulls_string_gen = SetValuesGen(StringType(), [None]) empty_or_null_string_gen = SetValuesGen(StringType(), [None, ""]) all_empty_string_gen = SetValuesGen(StringType(), [""]) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala index e5aa52c727d..2d6cb903b75 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala @@ -391,6 +391,7 @@ class GpuParquetWriter( val writeContext = new ParquetWriteSupport().init(conf) val builder = SchemaUtils .writerOptionsFromSchema(ParquetWriterOptions.builder(), dataSchema, + nullable = false, ParquetOutputTimestampType.INT96 == SQLConf.get.parquetOutputTimestampType, parquetFieldIdEnabled) .withMetadata(writeContext.getExtraMetaData) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ParquetCachedBatchSerializer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ParquetCachedBatchSerializer.scala index d88f21922ce..861905f45f7 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ParquetCachedBatchSerializer.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ParquetCachedBatchSerializer.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -430,7 +430,8 @@ protected class ParquetCachedBatchSerializer extends GpuCachedBatchSerializer { schema: StructType): ParquetWriterOptions = { val compressionType = if (useCompression) CompressionType.SNAPPY else CompressionType.NONE SchemaUtils - .writerOptionsFromSchema(ParquetWriterOptions.builder(), schema, writeInt96 = false) + .writerOptionsFromSchema(ParquetWriterOptions.builder(), schema, nullable = false, + writeInt96 = false) .withCompressionType(compressionType) .withStatisticsFrequency(StatisticsFrequency.ROWGROUP).build() } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SchemaUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SchemaUtils.scala index 22047f22e68..cc36fc7c848 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SchemaUtils.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SchemaUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2023, NVIDIA CORPORATION. + * Copyright (c) 2021-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -247,19 +247,19 @@ object SchemaUtils { dataType match { case dt: DecimalType => - if(parquetFieldIdWriteEnabled && parquetFieldId.nonEmpty) { + if (parquetFieldIdWriteEnabled && parquetFieldId.nonEmpty) { builder.withDecimalColumn(name, dt.precision, nullable, parquetFieldId.get) } else { builder.withDecimalColumn(name, dt.precision, nullable) } case TimestampType => - if(parquetFieldIdWriteEnabled && parquetFieldId.nonEmpty) { + if (parquetFieldIdWriteEnabled && parquetFieldId.nonEmpty) { builder.withTimestampColumn(name, writeInt96, nullable, parquetFieldId.get) } else { builder.withTimestampColumn(name, writeInt96, nullable) } case s: StructType => - val structB = if(parquetFieldIdWriteEnabled && parquetFieldId.nonEmpty) { + val structB = if (parquetFieldIdWriteEnabled && parquetFieldId.nonEmpty) { structBuilder(name, nullable, parquetFieldId.get) } else { structBuilder(name, nullable) @@ -267,7 +267,9 @@ object SchemaUtils { builder.withStructColumn(writerOptionsFromSchema( structB, s, - writeInt96, parquetFieldIdWriteEnabled).build()) + nullable = nullable, + writeInt96, + parquetFieldIdWriteEnabled).build()) case a: ArrayType => builder.withListColumn( writerOptionsFromField( @@ -328,11 +330,14 @@ object SchemaUtils { def writerOptionsFromSchema[T <: NestedBuilder[T, V], V <: ColumnWriterOptions]( builder: NestedBuilder[T, V], schema: StructType, + nullable: Boolean, writeInt96: Boolean = false, parquetFieldIdEnabled: Boolean = false): T = { schema.foreach(field => - writerOptionsFromField(builder, field.dataType, field.name, field.nullable, writeInt96, - field.metadata, parquetFieldIdEnabled) + // CUDF has issues if the child of a struct is not-nullable, but the struct itself is + // So we have to work around it and tell CUDF what it expects. + writerOptionsFromField(builder, field.dataType, field.name, nullable || field.nullable, + writeInt96, field.metadata, parquetFieldIdWriteEnabled = parquetFieldIdEnabled) ) builder.asInstanceOf[T] } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala index d39050a0c32..9c6882ca4a3 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala @@ -227,6 +227,7 @@ class GpuHiveParquetWriter(override val path: String, dataSchema: StructType, override protected val tableWriter: CudfTableWriter = { val optionsBuilder = SchemaUtils .writerOptionsFromSchema(ParquetWriterOptions.builder(), dataSchema, + nullable = false, writeInt96 = true, // Hive 1.2 write timestamp as INT96 parquetFieldIdEnabled = false) .withCompressionType(compType) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala index 1d4bc66a1da..6e9d30296ff 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala @@ -210,7 +210,7 @@ class GpuOrcWriter(override val path: String, override val tableWriter: TableWriter = { val builder = SchemaUtils - .writerOptionsFromSchema(ORCWriterOptions.builder(), dataSchema) + .writerOptionsFromSchema(ORCWriterOptions.builder(), dataSchema, nullable = false) .withCompressionType(CompressionType.valueOf(OrcConf.COMPRESS.getString(conf))) Table.writeORCChunked(builder.build(), this) } From 3f26d33aa6bbcde95b1091f58d6f78520f5a7061 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Wed, 18 Dec 2024 07:44:18 -0600 Subject: [PATCH 16/47] Add support for dumping write data to try and reproduce error cases (#11864) Signed-off-by: Robert (Bobby) Evans --- .../delta20x/GpuOptimisticTransaction.scala | 7 +-- .../delta21x/GpuOptimisticTransaction.scala | 7 +-- .../delta22x/GpuOptimisticTransaction.scala | 7 +-- .../delta23x/GpuOptimisticTransaction.scala | 7 +-- .../delta24x/GpuOptimisticTransaction.scala | 7 +-- .../rapids/GpuOptimisticTransaction.scala | 7 +-- .../rapids/GpuOptimisticTransaction.scala | 7 +-- .../rapids/GpuOptimisticTransaction.scala | 7 +-- .../rapids/GpuOptimisticTransaction.scala | 5 ++- .../spark/rapids/ColumnarOutputWriter.scala | 45 +++++++++++++++++-- .../nvidia/spark/rapids/GpuOverrides.scala | 3 +- .../spark/rapids/GpuParquetFileFormat.scala | 10 +++-- .../com/nvidia/spark/rapids/RapidsConf.scala | 11 +++++ .../sql/hive/rapids/GpuHiveFileFormat.scala | 20 +++++---- .../sql/hive/rapids/GpuSaveAsHiveFile.scala | 4 +- .../sql/rapids/GpuFileFormatDataWriter.scala | 34 +++++++++++--- ...GpuInsertIntoHadoopFsRelationCommand.scala | 6 ++- .../spark/sql/rapids/GpuOrcFileFormat.scala | 10 +++-- ...aSourceTableAsSelectCommandMetaShims.scala | 3 +- .../rapids/shims/GpuInsertIntoHiveTable.scala | 11 +++-- .../spark/sql/rapids/GpuDataSource.scala | 11 +++-- .../sql/rapids/GpuFileFormatWriter.scala | 17 ++++--- ...eDataSourceTableAsSelectCommandShims.scala | 5 ++- .../rapids/shims/GpuInsertIntoHiveTable.scala | 10 +++-- .../execution/datasources/GpuWriteFiles.scala | 11 +++-- .../sql/rapids/GpuFileFormatWriter.scala | 23 ++++++---- .../rapids/GpuFileFormatDataWriterSuite.scala | 17 ++++--- 27 files changed, 218 insertions(+), 94 deletions(-) diff --git a/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala b/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala index 282368fc95b..ee0fc6f1cae 100644 --- a/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * Copyright (c) 2022-2024, NVIDIA CORPORATION. * * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala * in the Delta Lake project at https://github.com/delta-io/delta. @@ -219,8 +219,9 @@ class GpuOptimisticTransaction bucketSpec = None, statsTrackers = optionalStatsTracker.toSeq ++ statsTrackers, options = options, - rapidsConf.stableSort, - rapidsConf.concurrentWriterPartitionFlushSize) + useStableSort = rapidsConf.stableSort, + concurrentWriterPartitionFlushSize = rapidsConf.concurrentWriterPartitionFlushSize, + baseDebugOutputPath = rapidsConf.outputDebugDumpPrefix) } catch { case s: SparkException => // Pull an InvariantViolationException up to the top level if it was the root cause. diff --git a/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala b/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala index 9acded983d0..7433594aa73 100644 --- a/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala * in the Delta Lake project at https://github.com/delta-io/delta. @@ -219,8 +219,9 @@ class GpuOptimisticTransaction bucketSpec = None, statsTrackers = optionalStatsTracker.toSeq ++ statsTrackers, options = options, - rapidsConf.stableSort, - rapidsConf.concurrentWriterPartitionFlushSize) + useStableSort = rapidsConf.stableSort, + concurrentWriterPartitionFlushSize = rapidsConf.concurrentWriterPartitionFlushSize, + baseDebugOutputPath = rapidsConf.outputDebugDumpPrefix) } catch { case s: SparkException => // Pull an InvariantViolationException up to the top level if it was the root cause. diff --git a/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala b/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala index 3ec00c5cb62..cb990ccc16d 100644 --- a/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala * in the Delta Lake project at https://github.com/delta-io/delta. @@ -241,8 +241,9 @@ class GpuOptimisticTransaction bucketSpec = None, statsTrackers = optionalStatsTracker.toSeq ++ statsTrackers, options = options, - rapidsConf.stableSort, - rapidsConf.concurrentWriterPartitionFlushSize) + useStableSort = rapidsConf.stableSort, + concurrentWriterPartitionFlushSize = rapidsConf.concurrentWriterPartitionFlushSize, + baseDebugOutputPath = rapidsConf.outputDebugDumpPrefix) } catch { case s: SparkException => // Pull an InvariantViolationException up to the top level if it was the root cause. diff --git a/delta-lake/delta-23x/src/main/scala/org/apache/spark/sql/delta/rapids/delta23x/GpuOptimisticTransaction.scala b/delta-lake/delta-23x/src/main/scala/org/apache/spark/sql/delta/rapids/delta23x/GpuOptimisticTransaction.scala index 38ee8a786c0..905329a6b01 100644 --- a/delta-lake/delta-23x/src/main/scala/org/apache/spark/sql/delta/rapids/delta23x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-23x/src/main/scala/org/apache/spark/sql/delta/rapids/delta23x/GpuOptimisticTransaction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala * in the Delta Lake project at https://github.com/delta-io/delta. @@ -241,8 +241,9 @@ class GpuOptimisticTransaction bucketSpec = None, statsTrackers = optionalStatsTracker.toSeq ++ statsTrackers, options = options, - rapidsConf.stableSort, - rapidsConf.concurrentWriterPartitionFlushSize) + useStableSort = rapidsConf.stableSort, + concurrentWriterPartitionFlushSize = rapidsConf.concurrentWriterPartitionFlushSize, + baseDebugOutputPath = rapidsConf.outputDebugDumpPrefix) } catch { case s: SparkException => // Pull an InvariantViolationException up to the top level if it was the root cause. diff --git a/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala b/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala index bc95a656271..fadc9bf99a8 100644 --- a/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala * in the Delta Lake project at https://github.com/delta-io/delta. @@ -243,8 +243,9 @@ class GpuOptimisticTransaction bucketSpec = None, statsTrackers = optionalStatsTracker.toSeq ++ statsTrackers, options = options, - rapidsConf.stableSort, - rapidsConf.concurrentWriterPartitionFlushSize) + useStableSort = rapidsConf.stableSort, + concurrentWriterPartitionFlushSize = rapidsConf.concurrentWriterPartitionFlushSize, + baseDebugOutputPath = rapidsConf.outputDebugDumpPrefix) } catch { case s: SparkException => // Pull an InvariantViolationException up to the top level if it was the root cause. diff --git a/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala index 9dff950ca5b..f4debdea1c1 100644 --- a/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * Copyright (c) 2022-2024, NVIDIA CORPORATION. * * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala * in the Delta Lake project at https://github.com/delta-io/delta. @@ -261,8 +261,9 @@ class GpuOptimisticTransaction( bucketSpec = None, statsTrackers = optionalStatsTracker.toSeq ++ identityTracker.toSeq ++ statsTrackers, options = options, - rapidsConf.stableSort, - rapidsConf.concurrentWriterPartitionFlushSize) + useStableSort = rapidsConf.stableSort, + concurrentWriterPartitionFlushSize = rapidsConf.concurrentWriterPartitionFlushSize, + baseDebugOutputPath = rapidsConf.outputDebugDumpPrefix) } catch { case s: SparkException => // Pull an InvariantViolationException up to the top level if it was the root cause. diff --git a/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala index fb9d3840917..48cebee6e41 100644 --- a/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala * in the Delta Lake project at https://github.com/delta-io/delta. @@ -258,8 +258,9 @@ class GpuOptimisticTransaction( bucketSpec = None, statsTrackers = optionalStatsTracker.toSeq ++ identityTracker.toSeq ++ statsTrackers, options = options, - rapidsConf.stableSort, - rapidsConf.concurrentWriterPartitionFlushSize) + useStableSort = rapidsConf.stableSort, + concurrentWriterPartitionFlushSize = rapidsConf.concurrentWriterPartitionFlushSize, + baseDebugOutputPath = rapidsConf.outputDebugDumpPrefix) } catch { case s: SparkException => // Pull an InvariantViolationException up to the top level if it was the root cause. diff --git a/delta-lake/delta-spark341db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark341db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala index 3e836056b6d..e1e5f28dec6 100644 --- a/delta-lake/delta-spark341db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-spark341db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala * in the Delta Lake project at https://github.com/delta-io/delta. @@ -259,8 +259,9 @@ class GpuOptimisticTransaction( bucketSpec = None, statsTrackers = optionalStatsTracker.toSeq ++ identityTracker.toSeq ++ statsTrackers, options = options, - rapidsConf.stableSort, - rapidsConf.concurrentWriterPartitionFlushSize) + useStableSort = rapidsConf.stableSort, + concurrentWriterPartitionFlushSize = rapidsConf.concurrentWriterPartitionFlushSize, + baseDebugOutputPath = rapidsConf.outputDebugDumpPrefix) } catch { case s: SparkException => // Pull an InvariantViolationException up to the top level if it was the root cause. diff --git a/delta-lake/delta-spark350db143/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark350db143/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala index e06aba55487..b49c5ebaa30 100644 --- a/delta-lake/delta-spark350db143/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-spark350db143/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -259,8 +259,9 @@ class GpuOptimisticTransaction( bucketSpec = None, statsTrackers = optionalStatsTracker.toSeq ++ identityTracker.toSeq ++ statsTrackers, options = options, - rapidsConf.stableSort, - rapidsConf.concurrentWriterPartitionFlushSize) + useStableSort = rapidsConf.stableSort, + concurrentWriterPartitionFlushSize = rapidsConf.concurrentWriterPartitionFlushSize, + baseDebugOutputPath = rapidsConf.outputDebugDumpPrefix) } catch { case s: SparkException => // Pull an InvariantViolationException up to the top level if it was the root cause. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala index 8d89583d9df..1b1965cfba0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala @@ -16,11 +16,11 @@ package com.nvidia.spark.rapids -import java.io.OutputStream +import java.io.{BufferedOutputStream, DataOutputStream, OutputStream} import scala.collection.mutable -import ai.rapids.cudf.{HostBufferConsumer, HostMemoryBuffer, NvtxColor, NvtxRange, TableWriter} +import ai.rapids.cudf.{HostBufferConsumer, HostMemoryBuffer, JCudfSerialization, NvtxColor, NvtxRange, TableWriter} import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.RapidsPluginImplicits._ @@ -61,7 +61,8 @@ abstract class ColumnarOutputWriterFactory extends Serializable { def newInstance( path: String, dataSchema: StructType, - context: TaskAttemptContext): ColumnarOutputWriter + context: TaskAttemptContext, + debugOutputPath: Option[String]): ColumnarOutputWriter } /** @@ -73,10 +74,44 @@ abstract class ColumnarOutputWriter(context: TaskAttemptContext, dataSchema: StructType, rangeName: String, includeRetry: Boolean, + debugDumpPath: Option[String], holdGpuBetweenBatches: Boolean = false, useAsyncWrite: Boolean = false) extends HostBufferConsumer with Logging { protected val tableWriter: TableWriter + private lazy val debugDumpOutputStream: Option[OutputStream] = try { + debugDumpPath.map { path => + val tc = TaskContext.get() + logWarning(s"DEBUG FILE OUTPUT $rangeName FOR " + + s"STAGE ${tc.stageId()} TASK ${tc.taskAttemptId()} is $path") + val hadoopPath = new Path(path) + val fs = hadoopPath.getFileSystem(conf) + new DataOutputStream(new BufferedOutputStream(fs.create(hadoopPath, false))) + } + } catch { + case e: Exception => + logError(s"Could Not Write Debug Table $debugDumpPath", e) + None + } + + /** + * Write out a debug batch to the debug output stream if it is configured. + * If it is not configured, this is a noop. If an exception happens the exception + * is ignored, but it is logged. + */ + private def debugWriteBatch(batch: ColumnarBatch): Unit = { + debugDumpOutputStream.foreach { output => + try { + withResource(GpuColumnVector.from(batch)) { table => + JCudfSerialization.writeToStream(table, output, 0, table.getRowCount) + } + output.flush() + } catch { + case t: Throwable => + logError(s"Could Not Write Debug Table $debugDumpPath", t) + } + } + } protected val conf: Configuration = context.getConfiguration @@ -222,6 +257,7 @@ abstract class ColumnarOutputWriter(context: TaskAttemptContext, // where corrupt files can be written if nothing is encoded via the writer. anythingWritten = true + debugWriteBatch(batch) // tableWriter.write() serializes the table into the HostMemoryBuffer, and buffers it // by calling handleBuffer() on the ColumnarOutputWriter. It may not write to the // output stream just yet. @@ -242,6 +278,9 @@ abstract class ColumnarOutputWriter(context: TaskAttemptContext, GpuSemaphore.releaseIfNecessary(TaskContext.get()) writeBufferedData() outputStream.close() + debugDumpOutputStream.foreach { os => + os.close() + } } /** 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 12ebfc01862..9d28b2f1bf3 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 @@ -369,7 +369,8 @@ final class InsertIntoHadoopFsRelationCommandMeta( cmd.fileIndex, cmd.outputColumnNames, conf.stableSort, - conf.concurrentWriterPartitionFlushSize) + conf.concurrentWriterPartitionFlushSize, + conf.outputDebugDumpPrefix) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala index 2d6cb903b75..a1d5098be73 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala @@ -280,10 +280,11 @@ class GpuParquetFileFormat extends ColumnarFileFormat with Logging { override def newInstance( path: String, dataSchema: StructType, - context: TaskAttemptContext): ColumnarOutputWriter = { + context: TaskAttemptContext, + debugOutputPath: Option[String]): ColumnarOutputWriter = { new GpuParquetWriter(path, dataSchema, compressionType, outputTimestampType.toString, dateTimeRebaseMode, timestampRebaseMode, context, parquetFieldIdWriteEnabled, - holdGpuBetweenBatches, asyncOutputWriteEnabled) + debugOutputPath, holdGpuBetweenBatches, asyncOutputWriteEnabled) } override def getFileExtension(context: TaskAttemptContext): String = { @@ -306,10 +307,11 @@ class GpuParquetWriter( timestampRebaseMode: DateTimeRebaseMode, context: TaskAttemptContext, parquetFieldIdEnabled: Boolean, + debugDumpPath: Option[String], holdGpuBetweenBatches: Boolean, useAsyncWrite: Boolean) - extends ColumnarOutputWriter(context, dataSchema, "Parquet", true, holdGpuBetweenBatches, - useAsyncWrite) { + extends ColumnarOutputWriter(context, dataSchema, "Parquet", true, + debugDumpPath, holdGpuBetweenBatches, useAsyncWrite) { override def throwIfRebaseNeededInExceptionMode(batch: ColumnarBatch): Unit = { val cols = GpuColumnVector.extractBases(batch) cols.foreach { col => 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 c34e461b258..c4199e3ea75 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 @@ -1682,6 +1682,15 @@ val GPU_COREDUMP_PIPE_PATTERN = conf("spark.rapids.gpu.coreDump.pipePattern") .booleanConf .createWithDefault(false) + val OUTPUT_DEBUG_DUMP_PREFIX = conf("spark.rapids.sql.output.debug.dumpPrefix") + .doc("A path prefix where data that is intended to be written out as the result " + + "of a query should be dumped for debugging. The format of this is based on " + + "JCudfSerialization and is trying to capture the underlying table so that if " + + "there are errors in the output format we can try to recreate it.") + .internal() + .stringConf + .createOptional + val PARQUET_DEBUG_DUMP_PREFIX = conf("spark.rapids.sql.parquet.debug.dumpPrefix") .doc("A path prefix where Parquet split file data is dumped for debugging.") .internal() @@ -2883,6 +2892,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val maxGpuColumnSizeBytes: Long = get(MAX_GPU_COLUMN_SIZE_BYTES) + lazy val outputDebugDumpPrefix: Option[String] = get(OUTPUT_DEBUG_DUMP_PREFIX) + lazy val parquetDebugDumpPrefix: Option[String] = get(PARQUET_DEBUG_DUMP_PREFIX) lazy val parquetDebugDumpAlways: Boolean = get(PARQUET_DEBUG_DUMP_ALWAYS) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala index 9c6882ca4a3..11637b9818a 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala @@ -213,16 +213,18 @@ class GpuHiveParquetFileFormat(compType: CompressionType) extends ColumnarFileFo override def newInstance(path: String, dataSchema: StructType, - context: TaskAttemptContext): ColumnarOutputWriter = { - new GpuHiveParquetWriter(path, dataSchema, context, compressionType) + context: TaskAttemptContext, + debugOutputPath: Option[String]): ColumnarOutputWriter = { + new GpuHiveParquetWriter(path, dataSchema, context, compressionType, debugOutputPath) } } } } class GpuHiveParquetWriter(override val path: String, dataSchema: StructType, - context: TaskAttemptContext, compType: CompressionType) - extends ColumnarOutputWriter(context, dataSchema, "HiveParquet", true) { + context: TaskAttemptContext, compType: CompressionType, + debugOutputPath: Option[String]) + extends ColumnarOutputWriter(context, dataSchema, "HiveParquet", true, debugOutputPath) { override protected val tableWriter: CudfTableWriter = { val optionsBuilder = SchemaUtils @@ -250,8 +252,9 @@ class GpuHiveTextFileFormat extends ColumnarFileFormat with Logging { override def newInstance(path: String, dataSchema: StructType, - context: TaskAttemptContext): ColumnarOutputWriter = { - new GpuHiveTextWriter(path, dataSchema, context) + context: TaskAttemptContext, + debugOutputPath: Option[String]): ColumnarOutputWriter = { + new GpuHiveTextWriter(path, dataSchema, context, debugOutputPath) } } } @@ -259,8 +262,9 @@ class GpuHiveTextFileFormat extends ColumnarFileFormat with Logging { class GpuHiveTextWriter(override val path: String, dataSchema: StructType, - context: TaskAttemptContext) - extends ColumnarOutputWriter(context, dataSchema, "HiveText", false) { + context: TaskAttemptContext, + debugOutputPath: Option[String]) + extends ColumnarOutputWriter(context, dataSchema, "HiveText", false, debugOutputPath) { /** * This reformats columns, to iron out inconsistencies between diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuSaveAsHiveFile.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuSaveAsHiveFile.scala index 08118cc11a0..75424714673 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuSaveAsHiveFile.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuSaveAsHiveFile.scala @@ -43,6 +43,7 @@ private[hive] trait GpuSaveAsHiveFile extends GpuDataWritingCommand with SaveAsH fileFormat: ColumnarFileFormat, outputLocation: String, forceHiveHashForBucketing: Boolean, + baseDebugOutputPath: Option[String], customPartitionLocations: Map[TablePartitionSpec,String] = Map.empty, partitionAttributes: Seq[Attribute] = Nil, bucketSpec: Option[BucketSpec] = None, @@ -67,7 +68,8 @@ private[hive] trait GpuSaveAsHiveFile extends GpuDataWritingCommand with SaveAsH options = options, useStableSort = false, // TODO: Fetch from RapidsConf. forceHiveHashForBucketing = forceHiveHashForBucketing, - concurrentWriterPartitionFlushSize = 0L // TODO: Fetch from RapidsConf. + concurrentWriterPartitionFlushSize = 0L, // TODO: Fetch from RapidsConf. + baseDebugOutputPath = baseDebugOutputPath ) } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala index 6b6580df68f..aca7d0e8751 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala @@ -228,7 +228,8 @@ class GpuEmptyDirectoryDataWriter( class GpuSingleDirectoryDataWriter( description: GpuWriteJobDescription, taskAttemptContext: TaskAttemptContext, - committer: FileCommitProtocol) + committer: FileCommitProtocol, + debugOutputBasePath: Option[String]) extends GpuFileFormatDataWriter(description, taskAttemptContext, committer) { // Initialize currentWriter and statsTrackers newOutputWriter() @@ -247,10 +248,16 @@ class GpuSingleDirectoryDataWriter( None, f"-c$fileCounter%03d" + ext) + val debugOutputPath = debugOutputBasePath.map { base => + base + s"/DEBUG_${taskAttemptContext.getTaskAttemptID}" + + f"_c$fileCounter%03d_${System.nanoTime()}.debug" + } + currentWriterStatus.writer = description.outputWriterFactory.newInstance( path = currentPath, dataSchema = description.dataColumns.toStructType, - context = taskAttemptContext) + context = taskAttemptContext, + debugOutputPath = debugOutputPath) statsTrackers.foreach(_.newFile(currentPath)) } @@ -293,7 +300,8 @@ class GpuSingleDirectoryDataWriter( class GpuDynamicPartitionDataSingleWriter( description: GpuWriteJobDescription, taskAttemptContext: TaskAttemptContext, - committer: FileCommitProtocol) + committer: FileCommitProtocol, + debugOutputBasePath: Option[String]) extends GpuFileFormatDataWriter(description, taskAttemptContext, committer) { /** Wrapper class to index a unique concurrent output writer. */ protected class WriterIndex( @@ -576,10 +584,22 @@ class GpuDynamicPartitionDataSingleWriter( committer.newTaskTempFile(taskAttemptContext, partDir, ext) } + val debugOutputPath = debugOutputBasePath.map { base => + if (customPath.isDefined) { + val hash = customPath.get.hashCode + base + s"/DEBUG_CUSTOM_${hash}_${taskAttemptContext.getTaskAttemptID}" + + f"_c$fileCounter%03d_${System.nanoTime()}.debug" + } else { + base + s"/${partDir.mkString("/")}/DEBUG_${taskAttemptContext.getTaskAttemptID}" + + f"_c$fileCounter%03d_${System.nanoTime()}.debug" + } + } + val outWriter = description.outputWriterFactory.newInstance( path = currentPath, dataSchema = description.dataColumns.toStructType, - context = taskAttemptContext) + context = taskAttemptContext, + debugOutputPath = debugOutputPath) statsTrackers.foreach(_.newFile(currentPath)) outWriter @@ -665,8 +685,10 @@ class GpuDynamicPartitionDataConcurrentWriter( description: GpuWriteJobDescription, taskAttemptContext: TaskAttemptContext, committer: FileCommitProtocol, - spec: GpuConcurrentOutputWriterSpec) - extends GpuDynamicPartitionDataSingleWriter(description, taskAttemptContext, committer) + spec: GpuConcurrentOutputWriterSpec, + debugOutputBasePath: Option[String]) + extends GpuDynamicPartitionDataSingleWriter(description, taskAttemptContext, + committer, debugOutputBasePath) with Logging { /** Wrapper class for status and caches of a unique concurrent output writer. */ diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala index 2671323ea8c..c8a9d0c7ed1 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala @@ -50,7 +50,8 @@ case class GpuInsertIntoHadoopFsRelationCommand( fileIndex: Option[FileIndex], outputColumnNames: Seq[String], useStableSort: Boolean, - concurrentWriterPartitionFlushSize: Long) + concurrentWriterPartitionFlushSize: Long, + baseDebugOutputPath: Option[String]) extends GpuDataWritingCommand { override def runColumnar(sparkSession: SparkSession, child: SparkPlan): Seq[ColumnarBatch] = { @@ -184,7 +185,8 @@ case class GpuInsertIntoHadoopFsRelationCommand( useStableSort = useStableSort, concurrentWriterPartitionFlushSize = concurrentWriterPartitionFlushSize, forceHiveHashForBucketing = forceHiveHashForBucketing, - numStaticPartitionCols = staticPartitions.size) + numStaticPartitionCols = staticPartitions.size, + baseDebugOutputPath = baseDebugOutputPath) // update metastore partition metadata diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala index 6e9d30296ff..5ac2aa1fe98 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala @@ -183,8 +183,9 @@ class GpuOrcFileFormat extends ColumnarFileFormat with Logging { new ColumnarOutputWriterFactory { override def newInstance(path: String, dataSchema: StructType, - context: TaskAttemptContext): ColumnarOutputWriter = { - new GpuOrcWriter(path, dataSchema, context) + context: TaskAttemptContext, + debugOutputPath: Option[String]): ColumnarOutputWriter = { + new GpuOrcWriter(path, dataSchema, context, debugOutputPath) } override def getFileExtension(context: TaskAttemptContext): String = { @@ -205,8 +206,9 @@ class GpuOrcFileFormat extends ColumnarFileFormat with Logging { class GpuOrcWriter(override val path: String, dataSchema: StructType, - context: TaskAttemptContext) - extends ColumnarOutputWriter(context, dataSchema, "ORC", true) { + context: TaskAttemptContext, + debugOutputPath: Option[String]) + extends ColumnarOutputWriter(context, dataSchema, "ORC", true, debugOutputPath) { override val tableWriter: TableWriter = { val builder = SchemaUtils diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala index c5d27c1e570..fde588dee62 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala @@ -87,6 +87,7 @@ final class CreateDataSourceTableAsSelectCommandMeta( origProvider, newProvider, conf.stableSort, - conf.concurrentWriterPartitionFlushSize) + conf.concurrentWriterPartitionFlushSize, + conf.outputDebugDumpPrefix) } } diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala index 9dd038b8874..d7a29bbc500 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala @@ -82,7 +82,8 @@ final class GpuInsertIntoHiveTableMeta(cmd: InsertIntoHiveTable, query = wrapped.query, overwrite = wrapped.overwrite, ifPartitionNotExists = wrapped.ifPartitionNotExists, - outputColumnNames = wrapped.outputColumnNames + outputColumnNames = wrapped.outputColumnNames, + baseOutputDebugPath = conf.outputDebugDumpPrefix ) } @@ -96,7 +97,8 @@ case class GpuInsertIntoHiveTable( query: LogicalPlan, overwrite: Boolean, ifPartitionNotExists: Boolean, - outputColumnNames: Seq[String]) extends GpuSaveAsHiveFile { + outputColumnNames: Seq[String], + baseOutputDebugPath: Option[String]) extends GpuSaveAsHiveFile { /** * Inserts all the rows in the table into Hive. Row objects are properly serialized with the @@ -219,7 +221,8 @@ case class GpuInsertIntoHiveTable( forceHiveHashForBucketing = forceHiveHashForBucketing, partitionAttributes = partitionAttributes, bucketSpec = BucketSpecForHiveShim.getBucketSpec(table, forceHiveHashForBucketing), - options = BucketingUtilsShim.getOptionsWithHiveBucketWrite(table.bucketSpec)) + options = BucketingUtilsShim.getOptionsWithHiveBucketWrite(table.bucketSpec), + baseDebugOutputPath = baseOutputDebugPath) if (partition.nonEmpty) { if (numDynamicPartitions > 0) { @@ -363,4 +366,4 @@ case class GpuInsertIntoHiveTable( } override def requireSingleBatch: Boolean = false // TODO: Re-evaluate. If partitioned or bucketed? -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/GpuDataSource.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/GpuDataSource.scala index 0bbdc614967..8a6be9fc241 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/GpuDataSource.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/GpuDataSource.scala @@ -67,7 +67,8 @@ case class GpuDataSource( format: ColumnarFileFormat, mode: SaveMode, data: LogicalPlan, useStableSort: Boolean, - concurrentWriterPartitionFlushSize: Long): GpuInsertIntoHadoopFsRelationCommand = { + concurrentWriterPartitionFlushSize: Long, + baseDebugOutputPath: Option[String]): GpuInsertIntoHadoopFsRelationCommand = { // Don't glob path for the write path. The contracts here are: // 1. Only one output path can be specified on the write path; // 2. Output path must be a legal HDFS style file system path; @@ -107,7 +108,8 @@ case class GpuDataSource( fileIndex = fileIndex, outputColumnNames = data.output.map(_.name), useStableSort = useStableSort, - concurrentWriterPartitionFlushSize = concurrentWriterPartitionFlushSize) + concurrentWriterPartitionFlushSize = concurrentWriterPartitionFlushSize, + baseDebugOutputPath = baseDebugOutputPath) } /** @@ -131,7 +133,8 @@ case class GpuDataSource( outputColumnNames: Seq[String], physicalPlan: SparkPlan, useStableSort: Boolean, - concurrentWriterPartitionFlushSize: Long): BaseRelation = { + concurrentWriterPartitionFlushSize: Long, + baseDebugOutputPath: Option[String]): BaseRelation = { val outputColumns = DataWritingCommand.logicalPlanOutputWithNames(data, outputColumnNames) if (outputColumns.map(_.dataType).exists(_.isInstanceOf[CalendarIntervalType])) { throw new AnalysisException("Cannot save interval data type into external storage.") @@ -139,7 +142,7 @@ case class GpuDataSource( // Only currently support ColumnarFileFormat val cmd = planForWritingFileFormat(gpuFileFormat, mode, data, useStableSort, - concurrentWriterPartitionFlushSize) + concurrentWriterPartitionFlushSize, baseDebugOutputPath) val resolvedPartCols = cmd.partitionColumns.map { col => // The partition columns created in `planForWritingFileFormat` should always be // `UnresolvedAttribute` with a single name part. diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala index 71d2892e50e..6188919d2d4 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala @@ -104,7 +104,8 @@ object GpuFileFormatWriter extends Logging { useStableSort: Boolean, concurrentWriterPartitionFlushSize: Long, forceHiveHashForBucketing: Boolean = false, - numStaticPartitionCols: Int = 0): Set[String] = { + numStaticPartitionCols: Int = 0, + baseDebugOutputPath: Option[String]): Set[String] = { require(partitionColumns.size >= numStaticPartitionCols) val job = Job.getInstance(hadoopConf) @@ -257,7 +258,8 @@ object GpuFileFormatWriter extends Logging { sparkAttemptNumber = taskContext.taskAttemptId().toInt & Integer.MAX_VALUE, committer, iterator = iter, - concurrentOutputWriterSpec = concurrentOutputWriterSpec) + concurrentOutputWriterSpec = concurrentOutputWriterSpec, + baseDebugOutputPath = baseDebugOutputPath) }, rddWithNonEmptyPartitions.partitions.indices, (index, res: WriteTaskResult) => { @@ -291,7 +293,8 @@ object GpuFileFormatWriter extends Logging { sparkAttemptNumber: Int, committer: FileCommitProtocol, iterator: Iterator[ColumnarBatch], - concurrentOutputWriterSpec: Option[GpuConcurrentOutputWriterSpec]): WriteTaskResult = { + concurrentOutputWriterSpec: Option[GpuConcurrentOutputWriterSpec], + baseDebugOutputPath: Option[String]): WriteTaskResult = { val jobId = RapidsHadoopWriterUtils.createJobID(jobTrackerId, sparkStageId) val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId) @@ -317,14 +320,16 @@ object GpuFileFormatWriter extends Logging { // In case of empty job, leave first partition to save meta for file format like parquet. new GpuEmptyDirectoryDataWriter(description, taskAttemptContext, committer) } else if (description.partitionColumns.isEmpty && description.bucketSpec.isEmpty) { - new GpuSingleDirectoryDataWriter(description, taskAttemptContext, committer) + new GpuSingleDirectoryDataWriter(description, taskAttemptContext, committer, + baseDebugOutputPath) } else { concurrentOutputWriterSpec match { case Some(spec) => new GpuDynamicPartitionDataConcurrentWriter(description, taskAttemptContext, - committer, spec) + committer, spec, baseDebugOutputPath) case _ => - new GpuDynamicPartitionDataSingleWriter(description, taskAttemptContext, committer) + new GpuDynamicPartitionDataSingleWriter(description, taskAttemptContext, committer, + baseDebugOutputPath) } } diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala index 1123f1013f9..9d9182f32c1 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala @@ -55,7 +55,8 @@ case class GpuCreateDataSourceTableAsSelectCommand( origProvider: Class[_], gpuFileFormat: ColumnarFileFormat, useStableSort: Boolean, - concurrentWriterPartitionFlushSize: Long) + concurrentWriterPartitionFlushSize: Long, + baseDebugOutputPath: Option[String]) extends GpuDataWritingCommand { override def runColumnar(sparkSession: SparkSession, child: SparkPlan): Seq[ColumnarBatch] = { @@ -138,7 +139,7 @@ case class GpuCreateDataSourceTableAsSelectCommand( gpuFileFormat = gpuFileFormat) try { dataSource.writeAndRead(mode, query, outputColumnNames, physicalPlan, useStableSort, - concurrentWriterPartitionFlushSize) + concurrentWriterPartitionFlushSize, baseDebugOutputPath) } catch { case ex: AnalysisException => logError(s"Failed to write to table ${table.identifier.unquotedString}", ex) diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala index e7aecf0675f..43c515998a8 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala @@ -80,7 +80,8 @@ final class GpuInsertIntoHiveTableMeta(cmd: InsertIntoHiveTable, overwrite = wrapped.overwrite, ifPartitionNotExists = wrapped.ifPartitionNotExists, outputColumnNames = wrapped.outputColumnNames, - tmpLocation = cmd.hiveTmpPath.externalTempPath + tmpLocation = cmd.hiveTmpPath.externalTempPath, + baseOutputDebugPath = conf.outputDebugDumpPrefix ) } @@ -95,7 +96,8 @@ case class GpuInsertIntoHiveTable( overwrite: Boolean, ifPartitionNotExists: Boolean, outputColumnNames: Seq[String], - tmpLocation: Path) extends GpuSaveAsHiveFile { + tmpLocation: Path, + baseOutputDebugPath: Option[String]) extends GpuSaveAsHiveFile { /** * Inserts all the rows in the table into Hive. Row objects are properly serialized with the @@ -206,7 +208,6 @@ case class GpuInsertIntoHiveTable( val forceHiveHashForBucketing = RapidsConf.FORCE_HIVE_HASH_FOR_BUCKETED_WRITE.get(sparkSession.sessionState.conf) - val writtenParts = gpuSaveAsHiveFile( sparkSession = sparkSession, plan = child, @@ -216,7 +217,8 @@ case class GpuInsertIntoHiveTable( forceHiveHashForBucketing = forceHiveHashForBucketing, partitionAttributes = partitionAttributes, bucketSpec = table.bucketSpec, - options = BucketingUtilsShim.getOptionsWithHiveBucketWrite(table.bucketSpec)) + options = BucketingUtilsShim.getOptionsWithHiveBucketWrite(table.bucketSpec), + baseDebugOutputPath = baseOutputDebugPath) if (partition.nonEmpty) { if (numDynamicPartitions > 0) { diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala index 2b4036e042b..abcf6178248 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala @@ -75,7 +75,8 @@ class GpuWriteFilesMeta( writeFilesExec.partitionColumns, writeFilesExec.bucketSpec, writeFilesExec.options, - writeFilesExec.staticPartitions + writeFilesExec.staticPartitions, + conf.outputDebugDumpPrefix ) } } @@ -89,7 +90,8 @@ case class GpuWriteFilesExec( partitionColumns: Seq[Attribute], bucketSpec: Option[BucketSpec], options: Map[String, String], - staticPartitions: TablePartitionSpec) extends ShimUnaryExecNode with GpuExec { + staticPartitions: TablePartitionSpec, + baseOutputDebugPath: Option[String]) extends ShimUnaryExecNode with GpuExec { override def output: Seq[Attribute] = Seq.empty @@ -133,6 +135,8 @@ case class GpuWriteFilesExec( val description = writeFilesSpec.description val committer = writeFilesSpec.committer val jobTrackerID = SparkHadoopWriterUtils.createJobTrackerID(new Date()) + val localBaseOutputDebugPath = baseOutputDebugPath + rddWithNonEmptyPartitions.mapPartitionsInternal { iterator => val sparkStageId = TaskContext.get().stageId() val sparkPartitionId = TaskContext.get().partitionId() @@ -145,7 +149,8 @@ case class GpuWriteFilesExec( sparkAttemptNumber, committer, iterator, - concurrentOutputWriterSpec + concurrentOutputWriterSpec, + localBaseOutputDebugPath ) Iterator(ret) diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala index fc4e9273281..6da9ce792ab 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala @@ -104,7 +104,8 @@ object GpuFileFormatWriter extends Logging { useStableSort: Boolean, concurrentWriterPartitionFlushSize: Long, forceHiveHashForBucketing: Boolean = false, - numStaticPartitionCols: Int = 0): Set[String] = { + numStaticPartitionCols: Int = 0, + baseDebugOutputPath: Option[String]): Set[String] = { require(partitionColumns.size >= numStaticPartitionCols) val job = Job.getInstance(hadoopConf) @@ -208,7 +209,8 @@ object GpuFileFormatWriter extends Logging { // In this path, Spark version is less than 340 or 'spark.sql.optimizer.plannedWrite.enabled' // is disabled, should sort the data if necessary. executeWrite(sparkSession, plan, job, description, committer, outputSpec, - requiredOrdering, partitionColumns, sortColumns, orderingMatched, useStableSort) + requiredOrdering, partitionColumns, sortColumns, orderingMatched, useStableSort, + baseDebugOutputPath) } } @@ -223,7 +225,8 @@ object GpuFileFormatWriter extends Logging { partitionColumns: Seq[Attribute], sortColumns: Seq[Attribute], orderingMatched: Boolean, - useStableSort: Boolean): Set[String] = { + useStableSort: Boolean, + baseDebugOutputPath: Option[String]): Set[String] = { val partitionSet = AttributeSet(partitionColumns) val hasGpuEmpty2Null = plan.find(p => GpuV1WriteUtils.hasGpuEmptyToNull(p.expressions)) .isDefined @@ -282,7 +285,8 @@ object GpuFileFormatWriter extends Logging { sparkAttemptNumber = taskContext.taskAttemptId().toInt & Integer.MAX_VALUE, committer, iterator = iter, - concurrentOutputWriterSpec = concurrentOutputWriterSpec) + concurrentOutputWriterSpec = concurrentOutputWriterSpec, + baseDebugOutputPath = baseDebugOutputPath) }, rddWithNonEmptyPartitions.partitions.indices, (index, res: WriteTaskResult) => { @@ -389,7 +393,8 @@ object GpuFileFormatWriter extends Logging { sparkAttemptNumber: Int, committer: FileCommitProtocol, iterator: Iterator[ColumnarBatch], - concurrentOutputWriterSpec: Option[GpuConcurrentOutputWriterSpec]): WriteTaskResult = { + concurrentOutputWriterSpec: Option[GpuConcurrentOutputWriterSpec], + baseDebugOutputPath: Option[String]): WriteTaskResult = { val jobId = SparkHadoopWriterUtils.createJobID(jobTrackerID, sparkStageId) val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId) @@ -415,14 +420,16 @@ object GpuFileFormatWriter extends Logging { // In case of empty job, leave first partition to save meta for file format like parquet. new GpuEmptyDirectoryDataWriter(description, taskAttemptContext, committer) } else if (description.partitionColumns.isEmpty && description.bucketSpec.isEmpty) { - new GpuSingleDirectoryDataWriter(description, taskAttemptContext, committer) + new GpuSingleDirectoryDataWriter(description, taskAttemptContext, committer, + baseDebugOutputPath) } else { concurrentOutputWriterSpec match { case Some(spec) => new GpuDynamicPartitionDataConcurrentWriter(description, taskAttemptContext, - committer, spec) + committer, spec, baseDebugOutputPath) case _ => - new GpuDynamicPartitionDataSingleWriter(description, taskAttemptContext, committer) + new GpuDynamicPartitionDataSingleWriter(description, taskAttemptContext, committer, + baseDebugOutputPath) } } diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriterSuite.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriterSuite.scala index 033173468fd..645e17a8d6f 100644 --- a/tests/src/test/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriterSuite.scala +++ b/tests/src/test/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriterSuite.scala @@ -59,7 +59,8 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { context, dataSchema, rangeName, - includeRetry) { + includeRetry, + None) { // this writer (for tests) doesn't do anything and passes through the // batch passed to it when asked to transform, which is done to @@ -93,7 +94,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { types, "", includeRetry)) - when(mockOutputWriterFactory.newInstance(any(), any(), any())) + when(mockOutputWriterFactory.newInstance(any(), any(), any(), any())) .thenAnswer(_ => mockOutputWriter) } @@ -229,7 +230,8 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { spy(new GpuDynamicPartitionDataSingleWriter( mockJobDescription, mockTaskAttemptContext, - mockCommitter)) + mockCommitter, + None)) } def prepareDynamicPartitionConcurrentWriter(maxWriters: Int, batchSize: Long): @@ -247,7 +249,8 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { mockJobDescription, mockTaskAttemptContext, mockCommitter, - concurrentSpec)) + concurrentSpec, + None)) } test("empty directory data writer") { @@ -289,7 +292,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { withColumnarBatchesVerifyClosed(cbs) { withResource(cbs) { _ => val singleWriter = spy(new GpuSingleDirectoryDataWriter( - mockJobDescription, mockTaskAttemptContext, mockCommitter)) + mockJobDescription, mockTaskAttemptContext, mockCommitter, None)) singleWriter.writeWithIterator(Iterator.empty) singleWriter.commit() } @@ -304,7 +307,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { val cbs = Seq(spy(cb), spy(cb2)) withColumnarBatchesVerifyClosed(cbs) { val singleWriter = spy(new GpuSingleDirectoryDataWriter( - mockJobDescription, mockTaskAttemptContext, mockCommitter)) + mockJobDescription, mockTaskAttemptContext, mockCommitter, None)) singleWriter.writeWithIterator(cbs.iterator) singleWriter.commit() // we write 2 batches @@ -324,7 +327,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { // setting this to 5 makes the single writer have to split at the 5 row boundary when(mockJobDescription.maxRecordsPerFile).thenReturn(5) val singleWriter = spy(new GpuSingleDirectoryDataWriter( - mockJobDescription, mockTaskAttemptContext, mockCommitter)) + mockJobDescription, mockTaskAttemptContext, mockCommitter, None)) singleWriter.writeWithIterator(cbs.iterator) singleWriter.commit() From 231a9c67e4abe9e030fbc052997b4337a19c093e Mon Sep 17 00:00:00 2001 From: Alfred Xu Date: Thu, 19 Dec 2024 08:05:53 +0800 Subject: [PATCH 17/47] Add metrics GpuPartitioning.CopyToHostTime (#11882) Add metrics GpuPartitioning.CopyToHostTime Signed-off-by: sperlingxx Co-authored-by: Jason Lowe --- .../com/nvidia/spark/rapids/GpuExec.scala | 2 ++ .../nvidia/spark/rapids/GpuPartitioning.scala | 28 ++++++++++++++++--- .../GpuShuffleExchangeExecBase.scala | 9 +++++- 3 files changed, 34 insertions(+), 5 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala index 850a04f390f..bc67366d347 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala @@ -92,6 +92,7 @@ object GpuMetric extends Logging { val DELETION_VECTOR_SIZE = "deletionVectorSize" val CONCAT_HEADER_TIME = "concatHeaderTime" val CONCAT_BUFFER_TIME = "concatBufferTime" + val COPY_TO_HOST_TIME = "d2hMemCopyTime" // Metric Descriptions. val DESCRIPTION_BUFFER_TIME = "buffer time" @@ -133,6 +134,7 @@ object GpuMetric extends Logging { val DESCRIPTION_DELETION_VECTOR_SIZE = "deletion vector size" val DESCRIPTION_CONCAT_HEADER_TIME = "concat header time" val DESCRIPTION_CONCAT_BUFFER_TIME = "concat buffer time" + val DESCRIPTION_COPY_TO_HOST_TIME = "deviceToHost memory copy time" def unwrap(input: GpuMetric): SQLMetric = input match { case w :WrappedGpuMetric => w.sqlMetric diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuPartitioning.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuPartitioning.scala index 6394e2974b4..4fbc612591b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuPartitioning.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuPartitioning.scala @@ -126,13 +126,23 @@ trait GpuPartitioning extends Partitioning { val totalInputSize = GpuColumnVector.getTotalDeviceMemoryUsed(partitionColumns) val mightNeedToSplit = totalInputSize > GpuPartitioning.MaxCpuBatchSize - val hostPartColumns = withResource(partitionColumns) { _ => - withRetryNoSplit { - partitionColumns.safeMap(_.copyToHostAsync(Cuda.DEFAULT_STREAM)) + // We have to wrap the NvtxWithMetrics over both copyToHostAsync and corresponding CudaSync, + // because the copyToHostAsync calls above are not guaranteed to be asynchronous (e.g.: when + // the copy is from pageable memory, and we're not guaranteed to be using pinned memory). + val hostPartColumns = withResource( + new NvtxWithMetrics("PartitionD2H", NvtxColor.CYAN, memCopyTime)) { _ => + val hostColumns = withResource(partitionColumns) { _ => + withRetryNoSplit { + partitionColumns.safeMap(_.copyToHostAsync(Cuda.DEFAULT_STREAM)) + } + } + closeOnExcept(hostColumns) { _ => + Cuda.DEFAULT_STREAM.sync() } + hostColumns } + withResource(hostPartColumns) { _ => - Cuda.DEFAULT_STREAM.sync() // Leaving the GPU for a while GpuSemaphore.releaseIfNecessary(TaskContext.get()) @@ -241,4 +251,14 @@ trait GpuPartitioning extends Partitioning { } } } + + private var memCopyTime: GpuMetric = NoopMetric + + /** + * Setup sub-metrics for the performance debugging of GpuPartition. This method is expected to + * be called at the query planning stage. Therefore, this method is NOT thread safe. + */ + def setupDebugMetrics(metrics: Map[String, GpuMetric]): Unit = { + metrics.get(GpuMetric.COPY_TO_HOST_TIME).foreach(memCopyTime = _) + } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExecBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExecBase.scala index 332545a99e1..0e1b857317c 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExecBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExecBase.scala @@ -208,7 +208,8 @@ abstract class GpuShuffleExchangeExecBase( PARTITION_SIZE -> createMetric(ESSENTIAL_LEVEL, DESCRIPTION_PARTITION_SIZE), NUM_PARTITIONS -> createMetric(ESSENTIAL_LEVEL, DESCRIPTION_NUM_PARTITIONS), NUM_OUTPUT_ROWS -> createMetric(ESSENTIAL_LEVEL, DESCRIPTION_NUM_OUTPUT_ROWS), - NUM_OUTPUT_BATCHES -> createMetric(MODERATE_LEVEL, DESCRIPTION_NUM_OUTPUT_BATCHES) + NUM_OUTPUT_BATCHES -> createMetric(MODERATE_LEVEL, DESCRIPTION_NUM_OUTPUT_BATCHES), + COPY_TO_HOST_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_COPY_TO_HOST_TIME) ) ++ additionalMetrics override def nodeName: String = "GpuColumnarExchange" @@ -364,6 +365,12 @@ object GpuShuffleExchangeExecBase { rdd } val partitioner: GpuExpression = getPartitioner(newRdd, outputAttributes, newPartitioning) + // Inject debugging subMetrics, such as D2HTime before SliceOnCpu + // The injected metrics will be serialized as the members of GpuPartitioning + partitioner match { + case pt: GpuPartitioning => pt.setupDebugMetrics(metrics) + case _ => + } val partitionTime: GpuMetric = metrics(METRIC_SHUFFLE_PARTITION_TIME) def getPartitioned: ColumnarBatch => Any = { batch => partitionTime.ns { From f9c5ca64ed47a8d6f339fe109d34d767388acbfb Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Thu, 19 Dec 2024 10:15:39 +0800 Subject: [PATCH 18/47] Use faster multi-contains in `rlike` regex rewrite (#11810) * use multiple contains in rlike rewrite Signed-off-by: Haoyang Li * memory leak Signed-off-by: Haoyang Li * address comment Signed-off-by: Haoyang Li * save a temp columnvector Signed-off-by: Haoyang Li * foreach Signed-off-by: Haoyang Li * foldLeft again Signed-off-by: Haoyang Li * withResource Signed-off-by: Haoyang Li * Use AST Signed-off-by: Haoyang Li --------- Signed-off-by: Haoyang Li --- .../com/nvidia/spark/rapids/RegexParser.scala | 11 ++++--- .../spark/sql/rapids/stringFunctions.scala | 33 +++++++++++-------- .../RegularExpressionRewriteSuite.scala | 16 +++++---- 3 files changed, 35 insertions(+), 25 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala index 89fd5bf9191..2b0b46f55ea 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala @@ -22,6 +22,8 @@ import scala.collection.mutable.ListBuffer import com.nvidia.spark.rapids.GpuOverrides.regexMetaChars import com.nvidia.spark.rapids.RegexParser.toReadableString +import org.apache.spark.unsafe.types.UTF8String + /** * Regular expression parser based on a Pratt Parser design. * @@ -1988,7 +1990,7 @@ object RegexOptimizationType { case class Contains(literal: String) extends RegexOptimizationType case class PrefixRange(literal: String, length: Int, rangeStart: Int, rangeEnd: Int) extends RegexOptimizationType - case class MultipleContains(literals: Seq[String]) extends RegexOptimizationType + case class MultipleContains(literals: Seq[UTF8String]) extends RegexOptimizationType case object NoOptimization extends RegexOptimizationType } @@ -2057,16 +2059,17 @@ object RegexRewrite { } } - private def getMultipleContainsLiterals(ast: RegexAST): Seq[String] = { + private def getMultipleContainsLiterals(ast: RegexAST): Seq[UTF8String] = { ast match { case RegexGroup(_, term, _) => getMultipleContainsLiterals(term) case RegexChoice(RegexSequence(parts), ls) if isLiteralString(parts) => { getMultipleContainsLiterals(ls) match { case Seq() => Seq.empty - case literals => RegexCharsToString(parts) +: literals + case literals => UTF8String.fromString(RegexCharsToString(parts)) +: literals } } - case RegexSequence(parts) if (isLiteralString(parts)) => Seq(RegexCharsToString(parts)) + case RegexSequence(parts) if (isLiteralString(parts)) => + Seq(UTF8String.fromString(RegexCharsToString(parts))) case _ => Seq.empty } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala index 79db87f1736..f668195abc7 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala @@ -24,7 +24,7 @@ import scala.annotation.tailrec import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import ai.rapids.cudf.{BinaryOp, BinaryOperable, CaptureGroups, ColumnVector, ColumnView, DType, PadSide, RegexFlag, RegexProgram, RoundMode, Scalar} +import ai.rapids.cudf.{ast, BinaryOp, BinaryOperable, CaptureGroups, ColumnVector, ColumnView, DType, PadSide, RegexFlag, RegexProgram, RoundMode, Scalar, Table} import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.Arm._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ @@ -1202,7 +1202,7 @@ class GpuRLikeMeta( } case StartsWith(s) => GpuStartsWith(lhs, GpuLiteral(s, StringType)) case Contains(s) => GpuContains(lhs, GpuLiteral(UTF8String.fromString(s), StringType)) - case MultipleContains(ls) => GpuMultipleContains(lhs, ls) + case MultipleContains(ls) => GpuContainsAny(lhs, ls) case PrefixRange(s, length, start, end) => GpuLiteralRangePattern(lhs, GpuLiteral(s, StringType), length, start, end) case _ => throw new IllegalStateException("Unexpected optimization type") @@ -1233,7 +1233,7 @@ case class GpuRLike(left: Expression, right: Expression, pattern: String) override def dataType: DataType = BooleanType } -case class GpuMultipleContains(input: Expression, searchList: Seq[String]) +case class GpuContainsAny(input: Expression, targets: Seq[UTF8String]) extends GpuUnaryExpression with ImplicitCastInputTypes with NullIntolerantShim { override def dataType: DataType = BooleanType @@ -1242,19 +1242,24 @@ case class GpuMultipleContains(input: Expression, searchList: Seq[String]) override def inputTypes: Seq[AbstractDataType] = Seq(StringType) + def multiOrsAst: ast.AstExpression = { + (1 until targets.length) + .foldLeft(new ast.ColumnReference(0).asInstanceOf[ast.AstExpression]) { (acc, id) => + new ast.BinaryOperation(ast.BinaryOperator.NULL_LOGICAL_OR, acc, new ast.ColumnReference(id)) + } + } + override def doColumnar(input: GpuColumnVector): ColumnVector = { - assert(searchList.length > 1) - val accInit = withResource(Scalar.fromString(searchList.head)) { searchScalar => - input.getBase.stringContains(searchScalar) + val targetsBytes = targets.map(t => t.getBytes).toArray + val boolCvs = withResource(ColumnVector.fromUTF8Strings(targetsBytes: _*)) { targetsCv => + input.getBase.stringContains(targetsCv) } - searchList.tail.foldLeft(accInit) { (acc, search) => - val containsSearch = withResource(Scalar.fromString(search)) { searchScalar => - input.getBase.stringContains(searchScalar) - } - withResource(acc) { _ => - withResource(containsSearch) { _ => - acc.or(containsSearch) - } + val boolTable = withResource(boolCvs) { _ => + new Table(boolCvs: _*) + } + withResource(boolTable) { _ => + withResource(multiOrsAst.compile()) { compiledAst => + compiledAst.computeColumn(boolTable) } } } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionRewriteSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionRewriteSuite.scala index a55815b95ef..12e12fd957f 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionRewriteSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionRewriteSuite.scala @@ -17,10 +17,12 @@ package com.nvidia.spark.rapids import org.scalatest.funsuite.AnyFunSuite +import org.apache.spark.unsafe.types.UTF8String + class RegularExpressionRewriteSuite extends AnyFunSuite { - private def verifyRewritePattern(patterns: Seq[String], excepted: Seq[RegexOptimizationType]): - Unit = { + private def verifyRewritePattern(patterns: Seq[String], + excepted: Seq[RegexOptimizationType]): Unit = { val results = patterns.map { pattern => val ast = new RegexParser(pattern).parse() RegexRewrite.matchSimplePattern(ast) @@ -87,11 +89,11 @@ class RegularExpressionRewriteSuite extends AnyFunSuite { "(火花|急流)" ) val excepted = Seq( - MultipleContains(Seq("abc", "def")), - MultipleContains(Seq("abc", "def", "ghi")), - MultipleContains(Seq("abc", "def")), - MultipleContains(Seq("abc", "def")), - MultipleContains(Seq("火花", "急流")) + MultipleContains(Seq("abc", "def").map(UTF8String.fromString)), + MultipleContains(Seq("abc", "def", "ghi").map(UTF8String.fromString)), + MultipleContains(Seq("abc", "def").map(UTF8String.fromString)), + MultipleContains(Seq("abc", "def").map(UTF8String.fromString)), + MultipleContains(Seq("火花", "急流").map(UTF8String.fromString)) ) verifyRewritePattern(patterns, excepted) } From 62b23969847d2eec94027a929a25bc18cc0c3ef0 Mon Sep 17 00:00:00 2001 From: YanxuanLiu Date: Thu, 19 Dec 2024 16:32:06 +0800 Subject: [PATCH 19/47] Fix bug: add timeout for cache deps steps [skip ci] (#11894) * add timeout for populate cache step Signed-off-by: Yanxuan Liu * add continue to cache steps Signed-off-by: Yanxuan Liu --------- Signed-off-by: Yanxuan Liu --- .github/workflows/mvn-verify-check.yml | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/.github/workflows/mvn-verify-check.yml b/.github/workflows/mvn-verify-check.yml index b58799c6110..c1c47f7bae4 100644 --- a/.github/workflows/mvn-verify-check.yml +++ b/.github/workflows/mvn-verify-check.yml @@ -58,12 +58,15 @@ jobs: echo "dailyCacheKey=$cacheKey" | tee $GITHUB_ENV $GITHUB_OUTPUT - name: Cache local Maven repository id: cache + continue-on-error: true uses: actions/cache@v4 with: path: ~/.m2 key: ${{ env.dailyCacheKey }} restore-keys: ${{ runner.os }}-maven- - name: populate-daily-cache + timeout-minutes: 30 + continue-on-error: true if: steps.cache.outputs.cache-hit != 'true' env: SCALA_VER: '2.12' @@ -118,6 +121,7 @@ jobs: java-version: 8 - name: Cache local Maven repository + continue-on-error: true uses: actions/cache@v4 with: path: ~/.m2 @@ -171,12 +175,15 @@ jobs: echo "scala213dailyCacheKey=$cacheKey" | tee $GITHUB_ENV $GITHUB_OUTPUT - name: Cache local Maven repository id: cache + continue-on-error: true uses: actions/cache@v4 with: path: ~/.m2 key: ${{ env.scala213dailyCacheKey }} restore-keys: ${{ runner.os }}-maven- - name: populate-daily-cache + timeout-minutes: 30 + continue-on-error: true if: steps.cache.outputs.cache-hit != 'true' env: SCALA_VER: '2.13' @@ -223,6 +230,7 @@ jobs: java-version: 17 - name: Cache local Maven repository + continue-on-error: true uses: actions/cache@v4 with: path: ~/.m2 @@ -278,6 +286,7 @@ jobs: java-version: 17 - name: Cache local Maven repository + continue-on-error: true uses: actions/cache@v4 with: path: ~/.m2 @@ -332,6 +341,7 @@ jobs: java-version: ${{ matrix.java-version }} - name: Cache local Maven repository + continue-on-error: true uses: actions/cache@v4 with: path: ~/.m2 @@ -379,6 +389,7 @@ jobs: java-version: 11 - name: Cache local Maven repository + continue-on-error: true uses: actions/cache@v4 with: path: ~/.m2 From 0e5b5cb4ed1301bdb79aca5282a57aa816db8e22 Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Thu, 19 Dec 2024 11:01:40 -0600 Subject: [PATCH 20/47] Make sure that the chunked packer bounce buffer is realease after the synchronize (#11887) Signed-off-by: Alessandro Bellina --- .../scala/com/nvidia/spark/rapids/spill/SpillFramework.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/spill/SpillFramework.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/spill/SpillFramework.scala index 57f2a823432..91421a37c64 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/spill/SpillFramework.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/spill/SpillFramework.scala @@ -1719,7 +1719,7 @@ class ChunkedPacker(table: Table, } override def next(): (DeviceBounceBuffer, Long) = { - withResource(bounceBufferPool.nextBuffer()) { bounceBuffer => + closeOnExcept(bounceBufferPool.nextBuffer()) { bounceBuffer => if (closed) { throw new IllegalStateException(s"ChunkedPacker is closed") } From 6244613be45b39b74611b0279d4dba164d6adcba Mon Sep 17 00:00:00 2001 From: MithunR Date: Thu, 19 Dec 2024 10:06:55 -0800 Subject: [PATCH 21/47] Support group-limit optimization for `ROW_NUMBER` (#11886) * Support group-limit optimization for `ROW_NUMBER` Fixes #10505. This is a follow-up to #10500, which added support for WindowGroupLimit optimizations for `RANK` and `DENSE_RANK` window functions. The same optimization was not extended to `ROW_NUMBER` at that time. This commit now allows the output from `ROW_NUMBER` to be filtered map-side, in case there is a `<` predicate on its return value. The following is an example of the kind of query that is affected by this change: ```sql SELECT * FROM ( SELECT *, ROW_NUMBER() OVER (PARTITION BY p ORDER BY o) AS rn FROM mytable ) WHERE rn < 10; ``` This is per the optimization in [SPARK-37099](https://issues.apache.org/jira/browse/SPARK-37099) in Apache Spark. With this, the output from the window function could potentially be drastically smaller than the input, thus saving on shuffle traffic. Note that this optimization does not kick in on Apache Spark or in `spark-rapids` if the `ROW_NUMBER` phrase does not include a `PARTITION BY` clause. `spark-rapids` remains consistent with Apache Spark in this regard. Signed-off-by: MithunR --- .../src/main/python/window_function_test.py | 37 ++----------------- .../shims/GpuWindowGroupLimitExec.scala | 14 ++++--- 2 files changed, 12 insertions(+), 39 deletions(-) diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py index f792bd2304d..44efe841beb 100644 --- a/integration_tests/src/main/python/window_function_test.py +++ b/integration_tests/src/main/python/window_function_test.py @@ -2107,7 +2107,9 @@ def assert_query_runs_on(exec, conf): 'DENSE_RANK() OVER (PARTITION BY a ORDER BY b, c) ', 'RANK() OVER (ORDER BY a,b,c) ', 'DENSE_RANK() OVER (ORDER BY a,b,c) ', - ]) + # ROW_NUMBER() on an un-partitioned window does not invoke WindowGroupLimit optimization. + 'ROW_NUMBER() OVER (PARTITION BY a ORDER BY b,c) ', +]) def test_window_group_limits_for_ranking_functions(data_gen, batch_size, rank_clause): """ This test verifies that window group limits are applied for queries with ranking-function based @@ -2133,39 +2135,6 @@ def test_window_group_limits_for_ranking_functions(data_gen, batch_size, rank_cl conf=conf) -@allow_non_gpu('WindowGroupLimitExec') -@pytest.mark.skipif(condition=not (is_spark_350_or_later() or is_databricks133_or_later()), - reason="WindowGroupLimit not available for spark.version < 3.5 " - " and Databricks version < 13.3") -@ignore_order(local=True) -@approximate_float -def test_window_group_limits_fallback_for_row_number(): - """ - This test verifies that window group limits are applied for queries with ranking-function based - row filters. - This test covers RANK() and DENSE_RANK(), for window function with and without `PARTITIONED BY` - clauses. - """ - conf = {'spark.rapids.sql.batchSizeBytes': '1g', - 'spark.rapids.sql.castFloatToDecimal.enabled': True} - - data_gen = _grpkey_longs_with_no_nulls - query = """ - SELECT * FROM ( - SELECT *, ROW_NUMBER() OVER (PARTITION BY a ORDER BY b, c) AS rnk - FROM window_agg_table - ) - WHERE rnk < 3 - """ - - assert_gpu_sql_fallback_collect( - lambda spark: gen_df(spark, data_gen, length=512), - cpu_fallback_class_name="WindowGroupLimitExec", - table_name="window_agg_table", - sql=query, - conf=conf) - - def test_lru_cache_datagen(): # log cache info at the end of integration tests, not related to window functions info = gen_df_help.cache_info() diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuWindowGroupLimitExec.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuWindowGroupLimitExec.scala index d8d63fe2be0..eca69254d61 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuWindowGroupLimitExec.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuWindowGroupLimitExec.scala @@ -30,12 +30,12 @@ import ai.rapids.cudf.GroupByOptions import com.nvidia.spark.rapids.{BaseExprMeta, DataFromReplacementRule, GpuBindReferences, GpuBoundReference, GpuColumnVector, GpuExec, GpuExpression, GpuMetric, GpuOverrides, GpuProjectExec, RapidsConf, RapidsMeta, SparkPlanMeta, SpillableColumnarBatch, SpillPriorities} import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitSpillableInHalfByRows, withRetry} -import com.nvidia.spark.rapids.window.{GpuDenseRank, GpuRank} +import com.nvidia.spark.rapids.window.{GpuDenseRank, GpuRank, GpuRowNumber} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, DenseRank, Expression, Rank, SortOrder} +import org.apache.spark.sql.catalyst.expressions.{Attribute, DenseRank, Expression, Rank, RowNumber, SortOrder} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.window.{Final, Partial, WindowGroupLimitExec, WindowGroupLimitMode} import org.apache.spark.sql.types.DataType @@ -44,6 +44,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch sealed trait RankFunctionType case object RankFunction extends RankFunctionType case object DenseRankFunction extends RankFunctionType +case object RowNumberFunction extends RankFunctionType class GpuWindowGroupLimitExecMeta(limitExec: WindowGroupLimitExec, conf: RapidsConf, @@ -63,8 +64,8 @@ class GpuWindowGroupLimitExecMeta(limitExec: WindowGroupLimitExec, wrapped.rankLikeFunction match { case DenseRank(_) => case Rank(_) => - // case RowNumber() => // TODO: Future. - case _ => willNotWorkOnGpu("Only Rank() and DenseRank() are " + + case RowNumber() => + case _ => willNotWorkOnGpu("Only rank, dense_rank and row_number are " + "currently supported for window group limits") } @@ -166,6 +167,7 @@ class GpuWindowGroupLimitingIterator(input: Iterator[ColumnarBatch], rankFunctionType match { case RankFunction => GpuRank(sortColumns) case DenseRankFunction => GpuDenseRank(sortColumns) + case RowNumberFunction => GpuRowNumber case _ => throw new IllegalArgumentException("Unexpected ranking function") } } @@ -300,8 +302,10 @@ case class GpuWindowGroupLimitExec( private def getRankFunctionType(expr: Expression): RankFunctionType = expr match { case GpuRank(_) => RankFunction case GpuDenseRank(_) => DenseRankFunction + case GpuRowNumber => RowNumberFunction case _ => - throw new UnsupportedOperationException("Only Rank() is currently supported for group limits") + throw new UnsupportedOperationException("Only rank, dense_rank and row_number are " + + "currently supported for group limits") } override protected def internalDoExecuteColumnar(): RDD[ColumnarBatch] = { From ea007dab0839227cf48043cedd4f0b56cfc6b3c1 Mon Sep 17 00:00:00 2001 From: MithunR Date: Thu, 19 Dec 2024 14:35:31 -0800 Subject: [PATCH 22/47] Update operatorsScore,supportedExprs for TruncDate, TruncTimestamp (#11890) This change updates `operatorsScore.csv` and `supportedExprs.csv` to include `TruncDate` and `TruncTimestamp`, for all shim versions. This seems to have been left out of #11833. Signed-off-by: MithunR --- tools/generated_files/321cdh/operatorsScore.csv | 2 ++ tools/generated_files/321cdh/supportedExprs.csv | 6 ++++++ tools/generated_files/322/operatorsScore.csv | 2 ++ tools/generated_files/322/supportedExprs.csv | 6 ++++++ tools/generated_files/323/operatorsScore.csv | 2 ++ tools/generated_files/323/supportedExprs.csv | 6 ++++++ tools/generated_files/324/operatorsScore.csv | 2 ++ tools/generated_files/324/supportedExprs.csv | 6 ++++++ tools/generated_files/330cdh/operatorsScore.csv | 2 ++ tools/generated_files/330cdh/supportedExprs.csv | 6 ++++++ tools/generated_files/332/operatorsScore.csv | 2 ++ tools/generated_files/332/supportedExprs.csv | 6 ++++++ tools/generated_files/332cdh/operatorsScore.csv | 2 ++ tools/generated_files/332cdh/supportedExprs.csv | 6 ++++++ tools/generated_files/333/operatorsScore.csv | 2 ++ tools/generated_files/333/supportedExprs.csv | 6 ++++++ tools/generated_files/334/operatorsScore.csv | 2 ++ tools/generated_files/334/supportedExprs.csv | 6 ++++++ tools/generated_files/341/operatorsScore.csv | 2 ++ tools/generated_files/341/supportedExprs.csv | 6 ++++++ tools/generated_files/343/operatorsScore.csv | 2 ++ tools/generated_files/343/supportedExprs.csv | 6 ++++++ tools/generated_files/344/operatorsScore.csv | 2 ++ tools/generated_files/344/supportedExprs.csv | 6 ++++++ tools/generated_files/350/operatorsScore.csv | 2 ++ tools/generated_files/350/supportedExprs.csv | 6 ++++++ tools/generated_files/351/operatorsScore.csv | 2 ++ tools/generated_files/351/supportedExprs.csv | 6 ++++++ tools/generated_files/352/operatorsScore.csv | 2 ++ tools/generated_files/352/supportedExprs.csv | 6 ++++++ tools/generated_files/353/operatorsScore.csv | 2 ++ tools/generated_files/353/supportedExprs.csv | 6 ++++++ 32 files changed, 128 insertions(+) diff --git a/tools/generated_files/321cdh/operatorsScore.csv b/tools/generated_files/321cdh/operatorsScore.csv index 19c999aa796..d8c4ca63adc 100644 --- a/tools/generated_files/321cdh/operatorsScore.csv +++ b/tools/generated_files/321cdh/operatorsScore.csv @@ -265,6 +265,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/321cdh/supportedExprs.csv b/tools/generated_files/321cdh/supportedExprs.csv index 573367d1d70..39d69c0836b 100644 --- a/tools/generated_files/321cdh/supportedExprs.csv +++ b/tools/generated_files/321cdh/supportedExprs.csv @@ -606,6 +606,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NS,NS UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NS,NS UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/322/operatorsScore.csv b/tools/generated_files/322/operatorsScore.csv index 19c999aa796..d8c4ca63adc 100644 --- a/tools/generated_files/322/operatorsScore.csv +++ b/tools/generated_files/322/operatorsScore.csv @@ -265,6 +265,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/322/supportedExprs.csv b/tools/generated_files/322/supportedExprs.csv index 573367d1d70..39d69c0836b 100644 --- a/tools/generated_files/322/supportedExprs.csv +++ b/tools/generated_files/322/supportedExprs.csv @@ -606,6 +606,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NS,NS UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NS,NS UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/323/operatorsScore.csv b/tools/generated_files/323/operatorsScore.csv index 19c999aa796..d8c4ca63adc 100644 --- a/tools/generated_files/323/operatorsScore.csv +++ b/tools/generated_files/323/operatorsScore.csv @@ -265,6 +265,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/323/supportedExprs.csv b/tools/generated_files/323/supportedExprs.csv index 573367d1d70..39d69c0836b 100644 --- a/tools/generated_files/323/supportedExprs.csv +++ b/tools/generated_files/323/supportedExprs.csv @@ -606,6 +606,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NS,NS UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NS,NS UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/324/operatorsScore.csv b/tools/generated_files/324/operatorsScore.csv index 19c999aa796..d8c4ca63adc 100644 --- a/tools/generated_files/324/operatorsScore.csv +++ b/tools/generated_files/324/operatorsScore.csv @@ -265,6 +265,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/324/supportedExprs.csv b/tools/generated_files/324/supportedExprs.csv index 573367d1d70..39d69c0836b 100644 --- a/tools/generated_files/324/supportedExprs.csv +++ b/tools/generated_files/324/supportedExprs.csv @@ -606,6 +606,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NS,NS UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NS,NS UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/330cdh/operatorsScore.csv b/tools/generated_files/330cdh/operatorsScore.csv index e5978fb9f1a..e86e30e606c 100644 --- a/tools/generated_files/330cdh/operatorsScore.csv +++ b/tools/generated_files/330cdh/operatorsScore.csv @@ -275,6 +275,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/330cdh/supportedExprs.csv b/tools/generated_files/330cdh/supportedExprs.csv index e1a4492676c..c8df29cfca5 100644 --- a/tools/generated_files/330cdh/supportedExprs.csv +++ b/tools/generated_files/330cdh/supportedExprs.csv @@ -627,6 +627,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/332/operatorsScore.csv b/tools/generated_files/332/operatorsScore.csv index b988344e702..229201ba885 100644 --- a/tools/generated_files/332/operatorsScore.csv +++ b/tools/generated_files/332/operatorsScore.csv @@ -276,6 +276,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/332/supportedExprs.csv b/tools/generated_files/332/supportedExprs.csv index 7329b2c4756..e217b6ce31d 100644 --- a/tools/generated_files/332/supportedExprs.csv +++ b/tools/generated_files/332/supportedExprs.csv @@ -629,6 +629,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/332cdh/operatorsScore.csv b/tools/generated_files/332cdh/operatorsScore.csv index b988344e702..229201ba885 100644 --- a/tools/generated_files/332cdh/operatorsScore.csv +++ b/tools/generated_files/332cdh/operatorsScore.csv @@ -276,6 +276,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/332cdh/supportedExprs.csv b/tools/generated_files/332cdh/supportedExprs.csv index 7329b2c4756..e217b6ce31d 100644 --- a/tools/generated_files/332cdh/supportedExprs.csv +++ b/tools/generated_files/332cdh/supportedExprs.csv @@ -629,6 +629,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/333/operatorsScore.csv b/tools/generated_files/333/operatorsScore.csv index b988344e702..229201ba885 100644 --- a/tools/generated_files/333/operatorsScore.csv +++ b/tools/generated_files/333/operatorsScore.csv @@ -276,6 +276,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/333/supportedExprs.csv b/tools/generated_files/333/supportedExprs.csv index 7329b2c4756..e217b6ce31d 100644 --- a/tools/generated_files/333/supportedExprs.csv +++ b/tools/generated_files/333/supportedExprs.csv @@ -629,6 +629,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/334/operatorsScore.csv b/tools/generated_files/334/operatorsScore.csv index b988344e702..229201ba885 100644 --- a/tools/generated_files/334/operatorsScore.csv +++ b/tools/generated_files/334/operatorsScore.csv @@ -276,6 +276,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/334/supportedExprs.csv b/tools/generated_files/334/supportedExprs.csv index 7329b2c4756..e217b6ce31d 100644 --- a/tools/generated_files/334/supportedExprs.csv +++ b/tools/generated_files/334/supportedExprs.csv @@ -629,6 +629,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/341/operatorsScore.csv b/tools/generated_files/341/operatorsScore.csv index b1e9198e58b..16ac93a02ba 100644 --- a/tools/generated_files/341/operatorsScore.csv +++ b/tools/generated_files/341/operatorsScore.csv @@ -277,6 +277,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/341/supportedExprs.csv b/tools/generated_files/341/supportedExprs.csv index 22b1f73d68b..0c2d6a74bc1 100644 --- a/tools/generated_files/341/supportedExprs.csv +++ b/tools/generated_files/341/supportedExprs.csv @@ -629,6 +629,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/343/operatorsScore.csv b/tools/generated_files/343/operatorsScore.csv index b1e9198e58b..16ac93a02ba 100644 --- a/tools/generated_files/343/operatorsScore.csv +++ b/tools/generated_files/343/operatorsScore.csv @@ -277,6 +277,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/343/supportedExprs.csv b/tools/generated_files/343/supportedExprs.csv index 22b1f73d68b..0c2d6a74bc1 100644 --- a/tools/generated_files/343/supportedExprs.csv +++ b/tools/generated_files/343/supportedExprs.csv @@ -629,6 +629,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/344/operatorsScore.csv b/tools/generated_files/344/operatorsScore.csv index b1e9198e58b..16ac93a02ba 100644 --- a/tools/generated_files/344/operatorsScore.csv +++ b/tools/generated_files/344/operatorsScore.csv @@ -277,6 +277,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/344/supportedExprs.csv b/tools/generated_files/344/supportedExprs.csv index 22b1f73d68b..0c2d6a74bc1 100644 --- a/tools/generated_files/344/supportedExprs.csv +++ b/tools/generated_files/344/supportedExprs.csv @@ -629,6 +629,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/350/operatorsScore.csv b/tools/generated_files/350/operatorsScore.csv index 3b0b82d58bf..cfb5b486942 100644 --- a/tools/generated_files/350/operatorsScore.csv +++ b/tools/generated_files/350/operatorsScore.csv @@ -279,6 +279,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/350/supportedExprs.csv b/tools/generated_files/350/supportedExprs.csv index 3c7ee2a51eb..09cea891f97 100644 --- a/tools/generated_files/350/supportedExprs.csv +++ b/tools/generated_files/350/supportedExprs.csv @@ -637,6 +637,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/351/operatorsScore.csv b/tools/generated_files/351/operatorsScore.csv index 3b0b82d58bf..cfb5b486942 100644 --- a/tools/generated_files/351/operatorsScore.csv +++ b/tools/generated_files/351/operatorsScore.csv @@ -279,6 +279,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/351/supportedExprs.csv b/tools/generated_files/351/supportedExprs.csv index 3c7ee2a51eb..09cea891f97 100644 --- a/tools/generated_files/351/supportedExprs.csv +++ b/tools/generated_files/351/supportedExprs.csv @@ -637,6 +637,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/352/operatorsScore.csv b/tools/generated_files/352/operatorsScore.csv index 3b0b82d58bf..cfb5b486942 100644 --- a/tools/generated_files/352/operatorsScore.csv +++ b/tools/generated_files/352/operatorsScore.csv @@ -279,6 +279,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/352/supportedExprs.csv b/tools/generated_files/352/supportedExprs.csv index 3c7ee2a51eb..09cea891f97 100644 --- a/tools/generated_files/352/supportedExprs.csv +++ b/tools/generated_files/352/supportedExprs.csv @@ -637,6 +637,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS diff --git a/tools/generated_files/353/operatorsScore.csv b/tools/generated_files/353/operatorsScore.csv index 3b0b82d58bf..cfb5b486942 100644 --- a/tools/generated_files/353/operatorsScore.csv +++ b/tools/generated_files/353/operatorsScore.csv @@ -279,6 +279,8 @@ ToUTCTimestamp,4 ToUnixTimestamp,4 TransformKeys,4 TransformValues,4 +TruncDate,4 +TruncTimestamp,4 UnaryMinus,4 UnaryPositive,4 UnboundedFollowing$,4 diff --git a/tools/generated_files/353/supportedExprs.csv b/tools/generated_files/353/supportedExprs.csv index 3c7ee2a51eb..09cea891f97 100644 --- a/tools/generated_files/353/supportedExprs.csv +++ b/tools/generated_files/353/supportedExprs.csv @@ -637,6 +637,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS From adb89aadabb887e6f1577a6ff200eaaa3be7b2ba Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Fri, 20 Dec 2024 07:40:09 +0800 Subject: [PATCH 23/47] Xxhash64 supports nested types [databricks] (#11859) Signed-off-by: Chong Gao Co-authored-by: Chong Gao --- docs/supported_ops.md | 6 +- .../src/main/python/hashing_test.py | 140 +++++++++++++++--- .../nvidia/spark/rapids/GpuOverrides.scala | 15 ++ .../spark/sql/rapids/HashFunctions.scala | 25 ++++ .../spark/rapids/shims/XxHash64Shims.scala | 3 +- tools/generated_files/320/supportedExprs.csv | 2 +- tools/generated_files/321/supportedExprs.csv | 2 +- .../generated_files/321cdh/supportedExprs.csv | 2 +- tools/generated_files/322/supportedExprs.csv | 2 +- tools/generated_files/323/supportedExprs.csv | 2 +- tools/generated_files/324/supportedExprs.csv | 2 +- tools/generated_files/330/supportedExprs.csv | 2 +- .../generated_files/330cdh/supportedExprs.csv | 2 +- tools/generated_files/331/supportedExprs.csv | 2 +- tools/generated_files/332/supportedExprs.csv | 2 +- .../generated_files/332cdh/supportedExprs.csv | 2 +- tools/generated_files/333/supportedExprs.csv | 2 +- tools/generated_files/334/supportedExprs.csv | 2 +- tools/generated_files/340/supportedExprs.csv | 2 +- tools/generated_files/341/supportedExprs.csv | 2 +- tools/generated_files/342/supportedExprs.csv | 2 +- tools/generated_files/343/supportedExprs.csv | 2 +- tools/generated_files/344/supportedExprs.csv | 2 +- tools/generated_files/350/supportedExprs.csv | 2 +- tools/generated_files/351/supportedExprs.csv | 2 +- tools/generated_files/352/supportedExprs.csv | 2 +- tools/generated_files/353/supportedExprs.csv | 2 +- tools/generated_files/supportedExprs.csv | 2 +- 28 files changed, 187 insertions(+), 48 deletions(-) diff --git a/docs/supported_ops.md b/docs/supported_ops.md index 2ac5a27daa7..2bc819b4a41 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -18553,9 +18553,9 @@ are limited. S NS NS -NS -NS -NS +PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT, DAYTIME, YEARMONTH
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT, DAYTIME, YEARMONTH
+PS
UTC is only supported TZ for child TIMESTAMP;
unsupported child types BINARY, CALENDAR, UDT, DAYTIME, YEARMONTH
NS NS NS diff --git a/integration_tests/src/main/python/hashing_test.py b/integration_tests/src/main/python/hashing_test.py index 6bd56da933d..e3ef67bc9f9 100644 --- a/integration_tests/src/main/python/hashing_test.py +++ b/integration_tests/src/main/python/hashing_test.py @@ -1,4 +1,4 @@ -# Copyright (c) 2023, NVIDIA CORPORATION. +# Copyright (c) 2023-2024, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -17,10 +17,8 @@ from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_fallback_collect from data_gen import * from marks import allow_non_gpu, ignore_order -from spark_session import is_before_spark_320 -# Spark 3.1.x does not normalize -0.0 and 0.0 but GPU version does -_xxhash_gens = [ +_atomic_gens = [ null_gen, boolean_gen, byte_gen, @@ -31,36 +29,136 @@ timestamp_gen, decimal_gen_32bit, decimal_gen_64bit, - decimal_gen_128bit] -if not is_before_spark_320(): - _xxhash_gens += [float_gen, double_gen] + decimal_gen_128bit, + float_gen, + double_gen +] -_struct_of_xxhash_gens = StructGen([(f"c{i}", g) for i, g in enumerate(_xxhash_gens)]) +_struct_of_xxhash_gens = StructGen([(f"c{i}", g) for i, g in enumerate(_atomic_gens)]) + +# will be used by HyperLogLogPlusPLus(approx_count_distinct) +xxhash_gens = (_atomic_gens + [_struct_of_xxhash_gens] + single_level_array_gens + + nested_array_gens_sample + [ + all_basic_struct_gen, + struct_array_gen, + _struct_of_xxhash_gens + ] + map_gens_sample) -_xxhash_fallback_gens = single_level_array_gens + nested_array_gens_sample + [ - all_basic_struct_gen, - struct_array_gen, - _struct_of_xxhash_gens] -if is_before_spark_320(): - _xxhash_fallback_gens += [float_gen, double_gen] @ignore_order(local=True) -@pytest.mark.parametrize("gen", _xxhash_gens, ids=idfn) +@pytest.mark.parametrize("gen", xxhash_gens, ids=idfn) def test_xxhash64_single_column(gen): assert_gpu_and_cpu_are_equal_collect( - lambda spark : unary_op_df(spark, gen).selectExpr("a", "xxhash64(a)")) + lambda spark: unary_op_df(spark, gen).selectExpr("a", "xxhash64(a)"), + {"spark.sql.legacy.allowHashOnMapType": True}) + @ignore_order(local=True) def test_xxhash64_multi_column(): gen = StructGen(_struct_of_xxhash_gens.children, nullable=False) col_list = ",".join(gen.data_type.fieldNames()) assert_gpu_and_cpu_are_equal_collect( - lambda spark : gen_df(spark, gen).selectExpr("c0", f"xxhash64({col_list})")) + lambda spark: gen_df(spark, gen).selectExpr("c0", f"xxhash64({col_list})"), + {"spark.sql.legacy.allowHashOnMapType": True}) + + +def test_xxhash64_8_depth(): + gen_8_depth = ( + StructGen([('l1', # level 1 + StructGen([('l2', + StructGen([('l3', + StructGen([('l4', + StructGen([('l5', + StructGen([('l6', + StructGen([('l7', + int_gen)]))]))]))]))]))]))])) # level 8 + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, gen_8_depth).selectExpr("a", "xxhash64(a)")) + @allow_non_gpu("ProjectExec") -@ignore_order(local=True) -@pytest.mark.parametrize("gen", _xxhash_fallback_gens, ids=idfn) -def test_xxhash64_fallback(gen): +def test_xxhash64_fallback_exceeds_stack_size_array_of_structure(): + gen_9_depth = ( + ArrayGen( # depth += 1 + StructGen([('c', # depth += 1 + ArrayGen( # depth += 1 + StructGen([('c', # depth += 1 + ArrayGen( # depth += 1 + StructGen([('c', # depth += 1 + ArrayGen( # depth += 1 + StructGen([('c', # depth += 1 + int_gen)]), # depth += 1 + max_length=1))]), + max_length=1))]), + max_length=1))]), + max_length=1)) + assert_gpu_fallback_collect( + lambda spark: unary_op_df(spark, gen_9_depth).selectExpr("a", "xxhash64(a)"), + "ProjectExec") + + +@allow_non_gpu("ProjectExec") +def test_xxhash64_array_of_other(): + gen_9_depth = ( + ArrayGen( # array(other: not struct): depth += 0 + ArrayGen( # array(other: not struct): depth += 0 + ArrayGen( # array(other: not struct): depth += 0 + MapGen( # map: depth += 2 + IntegerGen(nullable=False), + ArrayGen( # array(other: not struct): depth += 0 + MapGen( # map: depth += 2 + IntegerGen(nullable=False), + ArrayGen( # array(other: not struct): depth += 0 + MapGen( # map: depth += 2 + IntegerGen(nullable=False), + int_gen, # primitive: depth += 1 + max_length=1), + max_length=1), + max_length=1), + max_length=1), + max_length=1), + max_length=1), + max_length=1), + max_length=1)) + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, gen_9_depth).selectExpr("a", "xxhash64(a)"), + {"spark.sql.legacy.allowHashOnMapType": True}) + + +@allow_non_gpu("ProjectExec") +def test_xxhash64_fallback_exceeds_stack_size_structure(): + gen_9_depth = ( + StructGen([('l1', # level 1 + StructGen([('l2', + StructGen([('l3', + StructGen([('l4', + StructGen([('l5', + StructGen([('l6', + StructGen([('l7', + StructGen([('l8', + int_gen)]))]))]))]))]))]))]))])) # level 9 assert_gpu_fallback_collect( - lambda spark : unary_op_df(spark, gen).selectExpr("a", "xxhash64(a)"), + lambda spark: unary_op_df(spark, gen_9_depth).selectExpr("a", "xxhash64(a)"), "ProjectExec") + + +@allow_non_gpu("ProjectExec") +def test_xxhash64_fallback_exceeds_stack_size_map(): + gen_9_depth = ( + MapGen( # depth += 2 + IntegerGen(nullable=False), + MapGen( # depth += 2 + IntegerGen(nullable=False), + MapGen( # depth += 2 + IntegerGen(nullable=False), + MapGen( # depth += 2 + IntegerGen(nullable=False), # depth += 1 + IntegerGen(nullable=False), + max_length=1), + max_length=1), + max_length=1), + max_length=1)) + assert_gpu_fallback_collect( + lambda spark: unary_op_df(spark, gen_9_depth).selectExpr("a", "xxhash64(a)"), + "ProjectExec", + {"spark.sql.legacy.allowHashOnMapType": True}) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 9d28b2f1bf3..c1ede7554a6 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 @@ -3329,6 +3329,21 @@ object GpuOverrides extends Logging { override val childExprs: Seq[BaseExprMeta[_]] = a.children .map(GpuOverrides.wrapExpr(_, this.conf, Some(this))) + override def tagExprForGpu(): Unit = { + val maxDepth = a.children.map( + c => XxHash64Utils.computeMaxStackSize(c.dataType)).max + if (maxDepth > Hash.MAX_STACK_DEPTH) { + willNotWorkOnGpu(s"The data type requires a stack depth of $maxDepth, " + + s"which exceeds the GPU limit of ${Hash.MAX_STACK_DEPTH}. " + + "The algorithm to calculate stack depth: " + + "1: Primitive type counts 1 depth; " + + "2: Array of Structure counts: 1 + depthOf(Structure); " + + "3: Array of Other counts: depthOf(Other); " + + "4: Structure counts: 1 + max of depthOf(child); " + + "5: Map counts: 2 + max(depthOf(key), depthOf(value)); " + ) + } + } def convertToGpu(): GpuExpression = GpuXxHash64(childExprs.map(_.convertToGpu()), a.seed) }), diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/HashFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/HashFunctions.scala index 854b905baf6..26a864a7c36 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/HashFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/HashFunctions.scala @@ -108,6 +108,31 @@ case class GpuXxHash64(children: Seq[Expression], seed: Long) extends GpuHashExp } } +object XxHash64Utils { + /** + * Compute the max stack size that `inputType` will use, + * refer to the function `check_nested_depth` in src/main/cpp/src/xxhash64.cu + * in spark-rapids-jni repo. + * Note: + * - This should be sync with `check_nested_depth` + * - Map in cuDF is list of struct + * + * @param inputType the input type + * @return the max stack size that inputType will use for this input type. + */ + def computeMaxStackSize(inputType: DataType): Int = { + inputType match { + case ArrayType(c: StructType, _) => 1 + computeMaxStackSize(c) + case ArrayType(c, _) => computeMaxStackSize(c) + case st: StructType => + 1 + st.map(f => computeMaxStackSize(f.dataType)).max + case mt: MapType => + 2 + math.max(computeMaxStackSize(mt.keyType), computeMaxStackSize(mt.valueType)) + case _ => 1 // primitive types + } + } +} + case class GpuHiveHash(children: Seq[Expression]) extends GpuHashExpression { override def dataType: DataType = IntegerType diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/XxHash64Shims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/XxHash64Shims.scala index 18959d2c10f..f470e9868a1 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/XxHash64Shims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/XxHash64Shims.scala @@ -48,5 +48,6 @@ package com.nvidia.spark.rapids.shims import com.nvidia.spark.rapids.TypeSig object XxHash64Shims { - val supportedTypes: TypeSig = TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + val supportedTypes: TypeSig = (TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP).nested() } diff --git a/tools/generated_files/320/supportedExprs.csv b/tools/generated_files/320/supportedExprs.csv index 39d69c0836b..a9b6720be13 100644 --- a/tools/generated_files/320/supportedExprs.csv +++ b/tools/generated_files/320/supportedExprs.csv @@ -637,7 +637,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/321/supportedExprs.csv b/tools/generated_files/321/supportedExprs.csv index 39d69c0836b..a9b6720be13 100644 --- a/tools/generated_files/321/supportedExprs.csv +++ b/tools/generated_files/321/supportedExprs.csv @@ -637,7 +637,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/321cdh/supportedExprs.csv b/tools/generated_files/321cdh/supportedExprs.csv index 39d69c0836b..a9b6720be13 100644 --- a/tools/generated_files/321cdh/supportedExprs.csv +++ b/tools/generated_files/321cdh/supportedExprs.csv @@ -637,7 +637,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/322/supportedExprs.csv b/tools/generated_files/322/supportedExprs.csv index 39d69c0836b..a9b6720be13 100644 --- a/tools/generated_files/322/supportedExprs.csv +++ b/tools/generated_files/322/supportedExprs.csv @@ -637,7 +637,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/323/supportedExprs.csv b/tools/generated_files/323/supportedExprs.csv index 39d69c0836b..a9b6720be13 100644 --- a/tools/generated_files/323/supportedExprs.csv +++ b/tools/generated_files/323/supportedExprs.csv @@ -637,7 +637,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/324/supportedExprs.csv b/tools/generated_files/324/supportedExprs.csv index 39d69c0836b..a9b6720be13 100644 --- a/tools/generated_files/324/supportedExprs.csv +++ b/tools/generated_files/324/supportedExprs.csv @@ -637,7 +637,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/330/supportedExprs.csv b/tools/generated_files/330/supportedExprs.csv index c8df29cfca5..aa744638220 100644 --- a/tools/generated_files/330/supportedExprs.csv +++ b/tools/generated_files/330/supportedExprs.csv @@ -658,7 +658,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/330cdh/supportedExprs.csv b/tools/generated_files/330cdh/supportedExprs.csv index c8df29cfca5..aa744638220 100644 --- a/tools/generated_files/330cdh/supportedExprs.csv +++ b/tools/generated_files/330cdh/supportedExprs.csv @@ -658,7 +658,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/331/supportedExprs.csv b/tools/generated_files/331/supportedExprs.csv index e217b6ce31d..82aad5f070b 100644 --- a/tools/generated_files/331/supportedExprs.csv +++ b/tools/generated_files/331/supportedExprs.csv @@ -660,7 +660,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/332/supportedExprs.csv b/tools/generated_files/332/supportedExprs.csv index e217b6ce31d..82aad5f070b 100644 --- a/tools/generated_files/332/supportedExprs.csv +++ b/tools/generated_files/332/supportedExprs.csv @@ -660,7 +660,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/332cdh/supportedExprs.csv b/tools/generated_files/332cdh/supportedExprs.csv index e217b6ce31d..82aad5f070b 100644 --- a/tools/generated_files/332cdh/supportedExprs.csv +++ b/tools/generated_files/332cdh/supportedExprs.csv @@ -660,7 +660,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/333/supportedExprs.csv b/tools/generated_files/333/supportedExprs.csv index e217b6ce31d..82aad5f070b 100644 --- a/tools/generated_files/333/supportedExprs.csv +++ b/tools/generated_files/333/supportedExprs.csv @@ -660,7 +660,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/334/supportedExprs.csv b/tools/generated_files/334/supportedExprs.csv index e217b6ce31d..82aad5f070b 100644 --- a/tools/generated_files/334/supportedExprs.csv +++ b/tools/generated_files/334/supportedExprs.csv @@ -660,7 +660,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/340/supportedExprs.csv b/tools/generated_files/340/supportedExprs.csv index 0c2d6a74bc1..9f9a3c37ed9 100644 --- a/tools/generated_files/340/supportedExprs.csv +++ b/tools/generated_files/340/supportedExprs.csv @@ -660,7 +660,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/341/supportedExprs.csv b/tools/generated_files/341/supportedExprs.csv index 0c2d6a74bc1..9f9a3c37ed9 100644 --- a/tools/generated_files/341/supportedExprs.csv +++ b/tools/generated_files/341/supportedExprs.csv @@ -660,7 +660,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/342/supportedExprs.csv b/tools/generated_files/342/supportedExprs.csv index 0c2d6a74bc1..9f9a3c37ed9 100644 --- a/tools/generated_files/342/supportedExprs.csv +++ b/tools/generated_files/342/supportedExprs.csv @@ -660,7 +660,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/343/supportedExprs.csv b/tools/generated_files/343/supportedExprs.csv index 0c2d6a74bc1..9f9a3c37ed9 100644 --- a/tools/generated_files/343/supportedExprs.csv +++ b/tools/generated_files/343/supportedExprs.csv @@ -660,7 +660,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/344/supportedExprs.csv b/tools/generated_files/344/supportedExprs.csv index 0c2d6a74bc1..9f9a3c37ed9 100644 --- a/tools/generated_files/344/supportedExprs.csv +++ b/tools/generated_files/344/supportedExprs.csv @@ -660,7 +660,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/350/supportedExprs.csv b/tools/generated_files/350/supportedExprs.csv index 09cea891f97..9e3939f4566 100644 --- a/tools/generated_files/350/supportedExprs.csv +++ b/tools/generated_files/350/supportedExprs.csv @@ -668,7 +668,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/351/supportedExprs.csv b/tools/generated_files/351/supportedExprs.csv index 09cea891f97..9e3939f4566 100644 --- a/tools/generated_files/351/supportedExprs.csv +++ b/tools/generated_files/351/supportedExprs.csv @@ -668,7 +668,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/352/supportedExprs.csv b/tools/generated_files/352/supportedExprs.csv index 09cea891f97..9e3939f4566 100644 --- a/tools/generated_files/352/supportedExprs.csv +++ b/tools/generated_files/352/supportedExprs.csv @@ -668,7 +668,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/353/supportedExprs.csv b/tools/generated_files/353/supportedExprs.csv index 09cea891f97..9e3939f4566 100644 --- a/tools/generated_files/353/supportedExprs.csv +++ b/tools/generated_files/353/supportedExprs.csv @@ -668,7 +668,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/supportedExprs.csv b/tools/generated_files/supportedExprs.csv index 39d69c0836b..a9b6720be13 100644 --- a/tools/generated_files/supportedExprs.csv +++ b/tools/generated_files/supportedExprs.csv @@ -637,7 +637,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA From 01f9fd2e50c99de31ab6ee4cc69634ec7fd1f6c0 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 19 Dec 2024 19:58:46 -0800 Subject: [PATCH 24/47] Add missing json reader options for JsonScanRetrySuite (#11898) Signed-off-by: Jihoon Son --- .../spark/sql/rapids/GpuJsonReadCommon.scala | 14 +++++++++----- .../nvidia/spark/rapids/JsonScanRetrySuite.scala | 4 ++-- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonReadCommon.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonReadCommon.scala index 017d9722257..1e4f5579be5 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonReadCommon.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonReadCommon.scala @@ -193,18 +193,22 @@ object GpuJsonReadCommon { val allowUnquotedControlChars = options.buildJsonFactory() .isEnabled(JsonParser.Feature.ALLOW_UNQUOTED_CONTROL_CHARS) + baseCudfJsonOptionsBuilder() + .withNormalizeSingleQuotes(options.allowSingleQuotes) + .withLeadingZeros(options.allowNumericLeadingZeros) + .withNonNumericNumbers(options.allowNonNumericNumbers) + .withUnquotedControlChars(allowUnquotedControlChars) + .build() + } + + def baseCudfJsonOptionsBuilder(): ai.rapids.cudf.JSONOptions.Builder = { ai.rapids.cudf.JSONOptions.builder() .withRecoverWithNull(true) .withMixedTypesAsStrings(true) .withNormalizeWhitespace(true) .withKeepQuotes(true) - .withNormalizeSingleQuotes(options.allowSingleQuotes) .withStrictValidation(true) - .withLeadingZeros(options.allowNumericLeadingZeros) - .withNonNumericNumbers(options.allowNonNumericNumbers) - .withUnquotedControlChars(allowUnquotedControlChars) .withCudfPruneSchema(true) .withExperimental(true) - .build() } } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/JsonScanRetrySuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/JsonScanRetrySuite.scala index 47546f25513..1384a90f3cc 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/JsonScanRetrySuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/JsonScanRetrySuite.scala @@ -16,10 +16,10 @@ package com.nvidia.spark.rapids -import ai.rapids.cudf.JSONOptions import com.nvidia.spark.rapids.jni.RmmSpark import org.apache.spark.sql.catalyst.json.rapids.JsonPartitionReader +import org.apache.spark.sql.rapids.GpuJsonReadCommon import org.apache.spark.sql.types._ class JsonScanRetrySuite extends RmmSparkRetrySuiteBase { @@ -29,7 +29,7 @@ class JsonScanRetrySuite extends RmmSparkRetrySuiteBase { val cudfSchema = GpuColumnVector.from(StructType(Seq(StructField("a", IntegerType), StructField("b", IntegerType)))) - val opts = JSONOptions.builder().withLines(true).build() + val opts = GpuJsonReadCommon.baseCudfJsonOptionsBuilder().withLines(true).build() RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 1, RmmSpark.OomInjectionType.GPU.ordinal, 0) val table = JsonPartitionReader.readToTable(bufferer, cudfSchema, NoopMetric, From 32aa3e142e4d165f75430a28b62354fc9cbac380 Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Mon, 23 Dec 2024 09:33:28 +0800 Subject: [PATCH 25/47] Improve the retry support for nondeterministic expressions (#11789) Contributes to #11649 This PR is trying to address some requirements described in issue #11649, but not all of them. It introduces two new classes named "GpuExpressionRetryable" and "RetryStateTracker" to initially set up a fundamental support to detect the context requirement for nondeterministic expressions, and adds in the relevant unit tests. And it also adds the integration tests for the function "rand()" being used in HashAggregate, Generate, Projection, ArrowEvalPython and Filter. It still does not cover all the cases where a nondeterministic expression can be used, but we are closer than before. --------- Signed-off-by: Firestarman --- .../src/main/python/rand_test.py | 85 +++++++++++++++++++ .../src/main/python/spark_session.py | 1 + integration_tests/src/main/python/udf_test.py | 18 +++- .../spark/rapids/GpuAggregateExec.scala | 23 ++--- .../nvidia/spark/rapids/GpuOverrides.scala | 3 +- .../com/nvidia/spark/rapids/RapidsConf.scala | 11 +++ .../spark/rapids/RmmRapidsRetryIterator.scala | 20 +++++ .../expressions/GpuRandomExpressions.scala | 54 ++++++++++-- .../python/GpuArrowEvalPythonExec.scala | 4 +- .../rapids/NonDeterministicRetrySuite.scala | 76 ++++++++++++++++- 10 files changed, 269 insertions(+), 26 deletions(-) create mode 100644 integration_tests/src/main/python/rand_test.py diff --git a/integration_tests/src/main/python/rand_test.py b/integration_tests/src/main/python/rand_test.py new file mode 100644 index 00000000000..a91bc518b04 --- /dev/null +++ b/integration_tests/src/main/python/rand_test.py @@ -0,0 +1,85 @@ +# Copyright (c) 2024, 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 +from data_gen import * +from marks import * +from spark_session import is_before_spark_351 + +import pyspark.sql.functions as f + + +@ignore_order(local=True) +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 +def test_group_agg_with_rand(): + # GPU and CPU produce the same grouping rows but in different orders after Shuffle, + # while the rand() always generates the same sequence. Then CPU and GPU will produce + # different final rows after aggregation. See as below: + # GPU output: + # +---+-------------------+ + # | a| random| + # +---+-------------------+ + # | 3| 0.619189370225301| + # | 5| 0.5096018842446481| + # | 2| 0.8325259388871524| + # | 4|0.26322809041172357| + # | 1| 0.6702867696264135| + # +---+-------------------+ + # CPU output: + # +---+-------------------+ + # | a| random| + # +---+-------------------+ + # | 1| 0.619189370225301| + # | 2| 0.5096018842446481| + # | 3| 0.8325259388871524| + # | 4|0.26322809041172357| + # | 5| 0.6702867696264135| + # +---+-------------------+ + # To make the output comparable, here builds a generator to generate only one group. + const_int_gen = IntegerGen(nullable=False, min_val=1, max_val=1, special_cases=[]) + + def test(spark): + return unary_op_df(spark, const_int_gen, num_slices=1).groupby('a').agg(f.rand(42)) + assert_gpu_and_cpu_are_equal_collect(test) + + +@ignore_order(local=True) +def test_project_with_rand(): + # To make the output comparable, here build a generator to generate only one value. + # Not sure if Project could have the same order issue as groupBy, but still just in case. + const_int_gen = IntegerGen(nullable=False, min_val=1, max_val=1, special_cases=[]) + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, const_int_gen, num_slices=1).select('a', f.rand(42)) + ) + + +@ignore_order(local=True) +def test_filter_with_rand(): + const_int_gen = IntegerGen(nullable=False, min_val=1, max_val=1, special_cases=[]) + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, const_int_gen, num_slices=1).filter(f.rand(42) > 0.5) + ) + +# See https://github.com/apache/spark/commit/9c0b803ba124a6e70762aec1e5559b0d66529f4d +@ignore_order(local=True) +@pytest.mark.skipif(is_before_spark_351(), + reason='Generate supports nondeterministic inputs from Spark 3.5.1') +def test_generate_with_rand(): + const_int_gen = IntegerGen(nullable=False, min_val=1, max_val=1, special_cases=[]) + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, const_int_gen, num_slices=1).select( + f.explode(f.array(f.rand(42)))) + ) diff --git a/integration_tests/src/main/python/spark_session.py b/integration_tests/src/main/python/spark_session.py index 831680e4feb..d292dba46d3 100644 --- a/integration_tests/src/main/python/spark_session.py +++ b/integration_tests/src/main/python/spark_session.py @@ -41,6 +41,7 @@ def _from_scala_map(scala_map): # Many of these are redundant with default settings for the configs but are set here explicitly # to ensure any cluster settings do not interfere with tests that assume the defaults. _default_conf = { + 'spark.rapids.sql.test.retryContextCheck.enabled': 'true', 'spark.rapids.sql.castDecimalToFloat.enabled': 'false', 'spark.rapids.sql.castFloatToDecimal.enabled': 'false', 'spark.rapids.sql.castFloatToIntegralTypes.enabled': 'false', diff --git a/integration_tests/src/main/python/udf_test.py b/integration_tests/src/main/python/udf_test.py index 7ca3e84e9ba..8060198c789 100644 --- a/integration_tests/src/main/python/udf_test.py +++ b/integration_tests/src/main/python/udf_test.py @@ -16,7 +16,7 @@ from pyspark import BarrierTaskContext, TaskContext from conftest import is_at_least_precommit_run, is_databricks_runtime -from spark_session import is_before_spark_330, is_before_spark_350, is_spark_341 +from spark_session import is_before_spark_330, is_before_spark_331, is_before_spark_350, is_spark_341 from pyspark.sql.pandas.utils import require_minimum_pyarrow_version, require_minimum_pandas_version @@ -474,3 +474,19 @@ def add_one(a): lambda spark: unary_op_df(spark, int_gen, num_slices=4, length=52345) .select(my_udf(f.lit(0))), conf=arrow_udf_conf) + + +# Python UDFs support nondeterministic expressions from Spark 3.3.1. +# See https://github.com/apache/spark/commit/1a01a492c051bb861c480f224a3c310e133e4d01 +@ignore_order(local=True) +@pytest.mark.skipif(is_before_spark_331(), + reason='Nondeterministic expressions are supported from Spark 3.3.1') +def test_pandas_math_udf_with_rand(): + def add(rand_value): + return rand_value + my_udf = f.pandas_udf(add, returnType=IntegerType()) + assert_gpu_and_cpu_are_equal_collect( + # Ensure there is only one partition to make the output comparable. + lambda spark: unary_op_df(spark, int_gen, length=10, num_slices=1).select( + my_udf(f.rand(42))), + conf=arrow_udf_conf) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala index e3ca330b409..0b8896b4a63 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala @@ -816,13 +816,13 @@ object GpuAggFinalPassIterator { boundResultReferences) } - private[this] def reorderFinalBatch(finalBatch: ColumnarBatch, + private[this] def reorderFinalBatch(finalBatch: SpillableColumnarBatch, boundExpressions: BoundExpressionsModeAggregates, metrics: GpuHashAggregateMetrics): ColumnarBatch = { // Perform the last project to get the correct shape that Spark expects. Note this may // add things like literals that were not part of the aggregate into the batch. - closeOnExcept(GpuProjectExec.projectAndClose(finalBatch, - boundExpressions.boundResultReferences, NoopMetric)) { ret => + closeOnExcept(GpuProjectExec.projectAndCloseWithRetrySingleBatch(finalBatch, + boundExpressions.boundResultReferences)) { ret => metrics.numOutputRows += ret.numRows() metrics.numOutputBatches += 1 ret @@ -838,9 +838,12 @@ object GpuAggFinalPassIterator { withResource(new NvtxWithMetrics("finalize agg", NvtxColor.DARK_GREEN, aggTime, opTime)) { _ => val finalBatch = boundExpressions.boundFinalProjections.map { exprs => - GpuProjectExec.projectAndClose(batch, exprs, NoopMetric) + GpuProjectExec.projectAndCloseWithRetrySingleBatch( + SpillableColumnarBatch(batch, SpillPriorities.ACTIVE_BATCHING_PRIORITY), exprs) }.getOrElse(batch) - reorderFinalBatch(finalBatch, boundExpressions, metrics) + val finalSCB = + SpillableColumnarBatch(finalBatch, SpillPriorities.ACTIVE_BATCHING_PRIORITY) + reorderFinalBatch(finalSCB, boundExpressions, metrics) } } } @@ -854,12 +857,10 @@ object GpuAggFinalPassIterator { withResource(new NvtxWithMetrics("finalize agg", NvtxColor.DARK_GREEN, aggTime, opTime)) { _ => val finalBatch = boundExpressions.boundFinalProjections.map { exprs => - GpuProjectExec.projectAndCloseWithRetrySingleBatch(sb, exprs) - }.getOrElse { - withRetryNoSplit(sb) { _ => - sb.getColumnarBatch() - } - } + SpillableColumnarBatch( + GpuProjectExec.projectAndCloseWithRetrySingleBatch(sb, exprs), + SpillPriorities.ACTIVE_BATCHING_PRIORITY) + }.getOrElse(sb) reorderFinalBatch(finalBatch, boundExpressions, metrics) } } 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 c1ede7554a6..c23728ba43a 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 @@ -2432,7 +2432,8 @@ object GpuOverrides extends Logging { (TypeSig.INT + TypeSig.LONG).withAllLit(), (TypeSig.INT + TypeSig.LONG).withAllLit()))), (a, conf, p, r) => new UnaryExprMeta[Rand](a, conf, p, r) { - override def convertToGpu(child: Expression): GpuExpression = GpuRand(child) + override def convertToGpu(child: Expression): GpuExpression = + GpuRand(child, this.conf.isRetryContextCheckEnabled) }), expr[SparkPartitionID] ( "Returns the current partition id", 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 c4199e3ea75..609155e4b86 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 @@ -1643,6 +1643,15 @@ val GPU_COREDUMP_PIPE_PATTERN = conf("spark.rapids.gpu.coreDump.pipePattern") .booleanConf .createWithDefault(false) + val TEST_RETRY_CONTEXT_CHECK_ENABLED = conf("spark.rapids.sql.test.retryContextCheck.enabled") + .doc("Only to be used in tests. When set to true, enable the context check for " + + "GPU nondeterministic expressions but declaring to be retryable. A GPU retryable " + + "nondeterministic expression should run inside a checkpoint-restore context. And it " + + "will blow up when the context does not satisfy.") + .internal() + .booleanConf + .createWithDefault(false) + val TEST_CONF = conf("spark.rapids.sql.test.enabled") .doc("Intended to be used by unit tests, if enabled all operations must run on the " + "GPU or an error happens.") @@ -2733,6 +2742,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isTestEnabled: Boolean = get(TEST_CONF) + lazy val isRetryContextCheckEnabled: Boolean = get(TEST_RETRY_CONTEXT_CHECK_ENABLED) + lazy val isFoldableNonLitAllowed: Boolean = get(FOLDABLE_NON_LIT_ALLOWED) lazy val asyncWriteMaxInFlightHostMemoryBytes: Long = 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 04bc56af0c4..68a83cafd86 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 @@ -606,6 +606,7 @@ object RmmRapidsRetryIterator extends Logging { var doSplit = false var isFromGpuOom = true while (result.isEmpty && attemptIter.hasNext) { + RetryStateTracker.setCurThreadRetrying(!firstAttempt) if (!firstAttempt) { // call thread block API try { @@ -685,6 +686,7 @@ object RmmRapidsRetryIterator extends Logging { // else another exception wrapped a retry. So we are going to try again } } + RetryStateTracker.clearCurThreadRetrying() if (result.isEmpty) { // then lastException must be set, throw it. throw lastException @@ -791,3 +793,21 @@ object RmmRapidsRetryIterator extends Logging { case class AutoCloseableTargetSize(targetSize: Long, minSize: Long) extends AutoCloseable { override def close(): Unit = () } + +/** + * This leverages a ThreadLocal of boolean to track if a task thread is currently + * executing a retry. And the boolean state will be used by all the + * `GpuExpressionRetryable`s to determine if the context is safe to retry the evaluation. + */ +object RetryStateTracker { + private val localIsRetrying = new ThreadLocal[java.lang.Boolean]() + + def isCurThreadRetrying: Boolean = { + val ret = localIsRetrying.get() + ret != null && ret + } + + def setCurThreadRetrying(retrying: Boolean): Unit = localIsRetrying.set(retrying) + + def clearCurThreadRetrying(): Unit = localIsRetrying.remove() +} diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala index efc59749d2d..415a171b034 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.rapids.catalyst.expressions import ai.rapids.cudf.{DType, HostColumnVector, NvtxColor, NvtxRange} import com.nvidia.spark.Retryable -import com.nvidia.spark.rapids.{GpuColumnVector, GpuExpression, GpuLiteral} +import com.nvidia.spark.rapids.{GpuColumnVector, GpuExpression, GpuLiteral, RetryStateTracker} import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.shims.ShimUnaryExpression @@ -30,13 +30,51 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils import org.apache.spark.util.random.rapids.RapidsXORShiftRandom +/** + * An expression expected to be evaluated inside a retry with checkpoint-restore context. + * It will throw an exception if it is retried without being checkpointed. + * All the nondeterministic GPU expressions that support Retryable should extend from + * this trait. + */ +trait GpuExpressionRetryable extends GpuExpression with Retryable { + private var checked = false + + def doColumnarEval(batch: ColumnarBatch): GpuColumnVector + def doCheckpoint(): Unit + def doRestore(): Unit + + def doContextCheck(): Boolean // For tests + + override final def columnarEval(batch: ColumnarBatch): GpuColumnVector = { + if (doContextCheck && !checked) { // This is for tests + throw new IllegalStateException( + "The Retryable was called outside of a checkpoint-restore context") + } + if (!checked && RetryStateTracker.isCurThreadRetrying) { + // It is retrying the evaluation without checkpointing, which is not allowed. + throw new IllegalStateException( + "The Retryable should be retried only inside a checkpoint-restore context") + } + doColumnarEval(batch) + } + + override final def checkpoint(): Unit = { + checked = true + doCheckpoint() + } + + override final def restore(): Unit = doRestore() +} + /** Generate a random column with i.i.d. uniformly distributed values in [0, 1). */ -case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpression - with ExpectsInputTypes with ExpressionWithRandomSeed with Retryable { +case class GpuRand(child: Expression, doContextCheck: Boolean) extends ShimUnaryExpression + with ExpectsInputTypes with ExpressionWithRandomSeed with GpuExpressionRetryable { - def this() = this(GpuLiteral(Utils.random.nextLong(), LongType)) + def this(doContextCheck: Boolean) = this(GpuLiteral(Utils.random.nextLong(), LongType), + doContextCheck) - override def withNewSeed(seed: Long): GpuRand = GpuRand(GpuLiteral(seed, LongType)) + override def withNewSeed(seed: Long): GpuRand = GpuRand(GpuLiteral(seed, LongType), + doContextCheck) def seedExpression: Expression = child @@ -76,7 +114,7 @@ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpres } } - override def columnarEval(batch: ColumnarBatch): GpuColumnVector = { + override def doColumnarEval(batch: ColumnarBatch): GpuColumnVector = { if (curXORShiftRandomSeed.isEmpty) { // checkpoint not called, need to init the random generator here initRandom() @@ -93,14 +131,14 @@ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpres } } - override def checkpoint(): Unit = { + override def doCheckpoint(): Unit = { // In a task, checkpoint is called before columnarEval, so need to try to // init the random generator here. initRandom() curXORShiftRandomSeed = Some(rng.currentSeed) } - override def restore(): Unit = { + override def doRestore(): Unit = { assert(wasInitialized && curXORShiftRandomSeed.isDefined) rng.setHashedSeed(curXORShiftRandomSeed.get) } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuArrowEvalPythonExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuArrowEvalPythonExec.scala index c99d0403ed0..50dcdfb5ab5 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuArrowEvalPythonExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuArrowEvalPythonExec.scala @@ -434,7 +434,9 @@ case class GpuArrowEvalPythonExec( new RebatchingRoundoffIterator(iter, inputSchema, targetBatchSize, numInputRows, numInputBatches)) val pyInputIterator = batchProducer.asIterator.map { batch => - withResource(batch)(GpuProjectExec.project(_, boundReferences)) + GpuProjectExec.projectAndCloseWithRetrySingleBatch( + SpillableColumnarBatch(batch, SpillPriorities.ACTIVE_BATCHING_PRIORITY), + boundReferences) } if (isPythonOnGpuEnabled) { diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/NonDeterministicRetrySuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/NonDeterministicRetrySuite.scala index d018726ef35..1b91d9736ee 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/NonDeterministicRetrySuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/NonDeterministicRetrySuite.scala @@ -38,8 +38,11 @@ class NonDeterministicRetrySuite extends RmmSparkRetrySuiteBase { Array(GpuColumnVector.from(ColumnVector.fromInts(ints: _*), IntegerType)), ints.length) } + private def newGpuRand(ctxCheck: Boolean=false) = + GpuRand(GpuLiteral(RAND_SEED, IntegerType), ctxCheck) + test("GPU rand outputs the same sequence with checkpoint and restore") { - val gpuRand = GpuRand(GpuLiteral(RAND_SEED, IntegerType)) + val gpuRand = newGpuRand() withResource(buildBatch()) { inputCB => // checkpoint the state gpuRand.checkpoint() @@ -65,8 +68,7 @@ class NonDeterministicRetrySuite extends RmmSparkRetrySuiteBase { } test("GPU project retry with GPU rand") { - def projectRand(): Seq[GpuExpression] = Seq( - GpuAlias(GpuRand(GpuLiteral(RAND_SEED)), "rand")()) + def projectRand(): Seq[GpuExpression] = Seq(GpuAlias(newGpuRand(), "rand")()) Seq(true, false).foreach { useTieredProject => val conf = new SQLConf() @@ -109,7 +111,7 @@ class NonDeterministicRetrySuite extends RmmSparkRetrySuiteBase { test("GPU filter retry with GPU rand") { def filterRand(): Seq[GpuExpression] = Seq( GpuGreaterThan( - GpuRand(GpuLiteral.create(RAND_SEED, IntegerType)), + newGpuRand(), GpuLiteral.create(0.1d, DoubleType))) Seq(true, false).foreach { useTieredProject => @@ -155,4 +157,70 @@ class NonDeterministicRetrySuite extends RmmSparkRetrySuiteBase { } } + test("GPU project with GPU rand for context check enabled") { + // We dont check the output correctness, so it is ok to reuse the bound expressions. + val boundCheckExprs = GpuBindReferences.bindGpuReferences( + Seq(GpuAlias(newGpuRand(true), "rand")()), + batchAttrs) + + // 1) Context check + no-retry + no checkpoint-restore + assertThrows[IllegalStateException] { + GpuProjectExec.projectAndClose(buildBatch(), boundCheckExprs, NoopMetric) + } + + // 2) Context check + retry + no checkpoint-restore + assertThrows[IllegalStateException] { + RmmRapidsRetryIterator.withRetryNoSplit(buildBatch()) { cb => + GpuProjectExec.project(cb, boundCheckExprs) + } + } + + // 3) Context check + retry + checkpoint-restore + // This is the expected usage for the GPU Rand. + Seq(true, false).foreach { forceOOM => + val scb = SpillableColumnarBatch(buildBatch(), SpillPriorities.ACTIVE_ON_DECK_PRIORITY) + if (forceOOM) { // make a retrying really happen during the projection + RmmSpark.forceRetryOOM( + RmmSpark.getCurrentThreadId, 1, RmmSpark.OomInjectionType.GPU.ordinal, 0) + } + GpuProjectExec.projectAndCloseWithRetrySingleBatch(scb, boundCheckExprs).close() + } + } + + test("GPU project with GPU rand for context check disabled") { + // We dont check the output correctness, so it is ok to reuse the bound expressions. + val boundExprs = GpuBindReferences.bindGpuReferences( + Seq(GpuAlias(newGpuRand(false), "rand")()), + batchAttrs) + + // 1) No context check + no retry + no checkpoint-restore + // It works but not the expected usage for the GPU Rand + GpuProjectExec.projectAndClose(buildBatch(), boundExprs, NoopMetric).close() + + // 2) No context check + retry (no real retrying) + no checkpoint-restore + // It works but not the expected usage for the GPU Rand + RmmRapidsRetryIterator.withRetryNoSplit(buildBatch()) { cb => + GpuProjectExec.project(cb, boundExprs) + }.close() + + // 3) No context check + retry (A retrying happens) + no checkpoint-restore + assertThrows[IllegalStateException] { + val cb = buildBatch() + // Make a retrying really happen + RmmSpark.forceRetryOOM( + RmmSpark.getCurrentThreadId, 1, RmmSpark.OomInjectionType.GPU.ordinal, 0) + RmmRapidsRetryIterator.withRetryNoSplit(cb)(GpuProjectExec.project(_, boundExprs)) + } + + // 4) No context check + retry + checkpoint-restore + // This is the expected usage for the GPU Rand + Seq(true, false).foreach { forceOOM => + val scb = SpillableColumnarBatch(buildBatch(), SpillPriorities.ACTIVE_ON_DECK_PRIORITY) + if (forceOOM) { // make a retrying really happen during the projection + RmmSpark.forceRetryOOM( + RmmSpark.getCurrentThreadId, 1, RmmSpark.OomInjectionType.GPU.ordinal, 0) + } + GpuProjectExec.projectAndCloseWithRetrySingleBatch(scb, boundExprs).close() + } + } } From f0c35ffa5aefcda5f5947c914c1527d8b4b56a5a Mon Sep 17 00:00:00 2001 From: "Hongbin Ma (Mahone)" Date: Mon, 23 Dec 2024 14:09:49 +0800 Subject: [PATCH 26/47] address some comments for 11792 (#11816) * address some comments for 11792 Signed-off-by: Hongbin Ma (Mahone) * refine warning message Signed-off-by: Hongbin Ma (Mahone) * address comments Signed-off-by: Hongbin Ma (Mahone) --------- Signed-off-by: Hongbin Ma (Mahone) --- .../spark/rapids/GpuAggregateExec.scala | 59 ++++++++++++------- 1 file changed, 39 insertions(+), 20 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala index 0b8896b4a63..8fc5326705e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala @@ -214,10 +214,12 @@ object AggregateUtils extends Logging { helper: AggHelper, hashKeys: Seq[GpuExpression], hashBucketNum: Int, - hashSeed: Int, + baseHashSeed: Int, + recursiveDepth: Int, + maxRecursiveDepth: Int, batchesByBucket: ArrayBuffer[AutoClosableArrayBuffer[SpillableColumnarBatch]] ): Boolean = { - + val hashSeed = baseHashSeed + recursiveDepth * 7 var repartitionHappened = false def repartitionAndClose(batch: SpillableColumnarBatch): Unit = { @@ -277,12 +279,19 @@ object AggregateUtils extends Logging { val newBuckets = batchesByBucket.flatMap(bucket => { if (needRepartitionAgain(bucket)) { - if (hashSeed + 7 > 200) { - log.warn("Too many times of repartition, may hit a bug? Size for each batch in " + - "current bucket: " + bucket.map(_.sizeInBytes).mkString(", ") + " rows: " + - bucket.map(_.numRows()).mkString(", ") + " targetMergeBatchSize: " - + targetMergeBatchSize) - ArrayBuffer.apply(bucket) + if (recursiveDepth >= maxRecursiveDepth) { + // Normally this should not happen, because we are repartitioning data that has + // already gone through first round of aggregation, so there shouldn't be too many + // duplicated rows (the duplication only happens in different batches) to prevent + // repartitioning out (considering we're changing seed each time we repartition). + // However, for some test cases with really small batch size, this can happen. So + // we're just logging some warnings here. + log.warn(s"The bucket is still too large after $recursiveDepth repartitions. " + + s"See https://github.com/NVIDIA/spark-rapids/issues/11834. " + + s"Sizes for each batch in current bucket: ${bucket.map(_.sizeInBytes).mkString(", ")}" + + s" rows: ${bucket.map(_.numRows()).mkString(", ")}" + + s" targetMergeBatchSize: $targetMergeBatchSize") + ArrayBuffer(bucket) } else { val nextLayerBuckets = ArrayBuffer.fill(hashBucketNum)(new AutoClosableArrayBuffer[SpillableColumnarBatch]()) @@ -290,12 +299,12 @@ object AggregateUtils extends Logging { repartitionHappened = iterateAndRepartition( new CloseableBufferedIterator(bucket.iterator), metrics, targetMergeBatchSize, - helper, hashKeys, hashBucketNum, hashSeed + 7, - nextLayerBuckets) || repartitionHappened + helper, hashKeys, hashBucketNum, baseHashSeed, recursiveDepth + 1, + maxRecursiveDepth, nextLayerBuckets) || repartitionHappened nextLayerBuckets } } else { - ArrayBuffer.apply(bucket) + ArrayBuffer(bucket) } }) batchesByBucket.clear() @@ -921,7 +930,8 @@ class GpuMergeAggregateIterator( private[this] val targetMergeBatchSize = computeTargetMergeBatchSize(configuredTargetBatchSize) private[this] val defaultHashBucketNum = 16 - private[this] val defaultHashSeed = 107 + private[this] val maxLevelsOfRepartition = 10 // this is the max level for recursive repartition + private[this] val baseHashSeed = 107 // this is the seed used for first level for repartition private[this] var batchesByBucket = ArrayBuffer.fill(defaultHashBucketNum)(new AutoClosableArrayBuffer[SpillableColumnarBatch]()) @@ -953,7 +963,7 @@ class GpuMergeAggregateIterator( // Handle reduction-only aggregation if (isReductionOnly) { - val batches = ArrayBuffer.apply[SpillableColumnarBatch]() + val batches = ArrayBuffer[SpillableColumnarBatch]() while (firstPassIter.hasNext) { batches += firstPassIter.next() } @@ -985,7 +995,7 @@ class GpuMergeAggregateIterator( s"$firstPassReductionRatioEstimate") // if so, skip any aggregation, return the origin batch directly - realIter = Some(ConcatIterator.apply(firstPassIter, configuredTargetBatchSize)) + realIter = Some(ConcatIterator(firstPassIter, configuredTargetBatchSize)) metrics.numTasksSkippedAgg += 1 return realIter.get.next() } else { @@ -1010,12 +1020,12 @@ class GpuMergeAggregateIterator( configuredTargetBatchSize, concatAndMergeHelper) , metrics, targetMergeBatchSize, concatAndMergeHelper, - hashKeys, defaultHashBucketNum, defaultHashSeed, batchesByBucket) + hashKeys, defaultHashBucketNum, baseHashSeed, 0, maxLevelsOfRepartition, batchesByBucket) if (repartitionHappened) { metrics.numTasksRepartitioned += 1 } - realIter = Some(ConcatIterator.apply( + realIter = Some(ConcatIterator( new CloseableBufferedIterator(buildBucketIterator()), configuredTargetBatchSize)) realIter.get.next() } @@ -1048,8 +1058,12 @@ class GpuMergeAggregateIterator( val spillCbs = ArrayBuffer[SpillableColumnarBatch]() var totalBytes = 0L closeOnExcept(spillCbs) { _ => - while (input.hasNext && (spillCbs.isEmpty || - (totalBytes + input.head.sizeInBytes) < targetSize)) { + while (input.hasNext && ( + // for some test cases targetSize is too small to fit any SpillableColumnarBatch, + // in this case we put the first SpillableColumnarBatch into spillCbs anyway + // refer to https://github.com/NVIDIA/spark-rapids/issues/11790 for examples + spillCbs.isEmpty || + (totalBytes + input.head.sizeInBytes) <= targetSize)) { val tmp = input.next totalBytes += tmp.sizeInBytes spillCbs += tmp @@ -1081,8 +1095,13 @@ class GpuMergeAggregateIterator( closeOnExcept(new ArrayBuffer[AutoClosableArrayBuffer[SpillableColumnarBatch]]) { toAggregateBuckets => var currentSize = 0L - while (batchesByBucket.nonEmpty && (toAggregateBuckets.isEmpty || - batchesByBucket.last.size() + currentSize < targetMergeBatchSize)) { + while (batchesByBucket.nonEmpty && + ( + // for some test cases targetMergeBatchSize is too small to fit any bucket, + // in this case we put the first bucket into toAggregateBuckets anyway + // refer to https://github.com/NVIDIA/spark-rapids/issues/11790 for examples + toAggregateBuckets.isEmpty || + batchesByBucket.last.size() + currentSize <= targetMergeBatchSize)) { val bucket = batchesByBucket.remove(batchesByBucket.size - 1) currentSize += bucket.map(_.sizeInBytes).sum toAggregateBuckets += bucket From 0f702cd1bc1f1a1c52c3475db9f0a94563bf8012 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 24 Dec 2024 10:49:42 -0800 Subject: [PATCH 27/47] Async write support for ORC (#11865) * Async write support for ORC writer Signed-off-by: Jihoon Son * doc change * remove unnecessary coalesce in the tests * revert unrelated change * sort results --------- Signed-off-by: Jihoon Son --- .../src/main/python/orc_write_test.py | 17 +++++++++++++ .../src/main/python/parquet_write_test.py | 4 +-- .../com/nvidia/spark/rapids/RapidsConf.scala | 2 +- .../spark/sql/rapids/GpuOrcFileFormat.scala | 25 +++++++++++++------ 4 files changed, 38 insertions(+), 10 deletions(-) diff --git a/integration_tests/src/main/python/orc_write_test.py b/integration_tests/src/main/python/orc_write_test.py index 103cae474a3..b3c82675869 100644 --- a/integration_tests/src/main/python/orc_write_test.py +++ b/integration_tests/src/main/python/orc_write_test.py @@ -360,6 +360,23 @@ def create_empty_df(spark, path): conf={'spark.rapids.sql.format.orc.write.enabled': True}) +hold_gpu_configs = [True, False] +@pytest.mark.parametrize('hold_gpu', hold_gpu_configs, ids=idfn) +def test_async_writer(spark_tmp_path, hold_gpu): + data_path = spark_tmp_path + '/ORC_DATA' + num_rows = 2048 + num_cols = 10 + orc_gen = [int_gen for _ in range(num_cols)] + gen_list = [('_c' + str(i), gen) for i, gen in enumerate(orc_gen)] + assert_gpu_and_cpu_writes_are_equal_collect( + lambda spark, path: gen_df(spark, gen_list, length=num_rows).write.orc(path), + lambda spark, path: spark.read.orc(path).orderBy([('_c' + str(i)) for i in range(num_cols)]), + data_path, + conf={"spark.rapids.sql.asyncWrite.queryOutput.enabled": "true", + "spark.rapids.sql.batchSizeBytes": 4 * num_cols * 100, # 100 rows per batch + "spark.rapids.sql.queryOutput.holdGpuInTask": hold_gpu}) + + @ignore_order @pytest.mark.skipif(is_before_spark_320(), reason="is only supported in Spark 320+") def test_concurrent_writer(spark_tmp_path): diff --git a/integration_tests/src/main/python/parquet_write_test.py b/integration_tests/src/main/python/parquet_write_test.py index e5719d267b4..9b43fabd26d 100644 --- a/integration_tests/src/main/python/parquet_write_test.py +++ b/integration_tests/src/main/python/parquet_write_test.py @@ -705,8 +705,8 @@ def test_async_writer(spark_tmp_path, hold_gpu): parquet_gen = [int_gen for _ in range(num_cols)] gen_list = [('_c' + str(i), gen) for i, gen in enumerate(parquet_gen)] assert_gpu_and_cpu_writes_are_equal_collect( - lambda spark, path: gen_df(spark, gen_list, length=num_rows).coalesce(1).write.parquet(path), - lambda spark, path: spark.read.parquet(path), + lambda spark, path: gen_df(spark, gen_list, length=num_rows).write.parquet(path), + lambda spark, path: spark.read.parquet(path).orderBy([('_c' + str(i)) for i in range(num_cols)]), data_path, copy_and_update( writer_confs, 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 609155e4b86..eeeb930cf9a 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 @@ -2462,7 +2462,7 @@ val SHUFFLE_COMPRESSION_LZ4_CHUNK_SIZE = conf("spark.rapids.shuffle.compression. .doc("Option to turn on the async query output write. During the final output write, the " + "task first copies the output to the host memory, and then writes it into the storage. " + "When this option is enabled, the task will asynchronously write the output in the host " + - "memory to the storage. Only the Parquet format is supported currently.") + "memory to the storage. Only the Parquet and ORC formats are supported currently.") .internal() .booleanConf .createWithDefault(false) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala index 5ac2aa1fe98..422f6c2337e 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala @@ -169,7 +169,8 @@ class GpuOrcFileFormat extends ColumnarFileFormat with Logging { options: Map[String, String], dataSchema: StructType): ColumnarOutputWriterFactory = { - val orcOptions = new OrcOptions(options, sparkSession.sessionState.conf) + val sqlConf = sparkSession.sessionState.conf + val orcOptions = new OrcOptions(options, sqlConf) val conf = job.getConfiguration @@ -180,12 +181,18 @@ class GpuOrcFileFormat extends ColumnarFileFormat with Logging { conf.asInstanceOf[JobConf] .setOutputFormat(classOf[org.apache.orc.mapred.OrcOutputFormat[OrcStruct]]) + val asyncOutputWriteEnabled = RapidsConf.ENABLE_ASYNC_OUTPUT_WRITE.get(sqlConf) + // holdGpuBetweenBatches is on by default if asyncOutputWriteEnabled is on + val holdGpuBetweenBatches = RapidsConf.ASYNC_QUERY_OUTPUT_WRITE_HOLD_GPU_IN_TASK.get(sqlConf) + .getOrElse(asyncOutputWriteEnabled) + new ColumnarOutputWriterFactory { override def newInstance(path: String, dataSchema: StructType, context: TaskAttemptContext, debugOutputPath: Option[String]): ColumnarOutputWriter = { - new GpuOrcWriter(path, dataSchema, context, debugOutputPath) + new GpuOrcWriter(path, dataSchema, context, debugOutputPath, holdGpuBetweenBatches, + asyncOutputWriteEnabled) } override def getFileExtension(context: TaskAttemptContext): String = { @@ -204,11 +211,15 @@ class GpuOrcFileFormat extends ColumnarFileFormat with Logging { } } -class GpuOrcWriter(override val path: String, - dataSchema: StructType, - context: TaskAttemptContext, - debugOutputPath: Option[String]) - extends ColumnarOutputWriter(context, dataSchema, "ORC", true, debugOutputPath) { +class GpuOrcWriter( + override val path: String, + dataSchema: StructType, + context: TaskAttemptContext, + debugOutputPath: Option[String], + holdGpuBetweenBatches: Boolean, + useAsyncWrite: Boolean) + extends ColumnarOutputWriter(context, dataSchema, "ORC", true, debugOutputPath, + holdGpuBetweenBatches, useAsyncWrite) { override val tableWriter: TableWriter = { val builder = SchemaUtils From 2b7a0e26486857b51ff383e9ef80988e1d9c05bb Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Tue, 24 Dec 2024 11:16:57 -0800 Subject: [PATCH 28/47] Eagerly initialize RapidsShuffleManager for SPARK-45762 [databricks] (#11904) Fixes #11107 - Initialize RapidsShuffleManager on construction for Spark 4.0 and Databricks 14.3 - Disable lazy initialization and conf validation in the case above Signed-off-by: Gera Shegalov --- .../spark/sql/rapids/GpuShuffleEnv.scala | 35 +++++++------ .../spark/rapids/RapidsShuffleManager.scala | 2 - .../shims/ShuffleManagerShimUtils.scala | 49 +++++++++++++++++++ .../spark/rapids/RapidsShuffleManager.scala | 32 ++++++++++++ .../shims/ShuffleManagerShimUtils.scala | 25 ++++++++++ 5 files changed, 126 insertions(+), 17 deletions(-) create mode 100644 sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShuffleManagerShimUtils.scala create mode 100644 sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala create mode 100644 sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/ShuffleManagerShimUtils.scala diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala index 7f8733b9e00..3e0e2afa9c7 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.rapids import java.util.Locale import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.shims.ShuffleManagerShimUtils import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging @@ -105,21 +106,25 @@ object GpuShuffleEnv extends Logging { // def initShuffleManager(): Unit = { val shuffleManager = SparkEnv.get.shuffleManager - shuffleManager match { - case rapidsShuffleManager: RapidsShuffleManagerLike => - rapidsShuffleManager.initialize - case _ => - val rsmLoaderViaShuffleManager = shuffleManager.getClass.getSuperclass.getInterfaces - .collectFirst { - case c if c.getName == classOf[RapidsShuffleManagerLike].getName => c.getClassLoader - } - val rsmLoaderDirect = classOf[RapidsShuffleManagerLike].getClassLoader - - throw new IllegalStateException(s"Cannot initialize the RAPIDS Shuffle Manager " + - s"${shuffleManager}! Expected: an instance of RapidsShuffleManagerLike loaded by " + - s"${rsmLoaderDirect}. Actual: ${shuffleManager} tagged with RapidsShuffleManagerLike " + - s"loaded by: ${rsmLoaderViaShuffleManager}" - ) + if (ShuffleManagerShimUtils.eagerlyInitialized) { + // skip deferred init + } else { + shuffleManager match { + case rapidsShuffleManager: RapidsShuffleManagerLike => + rapidsShuffleManager.initialize + case _ => + val rsmLoaderViaShuffleManager = shuffleManager.getClass.getSuperclass.getInterfaces + .collectFirst { + case c if c.getName == classOf[RapidsShuffleManagerLike].getName => c.getClassLoader + } + val rsmLoaderDirect = classOf[RapidsShuffleManagerLike].getClassLoader + + throw new IllegalStateException(s"Cannot initialize the RAPIDS Shuffle Manager " + + s"${shuffleManager}! Expected: an instance of RapidsShuffleManagerLike loaded by " + + s"${rsmLoaderDirect}. Actual: ${shuffleManager} tagged with RapidsShuffleManagerLike " + + s"loaded by: ${rsmLoaderViaShuffleManager}" + ) + } } } diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala index 155eb35a516..e0ca739463f 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala @@ -37,11 +37,9 @@ {"spark": "343"} {"spark": "344"} {"spark": "350"} -{"spark": "350db143"} {"spark": "351"} {"spark": "352"} {"spark": "353"} -{"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.$_spark.version.classifier_ diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShuffleManagerShimUtils.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShuffleManagerShimUtils.scala new file mode 100644 index 00000000000..4c04377ff4c --- /dev/null +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShuffleManagerShimUtils.scala @@ -0,0 +1,49 @@ +/* + * Copyright (c) 2024, 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. + */ + + +/*** spark-rapids-shim-json-lines +{"spark": "320"} +{"spark": "321"} +{"spark": "321cdh"} +{"spark": "322"} +{"spark": "323"} +{"spark": "324"} +{"spark": "330"} +{"spark": "330cdh"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332cdh"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "344"} +{"spark": "350"} +{"spark": "351"} +{"spark": "352"} +{"spark": "353"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +object ShuffleManagerShimUtils { + def eagerlyInitialized = false +} diff --git a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala new file mode 100644 index 00000000000..ca4a9a1dc4b --- /dev/null +++ b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala @@ -0,0 +1,32 @@ +/* + * Copyright (c) 2024, 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "350db143"} +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.$_spark.version.classifier_ + +import org.apache.spark.SparkConf +import org.apache.spark.sql.rapids.ProxyRapidsShuffleInternalManagerBase + +/** A shuffle manager optimized for the RAPIDS Plugin for Apache Spark. */ +sealed class RapidsShuffleManager( + conf: SparkConf, + isDriver: Boolean +) extends ProxyRapidsShuffleInternalManagerBase(conf, isDriver) { + initialize +} diff --git a/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/ShuffleManagerShimUtils.scala b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/ShuffleManagerShimUtils.scala new file mode 100644 index 00000000000..1ceb7902042 --- /dev/null +++ b/sql-plugin/src/main/spark350db143/scala/com/nvidia/spark/rapids/shims/ShuffleManagerShimUtils.scala @@ -0,0 +1,25 @@ +/* + * Copyright (c) 2024, 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "350db143"} +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +object ShuffleManagerShimUtils { + def eagerlyInitialized = true +} From d9b5c53a8f4f09413266d00d24d35b820b81e0cf Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Mon, 30 Dec 2024 10:29:15 +0800 Subject: [PATCH 29/47] Fix the build error for spark 400 (#11905) Spark 400 introduces a new parameter in BasePythonRunner, so this PR creates a new shim for this change and specifies this to an empty map to pass the build. It is ok since this metric parameter is only for debugging. Signed-off-by: Firestarman --- .../python/shims/GpuBasePythonRunner.scala | 1 - .../python/shims/GpuBasePythonRunner.scala | 31 +++++++++++++++++++ 2 files changed, 31 insertions(+), 1 deletion(-) create mode 100644 sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala index 8493fe596b9..91872ae3ac6 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala @@ -21,7 +21,6 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} -{"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution.python.shims diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala new file mode 100644 index 00000000000..aa147b9269a --- /dev/null +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala @@ -0,0 +1,31 @@ +/* + * Copyright (c) 2024, 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids.execution.python.shims + +import org.apache.spark.api.python.{BasePythonRunner, ChainedPythonFunctions} +import org.apache.spark.sql.vectorized.ColumnarBatch + +abstract class GpuBasePythonRunner[IN]( + funcs: Seq[ChainedPythonFunctions], + evalType: Int, + argOffsets: Array[Array[Int]], + jobArtifactUUID: Option[String] +) extends BasePythonRunner[IN, ColumnarBatch](funcs, evalType, argOffsets, jobArtifactUUID, + Map.empty) From 9e2b28a4750e56b9f0ad80e512757704bd6cd15a Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Mon, 30 Dec 2024 17:27:55 +0800 Subject: [PATCH 30/47] Fix generated doc for Spark 400 (#11907) Signed-off-by: Chong Gao Co-authored-by: Chong Gao --- tools/generated_files/400/supportedExprs.csv | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/generated_files/400/supportedExprs.csv b/tools/generated_files/400/supportedExprs.csv index d3f1f1b851a..926bb4f6c36 100644 --- a/tools/generated_files/400/supportedExprs.csv +++ b/tools/generated_files/400/supportedExprs.csv @@ -666,7 +666,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA From e215ae4abd386136ca395d575fc490cf22e81995 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Thu, 2 Jan 2025 15:18:52 -0600 Subject: [PATCH 31/47] Update copyright year in NOTICE (#11909) Signed-off-by: Jason Lowe --- NOTICE | 2 +- NOTICE-binary | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/NOTICE b/NOTICE index 657bfecd06a..cfed5e14d14 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ RAPIDS plugin for Apache Spark -Copyright (c) 2019-2024, NVIDIA CORPORATION +Copyright (c) 2019-2025, NVIDIA CORPORATION -------------------------------------------------------------------------------- diff --git a/NOTICE-binary b/NOTICE-binary index fe2f05c0882..eee217b29d5 100644 --- a/NOTICE-binary +++ b/NOTICE-binary @@ -1,5 +1,5 @@ RAPIDS plugin for Apache Spark -Copyright (c) 2019-2024, NVIDIA CORPORATION +Copyright (c) 2019-2025, NVIDIA CORPORATION // ------------------------------------------------------------------ // NOTICE file corresponding to the section 4d of The Apache License, From 4df6d601150fbe678237dcd54585ca26870a480a Mon Sep 17 00:00:00 2001 From: YanxuanLiu Date: Fri, 3 Jan 2025 13:26:21 +0800 Subject: [PATCH 32/47] fix bug: enable if_modified_files check for all shims in github actions [skip ci] (#11910) * change if_modified_files to package phase Signed-off-by: Yanxuan Liu * update copyright year Signed-off-by: Yanxuan Liu * update dist/pom Signed-off-by: Yanxuan Liu --------- Signed-off-by: Yanxuan Liu --- dist/pom.xml | 4 ++-- scala2.13/dist/pom.xml | 4 ++-- scala2.13/tools/pom.xml | 4 ++-- tools/pom.xml | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dist/pom.xml b/dist/pom.xml index b34292a25cd..6480d261a98 100644 --- a/dist/pom.xml +++ b/dist/pom.xml @@ -1,6 +1,6 @@ - org.alluxio - alluxio-shaded-client - ${alluxio.client.version} - provided - - org.alluxio - alluxio-shaded-client - ${alluxio.client.version} - provided - - org.alluxio - alluxio-shaded-client - org.mockito mockito-core diff --git a/sql-plugin/pom.xml b/sql-plugin/pom.xml index 2b0a62a5b90..a4225585439 100644 --- a/sql-plugin/pom.xml +++ b/sql-plugin/pom.xml @@ -1,6 +1,6 @@ - org.alluxio - alluxio-shaded-client - org.mockito mockito-core diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java index b32e5e755cb..a928f702bb0 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuMultiFileBatchReader.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -358,8 +358,6 @@ protected FilePartitionReaderBase createRapidsReader(PartitionedFile[] pFiles, false, // ignoreMissingFiles false, // ignoreCorruptFiles false, // useFieldId - scala.collection.immutable.Map$.MODULE$.empty(), // alluxioPathReplacementMap - false, // alluxioReplacementTaskTime queryUsesInputFile, true, // keepReadsInOrder new CombineConf( diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java index b7544675d41..cd5694370b4 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -270,7 +270,7 @@ private scala.Tuple3 multiFileReadCheck(ReadTask r .toArray(String[]::new); // Get the final decision for the subtype of the Rapids reader. boolean useMultiThread = MultiFileReaderUtils.useMultiThreadReader( - canUseCoalescing, canUseMultiThread, files, allCloudSchemes, false); + canUseCoalescing, canUseMultiThread, files, allCloudSchemes); return scala.Tuple3.apply(canAccelerateRead, useMultiThread, ff); } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioCfgUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioCfgUtils.scala deleted file mode 100644 index 7cdab3e58b6..00000000000 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioCfgUtils.scala +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Copyright (c) 2022, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.nvidia.spark.rapids - -import org.apache.spark.sql.execution.datasources.FileFormat -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat - -/* - * Utils for the alluxio configurations. - * If alluxio feature is disabled, we can tolerate the client jar is absent. - * Use the following pattern to avoid `NoClassDefFoundError` if alluxio feature is disabled: - * if(AlluxioCfgUtils.enabledAlluxio* functions) { - * // AlluxioCfgUtils does not import any alluxio class - * // Only AlluxioUtils imports alluxio classes - * AlluxioUtils.doSomething; - * } - */ -object AlluxioCfgUtils { - def checkAlluxioNotSupported(rapidsConf: RapidsConf): Unit = { - if (rapidsConf.isParquetPerFileReadEnabled && - (rapidsConf.getAlluxioAutoMountEnabled || rapidsConf.getAlluxioPathsToReplace.isDefined)) { - throw new IllegalArgumentException("Alluxio is currently not supported with the PERFILE " + - "reader, please use one of the other reader types.") - } - } - - /** - * Returns whether alluxio convert time algorithm should be enabled - * Note: should also check whether the auto-mount or replace path is enabled. - * - * @param conf the rapids conf - * @return Returns whether alluxio convert time algorithm should be enabled - */ - def enabledAlluxioReplacementAlgoConvertTime(conf: RapidsConf): Boolean = { - conf.isAlluxioReplacementAlgoConvertTime && - (conf.getAlluxioAutoMountEnabled || conf.getAlluxioPathsToReplace.isDefined) - } - - def enabledAlluxioReplacementAlgoTaskTime(conf: RapidsConf): Boolean = { - conf.isAlluxioReplacementAlgoTaskTime && - (conf.getAlluxioAutoMountEnabled || conf.getAlluxioPathsToReplace.isDefined) - } - - def isAlluxioAutoMountTaskTime(rapidsConf: RapidsConf, - fileFormat: FileFormat): Boolean = { - rapidsConf.getAlluxioAutoMountEnabled && rapidsConf.isAlluxioReplacementAlgoTaskTime && - fileFormat.isInstanceOf[ParquetFileFormat] - } - - def isAlluxioPathsToReplaceTaskTime(rapidsConf: RapidsConf, - fileFormat: FileFormat): Boolean = { - rapidsConf.getAlluxioPathsToReplace.isDefined && rapidsConf.isAlluxioReplacementAlgoTaskTime && - fileFormat.isInstanceOf[ParquetFileFormat] - } - - def isConfiguredReplacementMap(conf: RapidsConf): Boolean = { - conf.getAlluxioPathsToReplace.isDefined - } -} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioConfigReader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioConfigReader.scala deleted file mode 100644 index 50a681f6218..00000000000 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioConfigReader.scala +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Copyright (c) 2022-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 com.nvidia.spark.rapids - -import java.io.FileNotFoundException -import java.util.Properties - -import scala.io.BufferedSource - -/** - * Alluxio master address and port reader. - * It reads from `/opt/alluxio/conf/alluxio-site.properties` - */ -class AlluxioConfigReader { - - def readAlluxioMasterAndPort(conf: RapidsConf): (String, String) = { - if (conf.getAlluxioMaster.isEmpty) { - // Read from local Alluxio home - readMasterAndPort(conf.getAlluxioHome) - } else { - // Alluxio master separately deployed and not co-located with Spark Driver. - // Like: EKS Env, - (conf.getAlluxioMaster, conf.getAlluxioMasterPort.toString) - } - } - - // By default, read from /opt/alluxio, refer to `spark.rapids.alluxio.home` config in `RapidsConf` - // The default port is 19998 - private[rapids] def readMasterAndPort(homePath: String): (String, String) = { - var buffered_source: BufferedSource = null - try { - buffered_source = scala.io.Source.fromFile(homePath + "/conf/alluxio-site.properties") - val prop: Properties = new Properties() - prop.load(buffered_source.bufferedReader()) - val alluxio_master = prop.getProperty("alluxio.master.hostname") - if (alluxio_master == null) { - throw new RuntimeException( - s"Can't find alluxio.master.hostname from $homePath/conf/alluxio-site.properties.") - } - - val alluxio_port = prop.getProperty("alluxio.master.rpc.port", "19998") - (alluxio_master, alluxio_port) - } catch { - case _: FileNotFoundException => - throw new RuntimeException(s"Alluxio config file not found in " + - s"$homePath/conf/alluxio-site.properties, " + - "the default value of `spark.rapids.alluxio.home` is /opt/alluxio, " + - "please create a link `/opt/alluxio` to Alluxio installation home, " + - "or set `spark.rapids.alluxio.home` to Alluxio installation home") - } finally { - if (buffered_source != null) buffered_source.close - } - } -} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioFS.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioFS.scala deleted file mode 100644 index 88994bae182..00000000000 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioFS.scala +++ /dev/null @@ -1,103 +0,0 @@ -/* - * Copyright (c) 2022-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 com.nvidia.spark.rapids - -import scala.collection.JavaConverters.mapAsScalaMapConverter -import scala.collection.mutable - -import alluxio.AlluxioURI -import alluxio.conf.{AlluxioProperties, InstancedConfiguration, PropertyKey} -import alluxio.grpc.MountPOptions -import com.nvidia.spark.rapids.Arm.withResource - -/** - * interfaces for Alluxio file system. - * Currently contains interfaces: - * get mount points - * mount - */ -class AlluxioFS { - private var masterHost: String = _ - private var masterPort: Int = _ - private var masterHostAndPort: Option[String] = None - - private var alluxioUser: String = "" - private var s3AccessKey: Option[String] = None - private var s3SecretKey: Option[String] = None - - def setHostAndPort(masterHost: String, masterPort: Int): Unit = { - this.masterHost = masterHost - this.masterPort = masterPort - this.masterHostAndPort = Some(masterHost + ":" + masterPort) - } - - def setUserAndKeys(alluxioUser: String, s3AccessKey: Option[String], - s3SecretKey: Option[String]): Unit = { - this.alluxioUser = alluxioUser - this.s3AccessKey = s3AccessKey - this.s3SecretKey = s3SecretKey - } - - private def getS3ClientConf(): InstancedConfiguration = { - val p = new AlluxioProperties() - p.set(PropertyKey.MASTER_HOSTNAME, masterHost) - p.set(PropertyKey.MASTER_RPC_PORT, masterPort) - s3AccessKey.foreach(access => p.set(PropertyKey.S3A_ACCESS_KEY, access)) - s3SecretKey.foreach(secret => p.set(PropertyKey.S3A_SECRET_KEY, secret)) - p.set(PropertyKey.SECURITY_LOGIN_USERNAME, alluxioUser) - new InstancedConfiguration(p) - } - - /** - * Get S3 mount points by Alluxio client - * - * @return mount points map, key of map is Alluxio path, value of map is S3 path. - * E.g.: returns a map: {'/bucket_1': 's3://bucket_1'} - */ - def getExistingMountPoints(): mutable.Map[String, String] = { - val conf = getS3ClientConf() - // get s3 mount points by Alluxio client - withResource(alluxio.client.file.FileSystem.Factory.create(conf)) { fs => - val mountTable = fs.getMountTable - mountTable.asScala.filter { case (_, mountPoint) => - // checked the alluxio code, the type should be s3 - // anyway let's keep both of them - mountPoint.getUfsType == "s3" || mountPoint.getUfsType == "s3a" - }.map { case (alluxioPath, s3Point) => - (alluxioPath, s3Point.getUfsUri) - } - } - } - - /** - * Mount an S3 path to Alluxio - * - * @param alluxioPath Alluxio path - * @param s3Path S3 path - */ - def mount(alluxioPath: String, s3Path: String): Unit = { - val conf = getS3ClientConf() - withResource(alluxio.client.file.FileSystem.Factory.create(conf)) { fs => - val mountOptionsBuilder = MountPOptions.newBuilder().setReadOnly(true) - s3AccessKey.foreach(e => mountOptionsBuilder.putProperties("s3a.accessKeyId", e)) - s3SecretKey.foreach(e => mountOptionsBuilder.putProperties("s3a.secretKey", e)) - fs.mount(new AlluxioURI(alluxioPath), new AlluxioURI(s3Path), - mountOptionsBuilder.build()) - } - } - - def getMasterHostAndPort(): Option[String] = masterHostAndPort -} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala deleted file mode 100644 index b717eb994c3..00000000000 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AlluxioUtils.scala +++ /dev/null @@ -1,671 +0,0 @@ -/* - * Copyright (c) 2022-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 com.nvidia.spark.rapids - -import java.io.FileNotFoundException - -import scala.util.control.NonFatal - -import com.nvidia.spark.rapids.shims.PartitionedFileUtilsShim -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path - -import org.apache.spark.internal.Logging -import org.apache.spark.sql.RuntimeConfig -import org.apache.spark.sql.catalyst.expressions.{DynamicPruning, Expression, PlanExpression} -import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, FileIndex, HadoopFsRelation, InMemoryFileIndex, PartitionDirectory, PartitionedFile, PartitionSpec} -import org.apache.spark.sql.execution.datasources.rapids.GpuPartitioningUtils - -/* - * Utilities for using Alluxio with the plugin for reading. - * Currently we only support Alluxio with the Datasource v1 Parquet reader. - * We currently support 2 different replacement algorithms: - * CONVERT_TIME: this replaces the file path when we convert a FileSourceScanExec to - * a GpuFileSourceScanExec. This will create an entirely new FileIndex and potentially - * has to re-infer the partitioning if its not a FileIndex type we know. So this can - * cause an extra list leaf files which for many files will run another job and thus - * has additional overhead. This will update the file locations to be the - * Alluxio specific ones if the data is already cached. In order to support the - * input_file_name functionality we have to convert the alluxio:// path back to its - * original url when we go to actually read the file. - * TASK_TIME: this replaces the file path as late as possible on the task side when - * we actually go to read the file. This makes is so that the original non-Alluxio - * path gets reported for the input_file_name properly without having to convert - * paths back to the original. This also has the benefit that it can be more performant - * if it doesn't have to do the extra list leaf files, but you don't get the - * locality information updated. So for small Alluxio clusters or with Spark - * clusters short on task slots this may be a better fit. - * - * The way we do the actual replacement algorithm differs depending on the file reader - * type we use: PERFILE, COALESCING or MULTITHREADED. - * PERFILE is not supported with Alluxio due to not easily being able to fix up - * input_file_name. We could but would require copying the FileScanRDD so skip for now. - * The COALESCING reader is not support when input_file_name is requested so it falls - * back to the MULTITHREADED reader if that is used, when input_file_name is not requested, - * we replace the paths properly based on the replacement algorithm and don't have to worry - * about calculating the original path. The MULTITHREADED reader supports input_file_name - * so it handles calculating the original file path in the case of the convert time algorithm. - * In order to do the replacement at task time and to output the original path for convert - * time, we need to have a mapping of the original scheme to the alluxio scheme. This has been - * made a parameter to many of the readers. With auto mount and task time replacement, - * we make a pass through the files on the driver side in GpuFileSourceScanExec in order to - * do the mounting before the tasks try to access alluxio. - * Note that Delta Lake uses the input_file_name functionality to do things like - * Updates and Deletes and will fail if the path has the alluxio:// in it. - * - * Below we support 2 configs to turn on Alluxio, we have the automount which uses a regex - * to replace paths and then we have the config that specifies direct paths to replace and - * user has to manually mount those. - */ -object AlluxioUtils extends Logging { - private val checkedAlluxioPath = scala.collection.mutable.HashSet[String]() - private val ALLUXIO_SCHEME = "alluxio://" - private val mountedBuckets: scala.collection.mutable.Map[String, String] = - scala.collection.mutable.Map() - private var alluxioPathsToReplaceMap: Option[Map[String, String]] = None - private var alluxioBucketRegex: Option[String] = None - private var isInitReplaceMap: Boolean = false - private var isInitMountPointsForAutoMount: Boolean = false - private var alluxioFS: AlluxioFS = new AlluxioFS() - private var alluxioMasterAndPortReader = new AlluxioConfigReader() - - private def checkAlluxioMounted( - hadoopConfiguration: Configuration, - alluxio_path: String): Unit = { - this.synchronized { - if (!checkedAlluxioPath.contains(alluxio_path)) { - val path = new Path(alluxio_path) - val fs = path.getFileSystem(hadoopConfiguration) - if (!fs.exists(path)) { - throw new FileNotFoundException( - s"Alluxio path $alluxio_path does not exist, maybe forgot to mount it") - } - logDebug(s"Alluxio path $alluxio_path is mounted") - checkedAlluxioPath.add(alluxio_path) - } else { - logDebug(s"Alluxio path $alluxio_path already mounted") - } - } - } - - // By default, read from /opt/alluxio, refer to `spark.rapids.alluxio.home` config in `RapidsConf` - private def readAlluxioMasterAndPort(conf: RapidsConf): (String, String) = { - alluxioMasterAndPortReader.readAlluxioMasterAndPort(conf) - } - - // Read out alluxio.master.hostname, alluxio.master.rpc.port - // from Alluxio's conf alluxio-site.properties - // We require an environment variable "ALLUXIO_HOME" - // This function will only read once from ALLUXIO/conf. - private def initAlluxioInfo(conf: RapidsConf, hadoopConf: Configuration, - runtimeConf: RuntimeConfig): Unit = { - this.synchronized { - // left outside isInit to allow changing at runtime - AlluxioCfgUtils.checkAlluxioNotSupported(conf) - - if (AlluxioCfgUtils.isConfiguredReplacementMap(conf)) { - // replace-map is enabled, if set this will invalid the auto-mount - if (!isInitReplaceMap) { - alluxioPathsToReplaceMap = getReplacementMapOption(conf) - isInitReplaceMap = true - } - } else if (conf.getAlluxioAutoMountEnabled) { - // auto-mount is enabled - if (!isInitMountPointsForAutoMount) { - val (alluxioMasterHostStr, alluxioMasterPortStr) = readAlluxioMasterAndPort(conf) - alluxioBucketRegex = Some(conf.getAlluxioBucketRegex) - // load mounted point by call Alluxio client. - try { - val (access_key, secret_key) = getKeyAndSecret(hadoopConf, runtimeConf) - // get existing mount points - alluxioFS.setHostAndPort(alluxioMasterHostStr, alluxioMasterPortStr.toInt) - alluxioFS.setUserAndKeys(conf.getAlluxioUser, access_key, secret_key) - val mountPoints = alluxioFS.getExistingMountPoints() - - mountPoints.foreach { case (alluxioPath, s3Path) => - // record map info from alluxio path to s3 path - mountedBuckets(alluxioPath) = s3Path - logInfo(s"Found mounted bucket $s3Path to $alluxioPath") - } - } catch { - case NonFatal(e) => logWarning(s"Failed to get alluxio mount table", e) - } - isInitMountPointsForAutoMount = true - } - } else { - // disabled Alluxio feature, do nothing - } - } - } - - // The path should be like s3://bucket/... or s3a://bucket/... - private def getSchemeAndBucketFromPath(path: String): (String, String) = { - val i = path.split("://") - val scheme = i(0) - if (i.length <= 1) { - throw new RuntimeException(s"path $path is not expected for Alluxio auto mount") - } - val bucket = i(1).split("/")(0) - (scheme + "://", bucket) - } - - // path is like "s3://foo/test...", it mounts bucket "foo" by calling the alluxio CLI - // And we'll append --option to set access_key and secret_key if existing. - // Suppose the key doesn't exist when using like Databricks's instance profile - private def autoMountBucket(alluxioUser: String, scheme: String, bucket: String, - s3AccessKey: Option[String], s3SecretKey: Option[String]): Unit = { - - // to match the output of alluxio fs mount, append / to remote_path - // and add / before bucket name for absolute path in Alluxio - val remote_path = scheme + bucket + "/" - val local_bucket = "/" + bucket - this.synchronized { - if (!mountedBuckets.contains(local_bucket)) { - try { - // not mount yet, call mount command - // we only support s3 or s3a bucket for now. - // To support other cloud storage, - // we need to support credential parameters for the others - alluxioFS.setUserAndKeys(alluxioUser, s3AccessKey, s3SecretKey) - alluxioFS.mount(local_bucket, remote_path) - logInfo(s"Mounted bucket $remote_path to $local_bucket in Alluxio") - mountedBuckets(local_bucket) = remote_path - } catch { - case NonFatal(e) => - throw new RuntimeException(s"Mount bucket $remote_path to $local_bucket failed", e) - } - } else if (mountedBuckets(local_bucket).equals(remote_path)) { - logDebug(s"Already mounted bucket $remote_path to $local_bucket in Alluxio") - } else { - throw new RuntimeException(s"Found a same bucket name in $remote_path " + - s"and ${mountedBuckets(local_bucket)}") - } - } - } - - // first try to get fs.s3a.access.key from spark config - // second try to get from environment variables - private def getKeyAndSecret( - hadoopConfiguration: Configuration, - runtimeConf: RuntimeConfig): (Option[String], Option[String]) = { - val hadoopAccessKey = - hadoopConfiguration.get("fs.s3a.access.key") - val hadoopSecretKey = - hadoopConfiguration.get("fs.s3a.secret.key") - if (hadoopAccessKey != null && hadoopSecretKey != null) { - (Some(hadoopAccessKey), Some(hadoopSecretKey)) - } else { - val accessKey = runtimeConf.getOption("spark.hadoop.fs.s3a.access.key") - val secretKey = runtimeConf.getOption("spark.hadoop.fs.s3a.secret.key") - if (accessKey.isDefined && secretKey.isDefined) { - (accessKey, secretKey) - } else { - val envAccessKey = scala.util.Properties.envOrNone("AWS_ACCESS_KEY_ID") - val envSecretKey = scala.util.Properties.envOrNone("AWS_ACCESS_SECRET_KEY") - (envAccessKey, envSecretKey) - } - } - } - - private def replaceSchemeWithAlluxio(file: String, scheme: String, masterPort: String): String = { - // replace s3://foo/.. to alluxio://alluxioMasterHostAndPort/foo/... - val newFile = file.replaceFirst(scheme, ALLUXIO_SCHEME + masterPort + "/") - logDebug(s"Replace $file to $newFile") - newFile - } - - private def genFuncForPathReplacement( - replaceMapOption: Option[Map[String, String]]) - : Option[Path => AlluxioPathReplaceConvertTime] = { - if (replaceMapOption.isDefined) { - Some((f: Path) => { - val pathStr = f.toString - val matchedSet = replaceMapOption.get.filter(a => pathStr.startsWith(a._1)) - if (matchedSet.size > 1) { - // never reach here since replaceMap is a Map - throw new IllegalArgumentException(s"Found ${matchedSet.size} same replacing rules " + - s"from ${RapidsConf.ALLUXIO_PATHS_REPLACE.key} which requires only 1 rule " + - s"for each file path") - } else if (matchedSet.size == 1) { - val res = AlluxioPathReplaceConvertTime( - new Path(pathStr.replaceFirst(matchedSet.head._1, matchedSet.head._2)), - Some(matchedSet.head._1)) - logDebug(s"Specific path replacement, replacing paths with: $res") - res - } else { - AlluxioPathReplaceConvertTime(f, None) - } - }) - } else { - None - } - } - - private def genFuncForAutoMountReplacement( - alluxioUser: String, - runtimeConf: RuntimeConfig, - hadoopConf: Configuration): Option[Path => AlluxioPathReplaceConvertTime] = { - assert(alluxioFS.getMasterHostAndPort().isDefined) - assert(alluxioBucketRegex.isDefined) - - Some((f: Path) => { - val pathStr = f.toString - val res = if (pathStr.matches(alluxioBucketRegex.get)) { - val (access_key, secret_key) = getKeyAndSecret(hadoopConf, runtimeConf) - val (scheme, bucket) = getSchemeAndBucketFromPath(pathStr) - autoMountBucket(alluxioUser, scheme, bucket, access_key, secret_key) - AlluxioPathReplaceConvertTime( - new Path(replaceSchemeWithAlluxio(pathStr, scheme, alluxioFS.getMasterHostAndPort().get)), - Some(scheme)) - } else { - AlluxioPathReplaceConvertTime(f, None) - } - logDebug(s"Automount replacing paths: $res") - res - }) - } - - // Contains the file string to read and contains a boolean indicating if the - // path was updated to an alluxio:// path. - private case class AlluxioPathReplaceTaskTime(fileStr: String, wasReplaced: Boolean) - - // Contains the file Path to read and optionally contains the prefix of the original path. - // The original path is needed when using the input_file_name option with the reader so - // it reports the original path and not the alluxio version - case class AlluxioPathReplaceConvertTime(filePath: Path, origPrefix: Option[String]) - - // Replaces the file name with Alluxio one if it matches. - // Returns a tuple with the file path and whether or not it replaced the - // scheme with the Alluxio one. - private def genFuncForTaskTimeReplacement(pathsToReplace: Map[String, String]) - : String => AlluxioPathReplaceTaskTime = { - (pathStr: String) => { - // pathsToReplace contain strings of exact paths to replace - val matchedSet = pathsToReplace.filter { case (pattern, _) => pathStr.startsWith(pattern) } - if (matchedSet.size > 1) { - // never reach here since replaceMap is a Map - throw new IllegalArgumentException(s"Found ${matchedSet.size} same replacing rules " + - s"from ${RapidsConf.ALLUXIO_PATHS_REPLACE.key} which requires only 1 rule " + - s"for each file path") - } else if (matchedSet.size == 1) { - AlluxioPathReplaceTaskTime( - pathStr.replaceFirst(matchedSet.head._1, matchedSet.head._2), wasReplaced = true) - } else { - AlluxioPathReplaceTaskTime(pathStr, wasReplaced = false) - } - } - } - - private def getReplacementMapOption(conf: RapidsConf): Option[Map[String, String]] = { - val alluxioPathsReplace: Option[Seq[String]] = conf.getAlluxioPathsToReplace - // alluxioPathsReplace: Seq("key->value", "key1->value1") - // turn the rules to the Map with eg - // { s3://foo -> alluxio://0.1.2.3:19998/foo, - // gs://bar -> alluxio://0.1.2.3:19998/bar } - if (alluxioPathsReplace.isDefined) { - alluxioPathsReplace.map(rules => { - rules.map(rule => { - val split = rule.split("->") - if (split.size == 2) { - split(0).trim -> split(1).trim - } else { - throw new IllegalArgumentException(s"Invalid setting for " + - s"${RapidsConf.ALLUXIO_PATHS_REPLACE.key}") - } - }).toMap - }) - } else { - None - } - } - - private def getReplacementFunc( - conf: RapidsConf, - runtimeConf: RuntimeConfig, - hadoopConf: Configuration): Option[Path => AlluxioPathReplaceConvertTime] = { - if (conf.getAlluxioPathsToReplace.isDefined) { - genFuncForPathReplacement(alluxioPathsToReplaceMap) - } else if (conf.getAlluxioAutoMountEnabled) { - genFuncForAutoMountReplacement(conf.getAlluxioUser, runtimeConf, hadoopConf) - } else { - None - } - } - - // assumes Alluxio directories already mounted at this point - def updateFilesTaskTimeIfAlluxio( - origFiles: Array[PartitionedFile], - alluxioPathReplacementMap: Option[Map[String, String]]) - : Array[PartitionedFileInfoOptAlluxio] = { - val res: Array[PartitionedFileInfoOptAlluxio] = - alluxioPathReplacementMap.map { pathsToReplace => - replacePathInPartitionFileTaskTimeIfNeeded(pathsToReplace, origFiles) - }.getOrElse(origFiles.map(PartitionedFileInfoOptAlluxio(_, None))) - logDebug(s"Updated files at TASK_TIME for Alluxio: ${res.mkString(",")}") - res - } - - // Replaces the path if needed and returns the replaced path and optionally the - // original file if it replaced the scheme with an Alluxio scheme. - private def replacePathInPartitionFileTaskTimeIfNeeded( - pathsToReplace: Map[String, String], - files: Array[PartitionedFile]): Array[PartitionedFileInfoOptAlluxio] = { - val replaceFunc = genFuncForTaskTimeReplacement(pathsToReplace) - - files.map { file => - val replacedFileInfo = replaceFunc(file.filePath.toString()) - if (replacedFileInfo.wasReplaced) { - logDebug(s"TASK_TIME replaced ${file.filePath} with ${replacedFileInfo.fileStr}") - PartitionedFileInfoOptAlluxio( - PartitionedFileUtilsShim.newPartitionedFile(file.partitionValues, - replacedFileInfo.fileStr, file.start, file.length), - Some(file)) - } else { - PartitionedFileInfoOptAlluxio(file, None) - } - } - } - - // if auto-mount, use this to check if need replacements - def autoMountIfNeeded( - conf: RapidsConf, - pds: Seq[PartitionDirectory], - hadoopConf: Configuration, - runtimeConf: RuntimeConfig): Option[Map[String, String]] = { - val alluxioAutoMountEnabled = conf.getAlluxioAutoMountEnabled - initAlluxioInfo(conf, hadoopConf, runtimeConf) - if (alluxioAutoMountEnabled) { - val (access_key, secret_key) = getKeyAndSecret(hadoopConf, runtimeConf) - val replacedSchemes = pds.flatMap { pd => - pd.files.map(_.getPath.toString).flatMap { file => - if (file.matches(alluxioBucketRegex.get)) { - val (scheme, bucket) = getSchemeAndBucketFromPath(file) - autoMountBucket(conf.getAlluxioUser, scheme, bucket, access_key, secret_key) - Some(scheme) - } else { - None - } - } - } - if (replacedSchemes.nonEmpty) { - val alluxioMasterHostAndPort = alluxioFS.getMasterHostAndPort() - Some(replacedSchemes.map(_ -> (ALLUXIO_SCHEME + alluxioMasterHostAndPort.get + "/")).toMap) - } else { - None - } - } else { - None - } - } - - // If specified replace map, use this to check if need replacements - def checkIfNeedsReplaced( - conf: RapidsConf, - pds: Seq[PartitionDirectory], - hadoopConf: Configuration, - runtimeConf: RuntimeConfig): Option[Map[String, String]] = { - initAlluxioInfo(conf, hadoopConf, runtimeConf) - val anyToReplace = pds.exists { pd => - pd.files.map(_.getPath.toString).exists { file => - val matchedSet = alluxioPathsToReplaceMap.get.filter(a => file.startsWith(a._1)) - if (matchedSet.size > 1) { - // never reach here since replaceMap is a Map - throw new IllegalArgumentException(s"Found ${matchedSet.size} same replacing rules " + - s"from ${RapidsConf.ALLUXIO_PATHS_REPLACE.key} which requires only 1 rule " + - s"for each file path") - } else if (matchedSet.size == 1) { - true - } else { - false - } - } - } - if (anyToReplace) { - alluxioPathsToReplaceMap - } else { - None - } - } - - // reverse the replacePathIfNeeded, returns a tuple of the file passed in and then if it - // was replaced the original file - def getOrigPathFromReplaced(pfs: Array[PartitionedFile], - pathsToReplace: Map[String, String]): Array[PartitionedFileInfoOptAlluxio] = { - pfs.map { pf => - val file = pf.filePath.toString() - // pathsToReplace contain strings of exact paths to replace - val matchedSet = pathsToReplace.filter { case (_, alluxioPattern) => - file.startsWith(alluxioPattern) - } - if (matchedSet.size > 1) { - // never reach here since replaceMap is a Map - throw new IllegalArgumentException(s"Found ${matchedSet.size} same replacing rules " + - s"from ${RapidsConf.ALLUXIO_PATHS_REPLACE.key} which requires only 1 rule " + - s"for each file path") - } else if (matchedSet.size == 1) { - val replacedFile = file.replaceFirst(matchedSet.head._2, matchedSet.head._1) - logDebug(s"getOrigPath replacedFile: $replacedFile") - PartitionedFileInfoOptAlluxio(pf, - Some(PartitionedFileUtilsShim.newPartitionedFile(pf.partitionValues, replacedFile, - pf.start, file.length))) - } else { - PartitionedFileInfoOptAlluxio(pf, None) - } - } - } - - // This is used when replacement algorithm is CONVERT_TIME and causes - // a new lookup on the alluxio files. For unknown FileIndex types it can - // also cause us to have to infer the partitioning again. - def replacePathIfNeeded( - conf: RapidsConf, - relation: HadoopFsRelation, - partitionFilters: Seq[Expression], - dataFilters: Seq[Expression]): (FileIndex, Option[Map[String, String]]) = { - val hadoopConf = relation.sparkSession.sparkContext.hadoopConfiguration - val runtimeConf = relation.sparkSession.conf - initAlluxioInfo(conf, hadoopConf, runtimeConf) - val replaceFunc = getReplacementFunc(conf, runtimeConf, hadoopConf) - - val (location, allReplacedPrefixes) = if (replaceFunc.isDefined) { - def replacePathsInPartitionSpec(spec: PartitionSpec): (PartitionSpec, Seq[String]) = { - val partitionsWithPathsReplaced = spec.partitions.map { p => - val replacedPathAndPrefix = replaceFunc.get(p.path) - (org.apache.spark.sql.execution.datasources.PartitionPath(p.values, - replacedPathAndPrefix.filePath), - replacedPathAndPrefix.origPrefix) - } - val paths = partitionsWithPathsReplaced.map(_._1) - val replacedPrefixes = partitionsWithPathsReplaced.flatMap(_._2) - (PartitionSpec(spec.partitionColumns, paths), replacedPrefixes) - } - - def createNewFileIndexWithPathsReplaced( - spec: PartitionSpec, - rootPaths: Seq[Path]): (InMemoryFileIndex, Seq[String]) = { - val (specAdjusted, replacedPrefixes) = replacePathsInPartitionSpec(spec) - val replacedPathsAndIndicator = rootPaths.map(replaceFunc.get) - val replacedPaths = replacedPathsAndIndicator.map(_.filePath) - val didReplaceAnyRoots = replacedPathsAndIndicator.flatMap(_.origPrefix) - val fi = new InMemoryFileIndex( - relation.sparkSession, - replacedPaths, - relation.options, - Option(relation.dataSchema), - userSpecifiedPartitionSpec = Some(specAdjusted)) - (fi, didReplaceAnyRoots ++ replacedPrefixes) - } - - // Before this change https://github.com/NVIDIA/spark-rapids/pull/6806, - // if we know the type of file index, try to reuse as much of the existing - // FileIndex as we can to keep from having to recalculate it and potentially - // mess it up. But this causes always reading old files, see the issue of this PR. - // - // Now, because we have the Task time replacement algorithm, - // just fall back to inferring partitioning for all FileIndex types except - // CatalogFileIndex. - // We use this approach to handle all the file index types for all CSPs like Databricks. - relation.location match { - case cfi: CatalogFileIndex => - logDebug("Handling CatalogFileIndex") - val memFI = cfi.filterPartitions(Nil) - createNewFileIndexWithPathsReplaced(memFI.partitionSpec(), memFI.rootPaths) - case _ => - logDebug(s"Handling file index type: ${relation.location.getClass}") - - // With the base Spark FileIndex type we don't know how to modify it to - // just replace the paths so we have to try to recompute. - def isDynamicPruningFilter(e: Expression): Boolean = { - e.isInstanceOf[DynamicPruning] || e.find(_.isInstanceOf[PlanExpression[_]]).isDefined - } - - val partitionDirs = relation.location.listFiles( - partitionFilters.filterNot(isDynamicPruningFilter), dataFilters) - - // replace all of input files - val inputFilesAndDidReplace = partitionDirs.flatMap(partitionDir => { - partitionDir.files.map(f => replaceFunc.get(f.getPath)) - }) - val inputFiles = inputFilesAndDidReplace.map(_.filePath) - val didReplaceAny = inputFilesAndDidReplace.flatMap(_.origPrefix) - - // replace all of rootPaths which are already unique - val rootPathsAndDidReplace = relation.location.rootPaths.map(replaceFunc.get) - val rootPaths = rootPathsAndDidReplace.map(_.filePath) - val rootPathsDidReplace = rootPathsAndDidReplace.flatMap(_.origPrefix) - - // check the alluxio paths in root paths exist or not - // throw out an exception to stop the job when any of them is not mounted - if (alluxioPathsToReplaceMap.isDefined) { - rootPaths.foreach { rootPath => - alluxioPathsToReplaceMap.get.values. - find(value => rootPath.toString.startsWith(value)). - foreach(matched => checkAlluxioMounted(hadoopConf, matched)) - } - } - - val parameters: Map[String, String] = relation.options - - // infer PartitionSpec - val (partitionSpec, replacedBasePath) = GpuPartitioningUtils.inferPartitioning( - relation.sparkSession, - rootPaths, - inputFiles, - parameters, - Option(relation.dataSchema), - replaceFunc.get) - - val allReplacedPrefixes = didReplaceAny ++ rootPathsDidReplace ++ replacedBasePath - // generate a new InMemoryFileIndex holding paths with alluxio schema - val fi = new InMemoryFileIndex( - relation.sparkSession, - inputFiles, - parameters, - Option(relation.dataSchema), - userSpecifiedPartitionSpec = Some(partitionSpec)) - (fi, allReplacedPrefixes) - } - } else { - (relation.location, Seq.empty) - } - val mapIfReplacedPaths = if (allReplacedPrefixes.nonEmpty) { - // with alluxio.automount.enabled we only have a regex so we need to track - // the exact schemes we replaced in order to set the input_file_name properly, - // with the alluxio.pathsToReplace it already contains the exact paths - if (conf.getAlluxioAutoMountEnabled) { - Some(allReplacedPrefixes.map( - _ -> (ALLUXIO_SCHEME + alluxioFS.getMasterHostAndPort().get + "/")).toMap) - } else { - alluxioPathsToReplaceMap - } - } else { - None - } - (location, mapIfReplacedPaths) - } - - // If reading large s3 files on a cluster with slower disks, skip using Alluxio. - def shouldReadDirectlyFromS3(rapidsConf: RapidsConf, pds: Seq[PartitionDirectory]): Boolean = { - if (!rapidsConf.enableAlluxioSlowDisk) { - logInfo(s"Skip reading directly from S3 because spark.rapids.alluxio.slow.disk is disabled") - false - } else { - val filesWithoutDir = pds.flatMap(pd => pd.files).filter { file => - // skip directory - !file.isDirectory - } - - val files = filesWithoutDir.filter { f => - /** - * Determines whether a file should be calculated for the average file size. - * This is used to filter out some unrelated files, - * such as transaction log files in the Delta file type. - * However Delta files has other unrelated - * files such as old regular parquet files. - * Limitation: This is not OK for Delta file type, json file type, Avro file type...... - * Currently only care about parquet and orc files. - * Note: It's hard to extract this into a method, because in Databricks 312 the files in - * `PartitionDirectory` are in type of - * `org.apache.spark.sql.execution.datasources.SerializableFileStatus` - * instead of `org.apache.hadoop.fs.FileStatus` - */ - f.getPath.getName.endsWith(".parquet") || f.getPath.getName.endsWith(".orc") - } - - val totalSize = files.map(_.getLen).sum - - val avgSize = if (files.isEmpty) 0 else totalSize / files.length - if (avgSize > rapidsConf.getAlluxioLargeFileThreshold) { - // if files are large - logInfo(s"Reading directly from S3, " + - s"average file size $avgSize > threshold ${rapidsConf.getAlluxioLargeFileThreshold}") - true - } else { - logInfo(s"Skip reading directly from S3, " + - s"average file size $avgSize <= threshold ${rapidsConf.getAlluxioLargeFileThreshold}") - false - } - } - } - - - /** - * For test purpose only - */ - def setAlluxioFS(alluxioFSMock: AlluxioFS): Unit = { - alluxioFS = alluxioFSMock - } - - /** - * For test purpose only - */ - def setAlluxioMasterAndPortReader( - alluxioMasterAndPortReaderMock: AlluxioConfigReader): Unit = { - alluxioMasterAndPortReader = alluxioMasterAndPortReaderMock - } - - /** - * For test purpose only - */ - def resetInitInfo(): Unit = { - isInitReplaceMap = false - isInitMountPointsForAutoMount = false - alluxioPathsToReplaceMap = None - alluxioBucketRegex = None - mountedBuckets.clear() - } -} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala index ab02d1f0eea..6d8ab11c60c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMultiFileReader.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -69,8 +69,6 @@ object SingleHMBAndMeta { trait HostMemoryBuffersWithMetaDataBase { // PartitionedFile to be read def partitionedFile: PartitionedFile - // Original PartitionedFile if path was replaced with Alluxio - def origPartitionedFile: Option[PartitionedFile] = None // An array of BlockChunk(HostMemoryBuffer and its data size) read from PartitionedFile def memBuffersAndSizes: Array[SingleHMBAndMeta] // Total bytes read @@ -174,16 +172,12 @@ object MultiFileReaderUtils { filePaths.exists(fp => cloudSchemes.contains(fp.getScheme)) } - // If Alluxio is enabled and we do task time replacement we have to take that - // into account here so we use the Coalescing reader instead of the MultiThreaded reader. def useMultiThreadReader( coalescingEnabled: Boolean, multiThreadEnabled: Boolean, files: Array[String], - cloudSchemes: Set[String], - anyAlluxioPathsReplaced: Boolean = false): Boolean = - !coalescingEnabled || (multiThreadEnabled && - (!anyAlluxioPathsReplaced && hasPathInCloud(files, cloudSchemes))) + cloudSchemes: Set[String]): Boolean = + !coalescingEnabled || (multiThreadEnabled && hasPathInCloud(files, cloudSchemes)) } /** @@ -193,14 +187,11 @@ object MultiFileReaderUtils { * @param sqlConf the SQLConf * @param broadcastedConf the Hadoop configuration * @param rapidsConf the Rapids configuration - * @param alluxioPathReplacementMap Optional map containing mapping of DFS - * scheme to Alluxio scheme */ abstract class MultiFilePartitionReaderFactoryBase( @transient sqlConf: SQLConf, broadcastedConf: Broadcast[SerializableConfiguration], - @transient rapidsConf: RapidsConf, - alluxioPathReplacementMap: Option[Map[String, String]] = None) + @transient rapidsConf: RapidsConf) extends PartitionReaderFactory with Logging { protected val maxReadBatchSizeRows: Int = rapidsConf.maxReadBatchSizeRows @@ -283,7 +274,7 @@ abstract class MultiFilePartitionReaderFactoryBase( /** for testing */ private[rapids] def useMultiThread(filePaths: Array[String]): Boolean = MultiFileReaderUtils.useMultiThreadReader(canUseCoalesceFilesReader, - canUseMultiThreadReader, filePaths, allCloudSchemes, alluxioPathReplacementMap.isDefined) + canUseMultiThreadReader, filePaths, allCloudSchemes) } /** @@ -310,11 +301,6 @@ abstract class FilePartitionReaderBase(conf: Configuration, execMetrics: Map[Str } } -// Contains the actual file path to read from and then an optional original path if its read from -// Alluxio. To make it transparent to the user, we return the original non-Alluxio path -// for input_file_name. -case class PartitionedFileInfoOptAlluxio(toRead: PartitionedFile, original: Option[PartitionedFile]) - case class CombineConf( combineThresholdSize: Long, // The size to combine to when combining small files combineWaitTime: Int) // The amount of time to wait for other files ready for combination. @@ -334,8 +320,6 @@ case class CombineConf( * @param filters push down filters * @param execMetrics the metrics * @param ignoreCorruptFiles Whether to ignore corrupt files when GPU failed to decode the files - * @param alluxioPathReplacementMap Map containing mapping of DFS scheme to Alluxio scheme - * @param alluxioReplacementTaskTime Whether the Alluxio replacement algorithm is set to task time * @param keepReadsInOrder Whether to require the files to be read in the same order as Spark. * Defaults to true for formats that don't explicitly handle this. * @param combineConf configs relevant to combination @@ -350,8 +334,6 @@ abstract class MultiFileCloudPartitionReaderBase( maxReadBatchSizeRows: Int, maxReadBatchSizeBytes: Long, ignoreCorruptFiles: Boolean = false, - alluxioPathReplacementMap: Map[String, String] = Map.empty, - alluxioReplacementTaskTime: Boolean = false, keepReadsInOrder: Boolean = true, combineConf: CombineConf = CombineConf(-1, -1)) extends FilePartitionReaderBase(conf, execMetrics) { @@ -369,23 +351,9 @@ abstract class MultiFileCloudPartitionReaderBase( // like in the case of a limit call and we don't read all files private var fcs: ExecutorCompletionService[HostMemoryBuffersWithMetaDataBase] = null - private val files: Array[PartitionedFileInfoOptAlluxio] = { - if (alluxioPathReplacementMap.nonEmpty) { - if (alluxioReplacementTaskTime) { - AlluxioUtils.updateFilesTaskTimeIfAlluxio(inputFiles, Some(alluxioPathReplacementMap)) - } else { - // was done at CONVERT_TIME, need to recalculate the original path to set for - // input_file_name - AlluxioUtils.getOrigPathFromReplaced(inputFiles, alluxioPathReplacementMap) - } - } else { - inputFiles.map(PartitionedFileInfoOptAlluxio(_, None)) - } - } - private def initAndStartReaders(): Unit = { // limit the number we submit at once according to the config if set - val limit = math.min(maxNumFileProcessed, files.length) + val limit = math.min(maxNumFileProcessed, inputFiles.length) val tc = TaskContext.get if (!keepReadsInOrder) { logDebug("Not keeping reads in order") @@ -404,25 +372,25 @@ abstract class MultiFileCloudPartitionReaderBase( // the files in the future. ie try to start some of the larger files but we may not want // them all to be large for (i <- 0 until limit) { - val file = files(i) - logDebug(s"MultiFile reader using file ${file.toRead}, orig file is ${file.original}") + val file = inputFiles(i) + logDebug(s"MultiFile reader using file $file") if (!keepReadsInOrder) { - val futureRunner = fcs.submit(getBatchRunner(tc, file.toRead, file.original, conf, filters)) + val futureRunner = fcs.submit(getBatchRunner(tc, file, conf, filters)) tasks.add(futureRunner) } else { // Add these in the order as we got them so that we can make sure // we process them in the same order as CPU would. val threadPool = MultiFileReaderThreadPool.getOrCreateThreadPool(numThreads) - tasks.add(threadPool.submit(getBatchRunner(tc, file.toRead, file.original, conf, filters))) + tasks.add(threadPool.submit(getBatchRunner(tc, file, conf, filters))) } } // queue up any left to add once others finish - for (i <- limit until files.length) { - val file = files(i) - tasksToRun.enqueue(getBatchRunner(tc, file.toRead, file.original, conf, filters)) + for (i <- limit until inputFiles.length) { + val file = inputFiles(i) + tasksToRun.enqueue(getBatchRunner(tc, file, conf, filters)) } isInitted = true - filesToRead = files.length + filesToRead = inputFiles.length } // Each format should implement combineHMBs and canUseCombine if they support combining @@ -441,7 +409,6 @@ abstract class MultiFileCloudPartitionReaderBase( * * @param tc task context to use * @param file file to be read - * @param origFile optional original unmodified file if replaced with Alluxio * @param conf the Configuration parameters * @param filters push down filters * @return Callable[HostMemoryBuffersWithMetaDataBase] @@ -449,7 +416,6 @@ abstract class MultiFileCloudPartitionReaderBase( def getBatchRunner( tc: TaskContext, file: PartitionedFile, - origFile: Option[PartitionedFile], conf: Configuration, filters: Array[Filter]): Callable[HostMemoryBuffersWithMetaDataBase] @@ -618,8 +584,7 @@ abstract class MultiFileCloudPartitionReaderBase( TrampolineUtil.incBytesRead(inputMetrics, fileBufsAndMeta.bytesRead) // this is combine mode so input file shouldn't be used at all but update to // what would be closest so we at least don't have same file as last batch - val inputFileToSet = - fileBufsAndMeta.origPartitionedFile.getOrElse(fileBufsAndMeta.partitionedFile) + val inputFileToSet = fileBufsAndMeta.partitionedFile InputFileUtils.setInputFileBlock( inputFileToSet.filePath.toString(), inputFileToSet.start, @@ -663,12 +628,7 @@ abstract class MultiFileCloudPartitionReaderBase( } TrampolineUtil.incBytesRead(inputMetrics, fileBufsAndMeta.bytesRead) - // if we replaced the path with Alluxio, set it to the original filesystem file - // since Alluxio replacement is supposed to be transparent to the user - // Note that combine mode would have fallen back to not use combine mode if - // the inputFile was required. - val inputFileToSet = - fileBufsAndMeta.origPartitionedFile.getOrElse(fileBufsAndMeta.partitionedFile) + val inputFileToSet = fileBufsAndMeta.partitionedFile InputFileUtils.setInputFileBlock( inputFileToSet.filePath.toString(), inputFileToSet.start, diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index ababb052f27..550f6bc5591 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2024, NVIDIA CORPORATION. + * Copyright (c) 2019-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -2150,7 +2150,6 @@ class MultiFileCloudOrcPartitionReader( override def getBatchRunner( tc: TaskContext, file: PartitionedFile, - origFile: Option[PartitionedFile], conf: Configuration, filters: Array[Filter]): Callable[HostMemoryBuffersWithMetaDataBase] = { new ReadBatchRunner(tc, file, conf, filters) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index 03eb48de6fb..f37c00b8f5a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2024, NVIDIA CORPORATION. + * Copyright (c) 2019-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -122,11 +122,11 @@ case class GpuParquetScan( logInfo("Using the original per file parquet reader") GpuParquetPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, dataSchema, readDataSchema, readPartitionSchema, pushedFilters, rapidsConf, metrics, - options.asScala.toMap, None) + options.asScala.toMap) } else { GpuParquetMultiFilePartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, dataSchema, readDataSchema, readPartitionSchema, pushedFilters, rapidsConf, metrics, - queryUsesInputFile, None) + queryUsesInputFile) } } @@ -1066,10 +1066,8 @@ case class GpuParquetMultiFilePartitionReaderFactory( filters: Array[Filter], @transient rapidsConf: RapidsConf, metrics: Map[String, GpuMetric], - queryUsesInputFile: Boolean, - alluxioPathReplacementMap: Option[Map[String, String]]) - extends MultiFilePartitionReaderFactoryBase(sqlConf, broadcastedConf, - rapidsConf, alluxioPathReplacementMap) { + queryUsesInputFile: Boolean) + extends MultiFilePartitionReaderFactoryBase(sqlConf, broadcastedConf, rapidsConf) { private val isCaseSensitive = sqlConf.caseSensitiveAnalysis private val debugDumpPrefix = rapidsConf.parquetDebugDumpPrefix @@ -1107,15 +1105,11 @@ case class GpuParquetMultiFilePartitionReaderFactory( "the deprecated one will be honored if both are set.") deprecatedVal }.getOrElse(rapidsConf.getMultithreadedReaderKeepOrder) - private val alluxioReplacementTaskTime = - AlluxioCfgUtils.enabledAlluxioReplacementAlgoTaskTime(rapidsConf) private val compressCfg = CpuCompressionConfig.forParquet(rapidsConf) // We can't use the coalescing files reader when InputFileName, InputFileBlockStart, // or InputFileBlockLength because we are combining all the files into a single buffer - // and we don't know which file is associated with each row. If this changes we need to - // make sure the Alluxio path replacement also handles setting the input file name to - // the non-Alluxio path like the multi-threaded reader does. + // and we don't know which file is associated with each row. override val canUseCoalesceFilesReader: Boolean = rapidsConf.isParquetCoalesceFileReadEnabled && !(queryUsesInputFile || ignoreCorruptFiles) @@ -1143,8 +1137,8 @@ case class GpuParquetMultiFilePartitionReaderFactory( targetBatchSizeBytes, maxGpuColumnSizeBytes, useChunkedReader, maxChunkedReaderMemoryUsageSizeBytes, compressCfg, metrics, partitionSchema, numThreads, maxNumFileProcessed, ignoreMissingFiles, - ignoreCorruptFiles, readUseFieldId, alluxioPathReplacementMap.getOrElse(Map.empty), - alluxioReplacementTaskTime, queryUsesInputFile, keepReadsInOrderFromConf, combineConf) + ignoreCorruptFiles, readUseFieldId, queryUsesInputFile, keepReadsInOrderFromConf, + combineConf) } private def filterBlocksForCoalescingReader( @@ -1209,18 +1203,8 @@ case class GpuParquetMultiFilePartitionReaderFactory( * @return coalescing reading PartitionReader */ override def buildBaseColumnarReaderForCoalescing( - origFiles: Array[PartitionedFile], + files: Array[PartitionedFile], conf: Configuration): PartitionReader[ColumnarBatch] = { - // update the file paths for Alluxio if needed, the coalescing reader doesn't support - // input_file_name so no need to track what the non Alluxio file name is - val files = if (alluxioReplacementTaskTime) { - AlluxioUtils.updateFilesTaskTimeIfAlluxio(origFiles, alluxioPathReplacementMap).map(_.toRead) - } else { - // Since coalescing reader isn't supported if input_file_name is used, so won't - // ever get here with that. So with convert time or no Alluxio just use the files as - // passed in. - origFiles - } val clippedBlocks = ArrayBuffer[ParquetSingleDataBlockMeta]() val startTime = System.nanoTime() val metaAndFilesArr = if (numFilesFilterParallel > 0) { @@ -1289,8 +1273,7 @@ case class GpuParquetPartitionReaderFactory( filters: Array[Filter], @transient rapidsConf: RapidsConf, metrics: Map[String, GpuMetric], - @transient params: Map[String, String], - alluxioPathReplacementMap: Option[Map[String, String]]) + @transient params: Map[String, String]) extends ShimFilePartitionReaderFactory(params) with Logging { private val isCaseSensitive = sqlConf.caseSensitiveAnalysis @@ -2336,8 +2319,6 @@ class MultiFileParquetPartitionReader( * @param ignoreMissingFiles Whether to ignore missing files * @param ignoreCorruptFiles Whether to ignore corrupt files * @param useFieldId Whether to use field id for column matching - * @param alluxioPathReplacementMap Map containing mapping of DFS scheme to Alluxio scheme - * @param alluxioReplacementTaskTime Whether the Alluxio replacement algorithm is set to task time * @param queryUsesInputFile Whether the query requires the input file name functionality * @param keepReadsInOrder Whether to require the files to be read in the same order as Spark. * Defaults to true for formats that don't explicitly handle this. @@ -2364,14 +2345,12 @@ class MultiFileCloudParquetPartitionReader( ignoreMissingFiles: Boolean, ignoreCorruptFiles: Boolean, useFieldId: Boolean, - alluxioPathReplacementMap: Map[String, String], - alluxioReplacementTaskTime: Boolean, queryUsesInputFile: Boolean, keepReadsInOrder: Boolean, combineConf: CombineConf) extends MultiFileCloudPartitionReaderBase(conf, files, numThreads, maxNumFileProcessed, null, execMetrics, maxReadBatchSizeRows, maxReadBatchSizeBytes, ignoreCorruptFiles, - alluxioPathReplacementMap, alluxioReplacementTaskTime, keepReadsInOrder, combineConf) + keepReadsInOrder, combineConf) with ParquetPartitionReaderBase { def checkIfNeedToSplit(current: HostMemoryBuffersWithMetaData, @@ -2489,7 +2468,6 @@ class MultiFileCloudParquetPartitionReader( combinedMeta.allPartValues.map(_._1).sum, Seq.empty) val newHmbMeta = HostMemoryBuffersWithMetaData( metaToUse.partitionedFile, - metaToUse.origPartitionedFile, // this doesn't matter since already read Array(newHmbBufferInfo), offset, metaToUse.dateRebaseMode, @@ -2589,7 +2567,6 @@ class MultiFileCloudParquetPartitionReader( if (combinedEmptyMeta.allEmpty) { val metaForEmpty = combinedEmptyMeta.metaForEmpty HostMemoryEmptyMetaData(metaForEmpty.partitionedFile, // just pick one since not used - metaForEmpty.origPartitionedFile, combinedEmptyMeta.emptyBufferSize, combinedEmptyMeta.emptyTotalBytesRead, metaForEmpty.dateRebaseMode, // these shouldn't matter since data is empty @@ -2608,7 +2585,6 @@ class MultiFileCloudParquetPartitionReader( private case class HostMemoryEmptyMetaData( override val partitionedFile: PartitionedFile, - override val origPartitionedFile: Option[PartitionedFile], bufferSize: Long, override val bytesRead: Long, dateRebaseMode: DateTimeRebaseMode, @@ -2625,7 +2601,6 @@ class MultiFileCloudParquetPartitionReader( case class HostMemoryBuffersWithMetaData( override val partitionedFile: PartitionedFile, - override val origPartitionedFile: Option[PartitionedFile], override val memBuffersAndSizes: Array[SingleHMBAndMeta], override val bytesRead: Long, dateRebaseMode: DateTimeRebaseMode, @@ -2638,7 +2613,6 @@ class MultiFileCloudParquetPartitionReader( private class ReadBatchRunner( file: PartitionedFile, - origPartitionedFile: Option[PartitionedFile], filterFunc: PartitionedFile => ParquetFileInfoWithBlockMeta, taskContext: TaskContext) extends Callable[HostMemoryBuffersWithMetaDataBase] with Logging { @@ -2659,7 +2633,7 @@ class MultiFileCloudParquetPartitionReader( } catch { case e: FileNotFoundException if ignoreMissingFiles => logWarning(s"Skipped missing file: ${file.filePath}", e) - HostMemoryEmptyMetaData(file, origPartitionedFile, 0, 0, + HostMemoryEmptyMetaData(file, 0, 0, DateTimeRebaseLegacy, DateTimeRebaseLegacy, hasInt96Timestamps = false, null, null, 0) // Throw FileNotFoundException even if `ignoreCorruptFiles` is true @@ -2667,7 +2641,7 @@ class MultiFileCloudParquetPartitionReader( case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles => logWarning( s"Skipped the rest of the content in the corrupted file: ${file.filePath}", e) - HostMemoryEmptyMetaData(file, origPartitionedFile, 0, 0, + HostMemoryEmptyMetaData(file, 0, 0, DateTimeRebaseLegacy, DateTimeRebaseLegacy, hasInt96Timestamps = false, null, null, 0) } finally { @@ -2688,7 +2662,7 @@ class MultiFileCloudParquetPartitionReader( if (fileBlockMeta.blocks.isEmpty) { val bytesRead = fileSystemBytesRead() - startingBytesRead // no blocks so return null buffer and size 0 - HostMemoryEmptyMetaData(file, origPartitionedFile, 0, bytesRead, + HostMemoryEmptyMetaData(file, 0, bytesRead, fileBlockMeta.dateRebaseMode, fileBlockMeta.timestampRebaseMode, fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema, 0) } else { @@ -2696,14 +2670,14 @@ class MultiFileCloudParquetPartitionReader( if (isDone) { val bytesRead = fileSystemBytesRead() - startingBytesRead // got close before finishing - HostMemoryEmptyMetaData(file, origPartitionedFile, 0, bytesRead, + HostMemoryEmptyMetaData(file, 0, bytesRead, fileBlockMeta.dateRebaseMode, fileBlockMeta.timestampRebaseMode, fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema, 0) } else { if (fileBlockMeta.schema.getFieldCount == 0) { val bytesRead = fileSystemBytesRead() - startingBytesRead val numRows = fileBlockMeta.blocks.map(_.getRowCount).sum.toInt - HostMemoryEmptyMetaData(file, origPartitionedFile, 0, bytesRead, + HostMemoryEmptyMetaData(file, 0, bytesRead, fileBlockMeta.dateRebaseMode, fileBlockMeta.timestampRebaseMode, fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema, numRows) @@ -2722,12 +2696,12 @@ class MultiFileCloudParquetPartitionReader( if (isDone) { // got close before finishing hostBuffers.foreach(_.hmb.safeClose()) - HostMemoryEmptyMetaData(file, origPartitionedFile, 0, bytesRead, + HostMemoryEmptyMetaData(file, 0, bytesRead, fileBlockMeta.dateRebaseMode, fileBlockMeta.timestampRebaseMode, fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema, 0) } else { - HostMemoryBuffersWithMetaData(file, origPartitionedFile, hostBuffers.toArray, + HostMemoryBuffersWithMetaData(file, hostBuffers.toArray, bytesRead, fileBlockMeta.dateRebaseMode, fileBlockMeta.timestampRebaseMode, fileBlockMeta.hasInt96Timestamps, fileBlockMeta.schema, fileBlockMeta.readSchema, None) @@ -2758,10 +2732,9 @@ class MultiFileCloudParquetPartitionReader( override def getBatchRunner( tc: TaskContext, file: PartitionedFile, - origFile: Option[PartitionedFile], conf: Configuration, filters: Array[Filter]): Callable[HostMemoryBuffersWithMetaDataBase] = { - new ReadBatchRunner(file, origFile, filterFunc, tc) + new ReadBatchRunner(file, filterFunc, tc) } /** diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala index 46b659c8fcb..ccc6e120d25 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -44,8 +44,7 @@ class GpuReadCSVFileFormat extends CSVFileFormat with GpuReadFileFormatWithMetri filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration, - metrics: Map[String, GpuMetric], - alluxioPathReplacementMap: Option[Map[String, String]] = None) + metrics: Map[String, GpuMetric]) : PartitionedFile => Iterator[InternalRow] = { val sqlConf = sparkSession.sessionState.conf val broadcastedHadoopConf = diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala index 9b384916ab5..09db30759c7 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2025, NVIDIA CORPORATION. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -63,8 +63,7 @@ trait GpuReadFileFormatWithMetrics extends FileFormat { filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration, - metrics: Map[String, GpuMetric], - alluxioPathReplacementMap: Option[Map[String, String]]) + metrics: Map[String, GpuMetric]) : PartitionedFile => Iterator[InternalRow] def isPerFileReadEnabled(conf: RapidsConf): Boolean diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala index 1792f36e09b..940d977c769 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -43,8 +43,7 @@ class GpuReadOrcFileFormat extends OrcFileFormat with GpuReadFileFormatWithMetri filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration, - metrics: Map[String, GpuMetric], - alluxioPathReplacementMap: Option[Map[String, String]] = None) + metrics: Map[String, GpuMetric]) : PartitionedFile => Iterator[InternalRow] = { val sqlConf = sparkSession.sessionState.conf val broadcastedHadoopConf = diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala index e9b8beba036..52f8b0e8d54 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -43,8 +43,7 @@ class GpuReadParquetFileFormat extends ParquetFileFormat with GpuReadFileFormatW filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration, - metrics: Map[String, GpuMetric], - alluxioPathReplacementMap: Option[Map[String, String]]) + metrics: Map[String, GpuMetric]) : PartitionedFile => Iterator[InternalRow] = { val sqlConf = sparkSession.sessionState.conf val broadcastedHadoopConf = @@ -58,8 +57,7 @@ class GpuReadParquetFileFormat extends ParquetFileFormat with GpuReadFileFormatW filters.toArray, new RapidsConf(sqlConf), metrics, - options, - alluxioPathReplacementMap) + options) PartitionReaderIterator.buildReader(factory) } @@ -78,8 +76,7 @@ class GpuReadParquetFileFormat extends ParquetFileFormat with GpuReadFileFormatW pushedFilters, fileScan.rapidsConf, fileScan.allMetrics, - fileScan.queryUsesInputFile, - fileScan.alluxioPathsMap) + fileScan.queryUsesInputFile) } } 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 eeeb930cf9a..6c2ef2a0a0d 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 @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2024, NVIDIA CORPORATION. + * Copyright (c) 2019-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -1994,108 +1994,6 @@ val SHUFFLE_COMPRESSION_LZ4_CHUNK_SIZE = conf("spark.rapids.shuffle.compression. .booleanConf .createWithDefault(false) - // ALLUXIO CONFIGS - val ALLUXIO_MASTER = conf("spark.rapids.alluxio.master") - .doc("The Alluxio master hostname. If not set, read Alluxio master URL from " + - "spark.rapids.alluxio.home locally. This config is useful when Alluxio master " + - "and Spark driver are not co-located.") - .startupOnly() - .stringConf - .createWithDefault("") - - val ALLUXIO_MASTER_PORT = conf("spark.rapids.alluxio.master.port") - .doc("The Alluxio master port. If not set, read Alluxio master port from " + - "spark.rapids.alluxio.home locally. This config is useful when Alluxio master " + - "and Spark driver are not co-located.") - .startupOnly() - .integerConf - .createWithDefault(19998) - - val ALLUXIO_HOME = conf("spark.rapids.alluxio.home") - .doc("The Alluxio installation home path or link to the installation home path. ") - .startupOnly() - .stringConf - .createWithDefault("/opt/alluxio") - - val ALLUXIO_PATHS_REPLACE = conf("spark.rapids.alluxio.pathsToReplace") - .doc("List of paths to be replaced with corresponding Alluxio scheme. " + - "E.g. when configure is set to " + - "\"s3://foo->alluxio://0.1.2.3:19998/foo,gs://bar->alluxio://0.1.2.3:19998/bar\", " + - "it means: " + - "\"s3://foo/a.csv\" will be replaced to \"alluxio://0.1.2.3:19998/foo/a.csv\" and " + - "\"gs://bar/b.csv\" will be replaced to \"alluxio://0.1.2.3:19998/bar/b.csv\". " + - "To use this config, you have to mount the buckets to Alluxio by yourself. " + - "If you set this config, spark.rapids.alluxio.automount.enabled won't be valid.") - .startupOnly() - .stringConf - .toSequence - .createOptional - - val ALLUXIO_AUTOMOUNT_ENABLED = conf("spark.rapids.alluxio.automount.enabled") - .doc("Enable the feature of auto mounting the cloud storage to Alluxio. " + - "It requires the Alluxio master is the same node of Spark driver node. " + - "The Alluxio master's host and port will be read from alluxio.master.hostname and " + - "alluxio.master.rpc.port(default: 19998) from ALLUXIO_HOME/conf/alluxio-site.properties, " + - "then replace a cloud path which matches spark.rapids.alluxio.bucket.regex like " + - "\"s3://bar/b.csv\" to \"alluxio://0.1.2.3:19998/bar/b.csv\", " + - "and the bucket \"s3://bar\" will be mounted to \"/bar\" in Alluxio automatically.") - .booleanConf - .createWithDefault(false) - - val ALLUXIO_BUCKET_REGEX = conf("spark.rapids.alluxio.bucket.regex") - .doc("A regex to decide which bucket should be auto-mounted to Alluxio. " + - "E.g. when setting as \"^s3://bucket.*\", " + - "the bucket which starts with \"s3://bucket\" will be mounted to Alluxio " + - "and the path \"s3://bucket-foo/a.csv\" will be replaced to " + - "\"alluxio://0.1.2.3:19998/bucket-foo/a.csv\". " + - "It's only valid when setting spark.rapids.alluxio.automount.enabled=true. " + - "The default value matches all the buckets in \"s3://\" or \"s3a://\" scheme.") - .stringConf - .createWithDefault("^s3a{0,1}://.*") - - val ALLUXIO_USER = conf("spark.rapids.alluxio.user") - .doc("Alluxio user is set on the Alluxio client, " + - "which is used to mount or get information. " + - "By default it should be the user that running the Alluxio processes. " + - "The default value is ubuntu.") - .stringConf - .createWithDefault("ubuntu") - - val ALLUXIO_REPLACEMENT_ALGO = conf("spark.rapids.alluxio.replacement.algo") - .doc("The algorithm used when replacing the UFS path with the Alluxio path. CONVERT_TIME " + - "and TASK_TIME are the valid options. CONVERT_TIME indicates that we do it " + - "when we convert it to a GPU file read, this has extra overhead of creating an entirely " + - "new file index, which requires listing the files and getting all new file info from " + - "Alluxio. TASK_TIME replaces the path as late as possible inside of the task. " + - "By waiting and replacing it at task time, it just replaces " + - "the path without fetching the file information again, this is faster " + - "but doesn't update locality information if that has a bit impact on performance.") - .stringConf - .checkValues(Set("CONVERT_TIME", "TASK_TIME")) - .createWithDefault("TASK_TIME") - - val ALLUXIO_LARGE_FILE_THRESHOLD = conf("spark.rapids.alluxio.large.file.threshold") - .doc("The threshold is used to identify whether average size of files is large " + - "when reading from S3. If reading large files from S3 and " + - "the disks used by Alluxio are slow, " + - "directly reading from S3 is better than reading caches from Alluxio, " + - "because S3 network bandwidth is faster than local disk. " + - "This improvement takes effect when spark.rapids.alluxio.slow.disk is enabled.") - .bytesConf(ByteUnit.BYTE) - .createWithDefault(64 * 1024 * 1024) // 64M - - val ALLUXIO_SLOW_DISK = conf("spark.rapids.alluxio.slow.disk") - .doc("Indicates whether the disks used by Alluxio are slow. " + - "If it's true and reading S3 large files, " + - "Rapids Accelerator reads from S3 directly instead of reading from Alluxio caches. " + - "Refer to spark.rapids.alluxio.large.file.threshold which defines a threshold that " + - "identifying whether files are large. " + - "Typically, it's slow disks if speed is less than 300M/second. " + - "If using convert time spark.rapids.alluxio.replacement.algo, " + - "this may not apply to all file types like Delta files") - .booleanConf - .createWithDefault(true) - // USER FACING DEBUG CONFIGS val SHUFFLE_COMPRESSION_MAX_BATCH_MEMORY = @@ -2294,8 +2192,7 @@ val SHUFFLE_COMPRESSION_LZ4_CHUNK_SIZE = conf("spark.rapids.shuffle.compression. "the files will be filtered in a multithreaded manner where each thread filters " + "the number of files set by this config. If this is set to zero the files are " + "filtered serially. This uses the same thread pool as the multithreaded reader, " + - s"see $MULTITHREAD_READ_NUM_THREADS. Note that filtering multithreaded " + - "is useful with Alluxio.") + s"see $MULTITHREAD_READ_NUM_THREADS.") .integerConf .createWithDefault(value = 0) @@ -3240,32 +3137,6 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val gpuWriteMemorySpeed: Double = get(OPTIMIZER_GPU_WRITE_SPEED) - lazy val getAlluxioHome: String = get(ALLUXIO_HOME) - - lazy val getAlluxioMaster: String = get(ALLUXIO_MASTER) - - lazy val getAlluxioMasterPort: Int = get(ALLUXIO_MASTER_PORT) - - lazy val getAlluxioPathsToReplace: Option[Seq[String]] = get(ALLUXIO_PATHS_REPLACE) - - lazy val getAlluxioAutoMountEnabled: Boolean = get(ALLUXIO_AUTOMOUNT_ENABLED) - - lazy val getAlluxioBucketRegex: String = get(ALLUXIO_BUCKET_REGEX) - - lazy val getAlluxioUser: String = get(ALLUXIO_USER) - - lazy val getAlluxioReplacementAlgo: String = get(ALLUXIO_REPLACEMENT_ALGO) - - lazy val isAlluxioReplacementAlgoConvertTime: Boolean = - get(ALLUXIO_REPLACEMENT_ALGO) == "CONVERT_TIME" - - lazy val isAlluxioReplacementAlgoTaskTime: Boolean = - get(ALLUXIO_REPLACEMENT_ALGO) == "TASK_TIME" - - lazy val getAlluxioLargeFileThreshold: Long = get(ALLUXIO_LARGE_FILE_THRESHOLD) - - lazy val enableAlluxioSlowDisk: Boolean = get(ALLUXIO_SLOW_DISK) - lazy val driverTimeZone: Option[String] = get(DRIVER_TIMEZONE) lazy val isRangeWindowByteEnabled: Boolean = get(ENABLE_RANGE_WINDOW_BYTES) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala index b7b7d63c3cb..68a68da2073 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -46,8 +46,7 @@ class GpuReadJsonFileFormat extends JsonFileFormat with GpuReadFileFormatWithMet filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration, - metrics: Map[String, GpuMetric], - alluxioPathReplacementMap: Option[Map[String, String]] = None) + metrics: Map[String, GpuMetric]) : PartitionedFile => Iterator[InternalRow] = { val sqlConf = sparkSession.sessionState.conf val broadcastedHadoopConf = diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/execution/datasources/rapids/GpuPartitioningUtils.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/execution/datasources/rapids/GpuPartitioningUtils.scala deleted file mode 100644 index 83ae5fb2afd..00000000000 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/execution/datasources/rapids/GpuPartitioningUtils.scala +++ /dev/null @@ -1,627 +0,0 @@ -/* - * Copyright (c) 2021-2024, 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.execution.datasources.rapids - -import java.lang.{Double => JDouble, Long => JLong} -import java.math.{BigDecimal => JBigDecimal} -import java.time.ZoneId -import java.util.Locale - -import scala.collection.mutable.ArrayBuffer -import scala.util.Try -import scala.util.control.NonFatal - -import com.nvidia.spark.rapids.AlluxioUtils.AlluxioPathReplaceConvertTime -import com.nvidia.spark.rapids.shims.FileIndexOptionsShims.BASE_PATH_PARAM -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper} -import org.apache.spark.sql.catalyst.analysis.TypeCoercion -import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.{unescapePathName, DEFAULT_PARTITION_NAME} -import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} -import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils, TimestampFormatter} -import org.apache.spark.sql.catalyst.util.rapids.DateFormatter -import org.apache.spark.sql.execution.datasources.{PartitionPath, PartitionSpec} -import org.apache.spark.sql.execution.datasources.PartitioningUtils.timestampPartitionPattern -import org.apache.spark.sql.types._ -import org.apache.spark.sql.types.shims.PartitionValueCastShims -import org.apache.spark.unsafe.types.UTF8String - -object GpuPartitioningUtils extends SQLConfHelper { - - case class TypedPartValue(value: String, dataType: DataType) - - case class PartitionValues(columnNames: Seq[String], typedValues: Seq[TypedPartValue]) - - /** - * - * @param sparkSession - * @param rootPaths the list of root input paths from which the catalog will get files - * @param leafFiles leaf file paths - * @param parameters a set of options to control partition discovery - * @param userSpecifiedSchema an optional user specified schema that will be use to provide - * types for the discovered partitions - * @param replaceFunc the alluxio replace function - * @return the specification of the partitions inferred from the data and if it was replaced the - * original path - * - * Mainly copied from PartitioningAwareFileIndex.inferPartitioning - */ - def inferPartitioning( - sparkSession: SparkSession, - rootPaths: Seq[Path], - leafFiles: Seq[Path], - parameters: Map[String, String], - userSpecifiedSchema: Option[StructType], - replaceFunc: Path => AlluxioPathReplaceConvertTime) - : (PartitionSpec, Option[String]) = { - - val recursiveFileLookup = parameters.getOrElse("recursiveFileLookup", "false").toBoolean - - if (recursiveFileLookup) { - (PartitionSpec.emptySpec, None) - } else { - val caseInsensitiveOptions = CaseInsensitiveMap(parameters) - val timeZoneId = caseInsensitiveOptions.get(DateTimeUtils.TIMEZONE_OPTION) - .getOrElse(sparkSession.sessionState.conf.sessionLocalTimeZone) - - // filter out non-data path and get unique leaf dirs of inputFiles - val leafDirs: Seq[Path] = leafFiles.filter(isDataPath).map(_.getParent).distinct - - val basePathAndAnyReplacedOption = parameters.get(BASE_PATH_PARAM).map { file => - // need to replace the base path - replaceFunc(new Path(file)) - } - val basePathOption = basePathAndAnyReplacedOption.map(_.filePath) - val anyReplacedBase = basePathAndAnyReplacedOption.flatMap(_.origPrefix) - - val basePaths = getBasePaths(sparkSession.sessionState.newHadoopConfWithOptions(parameters), - basePathOption, rootPaths, leafFiles) - - val parsed = parsePartitions( - leafDirs, - typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled, - basePaths = basePaths, - userSpecifiedSchema = userSpecifiedSchema, - caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis, - validatePartitionColumns = sparkSession.sessionState.conf.validatePartitionColumns, - timeZoneId = timeZoneId) - (parsed, anyReplacedBase) - } - } - - // SPARK-15895: Metadata files (e.g. Parquet summary files) and temporary files should not be - // counted as data files, so that they shouldn't participate partition discovery. - // Copied from PartitioningAwareFileIndex.isDataPath - def isDataPath(path: Path): Boolean = { - isDataPath(path.getName) - } - - def isDataPath(name: String): Boolean = { - !((name.startsWith("_") && !name.contains("=")) || name.startsWith(".")) - } - - /** - * Contains a set of paths that are considered as the base dirs of the input datasets. - * The partitioning discovery logic will make sure it will stop when it reaches any - * base path. - * - * By default, the paths of the dataset provided by users will be base paths. - * Below are three typical examples, - * Case 1) `spark.read.parquet("/path/something=true/")`: the base path will be - * `/path/something=true/`, and the returned DataFrame will not contain a column of `something`. - * Case 2) `spark.read.parquet("/path/something=true/a.parquet")`: the base path will be - * still `/path/something=true/`, and the returned DataFrame will also not contain a column of - * `something`. - * Case 3) `spark.read.parquet("/path/")`: the base path will be `/path/`, and the returned - * DataFrame will have the column of `something`. - * - * Users also can override the basePath by setting `basePath` in the options to pass the new base - * path to the data source. - * For example, `spark.read.option("basePath", "/path/").parquet("/path/something=true/")`, - * and the returned DataFrame will have the column of `something`. - * - * mainly copied from PartitioningAwareFileIndex.basePaths - */ - private def getBasePaths( - hadoopConf: Configuration, - basePathOption: Option[Path], - rootPaths: Seq[Path], - leafFiles: Seq[Path]): Set[Path] = { - basePathOption match { - case Some(userDefinedBasePath) => - val fs = userDefinedBasePath.getFileSystem(hadoopConf) - if (!fs.getFileStatus(userDefinedBasePath).isDirectory) { - throw new IllegalArgumentException(s"Option '$BASE_PATH_PARAM' must be a directory") - } - val qualifiedBasePath = fs.makeQualified(userDefinedBasePath) - val qualifiedBasePathStr = qualifiedBasePath.toString - rootPaths - .find(!fs.makeQualified(_).toString.startsWith(qualifiedBasePathStr)) - .foreach { rp => - throw new IllegalArgumentException( - s"Wrong basePath $userDefinedBasePath for the root path: $rp") - } - Set(qualifiedBasePath) - - case None => - rootPaths.map { path => - // Make the path qualified (consistent with listLeafFiles and bulkListLeafFiles). - val qualifiedPath = path.getFileSystem(hadoopConf).makeQualified(path) - if (leafFiles.contains(qualifiedPath)) qualifiedPath.getParent else qualifiedPath }.toSet - } - } - - // To fix issue: https://github.com/NVIDIA/spark-rapids/issues/6026 and - // https://issues.apache.org/jira/browse/SPARK-39012 - // Port latest Spark code into spark-rapids - def castPartValueToDesiredType( - desiredType: DataType, - value: String, - zoneId: ZoneId): Any = desiredType match { - case _ if value == DEFAULT_PARTITION_NAME => null - case NullType => null - case StringType => UTF8String.fromString(unescapePathName(value)) - case ByteType => Integer.parseInt(value).toByte - case ShortType => Integer.parseInt(value).toShort - case IntegerType => Integer.parseInt(value) - case LongType => JLong.parseLong(value) - case FloatType => JDouble.parseDouble(value).toFloat - case DoubleType => JDouble.parseDouble(value) - case _: DecimalType => Literal(new JBigDecimal(value)).value - case DateType => - Cast(Literal(value), DateType, Some(zoneId.getId)).eval() - case BinaryType => value.getBytes() - case BooleanType => value.toBoolean - case t if PartitionValueCastShims.isSupportedType(t) => - PartitionValueCastShims.castTo(t, value, zoneId) - case dt => throw new IllegalArgumentException(s"Unexpected type $dt") - } - - /** - * Given a group of qualified paths, tries to parse them and returns a partition specification. - * For example, given: - * {{{ - * hdfs://:/path/to/partition/a=1/b=hello/c=3.14 - * hdfs://:/path/to/partition/a=2/b=world/c=6.28 - * }}} - * it returns: - * {{{ - * PartitionSpec( - * partitionColumns = StructType( - * StructField(name = "a", dataType = IntegerType, nullable = true), - * StructField(name = "b", dataType = StringType, nullable = true), - * StructField(name = "c", dataType = DoubleType, nullable = true)), - * partitions = Seq( - * Partition( - * values = Row(1, "hello", 3.14), - * path = "hdfs://:/path/to/partition/a=1/b=hello/c=3.14"), - * Partition( - * values = Row(2, "world", 6.28), - * path = "hdfs://:/path/to/partition/a=2/b=world/c=6.28"))) - * }}} - */ - private[datasources] def parsePartitions( - paths: Seq[Path], - typeInference: Boolean, - basePaths: Set[Path], - userSpecifiedSchema: Option[StructType], - caseSensitive: Boolean, - validatePartitionColumns: Boolean, - timeZoneId: String): PartitionSpec = { - parsePartitions(paths, typeInference, basePaths, userSpecifiedSchema, caseSensitive, - validatePartitionColumns, DateTimeUtils.getZoneId(timeZoneId)) - } - - private[datasources] def parsePartitions( - paths: Seq[Path], - typeInference: Boolean, - basePaths: Set[Path], - userSpecifiedSchema: Option[StructType], - caseSensitive: Boolean, - validatePartitionColumns: Boolean, - zoneId: ZoneId): PartitionSpec = { - val userSpecifiedDataTypes = if (userSpecifiedSchema.isDefined) { - val nameToDataType = userSpecifiedSchema.get.fields.map(f => f.name -> f.dataType).toMap - if (!caseSensitive) { - CaseInsensitiveMap(nameToDataType) - } else { - nameToDataType - } - } else { - Map.empty[String, DataType] - } - - // SPARK-26990: use user specified field names if case insensitive. - val userSpecifiedNames = if (userSpecifiedSchema.isDefined && !caseSensitive) { - CaseInsensitiveMap(userSpecifiedSchema.get.fields.map(f => f.name -> f.name).toMap) - } else { - Map.empty[String, String] - } - - val dateFormatter = DateFormatter(DateFormatter.defaultPattern) - val timestampFormatter = TimestampFormatter( - timestampPartitionPattern, - zoneId, - isParsing = true) - // First, we need to parse every partition's path and see if we can find partition values. - val (partitionValues, optDiscoveredBasePaths) = paths.map { path => - parsePartition(path, typeInference, basePaths, userSpecifiedDataTypes, - validatePartitionColumns, zoneId, dateFormatter, timestampFormatter) - }.unzip - - // We create pairs of (path -> path's partition value) here - // If the corresponding partition value is None, the pair will be skipped - val pathsWithPartitionValues = paths.zip(partitionValues).flatMap(x => x._2.map(x._1 -> _)) - - if (pathsWithPartitionValues.isEmpty) { - // This dataset is not partitioned. - PartitionSpec.emptySpec - } else { - // This dataset is partitioned. We need to check whether all partitions have the same - // partition columns and resolve potential type conflicts. - - // Check if there is conflicting directory structure. - // For the paths such as: - // var paths = Seq( - // "hdfs://host:9000/invalidPath", - // "hdfs://host:9000/path/a=10/b=20", - // "hdfs://host:9000/path/a=10.5/b=hello") - // It will be recognised as conflicting directory structure: - // "hdfs://host:9000/invalidPath" - // "hdfs://host:9000/path" - // TODO: Selective case sensitivity. - val discoveredBasePaths = optDiscoveredBasePaths.flatten.map(_.toString.toLowerCase()) - assert( - discoveredBasePaths.distinct.size == 1, - "Conflicting directory structures detected. Suspicious paths:\b" + - discoveredBasePaths.distinct.mkString("\n\t", "\n\t", "\n\n") + - "If provided paths are partition directories, please set " + - "\"basePath\" in the options of the data source to specify the " + - "root directory of the table. If there are multiple root directories, " + - "please load them separately and then union them.") - - val resolvedPartitionValues = resolvePartitions(pathsWithPartitionValues, caseSensitive) - - // Creates the StructType which represents the partition columns. - val fields = { - val PartitionValues(columnNames, typedValues) = resolvedPartitionValues.head - columnNames.zip(typedValues).map { case (name, TypedPartValue(_, dataType)) => - // We always assume partition columns are nullable since we've no idea whether null values - // will be appended in the future. - val resultName = userSpecifiedNames.getOrElse(name, name) - val resultDataType = userSpecifiedDataTypes.getOrElse(name, dataType) - StructField(resultName, resultDataType, nullable = true) - } - } - - // Finally, we create `Partition`s based on paths and resolved partition values. - val partitions = resolvedPartitionValues.zip(pathsWithPartitionValues).map { - case (PartitionValues(columnNames, typedValues), (path, _)) => - val rowValues = columnNames.zip(typedValues).map { case (columnName, typedValue) => - try { - castPartValueToDesiredType(typedValue.dataType, typedValue.value, zoneId) - } catch { - case NonFatal(_) => - if (validatePartitionColumns) { - throw new RuntimeException(s"Failed to cast value `$typedValue.value` to " + - s"`$typedValue.dataType` for partition column `$columnName`") - } else null - } - } - PartitionPath(InternalRow.fromSeq(rowValues), path) - } - - PartitionSpec(StructType(fields), partitions) - } - } - - /** - * Parses a single partition, returns column names and values of each partition column, also - * the path when we stop partition discovery. For example, given: - * {{{ - * path = hdfs://:/path/to/partition/a=42/b=hello/c=3.14 - * }}} - * it returns the partition: - * {{{ - * PartitionValues( - * Seq("a", "b", "c"), - * Seq( - * Literal.create(42, IntegerType), - * Literal.create("hello", StringType), - * Literal.create(3.14, DoubleType))) - * }}} - * and the path when we stop the discovery is: - * {{{ - * hdfs://:/path/to/partition - * }}} - */ - private[datasources] def parsePartition( - path: Path, - typeInference: Boolean, - basePaths: Set[Path], - userSpecifiedDataTypes: Map[String, DataType], - validatePartitionColumns: Boolean, - zoneId: ZoneId, - dateFormatter: DateFormatter, - timestampFormatter: TimestampFormatter): (Option[PartitionValues], Option[Path]) = { - val columns = ArrayBuffer.empty[(String, TypedPartValue)] - // Old Hadoop versions don't have `Path.isRoot` - var finished = path.getParent == null - // currentPath is the current path that we will use to parse partition column value. - var currentPath: Path = path - - while (!finished) { - // Sometimes (e.g., when speculative task is enabled), temporary directories may be left - // uncleaned. Here we simply ignore them. - if (currentPath.getName.toLowerCase(Locale.ROOT) == "_temporary") { - return (None, None) - } - - if (basePaths.contains(currentPath)) { - // If the currentPath is one of base paths. We should stop. - finished = true - } else { - // Let's say currentPath is a path of "/table/a=1/", currentPath.getName will give us a=1. - // Once we get the string, we try to parse it and find the partition column and value. - val maybeColumn = - parsePartitionColumn(currentPath.getName, typeInference, userSpecifiedDataTypes, - zoneId, dateFormatter, timestampFormatter) - maybeColumn.foreach(columns += _) - - // Now, we determine if we should stop. - // When we hit any of the following cases, we will stop: - // - In this iteration, we could not parse the value of partition column and value, - // i.e. maybeColumn is None, and columns is not empty. At here we check if columns is - // empty to handle cases like /table/a=1/_temporary/something (we need to find a=1 in - // this case). - // - After we get the new currentPath, this new currentPath represent the top level dir - // i.e. currentPath.getParent == null. For the example of "/table/a=1/", - // the top level dir is "/table". - finished = - (maybeColumn.isEmpty && !columns.isEmpty) || currentPath.getParent == null - - if (!finished) { - // For the above example, currentPath will be "/table/". - currentPath = currentPath.getParent - } - } - } - - if (columns.isEmpty) { - (None, Some(path)) - } else { - val (columnNames, values) = columns.reverse.unzip - (Some(PartitionValues(columnNames.toSeq, values.toSeq)), Some(currentPath)) - } - } - - private def parsePartitionColumn( - columnSpec: String, - typeInference: Boolean, - userSpecifiedDataTypes: Map[String, DataType], - zoneId: ZoneId, - dateFormatter: DateFormatter, - timestampFormatter: TimestampFormatter): Option[(String, TypedPartValue)] = { - val equalSignIndex = columnSpec.indexOf('=') - if (equalSignIndex == -1) { - None - } else { - val columnName = unescapePathName(columnSpec.take(equalSignIndex)) - assert(columnName.nonEmpty, s"Empty partition column name in '$columnSpec'") - - val rawColumnValue = columnSpec.drop(equalSignIndex + 1) - assert(rawColumnValue.nonEmpty, s"Empty partition column value in '$columnSpec'") - - val dataType = if (userSpecifiedDataTypes.contains(columnName)) { - // SPARK-26188: if user provides corresponding column schema, get the column value without - // inference, and then cast it as user specified data type. - userSpecifiedDataTypes(columnName) - } else { - inferPartitionColumnValue( - rawColumnValue, - typeInference, - zoneId, - dateFormatter, - timestampFormatter) - } - Some(columnName -> TypedPartValue(rawColumnValue, dataType)) - } - } - - private[datasources] def listConflictingPartitionColumns( - pathWithPartitionValues: Seq[(Path, PartitionValues)]): String = { - val distinctPartColNames = pathWithPartitionValues.map(_._2.columnNames).distinct - - def groupByKey[K, V](seq: Seq[(K, V)]): Map[K, Iterable[V]] = - seq.groupBy { case (key, _) => key }.mapValues(_.map { case (_, value) => value }).toMap - - val partColNamesToPaths = groupByKey(pathWithPartitionValues.map { - case (path, partValues) => partValues.columnNames -> path - }) - - val distinctPartColLists = distinctPartColNames.map(_.mkString(", ")).zipWithIndex.map { - case (names, index) => - s"Partition column name list #$index: $names" - } - - // Lists out those non-leaf partition directories that also contain files - val suspiciousPaths = distinctPartColNames.sortBy(_.length).flatMap(partColNamesToPaths) - - s"Conflicting partition column names detected:\n" + - distinctPartColLists.mkString("\n\t", "\n\t", "\n\n") + - "For partitioned table directories, data files should only live in leaf directories.\n" + - "And directories at the same level should have the same partition column name.\n" + - "Please check the following directories for unexpected files or " + - "inconsistent partition column names:\n" + - suspiciousPaths.map("\t" + _).mkString("\n", "\n", "") - } - - // scalastyle:off line.size.limit - /** - * Converts a string to a [[Literal]] with automatic type inference. Currently only supports - * [[NullType]], [[IntegerType]], [[LongType]], [[DoubleType]], [[DecimalType]], [[DateType]] - * [[TimestampType]], and [[StringType]]. - * - * When resolving conflicts, it follows the table below: - * - * +--------------------+-------------------+-------------------+-------------------+--------------------+------------+---------------+---------------+------------+ - * | InputA \ InputB | NullType | IntegerType | LongType | DecimalType(38,0)* | DoubleType | DateType | TimestampType | StringType | - * +--------------------+-------------------+-------------------+-------------------+--------------------+------------+---------------+---------------+------------+ - * | NullType | NullType | IntegerType | LongType | DecimalType(38,0) | DoubleType | DateType | TimestampType | StringType | - * | IntegerType | IntegerType | IntegerType | LongType | DecimalType(38,0) | DoubleType | StringType | StringType | StringType | - * | LongType | LongType | LongType | LongType | DecimalType(38,0) | StringType | StringType | StringType | StringType | - * | DecimalType(38,0)* | DecimalType(38,0) | DecimalType(38,0) | DecimalType(38,0) | DecimalType(38,0) | StringType | StringType | StringType | StringType | - * | DoubleType | DoubleType | DoubleType | StringType | StringType | DoubleType | StringType | StringType | StringType | - * | DateType | DateType | StringType | StringType | StringType | StringType | DateType | TimestampType | StringType | - * | TimestampType | TimestampType | StringType | StringType | StringType | StringType | TimestampType | TimestampType | StringType | - * | StringType | StringType | StringType | StringType | StringType | StringType | StringType | StringType | StringType | - * +--------------------+-------------------+-------------------+-------------------+--------------------+------------+---------------+---------------+------------+ - * Note that, for DecimalType(38,0)*, the table above intentionally does not cover all other - * combinations of scales and precisions because currently we only infer decimal type like - * `BigInteger`/`BigInt`. For example, 1.1 is inferred as double type. - */ - // scalastyle:on line.size.limit - private[datasources] def inferPartitionColumnValue( - raw: String, - typeInference: Boolean, - zoneId: ZoneId, - dateFormatter: DateFormatter, - timestampFormatter: TimestampFormatter): DataType = { - val decimalTry = Try { - // `BigDecimal` conversion can fail when the `field` is not a form of number. - val bigDecimal = new JBigDecimal(raw) - // It reduces the cases for decimals by disallowing values having scale (e.g. `1.1`). - require(bigDecimal.scale <= 0) - // `DecimalType` conversion can fail when - // 1. The precision is bigger than 38. - // 2. scale is bigger than precision. - DecimalType.fromDecimal(Decimal(bigDecimal)) - } - - val dateTry = Try { - // try and parse the date, if no exception occurs this is a candidate to be resolved as - // DateType - dateFormatter.parse(raw) - // SPARK-23436: Casting the string to date may still return null if a bad Date is provided. - // This can happen since DateFormat.parse may not use the entire text of the given string: - // so if there are extra-characters after the date, it returns correctly. - // We need to check that we can cast the raw string since we later can use Cast to get - // the partition values with the right DataType (see - // org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex.inferPartitioning) - val dateValue = Cast(Literal(raw), DateType, Some(zoneId.getId)).eval() - // Disallow DateType if the cast returned null - require(dateValue != null) - DateType - } - - val timestampTry = Try { - val unescapedRaw = unescapePathName(raw) - // the inferred data type is consistent with the default timestamp type - val timestampType = TimestampType - // try and parse the date, if no exception occurs this is a candidate to be resolved as - // TimestampType or TimestampNTZType - timestampType match { - case TimestampType => timestampFormatter.parse(unescapedRaw) - // comment the NTZ type for very limited use case of "SPARK_TESTING" to reduce duplicated - // code copied from Spark - // case TimestampNTZType => timestampFormatter.parseWithoutTimeZone(unescapedRaw) - } - // SPARK-23436: see comment for date - val timestampValue = Cast(Literal(unescapedRaw), timestampType, Some(zoneId.getId)).eval() - // Disallow TimestampType if the cast returned null - require(timestampValue != null) - timestampType - } - - if (typeInference) { - // First tries integral types - Try({ Integer.parseInt(raw); IntegerType }) - .orElse(Try { JLong.parseLong(raw); LongType }) - .orElse(decimalTry) - // Then falls back to fractional types - .orElse(Try { JDouble.parseDouble(raw); DoubleType }) - // Then falls back to date/timestamp types - .orElse(timestampTry) - .orElse(dateTry) - // Then falls back to string - .getOrElse { - if (raw == DEFAULT_PARTITION_NAME) NullType else StringType - } - } else { - if (raw == DEFAULT_PARTITION_NAME) NullType else StringType - } - } - - /** - * Resolves possible type conflicts between partitions by up-casting "lower" types using - * [[findWiderTypeForPartitionColumn]]. - */ - def resolvePartitions( - pathsWithPartitionValues: Seq[(Path, PartitionValues)], - caseSensitive: Boolean): Seq[PartitionValues] = { - if (pathsWithPartitionValues.isEmpty) { - Seq.empty - } else { - val partColNames = if (caseSensitive) { - pathsWithPartitionValues.map(_._2.columnNames) - } else { - pathsWithPartitionValues.map(_._2.columnNames.map(_.toLowerCase())) - } - assert( - partColNames.distinct.size == 1, - listConflictingPartitionColumns(pathsWithPartitionValues)) - - // Resolves possible type conflicts for each column - val values = pathsWithPartitionValues.map(_._2) - val columnCount = values.head.columnNames.size - val resolvedValues = (0 until columnCount).map { i => - resolveTypeConflicts(values.map(_.typedValues(i))) - } - - // Fills resolved literals back to each partition - values.zipWithIndex.map { case (d, index) => - d.copy(typedValues = resolvedValues.map(_(index))) - } - } - } - - /** - * Given a collection of [[Literal]]s, resolves possible type conflicts by - * [[findWiderTypeForPartitionColumn]]. - */ - private def resolveTypeConflicts(typedValues: Seq[TypedPartValue]): Seq[TypedPartValue] = { - val dataTypes = typedValues.map(_.dataType) - val desiredType = dataTypes.reduce(findWiderTypeForPartitionColumn) - - typedValues.map(tv => tv.copy(dataType = desiredType)) - } - - /** - * Type widening rule for partition column types. It is similar to - * [[TypeCoercion.findWiderTypeForTwo]] but the main difference is that here we disallow - * precision loss when widening double/long and decimal, and fall back to string. - */ - private val findWiderTypeForPartitionColumn: (DataType, DataType) => DataType = { - case (DoubleType, _: DecimalType) | (_: DecimalType, DoubleType) => StringType - case (DoubleType, LongType) | (LongType, DoubleType) => StringType - case (t1, t2) => TypeCoercion.findWiderTypeForTwo(t1, t2).getOrElse(StringType) - } - -} diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala index 02153b5cb87..e81e2eb9821 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -700,7 +700,6 @@ class GpuMultiFileCloudAvroPartitionReader( override def getBatchRunner( tc: TaskContext, file: PartitionedFile, - origFile: Option[PartitionedFile], config: Configuration, filters: Array[Filter]): Callable[HostMemoryBuffersWithMetaDataBase] = new ReadBatchRunner(tc, file, config, filters) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala index 38306991db3..9f280e02c7b 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2024, NVIDIA CORPORATION. + * Copyright (c) 2020-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -63,7 +63,6 @@ import org.apache.spark.util.collection.BitSet * off in GpuTransitionOverrides if InputFileName, * InputFileBlockStart, or InputFileBlockLength are used * @param disableBucketedScan Disable bucketed scan based on physical query plan. - * @param alluxioPathsMap Map containing mapping of DFS scheme to Alluxio scheme */ case class GpuFileSourceScanExec( @transient relation: HadoopFsRelation, @@ -76,7 +75,6 @@ case class GpuFileSourceScanExec( tableIdentifier: Option[TableIdentifier], disableBucketedScan: Boolean = false, queryUsesInputFile: Boolean = false, - alluxioPathsMap: Option[Map[String, String]], requiredPartitionSchema: Option[StructType] = None)(@transient val rapidsConf: RapidsConf) extends GpuDataSourceScanExec with GpuExec { import GpuMetric._ @@ -92,12 +90,6 @@ case class GpuFileSourceScanExec( val readPartitionSchema = requiredPartitionSchema.getOrElse(relation.partitionSchema) - // this is set only when we either explicitly replaced a path for CONVERT_TIME - // or when TASK_TIME if one of the paths will be replaced. - // If reading large s3 files on a cluster with slower disks, - // should update this to None and read directly from s3 to get faster. - private var alluxioPathReplacementMap: Option[Map[String, String]] = alluxioPathsMap - @transient private val gpuFormat = relation.fileFormat match { case g: GpuReadFileFormatWithMetrics => g case f => throw new IllegalStateException(s"${f.getClass} is not a GPU format with metrics") @@ -135,31 +127,7 @@ case class GpuFileSourceScanExec( val startTime = System.nanoTime() val pds = relation.location.listFiles( partitionFilters.filterNot(isDynamicPruningFilter), dataFilters) - if (AlluxioCfgUtils.isAlluxioPathsToReplaceTaskTime(rapidsConf, relation.fileFormat)) { - // if should directly read from s3, should set `alluxioPathReplacementMap` as None - if (AlluxioUtils.shouldReadDirectlyFromS3(rapidsConf, pds)) { - alluxioPathReplacementMap = None - } else { - // this is not ideal, here we check to see if we will replace any paths, which is an - // extra iteration through paths - alluxioPathReplacementMap = AlluxioUtils.checkIfNeedsReplaced(rapidsConf, pds, - relation.sparkSession.sparkContext.hadoopConfiguration, - relation.sparkSession.conf) - } - } else if (AlluxioCfgUtils.isAlluxioAutoMountTaskTime(rapidsConf, relation.fileFormat)) { - // if should directly read from s3, should set `alluxioPathReplacementMap` as None - if (AlluxioUtils.shouldReadDirectlyFromS3(rapidsConf, pds)) { - alluxioPathReplacementMap = None - } else { - alluxioPathReplacementMap = AlluxioUtils.autoMountIfNeeded(rapidsConf, pds, - relation.sparkSession.sparkContext.hadoopConfiguration, - relation.sparkSession.conf) - } - } - - logDebug(s"File listing and possibly replace with Alluxio path " + - s"took: ${System.nanoTime() - startTime}") - + logDebug(s"File listing took: ${System.nanoTime() - startTime}") setFilesNumAndSizeMetric(pds, true) val timeTakenMs = NANOSECONDS.toMillis( (System.nanoTime() - startTime) + optimizerMetadataTimeNs) @@ -369,8 +337,7 @@ case class GpuFileSourceScanExec( options = relation.options, hadoopConf = relation.sparkSession.sessionState.newHadoopConfWithOptions(relation.options), - metrics = allMetrics, - alluxioPathReplacementMap) + metrics = allMetrics) Some(reader) } else { None @@ -645,8 +612,7 @@ case class GpuFileSourceScanExec( optionalNumCoalescedBuckets, QueryPlan.normalizePredicates(dataFilters, originalOutput), None, - queryUsesInputFile, - alluxioPathsMap = alluxioPathsMap)(rapidsConf) + queryUsesInputFile)(rapidsConf) } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala index d50b0b184ca..bfa17d1a8b4 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -47,8 +47,7 @@ class GpuReadAvroFileFormat extends AvroFileFormat with GpuReadFileFormatWithMet filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration, - metrics: Map[String, GpuMetric], - alluxioPathReplacementMap: Option[Map[String, String]] = None) + metrics: Map[String, GpuMetric]) : PartitionedFile => Iterator[InternalRow] = { val sqlConf = sparkSession.sessionState.conf val broadcastedHadoopConf = diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileIndexOptionsShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileIndexOptionsShims.scala deleted file mode 100644 index 4c39738e4d0..00000000000 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileIndexOptionsShims.scala +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Copyright (c) 2022-2024, 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. - */ - -/*** spark-rapids-shim-json-lines -{"spark": "320"} -{"spark": "321"} -{"spark": "321cdh"} -{"spark": "322"} -{"spark": "323"} -{"spark": "324"} -{"spark": "330"} -{"spark": "330cdh"} -{"spark": "330db"} -{"spark": "331"} -{"spark": "332"} -{"spark": "332cdh"} -{"spark": "333"} -{"spark": "334"} -spark-rapids-shim-json-lines ***/ -package com.nvidia.spark.rapids.shims - -import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex - -object FileIndexOptionsShims { - val BASE_PATH_PARAM = PartitioningAwareFileIndex.BASE_PATH_PARAM -} diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala index 4fc5c3efaf2..386b1209f76 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -45,7 +45,7 @@ import com.nvidia.spark.rapids._ import org.apache.spark.sql.catalyst.expressions.DynamicPruningExpression import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex} +import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.rapids.GpuFileSourceScanExec class FileSourceScanExecMeta(plan: FileSourceScanExec, @@ -95,46 +95,8 @@ class FileSourceScanExecMeta(plan: FileSourceScanExec, override def convertToGpu(): GpuExec = { val sparkSession = wrapped.relation.sparkSession val options = wrapped.relation.options - val (location, alluxioPathsToReplaceMap) = - if (AlluxioCfgUtils.enabledAlluxioReplacementAlgoConvertTime(conf)) { - val shouldReadFromS3 = wrapped.relation.location match { - // Only handle InMemoryFileIndex - // - // skip handle `MetadataLogFileIndex`, from the description of this class: - // it's about the files generated by the `FileStreamSink`. - // The streaming data source is not in our scope. - // - // For CatalogFileIndex and FileIndex of `delta` data source, - // need more investigation. - case inMemory: InMemoryFileIndex => - // List all the partitions to reduce overhead, pass in 2 empty filters. - // Subsequent process will do the right partition pruning. - // This operation is fast, because it lists files from the caches and the caches - // already exist in the `InMemoryFileIndex`. - val pds = inMemory.listFiles(Seq.empty, Seq.empty) - AlluxioUtils.shouldReadDirectlyFromS3(conf, pds) - case _ => - false - } - - if (!shouldReadFromS3) { - // it's convert time algorithm and some paths are not large tables - AlluxioUtils.replacePathIfNeeded( - conf, - wrapped.relation, - partitionFilters, - wrapped.dataFilters) - } else { - // convert time algorithm and read large files - (wrapped.relation.location, None) - } - } else { - // it's not convert time algorithm or read large files, do not replace - (wrapped.relation.location, None) - } - val newRelation = HadoopFsRelation( - location, + wrapped.relation.location, wrapped.relation.partitionSchema, wrapped.relation.dataSchema, wrapped.relation.bucketSpec, @@ -151,7 +113,6 @@ class FileSourceScanExecMeta(plan: FileSourceScanExec, wrapped.dataFilters, wrapped.tableIdentifier, wrapped.disableBucketedScan, - queryUsesInputFile = false, - alluxioPathsToReplaceMap)(conf) + queryUsesInputFile = false)(conf) } } diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/types/shims/PartitionValueCastShims.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/types/shims/PartitionValueCastShims.scala deleted file mode 100644 index 31b7461aea3..00000000000 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/types/shims/PartitionValueCastShims.scala +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Copyright (c) 2022-2024, 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. - */ - -/*** spark-rapids-shim-json-lines -{"spark": "320"} -{"spark": "321"} -{"spark": "321cdh"} -{"spark": "322"} -{"spark": "323"} -{"spark": "324"} -spark-rapids-shim-json-lines ***/ -package org.apache.spark.sql.types.shims - -import java.time.ZoneId - -import scala.util.Try - -import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.unescapePathName -import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} -import org.apache.spark.sql.types.{AnyTimestampType, DataType, DateType} - -object PartitionValueCastShims { - def isSupportedType(dt: DataType): Boolean = dt match { - // Timestamp types - case dt if AnyTimestampType.acceptsType(dt) => true - case _ => false - } - - // Only for TimestampType and TimestampNTZType - def castTo(desiredType: DataType, value: String, zoneId: ZoneId): Any = desiredType match { - // Copied from org/apache/spark/sql/execution/datasources/PartitionUtils.scala - case dt if AnyTimestampType.acceptsType(desiredType) => - Try { - Cast(Literal(unescapePathName(value)), dt, Some(zoneId.getId)).eval() - }.getOrElse { - Cast(Cast(Literal(value), DateType, Some(zoneId.getId)), dt).eval() - } - } -} diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/types/shims/PartitionValueCastShims.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/types/shims/PartitionValueCastShims.scala deleted file mode 100644 index e1e1247b759..00000000000 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/types/shims/PartitionValueCastShims.scala +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Copyright (c) 2022-2024, 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. - */ - -/*** spark-rapids-shim-json-lines -{"spark": "330"} -{"spark": "330cdh"} -{"spark": "330db"} -{"spark": "331"} -{"spark": "332"} -{"spark": "332cdh"} -{"spark": "332db"} -{"spark": "333"} -{"spark": "334"} -{"spark": "340"} -{"spark": "341"} -{"spark": "341db"} -{"spark": "342"} -{"spark": "343"} -{"spark": "344"} -{"spark": "350"} -{"spark": "350db143"} -{"spark": "351"} -{"spark": "352"} -{"spark": "353"} -{"spark": "400"} -spark-rapids-shim-json-lines ***/ -package org.apache.spark.sql.types.shims - -import java.time.ZoneId - -import scala.util.Try - -import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.unescapePathName -import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} -import org.apache.spark.sql.types.{AnsiIntervalType, AnyTimestampType, DataType, DateType} - -object PartitionValueCastShims { - def isSupportedType(dt: DataType): Boolean = dt match { - case dt if AnyTimestampType.acceptsType(dt) => true - case _: AnsiIntervalType => true - case _ => false - } - - // Only for AnsiIntervalType - def castTo(desiredType: DataType, value: String, zoneId: ZoneId): Any = desiredType match { - // Copied from org/apache/spark/sql/execution/datasources/PartitionUtils.scala - case dt if AnyTimestampType.acceptsType(desiredType) => - Try { - Cast(Literal(unescapePathName(value)), dt, Some(zoneId.getId)).eval() - }.getOrElse { - Cast(Cast(Literal(value), DateType, Some(zoneId.getId)), dt).eval() - } - case it: AnsiIntervalType => - Cast(Literal(unescapePathName(value)), it).eval() - } -} diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala index 2763af7fd56..64fdea89a34 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,7 +27,7 @@ import com.nvidia.spark.rapids._ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InMemoryFileIndex} +import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.rapids.GpuFileSourceScanExec @@ -134,46 +134,8 @@ class FileSourceScanExecMeta(plan: FileSourceScanExec, override def convertToGpu(): GpuExec = { val sparkSession = wrapped.relation.sparkSession val options = wrapped.relation.options - val (location, alluxioPathsToReplaceMap) = - if (AlluxioCfgUtils.enabledAlluxioReplacementAlgoConvertTime(conf)) { - val shouldReadFromS3 = wrapped.relation.location match { - // Only handle InMemoryFileIndex - // - // skip handle `MetadataLogFileIndex`, from the description of this class: - // it's about the files generated by the `FileStreamSink`. - // The streaming data source is not in our scope. - // - // For CatalogFileIndex and FileIndex of `delta` data source, - // need more investigation. - case inMemory: InMemoryFileIndex => - // List all the partitions to reduce overhead, pass in 2 empty filters. - // Subsequent process will do the right partition pruning. - // This operation is fast, because it lists files from the caches and the caches - // already exist in the `InMemoryFileIndex`. - val pds = inMemory.listFiles(Seq.empty, Seq.empty) - AlluxioUtils.shouldReadDirectlyFromS3(conf, pds) - case _ => - false - } - - if (!shouldReadFromS3) { - // it's convert time algorithm and some paths are not large tables - AlluxioUtils.replacePathIfNeeded( - conf, - wrapped.relation, - partitionFilters, - dataFilters) - } else { - // convert time algorithm and read large files - (wrapped.relation.location, None) - } - } else { - // it's not convert time algorithm or read large files, do not replace - (wrapped.relation.location, None) - } - val newRelation = HadoopFsRelation( - location, + wrapped.relation.location, wrapped.relation.partitionSchema, wrapped.relation.dataSchema, wrapped.relation.bucketSpec, @@ -191,7 +153,6 @@ class FileSourceScanExecMeta(plan: FileSourceScanExec, dataFilters, wrapped.tableIdentifier, wrapped.disableBucketedScan, - queryUsesInputFile = false, - alluxioPathsToReplaceMap)(conf) + queryUsesInputFile = false)(conf) } } diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/FileIndexOptionsShims.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/FileIndexOptionsShims.scala deleted file mode 100644 index 5cd4cafe7f0..00000000000 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/FileIndexOptionsShims.scala +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Copyright (c) 2022-2024, 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. - */ - -/*** spark-rapids-shim-json-lines -{"spark": "332db"} -{"spark": "340"} -{"spark": "341"} -{"spark": "341db"} -{"spark": "342"} -{"spark": "343"} -{"spark": "344"} -{"spark": "350"} -{"spark": "350db143"} -{"spark": "351"} -{"spark": "352"} -{"spark": "353"} -{"spark": "400"} -spark-rapids-shim-json-lines ***/ -package com.nvidia.spark.rapids.shims - -import org.apache.spark.sql.execution.datasources.FileIndexOptions - -object FileIndexOptionsShims { - val BASE_PATH_PARAM = FileIndexOptions.BASE_PATH_PARAM -} diff --git a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/AlluxioConfigReaderSuite.scala b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/AlluxioConfigReaderSuite.scala deleted file mode 100644 index ec8db0bafc2..00000000000 --- a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/AlluxioConfigReaderSuite.scala +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Copyright (c) 2022-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 com.nvidia.spark.rapids - -import java.io.File -import java.nio.charset.StandardCharsets -import java.nio.file.Files - -import org.scalatest.funsuite.AnyFunSuite - -class AlluxioConfigReaderSuite extends AnyFunSuite { - - test("testReadAlluxioMasterAndPort") { - val homeDir = Files.createTempDirectory("tmpAlluxioHomePrefix") - val confDir = new File(homeDir.toFile, "conf") - assert(confDir.mkdir()) - val f = new File(confDir, "alluxio-site.properties") - - try { - val content = - """ -alluxio.master.hostname=host1.com -alluxio.master.rpc.port=200 - """ - Files.write(f.toPath, content.getBytes(StandardCharsets.UTF_8)) - val (host, port) = new AlluxioConfigReader() - .readMasterAndPort(homeDir.toFile.getAbsolutePath) - assert(host.equals("host1.com")) - assert(port == "200") - } finally { - f.delete() - confDir.delete() - homeDir.toFile.delete() - } - } - - test("testReadAlluxioMasterAndPort, get default port") { - val homeDir = Files.createTempDirectory("tmpAlluxioHomePrefix") - val confDir = new File(homeDir.toFile, "conf") - assert(confDir.mkdir()) - val f = new File(confDir, "alluxio-site.properties") - - try { - val content = - """ -alluxio.master.hostname=host1.com - """ - Files.write(f.toPath, content.getBytes(StandardCharsets.UTF_8)) - val (host, port) = new AlluxioConfigReader() - .readMasterAndPort(homeDir.toFile.getAbsolutePath) - assert(host.equals("host1.com")) - assert(port == "19998") - } finally { - f.delete() - confDir.delete() - homeDir.toFile.delete() - } - } - - test("testReadAlluxioMasterAndPort, cfg does not specify master") { - val homeDir = Files.createTempDirectory("tmpAlluxioHomePrefix") - val confDir = new File(homeDir.toFile, "conf") - assert(confDir.mkdir()) - val f = new File(confDir, "alluxio-site.properties") - - try { - val content = - """ -xxx=yyy - """ - Files.write(f.toPath, content.getBytes(StandardCharsets.UTF_8)) - try { - new AlluxioConfigReader() - .readMasterAndPort(homeDir.toFile.getAbsolutePath) - assert(false) - } catch { - case e: RuntimeException => - assert(e.getMessage.contains("Can't find alluxio.master.hostname")) - } - } finally { - f.delete() - confDir.delete() - homeDir.toFile.delete() - } - } - - test("testReadAlluxioMasterAndPort, cfg file does not exist") { - val homeDir = Files.createTempDirectory("tmpAlluxioHomePrefix") - val confDir = new File(homeDir.toFile, "conf") - assert(confDir.mkdir()) - - try { - try { - new AlluxioConfigReader() - .readMasterAndPort(homeDir.toFile.getAbsolutePath) - assert(false) - } catch { - case e: RuntimeException => - assert(e.getMessage.contains("Alluxio config file not found in")) - } - } finally { - confDir.delete() - homeDir.toFile.delete() - } - } -} diff --git a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/AlluxioUtilsSuite.scala b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/AlluxioUtilsSuite.scala deleted file mode 100644 index 609778e8a0a..00000000000 --- a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/AlluxioUtilsSuite.scala +++ /dev/null @@ -1,242 +0,0 @@ -/* - * Copyright (c) 2022-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 com.nvidia.spark.rapids - -import scala.collection.mutable - -import com.nvidia.spark.rapids.shims.PartitionedFileUtilsShim -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, Path} -import org.mockito.Mockito._ -import org.scalatest.funsuite.AnyFunSuite -import org.scalatestplus.mockito.MockitoSugar.mock - -import org.apache.spark.sql.RuntimeConfig -import org.apache.spark.sql.execution.datasources.{PartitionDirectory, PartitionedFile} -import org.apache.spark.sql.internal.SQLConf - -class AlluxioMasterAndPortReaderMock(master: String, port: String) - extends AlluxioConfigReader { - override def readAlluxioMasterAndPort(conf: RapidsConf): (String, String) = (master, port) -} - -class AlluxioFSMock extends AlluxioFS { - private val mountPoints = mutable.Map[String, String]() - - /** - * Get S3 mount points by Alluxio client - * - * @return mount points map, key of map is Alluxio path, value of map is S3 path. - * E.g.: returns a map: {'/bucket_1': 's3://bucket_1'} - */ - override def getExistingMountPoints(): mutable.Map[String, String] = { - mountPoints - } - - /** - * Mount an S3 path to Alluxio - * - * @param alluxioPath Alluxio path - * @param s3Path S3 path - */ - override def mount(alluxioPath: String, s3Path: String): Unit = { - mountPoints(alluxioPath) = s3Path - } - - def getMountPoints(): mutable.Map[String, String] = { - mountPoints - } -} - -class AlluxioUtilsSuite extends AnyFunSuite { - - def setMockOnAlluxioUtils(): Unit = { - AlluxioUtils.setAlluxioFS(new AlluxioFSMock()) - AlluxioUtils.setAlluxioMasterAndPortReader( - new AlluxioMasterAndPortReaderMock("localhost", "19998")) - } - - test("updateFilesTaskTimeIfAlluxio") { - setMockOnAlluxioUtils() - AlluxioUtils.resetInitInfo() - val replaceMap = Map[String, String](("s3a://", "alluxio://localhost:19998/")) - val partitionedFiles = Array[PartitionedFile]( - PartitionedFileUtilsShim.newPartitionedFile(null, "s3a://bucket_1/a.file", 0, 0), - PartitionedFileUtilsShim.newPartitionedFile(null, "s3a://bucket_2/b.file", 0, 0), - PartitionedFileUtilsShim.newPartitionedFile(null, "myScheme://bucket_1/1.file", 0, 0) - ) - val replaced = AlluxioUtils.updateFilesTaskTimeIfAlluxio(partitionedFiles, Option(replaceMap)) - assert(replaced.size == 3) - assert(replaced(0).toRead.filePath.toString === "alluxio://localhost:19998/bucket_1/a.file") - assert(replaced(0).original.get.filePath.toString === "s3a://bucket_1/a.file") - assert(replaced(1).toRead.filePath.toString === "alluxio://localhost:19998/bucket_2/b.file") - assert(replaced(1).original.get.filePath.toString === "s3a://bucket_2/b.file") - assert(replaced(2).toRead.filePath.toString === "myScheme://bucket_1/1.file") - assert(replaced(2).original.isEmpty) - } - - test("updateFilesTaskTimeIfAlluxio, multiple replacing rules") { - setMockOnAlluxioUtils() - AlluxioUtils.resetInitInfo() - val replaceMap = Map[String, String]( - ("s3a://", "alluxio://localhost:19998/"), // the first rule - ("s3a://bucket_1", "alluxio://localhost:19998/") // should not specify this rule! - ) - val partitionedFiles = Array[PartitionedFile]( - PartitionedFileUtilsShim.newPartitionedFile(null, "s3a://bucket_1/a.file", 0, 0) - ) - try { - AlluxioUtils.updateFilesTaskTimeIfAlluxio(partitionedFiles, Option(replaceMap)) - assert(false) - } catch { - case e: IllegalArgumentException => - assert(e.getMessage.contains("same replacing rules")) - } - } - - test("checkIfNeedsReplaced for PathsToReplace map, true") { - setMockOnAlluxioUtils() - AlluxioUtils.resetInitInfo() - val sqlConf = new SQLConf() - sqlConf.setConfString("spark.rapids.alluxio.pathsToReplace", - "s3a://bucket_1->alluxio://0.1.2.3:19998/foo") - val rapidsConf = new RapidsConf(sqlConf) - val fs = new FileStatus(0, false, 1, 1024L, 0L, - new Path("s3a://bucket_1/a.parquet")) - val pds = Seq(PartitionDirectory(null, Array(fs))) - val configuration = new Configuration() - val runtimeConfig = mock[RuntimeConfig] - when(runtimeConfig.getOption("spark.hadoop.fs.s3a.access.key")).thenReturn(Some("access key")) - when(runtimeConfig.getOption("spark.hadoop.fs.s3a.secret.key")).thenReturn(Some("secret key")) - - assert(AlluxioUtils - .checkIfNeedsReplaced(rapidsConf, pds, configuration, runtimeConfig).isDefined) - } - - test("checkIfNeedsReplaced for PathsToReplace map, false") { - setMockOnAlluxioUtils() - AlluxioUtils.resetInitInfo() - val sqlConf = new SQLConf() - sqlConf.setConfString("spark.rapids.alluxio.pathsToReplace", - "s3a://bucket_1->alluxio://0.1.2.3:19998/foo") - val rapidsConf = new RapidsConf(sqlConf) - val fs = new FileStatus(0, false, 1, 1024L, 0L, - new Path("s3a://not_found/a.parquet")) - val pds = Seq(PartitionDirectory(null, Array(fs))) - val configuration = new Configuration() - val runtimeConfig = mock[RuntimeConfig] - when(runtimeConfig.getOption("spark.hadoop.fs.s3a.access.key")).thenReturn(Some("access key")) - when(runtimeConfig.getOption("spark.hadoop.fs.s3a.secret.key")).thenReturn(Some("secret key")) - - assert(AlluxioUtils.checkIfNeedsReplaced(rapidsConf, pds, configuration, runtimeConfig).isEmpty) - } - - test("checkIfNeedsReplaced for PathsToReplace map, exception") { - setMockOnAlluxioUtils() - AlluxioUtils.resetInitInfo() - val sqlConf = new SQLConf() - sqlConf.setConfString("spark.rapids.alluxio.pathsToReplace", - "s3a://bucket_1->alluxio://0.1.2.3:19998/dir1," + - "s3a://bucket_1/dir1->alluxio://4.4.4.4:19998/dir1" - ) - val rapidsConf = new RapidsConf(sqlConf) - val fs = new FileStatus(0, false, 1, 1024L, 0L, - new Path("s3a://bucket_1/dir1/a.parquet")) // matches 2 rules - val pds = Seq(PartitionDirectory(null, Array(fs))) - val configuration = new Configuration() - val runtimeConfig = mock[RuntimeConfig] - when(runtimeConfig.getOption("spark.hadoop.fs.s3a.access.key")).thenReturn(Some("access key")) - when(runtimeConfig.getOption("spark.hadoop.fs.s3a.secret.key")).thenReturn(Some("secret key")) - - try { - AlluxioUtils.checkIfNeedsReplaced(rapidsConf, pds, configuration, runtimeConfig).isEmpty - assert(false) - } catch { - case e: IllegalArgumentException => - assert(e.getMessage.contains("same replacing rules")) - } - } - - test("checkIfNeedsReplaced for PathsToReplace map, invalid setting") { - setMockOnAlluxioUtils() - AlluxioUtils.resetInitInfo() - val sqlConf = new SQLConf() - sqlConf.setConfString("spark.rapids.alluxio.pathsToReplace", - "s3a://bucket_1->alluxio://0.1.2.3:19998/->dir1" // contains 2 `->` - ) - val rapidsConf = new RapidsConf(sqlConf) - val fs = new FileStatus(0, false, 1, 1024L, 0L, - new Path("s3a://bucket_1/dir1/a.parquet")) // matches 2 rules - val pds = Seq(PartitionDirectory(null, Array(fs))) - val configuration = new Configuration() - val runtimeConfig = mock[RuntimeConfig] - when(runtimeConfig.getOption("spark.hadoop.fs.s3a.access.key")).thenReturn(Some("access key")) - when(runtimeConfig.getOption("spark.hadoop.fs.s3a.secret.key")).thenReturn(Some("secret key")) - - try { - AlluxioUtils.checkIfNeedsReplaced(rapidsConf, pds, configuration, runtimeConfig).isEmpty - assert(false) - } catch { - case e: IllegalArgumentException => - assert(e.getMessage.contains("Invalid setting")) - } - } - - test("autoMountIfNeeded, auto-mount is false") { - setMockOnAlluxioUtils() - AlluxioUtils.resetInitInfo() - val sqlConf = new SQLConf() - sqlConf.setConfString("spark.rapids.alluxio.automount.enabled", "false") - val rapidsConf = new RapidsConf(sqlConf) - val fs = new FileStatus(0, false, 1, 1024L, 0L, - new Path("s3a://bucket_1/a.parquet")) - val pds = Seq(PartitionDirectory(null, Array(fs))) - val configuration = new Configuration() - val runtimeConfig = mock[RuntimeConfig] - when(runtimeConfig.getOption("spark.hadoop.fs.s3a.access.key")).thenReturn(Some("access key")) - when(runtimeConfig.getOption("spark.hadoop.fs.s3a.secret.key")).thenReturn(Some("secret key")) - - assert(AlluxioUtils.autoMountIfNeeded(rapidsConf, pds, configuration, runtimeConfig).isEmpty) - } - - test("autoMountIfNeeded, auto-mount is true") { - setMockOnAlluxioUtils() - val alluxioFSMock = new AlluxioFSMock() - AlluxioUtils.setAlluxioFS(alluxioFSMock) - AlluxioUtils.resetInitInfo() - val sqlConf = new SQLConf() - sqlConf.setConfString("spark.rapids.alluxio.automount.enabled", "true") - val rapidsConf = new RapidsConf(sqlConf) - - val configuration = new Configuration() - val runtimeConfig = mock[RuntimeConfig] - when(runtimeConfig.getOption("spark.hadoop.fs.s3a.access.key")).thenReturn(Some("access key")) - when(runtimeConfig.getOption("spark.hadoop.fs.s3a.secret.key")).thenReturn(Some("secret key")) - - assert(alluxioFSMock.getMountPoints().isEmpty) - val fs = new FileStatus(0, false, 1, 1024L, 0L, - new Path("s3a://bucket_1/a.parquet")) - val pds = Seq(PartitionDirectory(null, Array(fs))) - assert(AlluxioUtils.autoMountIfNeeded(rapidsConf, pds, configuration, runtimeConfig).isDefined) - assert(alluxioFSMock.getMountPoints().contains("/bucket_1")) - - val fs2 = new FileStatus(0, false, 1, 1024L, 0L, - new Path("myScheme://bucket_2/a.parquet")) - val pds2 = Seq(PartitionDirectory(null, Array(fs2))) - assert(AlluxioUtils.autoMountIfNeeded(rapidsConf, pds2, configuration, runtimeConfig).isEmpty) - assert(alluxioFSMock.getMountPoints().size == 1) - } -} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuMultiFileReaderSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuMultiFileReaderSuite.scala index 21f364e085b..ae1b5a47011 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuMultiFileReaderSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuMultiFileReaderSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2023, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -58,7 +58,6 @@ class GpuMultiFileReaderSuite extends AnyFunSuite { override def getBatchRunner( tc: TaskContext, file: PartitionedFile, - origFile: Option[PartitionedFile], conf: Configuration, filters: Array[Filter]): Callable[HostMemoryBuffersWithMetaDataBase] = { () => null diff --git a/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala index e303c207f63..a2dcaa09e45 100644 --- a/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala +++ b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -90,11 +90,11 @@ class DynamicPruningSuite case BroadcastQueryStageExec(id, plan, _canonicalized) => val newPlan = replaceSubquery(plan) BroadcastQueryStageExec(id, newPlan, _canonicalized) - case g @ GpuFileSourceScanExec(r, o, rs, pf, obs, oncb, df, ti, dbs, quif, apm, rps) => + case g @ GpuFileSourceScanExec(r, o, rs, pf, obs, oncb, df, ti, dbs, quif, rps) => val newPartitionFilters = updatePartitionFilters(pf) val rc = g.rapidsConf GpuFileSourceScanExec(r, o, rs, newPartitionFilters, - obs, oncb, df, ti, dbs, quif, apm, rps)(rc) + obs, oncb, df, ti, dbs, quif, rps)(rc) case FileSourceScanExec(r, o, rs, pf, obs, oncb, df, ti, dbs) => val newPartitionFilters = updatePartitionFilters(pf) FileSourceScanExec(r, o, rs, newPartitionFilters, obs, oncb, df, ti, dbs) From 66c37e3281bab9ee082e905a40333cebcea0817b Mon Sep 17 00:00:00 2001 From: liyuan <84758614+nvliyuan@users.noreply.github.com> Date: Thu, 9 Jan 2025 03:09:05 +0800 Subject: [PATCH 37/47] [DOC] update release note to add spark 353 support [skip ci] (#11939) * update release note to add spark 353 support in v24.12 Signed-off-by: liyuan * Update docs/download.md Co-authored-by: Sameer Raheja --------- Signed-off-by: liyuan Co-authored-by: Sameer Raheja --- docs/download.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/download.md b/docs/download.md index e16c94e5b90..a44579cf400 100644 --- a/docs/download.md +++ b/docs/download.md @@ -42,8 +42,8 @@ The plugin is tested on the following architectures: 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, 3.3.4 - Apache Spark 3.4.0, 3.4.1, 3.4.2, 3.4.3 - Apache Spark 3.5.0, 3.5.1, 3.5.2 + Apache Spark 3.4.0, 3.4.1, 3.4.2, 3.4.3, 3.4.4 + Apache Spark 3.5.0, 3.5.1, 3.5.2, 3.5.3 Supported Databricks runtime versions for Azure and AWS: Databricks 11.3 ML LTS (GPU, Scala 2.12, Spark 3.3.0) From 6d888074f95cbc9e45e1af002361e9004b804be5 Mon Sep 17 00:00:00 2001 From: "Hongbin Ma (Mahone)" Date: Fri, 10 Jan 2025 08:20:10 +0800 Subject: [PATCH 38/47] make maxCpuBatchSize in GpuPartitioning configurable (#11929) This PR closes https://github.com/NVIDIA/spark-rapids/issues/11928 --------- Signed-off-by: Hongbin Ma (Mahone) --- .../nvidia/spark/rapids/GpuPartitioning.scala | 17 +++++++---------- .../com/nvidia/spark/rapids/RapidsConf.scala | 16 ++++++++++++++++ 2 files changed, 23 insertions(+), 10 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuPartitioning.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuPartitioning.scala index 4fbc612591b..4d691703b74 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuPartitioning.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuPartitioning.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2024, NVIDIA CORPORATION. + * Copyright (c) 2020-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -29,15 +29,12 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.GpuShuffleEnv import org.apache.spark.sql.vectorized.ColumnarBatch -object GpuPartitioning { - // The maximum size of an Array minus a bit for overhead for metadata - val MaxCpuBatchSize = 2147483639L - 2048L -} - trait GpuPartitioning extends Partitioning { - private[this] val (maxCompressionBatchSize, _useGPUShuffle, _useMultiThreadedShuffle) = { + private[this] val ( + maxCpuBatchSize, maxCompressionBatchSize, _useGPUShuffle, _useMultiThreadedShuffle) = { val rapidsConf = new RapidsConf(SQLConf.get) - (rapidsConf.shuffleCompressionMaxBatchMemory, + (rapidsConf.shuffleParitioningMaxCpuBatchSize, + rapidsConf.shuffleCompressionMaxBatchMemory, GpuShuffleEnv.useGPUShuffle(rapidsConf), GpuShuffleEnv.useMultiThreadedShuffle(rapidsConf)) } @@ -124,7 +121,7 @@ trait GpuPartitioning extends Partitioning { // This should be a temp work around. partitionColumns.foreach(_.getBase.getNullCount) val totalInputSize = GpuColumnVector.getTotalDeviceMemoryUsed(partitionColumns) - val mightNeedToSplit = totalInputSize > GpuPartitioning.MaxCpuBatchSize + val mightNeedToSplit = totalInputSize > maxCpuBatchSize // We have to wrap the NvtxWithMetrics over both copyToHostAsync and corresponding CudaSync, // because the copyToHostAsync calls above are not guaranteed to be asynchronous (e.g.: when @@ -164,7 +161,7 @@ trait GpuPartitioning extends Partitioning { case (batch, part) => val totalSize = SlicedGpuColumnVector.getTotalHostMemoryUsed(batch) val numOutputBatches = - math.ceil(totalSize.toDouble / GpuPartitioning.MaxCpuBatchSize).toInt + math.ceil(totalSize.toDouble / maxCpuBatchSize).toInt if (numOutputBatches > 1) { // For now we are going to slice it on number of rows instead of looking // at each row to try and decide. If we get in trouble we can probably 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 6c2ef2a0a0d..b4e78a17699 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 @@ -1976,6 +1976,20 @@ val SHUFFLE_COMPRESSION_LZ4_CHUNK_SIZE = conf("spark.rapids.shuffle.compression. .integerConf .createWithDefault(20) + val SHUFFLE_PARTITIONING_MAX_CPU_BATCH_SIZE = + conf("spark.rapids.shuffle.partitioning.maxCpuBatchSize") + .doc("The maximum size of a sliced batch output to the CPU side " + + "when GPU partitioning shuffle data. This can be used to limit the peak on-heap memory " + + "used by CPU to serialize the shuffle data, especially for skew data cases. " + + "The default value is maximum size of an Array minus 2k overhead (2147483639L - 2048L), " + + "user should only set a smaller value than default value to avoid subsequent failures.") + .internal() + .bytesConf(ByteUnit.BYTE) + .checkValue(v => v > 0 && v <= 2147483639L - 2048L, + s"maxCpuBatchSize must be positive and not exceed ${2147483639L - 2048L} bytes.") + // The maximum size of an Array minus a bit for overhead for metadata + .createWithDefault(2147483639L - 2048L) + val SHUFFLE_MULTITHREADED_READER_THREADS = conf("spark.rapids.shuffle.multiThreaded.reader.threads") .doc("The number of threads to use for reading shuffle blocks per executor in the " + @@ -3073,6 +3087,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val shuffleMultiThreadedReaderThreads: Int = get(SHUFFLE_MULTITHREADED_READER_THREADS) + lazy val shuffleParitioningMaxCpuBatchSize: Long = get(SHUFFLE_PARTITIONING_MAX_CPU_BATCH_SIZE) + lazy val shuffleKudoSerializerEnabled: Boolean = get(SHUFFLE_KUDO_SERIALIZER_ENABLED) def isUCXShuffleManagerMode: Boolean = From dfc957c917d04ddf9fbacfc1f9d0b618eb589117 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 9 Jan 2025 16:24:45 -0800 Subject: [PATCH 39/47] Add throttle time metrics for async write (#11936) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ![Screenshot 2025-01-07 at 2 38 25 PM](https://github.com/user-attachments/assets/799bc726-b5c4-4e64-b38e-db30f69d9b4b) The async write added in https://github.com/NVIDIA/spark-rapids/pull/11730 supports throttling based on the number in-flight bytes to write. Knowing how much time the query spends on waiting in the throttle would be useful to understand query performance. This PR adds these new metrics which are also seen in the screenshot above: - total throttle time: total throttle time in a task - avg/min/max throttle time per async write: average/min/max throttle time per async write request The new metrics are added as debug metrics. To do so, The `create*Metrics()` in `GpuExec` has been moved to `GpuMetric` so that they can be used in other places than `GpuExec`. To put related things together, `GpuMetric` and `MetricsLevel` have been moved to a new `GpuMetrics.scala` file. --------- Signed-off-by: Jihoon Son --- .../spark/rapids/delta/RapidsDeltaWrite.scala | 9 +- .../delta20x/GpuOptimisticTransaction.scala | 6 +- .../delta21x/GpuOptimisticTransaction.scala | 6 +- .../delta22x/GpuOptimisticTransaction.scala | 6 +- .../delta23x/GpuOptimisticTransaction.scala | 6 +- .../delta24x/GpuOptimisticTransaction.scala | 6 +- .../rapids/GpuOptimisticTransaction.scala | 6 +- .../rapids/GpuOptimisticTransaction.scala | 6 +- .../rapids/GpuOptimisticTransaction.scala | 6 +- .../rapids/GpuOptimisticTransaction.scala | 6 +- .../spark/rapids/ColumnarOutputWriter.scala | 15 +- .../rapids/GpuDataWritingCommandExec.scala | 8 +- .../com/nvidia/spark/rapids/GpuExec.scala | 268 +--------------- .../com/nvidia/spark/rapids/GpuMetrics.scala | 291 ++++++++++++++++++ .../spark/rapids/GpuParquetFileFormat.scala | 9 +- .../spark/rapids/GpuRunnableCommandExec.scala | 8 +- .../rapids/io/async/AsyncOutputStream.scala | 10 +- .../rapids/io/async/ThrottlingExecutor.scala | 36 ++- .../sql/hive/rapids/GpuHiveFileFormat.scala | 18 +- .../BasicColumnarWriteStatsTracker.scala | 39 ++- .../sql/rapids/GpuFileFormatDataWriter.scala | 6 +- .../spark/sql/rapids/GpuOrcFileFormat.scala | 10 +- .../sql/rapids/GpuWriteStatsTracker.scala | 55 +++- .../io/async/AsyncOutputStreamSuite.scala | 10 +- .../io/async/ThrottlingExecutorSuite.scala | 72 ++++- .../rapids/GpuFileFormatDataWriterSuite.scala | 92 +++--- 26 files changed, 607 insertions(+), 403 deletions(-) create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMetrics.scala diff --git a/delta-lake/common/src/main/scala/com/nvidia/spark/rapids/delta/RapidsDeltaWrite.scala b/delta-lake/common/src/main/scala/com/nvidia/spark/rapids/delta/RapidsDeltaWrite.scala index bbf0ca58d4e..75ade9cfd4e 100644 --- a/delta-lake/common/src/main/scala/com/nvidia/spark/rapids/delta/RapidsDeltaWrite.scala +++ b/delta-lake/common/src/main/scala/com/nvidia/spark/rapids/delta/RapidsDeltaWrite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode} import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy, UnaryExecNode} import org.apache.spark.sql.execution.datasources.v2.V2CommandExec -import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.rapids.GpuWriteJobStatsTracker import org.apache.spark.sql.vectorized.ColumnarBatch @@ -106,11 +105,11 @@ case class GpuRapidsDeltaWriteExec(child: SparkPlan) extends V2CommandExec with UnaryExecNode with GpuExec { override def output: Seq[Attribute] = child.output - lazy val basicMetrics: Map[String, SQLMetric] = GpuWriteJobStatsTracker.basicMetrics - lazy val taskMetrics: Map[String, SQLMetric] = GpuWriteJobStatsTracker.taskMetrics + lazy val basicMetrics: Map[String, GpuMetric] = GpuWriteJobStatsTracker.basicMetrics + lazy val taskMetrics: Map[String, GpuMetric] = GpuWriteJobStatsTracker.taskMetrics override lazy val allMetrics: Map[String, GpuMetric] = - GpuMetric.wrap(basicMetrics ++ taskMetrics) + basicMetrics ++ taskMetrics override def internalDoExecuteColumnar(): RDD[ColumnarBatch] = { // This is just a stub node for planning purposes and does not actually perform diff --git a/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala b/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala index ee0fc6f1cae..26021ac4b84 100644 --- a/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala * in the Delta Lake project at https://github.com/delta-io/delta. @@ -179,8 +179,8 @@ class GpuOptimisticTransaction val serializableHadoopConf = new SerializableConfiguration(hadoopConf) val basicWriteJobStatsTracker = new BasicColumnarWriteJobStatsTracker( serializableHadoopConf, - BasicWriteJobStatsTracker.metrics) - registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics) + GpuMetric.wrap(BasicWriteJobStatsTracker.metrics)) + registerSQLMetrics(spark, GpuMetric.unwrap(basicWriteJobStatsTracker.driverSideMetrics)) statsTrackers.append(basicWriteJobStatsTracker) gpuRapidsWrite.foreach { grw => val tracker = new GpuWriteJobStatsTracker(serializableHadoopConf, diff --git a/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala b/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala index 7433594aa73..0667fd7a3d3 100644 --- a/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala * in the Delta Lake project at https://github.com/delta-io/delta. @@ -179,8 +179,8 @@ class GpuOptimisticTransaction val serializableHadoopConf = new SerializableConfiguration(hadoopConf) val basicWriteJobStatsTracker = new BasicColumnarWriteJobStatsTracker( serializableHadoopConf, - BasicWriteJobStatsTracker.metrics) - registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics) + GpuMetric.wrap(BasicWriteJobStatsTracker.metrics)) + registerSQLMetrics(spark, GpuMetric.unwrap(basicWriteJobStatsTracker.driverSideMetrics)) statsTrackers.append(basicWriteJobStatsTracker) gpuRapidsWrite.foreach { grw => val tracker = new GpuWriteJobStatsTracker(serializableHadoopConf, diff --git a/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala b/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala index cb990ccc16d..2c13cb23872 100644 --- a/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala * in the Delta Lake project at https://github.com/delta-io/delta. @@ -201,8 +201,8 @@ class GpuOptimisticTransaction val serializableHadoopConf = new SerializableConfiguration(hadoopConf) val basicWriteJobStatsTracker = new BasicColumnarWriteJobStatsTracker( serializableHadoopConf, - BasicWriteJobStatsTracker.metrics) - registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics) + GpuMetric.wrap(BasicWriteJobStatsTracker.metrics)) + registerSQLMetrics(spark, GpuMetric.unwrap(basicWriteJobStatsTracker.driverSideMetrics)) statsTrackers.append(basicWriteJobStatsTracker) gpuRapidsWrite.foreach { grw => val tracker = new GpuWriteJobStatsTracker(serializableHadoopConf, diff --git a/delta-lake/delta-23x/src/main/scala/org/apache/spark/sql/delta/rapids/delta23x/GpuOptimisticTransaction.scala b/delta-lake/delta-23x/src/main/scala/org/apache/spark/sql/delta/rapids/delta23x/GpuOptimisticTransaction.scala index 905329a6b01..75cef27b02e 100644 --- a/delta-lake/delta-23x/src/main/scala/org/apache/spark/sql/delta/rapids/delta23x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-23x/src/main/scala/org/apache/spark/sql/delta/rapids/delta23x/GpuOptimisticTransaction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala * in the Delta Lake project at https://github.com/delta-io/delta. @@ -201,8 +201,8 @@ class GpuOptimisticTransaction val serializableHadoopConf = new SerializableConfiguration(hadoopConf) val basicWriteJobStatsTracker = new BasicColumnarWriteJobStatsTracker( serializableHadoopConf, - BasicWriteJobStatsTracker.metrics) - registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics) + GpuMetric.wrap(BasicWriteJobStatsTracker.metrics)) + registerSQLMetrics(spark, GpuMetric.unwrap(basicWriteJobStatsTracker.driverSideMetrics)) statsTrackers.append(basicWriteJobStatsTracker) gpuRapidsWrite.foreach { grw => val tracker = new GpuWriteJobStatsTracker(serializableHadoopConf, diff --git a/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala b/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala index fadc9bf99a8..2985b6334be 100644 --- a/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala * in the Delta Lake project at https://github.com/delta-io/delta. @@ -203,8 +203,8 @@ class GpuOptimisticTransaction val serializableHadoopConf = new SerializableConfiguration(hadoopConf) val basicWriteJobStatsTracker = new BasicColumnarWriteJobStatsTracker( serializableHadoopConf, - BasicWriteJobStatsTracker.metrics) - registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics) + GpuMetric.wrap(BasicWriteJobStatsTracker.metrics)) + registerSQLMetrics(spark, GpuMetric.unwrap(basicWriteJobStatsTracker.driverSideMetrics)) statsTrackers.append(basicWriteJobStatsTracker) gpuRapidsWrite.foreach { grw => val tracker = new GpuWriteJobStatsTracker(serializableHadoopConf, diff --git a/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala index f4debdea1c1..c2ef25e5e34 100644 --- a/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala * in the Delta Lake project at https://github.com/delta-io/delta. @@ -220,8 +220,8 @@ class GpuOptimisticTransaction( val serializableHadoopConf = new SerializableConfiguration(hadoopConf) val basicWriteJobStatsTracker = new BasicColumnarWriteJobStatsTracker( serializableHadoopConf, - BasicWriteJobStatsTracker.metrics) - registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics) + GpuMetric.wrap(BasicWriteJobStatsTracker.metrics)) + registerSQLMetrics(spark, GpuMetric.unwrap(basicWriteJobStatsTracker.driverSideMetrics)) statsTrackers.append(basicWriteJobStatsTracker) gpuRapidsWrite.foreach { grw => val tracker = new GpuWriteJobStatsTracker(serializableHadoopConf, diff --git a/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala index 48cebee6e41..d7de53453b0 100644 --- a/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala * in the Delta Lake project at https://github.com/delta-io/delta. @@ -217,8 +217,8 @@ class GpuOptimisticTransaction( val serializableHadoopConf = new SerializableConfiguration(hadoopConf) val basicWriteJobStatsTracker = new BasicColumnarWriteJobStatsTracker( serializableHadoopConf, - BasicWriteJobStatsTracker.metrics) - registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics) + GpuMetric.wrap(BasicWriteJobStatsTracker.metrics)) + registerSQLMetrics(spark, GpuMetric.unwrap(basicWriteJobStatsTracker.driverSideMetrics)) statsTrackers.append(basicWriteJobStatsTracker) gpuRapidsWrite.foreach { grw => val tracker = new GpuWriteJobStatsTracker(serializableHadoopConf, diff --git a/delta-lake/delta-spark341db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark341db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala index e1e5f28dec6..0685e67e96a 100644 --- a/delta-lake/delta-spark341db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-spark341db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala * in the Delta Lake project at https://github.com/delta-io/delta. @@ -219,8 +219,8 @@ class GpuOptimisticTransaction( val serializableHadoopConf = new SerializableConfiguration(hadoopConf) val basicWriteJobStatsTracker = new BasicColumnarWriteJobStatsTracker( serializableHadoopConf, - BasicWriteJobStatsTracker.metrics) - registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics) + GpuMetric.wrap(BasicWriteJobStatsTracker.metrics)) + registerSQLMetrics(spark, GpuMetric.unwrap(basicWriteJobStatsTracker.driverSideMetrics)) statsTrackers.append(basicWriteJobStatsTracker) gpuRapidsWrite.foreach { grw => val tracker = new GpuWriteJobStatsTracker(serializableHadoopConf, diff --git a/delta-lake/delta-spark350db143/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark350db143/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala index b49c5ebaa30..2f499fca822 100644 --- a/delta-lake/delta-spark350db143/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-spark350db143/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * This file was derived from OptimisticTransaction.scala and TransactionalWrite.scala * in the Delta Lake project at https://github.com/delta-io/delta. @@ -219,8 +219,8 @@ class GpuOptimisticTransaction( val serializableHadoopConf = new SerializableConfiguration(hadoopConf) val basicWriteJobStatsTracker = new BasicColumnarWriteJobStatsTracker( serializableHadoopConf, - BasicWriteJobStatsTracker.metrics) - registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics) + GpuMetric.wrap(BasicWriteJobStatsTracker.metrics)) + registerSQLMetrics(spark, GpuMetric.unwrap(basicWriteJobStatsTracker.driverSideMetrics)) statsTrackers.append(basicWriteJobStatsTracker) gpuRapidsWrite.foreach { grw => val tracker = new GpuWriteJobStatsTracker(serializableHadoopConf, diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala index 1b1965cfba0..259982c1061 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2024, NVIDIA CORPORATION. + * Copyright (c) 2019-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -62,6 +62,7 @@ abstract class ColumnarOutputWriterFactory extends Serializable { path: String, dataSchema: StructType, context: TaskAttemptContext, + statsTrackers: Seq[ColumnarWriteTaskStatsTracker], debugOutputPath: Option[String]): ColumnarOutputWriter } @@ -74,6 +75,7 @@ abstract class ColumnarOutputWriter(context: TaskAttemptContext, dataSchema: StructType, rangeName: String, includeRetry: Boolean, + statsTrackers: Seq[ColumnarWriteTaskStatsTracker], debugDumpPath: Option[String], holdGpuBetweenBatches: Boolean = false, useAsyncWrite: Boolean = false) extends HostBufferConsumer with Logging { @@ -128,7 +130,7 @@ abstract class ColumnarOutputWriter(context: TaskAttemptContext, protected def getOutputStream: OutputStream = { if (useAsyncWrite) { logWarning("Async output write enabled") - new AsyncOutputStream(() => openOutputStream(), trafficController) + new AsyncOutputStream(() => openOutputStream(), trafficController, statsTrackers) } else { openOutputStream() } @@ -156,8 +158,7 @@ abstract class ColumnarOutputWriter(context: TaskAttemptContext, private[this] def updateStatistics( writeStartTime: Long, - gpuTime: Long, - statsTrackers: Seq[ColumnarWriteTaskStatsTracker]): Unit = { + gpuTime: Long): Unit = { // Update statistics val writeTime = System.nanoTime - writeStartTime - gpuTime statsTrackers.foreach { @@ -183,9 +184,7 @@ abstract class ColumnarOutputWriter(context: TaskAttemptContext, * during the distributed filesystem transfer to allow other tasks to start/continue * GPU processing. */ - def writeSpillableAndClose( - spillableBatch: SpillableColumnarBatch, - statsTrackers: Seq[ColumnarWriteTaskStatsTracker]): Long = { + def writeSpillableAndClose(spillableBatch: SpillableColumnarBatch): Long = { val writeStartTime = System.nanoTime closeOnExcept(spillableBatch) { _ => val cb = withRetryNoSplit[ColumnarBatch] { @@ -222,7 +221,7 @@ abstract class ColumnarOutputWriter(context: TaskAttemptContext, } writeBufferedData() - updateStatistics(writeStartTime, gpuTime, statsTrackers) + updateStatistics(writeStartTime, gpuTime) spillableBatch.numRows() } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala index 019f9b2e6b0..1f31b973428 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2024, NVIDIA CORPORATION. + * Copyright (c) 2019-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -39,10 +39,10 @@ import org.apache.spark.util.SerializableConfiguration * An extension of `DataWritingCommand` that allows columnar execution. */ trait GpuDataWritingCommand extends DataWritingCommand with ShimUnaryCommand { - lazy val basicMetrics: Map[String, SQLMetric] = GpuWriteJobStatsTracker.basicMetrics - lazy val taskMetrics: Map[String, SQLMetric] = GpuWriteJobStatsTracker.taskMetrics + lazy val basicMetrics: Map[String, GpuMetric] = GpuWriteJobStatsTracker.basicMetrics + lazy val taskMetrics: Map[String, GpuMetric] = GpuWriteJobStatsTracker.taskMetrics - override lazy val metrics: Map[String, SQLMetric] = basicMetrics ++ taskMetrics + override lazy val metrics: Map[String, SQLMetric] = GpuMetric.unwrap(basicMetrics ++ taskMetrics) override final def run(sparkSession: SparkSession, child: SparkPlan): Seq[Row] = { Arm.withResource(runColumnar(sparkSession, child)) { batches => diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala index bc67366d347..b3fb81e046e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2024, NVIDIA CORPORATION. + * Copyright (c) 2019-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,17 +16,12 @@ package com.nvidia.spark.rapids -import scala.collection.immutable.TreeMap - -import ai.rapids.cudf.NvtxColor -import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.filecache.FileCacheConf import com.nvidia.spark.rapids.lore.{GpuLore, GpuLoreDumpRDD} import com.nvidia.spark.rapids.lore.GpuLore.{loreIdOf, LORE_DUMP_PATH_TAG, LORE_DUMP_RDD_TAG} import com.nvidia.spark.rapids.shims.SparkShimImpl import org.apache.hadoop.fs.Path -import org.apache.spark.internal.Logging import org.apache.spark.rapids.LocationPreservingMapPartitionsRDD import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession @@ -34,246 +29,10 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Exp import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.rapids.GpuTaskMetrics import org.apache.spark.sql.vectorized.ColumnarBatch -sealed class MetricsLevel(val num: Integer) extends Serializable { - def >=(other: MetricsLevel): Boolean = - num >= other.num -} - -object MetricsLevel { - def apply(str: String): MetricsLevel = str match { - case "ESSENTIAL" => GpuMetric.ESSENTIAL_LEVEL - case "MODERATE" => GpuMetric.MODERATE_LEVEL - case _ => GpuMetric.DEBUG_LEVEL - } -} - -object GpuMetric extends Logging { - // Metric names. - val BUFFER_TIME = "bufferTime" - val COPY_BUFFER_TIME = "copyBufferTime" - val GPU_DECODE_TIME = "gpuDecodeTime" - val NUM_INPUT_ROWS = "numInputRows" - val NUM_INPUT_BATCHES = "numInputBatches" - val NUM_OUTPUT_ROWS = "numOutputRows" - val NUM_OUTPUT_BATCHES = "numOutputBatches" - val PARTITION_SIZE = "partitionSize" - val NUM_PARTITIONS = "numPartitions" - val OP_TIME = "opTime" - val COLLECT_TIME = "collectTime" - val CONCAT_TIME = "concatTime" - val SORT_TIME = "sortTime" - val REPARTITION_TIME = "repartitionTime" - val AGG_TIME = "computeAggTime" - val JOIN_TIME = "joinTime" - val FILTER_TIME = "filterTime" - val BUILD_DATA_SIZE = "buildDataSize" - val BUILD_TIME = "buildTime" - val STREAM_TIME = "streamTime" - val NUM_TASKS_FALL_BACKED = "numTasksFallBacked" - val NUM_TASKS_REPARTITIONED = "numTasksRepartitioned" - val NUM_TASKS_SKIPPED_AGG = "numTasksSkippedAgg" - val READ_FS_TIME = "readFsTime" - val WRITE_BUFFER_TIME = "writeBufferTime" - val FILECACHE_FOOTER_HITS = "filecacheFooterHits" - val FILECACHE_FOOTER_HITS_SIZE = "filecacheFooterHitsSize" - val FILECACHE_FOOTER_MISSES = "filecacheFooterMisses" - val FILECACHE_FOOTER_MISSES_SIZE = "filecacheFooterMissesSize" - val FILECACHE_DATA_RANGE_HITS = "filecacheDataRangeHits" - val FILECACHE_DATA_RANGE_HITS_SIZE = "filecacheDataRangeHitsSize" - val FILECACHE_DATA_RANGE_MISSES = "filecacheDataRangeMisses" - val FILECACHE_DATA_RANGE_MISSES_SIZE = "filecacheDataRangeMissesSize" - val FILECACHE_FOOTER_READ_TIME = "filecacheFooterReadTime" - val FILECACHE_DATA_RANGE_READ_TIME = "filecacheDataRangeReadTime" - val DELETION_VECTOR_SCATTER_TIME = "deletionVectorScatterTime" - val DELETION_VECTOR_SIZE = "deletionVectorSize" - val CONCAT_HEADER_TIME = "concatHeaderTime" - val CONCAT_BUFFER_TIME = "concatBufferTime" - val COPY_TO_HOST_TIME = "d2hMemCopyTime" - - // Metric Descriptions. - val DESCRIPTION_BUFFER_TIME = "buffer time" - val DESCRIPTION_COPY_BUFFER_TIME = "copy buffer time" - val DESCRIPTION_GPU_DECODE_TIME = "GPU decode time" - val DESCRIPTION_NUM_INPUT_ROWS = "input rows" - val DESCRIPTION_NUM_INPUT_BATCHES = "input columnar batches" - val DESCRIPTION_NUM_OUTPUT_ROWS = "output rows" - val DESCRIPTION_NUM_OUTPUT_BATCHES = "output columnar batches" - val DESCRIPTION_PARTITION_SIZE = "partition data size" - val DESCRIPTION_NUM_PARTITIONS = "partitions" - val DESCRIPTION_OP_TIME = "op time" - val DESCRIPTION_COLLECT_TIME = "collect batch time" - val DESCRIPTION_CONCAT_TIME = "concat batch time" - val DESCRIPTION_SORT_TIME = "sort time" - val DESCRIPTION_REPARTITION_TIME = "repartition time" - val DESCRIPTION_AGG_TIME = "aggregation time" - val DESCRIPTION_JOIN_TIME = "join time" - val DESCRIPTION_FILTER_TIME = "filter time" - val DESCRIPTION_BUILD_DATA_SIZE = "build side size" - val DESCRIPTION_BUILD_TIME = "build time" - val DESCRIPTION_STREAM_TIME = "stream time" - val DESCRIPTION_NUM_TASKS_FALL_BACKED = "number of sort fallback tasks" - val DESCRIPTION_NUM_TASKS_REPARTITIONED = "number of tasks repartitioned for agg" - val DESCRIPTION_NUM_TASKS_SKIPPED_AGG = "number of tasks skipped aggregation" - val DESCRIPTION_READ_FS_TIME = "time to read fs data" - val DESCRIPTION_WRITE_BUFFER_TIME = "time to write data to buffer" - val DESCRIPTION_FILECACHE_FOOTER_HITS = "cached footer hits" - val DESCRIPTION_FILECACHE_FOOTER_HITS_SIZE = "cached footer hits size" - val DESCRIPTION_FILECACHE_FOOTER_MISSES = "cached footer misses" - val DESCRIPTION_FILECACHE_FOOTER_MISSES_SIZE = "cached footer misses size" - val DESCRIPTION_FILECACHE_DATA_RANGE_HITS = "cached data hits" - val DESCRIPTION_FILECACHE_DATA_RANGE_HITS_SIZE = "cached data hits size" - val DESCRIPTION_FILECACHE_DATA_RANGE_MISSES = "cached data misses" - val DESCRIPTION_FILECACHE_DATA_RANGE_MISSES_SIZE = "cached data misses size" - val DESCRIPTION_FILECACHE_FOOTER_READ_TIME = "cached footer read time" - val DESCRIPTION_FILECACHE_DATA_RANGE_READ_TIME = "cached data read time" - val DESCRIPTION_DELETION_VECTOR_SCATTER_TIME = "deletion vector scatter time" - val DESCRIPTION_DELETION_VECTOR_SIZE = "deletion vector size" - val DESCRIPTION_CONCAT_HEADER_TIME = "concat header time" - val DESCRIPTION_CONCAT_BUFFER_TIME = "concat buffer time" - val DESCRIPTION_COPY_TO_HOST_TIME = "deviceToHost memory copy time" - - def unwrap(input: GpuMetric): SQLMetric = input match { - case w :WrappedGpuMetric => w.sqlMetric - case i => throw new IllegalArgumentException(s"found unsupported GpuMetric ${i.getClass}") - } - - def unwrap(input: Map[String, GpuMetric]): Map[String, SQLMetric] = { - val ret = input.collect { - // remove the metrics that are not registered - case (k, w) if w != NoopMetric => (k, unwrap(w)) - } - val companions = input.collect { - // add the companions - case (k, w) if w.companionGpuMetric.isDefined => - (k + "_exSemWait", unwrap(w.companionGpuMetric.get)) - } - - TreeMap.apply((ret ++ companions).toSeq: _*) - } - - def wrap(input: SQLMetric): GpuMetric = WrappedGpuMetric(input) - - def wrap(input: Map[String, SQLMetric]): Map[String, GpuMetric] = input.map { - case (k, v) => (k, wrap(v)) - } - - def ns[T](metrics: GpuMetric*)(f: => T): T = { - val start = System.nanoTime() - try { - f - } finally { - val taken = System.nanoTime() - start - metrics.foreach(_.add(taken)) - } - } - - object DEBUG_LEVEL extends MetricsLevel(0) - object MODERATE_LEVEL extends MetricsLevel(1) - object ESSENTIAL_LEVEL extends MetricsLevel(2) -} - -sealed abstract class GpuMetric extends Serializable { - def value: Long - def set(v: Long): Unit - def +=(v: Long): Unit - def add(v: Long): Unit - - private var isTimerActive = false - - // For timing GpuMetrics, we additionally create a companion GpuMetric to track elapsed time - // excluding semaphore wait time - var companionGpuMetric: Option[GpuMetric] = None - private var semWaitTimeWhenActivated = 0L - - final def tryActivateTimer(): Boolean = { - if (!isTimerActive) { - isTimerActive = true - semWaitTimeWhenActivated = GpuTaskMetrics.get.getSemWaitTime() - true - } else { - false - } - } - - final def deactivateTimer(duration: Long): Unit = { - if (isTimerActive) { - isTimerActive = false - companionGpuMetric.foreach(c => - c.add(duration - (GpuTaskMetrics.get.getSemWaitTime() - semWaitTimeWhenActivated))) - semWaitTimeWhenActivated = 0L - add(duration) - } - } - - final def ns[T](f: => T): T = { - if (tryActivateTimer()) { - val start = System.nanoTime() - try { - f - } finally { - deactivateTimer(System.nanoTime() - start) - } - } else { - f - } - } -} - -object NoopMetric extends GpuMetric { - override def +=(v: Long): Unit = () - override def add(v: Long): Unit = () - override def set(v: Long): Unit = () - override def value: Long = 0 -} - -final case class WrappedGpuMetric(sqlMetric: SQLMetric, withMetricsExclSemWait: Boolean = false) - extends GpuMetric { - - if (withMetricsExclSemWait) { - // SQLMetrics.NS_TIMING_METRIC and SQLMetrics.TIMING_METRIC is private, - // so we have to use the string directly - if (sqlMetric.metricType == "nsTiming") { - companionGpuMetric = Some(WrappedGpuMetric.apply(SQLMetrics.createNanoTimingMetric( - SparkSession.getActiveSession.get.sparkContext, sqlMetric.name.get + " (excl. SemWait)"))) - } - } - - def +=(v: Long): Unit = sqlMetric.add(v) - def add(v: Long): Unit = sqlMetric.add(v) - override def set(v: Long): Unit = sqlMetric.set(v) - override def value: Long = sqlMetric.value -} - -/** A GPU metric class that just accumulates into a variable without implicit publishing. */ -final class LocalGpuMetric extends GpuMetric { - private var lval = 0L - override def value: Long = lval - override def set(v: Long): Unit = { lval = v } - override def +=(v: Long): Unit = { lval += v } - override def add(v: Long): Unit = { lval += v } -} - -class CollectTimeIterator[T]( - nvtxName: String, - it: Iterator[T], - collectTime: GpuMetric) extends Iterator[T] { - override def hasNext: Boolean = { - withResource(new NvtxWithMetrics(nvtxName, NvtxColor.BLUE, collectTime)) { _ => - it.hasNext - } - } - - override def next(): T = { - withResource(new NvtxWithMetrics(nvtxName, NvtxColor.BLUE, collectTime)) { _ => - it.next - } - } -} - object GpuExec { def outputBatching(sp: SparkPlan): CoalesceGoal = sp match { case gpu: GpuExec => gpu.outputBatching @@ -285,6 +44,7 @@ object GpuExec { trait GpuExec extends SparkPlan { import GpuMetric._ + def sparkSession: SparkSession = { SparkShimImpl.sessionFromPlan(this) } @@ -318,31 +78,23 @@ trait GpuExec extends SparkPlan { */ def outputBatching: CoalesceGoal = null - private [this] lazy val metricsConf = MetricsLevel(RapidsConf.METRICS_LEVEL.get(conf)) - - private [this] def createMetricInternal(level: MetricsLevel, f: => SQLMetric): GpuMetric = { - if (level >= metricsConf) { - // only enable companion metrics (excluding semaphore wait time) for DEBUG_LEVEL - WrappedGpuMetric(f, withMetricsExclSemWait = GpuMetric.DEBUG_LEVEL >= metricsConf) - } else { - NoopMetric - } - } + @transient private [this] lazy val metricFactory = new GpuMetricFactory( + MetricsLevel(RapidsConf.METRICS_LEVEL.get(conf)), sparkContext) def createMetric(level: MetricsLevel, name: String): GpuMetric = - createMetricInternal(level, SQLMetrics.createMetric(sparkContext, name)) + metricFactory.create(level, name) def createNanoTimingMetric(level: MetricsLevel, name: String): GpuMetric = - createMetricInternal(level, SQLMetrics.createNanoTimingMetric(sparkContext, name)) + metricFactory.createNanoTiming(level, name) def createSizeMetric(level: MetricsLevel, name: String): GpuMetric = - createMetricInternal(level, SQLMetrics.createSizeMetric(sparkContext, name)) + metricFactory.createSize(level, name) def createAverageMetric(level: MetricsLevel, name: String): GpuMetric = - createMetricInternal(level, SQLMetrics.createAverageMetric(sparkContext, name)) + metricFactory.createAverage(level, name) def createTimingMetric(level: MetricsLevel, name: String): GpuMetric = - createMetricInternal(level, SQLMetrics.createTimingMetric(sparkContext, name)) + metricFactory.createTiming(level, name) protected def createFileCacheMetrics(): Map[String, GpuMetric] = { if (FileCacheConf.FILECACHE_ENABLED.get(conf)) { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMetrics.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMetrics.scala new file mode 100644 index 00000000000..508d474c50b --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuMetrics.scala @@ -0,0 +1,291 @@ +/* + * Copyright (c) 2025, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids + +import scala.collection.immutable.TreeMap + +import ai.rapids.cudf.NvtxColor +import com.nvidia.spark.rapids.Arm.withResource + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.rapids.GpuTaskMetrics + +sealed class MetricsLevel(val num: Integer) extends Serializable { + def >=(other: MetricsLevel): Boolean = + num >= other.num +} + +object MetricsLevel { + def apply(str: String): MetricsLevel = str match { + case "ESSENTIAL" => GpuMetric.ESSENTIAL_LEVEL + case "MODERATE" => GpuMetric.MODERATE_LEVEL + case _ => GpuMetric.DEBUG_LEVEL + } +} + +class GpuMetricFactory(metricsConf: MetricsLevel, context: SparkContext) { + + private [this] def createInternal(level: MetricsLevel, f: => SQLMetric): GpuMetric = { + if (level >= metricsConf) { + // only enable companion metrics (excluding semaphore wait time) for DEBUG_LEVEL + WrappedGpuMetric(f, withMetricsExclSemWait = GpuMetric.DEBUG_LEVEL >= metricsConf) + } else { + NoopMetric + } + } + + def create(level: MetricsLevel, name: String): GpuMetric = + createInternal(level, SQLMetrics.createMetric(context, name)) + + def createNanoTiming(level: MetricsLevel, name: String): GpuMetric = + createInternal(level, SQLMetrics.createNanoTimingMetric(context, name)) + + def createSize(level: MetricsLevel, name: String): GpuMetric = + createInternal(level, SQLMetrics.createSizeMetric(context, name)) + + def createAverage(level: MetricsLevel, name: String): GpuMetric = + createInternal(level, SQLMetrics.createAverageMetric(context, name)) + + def createTiming(level: MetricsLevel, name: String): GpuMetric = + createInternal(level, SQLMetrics.createTimingMetric(context, name)) +} + +object GpuMetric extends Logging { + // Metric names. + val BUFFER_TIME = "bufferTime" + val COPY_BUFFER_TIME = "copyBufferTime" + val GPU_DECODE_TIME = "gpuDecodeTime" + val NUM_INPUT_ROWS = "numInputRows" + val NUM_INPUT_BATCHES = "numInputBatches" + val NUM_OUTPUT_ROWS = "numOutputRows" + val NUM_OUTPUT_BATCHES = "numOutputBatches" + val PARTITION_SIZE = "partitionSize" + val NUM_PARTITIONS = "numPartitions" + val OP_TIME = "opTime" + val COLLECT_TIME = "collectTime" + val CONCAT_TIME = "concatTime" + val SORT_TIME = "sortTime" + val REPARTITION_TIME = "repartitionTime" + val AGG_TIME = "computeAggTime" + val JOIN_TIME = "joinTime" + val FILTER_TIME = "filterTime" + val BUILD_DATA_SIZE = "buildDataSize" + val BUILD_TIME = "buildTime" + val STREAM_TIME = "streamTime" + val NUM_TASKS_FALL_BACKED = "numTasksFallBacked" + val NUM_TASKS_REPARTITIONED = "numTasksRepartitioned" + val NUM_TASKS_SKIPPED_AGG = "numTasksSkippedAgg" + val READ_FS_TIME = "readFsTime" + val WRITE_BUFFER_TIME = "writeBufferTime" + val FILECACHE_FOOTER_HITS = "filecacheFooterHits" + val FILECACHE_FOOTER_HITS_SIZE = "filecacheFooterHitsSize" + val FILECACHE_FOOTER_MISSES = "filecacheFooterMisses" + val FILECACHE_FOOTER_MISSES_SIZE = "filecacheFooterMissesSize" + val FILECACHE_DATA_RANGE_HITS = "filecacheDataRangeHits" + val FILECACHE_DATA_RANGE_HITS_SIZE = "filecacheDataRangeHitsSize" + val FILECACHE_DATA_RANGE_MISSES = "filecacheDataRangeMisses" + val FILECACHE_DATA_RANGE_MISSES_SIZE = "filecacheDataRangeMissesSize" + val FILECACHE_FOOTER_READ_TIME = "filecacheFooterReadTime" + val FILECACHE_DATA_RANGE_READ_TIME = "filecacheDataRangeReadTime" + val DELETION_VECTOR_SCATTER_TIME = "deletionVectorScatterTime" + val DELETION_VECTOR_SIZE = "deletionVectorSize" + val CONCAT_HEADER_TIME = "concatHeaderTime" + val CONCAT_BUFFER_TIME = "concatBufferTime" + val COPY_TO_HOST_TIME = "d2hMemCopyTime" + + // Metric Descriptions. + val DESCRIPTION_BUFFER_TIME = "buffer time" + val DESCRIPTION_COPY_BUFFER_TIME = "copy buffer time" + val DESCRIPTION_GPU_DECODE_TIME = "GPU decode time" + val DESCRIPTION_NUM_INPUT_ROWS = "input rows" + val DESCRIPTION_NUM_INPUT_BATCHES = "input columnar batches" + val DESCRIPTION_NUM_OUTPUT_ROWS = "output rows" + val DESCRIPTION_NUM_OUTPUT_BATCHES = "output columnar batches" + val DESCRIPTION_PARTITION_SIZE = "partition data size" + val DESCRIPTION_NUM_PARTITIONS = "partitions" + val DESCRIPTION_OP_TIME = "op time" + val DESCRIPTION_COLLECT_TIME = "collect batch time" + val DESCRIPTION_CONCAT_TIME = "concat batch time" + val DESCRIPTION_SORT_TIME = "sort time" + val DESCRIPTION_REPARTITION_TIME = "repartition time" + val DESCRIPTION_AGG_TIME = "aggregation time" + val DESCRIPTION_JOIN_TIME = "join time" + val DESCRIPTION_FILTER_TIME = "filter time" + val DESCRIPTION_BUILD_DATA_SIZE = "build side size" + val DESCRIPTION_BUILD_TIME = "build time" + val DESCRIPTION_STREAM_TIME = "stream time" + val DESCRIPTION_NUM_TASKS_FALL_BACKED = "number of sort fallback tasks" + val DESCRIPTION_NUM_TASKS_REPARTITIONED = "number of tasks repartitioned for agg" + val DESCRIPTION_NUM_TASKS_SKIPPED_AGG = "number of tasks skipped aggregation" + val DESCRIPTION_READ_FS_TIME = "time to read fs data" + val DESCRIPTION_WRITE_BUFFER_TIME = "time to write data to buffer" + val DESCRIPTION_FILECACHE_FOOTER_HITS = "cached footer hits" + val DESCRIPTION_FILECACHE_FOOTER_HITS_SIZE = "cached footer hits size" + val DESCRIPTION_FILECACHE_FOOTER_MISSES = "cached footer misses" + val DESCRIPTION_FILECACHE_FOOTER_MISSES_SIZE = "cached footer misses size" + val DESCRIPTION_FILECACHE_DATA_RANGE_HITS = "cached data hits" + val DESCRIPTION_FILECACHE_DATA_RANGE_HITS_SIZE = "cached data hits size" + val DESCRIPTION_FILECACHE_DATA_RANGE_MISSES = "cached data misses" + val DESCRIPTION_FILECACHE_DATA_RANGE_MISSES_SIZE = "cached data misses size" + val DESCRIPTION_FILECACHE_FOOTER_READ_TIME = "cached footer read time" + val DESCRIPTION_FILECACHE_DATA_RANGE_READ_TIME = "cached data read time" + val DESCRIPTION_DELETION_VECTOR_SCATTER_TIME = "deletion vector scatter time" + val DESCRIPTION_DELETION_VECTOR_SIZE = "deletion vector size" + val DESCRIPTION_CONCAT_HEADER_TIME = "concat header time" + val DESCRIPTION_CONCAT_BUFFER_TIME = "concat buffer time" + val DESCRIPTION_COPY_TO_HOST_TIME = "deviceToHost memory copy time" + + def unwrap(input: GpuMetric): SQLMetric = input match { + case w :WrappedGpuMetric => w.sqlMetric + case i => throw new IllegalArgumentException(s"found unsupported GpuMetric ${i.getClass}") + } + + def unwrap(input: Map[String, GpuMetric]): Map[String, SQLMetric] = { + val ret = input.collect { + // remove the metrics that are not registered + case (k, w) if w != NoopMetric => (k, unwrap(w)) + } + val companions = input.collect { + // add the companions + case (k, w) if w.companionGpuMetric.isDefined => + (k + "_exSemWait", unwrap(w.companionGpuMetric.get)) + } + + TreeMap.apply((ret ++ companions).toSeq: _*) + } + + def wrap(input: SQLMetric): GpuMetric = WrappedGpuMetric(input) + + def wrap(input: Map[String, SQLMetric]): Map[String, GpuMetric] = input.map { + case (k, v) => (k, wrap(v)) + } + + def ns[T](metrics: GpuMetric*)(f: => T): T = { + val start = System.nanoTime() + try { + f + } finally { + val taken = System.nanoTime() - start + metrics.foreach(_.add(taken)) + } + } + + object DEBUG_LEVEL extends MetricsLevel(0) + object MODERATE_LEVEL extends MetricsLevel(1) + object ESSENTIAL_LEVEL extends MetricsLevel(2) +} + +sealed abstract class GpuMetric extends Serializable { + def value: Long + def set(v: Long): Unit + def +=(v: Long): Unit + def add(v: Long): Unit + + private var isTimerActive = false + + // For timing GpuMetrics, we additionally create a companion GpuMetric to track elapsed time + // excluding semaphore wait time + var companionGpuMetric: Option[GpuMetric] = None + private var semWaitTimeWhenActivated = 0L + + final def tryActivateTimer(): Boolean = { + if (!isTimerActive) { + isTimerActive = true + semWaitTimeWhenActivated = GpuTaskMetrics.get.getSemWaitTime() + true + } else { + false + } + } + + final def deactivateTimer(duration: Long): Unit = { + if (isTimerActive) { + isTimerActive = false + companionGpuMetric.foreach(c => + c.add(duration - (GpuTaskMetrics.get.getSemWaitTime() - semWaitTimeWhenActivated))) + semWaitTimeWhenActivated = 0L + add(duration) + } + } + + final def ns[T](f: => T): T = { + if (tryActivateTimer()) { + val start = System.nanoTime() + try { + f + } finally { + deactivateTimer(System.nanoTime() - start) + } + } else { + f + } + } +} + +object NoopMetric extends GpuMetric { + override def +=(v: Long): Unit = () + override def add(v: Long): Unit = () + override def set(v: Long): Unit = () + override def value: Long = 0 +} + +final case class WrappedGpuMetric(sqlMetric: SQLMetric, withMetricsExclSemWait: Boolean = false) + extends GpuMetric { + + if (withMetricsExclSemWait) { + // SQLMetrics.NS_TIMING_METRIC and SQLMetrics.TIMING_METRIC is private, + // so we have to use the string directly + if (sqlMetric.metricType == "nsTiming") { + companionGpuMetric = Some(WrappedGpuMetric.apply(SQLMetrics.createNanoTimingMetric( + SparkSession.getActiveSession.get.sparkContext, sqlMetric.name.get + " (excl. SemWait)"))) + } + } + + def +=(v: Long): Unit = sqlMetric.add(v) + def add(v: Long): Unit = sqlMetric.add(v) + override def set(v: Long): Unit = sqlMetric.set(v) + override def value: Long = sqlMetric.value +} + +/** A GPU metric class that just accumulates into a variable without implicit publishing. */ +final class LocalGpuMetric extends GpuMetric { + private var lval = 0L + override def value: Long = lval + override def set(v: Long): Unit = { lval = v } + override def +=(v: Long): Unit = { lval += v } + override def add(v: Long): Unit = { lval += v } +} + +class CollectTimeIterator[T]( + nvtxName: String, + it: Iterator[T], + collectTime: GpuMetric) extends Iterator[T] { + override def hasNext: Boolean = { + withResource(new NvtxWithMetrics(nvtxName, NvtxColor.BLUE, collectTime)) { _ => + it.hasNext + } + } + + override def next(): T = { + withResource(new NvtxWithMetrics(nvtxName, NvtxColor.BLUE, collectTime)) { _ => + it.next + } + } +} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala index a1d5098be73..28c0c09eb76 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2024, NVIDIA CORPORATION. + * Copyright (c) 2019-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ import org.apache.spark.sql.execution.datasources.DataSourceUtils import org.apache.spark.sql.execution.datasources.parquet.{ParquetOptions, ParquetWriteSupport} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType +import org.apache.spark.sql.rapids.ColumnarWriteTaskStatsTracker import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch @@ -281,10 +282,11 @@ class GpuParquetFileFormat extends ColumnarFileFormat with Logging { path: String, dataSchema: StructType, context: TaskAttemptContext, + statsTrackers: Seq[ColumnarWriteTaskStatsTracker], debugOutputPath: Option[String]): ColumnarOutputWriter = { new GpuParquetWriter(path, dataSchema, compressionType, outputTimestampType.toString, dateTimeRebaseMode, timestampRebaseMode, context, parquetFieldIdWriteEnabled, - debugOutputPath, holdGpuBetweenBatches, asyncOutputWriteEnabled) + statsTrackers, debugOutputPath, holdGpuBetweenBatches, asyncOutputWriteEnabled) } override def getFileExtension(context: TaskAttemptContext): String = { @@ -307,10 +309,11 @@ class GpuParquetWriter( timestampRebaseMode: DateTimeRebaseMode, context: TaskAttemptContext, parquetFieldIdEnabled: Boolean, + statsTrackers: Seq[ColumnarWriteTaskStatsTracker], debugDumpPath: Option[String], holdGpuBetweenBatches: Boolean, useAsyncWrite: Boolean) - extends ColumnarOutputWriter(context, dataSchema, "Parquet", true, + extends ColumnarOutputWriter(context, dataSchema, "Parquet", true, statsTrackers, debugDumpPath, holdGpuBetweenBatches, useAsyncWrite) { override def throwIfRebaseNeededInExceptionMode(batch: ColumnarBatch): Unit = { val cols = GpuColumnVector.extractBases(batch) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala index 43bd593c0b5..bda9a3c1e89 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -39,10 +39,10 @@ import org.apache.spark.util.SerializableConfiguration * An extension of `RunnableCommand` that allows columnar execution. */ trait GpuRunnableCommand extends RunnableCommand with ShimUnaryCommand { - lazy val basicMetrics: Map[String, SQLMetric] = GpuWriteJobStatsTracker.basicMetrics - lazy val taskMetrics: Map[String, SQLMetric] = GpuWriteJobStatsTracker.taskMetrics + lazy val basicMetrics: Map[String, GpuMetric] = GpuWriteJobStatsTracker.basicMetrics + lazy val taskMetrics: Map[String, GpuMetric] = GpuWriteJobStatsTracker.taskMetrics - override lazy val metrics: Map[String, SQLMetric] = basicMetrics ++ taskMetrics + override lazy val metrics: Map[String, SQLMetric] = GpuMetric.unwrap(basicMetrics ++ taskMetrics) override final def run(sparkSession: SparkSession): Seq[Row] = throw new UnsupportedOperationException( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/AsyncOutputStream.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/AsyncOutputStream.scala index 40904a96dd2..bde935d9b13 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/AsyncOutputStream.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/AsyncOutputStream.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.{AtomicLong, AtomicReference} import com.nvidia.spark.rapids.RapidsPluginImplicits._ +import org.apache.spark.sql.rapids.ColumnarWriteTaskStatsTracker import org.apache.spark.sql.rapids.execution.TrampolineUtil /** @@ -29,14 +30,15 @@ import org.apache.spark.sql.rapids.execution.TrampolineUtil * and executed in the order they were scheduled. This class is not thread-safe and should only be * used by a single thread. */ -class AsyncOutputStream(openFn: Callable[OutputStream], trafficController: TrafficController) +class AsyncOutputStream(openFn: Callable[OutputStream], trafficController: TrafficController, + statsTrackers: Seq[ColumnarWriteTaskStatsTracker]) extends OutputStream { private var closed = false private val executor = new ThrottlingExecutor( - TrampolineUtil.newDaemonCachedThreadPool("AsyncOutputStream", 1, 1), - trafficController) + TrampolineUtil.newDaemonCachedThreadPool("AsyncOutputStream", 1, 1), trafficController, + statsTrackers) // Open the underlying stream asynchronously as soon as the AsyncOutputStream is constructed, // so that the open can be done in parallel with other operations. This could help with diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/ThrottlingExecutor.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/ThrottlingExecutor.scala index 99c3cc9ea5e..4c127654cfe 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/ThrottlingExecutor.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/io/async/ThrottlingExecutor.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,17 +18,36 @@ package com.nvidia.spark.rapids.io.async import java.util.concurrent.{Callable, ExecutorService, Future, TimeUnit} +import org.apache.spark.sql.rapids.{ColumnarWriteTaskStatsTracker, GpuWriteTaskStatsTracker} + /** * Thin wrapper around an ExecutorService that adds throttling. * * The given executor is owned by this class and will be shutdown when this class is shutdown. */ -class ThrottlingExecutor( - val executor: ExecutorService, throttler: TrafficController) { +class ThrottlingExecutor(executor: ExecutorService, throttler: TrafficController, + statsTrackers: Seq[ColumnarWriteTaskStatsTracker]) { + + private var numTasksScheduled = 0 + private var accumulatedThrottleTimeNs = 0L + private var minThrottleTimeNs = Long.MaxValue + private var maxThrottleTimeNs = 0L + + private def blockUntilTaskRunnable(task: Task[_]): Unit = { + val blockStart = System.nanoTime() + throttler.blockUntilRunnable(task) + val blockTimeNs = System.nanoTime() - blockStart + accumulatedThrottleTimeNs += blockTimeNs + minThrottleTimeNs = Math.min(minThrottleTimeNs, blockTimeNs) + maxThrottleTimeNs = Math.max(maxThrottleTimeNs, blockTimeNs) + numTasksScheduled += 1 + updateMetrics() + } def submit[T](callable: Callable[T], hostMemoryBytes: Long): Future[T] = { val task = new Task[T](hostMemoryBytes, callable) - throttler.blockUntilRunnable(task) + blockUntilTaskRunnable(task) + executor.submit(() => { try { task.call() @@ -38,7 +57,16 @@ class ThrottlingExecutor( }) } + def updateMetrics(): Unit = { + statsTrackers.foreach { + case gpuStatsTracker: GpuWriteTaskStatsTracker => gpuStatsTracker.setAsyncWriteThrottleTimes( + numTasksScheduled, accumulatedThrottleTimeNs, minThrottleTimeNs, maxThrottleTimeNs) + case _ => + } + } + def shutdownNow(timeout: Long, timeUnit: TimeUnit): Unit = { + updateMetrics() executor.shutdownNow() executor.awaitTermination(timeout, timeUnit) } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala index 11637b9818a..b1a0171b7f1 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -31,6 +31,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions import org.apache.spark.sql.hive.rapids.shims.GpuInsertIntoHiveTableMeta +import org.apache.spark.sql.rapids.ColumnarWriteTaskStatsTracker import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.types.{DataType, Decimal, DecimalType, StringType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -214,8 +215,10 @@ class GpuHiveParquetFileFormat(compType: CompressionType) extends ColumnarFileFo override def newInstance(path: String, dataSchema: StructType, context: TaskAttemptContext, + statsTrackers: Seq[ColumnarWriteTaskStatsTracker], debugOutputPath: Option[String]): ColumnarOutputWriter = { - new GpuHiveParquetWriter(path, dataSchema, context, compressionType, debugOutputPath) + new GpuHiveParquetWriter(path, dataSchema, context, compressionType, statsTrackers, + debugOutputPath) } } } @@ -223,8 +226,10 @@ class GpuHiveParquetFileFormat(compType: CompressionType) extends ColumnarFileFo class GpuHiveParquetWriter(override val path: String, dataSchema: StructType, context: TaskAttemptContext, compType: CompressionType, + statsTrackers: Seq[ColumnarWriteTaskStatsTracker], debugOutputPath: Option[String]) - extends ColumnarOutputWriter(context, dataSchema, "HiveParquet", true, debugOutputPath) { + extends ColumnarOutputWriter(context, dataSchema, "HiveParquet", true, statsTrackers, + debugOutputPath) { override protected val tableWriter: CudfTableWriter = { val optionsBuilder = SchemaUtils @@ -253,8 +258,9 @@ class GpuHiveTextFileFormat extends ColumnarFileFormat with Logging { override def newInstance(path: String, dataSchema: StructType, context: TaskAttemptContext, + statsTrackers: Seq[ColumnarWriteTaskStatsTracker], debugOutputPath: Option[String]): ColumnarOutputWriter = { - new GpuHiveTextWriter(path, dataSchema, context, debugOutputPath) + new GpuHiveTextWriter(path, dataSchema, context, statsTrackers, debugOutputPath) } } } @@ -263,8 +269,10 @@ class GpuHiveTextFileFormat extends ColumnarFileFormat with Logging { class GpuHiveTextWriter(override val path: String, dataSchema: StructType, context: TaskAttemptContext, + statsTrackers: Seq[ColumnarWriteTaskStatsTracker], debugOutputPath: Option[String]) - extends ColumnarOutputWriter(context, dataSchema, "HiveText", false, debugOutputPath) { + extends ColumnarOutputWriter(context, dataSchema, "HiveText", false, statsTrackers, + debugOutputPath) { /** * This reformats columns, to iron out inconsistencies between diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/BasicColumnarWriteStatsTracker.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/BasicColumnarWriteStatsTracker.scala index 7d2f6f90036..27a00e50fb6 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/BasicColumnarWriteStatsTracker.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/BasicColumnarWriteStatsTracker.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2023, NVIDIA CORPORATION. + * Copyright (c) 2019-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,7 @@ import java.nio.charset.StandardCharsets import scala.collection.mutable +import com.nvidia.spark.rapids.{GpuMetric, GpuMetricFactory, MetricsLevel, RapidsConf} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} @@ -28,7 +29,7 @@ import org.apache.spark.{SparkContext, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.WriteTaskStats -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.rapids.BasicColumnarWriteJobStatsTracker._ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration @@ -52,7 +53,7 @@ case class BasicColumnarWriteTaskStats( */ class BasicColumnarWriteTaskStatsTracker( hadoopConf: Configuration, - taskCommitTimeMetric: Option[SQLMetric]) + taskCommitTimeMetric: Option[GpuMetric]) extends ColumnarWriteTaskStatsTracker with Logging { private[this] var numPartitions: Int = 0 private[this] var numFiles: Int = 0 @@ -184,13 +185,13 @@ class BasicColumnarWriteTaskStatsTracker( */ class BasicColumnarWriteJobStatsTracker( serializableHadoopConf: SerializableConfiguration, - @transient val driverSideMetrics: Map[String, SQLMetric], - taskCommitTimeMetric: SQLMetric) + @transient val driverSideMetrics: Map[String, GpuMetric], + taskCommitTimeMetric: GpuMetric) extends ColumnarWriteJobStatsTracker { def this( serializableHadoopConf: SerializableConfiguration, - metrics: Map[String, SQLMetric]) = { + metrics: Map[String, GpuMetric]) = { this(serializableHadoopConf, metrics - TASK_COMMIT_TIME, metrics(TASK_COMMIT_TIME)) } @@ -221,7 +222,8 @@ class BasicColumnarWriteJobStatsTracker( driverSideMetrics(BasicColumnarWriteJobStatsTracker.NUM_PARTS_KEY).add(numPartitions) val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, driverSideMetrics.values.toList) + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, + GpuMetric.unwrap(driverSideMetrics).values.toList) } } @@ -235,15 +237,24 @@ object BasicColumnarWriteJobStatsTracker { /** XAttr key of the data length header added in HADOOP-17414. */ val FILE_LENGTH_XATTR = "header.x-hadoop-s3a-magic-data-length" - def metrics: Map[String, SQLMetric] = { + def metrics: Map[String, GpuMetric] = { val sparkContext = SparkContext.getActive.get + val metricsConf = MetricsLevel(sparkContext.conf.get(RapidsConf.METRICS_LEVEL.key, + RapidsConf.METRICS_LEVEL.defaultValue)) + val metricFactory = new GpuMetricFactory(metricsConf, sparkContext) Map( - NUM_FILES_KEY -> SQLMetrics.createMetric(sparkContext, "number of written files"), - NUM_OUTPUT_BYTES_KEY -> SQLMetrics.createSizeMetric(sparkContext, "written output"), - NUM_OUTPUT_ROWS_KEY -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - NUM_PARTS_KEY -> SQLMetrics.createMetric(sparkContext, "number of dynamic part"), - TASK_COMMIT_TIME -> SQLMetrics.createTimingMetric(sparkContext, "task commit time"), - JOB_COMMIT_TIME -> SQLMetrics.createTimingMetric(sparkContext, "job commit time") + NUM_FILES_KEY -> metricFactory.create(GpuMetric.ESSENTIAL_LEVEL, + "number of written files"), + NUM_OUTPUT_BYTES_KEY -> metricFactory.createSize(GpuMetric.ESSENTIAL_LEVEL, + "written output"), + NUM_OUTPUT_ROWS_KEY -> metricFactory.create(GpuMetric.ESSENTIAL_LEVEL, + "number of output rows"), + NUM_PARTS_KEY -> metricFactory.create(GpuMetric.ESSENTIAL_LEVEL, + "number of dynamic part"), + TASK_COMMIT_TIME -> metricFactory.createTiming(GpuMetric.ESSENTIAL_LEVEL, + "task commit time"), + JOB_COMMIT_TIME -> metricFactory.createTiming(GpuMetric.ESSENTIAL_LEVEL, + "job commit time") ) } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala index aca7d0e8751..2f4775d9e8d 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2024, NVIDIA CORPORATION. + * Copyright (c) 2019-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -165,7 +165,7 @@ abstract class GpuFileFormatDataWriter( protected final def writeUpdateMetricsAndClose(scb: SpillableColumnarBatch, writerStatus: WriterAndStatus): Unit = { - writerStatus.recordsInFile += writerStatus.writer.writeSpillableAndClose(scb, statsTrackers) + writerStatus.recordsInFile += writerStatus.writer.writeSpillableAndClose(scb) } /** Release all resources. Public for testing */ @@ -257,6 +257,7 @@ class GpuSingleDirectoryDataWriter( path = currentPath, dataSchema = description.dataColumns.toStructType, context = taskAttemptContext, + statsTrackers = statsTrackers, debugOutputPath = debugOutputPath) statsTrackers.foreach(_.newFile(currentPath)) @@ -599,6 +600,7 @@ class GpuDynamicPartitionDataSingleWriter( path = currentPath, dataSchema = description.dataColumns.toStructType, context = taskAttemptContext, + statsTrackers = statsTrackers, debugOutputPath = debugOutputPath) statsTrackers.foreach(_.newFile(currentPath)) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala index 422f6c2337e..a4370cbcf17 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2024, NVIDIA CORPORATION. + * Copyright (c) 2020-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -190,9 +190,10 @@ class GpuOrcFileFormat extends ColumnarFileFormat with Logging { override def newInstance(path: String, dataSchema: StructType, context: TaskAttemptContext, + statsTrackers: Seq[ColumnarWriteTaskStatsTracker], debugOutputPath: Option[String]): ColumnarOutputWriter = { - new GpuOrcWriter(path, dataSchema, context, debugOutputPath, holdGpuBetweenBatches, - asyncOutputWriteEnabled) + new GpuOrcWriter(path, dataSchema, context, statsTrackers, debugOutputPath, + holdGpuBetweenBatches, asyncOutputWriteEnabled) } override def getFileExtension(context: TaskAttemptContext): String = { @@ -215,10 +216,11 @@ class GpuOrcWriter( override val path: String, dataSchema: StructType, context: TaskAttemptContext, + statsTrackers: Seq[ColumnarWriteTaskStatsTracker], debugOutputPath: Option[String], holdGpuBetweenBatches: Boolean, useAsyncWrite: Boolean) - extends ColumnarOutputWriter(context, dataSchema, "ORC", true, debugOutputPath, + extends ColumnarOutputWriter(context, dataSchema, "ORC", true, statsTrackers, debugOutputPath, holdGpuBetweenBatches, useAsyncWrite) { override val tableWriter: TableWriter = { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuWriteStatsTracker.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuWriteStatsTracker.scala index 6dcab18e79e..fb781ac15b0 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuWriteStatsTracker.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuWriteStatsTracker.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2023, NVIDIA CORPORATION. + * Copyright (c) 2019-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,11 +16,10 @@ package org.apache.spark.sql.rapids -import com.nvidia.spark.rapids.GpuDataWritingCommand +import com.nvidia.spark.rapids.{GpuDataWritingCommand, GpuMetric, GpuMetricFactory, MetricsLevel, RapidsConf} import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkContext -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.rapids.BasicColumnarWriteJobStatsTracker.TASK_COMMIT_TIME import org.apache.spark.util.SerializableConfiguration @@ -30,7 +29,7 @@ import org.apache.spark.util.SerializableConfiguration */ class GpuWriteTaskStatsTracker( hadoopConf: Configuration, - taskMetrics: Map[String, SQLMetric]) + taskMetrics: Map[String, GpuMetric]) extends BasicColumnarWriteTaskStatsTracker(hadoopConf, taskMetrics.get(TASK_COMMIT_TIME)) { def addGpuTime(nanos: Long): Unit = { taskMetrics(GpuWriteJobStatsTracker.GPU_TIME_KEY) += nanos @@ -39,6 +38,20 @@ class GpuWriteTaskStatsTracker( def addWriteTime(nanos: Long): Unit = { taskMetrics(GpuWriteJobStatsTracker.WRITE_TIME_KEY) += nanos } + + def setAsyncWriteThrottleTimes(numTasks: Int, accumulatedThrottleTimeNs: Long, minNs: Long, + maxNs: Long): Unit = { + val avg = if (numTasks > 0) { + accumulatedThrottleTimeNs.toDouble / numTasks + } else { + 0 + } + taskMetrics(GpuWriteJobStatsTracker.ASYNC_WRITE_TOTAL_THROTTLE_TIME_KEY).set( + accumulatedThrottleTimeNs) + taskMetrics(GpuWriteJobStatsTracker.ASYNC_WRITE_AVG_THROTTLE_TIME_KEY).set(avg.toLong) + taskMetrics(GpuWriteJobStatsTracker.ASYNC_WRITE_MIN_THROTTLE_TIME_KEY).set(minNs) + taskMetrics(GpuWriteJobStatsTracker.ASYNC_WRITE_MAX_THROTTLE_TIME_KEY).set(maxNs) + } } /** @@ -49,8 +62,8 @@ class GpuWriteTaskStatsTracker( */ class GpuWriteJobStatsTracker( serializableHadoopConf: SerializableConfiguration, - @transient driverSideMetrics: Map[String, SQLMetric], - taskMetrics: Map[String, SQLMetric]) + @transient driverSideMetrics: Map[String, GpuMetric], + taskMetrics: Map[String, GpuMetric]) extends BasicColumnarWriteJobStatsTracker(serializableHadoopConf, driverSideMetrics) { override def newTaskInstance(): ColumnarWriteTaskStatsTracker = { new GpuWriteTaskStatsTracker(serializableHadoopConf.value, taskMetrics) @@ -60,15 +73,31 @@ class GpuWriteJobStatsTracker( object GpuWriteJobStatsTracker { val GPU_TIME_KEY = "gpuTime" val WRITE_TIME_KEY = "writeTime" + val ASYNC_WRITE_TOTAL_THROTTLE_TIME_KEY = "asyncWriteTotalThrottleTime" + val ASYNC_WRITE_AVG_THROTTLE_TIME_KEY = "asyncWriteAvgThrottleTime" + val ASYNC_WRITE_MIN_THROTTLE_TIME_KEY = "asyncWriteMinThrottleTime" + val ASYNC_WRITE_MAX_THROTTLE_TIME_KEY = "asyncWriteMaxThrottleTime" - def basicMetrics: Map[String, SQLMetric] = BasicColumnarWriteJobStatsTracker.metrics + def basicMetrics: Map[String, GpuMetric] = BasicColumnarWriteJobStatsTracker.metrics - def taskMetrics: Map[String, SQLMetric] = { + def taskMetrics: Map[String, GpuMetric] = { val sparkContext = SparkContext.getActive.get + val metricsConf = MetricsLevel(sparkContext.conf.get(RapidsConf.METRICS_LEVEL.key, + RapidsConf.METRICS_LEVEL.defaultValue)) + val metricFactory = new GpuMetricFactory(metricsConf, sparkContext) Map( - GPU_TIME_KEY -> SQLMetrics.createNanoTimingMetric(sparkContext, "GPU time"), - WRITE_TIME_KEY -> SQLMetrics.createNanoTimingMetric(sparkContext, "write time"), - TASK_COMMIT_TIME -> basicMetrics(TASK_COMMIT_TIME) + GPU_TIME_KEY -> metricFactory.createNanoTiming(GpuMetric.ESSENTIAL_LEVEL, "GPU time"), + WRITE_TIME_KEY -> metricFactory.createNanoTiming(GpuMetric.ESSENTIAL_LEVEL, + "write time"), + TASK_COMMIT_TIME -> basicMetrics(TASK_COMMIT_TIME), + ASYNC_WRITE_TOTAL_THROTTLE_TIME_KEY -> metricFactory.createNanoTiming( + GpuMetric.DEBUG_LEVEL, "total throttle time"), + ASYNC_WRITE_AVG_THROTTLE_TIME_KEY -> metricFactory.createNanoTiming( + GpuMetric.DEBUG_LEVEL, "avg throttle time per async write"), + ASYNC_WRITE_MIN_THROTTLE_TIME_KEY -> metricFactory.createNanoTiming( + GpuMetric.DEBUG_LEVEL, "min throttle time per async write"), + ASYNC_WRITE_MAX_THROTTLE_TIME_KEY -> metricFactory.createNanoTiming( + GpuMetric.DEBUG_LEVEL, "max throttle time per async write") ) } @@ -77,7 +106,7 @@ object GpuWriteJobStatsTracker { new GpuWriteJobStatsTracker(serializableHadoopConf, command.basicMetrics, command.taskMetrics) def apply(serializableHadoopConf: SerializableConfiguration, - basicMetrics: Map[String, SQLMetric], - taskMetrics: Map[String, SQLMetric]): GpuWriteJobStatsTracker = + basicMetrics: Map[String, GpuMetric], + taskMetrics: Map[String, GpuMetric]): GpuWriteJobStatsTracker = new GpuWriteJobStatsTracker(serializableHadoopConf, basicMetrics, taskMetrics) } diff --git a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/AsyncOutputStreamSuite.scala b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/AsyncOutputStreamSuite.scala index a4fa35349ce..8d5a7beb01a 100644 --- a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/AsyncOutputStreamSuite.scala +++ b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/AsyncOutputStreamSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,7 +35,7 @@ class AsyncOutputStreamSuite extends AnyFunSuite with BeforeAndAfterEach { new AsyncOutputStream(() => { val file = File.createTempFile("async-write-test", "tmp") new BufferedOutputStream(new FileOutputStream(file)) - }, trafficController) + }, trafficController, Seq.empty) } test("open, write, and close") { @@ -108,7 +108,7 @@ class AsyncOutputStreamSuite extends AnyFunSuite with BeforeAndAfterEach { } test("write after error") { - val os = new AsyncOutputStream(() => new ThrowingOutputStream, trafficController) + val os = new AsyncOutputStream(() => new ThrowingOutputStream, trafficController, Seq.empty) // The first call to `write` should succeed os.write(buf) @@ -134,7 +134,7 @@ class AsyncOutputStreamSuite extends AnyFunSuite with BeforeAndAfterEach { } test("flush after error") { - val os = new AsyncOutputStream(() => new ThrowingOutputStream, trafficController) + val os = new AsyncOutputStream(() => new ThrowingOutputStream, trafficController, Seq.empty) // The first write should succeed os.write(buf) @@ -151,7 +151,7 @@ class AsyncOutputStreamSuite extends AnyFunSuite with BeforeAndAfterEach { } test("close after error") { - val os = new AsyncOutputStream(() => new ThrowingOutputStream, trafficController) + val os = new AsyncOutputStream(() => new ThrowingOutputStream, trafficController, Seq.empty) os.write(buf) diff --git a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/ThrottlingExecutorSuite.scala b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/ThrottlingExecutorSuite.scala index 86fb692cd64..8bf64387134 100644 --- a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/ThrottlingExecutorSuite.scala +++ b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/io/async/ThrottlingExecutorSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,9 +18,14 @@ package com.nvidia.spark.rapids.io.async import java.util.concurrent.{Callable, CountDownLatch, ExecutionException, Executors, Future, RejectedExecutionException, TimeUnit} +import com.nvidia.spark.rapids.{GpuMetric, RapidsConf} +import org.apache.hadoop.conf.Configuration import org.scalatest.BeforeAndAfterEach import org.scalatest.funsuite.AnyFunSuite +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.rapids.{GpuWriteJobStatsTracker, GpuWriteTaskStatsTracker} + class ThrottlingExecutorSuite extends AnyFunSuite with BeforeAndAfterEach { // Some tests might take longer than usual in the limited CI environment. @@ -31,6 +36,15 @@ class ThrottlingExecutorSuite extends AnyFunSuite with BeforeAndAfterEach { var trafficController: TrafficController = _ var executor: ThrottlingExecutor = _ + // Initialize SparkSession to initialize the GpuMetrics. + SparkSession.builder + .master("local") + .appName("ThrottlingExecutorSuite") + .config(RapidsConf.METRICS_LEVEL.key, "DEBUG") + .getOrCreate() + + val taskMetrics: Map[String, GpuMetric] = GpuWriteJobStatsTracker.taskMetrics + class TestTask extends Callable[Unit] { val latch = new CountDownLatch(1) override def call(): Unit = { @@ -43,7 +57,8 @@ class ThrottlingExecutorSuite extends AnyFunSuite with BeforeAndAfterEach { trafficController = new TrafficController(throttle) executor = new ThrottlingExecutor( Executors.newSingleThreadExecutor(), - trafficController + trafficController, + Seq(new GpuWriteTaskStatsTracker(new Configuration(), taskMetrics)) ) } @@ -142,4 +157,57 @@ class ThrottlingExecutorSuite extends AnyFunSuite with BeforeAndAfterEach { val e2 = intercept[ExecutionException](future2.get()) assertCause(e2, classOf[RejectedExecutionException]) } + + test("test task metrics") { + val exec = Executors.newSingleThreadExecutor() + // Run a task. Note that the first task never waits in ThrottlingExecutor. + var runningTask = new TestTask + exec.submit(new Runnable { + override def run(): Unit = executor.submit(runningTask, 100) + }) + var taskCount = 1 + + for (i <- 0 to 9) { + val sleepTimeMs = (i + 1) * 10L + val waitingTask = new TestTask + // Latch indicating that the Runnable has been submitted. + val runnableSubmitted = new CountDownLatch(1) + // Latch indicating that waitingTask has been submitted to ThrottlingExecutor. + val waitingTaskSubmitted = new CountDownLatch(1) + exec.submit(new Runnable { + override def run(): Unit = { + runnableSubmitted.countDown() + executor.submit(waitingTask, 100) + waitingTaskSubmitted.countDown() + } + }) + taskCount += 1 + // Wait until the Runnable is submitted, and then sleep. + // This is to ensure that the waitingTask will wait for at least sleepTimeMs. + runnableSubmitted.await(longTimeoutSec, TimeUnit.SECONDS) + // Let the waitingTask wait for sleepTimeMs. + Thread.sleep(sleepTimeMs) + // Finish the running task. + runningTask.latch.countDown() + // Wait until the waitingTask is submitted. + waitingTaskSubmitted.await(longTimeoutSec, TimeUnit.SECONDS) + executor.updateMetrics() + + // Skip the check on the min throttle time as the first task never waits. + + assert(TimeUnit.MILLISECONDS.toNanos(sleepTimeMs) <= + taskMetrics(GpuWriteJobStatsTracker.ASYNC_WRITE_MAX_THROTTLE_TIME_KEY).value + ) + + runningTask = waitingTask + } + + // Finish the last task. + runningTask.latch.countDown() + + // Verify the average throttle time. + executor.updateMetrics() + assert(Seq.range(0, 10).sum * TimeUnit.MILLISECONDS.toNanos(10).toDouble / taskCount <= + taskMetrics(GpuWriteJobStatsTracker.ASYNC_WRITE_AVG_THROTTLE_TIME_KEY).value) + } } diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriterSuite.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriterSuite.scala index 645e17a8d6f..a1b0f04b51a 100644 --- a/tests/src/test/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriterSuite.scala +++ b/tests/src/test/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriterSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -60,6 +60,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { dataSchema, rangeName, includeRetry, + mockJobDescription.statsTrackers.map(_.newTaskInstance()), None) { // this writer (for tests) doesn't do anything and passes through the @@ -94,7 +95,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { types, "", includeRetry)) - when(mockOutputWriterFactory.newInstance(any(), any(), any(), any())) + when(mockOutputWriterFactory.newInstance(any(), any(), any(), any(), any())) .thenAnswer(_ => mockOutputWriter) } @@ -312,7 +313,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { singleWriter.commit() // we write 2 batches verify(mockOutputWriter, times(2)) - .writeSpillableAndClose(any(), any()) + .writeSpillableAndClose(any()) verify(mockOutputWriter, times(1)).close() } } @@ -333,7 +334,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { singleWriter.commit() // twice for the first batch given the split, and once for the second batch verify(mockOutputWriter, times(3)) - .writeSpillableAndClose(any(), any()) + .writeSpillableAndClose(any()) // three because we wrote 3 files (15 rows, limit was 5 rows per file) verify(mockOutputWriter, times(3)).close() } @@ -355,7 +356,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { dynamicSingleWriter.commit() // we write 9 batches (4 partitions in the first bach, and 5 partitions in the second) verify(mockOutputWriter, times(9)) - .writeSpillableAndClose(any(), any()) + .writeSpillableAndClose(any()) verify(dynamicSingleWriter, times(9)).newWriter(any(), any(), any()) // it uses 9 writers because the single writer mode only keeps one writer open at a time // and once a new partition is seen, the old writer is closed and a new one is opened. @@ -385,7 +386,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { val dynamicSingleWriter = prepareDynamicPartitionSingleWriter() dynamicSingleWriter.writeWithIterator(cbs.iterator) dynamicSingleWriter.commit() - verify(mockOutputWriter, times(numWrites)).writeSpillableAndClose(any(), any()) + verify(mockOutputWriter, times(numWrites)).writeSpillableAndClose(any()) verify(dynamicSingleWriter, times(numNewWriters)).newWriter(any(), any(), any()) verify(mockOutputWriter, times(numNewWriters)).close() } @@ -406,7 +407,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { dynamicSingleWriter.commit() // we get 13 calls because we write 13 individual batches after splitting verify(mockOutputWriter, times(13)) - .writeSpillableAndClose(any(), any()) + .writeSpillableAndClose(any()) verify(dynamicSingleWriter, times(13)).newWriter(any(), any(), any()) // since we have a limit of 1 record per file, we write 13 files verify(mockOutputWriter, times(13)) @@ -430,7 +431,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { dynamicConcurrentWriter.commit() // we get 9 calls because we have 9 partitions total verify(mockOutputWriter, times(9)) - .writeSpillableAndClose(any(), any()) + .writeSpillableAndClose(any()) // we write 5 files because we write 1 file per partition, since this concurrent // writer was able to keep the writers alive verify(dynamicConcurrentWriter, times(5)).newWriter(any(), any(), any()) @@ -463,7 +464,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { prepareDynamicPartitionConcurrentWriter(maxWriters = 20, batchSize = 100) dynamicConcurrentWriter.writeWithIterator(cbs.iterator) dynamicConcurrentWriter.commit() - verify(mockOutputWriter, times(numWrites)).writeSpillableAndClose(any(), any()) + verify(mockOutputWriter, times(numWrites)).writeSpillableAndClose(any()) verify(dynamicConcurrentWriter, times(numNewWriters)).newWriter(any(), any(), any()) verify(mockOutputWriter, times(numNewWriters)).close() } @@ -487,7 +488,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { // we get 13 calls here because we write 1 row files verify(mockOutputWriter, times(13)) - .writeSpillableAndClose(any(), any()) + .writeSpillableAndClose(any()) verify(dynamicConcurrentWriter, times(13)).newWriter(any(), any(), any()) // we have to open 13 writers (1 per row) given the record limit of 1 @@ -513,7 +514,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { // 6 batches written, one per partition (no splitting) plus one written by // the concurrent writer. verify(mockOutputWriter, times(6)) - .writeSpillableAndClose(any(), any()) + .writeSpillableAndClose(any()) verify(dynamicConcurrentWriter, times(5)).newWriter(any(), any(), any()) // 5 files written because this is the single writer mode verify(mockOutputWriter, times(5)).close() @@ -537,7 +538,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { dynamicConcurrentWriter.commit() // 18 batches are written, once per row above given maxRecorsPerFile verify(mockOutputWriter, times(18)) - .writeSpillableAndClose(any(), any()) + .writeSpillableAndClose(any()) verify(dynamicConcurrentWriter, times(18)).newWriter(any(), any(), any()) // dynamic partitioning code calls close several times on the same ColumnarOutputWriter // that doesn't seem to be an issue right now, but verifying that the writer was closed @@ -557,22 +558,26 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { resetMocksWithAndWithoutRetry { val cb = buildBatchWithPartitionedCol(1, 1, 1, 1, 1, 1, 1, 1, 1) val cbs = Seq(spy(cb)) - withColumnarBatchesVerifyClosed(cbs) { - // I would like to not flush on the first iteration of the `write` method - when(mockJobDescription.concurrentWriterPartitionFlushSize).thenReturn(1000) - when(mockJobDescription.maxRecordsPerFile).thenReturn(9) - val statsTracker = mock[ColumnarWriteTaskStatsTracker] - val jobTracker = new ColumnarWriteJobStatsTracker { - override def newTaskInstance(): ColumnarWriteTaskStatsTracker = { - statsTracker - } - override def processStats(stats: Seq[WriteTaskStats], jobCommitTime: Long): Unit = {} + // Mock jobDescription before it is passed to the mockOutputWriter in + // withColumnarBatchesVerifyClosed(). + + // I would like to not flush on the first iteration of the `write` method + when(mockJobDescription.concurrentWriterPartitionFlushSize).thenReturn(1000) + when(mockJobDescription.maxRecordsPerFile).thenReturn(9) + + val statsTracker = mock[ColumnarWriteTaskStatsTracker] + val jobTracker = new ColumnarWriteJobStatsTracker { + override def newTaskInstance(): ColumnarWriteTaskStatsTracker = { + statsTracker } - when(mockJobDescription.statsTrackers) - .thenReturn(Seq(jobTracker)) + override def processStats(stats: Seq[WriteTaskStats], jobCommitTime: Long): Unit = {} + } + when(mockJobDescription.statsTrackers) + .thenReturn(Seq(jobTracker)) - // throw once from bufferBatchAndClose to simulate an exception after we call the + withColumnarBatchesVerifyClosed(cbs) { + // throw once from bufferBatchAndClose to simulate an exception after we call the // stats tracker mockOutputWriter.throwOnNextBufferBatchAndClose( new GpuSplitAndRetryOOM("mocking a split and retry")) @@ -591,7 +596,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { // 1 batch is written, all rows fit verify(mockOutputWriter, times(1)) - .writeSpillableAndClose(any(), any()) + .writeSpillableAndClose(any()) // we call newBatch once verify(statsTracker, times(1)).newBatch(any(), any()) if (includeRetry) { @@ -614,23 +619,28 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { resetMocksWithAndWithoutRetry { val cb = buildBatchWithPartitionedCol(1, 1, 1, 1, 1, 1, 1, 1, 1) val cbs = Seq(spy(cb)) - withColumnarBatchesVerifyClosed(cbs) { - // I would like to not flush on the first iteration of the `write` method - when(mockJobDescription.concurrentWriterPartitionFlushSize).thenReturn(1000) - when(mockJobDescription.maxRecordsPerFile).thenReturn(9) - val statsTracker = mock[ColumnarWriteTaskStatsTracker] - val jobTracker = new ColumnarWriteJobStatsTracker { - override def newTaskInstance(): ColumnarWriteTaskStatsTracker = { - statsTracker - } + // Mock jobDescription before mockOutputWriter is initialized in + // withColumnarBatchesVerifyClosed(). + + // I would like to not flush on the first iteration of the `write` method + when(mockJobDescription.concurrentWriterPartitionFlushSize).thenReturn(1000) + when(mockJobDescription.maxRecordsPerFile).thenReturn(9) - override def processStats(stats: Seq[WriteTaskStats], jobCommitTime: Long): Unit = {} + val statsTracker = mock[ColumnarWriteTaskStatsTracker] + val jobTracker = new ColumnarWriteJobStatsTracker { + override def newTaskInstance(): ColumnarWriteTaskStatsTracker = { + statsTracker } - when(mockJobDescription.statsTrackers) - .thenReturn(Seq(jobTracker)) - when(statsTracker.newBatch(any(), any())) - .thenThrow(new GpuRetryOOM("mocking a retry")) + + override def processStats(stats: Seq[WriteTaskStats], jobCommitTime: Long): Unit = {} + } + when(mockJobDescription.statsTrackers) + .thenReturn(Seq(jobTracker)) + when(statsTracker.newBatch(any(), any())) + .thenThrow(new GpuRetryOOM("mocking a retry")) + + withColumnarBatchesVerifyClosed(cbs) { val dynamicConcurrentWriter = prepareDynamicPartitionConcurrentWriter(maxWriters = 5, batchSize = 1) @@ -643,7 +653,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { verify(mockOutputWriter, times(0)).bufferBatchAndClose(any()) // we attempt to write one batch verify(mockOutputWriter, times(1)) - .writeSpillableAndClose(any(), any()) + .writeSpillableAndClose(any()) // we call newBatch once verify(statsTracker, times(1)).newBatch(any(), any()) } From 139dd589c0e3e7cac5bf121c92fa3aa59020437b Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Fri, 10 Jan 2025 08:42:05 +0800 Subject: [PATCH 40/47] Fix two potential OOM issues in GPU aggregate. (#11908) close https://github.com/NVIDIA/spark-rapids/issues/11903 The first one is by taking the nested literals into account when calculating the output size for pre-split. See the linked issue above for more details. The second one is by using the correct size for buffer size comparison when collecting the next bundle of batches in aggregate. The size return from the `batchesByBucket.last.size()` is not the actual buffer size in bytes, but the element number of an array. It can not be used for the buffer size comparison. I verified this PR locally by the toy query and it works well. --------- Signed-off-by: Firestarman --- .../nvidia/spark/rapids/DataTypeUtils.scala | 7 +- .../spark/rapids/GpuAggregateExec.scala | 21 +-- .../spark/rapids/basicPhysicalOperators.scala | 171 +++++++++++++++++- .../unit/LiteralSizeEstimationTest.scala | 121 +++++++++++++ 4 files changed, 305 insertions(+), 15 deletions(-) create mode 100644 tests/src/test/scala/com/nvidia/spark/rapids/unit/LiteralSizeEstimationTest.scala diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DataTypeUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DataTypeUtils.scala index a031a2aaeed..e3d71818315 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DataTypeUtils.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DataTypeUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -25,6 +25,11 @@ object DataTypeUtils { case _ => false } + def hasOffset(dataType: DataType): Boolean = dataType match { + case _: ArrayType | _: StringType | _: BinaryType => true + case _ => false + } + def hasNestedTypes(schema: StructType): Boolean = schema.exists(f => isNestedType(f.dataType)) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala index 8fc5326705e..847a21d81e6 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2024, NVIDIA CORPORATION. + * Copyright (c) 2019-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -1095,16 +1095,13 @@ class GpuMergeAggregateIterator( closeOnExcept(new ArrayBuffer[AutoClosableArrayBuffer[SpillableColumnarBatch]]) { toAggregateBuckets => var currentSize = 0L - while (batchesByBucket.nonEmpty && - ( - // for some test cases targetMergeBatchSize is too small to fit any bucket, - // in this case we put the first bucket into toAggregateBuckets anyway - // refer to https://github.com/NVIDIA/spark-rapids/issues/11790 for examples - toAggregateBuckets.isEmpty || - batchesByBucket.last.size() + currentSize <= targetMergeBatchSize)) { - val bucket = batchesByBucket.remove(batchesByBucket.size - 1) - currentSize += bucket.map(_.sizeInBytes).sum - toAggregateBuckets += bucket + var keepGoing = true + while (batchesByBucket.nonEmpty && keepGoing) { + currentSize += batchesByBucket.last.map(_.sizeInBytes).sum + keepGoing = currentSize <= targetMergeBatchSize || toAggregateBuckets.isEmpty + if (keepGoing) { + toAggregateBuckets += batchesByBucket.remove(batchesByBucket.size - 1) + } } AggregateUtils.concatenateAndMerge( @@ -2225,4 +2222,4 @@ class DynamicGpuPartialAggregateIterator( throw new NoSuchElementException() } } -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala index 891e837d7e1..9330924e293 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2024, NVIDIA CORPORATION. + * Copyright (c) 2019-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -23,6 +23,7 @@ import ai.rapids.cudf import ai.rapids.cudf._ import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} +import com.nvidia.spark.rapids.DataTypeUtils.hasOffset import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitSpillableInHalfByRows, withRestoreOnRetry, withRetry, withRetryNoSplit} @@ -35,11 +36,13 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, RangePartitioning, SinglePartition, UnknownPartitioning} +import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SampleExec, SparkPlan} import org.apache.spark.sql.rapids.{GpuPartitionwiseSampledRDD, GpuPoissonSampler} import org.apache.spark.sql.rapids.execution.TrampolineUtil -import org.apache.spark.sql.types.{DataType, LongType} +import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.random.BernoulliCellSampler class GpuProjectExecMeta( @@ -308,12 +311,176 @@ object PreProjectSplitIterator { } else { boundExprs.getPassThroughIndex(index).map { inputIndex => cb.column(inputIndex).asInstanceOf[GpuColumnVector].getBase.getDeviceMemorySize + }.orElse { + // A literal has an exact size that should be taken into account. + extractGpuLit(boundExprs.exprTiers.last(index)).map { gpuLit => + calcSizeForLiteral(gpuLit.value, gpuLit.dataType, numRows) + } }.getOrElse { GpuBatchUtils.minGpuMemory(dataType, true, numRows) } } }.sum } + + @scala.annotation.tailrec + def extractGpuLit(exp: Expression): Option[GpuLiteral] = exp match { + case gl: GpuLiteral => Some(gl) + case ga: GpuAlias => extractGpuLit(ga.child) + case _ => None + } + + private[rapids] def calcSizeForLiteral(litVal: Any, litType: DataType, numRows: Int): Long = { + // First calculate the meta buffers size + val metaSize = new LitMetaCollector(litVal, litType).collect.map { litMeta => + val expandedRowsNum = litMeta.getRowsNum * numRows + var totalSize = 0L + if (litMeta.hasNull) { + totalSize += GpuBatchUtils.calculateValidityBufferSize(expandedRowsNum) + } + if (litMeta.hasOffset) { + totalSize += GpuBatchUtils.calculateOffsetBufferSize(expandedRowsNum) + } + totalSize + }.sum + // finalSize = oneLitValueSize * numRows + metadata size + calcLitValueSize(litVal, litType) * numRows + metaSize + } + + /** + * Represent the metadata information of a literal or one of its children, + * which will be used to calculate the final metadata size after expanding + * this literal to a column. + */ + private class LitMeta(val hasNull: Boolean, val hasOffset: Boolean) { + private var rowsNum: Int = 0 + def incRowsNum(rows: Int = 1): Unit = rowsNum += rows + def getRowsNum: Int = rowsNum + + override def toString: String = + s"LitMeta{rowsNum: $rowsNum, hasNull: $hasNull, hasOffset: $hasOffset}" + } + + /** + * Collect the metadata information of a literal, the result also includes + * its children for a nested type literal. + */ + private class LitMetaCollector(litValue: Any, litType: DataType) { + private var collected = false + private val metaInfos: ArrayBuffer[LitMeta] = ArrayBuffer.empty + + def collect: Seq[LitMeta] = { + if (!collected) { + executeCollect(litValue, litType, litValue == null, 0) + collected = true + } + metaInfos.filter(_ != null).toSeq + } + + /** + * Go through the literal and all its children to collect the meta information and + * save to the cache, call "collect" to get the result. + * Each LitMeta indicates whether the literal or a child will has offset/validity + * buffers after being expanded to a column, along with the number of original rows. + * For nested types, it follows the type definition from + * https://github.com/rapidsai/cudf/blob/a0487be669326175982c8bfcdab4d61184c88e27/ + * cpp/doxygen/developer_guide/DEVELOPER_GUIDE.md#list-columns + */ + private def executeCollect(lit: Any, litTp: DataType, nullable: Boolean, + depth: Int): Unit = { + litTp match { + case ArrayType(elemType, hasNullElem) => + // It may be at a middle element of a nested array, so use the nullable + // from the parent. + getOrInitAt(depth, new LitMeta(nullable, true)).incRowsNum() + // Go into the child + val arrayData = lit.asInstanceOf[ArrayData] + if (arrayData != null) { // Only need to go into child when nonempty + (0 until arrayData.numElements()).foreach(i => + executeCollect(arrayData.get(i, elemType), elemType, hasNullElem, depth + 1) + ) + } + case StructType(fields) => + if (nullable) { + // Add a meta for only a nullable struct, and a struct doesn't have offsets. + getOrInitAt(depth, new LitMeta(nullable, false)).incRowsNum() + } + // Always go into children, which is different from array. + val stData = lit.asInstanceOf[InternalRow] + fields.zipWithIndex.foreach { case (f, i) => + val fLit = if (stData != null) stData.get(i, f.dataType) else null + executeCollect(fLit, f.dataType, f.nullable, depth + 1 + i) + } + case MapType(keyType, valType, hasNullValue) => + // Map is list of struct in cudf. But the nested struct has no offset or + // validity, so only need a meta for the top list. + getOrInitAt(depth, new LitMeta(nullable, true)).incRowsNum() + val mapData = lit.asInstanceOf[MapData] + if (mapData != null) { + mapData.foreach(keyType, valType, { case (key, value) => + executeCollect(key, keyType, false, depth + 1) + executeCollect(value, valType, hasNullValue, depth + 2) + }) + } + case otherType => // primitive types + val hasOffsetBuf = hasOffset(otherType) + if (nullable || hasOffsetBuf) { + getOrInitAt(depth, new LitMeta(nullable, hasOffsetBuf)).incRowsNum() + } + } + } + + private def getOrInitAt(pos: Int, initMeta: LitMeta): LitMeta = { + if (pos >= metaInfos.length) { + (metaInfos.length until pos).foreach { _ => + metaInfos.append(null) + } + metaInfos.append(initMeta) + } else if (metaInfos(pos) == null) { + metaInfos(pos) = initMeta + } + metaInfos(pos) + } + } + + private def calcLitValueSize(lit: Any, litTp: DataType): Long = { + litTp match { + case StringType => + if (lit == null) 0L else lit.asInstanceOf[UTF8String].numBytes() + case BinaryType => + if (lit == null) 0L else lit.asInstanceOf[Array[Byte]].length + case ArrayType(elemType, _) => + val arrayData = lit.asInstanceOf[ArrayData] + if (arrayData == null) { + 0L + } else { + (0 until arrayData.numElements()).map { idx => + calcLitValueSize(arrayData.get(idx, elemType), elemType) + }.sum + } + case MapType(keyType, valType, _) => + val mapData = lit.asInstanceOf[MapData] + if (mapData == null) { + 0L + } else { + val keyData = mapData.keyArray() + val valData = mapData.valueArray() + (0 until mapData.numElements()).map { i => + calcLitValueSize(keyData.get(i, keyType), keyType) + + calcLitValueSize(valData.get(i, valType), valType) + }.sum + } + case StructType(fields) => + // A special case that it should always go into children even lit is null. + // Because the children of fixed width will always take some memory. + val stData = lit.asInstanceOf[InternalRow] + fields.zipWithIndex.map { case (f, i) => + val fLit = if (stData == null) null else stData.get(i, f.dataType) + calcLitValueSize(fLit, f.dataType) + }.sum + case _ => litTp.defaultSize + } + } } /** diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/unit/LiteralSizeEstimationTest.scala b/tests/src/test/scala/com/nvidia/spark/rapids/unit/LiteralSizeEstimationTest.scala new file mode 100644 index 00000000000..b50e23ae78e --- /dev/null +++ b/tests/src/test/scala/com/nvidia/spark/rapids/unit/LiteralSizeEstimationTest.scala @@ -0,0 +1,121 @@ +/* + * Copyright (c) 2025, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids.unit + +import ai.rapids.cudf.ColumnVector +import com.nvidia.spark.rapids.{GpuScalar, GpuUnitTests, PreProjectSplitIterator} +import com.nvidia.spark.rapids.Arm.withResource + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData} +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +/** These tests only cover nested type literals for the PreProjectSplitIterator case */ +class LiteralSizeEstimationTest extends GpuUnitTests { + private val numRows = 1000 + + private def testLiteralSizeEstimate(lit: Any, litType: DataType): Unit = { + val col = withResource(GpuScalar.from(lit, litType))(ColumnVector.fromScalar(_, numRows)) + val actualSize = withResource(col)(_.getDeviceMemorySize) + val estimatedSize = PreProjectSplitIterator.calcSizeForLiteral(lit, litType, numRows) + assertResult(actualSize)(estimatedSize) + } + + test("estimate the array(int) literal size") { + val litType = ArrayType(IntegerType, true) + val lit = ArrayData.toArrayData(Array(null, 1, 2, null)) + testLiteralSizeEstimate(lit, litType) + } + + test("estimate the array(string) literal size") { + val litType = ArrayType(StringType, true) + val lit = ArrayData.toArrayData( + Array(null, UTF8String.fromString("s1"), UTF8String.fromString("s2"))) + testLiteralSizeEstimate(lit, litType) + } + + test("estimate the array(array(array(int))) literal size") { + val litType = ArrayType(ArrayType(ArrayType(IntegerType, true), true), true) + val nestedElem1 = ArrayData.toArrayData(Array(null, 1, 2, null)) + val nestedElem2 = ArrayData.toArrayData(Array(null)) + val nestedElem3 = ArrayData.toArrayData(Array()) + val elem1 = ArrayData.toArrayData(Array(nestedElem1, null)) + val elem2 = ArrayData.toArrayData(Array(nestedElem2, null, nestedElem3)) + val lit = ArrayData.toArrayData(Array(null, elem1, null, elem2, null)) + testLiteralSizeEstimate(lit, litType) + } + + test("estimate the array(array(array(string))) literal size") { + val litType = ArrayType(ArrayType(ArrayType(StringType, true), true), true) + val nestedElem1 = ArrayData.toArrayData( + Array(null, UTF8String.fromString("s1"), UTF8String.fromString("s2"))) + val nestedElem2 = ArrayData.toArrayData(Array(null)) + val nestedElem3 = ArrayData.toArrayData(Array()) + val elem1 = ArrayData.toArrayData(Array(nestedElem1, null)) + val elem2 = ArrayData.toArrayData(Array(nestedElem2, null, nestedElem3)) + val lit = ArrayData.toArrayData(Array(null, elem1, null, elem2, null)) + testLiteralSizeEstimate(lit, litType) + } + + test("estimate the struct(int, string) literal size") { + val litType = StructType(Seq( + StructField("int1", IntegerType), + StructField("string2", StringType) + )) + // null + testLiteralSizeEstimate(InternalRow(null, null), litType) + // normal case + testLiteralSizeEstimate(InternalRow(1, UTF8String.fromString("s1")), litType) + } + + test("estimate the struct(int, array(string)) literal size") { + val litType = StructType(Seq( + StructField("int1", IntegerType), + StructField("string2", ArrayType(StringType, true)) + )) + testLiteralSizeEstimate(InternalRow(null, null), litType) + val arrayLit = ArrayData.toArrayData( + Array(null, UTF8String.fromString("s1"), UTF8String.fromString("s2"))) + // normal case + testLiteralSizeEstimate(InternalRow(1, arrayLit), litType) + } + + test("estimate the list(struct(int, array(string))) literal size") { + val litType = ArrayType( + StructType(Seq( + StructField("int1", IntegerType), + StructField("string2", ArrayType(StringType, true)) + )), true) + val arrayLit = ArrayData.toArrayData( + Array(null, UTF8String.fromString("a1"), UTF8String.fromString("a2"))) + val elem1 = InternalRow(1, arrayLit) + val elem2 = InternalRow(null, null) + val lit = ArrayData.toArrayData(Array(null, elem1, elem2)) + testLiteralSizeEstimate(lit, litType) + } + + test("estimate the map(int, array(string)) literal size") { + val litType = MapType(IntegerType, ArrayType(StringType, true), true) + val arrayLit = ArrayData.toArrayData( + Array(null, UTF8String.fromString("s1"), UTF8String.fromString("s2"))) + val valueLit = ArrayData.toArrayData(Array(null, arrayLit)) + val keyLit = ArrayData.toArrayData(Array(1, 2)) + val lit = new ArrayBasedMapData(keyLit, valueLit) + testLiteralSizeEstimate(lit, litType) + } +} From b5075ed451213bc3c56064f11214e55a5fd8a745 Mon Sep 17 00:00:00 2001 From: Tim Liu Date: Fri, 10 Jan 2025 17:16:47 +0800 Subject: [PATCH 41/47] Update download page for 24.12.1 hot fix release [skip ci] (#11944) Update download page for 24.12.1 hot fix release Signed-off-by: Tim Liu --- docs/archive.md | 93 ++++++++++++++++++++++++++++++++++++++++++++++++ docs/download.md | 18 +++++----- 2 files changed, 102 insertions(+), 9 deletions(-) diff --git a/docs/archive.md b/docs/archive.md index 4f9c5ed49fc..ce5667810f9 100644 --- a/docs/archive.md +++ b/docs/archive.md @@ -5,6 +5,99 @@ nav_order: 15 --- Below are archived releases for RAPIDS Accelerator for Apache Spark. +## Release v24.12.0 +### Hardware Requirements: + +The plugin is tested on the following architectures: + + GPU Models: NVIDIA V100, T4, A10/A100, L4 and H100 GPUs + +### Software Requirements: + + OS: Spark RAPIDS is compatible with any Linux distribution with glibc >= 2.28 (Please check ldd --version output). glibc 2.28 was released August 1, 2018. + Tested on Ubuntu 20.04, Ubuntu 22.04, Rocky Linux 8 and Rocky Linux 9 + + NVIDIA Driver*: R470+ + + Runtime: + 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. + + 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, 3.3.4 + Apache Spark 3.4.0, 3.4.1, 3.4.2, 3.4.3 + Apache Spark 3.5.0, 3.5.1, 3.5.2 + + Supported Databricks runtime versions for Azure and AWS: + 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) + Databricks 13.3 ML LTS (GPU, Scala 2.12, Spark 3.4.1) + + Supported Dataproc versions (Debian/Ubuntu/Rocky): + GCP Dataproc 2.1 + GCP Dataproc 2.2 + + Supported Dataproc Serverless versions: + Spark runtime 1.1 LTS + Spark runtime 2.0 + Spark runtime 2.1 + Spark runtime 2.2 + +*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 RAPIDS Accelerator for Apache Spark v24.12.0 + +| Processor | Scala Version | Download Jar | Download Signature | +|-----------|---------------|--------------|--------------------| +| x86_64 | Scala 2.12 | [RAPIDS Accelerator v24.12.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.0/rapids-4-spark_2.12-24.12.0.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.0/rapids-4-spark_2.12-24.12.0.jar.asc) | +| x86_64 | Scala 2.13 | [RAPIDS Accelerator v24.12.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.0/rapids-4-spark_2.13-24.12.0.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.0/rapids-4-spark_2.13-24.12.0.jar.asc) | +| arm64 | Scala 2.12 | [RAPIDS Accelerator v24.12.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.0/rapids-4-spark_2.12-24.12.0-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.0/rapids-4-spark_2.12-24.12.0-cuda11-arm64.jar.asc) | +| arm64 | Scala 2.13 | [RAPIDS Accelerator v24.12.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.0/rapids-4-spark_2.13-24.12.0-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.0/rapids-4-spark_2.13-24.12.0-cuda11-arm64.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 [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com). +* Import the public key: `gpg --import PUB_KEY` +* Verify the signature for Scala 2.12 jar: + `gpg --verify rapids-4-spark_2.12-24.12.0.jar.asc rapids-4-spark_2.12-24.12.0.jar` +* Verify the signature for Scala 2.13 jar: + `gpg --verify rapids-4-spark_2.13-24.12.0.jar.asc rapids-4-spark_2.13-24.12.0.jar` + +The output of signature verify: + + gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) " + +### Release Notes +* Add repartition-based algorithm fallback in hash aggregate +* Support Spark function months_between +* Support asynchronous writing for Parquet files +* Add retry support to improve sub hash-join stability +* Improve JSON scan and from_json +* Improved performance for CASE WHEN statements comparing a string column against multiple values +* Falling back to the CPU for ORC boolean writes by the GPU due to a bug in cudf's ORC writer +* Fix a device memory leak in timestamp operator in `incompatibleDateFormats` case +* Fix a host memory leak in GpuBroadcastNestedLoopJoinExecBase when `spillableBuiltBatch` is 0 +* For updates on RAPIDS Accelerator Tools, please visit [this link](https://github.com/NVIDIA/spark-rapids-tools/releases) + +Note: There is a known issue in the 24.12.0 release when decompressing gzip files on H100 GPUs. +Please find more details in [issue-16661](https://github.com/rapidsai/cudf/issues/16661). + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + ## Release v24.10.1 ### Hardware Requirements: diff --git a/docs/download.md b/docs/download.md index e16c94e5b90..5ae5b6f786d 100644 --- a/docs/download.md +++ b/docs/download.md @@ -18,7 +18,7 @@ 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 v24.12.0 +## Release v24.12.1 ### Hardware Requirements: The plugin is tested on the following architectures: @@ -69,14 +69,14 @@ for your hardware's minimum driver version. ### 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 RAPIDS Accelerator for Apache Spark v24.12.0 +### Download RAPIDS Accelerator for Apache Spark v24.12.1 | Processor | Scala Version | Download Jar | Download Signature | |-----------|---------------|--------------|--------------------| -| x86_64 | Scala 2.12 | [RAPIDS Accelerator v24.12.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.0/rapids-4-spark_2.12-24.12.0.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.0/rapids-4-spark_2.12-24.12.0.jar.asc) | -| x86_64 | Scala 2.13 | [RAPIDS Accelerator v24.12.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.0/rapids-4-spark_2.13-24.12.0.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.0/rapids-4-spark_2.13-24.12.0.jar.asc) | -| arm64 | Scala 2.12 | [RAPIDS Accelerator v24.12.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.0/rapids-4-spark_2.12-24.12.0-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.0/rapids-4-spark_2.12-24.12.0-cuda11-arm64.jar.asc) | -| arm64 | Scala 2.13 | [RAPIDS Accelerator v24.12.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.0/rapids-4-spark_2.13-24.12.0-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.0/rapids-4-spark_2.13-24.12.0-cuda11-arm64.jar.asc) | +| x86_64 | Scala 2.12 | [RAPIDS Accelerator v24.12.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.1/rapids-4-spark_2.12-24.12.1.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.1/rapids-4-spark_2.12-24.12.1.jar.asc) | +| x86_64 | Scala 2.13 | [RAPIDS Accelerator v24.12.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.1/rapids-4-spark_2.13-24.12.1.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.1/rapids-4-spark_2.13-24.12.1.jar.asc) | +| arm64 | Scala 2.12 | [RAPIDS Accelerator v24.12.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.1/rapids-4-spark_2.12-24.12.1-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.12.1/rapids-4-spark_2.12-24.12.1-cuda11-arm64.jar.asc) | +| arm64 | Scala 2.13 | [RAPIDS Accelerator v24.12.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.1/rapids-4-spark_2.13-24.12.1-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.12.1/rapids-4-spark_2.13-24.12.1-cuda11-arm64.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. @@ -85,9 +85,9 @@ CUDA 11.8 through CUDA 12.0. * 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 for Scala 2.12 jar: - `gpg --verify rapids-4-spark_2.12-24.12.0.jar.asc rapids-4-spark_2.12-24.12.0.jar` + `gpg --verify rapids-4-spark_2.12-24.12.1.jar.asc rapids-4-spark_2.12-24.12.1.jar` * Verify the signature for Scala 2.13 jar: - `gpg --verify rapids-4-spark_2.13-24.12.0.jar.asc rapids-4-spark_2.13-24.12.0.jar` + `gpg --verify rapids-4-spark_2.13-24.12.1.jar.asc rapids-4-spark_2.13-24.12.1.jar` The output of signature verify: @@ -105,7 +105,7 @@ The output of signature verify: * Fix a host memory leak in GpuBroadcastNestedLoopJoinExecBase when `spillableBuiltBatch` is 0 * For updates on RAPIDS Accelerator Tools, please visit [this link](https://github.com/NVIDIA/spark-rapids-tools/releases) -Note: There is a known issue in the 24.12.0 release when decompressing gzip files on H100 GPUs. +Note: There is a known issue in the 24.12.1 release when decompressing gzip files on H100 GPUs. Please find more details in [issue-16661](https://github.com/rapidsai/cudf/issues/16661). For a detailed list of changes, please refer to the From db484e17d51ce030ac551840bb222df70e0d3e11 Mon Sep 17 00:00:00 2001 From: Jenkins Automation <70000568+nvauto@users.noreply.github.com> Date: Fri, 10 Jan 2025 17:24:27 +0800 Subject: [PATCH 42/47] Update rapids JNI dependency to 24.12.1 (#11943) Wait for the pre-merge CI job to SUCCEED --------- Signed-off-by: nvauto <70000568+nvauto@users.noreply.github.com> Signed-off-by: Tim Liu Co-authored-by: Tim Liu --- pom.xml | 4 ++-- scala2.13/pom.xml | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index 00d2777a58e..79883589299 100644 --- a/pom.xml +++ b/pom.xml @@ -1,6 +1,6 @@ --------- Signed-off-by: Niranjan Artal --- .../sql/tests/datagen/DataGenExprShims.scala | 3 +- docs/download.md | 2 +- pom.xml | 22 + scala2.13/pom.xml | 22 + .../shims/ShimSupportsRuntimeFiltering.java | 3 +- .../spark/rapids/RapidsShuffleManager.scala | 3 +- .../nvidia/spark/rapids/shims/AQEUtils.scala | 3 +- .../rapids/shims/AggregationTagging.scala | 3 +- .../spark/rapids/shims/CudfUnsafeRow.scala | 3 +- .../rapids/shims/CudfUnsafeRowBase.scala | 3 +- .../rapids/shims/DateTimeUtilsShims.scala | 3 +- .../spark/rapids/shims/DeltaLakeUtils.scala | 3 +- .../rapids/shims/FileSourceScanExecMeta.scala | 1 + .../rapids/shims/GpuBatchScanExecBase.scala | 3 +- .../shims/GpuFileFormatDataWriterShim.scala | 3 +- .../spark/rapids/shims/GpuOrcDataReader.scala | 3 +- .../shims/GpuOrcDataReader320Plus.scala | 3 +- .../rapids/shims/GpuOrcDataReaderBase.scala | 3 +- .../spark/rapids/shims/GpuParquetCrypto.scala | 3 +- .../rapids/shims/GpuWindowInPandasExec.scala | 3 +- .../nvidia/spark/rapids/shims/HashUtils.scala | 3 +- .../rapids/shims/NullIntolerantShim.scala | 3 +- .../shims/OffsetWindowFunctionMeta.scala | 3 +- .../spark/rapids/shims/OrcCastingShims.scala | 3 +- .../shims/OrcShims320untilAllBase.scala | 3 +- .../spark/rapids/shims/PlanShimsImpl.scala | 3 +- .../spark/rapids/shims/RaiseErrorShim.scala | 3 +- .../rapids/shims/RapidsCsvScanMeta.scala | 3 +- .../spark/rapids/shims/RebaseShims.scala | 3 +- .../rapids/shims/ShimAQEShuffleReadExec.scala | 1 + .../rapids/shims/ShimBaseSubqueryExec.scala | 3 +- .../shims/ShimBroadcastExchangeLike.scala | 3 +- .../spark/rapids/shims/ShimLeafExecNode.scala | 3 +- .../rapids/shims/ShimPredicateHelper.scala | 3 +- .../shims/ShuffleManagerShimUtils.scala | 3 +- .../rapids/shims/Spark320PlusNonDBShims.scala | 3 +- .../rapids/shims/Spark320PlusShims.scala | 3 +- .../rapids/shims/StaticPartitionShims.scala | 3 +- .../nvidia/spark/rapids/shims/TreeNode.scala | 3 +- .../spark/rapids/shims/TypeSigUtil.scala | 3 +- .../spark/rapids/shims/XxHash64Shims.scala | 3 +- .../spark/rapids/shims/YearParseUtil.scala | 3 +- .../rapids/shims/extractValueShims.scala | 3 +- .../spark/rapids/shims/gpuWindows.scala | 3 +- .../spark/rapids/v1FallbackWriters.scala | 3 +- .../shims/GpuShuffleBlockResolver.scala | 3 +- .../rapids/shims/GpuShuffleExchangeExec.scala | 3 +- .../rapids/shims/ShuffledBatchRDDUtil.scala | 3 +- .../shims/storage/ShimDiskBlockManager.scala | 3 +- .../hive/rapids/shims/CommandUtilsShim.scala | 3 +- .../shims/GpuRowBasedHiveGenericUDFShim.scala | 3 +- .../apache/spark/sql/nvidia/DFUDFShims.scala | 3 +- .../sql/rapids/RapidsShuffleWriter.scala | 5 +- .../execution/GpuSubqueryBroadcastMeta.scala | 3 +- .../python/shims/GpuArrowPythonOutput.scala | 3 +- .../python/shims/GpuArrowPythonRunner.scala | 3 +- .../shims/GpuCoGroupedArrowPythonRunner.scala | 3 +- .../shims/GpuGroupedPythonRunnerFactory.scala | 3 +- .../python/shims/WritePythonUDFUtils.scala | 3 +- .../spark/sql/rapids/shims/AvroUtils.scala | 3 +- .../rapids/shims/RapidsQueryErrorUtils.scala | 3 +- .../shims/RapidsShuffleThreadedWriter.scala | 3 +- .../sql/rapids/shims/Spark32XShimsUtils.scala | 3 +- .../rapids/shims/datetimeExpressions.scala | 3 +- .../apache/spark/sql/rapids/shims/misc.scala | 3 +- .../storage/RapidsPushBasedFetchHelper.scala | 3 +- .../RapidsShuffleBlockFetcherIterator.scala | 3 +- .../rapids/shims/Spark321PlusShims.scala | 3 +- .../spark/sql/rapids/shims/GpuAscii.scala | 3 +- .../nvidia/spark/rapids/GpuBloomFilter.scala | 3 +- .../rapids/GpuBloomFilterMightContain.scala | 3 +- .../spark/rapids/GpuInSubqueryExec.scala | 3 +- .../nvidia/spark/rapids/shims/AnsiUtil.scala | 3 +- .../spark/rapids/shims/BloomFilterShims.scala | 3 +- .../rapids/shims/BucketingUtilsShim.scala | 3 +- .../rapids/shims/CharVarcharUtilsShims.scala | 3 +- .../rapids/shims/DayTimeIntervalShims.scala | 3 +- .../spark/rapids/shims/DistributionUtil.scala | 3 +- .../rapids/shims/FilteredPartitions.scala | 3 +- .../spark/rapids/shims/GpuDataSourceRDD.scala | 3 +- .../rapids/shims/GpuHashPartitioning.scala | 3 +- .../rapids/shims/GpuIntervalUtilsBase.scala | 3 +- .../rapids/shims/GpuRangePartitioning.scala | 3 +- .../spark/rapids/shims/GpuTypeShims.scala | 3 +- .../spark/rapids/shims/InSubqueryShims.scala | 3 +- .../spark/rapids/shims/OrcReadingShims.scala | 3 +- .../nvidia/spark/rapids/shims/OrcShims.scala | 3 +- .../rapids/shims/ParquetFieldIdShims.scala | 3 +- .../rapids/shims/ParquetSchemaClipShims.scala | 3 +- .../shims/RapidsFileSourceMetaUtils.scala | 3 +- .../rapids/shims/RapidsOrcScanMeta.scala | 3 +- .../rapids/shims/RapidsParquetScanMeta.scala | 3 +- .../spark/rapids/shims/RoundingShims.scala | 3 +- .../spark/rapids/shims/ScanExecShims.scala | 3 +- .../rapids/shims/Spark330PlusNonDBShims.scala | 3 +- .../rapids/shims/Spark330PlusShims.scala | 3 +- .../RapidsVectorizedColumnReader.scala | 3 +- .../parquet/ShimCurrentBatchIterator.scala | 3 +- .../aggregate/GpuBloomFilterAggregate.scala | 3 +- .../shims/GpuPythonMapInArrowExec.scala | 3 +- .../shims/RapidsErrorUtilsFor330plus.scala | 3 +- .../shims/RapidsShuffleThreadedReader.scala | 3 +- .../rapids/shims/intervalExpressions.scala | 3 +- .../spark/rapids/shims/AnsiCastShim.scala | 3 +- .../rapids/shims/CastingConfigShim.scala | 3 +- .../shims/ColumnDefaultValuesShims.scala | 3 +- .../shims/DecimalArithmeticOverrides.scala | 3 +- .../spark/rapids/shims/GetMapValueMeta.scala | 3 +- .../spark/rapids/shims/GpuCastShims.scala | 3 +- .../rapids/shims/ParquetStringPredShims.scala | 3 +- .../ShimFilePartitionReaderFactory.scala | 3 +- .../spark/rapids/shims/TypeUtilsShims.scala | 3 +- .../rapids/DataSourceStrategyUtils.scala | 3 +- .../GpuCheckOverflowInTableInsert.scala | 3 +- .../rapids/aggregate/aggregateFunctions.scala | 3 +- .../apache/spark/sql/rapids/arithmetic.scala | 3 +- .../rapids/shims/Spark331PlusNonDBShims.scala | 3 +- ...aSourceTableAsSelectCommandMetaShims.scala | 3 +- .../rapids/shims/GpuInsertIntoHiveTable.scala | 3 +- .../spark/rapids/shims/GpuKnownNullable.scala | 3 +- ...dCreateHiveTableAsSelectCommandShims.scala | 3 +- .../spark/rapids/shims/LogicalPlanShims.scala | 3 +- .../execution/datasources/GpuWriteFiles.scala | 3 +- .../sql/hive/rapids/shims/HiveFileUtil.scala | 3 +- .../rapids/shims/HiveProviderCmdShims.scala | 3 +- .../sql/rapids/GpuFileFormatWriter.scala | 3 +- ...eDataSourceTableAsSelectCommandShims.scala | 3 +- .../sql/rapids/shims/GpuDataSource.scala | 3 +- .../sql/rapids/shims/SchemaUtilsShims.scala | 3 +- .../shims/SparkDateTimeExceptionShims.scala | 3 +- .../shims/SparkUpgradeExceptionShims.scala | 3 +- .../spark/rapids/shims/GetSequenceSize.scala | 3 +- ...eSizeTooLongUnsuccessfulErrorBuilder.scala | 3 +- .../spark/rapids/shims/CastCheckShims.scala | 3 +- .../spark/rapids/shims/GlobalLimitShims.scala | 3 +- .../spark/rapids/shims/GpuBatchScanExec.scala | 3 +- .../rapids/shims/GpuBroadcastJoinMeta.scala | 3 +- .../rapids/shims/OrcProtoWriterShim.scala | 3 +- .../shims/ParquetLegacyNanoAsLongShims.scala | 3 +- .../ParquetTimestampAnnotationShims.scala | 3 +- .../shims/ParquetTimestampNTZShims.scala | 3 +- .../shims/PartitionedFileUtilsShimBase.scala | 3 +- .../rapids/shims/ShuffleOriginUtil.scala | 3 +- .../rapids/shims/Spark340PlusNonDBShims.scala | 3 +- .../shims/TagScanForRuntimeFiltering.scala | 3 +- .../shuffle/RapidsShuffleIterator.scala | 3 +- .../spark/sql/catalyst/csv/GpuCsvUtils.scala | 3 +- .../sql/catalyst/json/GpuJsonUtils.scala | 3 +- .../sql/rapids/RapidsCachingReader.scala | 3 +- .../execution/GpuBroadcastHashJoinExec.scala | 3 +- .../GpuBroadcastNestedLoopJoinExec.scala | 3 +- .../rapids/execution/ShimTrampolineUtil.scala | 3 +- .../rapids/shims/GpuJsonToStructsShim.scala | 3 +- .../shims/RapidsErrorUtils340PlusBase.scala | 3 +- .../shims/GpuAggregateInPandasExecMeta.scala | 3 +- .../rapids/shims/GpuToPrettyString.scala | 3 +- .../shims/GpuWindowGroupLimitExec.scala | 3 +- .../spark/rapids/shims/PythonUDFShim.scala | 3 +- .../execution/rapids/shims/SplitFiles.scala | 3 +- .../hive/rapids/shims/CreateFunctions.scala | 3 +- .../hive/rapids/shims/FileSinkDescShim.scala | 3 +- .../rapids/shims/HiveInspectorsShim.scala | 3 +- .../python/shims/GpuBasePythonRunner.scala | 3 +- .../rapids/shims/DecimalMultiply128.scala | 3 +- .../sql/rapids/shims/RapidsErrorUtils.scala | 3 +- .../rapids/shims/BatchScanExecMeta.scala | 3 +- .../rapids/shims/BatchScanExecMetaBase.scala | 3 +- .../spark/rapids/shims/GpuIntervalUtils.scala | 3 +- .../shims/KeyGroupedPartitioningShim.scala | 3 +- .../shims/LegacyBehaviorPolicyShim.scala | 3 +- .../rapids/shims/NullOutputStreamShim.scala | 3 +- .../shims/PartitionedFileUtilsShim.scala | 3 +- .../shims/PythonMapInArrowExecShims.scala | 3 +- .../spark/rapids/shims/SparkShims.scala | 3 +- .../parquet/rapids/shims/ParquetCVShims.scala | 3 +- .../shims/ShimVectorizedColumnReader.scala | 3 +- .../GpuAtomicCreateTableAsSelectExec.scala | 3 +- .../GpuAtomicReplaceTableAsSelectExec.scala | 3 +- .../rapids/shims/FilePartitionShims.scala | 3 +- .../sql/rapids/execution/GpuShuffleMeta.scala | 3 +- .../sql/rapids/shims/ArrowUtilsShim.scala | 3 +- .../sql/rapids/shims/DataTypeUtilsShim.scala | 3 +- .../rapids/shims/GpuMapInPandasExecMeta.scala | 3 +- .../shims/GpuPythonMapInArrowExecMeta.scala | 3 +- .../rapids/shims/SchemaMetadataShims.scala | 3 +- .../sql/rapids/RapidsShuffleWriter.scala | 4 +- .../spark354/SparkShimServiceProvider.scala | 36 + .../shims/spark354/SparkShimsSuite.scala | 35 + .../spark/rapids/shims/OrcStatisticShim.scala | 3 +- .../sql/rapids/GpuInSubqueryExecSuite.scala | 3 +- .../shuffle/RapidsShuffleTestHelper.scala | 3 +- .../spark/rapids/ToPrettyStringSuite.scala | 3 +- ...eDataSourceTableAsSelectCommandSuite.scala | 3 +- tools/generated_files/354/operatorsScore.csv | 297 +++++++ .../354/supportedDataSource.csv | 13 + tools/generated_files/354/supportedExecs.csv | 57 ++ tools/generated_files/354/supportedExprs.csv | 808 ++++++++++++++++++ 197 files changed, 1666 insertions(+), 189 deletions(-) create mode 100644 sql-plugin/src/main/spark354/scala/com/nvidia/spark/rapids/shims/spark354/SparkShimServiceProvider.scala create mode 100644 sql-plugin/src/test/spark354/scala/com/nvidia/spark/rapids/shims/spark354/SparkShimsSuite.scala create mode 100644 tools/generated_files/354/operatorsScore.csv create mode 100644 tools/generated_files/354/supportedDataSource.csv create mode 100644 tools/generated_files/354/supportedExecs.csv create mode 100644 tools/generated_files/354/supportedExprs.csv diff --git a/datagen/src/main/spark320/scala/org/apache/spark/sql/tests/datagen/DataGenExprShims.scala b/datagen/src/main/spark320/scala/org/apache/spark/sql/tests/datagen/DataGenExprShims.scala index 5ae88606cfb..1abb1664fed 100644 --- a/datagen/src/main/spark320/scala/org/apache/spark/sql/tests/datagen/DataGenExprShims.scala +++ b/datagen/src/main/spark320/scala/org/apache/spark/sql/tests/datagen/DataGenExprShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.tests.datagen diff --git a/docs/download.md b/docs/download.md index 7f5d6cc8852..993c98efa45 100644 --- a/docs/download.md +++ b/docs/download.md @@ -43,7 +43,7 @@ The plugin is tested on the following architectures: 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, 3.3.4 Apache Spark 3.4.0, 3.4.1, 3.4.2, 3.4.3, 3.4.4 - Apache Spark 3.5.0, 3.5.1, 3.5.2, 3.5.3 + Apache Spark 3.5.0, 3.5.1, 3.5.2, 3.5.3, 3.5.4 Supported Databricks runtime versions for Azure and AWS: Databricks 11.3 ML LTS (GPU, Scala 2.12, Spark 3.3.0) diff --git a/pom.xml b/pom.xml index 401ddb601b5..82b4f51004f 100644 --- a/pom.xml +++ b/pom.xml @@ -694,6 +694,27 @@ delta-lake/delta-stub + + release354 + + + buildver + 354 + + + + 354 + ${spark354.version} + ${spark354.version} + 1.13.1 + rapids-4-spark-delta-stub + ${spark330.iceberg.version} + 2.0.7 + + + delta-lake/delta-stub + + diff --git a/scala2.13/pom.xml b/scala2.13/pom.xml index 098463afb8b..a5bd02a71c4 100644 --- a/scala2.13/pom.xml +++ b/scala2.13/pom.xml @@ -694,6 +694,27 @@ delta-lake/delta-stub + + release354 + + + buildver + 354 + + + + 354 + ${spark354.version} + ${spark354.version} + 1.13.1 + rapids-4-spark-delta-stub + ${spark330.iceberg.version} + 2.0.7 + + + delta-lake/delta-stub + + release400 @@ -901,6 +922,7 @@ 3.5.1 3.5.2 3.5.3 + 3.5.4 4.0.0-SNAPSHOT 3.12.4 diff --git a/sql-plugin/src/main/spark320/java/com/nvidia/spark/rapids/shims/ShimSupportsRuntimeFiltering.java b/sql-plugin/src/main/spark320/java/com/nvidia/spark/rapids/shims/ShimSupportsRuntimeFiltering.java index 38ae41e1124..e0206b5622b 100644 --- a/sql-plugin/src/main/spark320/java/com/nvidia/spark/rapids/shims/ShimSupportsRuntimeFiltering.java +++ b/sql-plugin/src/main/spark320/java/com/nvidia/spark/rapids/shims/ShimSupportsRuntimeFiltering.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims; diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala index e0ca739463f..32cbe00bc8c 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/RapidsShuffleManager.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -40,6 +40,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.$_spark.version.classifier_ diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala index 38fb5e2762f..5441f832b38 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/AQEUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -37,6 +37,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/AggregationTagging.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/AggregationTagging.scala index 8c22b51fbd4..4aa128bce34 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/AggregationTagging.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/AggregationTagging.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -37,6 +37,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala index 72760544f06..995c6010bd8 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2024, NVIDIA CORPORATION. + * Copyright (c) 2020-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -39,6 +39,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRowBase.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRowBase.scala index ae689f9e712..bc84442de58 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRowBase.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRowBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2024, NVIDIA CORPORATION. + * Copyright (c) 2020-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -40,6 +40,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/DateTimeUtilsShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/DateTimeUtilsShims.scala index c7d14935e24..d25e7dd35bf 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/DateTimeUtilsShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/DateTimeUtilsShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/DeltaLakeUtils.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/DeltaLakeUtils.scala index 245de9ddd1a..3659485114b 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/DeltaLakeUtils.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/DeltaLakeUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -37,6 +37,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala index 386b1209f76..242adbe1001 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/FileSourceScanExecMeta.scala @@ -37,6 +37,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExecBase.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExecBase.scala index b08c3828683..eb246c4aed7 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExecBase.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExecBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuFileFormatDataWriterShim.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuFileFormatDataWriterShim.scala index 859f80820ce..fefa9595c32 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuFileFormatDataWriterShim.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuFileFormatDataWriterShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -38,6 +38,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala index 245c2501c4d..c838b63dcc2 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -37,6 +37,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader320Plus.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader320Plus.scala index e0a7a910c14..d73d0e5ee96 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader320Plus.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader320Plus.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -37,6 +37,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReaderBase.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReaderBase.scala index 4bf6c9b5967..7b9e2c3fb17 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReaderBase.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReaderBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -40,6 +40,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuParquetCrypto.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuParquetCrypto.scala index 7594a324c47..f343415d8c5 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuParquetCrypto.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuParquetCrypto.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -37,6 +37,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuWindowInPandasExec.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuWindowInPandasExec.scala index a82a0772131..0c59033300b 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuWindowInPandasExec.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuWindowInPandasExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -37,6 +37,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/HashUtils.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/HashUtils.scala index feef4bafe97..44f5dbd6e5d 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/HashUtils.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/HashUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -40,6 +40,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/NullIntolerantShim.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/NullIntolerantShim.scala index b6f8aa79edc..488b1b37660 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/NullIntolerantShim.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/NullIntolerantShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OffsetWindowFunctionMeta.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OffsetWindowFunctionMeta.scala index 549f27b6623..e783a59b3ba 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OffsetWindowFunctionMeta.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OffsetWindowFunctionMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala index e1b27f32d8a..af7c8b5c027 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OrcCastingShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -38,6 +38,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OrcShims320untilAllBase.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OrcShims320untilAllBase.scala index d5aadda6b8a..4842432eee5 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OrcShims320untilAllBase.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/OrcShims320untilAllBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -38,6 +38,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala index 493e488e6a3..afcb9a7d7a4 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/PlanShimsImpl.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -39,6 +39,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala index 331e143ff6b..db9c2c99215 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -39,6 +39,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala index 78303640c82..680b63d0a57 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RebaseShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RebaseShims.scala index 52d1e70aa58..3b2d26fa15c 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RebaseShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RebaseShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -40,6 +40,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimAQEShuffleReadExec.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimAQEShuffleReadExec.scala index 7d4a2ab621f..dcbcbd14fff 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimAQEShuffleReadExec.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimAQEShuffleReadExec.scala @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimBaseSubqueryExec.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimBaseSubqueryExec.scala index db2959ad14d..10efab93fe6 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimBaseSubqueryExec.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimBaseSubqueryExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimBroadcastExchangeLike.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimBroadcastExchangeLike.scala index badfe383018..fae72fd2a61 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimBroadcastExchangeLike.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimBroadcastExchangeLike.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -37,6 +37,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala index a39aa5eddb7..764e9d16673 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimLeafExecNode.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -37,6 +37,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimPredicateHelper.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimPredicateHelper.scala index 272f92113ba..2db639617ee 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimPredicateHelper.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShimPredicateHelper.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShuffleManagerShimUtils.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShuffleManagerShimUtils.scala index 4c04377ff4c..7ea436fe225 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShuffleManagerShimUtils.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ShuffleManagerShimUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/Spark320PlusNonDBShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/Spark320PlusNonDBShims.scala index 9bf1dbc4774..110aea06bc2 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/Spark320PlusNonDBShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/Spark320PlusNonDBShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -36,6 +36,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala index 28497a03f48..f88316723ec 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/Spark320PlusShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/StaticPartitionShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/StaticPartitionShims.scala index a865e0e88c6..c837d2f4433 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/StaticPartitionShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/StaticPartitionShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -39,6 +39,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/TreeNode.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/TreeNode.scala index 6adca2ec151..2aa7a78b803 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/TreeNode.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/TreeNode.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/TypeSigUtil.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/TypeSigUtil.scala index 9f649528d23..802de9ed301 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/TypeSigUtil.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/TypeSigUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/XxHash64Shims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/XxHash64Shims.scala index f470e9868a1..27d82b47a2b 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/XxHash64Shims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/XxHash64Shims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/YearParseUtil.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/YearParseUtil.scala index bef82cfd0b3..2166d5696e6 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/YearParseUtil.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/YearParseUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -40,6 +40,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/extractValueShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/extractValueShims.scala index 96365ee261d..09310d09300 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/extractValueShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/extractValueShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -37,6 +37,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala index 62eab1c6153..0575132d604 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/gpuWindows.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/v1FallbackWriters.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/v1FallbackWriters.scala index f9767504d99..c4f1ab4609f 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/v1FallbackWriters.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/v1FallbackWriters.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/GpuShuffleBlockResolver.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/GpuShuffleBlockResolver.scala index 3d14707c88a..b6d5582fade 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/GpuShuffleBlockResolver.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/GpuShuffleBlockResolver.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala index b4f6246abe1..50d8a242367 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2024, NVIDIA CORPORATION. + * Copyright (c) 2020-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -37,6 +37,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala index 9b63c696371..26bc1fd761a 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/ShuffledBatchRDDUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/storage/ShimDiskBlockManager.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/storage/ShimDiskBlockManager.scala index e65b5da2840..e5ac9a17b80 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/storage/ShimDiskBlockManager.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/rapids/shims/storage/ShimDiskBlockManager.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.rapids.shims.storage diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala index 6f8c7dbfae9..33195590988 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.hive.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/hive/rapids/shims/GpuRowBasedHiveGenericUDFShim.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/hive/rapids/shims/GpuRowBasedHiveGenericUDFShim.scala index 5809d2679ff..29fe4d0ba16 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/hive/rapids/shims/GpuRowBasedHiveGenericUDFShim.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/hive/rapids/shims/GpuRowBasedHiveGenericUDFShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -36,6 +36,7 @@ {"spark": "344"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.hive.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/nvidia/DFUDFShims.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/nvidia/DFUDFShims.scala index 2bc7181cead..f03200eb57d 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/nvidia/DFUDFShims.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/nvidia/DFUDFShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.nvidia diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala index cae48a4d5de..d852ede4907 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -14,6 +14,7 @@ * limitations under the License. */ + /*** spark-rapids-shim-json-lines {"spark": "320"} {"spark": "321"} @@ -40,9 +41,9 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ - package org.apache.spark.sql.rapids import scala.collection.mutable diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala index 01f84cbe1bd..c076a1736af 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -36,6 +36,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonOutput.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonOutput.scala index 119801738c3..0c92b7f7933 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonOutput.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonOutput.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -39,6 +39,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution.python.shims diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala index 4858d496966..f9195e53066 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -39,6 +39,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution.python.shims diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala index 8c6e6bccf24..e9ae7740a65 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -39,6 +39,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution.python.shims diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala index 9818bcd7efb..e376c141451 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -37,6 +37,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution.python.shims diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala index 28099962f64..206f26c1802 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution.python.shims diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/AvroUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/AvroUtils.scala index a5ddd9a5b7f..0f2aeb3eab2 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/AvroUtils.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/AvroUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala index 3705f110cdd..a8d0aa83a8b 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala index 49bf12d5c5e..ddd6c189531 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedWriter.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/Spark32XShimsUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/Spark32XShimsUtils.scala index d86656e3072..0745e8d310c 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/Spark32XShimsUtils.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/Spark32XShimsUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/datetimeExpressions.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/datetimeExpressions.scala index 02705a88fe4..586c54d1af9 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/datetimeExpressions.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/datetimeExpressions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/misc.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/misc.scala index af5d8b000ec..c7d3f19d617 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/misc.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/misc.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -39,6 +39,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala index 451061924ab..c25fe41823c 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/storage/RapidsPushBasedFetchHelper.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.storage diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala index 2cba8ebb6ea..2bb0530c39c 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/storage/RapidsShuffleBlockFetcherIterator.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.storage diff --git a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/Spark321PlusShims.scala b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/Spark321PlusShims.scala index 57fbb1d9052..d9f2ef380ae 100644 --- a/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/Spark321PlusShims.scala +++ b/sql-plugin/src/main/spark321/scala/com/nvidia/spark/rapids/shims/Spark321PlusShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -40,6 +40,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark323/scala/org/apache/spark/sql/rapids/shims/GpuAscii.scala b/sql-plugin/src/main/spark323/scala/org/apache/spark/sql/rapids/shims/GpuAscii.scala index 9c3d699499c..0f3ffee0612 100644 --- a/sql-plugin/src/main/spark323/scala/org/apache/spark/sql/rapids/shims/GpuAscii.scala +++ b/sql-plugin/src/main/spark323/scala/org/apache/spark/sql/rapids/shims/GpuAscii.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilter.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilter.scala index 049ffa97b90..32589c86cd9 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilter.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilter.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilterMightContain.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilterMightContain.scala index ae1fe0b2a47..7fddb7ac817 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilterMightContain.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuBloomFilterMightContain.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuInSubqueryExec.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuInSubqueryExec.scala index c8add7d4538..2a0d96a43ed 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuInSubqueryExec.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/GpuInSubqueryExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -31,6 +31,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/AnsiUtil.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/AnsiUtil.scala index 0c522df0ec7..7bb32ec19fa 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/AnsiUtil.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/AnsiUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BloomFilterShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BloomFilterShims.scala index 647802bf881..24d28500bb0 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BloomFilterShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BloomFilterShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BucketingUtilsShim.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BucketingUtilsShim.scala index e0d8fcf8708..ace1322997f 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BucketingUtilsShim.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/BucketingUtilsShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala index f79479d050d..fae3c1e3e40 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/CharVarcharUtilsShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala index 591da3a8881..00656b1890f 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/DayTimeIntervalShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/DistributionUtil.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/DistributionUtil.scala index 088b254d68b..0487066117a 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/DistributionUtil.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/DistributionUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -31,6 +31,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/FilteredPartitions.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/FilteredPartitions.scala index fd4726a995a..699ff773277 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/FilteredPartitions.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/FilteredPartitions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuDataSourceRDD.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuDataSourceRDD.scala index 8ea99f0248f..fc1aecb6437 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuDataSourceRDD.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuDataSourceRDD.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuHashPartitioning.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuHashPartitioning.scala index 54c1384664c..32bdc257c0e 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuHashPartitioning.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuHashPartitioning.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtilsBase.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtilsBase.scala index 463834fccdc..d312aff7426 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtilsBase.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtilsBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -34,6 +34,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuRangePartitioning.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuRangePartitioning.scala index e800ac54ad5..a583ae5e95c 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuRangePartitioning.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuRangePartitioning.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala index 2bba890ff0d..c255fffedca 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -34,6 +34,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/InSubqueryShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/InSubqueryShims.scala index 0e7b922c614..770ed3eb792 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/InSubqueryShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/InSubqueryShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -31,6 +31,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcReadingShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcReadingShims.scala index 1d28bccc9b1..d16192dec39 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcReadingShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcReadingShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcShims.scala index 65b62dd1bcc..3b0e0993f49 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/OrcShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -32,6 +32,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala index 16ad098f2c8..1cacaf9202d 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -34,6 +34,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index 5845b4805b9..6e5bed9e124 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsFileSourceMetaUtils.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsFileSourceMetaUtils.scala index 32eb915ed65..768601c203b 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsFileSourceMetaUtils.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsFileSourceMetaUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala index 7f0c3d7456d..6e24e4812cd 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala index f65055a63ef..83a791a7adf 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RoundingShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RoundingShims.scala index e5217c8820d..1f1ce7c8ddb 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RoundingShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RoundingShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ScanExecShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ScanExecShims.scala index 0cafb47570a..b80e423b681 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ScanExecShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ScanExecShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusNonDBShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusNonDBShims.scala index 7361b0d512e..53539acf462 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusNonDBShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusNonDBShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -31,6 +31,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusShims.scala index f2f0f970c01..96e9899f8c9 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -31,6 +31,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/RapidsVectorizedColumnReader.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/RapidsVectorizedColumnReader.scala index a14ebb1eb54..bb47e96dc18 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/RapidsVectorizedColumnReader.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/RapidsVectorizedColumnReader.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -34,6 +34,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.datasources.parquet diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/ShimCurrentBatchIterator.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/ShimCurrentBatchIterator.scala index a675e89e769..67ff117d011 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/ShimCurrentBatchIterator.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/execution/datasources/parquet/ShimCurrentBatchIterator.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.datasources.parquet diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/aggregate/GpuBloomFilterAggregate.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/aggregate/GpuBloomFilterAggregate.scala index e9013b243a9..1bfe1be7df3 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/aggregate/GpuBloomFilterAggregate.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/aggregate/GpuBloomFilterAggregate.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.aggregate diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExec.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExec.scala index 5bf0c24f115..9e9d44c2a37 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExec.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -34,6 +34,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor330plus.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor330plus.scala index 1146e74177e..a9f14ac0a5f 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor330plus.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtilsFor330plus.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala index d5442e7eefa..404c8696113 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsShuffleThreadedReader.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/intervalExpressions.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/intervalExpressions.scala index 03783af712d..158a2fba225 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/intervalExpressions.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/intervalExpressions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,6 +35,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala index 3b605e88ce8..ba4484ff605 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/AnsiCastShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/CastingConfigShim.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/CastingConfigShim.scala index 7cc9e7496d0..2bc4ef0ced8 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/CastingConfigShim.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/CastingConfigShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -28,6 +28,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ColumnDefaultValuesShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ColumnDefaultValuesShims.scala index a2dbc63a056..ce131b9e05f 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ColumnDefaultValuesShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ColumnDefaultValuesShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DecimalArithmeticOverrides.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DecimalArithmeticOverrides.scala index c9605263edc..f3e8f17a526 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DecimalArithmeticOverrides.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/DecimalArithmeticOverrides.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -28,6 +28,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GetMapValueMeta.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GetMapValueMeta.scala index 7db246c653a..5aa3fe99fe6 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GetMapValueMeta.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GetMapValueMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -28,6 +28,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala index eee11aa32e2..20cd6b39f06 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -28,6 +28,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ParquetStringPredShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ParquetStringPredShims.scala index c1bca25b29d..16d4ea5441f 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ParquetStringPredShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ParquetStringPredShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -28,6 +28,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimFilePartitionReaderFactory.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimFilePartitionReaderFactory.scala index 65127a01f5e..40ce563908f 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimFilePartitionReaderFactory.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ShimFilePartitionReaderFactory.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -28,6 +28,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/TypeUtilsShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/TypeUtilsShims.scala index 39a72f9e6ce..7f771991cfa 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/TypeUtilsShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/TypeUtilsShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -28,6 +28,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/rapids/DataSourceStrategyUtils.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/rapids/DataSourceStrategyUtils.scala index 9f9dd92711c..b89fc88f453 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/rapids/DataSourceStrategyUtils.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/execution/datasources/rapids/DataSourceStrategyUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -28,6 +28,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.datasources.rapids diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/GpuCheckOverflowInTableInsert.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/GpuCheckOverflowInTableInsert.scala index e2a30be6f29..add45497ed4 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/GpuCheckOverflowInTableInsert.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/GpuCheckOverflowInTableInsert.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -33,6 +33,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/aggregate/aggregateFunctions.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/aggregate/aggregateFunctions.scala index 0f48f1df5c2..febb969667b 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/aggregate/aggregateFunctions.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/aggregate/aggregateFunctions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -28,6 +28,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.aggregate diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala index c0b62c6bffb..b15eed388f6 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -28,6 +28,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids diff --git a/sql-plugin/src/main/spark331/scala/com/nvidia/spark/rapids/shims/Spark331PlusNonDBShims.scala b/sql-plugin/src/main/spark331/scala/com/nvidia/spark/rapids/shims/Spark331PlusNonDBShims.scala index d59b3e33dd6..a8e61599f25 100644 --- a/sql-plugin/src/main/spark331/scala/com/nvidia/spark/rapids/shims/Spark331PlusNonDBShims.scala +++ b/sql-plugin/src/main/spark331/scala/com/nvidia/spark/rapids/shims/Spark331PlusNonDBShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -29,6 +29,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala index 09049a3a432..673a9445d50 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala index 43c515998a8..9d3fe5c8c0e 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.hive.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuKnownNullable.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuKnownNullable.scala index 225f0bc87ca..6f8e7a96a7f 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuKnownNullable.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuKnownNullable.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala index e987a940b70..f5e0e12e244 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/LogicalPlanShims.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/LogicalPlanShims.scala index 8f98321b4ba..f301686e305 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/LogicalPlanShims.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/LogicalPlanShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala index abcf6178248..04c49fb13fc 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.datasources diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveFileUtil.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveFileUtil.scala index 41a239ce0ab..a1673bb6db1 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveFileUtil.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveFileUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.hive.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveProviderCmdShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveProviderCmdShims.scala index ceb6ae8bc8e..501bab1fbcb 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveProviderCmdShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/hive/rapids/shims/HiveProviderCmdShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.hive.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala index 6da9ce792ab..12f06904984 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala index 3c950a3836b..97703b3b238 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuDataSource.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuDataSource.scala index 758a6826320..cf8339d3298 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuDataSource.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuDataSource.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaUtilsShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaUtilsShims.scala index 4e6f890c178..4ff4e6c2fe9 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaUtilsShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SchemaUtilsShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkDateTimeExceptionShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkDateTimeExceptionShims.scala index fd706764e23..e3c093957ce 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkDateTimeExceptionShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkDateTimeExceptionShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkUpgradeExceptionShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkUpgradeExceptionShims.scala index 507d7c97913..1bf0babb954 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkUpgradeExceptionShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/SparkUpgradeExceptionShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark334/scala/com/nvidia/spark/rapids/shims/GetSequenceSize.scala b/sql-plugin/src/main/spark334/scala/com/nvidia/spark/rapids/shims/GetSequenceSize.scala index 07da94bcf07..ab17ef1cc11 100644 --- a/sql-plugin/src/main/spark334/scala/com/nvidia/spark/rapids/shims/GetSequenceSize.scala +++ b/sql-plugin/src/main/spark334/scala/com/nvidia/spark/rapids/shims/GetSequenceSize.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -22,6 +22,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark334/scala/org/apache/spark/sql/rapids/shims/SequenceSizeTooLongUnsuccessfulErrorBuilder.scala b/sql-plugin/src/main/spark334/scala/org/apache/spark/sql/rapids/shims/SequenceSizeTooLongUnsuccessfulErrorBuilder.scala index 8d590a54075..810a7c06ba9 100644 --- a/sql-plugin/src/main/spark334/scala/org/apache/spark/sql/rapids/shims/SequenceSizeTooLongUnsuccessfulErrorBuilder.scala +++ b/sql-plugin/src/main/spark334/scala/org/apache/spark/sql/rapids/shims/SequenceSizeTooLongUnsuccessfulErrorBuilder.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -22,6 +22,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/CastCheckShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/CastCheckShims.scala index fb36768c880..eb8ecc5784b 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/CastCheckShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/CastCheckShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GlobalLimitShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GlobalLimitShims.scala index 9ef5f49aa9f..9e57676a805 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GlobalLimitShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GlobalLimitShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,6 +26,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index a341b145849..b0546682509 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -25,6 +25,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBroadcastJoinMeta.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBroadcastJoinMeta.scala index 8f96fddbbb7..c2675f65200 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBroadcastJoinMeta.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBroadcastJoinMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -23,6 +23,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/OrcProtoWriterShim.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/OrcProtoWriterShim.scala index ecc245ca54e..d4ce183dd92 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/OrcProtoWriterShim.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/OrcProtoWriterShim.scala @@ -1,6 +1,6 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetLegacyNanoAsLongShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetLegacyNanoAsLongShims.scala index 4a7cf9d9797..85dd780fe99 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetLegacyNanoAsLongShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetLegacyNanoAsLongShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,6 +26,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetTimestampAnnotationShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetTimestampAnnotationShims.scala index a8f8cf56e0e..1bef4fd613b 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetTimestampAnnotationShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetTimestampAnnotationShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,6 +26,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetTimestampNTZShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetTimestampNTZShims.scala index 8df78afc689..1d7dff124a8 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetTimestampNTZShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ParquetTimestampNTZShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,6 +26,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShimBase.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShimBase.scala index 2cb0af118eb..6072959b20a 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShimBase.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShimBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,6 +24,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ShuffleOriginUtil.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ShuffleOriginUtil.scala index bd5c71d40a7..65333bad548 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ShuffleOriginUtil.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/ShuffleOriginUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,6 +24,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/Spark340PlusNonDBShims.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/Spark340PlusNonDBShims.scala index f131c1303af..b1706f72d36 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/Spark340PlusNonDBShims.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/Spark340PlusNonDBShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,6 +24,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/TagScanForRuntimeFiltering.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/TagScanForRuntimeFiltering.scala index 80abfb4b730..b6ac90a7223 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/TagScanForRuntimeFiltering.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/TagScanForRuntimeFiltering.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,6 +26,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala index 868e5492e2b..0476f6f0fad 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,6 +26,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shuffle diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/csv/GpuCsvUtils.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/csv/GpuCsvUtils.scala index e011cedcba9..0d3bda9c463 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/csv/GpuCsvUtils.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/csv/GpuCsvUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,6 +26,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.catalyst.csv diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/json/GpuJsonUtils.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/json/GpuJsonUtils.scala index 1183725b5a2..447c8100149 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/json/GpuJsonUtils.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/catalyst/json/GpuJsonUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,6 +26,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.catalyst.json diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala index bc962e1bf5c..7547ebfc8de 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,6 +26,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids 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 d4531cf5d7f..1fdf8fa24f2 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 @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,6 +24,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala index 66e1e2d5e4a..ec06d52ca90 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,6 +24,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/execution/ShimTrampolineUtil.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/execution/ShimTrampolineUtil.scala index a88c31e6985..c3ef0de325b 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/execution/ShimTrampolineUtil.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/execution/ShimTrampolineUtil.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,6 +24,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala index f3e37aac24b..65a81f9e10e 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/GpuJsonToStructsShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -25,6 +25,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils340PlusBase.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils340PlusBase.scala index c344b3a365c..01dc896bcb5 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils340PlusBase.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils340PlusBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,6 +24,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuAggregateInPandasExecMeta.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuAggregateInPandasExecMeta.scala index b4b80da5b3a..8f8633e2abc 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuAggregateInPandasExecMeta.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuAggregateInPandasExecMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala index 18de0caa030..1e57ba5d32b 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuWindowGroupLimitExec.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuWindowGroupLimitExec.scala index eca69254d61..cb2aa5cbaa8 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuWindowGroupLimitExec.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/GpuWindowGroupLimitExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PythonUDFShim.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PythonUDFShim.scala index f8ea26b2876..52f7eac9443 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PythonUDFShim.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PythonUDFShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/SplitFiles.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/SplitFiles.scala index ad9d9818bde..6cd653bae9f 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/SplitFiles.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/SplitFiles.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -22,6 +22,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.rapids.shims diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/CreateFunctions.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/CreateFunctions.scala index 0f459a8418c..28e8547560a 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/CreateFunctions.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/CreateFunctions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.hive.rapids.shims diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/FileSinkDescShim.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/FileSinkDescShim.scala index 2fee066a722..7a6c5e4ee97 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/FileSinkDescShim.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/FileSinkDescShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -22,6 +22,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.hive.rapids.shims diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/HiveInspectorsShim.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/HiveInspectorsShim.scala index 54b6e469089..9d7b34ab110 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/HiveInspectorsShim.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/hive/rapids/shims/HiveInspectorsShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -22,6 +22,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.hive.rapids.shims diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala index 91872ae3ac6..931e52ccebb 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuBasePythonRunner.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution.python.shims diff --git a/sql-plugin/src/main/spark342/scala/com/nvidia/spark/rapids/shims/DecimalMultiply128.scala b/sql-plugin/src/main/spark342/scala/com/nvidia/spark/rapids/shims/DecimalMultiply128.scala index e6cd86e60f8..74bda17210f 100644 --- a/sql-plugin/src/main/spark342/scala/com/nvidia/spark/rapids/shims/DecimalMultiply128.scala +++ b/sql-plugin/src/main/spark342/scala/com/nvidia/spark/rapids/shims/DecimalMultiply128.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -22,6 +22,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark342/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark342/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index d67a0d7aee6..b8b51eece15 100644 --- a/sql-plugin/src/main/spark342/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark342/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala index fba9487a11f..c39f9306da3 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMetaBase.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMetaBase.scala index d87c7d5ccbf..f86da0df7fa 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMetaBase.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMetaBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,6 +20,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtils.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtils.scala index 9fe9e0b4d49..bba71174380 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtils.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,6 +18,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala index 910e6be1655..bce298add52 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,6 +18,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/LegacyBehaviorPolicyShim.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/LegacyBehaviorPolicyShim.scala index aa81ca0934f..af73434777e 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/LegacyBehaviorPolicyShim.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/LegacyBehaviorPolicyShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,6 +20,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/NullOutputStreamShim.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/NullOutputStreamShim.scala index 25486463349..a4d2b69c297 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/NullOutputStreamShim.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/NullOutputStreamShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,6 +20,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala index a44ba6db894..a5bd5d1ca27 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala index 9dd96079bd4..b87877623d7 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/SparkShims.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/SparkShims.scala index 63281ae04e0..c060359d3df 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/SparkShims.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/SparkShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ParquetCVShims.scala b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ParquetCVShims.scala index 322bbbad8be..5d52d3bf134 100644 --- a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ParquetCVShims.scala +++ b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ParquetCVShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.datasources.parquet diff --git a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimVectorizedColumnReader.scala b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimVectorizedColumnReader.scala index 21b597514e0..25beef93e01 100644 --- a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimVectorizedColumnReader.scala +++ b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/datasources/parquet/rapids/shims/ShimVectorizedColumnReader.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.datasources.parquet.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicCreateTableAsSelectExec.scala b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicCreateTableAsSelectExec.scala index cb6b8587915..203b2c03d78 100644 --- a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicCreateTableAsSelectExec.scala +++ b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicCreateTableAsSelectExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.datasources.v2.rapids diff --git a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicReplaceTableAsSelectExec.scala b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicReplaceTableAsSelectExec.scala index 4a86c7b6987..b180d884405 100644 --- a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicReplaceTableAsSelectExec.scala +++ b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/datasources/v2/rapids/GpuAtomicReplaceTableAsSelectExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.datasources.v2.rapids diff --git a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala index c3ad3bd5f43..613b3606532 100644 --- a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala +++ b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/execution/rapids/shims/FilePartitionShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/execution/GpuShuffleMeta.scala b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/execution/GpuShuffleMeta.scala index 36cc9aade00..b1effa9cc2a 100644 --- a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/execution/GpuShuffleMeta.scala +++ b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/execution/GpuShuffleMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.execution diff --git a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/ArrowUtilsShim.scala b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/ArrowUtilsShim.scala index 951d61769cb..ed1052e36cf 100644 --- a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/ArrowUtilsShim.scala +++ b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/ArrowUtilsShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,6 +20,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/DataTypeUtilsShim.scala b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/DataTypeUtilsShim.scala index a774c05cb5d..3c6aefb706a 100644 --- a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/DataTypeUtilsShim.scala +++ b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/DataTypeUtilsShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,6 +20,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/GpuMapInPandasExecMeta.scala b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/GpuMapInPandasExecMeta.scala index a34b65d6377..6cb5908190c 100644 --- a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/GpuMapInPandasExecMeta.scala +++ b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/GpuMapInPandasExecMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExecMeta.scala b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExecMeta.scala index eb560766a82..34fb5ba11f0 100644 --- a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExecMeta.scala +++ b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExecMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/SchemaMetadataShims.scala b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/SchemaMetadataShims.scala index 8bba14248f6..b856add7d75 100644 --- a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/SchemaMetadataShims.scala +++ b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/SchemaMetadataShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala index e333d6f3f0c..db7c64f2059 100644 --- a/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala +++ b/sql-plugin/src/main/spark350db143/scala/org/apache/spark/sql/rapids/RapidsShuffleWriter.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -14,10 +14,10 @@ * limitations under the License. */ + /*** spark-rapids-shim-json-lines {"spark": "350db143"} spark-rapids-shim-json-lines ***/ - package org.apache.spark.sql.rapids import scala.collection.mutable diff --git a/sql-plugin/src/main/spark354/scala/com/nvidia/spark/rapids/shims/spark354/SparkShimServiceProvider.scala b/sql-plugin/src/main/spark354/scala/com/nvidia/spark/rapids/shims/spark354/SparkShimServiceProvider.scala new file mode 100644 index 00000000000..46d9887b0c8 --- /dev/null +++ b/sql-plugin/src/main/spark354/scala/com/nvidia/spark/rapids/shims/spark354/SparkShimServiceProvider.scala @@ -0,0 +1,36 @@ +/* + * Copyright (c) 2025, 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "354"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims.spark354 + +import com.nvidia.spark.rapids.SparkShimVersion + +object SparkShimServiceProvider { + val VERSION = SparkShimVersion(3, 5, 4) + val VERSIONNAMES = Seq(s"$VERSION") +} + +class SparkShimServiceProvider extends com.nvidia.spark.rapids.SparkShimServiceProvider { + + override def getShimVersion: SparkShimVersion = SparkShimServiceProvider.VERSION + + override def matchesVersion(version: String): Boolean = { + SparkShimServiceProvider.VERSIONNAMES.contains(version) + } +} diff --git a/sql-plugin/src/test/spark354/scala/com/nvidia/spark/rapids/shims/spark354/SparkShimsSuite.scala b/sql-plugin/src/test/spark354/scala/com/nvidia/spark/rapids/shims/spark354/SparkShimsSuite.scala new file mode 100644 index 00000000000..842ba0afcf0 --- /dev/null +++ b/sql-plugin/src/test/spark354/scala/com/nvidia/spark/rapids/shims/spark354/SparkShimsSuite.scala @@ -0,0 +1,35 @@ +/* + * Copyright (c) 2025, 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "354"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims.spark354 + +import com.nvidia.spark.rapids._ +import org.scalatest.funsuite.AnyFunSuite + +class SparkShimsSuite extends AnyFunSuite with FQSuiteName { + test("spark shims version") { + assert(ShimLoader.getShimVersion === SparkShimVersion(3, 5, 4)) + } + + test("shuffle manager class") { + assert(ShimLoader.getRapidsShuffleManagerClass === + classOf[com.nvidia.spark.rapids.spark354.RapidsShuffleManager].getCanonicalName) + } + +} diff --git a/tests/src/test/spark320/scala/com/nvidia/spark/rapids/shims/OrcStatisticShim.scala b/tests/src/test/spark320/scala/com/nvidia/spark/rapids/shims/OrcStatisticShim.scala index b3b32ccb9f2..2a1bbef149d 100644 --- a/tests/src/test/spark320/scala/com/nvidia/spark/rapids/shims/OrcStatisticShim.scala +++ b/tests/src/test/spark320/scala/com/nvidia/spark/rapids/shims/OrcStatisticShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -38,6 +38,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/GpuInSubqueryExecSuite.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/GpuInSubqueryExecSuite.scala index 89262ef5bbe..6d2608be3e0 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/GpuInSubqueryExecSuite.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/GpuInSubqueryExecSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -31,6 +31,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids diff --git a/tests/src/test/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala b/tests/src/test/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala index 0efcb4f1d7d..e13311d1627 100644 --- a/tests/src/test/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala +++ b/tests/src/test/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTestHelper.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,6 +26,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shuffle diff --git a/tests/src/test/spark341db/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala b/tests/src/test/spark341db/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala index 7e802d89764..e5829730785 100644 --- a/tests/src/test/spark341db/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala +++ b/tests/src/test/spark341db/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,6 +21,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids diff --git a/tests/src/test/spark350/scala/org/apache/spark/sql/rapids/GpuCreateDataSourceTableAsSelectCommandSuite.scala b/tests/src/test/spark350/scala/org/apache/spark/sql/rapids/GpuCreateDataSourceTableAsSelectCommandSuite.scala index f95d8862a30..2bbbfc148f5 100644 --- a/tests/src/test/spark350/scala/org/apache/spark/sql/rapids/GpuCreateDataSourceTableAsSelectCommandSuite.scala +++ b/tests/src/test/spark350/scala/org/apache/spark/sql/rapids/GpuCreateDataSourceTableAsSelectCommandSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ {"spark": "351"} {"spark": "352"} {"spark": "353"} +{"spark": "354"} {"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids diff --git a/tools/generated_files/354/operatorsScore.csv b/tools/generated_files/354/operatorsScore.csv new file mode 100644 index 00000000000..cfb5b486942 --- /dev/null +++ b/tools/generated_files/354/operatorsScore.csv @@ -0,0 +1,297 @@ +CPUOperator,Score +CoalesceExec,3.0 +CollectLimitExec,3.0 +ExpandExec,3.0 +FileSourceScanExec,3.0 +FilterExec,2.8 +GenerateExec,3.0 +GlobalLimitExec,3.0 +LocalLimitExec,3.0 +ProjectExec,3.0 +RangeExec,3.0 +SampleExec,3.0 +SortExec,8.0 +SubqueryBroadcastExec,3.0 +TakeOrderedAndProjectExec,3.0 +UnionExec,3.0 +AQEShuffleReadExec,3.0 +HashAggregateExec,4.5 +ObjectHashAggregateExec,3.0 +SortAggregateExec,3.0 +InMemoryTableScanExec,3.0 +DataWritingCommandExec,3.0 +ExecutedCommandExec,3.0 +WriteFilesExec,3.0 +AppendDataExecV1,3.0 +AtomicCreateTableAsSelectExec,3.0 +AtomicReplaceTableAsSelectExec,3.0 +BatchScanExec,3.0 +OverwriteByExpressionExecV1,3.0 +BroadcastExchangeExec,3.0 +ShuffleExchangeExec,4.2 +BroadcastHashJoinExec,5.1 +BroadcastNestedLoopJoinExec,3.0 +CartesianProductExec,3.0 +ShuffledHashJoinExec,3.0 +SortMergeJoinExec,22.7 +AggregateInPandasExec,1.2 +ArrowEvalPythonExec,1.2 +FlatMapCoGroupsInPandasExec,3.0 +FlatMapGroupsInPandasExec,1.2 +MapInPandasExec,1.2 +PythonMapInArrowExec,3.0 +WindowInPandasExec,1.2 +WindowExec,3.0 +WindowGroupLimitExec,3.0 +HiveTableScanExec,3.0 +Abs,4 +Acos,4 +Acosh,4 +Add,4 +AggregateExpression,4 +Alias,4 +And,4 +ApproximatePercentile,4 +ArrayContains,4 +ArrayExcept,4 +ArrayExists,4 +ArrayFilter,4 +ArrayIntersect,4 +ArrayJoin,4 +ArrayMax,4 +ArrayMin,4 +ArrayRemove,4 +ArrayRepeat,4 +ArrayTransform,4 +ArrayUnion,4 +ArraysOverlap,4 +ArraysZip,4 +Ascii,4 +Asin,4 +Asinh,4 +AtLeastNNonNulls,4 +Atan,4 +Atanh,4 +AttributeReference,4 +Average,4 +BRound,4 +BitLength,4 +BitwiseAnd,4 +BitwiseNot,4 +BitwiseOr,4 +BitwiseXor,4 +BloomFilterAggregate,4 +BloomFilterMightContain,4 +BoundReference,4 +CaseWhen,4 +Cbrt,4 +Ceil,4 +CheckOverflowInTableInsert,4 +Coalesce,4 +CollectList,4 +CollectSet,4 +Concat,4 +ConcatWs,4 +Contains,4 +Conv,4 +Cos,4 +Cosh,4 +Cot,4 +Count,4 +CreateArray,4 +CreateMap,4 +CreateNamedStruct,4 +CurrentRow$,4 +DateAdd,4 +DateAddInterval,4 +DateDiff,4 +DateFormatClass,4 +DateSub,4 +DayOfMonth,4 +DayOfWeek,4 +DayOfYear,4 +DenseRank,4 +Divide,4 +DivideDTInterval,4 +DivideYMInterval,4 +DynamicPruningExpression,4 +ElementAt,4 +Empty2Null,4 +EndsWith,4 +EqualNullSafe,4 +EqualTo,4 +Exp,4 +Explode,4 +Expm1,4 +First,4 +Flatten,4 +Floor,4 +FormatNumber,4 +FromUTCTimestamp,4 +FromUnixTime,4 +GetArrayItem,4 +GetArrayStructFields,4 +GetJsonObject,4 +GetMapValue,4 +GetStructField,4 +GetTimestamp,4 +GreaterThan,4 +GreaterThanOrEqual,4 +Greatest,4 +HiveGenericUDF,4 +HiveHash,4 +HiveSimpleUDF,4 +Hour,4 +Hypot,4 +If,4 +In,4 +InSet,4 +InSubqueryExec,4 +InitCap,4 +InputFileBlockLength,4 +InputFileBlockStart,4 +InputFileName,4 +IntegralDivide,4 +IsNaN,4 +IsNotNull,4 +IsNull,4 +JsonToStructs,4 +JsonTuple,4 +KnownFloatingPointNormalized,4 +KnownNotNull,4 +KnownNullable,4 +Lag,4 +LambdaFunction,4 +Last,4 +LastDay,4 +Lead,4 +Least,4 +Length,4 +LessThan,4 +LessThanOrEqual,4 +Like,4 +Literal,4 +Log,4 +Log10,4 +Log1p,4 +Log2,4 +Logarithm,4 +Lower,4 +MakeDecimal,4 +MapConcat,4 +MapEntries,4 +MapFilter,4 +MapFromArrays,4 +MapKeys,4 +MapValues,4 +Max,4 +MaxBy,4 +Md5,4 +MicrosToTimestamp,4 +MillisToTimestamp,4 +Min,4 +MinBy,4 +Minute,4 +MonotonicallyIncreasingID,4 +Month,4 +MonthsBetween,4 +Multiply,4 +MultiplyDTInterval,4 +MultiplyYMInterval,4 +Murmur3Hash,4 +NaNvl,4 +NamedLambdaVariable,4 +NormalizeNaNAndZero,4 +Not,4 +NthValue,4 +OctetLength,4 +Or,4 +ParseUrl,4 +PercentRank,4 +Percentile,4 +PivotFirst,4 +Pmod,4 +PosExplode,4 +Pow,4 +PreciseTimestampConversion,4 +PythonUDAF,4 +PythonUDF,4 +Quarter,4 +RLike,4 +RaiseError,4 +Rand,4 +Rank,4 +RegExpExtract,4 +RegExpExtractAll,4 +RegExpReplace,4 +Remainder,4 +ReplicateRows,4 +Reverse,4 +Rint,4 +Round,4 +RoundCeil,4 +RoundFloor,4 +RowNumber,4 +ScalaUDF,4 +ScalarSubquery,4 +Second,4 +SecondsToTimestamp,4 +Sequence,4 +ShiftLeft,4 +ShiftRight,4 +ShiftRightUnsigned,4 +Signum,4 +Sin,4 +Sinh,4 +Size,4 +SortArray,4 +SortOrder,4 +SparkPartitionID,4 +SpecifiedWindowFrame,4 +Sqrt,4 +Stack,4 +StartsWith,4 +StddevPop,4 +StddevSamp,4 +StringInstr,4 +StringLPad,4 +StringLocate,4 +StringRPad,4 +StringRepeat,4 +StringReplace,4 +StringSplit,4 +StringToMap,4 +StringTranslate,4 +StringTrim,4 +StringTrimLeft,4 +StringTrimRight,4 +StructsToJson,4 +Substring,4 +SubstringIndex,4 +Subtract,4 +Sum,4 +Tan,4 +Tanh,4 +TimeAdd,4 +ToDegrees,4 +ToRadians,4 +ToUTCTimestamp,4 +ToUnixTimestamp,4 +TransformKeys,4 +TransformValues,4 +TruncDate,4 +TruncTimestamp,4 +UnaryMinus,4 +UnaryPositive,4 +UnboundedFollowing$,4 +UnboundedPreceding$,4 +UnixTimestamp,4 +UnscaledValue,4 +Upper,4 +VariancePop,4 +VarianceSamp,4 +WeekDay,4 +WindowExpression,4 +WindowSpecDefinition,4 +XxHash64,4 +Year,4 diff --git a/tools/generated_files/354/supportedDataSource.csv b/tools/generated_files/354/supportedDataSource.csv new file mode 100644 index 00000000000..82df521b39b --- /dev/null +++ b/tools/generated_files/354/supportedDataSource.csv @@ -0,0 +1,13 @@ +Format,Direction,BOOLEAN,BYTE,SHORT,INT,LONG,FLOAT,DOUBLE,DATE,TIMESTAMP,STRING,DECIMAL,NULL,BINARY,CALENDAR,ARRAY,MAP,STRUCT,UDT,DAYTIME,YEARMONTH +Avro,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO +CSV,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,NA,NA,NA,NA,NA,NA +Delta,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S +Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS +HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S +JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA +ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA +ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S +Parquet,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/tools/generated_files/354/supportedExecs.csv b/tools/generated_files/354/supportedExecs.csv new file mode 100644 index 00000000000..409fa3e45aa --- /dev/null +++ b/tools/generated_files/354/supportedExecs.csv @@ -0,0 +1,57 @@ +Exec,Supported,Notes,Params,BOOLEAN,BYTE,SHORT,INT,LONG,FLOAT,DOUBLE,DATE,TIMESTAMP,STRING,DECIMAL,NULL,BINARY,CALENDAR,ARRAY,MAP,STRUCT,UDT,DAYTIME,YEARMONTH +CoalesceExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +CollectLimitExec,NS,This is disabled by default because Collect Limit replacement can be slower on the GPU; if huge number of rows in a batch it could help by limiting the number of rows transferred from GPU to CPU,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +ExpandExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +FileSourceScanExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +FilterExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +GenerateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +GlobalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S +SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +UnionExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +AQEShuffleReadExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +HashAggregateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,PS,NS,PS,PS,PS,NS,NS,NS +ObjectHashAggregateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,PS,NS,PS,PS,PS,NS,NS,NS +SortAggregateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,PS,NS,PS,PS,PS,NS,NS,NS +InMemoryTableScanExec,NS,This is disabled by default because there could be complications when using it with AQE with Spark-3.5.0 and Spark-3.5.1. For more details please check https://github.com/NVIDIA/spark-rapids/issues/10603,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,PS,PS,PS,NS,S,S +DataWritingCommandExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,PS,NS,S,NS,PS,PS,PS,NS,S,S +ExecutedCommandExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +WriteFilesExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +AppendDataExecV1,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,S,NS,PS,PS,PS,NS,S,S +AtomicCreateTableAsSelectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,S,NS,PS,PS,PS,NS,S,S +AtomicReplaceTableAsSelectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,S,NS,PS,PS,PS,NS,S,S +BatchScanExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,S,NS,PS,PS,PS,NS,S,S +OverwriteByExpressionExecV1,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,S,NS,PS,PS,PS,NS,S,S +BroadcastExchangeExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +ShuffleExchangeExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +BroadcastHashJoinExec,S,None,leftKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NS,NS +BroadcastHashJoinExec,S,None,rightKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NS,NS +BroadcastHashJoinExec,S,None,condition,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BroadcastHashJoinExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +BroadcastNestedLoopJoinExec,S,None,condition(A non-inner join only is supported if the condition expression can be converted to a GPU AST expression),S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BroadcastNestedLoopJoinExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +CartesianProductExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +ShuffledHashJoinExec,S,None,leftKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NS,NS +ShuffledHashJoinExec,S,None,rightKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NS,NS +ShuffledHashJoinExec,S,None,condition,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShuffledHashJoinExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +SortMergeJoinExec,S,None,leftKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NS,NS +SortMergeJoinExec,S,None,rightKeys,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NS,NS +SortMergeJoinExec,S,None,condition,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SortMergeJoinExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +AggregateInPandasExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS +ArrowEvalPythonExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +FlatMapCoGroupsInPandasExec,NS,This is disabled by default because Performance is not ideal with many small groups,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS +FlatMapGroupsInPandasExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS +MapInPandasExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonMapInArrowExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +WindowInPandasExec,NS,This is disabled by default because it only supports row based frame for now,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,NS,NS,NS,NS +WindowExec,S,None,partitionSpec,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS +WindowExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +WindowGroupLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +HiveTableScanExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS diff --git a/tools/generated_files/354/supportedExprs.csv b/tools/generated_files/354/supportedExprs.csv new file mode 100644 index 00000000000..9e3939f4566 --- /dev/null +++ b/tools/generated_files/354/supportedExprs.csv @@ -0,0 +1,808 @@ +Expression,Supported,SQL Func,Notes,Context,Params,BOOLEAN,BYTE,SHORT,INT,LONG,FLOAT,DOUBLE,DATE,TIMESTAMP,STRING,DECIMAL,NULL,BINARY,CALENDAR,ARRAY,MAP,STRUCT,UDT,DAYTIME,YEARMONTH +Abs,S,`abs`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,S,S +Abs,S,`abs`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,S,S +Abs,S,`abs`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NS,NS +Abs,S,`abs`,None,AST,result,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NS,NS +Acos,S,`acos`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Acos,S,`acos`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Acos,S,`acos`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Acos,S,`acos`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Acosh,S,`acosh`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Acosh,S,`acosh`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Acosh,S,`acosh`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Acosh,S,`acosh`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Add,S,`+`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +Add,S,`+`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +Add,S,`+`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +Add,S,`+`,None,AST,lhs,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS +Add,S,`+`,None,AST,rhs,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS +Add,S,`+`,None,AST,result,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS +Alias,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +Alias,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +Alias,S, ,None,AST,input,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS,S,S +Alias,S, ,None,AST,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS,S,S +And,S,`and`,None,project,lhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +And,S,`and`,None,project,rhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +And,S,`and`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +And,S,`and`,None,AST,lhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +And,S,`and`,None,AST,rhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +And,S,`and`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ArrayContains,S,`array_contains`,None,project,array,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayContains,S,`array_contains`,None,project,key,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS +ArrayContains,S,`array_contains`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ArrayExcept,S,`array_except`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,array1,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayExcept,S,`array_except`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,array2,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayExcept,S,`array_except`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayExists,S,`exists`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayExists,S,`exists`,None,project,function,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ArrayExists,S,`exists`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ArrayFilter,S,`filter`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayFilter,S,`filter`,None,project,function,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ArrayFilter,S,`filter`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayIntersect,S,`array_intersect`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,array1,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayIntersect,S,`array_intersect`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,array2,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayIntersect,S,`array_intersect`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayJoin,S,`array_join`,None,project,array,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +ArrayJoin,S,`array_join`,None,project,delimiter,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ArrayJoin,S,`array_join`,None,project,nullReplacement,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ArrayJoin,S,`array_join`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ArrayMax,S,`array_max`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayMax,S,`array_max`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +ArrayMin,S,`array_min`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayMin,S,`array_min`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +ArrayRemove,S,`array_remove`,None,project,array,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS,PS,NS,NS,NS,NS,NS +ArrayRemove,S,`array_remove`,None,project,element,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +ArrayRemove,S,`array_remove`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayRepeat,S,`array_repeat`,None,project,left,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +ArrayRepeat,S,`array_repeat`,None,project,right,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ArrayRepeat,S,`array_repeat`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayTransform,S,`transform`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayTransform,S,`transform`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +ArrayTransform,S,`transform`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayUnion,S,`array_union`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,array1,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayUnion,S,`array_union`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,array2,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArrayUnion,S,`array_union`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArraysOverlap,S,`arrays_overlap`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,array1,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArraysOverlap,S,`arrays_overlap`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,array2,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArraysOverlap,S,`arrays_overlap`,This is not 100% compatible with the Spark version because the GPU implementation treats -0.0 and 0.0 as equal; but the CPU implementation currently does not (see SPARK-39845). Also; Apache Spark 3.1.3 fixed issue SPARK-36741 where NaNs in these set like operators were not treated as being equal. We have chosen to break with compatibility for the older versions of Spark in this instance and handle NaNs the same as 3.1.3+,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ArraysZip,S,`arrays_zip`,None,project,children,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ArraysZip,S,`arrays_zip`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Ascii,NS,`ascii`,This is disabled by default because it only supports strings starting with ASCII or Latin-1 characters after Spark 3.2.3; 3.3.1 and 3.4.0. Otherwise the results will not match the CPU.,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Ascii,NS,`ascii`,This is disabled by default because it only supports strings starting with ASCII or Latin-1 characters after Spark 3.2.3; 3.3.1 and 3.4.0. Otherwise the results will not match the CPU.,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Asin,S,`asin`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Asin,S,`asin`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Asin,S,`asin`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Asin,S,`asin`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Asinh,S,`asinh`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Asinh,S,`asinh`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Asinh,S,`asinh`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Asinh,S,`asinh`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +AtLeastNNonNulls,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +AtLeastNNonNulls,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Atan,S,`atan`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Atan,S,`atan`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Atan,S,`atan`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Atan,S,`atan`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Atanh,S,`atanh`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Atanh,S,`atanh`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Atanh,S,`atanh`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Atanh,S,`atanh`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +AttributeReference,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +AttributeReference,S, ,None,AST,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS,S,S +BRound,S,`bround`,None,project,value,NA,S,S,S,S,PS,PS,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +BRound,S,`bround`,None,project,scale,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BRound,S,`bround`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitLength,S,`bit_length`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA +BitLength,S,`bit_length`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseAnd,S,`&`,None,project,lhs,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseAnd,S,`&`,None,project,rhs,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseAnd,S,`&`,None,project,result,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseAnd,S,`&`,None,AST,lhs,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseAnd,S,`&`,None,AST,rhs,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseAnd,S,`&`,None,AST,result,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseNot,S,`~`,None,project,input,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseNot,S,`~`,None,project,result,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseNot,S,`~`,None,AST,input,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseNot,S,`~`,None,AST,result,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseOr,S,`\|`,None,project,lhs,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseOr,S,`\|`,None,project,rhs,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseOr,S,`\|`,None,project,result,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseOr,S,`\|`,None,AST,lhs,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseOr,S,`\|`,None,AST,rhs,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseOr,S,`\|`,None,AST,result,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseXor,S,`^`,None,project,lhs,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseXor,S,`^`,None,project,rhs,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseXor,S,`^`,None,project,result,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseXor,S,`^`,None,AST,lhs,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseXor,S,`^`,None,AST,rhs,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BitwiseXor,S,`^`,None,AST,result,NA,NS,NS,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BloomFilterMightContain,S, ,None,project,lhs,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA +BloomFilterMightContain,S, ,None,project,rhs,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +BloomFilterMightContain,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BoundReference,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +BoundReference,S, ,None,AST,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS,S,S +CaseWhen,S,`when`,None,project,predicate,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +CaseWhen,S,`when`,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +CaseWhen,S,`when`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Cbrt,S,`cbrt`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cbrt,S,`cbrt`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cbrt,S,`cbrt`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cbrt,S,`cbrt`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Ceil,S, ,None,project,input,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Ceil,S, ,None,project,result,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +CheckOverflowInTableInsert,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +CheckOverflowInTableInsert,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +Coalesce,S,`coalesce`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +Coalesce,S,`coalesce`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +Concat,S,`concat`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NS,NA,PS,NA,NA,NA,NA,NA +Concat,S,`concat`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NS,NA,PS,NA,NA,NA,NA,NA +ConcatWs,S,`concat_ws`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +ConcatWs,S,`concat_ws`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Contains,S, ,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Contains,S, ,None,project,search,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Contains,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Conv,NS,`conv`,This is disabled by default because GPU implementation is incomplete. We currently only support from/to_base values of 10 and 16. We fall back on CPU if the signed conversion is signalled via a negative to_base. GPU implementation does not check for an 64-bit signed/unsigned int overflow when performing the conversion to return `FFFFFFFFFFFFFFFF` or `18446744073709551615` or to throw an error in the ANSI mode. It is safe to enable if the overflow is not possible or detected externally. For instance decimal strings not longer than 18 characters / hexadecimal strings not longer than 15 characters disregarding the sign cannot cause an overflow. ,project,num,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Conv,NS,`conv`,This is disabled by default because GPU implementation is incomplete. We currently only support from/to_base values of 10 and 16. We fall back on CPU if the signed conversion is signalled via a negative to_base. GPU implementation does not check for an 64-bit signed/unsigned int overflow when performing the conversion to return `FFFFFFFFFFFFFFFF` or `18446744073709551615` or to throw an error in the ANSI mode. It is safe to enable if the overflow is not possible or detected externally. For instance decimal strings not longer than 18 characters / hexadecimal strings not longer than 15 characters disregarding the sign cannot cause an overflow. ,project,from_base,NA,PS,PS,PS,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Conv,NS,`conv`,This is disabled by default because GPU implementation is incomplete. We currently only support from/to_base values of 10 and 16. We fall back on CPU if the signed conversion is signalled via a negative to_base. GPU implementation does not check for an 64-bit signed/unsigned int overflow when performing the conversion to return `FFFFFFFFFFFFFFFF` or `18446744073709551615` or to throw an error in the ANSI mode. It is safe to enable if the overflow is not possible or detected externally. For instance decimal strings not longer than 18 characters / hexadecimal strings not longer than 15 characters disregarding the sign cannot cause an overflow. ,project,to_base,NA,PS,PS,PS,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Conv,NS,`conv`,This is disabled by default because GPU implementation is incomplete. We currently only support from/to_base values of 10 and 16. We fall back on CPU if the signed conversion is signalled via a negative to_base. GPU implementation does not check for an 64-bit signed/unsigned int overflow when performing the conversion to return `FFFFFFFFFFFFFFFF` or `18446744073709551615` or to throw an error in the ANSI mode. It is safe to enable if the overflow is not possible or detected externally. For instance decimal strings not longer than 18 characters / hexadecimal strings not longer than 15 characters disregarding the sign cannot cause an overflow. ,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cos,S,`cos`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cos,S,`cos`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cos,S,`cos`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cos,S,`cos`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cosh,S,`cosh`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cosh,S,`cosh`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cosh,S,`cosh`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cosh,S,`cosh`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cot,S,`cot`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cot,S,`cot`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cot,S,`cot`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Cot,S,`cot`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +CreateArray,S,`array`,None,project,arg,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,NS,PS,NS,NS,NS +CreateArray,S,`array`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +CreateMap,S,`map`,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,S,NA,NA,PS,NA,PS,NA,NA,NA +CreateMap,S,`map`,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NA,NA,PS,PS,PS,NA,NA,NA +CreateNamedStruct,S,`named_struct`; `struct`,None,project,name,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +CreateNamedStruct,S,`named_struct`; `struct`,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +CreateNamedStruct,S,`named_struct`; `struct`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA +CurrentRow$,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +DateAdd,S,`date_add`; `dateadd`,None,project,startDate,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateAdd,S,`date_add`; `dateadd`,None,project,days,NA,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateAdd,S,`date_add`; `dateadd`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateAddInterval,S, ,None,project,start,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateAddInterval,S, ,None,project,interval,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA +DateAddInterval,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateDiff,S,`date_diff`; `datediff`,None,project,lhs,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateDiff,S,`date_diff`; `datediff`,None,project,rhs,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateDiff,S,`date_diff`; `datediff`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateFormatClass,S,`date_format`,None,project,timestamp,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateFormatClass,S,`date_format`,None,project,strfmt,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateFormatClass,S,`date_format`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateSub,S,`date_sub`,None,project,startDate,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateSub,S,`date_sub`,None,project,days,NA,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DateSub,S,`date_sub`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DayOfMonth,S,`day`; `dayofmonth`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DayOfMonth,S,`day`; `dayofmonth`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DayOfWeek,S,`dayofweek`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DayOfWeek,S,`dayofweek`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DayOfYear,S,`dayofyear`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DayOfYear,S,`dayofyear`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DenseRank,S,`dense_rank`,None,window,ordering,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +DenseRank,S,`dense_rank`,None,window,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Divide,S,`/`,None,project,lhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Divide,S,`/`,None,project,rhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Divide,S,`/`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +DivideDTInterval,S, ,None,project,lhs,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA +DivideDTInterval,S, ,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +DivideDTInterval,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA +DivideYMInterval,S, ,None,project,lhs,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S +DivideYMInterval,S, ,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +DivideYMInterval,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S +DynamicPruningExpression,S, ,None,project,input,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +DynamicPruningExpression,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +ElementAt,S,`element_at`,None,project,array/map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,PS,NA,NA,NA,NA +ElementAt,S,`element_at`,None,project,index/key,PS,PS,PS,S,PS,PS,PS,PS,PS,PS,PS,NS,NS,NS,NS,NS,NS,NS,NS,NS +ElementAt,S,`element_at`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Empty2Null,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Empty2Null,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +EndsWith,S, ,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +EndsWith,S, ,None,project,search,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +EndsWith,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +EqualNullSafe,S,`<=>`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +EqualNullSafe,S,`<=>`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +EqualNullSafe,S,`<=>`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +EqualTo,S,`==`; `=`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +EqualTo,S,`==`; `=`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +EqualTo,S,`==`; `=`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +EqualTo,S,`==`; `=`,None,AST,lhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +EqualTo,S,`==`; `=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +EqualTo,S,`==`; `=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Exp,S,`exp`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Exp,S,`exp`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Exp,S,`exp`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Exp,S,`exp`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Explode,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,PS,NA,NA,NA,NA +Explode,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Expm1,S,`expm1`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Expm1,S,`expm1`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Expm1,S,`expm1`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Expm1,S,`expm1`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Flatten,S,`flatten`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Flatten,S,`flatten`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Floor,S, ,None,project,input,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Floor,S, ,None,project,result,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +FormatNumber,S,`format_number`,None,project,x,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +FormatNumber,S,`format_number`,None,project,d,NA,NA,NA,PS,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +FormatNumber,S,`format_number`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +FromUTCTimestamp,S,`from_utc_timestamp`,None,project,timestamp,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +FromUTCTimestamp,S,`from_utc_timestamp`,None,project,timezone,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +FromUTCTimestamp,S,`from_utc_timestamp`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +FromUnixTime,S,`from_unixtime`,None,project,sec,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +FromUnixTime,S,`from_unixtime`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +FromUnixTime,S,`from_unixtime`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetArrayItem,S, ,None,project,array,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +GetArrayItem,S, ,None,project,ordinal,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetArrayItem,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +GetArrayStructFields,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +GetArrayStructFields,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,path,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetJsonObject,S,`get_json_object`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetMapValue,S, ,None,project,map,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +GetMapValue,S, ,None,project,key,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS +GetMapValue,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +GetStructField,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA +GetStructField,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +GetTimestamp,S, ,None,project,timeExp,NA,NA,NA,NA,NA,NA,NA,S,PS,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetTimestamp,S, ,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GetTimestamp,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GreaterThan,S,`>`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +GreaterThan,S,`>`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +GreaterThan,S,`>`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GreaterThan,S,`>`,None,AST,lhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +GreaterThan,S,`>`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +GreaterThan,S,`>`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GreaterThanOrEqual,S,`>=`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +GreaterThanOrEqual,S,`>=`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +GreaterThanOrEqual,S,`>=`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +GreaterThanOrEqual,S,`>=`,None,AST,lhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +HiveHash,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS +HiveHash,S, ,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Hypot,S,`hypot`,None,project,lhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Hypot,S,`hypot`,None,project,rhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Hypot,S,`hypot`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +If,S,`if`,None,project,predicate,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +If,S,`if`,None,project,trueValue,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +If,S,`if`,None,project,falseValue,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +If,S,`if`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S +In,S,`in`,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +In,S,`in`,None,project,list,PS,PS,PS,PS,PS,PS,PS,PS,PS,PS,PS,NS,NS,NS,NS,NA,NS,NS,NA,NA +In,S,`in`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +InSet,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +InSet,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +InitCap,S,`initcap`,This is not 100% compatible with the Spark version because the Unicode version used by cuDF and the JVM may differ; resulting in some corner-case characters not changing case correctly.,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +InitCap,S,`initcap`,This is not 100% compatible with the Spark version because the Unicode version used by cuDF and the JVM may differ; resulting in some corner-case characters not changing case correctly.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +InputFileBlockLength,S,`input_file_block_length`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +InputFileBlockStart,S,`input_file_block_start`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +InputFileName,S,`input_file_name`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +IntegralDivide,S,`div`,None,project,lhs,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +IntegralDivide,S,`div`,None,project,rhs,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +IntegralDivide,S,`div`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +IsNaN,S,`isnan`,None,project,input,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +IsNaN,S,`isnan`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS +IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS +IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA +JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +KnownFloatingPointNormalized,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +KnownFloatingPointNormalized,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +KnownNotNull,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,NS,S,S,PS,PS,PS,NS,NS,NS +KnownNotNull,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,NS,S,S,PS,PS,PS,NS,NS,NS +KnownNullable,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +KnownNullable,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +Lag,S,`lag`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +Lag,S,`lag`,None,window,offset,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Lag,S,`lag`,None,window,default,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +Lag,S,`lag`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +LambdaFunction,S, ,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +LambdaFunction,S, ,None,project,arguments,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +LambdaFunction,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +LastDay,S,`last_day`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +LastDay,S,`last_day`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Lead,S,`lead`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +Lead,S,`lead`,None,window,offset,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Lead,S,`lead`,None,window,default,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +Lead,S,`lead`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +Least,S,`least`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +Least,S,`least`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +Length,S,`char_length`; `character_length`; `len`; `length`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA +Length,S,`char_length`; `character_length`; `len`; `length`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +LessThan,S,`<`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +LessThan,S,`<`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +LessThan,S,`<`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +LessThan,S,`<`,None,AST,lhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +LessThan,S,`<`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +LessThan,S,`<`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +LessThanOrEqual,S,`<=`,None,project,lhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +LessThanOrEqual,S,`<=`,None,project,rhs,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,PS,NS,NA,NA +LessThanOrEqual,S,`<=`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +LessThanOrEqual,S,`<=`,None,AST,lhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +LessThanOrEqual,S,`<=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA,NS,NS,NA,NA +LessThanOrEqual,S,`<=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Like,S,`like`,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Like,S,`like`,None,project,search,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Like,S,`like`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Literal,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,NS,S,S +Literal,S, ,None,AST,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,NS,NS,NS,NS,NS,NS +Log,S,`ln`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Log,S,`ln`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Log10,S,`log10`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Log10,S,`log10`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Log1p,S,`log1p`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Log1p,S,`log1p`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Log2,S,`log2`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Log2,S,`log2`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Logarithm,S,`log`,None,project,value,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Logarithm,S,`log`,None,project,base,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Logarithm,S,`log`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Lower,S,`lcase`; `lower`,This is not 100% compatible with the Spark version because the Unicode version used by cuDF and the JVM may differ; resulting in some corner-case characters not changing case correctly.,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Lower,S,`lcase`; `lower`,This is not 100% compatible with the Spark version because the Unicode version used by cuDF and the JVM may differ; resulting in some corner-case characters not changing case correctly.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MakeDecimal,S, ,None,project,input,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MakeDecimal,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA +MapConcat,S,`map_concat`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +MapConcat,S,`map_concat`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +MapEntries,S,`map_entries`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +MapEntries,S,`map_entries`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +MapFilter,S,`map_filter`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +MapFilter,S,`map_filter`,None,project,function,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MapFilter,S,`map_filter`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +MapFromArrays,S,`map_from_arrays`,None,project,keys,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +MapFromArrays,S,`map_from_arrays`,None,project,values,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +MapFromArrays,S,`map_from_arrays`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +MapKeys,S,`map_keys`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +MapKeys,S,`map_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +MapValues,S,`map_values`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +MapValues,S,`map_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Md5,S,`md5`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA +Md5,S,`md5`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MicrosToTimestamp,S,`timestamp_micros`,None,project,input,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MicrosToTimestamp,S,`timestamp_micros`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MillisToTimestamp,S,`timestamp_millis`,None,project,input,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MillisToTimestamp,S,`timestamp_millis`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Minute,S,`minute`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Multiply,S,`*`,None,AST,lhs,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +Multiply,S,`*`,None,AST,rhs,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +Multiply,S,`*`,None,AST,result,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +MultiplyDTInterval,S, ,None,project,lhs,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA +MultiplyDTInterval,S, ,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +MultiplyDTInterval,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA +MultiplyYMInterval,S, ,None,project,lhs,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S +MultiplyYMInterval,S, ,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +MultiplyYMInterval,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S +Murmur3Hash,S,`hash`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +Murmur3Hash,S,`hash`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +NaNvl,S,`nanvl`,None,project,lhs,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +NaNvl,S,`nanvl`,None,project,rhs,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +NaNvl,S,`nanvl`,None,project,result,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +NamedLambdaVariable,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +Not,S,`!`; `not`,None,project,input,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Not,S,`!`; `not`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Not,S,`!`; `not`,None,AST,input,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Not,S,`!`; `not`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +NthValue,S,`nth_value`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +NthValue,S,`nth_value`,None,window,offset,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +NthValue,S,`nth_value`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +OctetLength,S,`octet_length`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA +OctetLength,S,`octet_length`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Or,S,`or`,None,project,lhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Or,S,`or`,None,project,rhs,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Or,S,`or`,None,project,result,S,NA,NA,NA,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,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,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,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,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,NA,NA +ParseUrl,S,`parse_url`,None,project,key,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,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,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,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,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,NA,NA +Pmod,S,`pmod`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +Pmod,S,`pmod`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA +PosExplode,S,`posexplode_outer`; `posexplode`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,PS,NA,NA,NA,NA +PosExplode,S,`posexplode_outer`; `posexplode`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Pow,S,`pow`; `power`,None,project,lhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Pow,S,`pow`; `power`,None,project,rhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Pow,S,`pow`; `power`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Pow,S,`pow`; `power`,None,AST,lhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Pow,S,`pow`; `power`,None,AST,rhs,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Pow,S,`pow`; `power`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +PreciseTimestampConversion,S, ,None,project,input,NA,NA,NA,NA,S,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +PreciseTimestampConversion,S, ,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +PythonUDAF,S, ,None,aggregation,param,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonUDAF,S, ,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NA,PS,NS,PS,NA,NA,NA +PythonUDAF,S, ,None,reduction,param,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonUDAF,S, ,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NA,PS,NS,PS,NA,NA,NA +PythonUDAF,S, ,None,window,param,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonUDAF,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NA,PS,NS,PS,NA,NA,NA +PythonUDAF,S, ,None,project,param,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonUDAF,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NA,PS,NS,PS,NA,NA,NA +PythonUDF,S, ,None,aggregation,param,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonUDF,S, ,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NA,PS,NS,PS,NA,NA,NA +PythonUDF,S, ,None,reduction,param,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonUDF,S, ,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NA,PS,NS,PS,NA,NA,NA +PythonUDF,S, ,None,window,param,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonUDF,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NA,PS,NS,PS,NA,NA,NA +PythonUDF,S, ,None,project,param,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +PythonUDF,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NA,PS,NS,PS,NA,NA,NA +Quarter,S,`quarter`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Quarter,S,`quarter`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RLike,S,`regexp_like`; `regexp`; `rlike`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RLike,S,`regexp_like`; `regexp`; `rlike`,None,project,regexp,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RLike,S,`regexp_like`; `regexp`; `rlike`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RaiseError,S,`raise_error`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RaiseError,S,`raise_error`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +Rand,S,`rand`; `random`,None,project,seed,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Rand,S,`rand`; `random`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Rank,S,`rank`,None,window,ordering,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +Rank,S,`rank`,None,window,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpExtract,S,`regexp_extract`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpExtract,S,`regexp_extract`,None,project,regexp,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpExtract,S,`regexp_extract`,None,project,idx,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpExtract,S,`regexp_extract`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpExtractAll,S,`regexp_extract_all`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpExtractAll,S,`regexp_extract_all`,None,project,regexp,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpExtractAll,S,`regexp_extract_all`,None,project,idx,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpExtractAll,S,`regexp_extract_all`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +RegExpReplace,S,`regexp_replace`,None,project,regex,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpReplace,S,`regexp_replace`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpReplace,S,`regexp_replace`,None,project,pos,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpReplace,S,`regexp_replace`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RegExpReplace,S,`regexp_replace`,None,project,rep,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Remainder,S,`%`; `mod`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Remainder,S,`%`; `mod`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Remainder,S,`%`; `mod`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +ReplicateRows,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +ReplicateRows,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Reverse,S,`reverse`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Reverse,S,`reverse`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Rint,S,`rint`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Rint,S,`rint`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Rint,S,`rint`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Rint,S,`rint`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Round,S,`round`,None,project,value,NA,S,S,S,S,PS,PS,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Round,S,`round`,None,project,scale,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Round,S,`round`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +RoundCeil,S, ,None,project,value,NA,S,S,S,S,PS,PS,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +RoundCeil,S, ,None,project,scale,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RoundCeil,S, ,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +RoundFloor,S, ,None,project,value,NA,S,S,S,S,PS,PS,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +RoundFloor,S, ,None,project,scale,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +RoundFloor,S, ,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +RowNumber,S,`row_number`,None,window,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ScalaUDF,S, ,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,NS,NS,NS +ScalaUDF,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,NS,NS,NS +Second,S,`second`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Second,S,`second`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SecondsToTimestamp,S,`timestamp_seconds`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +SecondsToTimestamp,S,`timestamp_seconds`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sequence,S,`sequence`,None,project,start,NA,S,S,S,S,NA,NA,NS,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sequence,S,`sequence`,None,project,stop,NA,S,S,S,S,NA,NA,NS,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sequence,S,`sequence`,None,project,step,NA,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA +Sequence,S,`sequence`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +ShiftLeft,S,`shiftleft`,None,project,value,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShiftLeft,S,`shiftleft`,None,project,amount,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShiftLeft,S,`shiftleft`,None,project,result,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShiftRight,S,`shiftright`,None,project,value,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShiftRight,S,`shiftright`,None,project,amount,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShiftRight,S,`shiftright`,None,project,result,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShiftRightUnsigned,S,`shiftrightunsigned`,None,project,value,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShiftRightUnsigned,S,`shiftrightunsigned`,None,project,amount,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ShiftRightUnsigned,S,`shiftrightunsigned`,None,project,result,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Signum,S,`sign`; `signum`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Signum,S,`sign`; `signum`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sin,S,`sin`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sin,S,`sin`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sin,S,`sin`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sin,S,`sin`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sinh,S,`sinh`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sinh,S,`sinh`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sinh,S,`sinh`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sinh,S,`sinh`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Size,S,`cardinality`; `size`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,PS,NA,NA,NA,NA +Size,S,`cardinality`; `size`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SortArray,S,`sort_array`,None,project,array,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +SortArray,S,`sort_array`,None,project,ascendingOrder,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SortArray,S,`sort_array`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +SortOrder,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +SortOrder,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +SparkPartitionID,S,`spark_partition_id`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SpecifiedWindowFrame,S, ,None,project,lower,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,S,NA,NA,NA,NA,S,NS +SpecifiedWindowFrame,S, ,None,project,upper,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,S,NA,NA,NA,NA,S,NS +SpecifiedWindowFrame,S, ,None,project,result,NA,S,S,S,S,NS,NS,NA,NA,NA,NS,NA,NA,S,NA,NA,NA,NA,S,NS +Sqrt,S,`sqrt`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sqrt,S,`sqrt`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sqrt,S,`sqrt`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sqrt,S,`sqrt`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Stack,S,`stack`,None,project,n,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Stack,S,`stack`,None,project,expr,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +Stack,S,`stack`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +StartsWith,S, ,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StartsWith,S, ,None,project,search,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StartsWith,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringInstr,S,`instr`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringInstr,S,`instr`,None,project,substr,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringInstr,S,`instr`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringLPad,S, ,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringLPad,S, ,None,project,len,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringLPad,S, ,None,project,pad,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringLPad,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringLocate,S,`locate`; `position`,None,project,substr,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringLocate,S,`locate`; `position`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringLocate,S,`locate`; `position`,None,project,start,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringLocate,S,`locate`; `position`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringRPad,S, ,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringRPad,S, ,None,project,len,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringRPad,S, ,None,project,pad,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringRPad,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringRepeat,S,`repeat`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringRepeat,S,`repeat`,None,project,repeatTimes,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringRepeat,S,`repeat`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringReplace,S,`replace`,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringReplace,S,`replace`,None,project,search,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringReplace,S,`replace`,None,project,replace,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringReplace,S,`replace`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringSplit,S,`split`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringSplit,S,`split`,None,project,regexp,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringSplit,S,`split`,None,project,limit,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringSplit,S,`split`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +StringToMap,S,`str_to_map`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringToMap,S,`str_to_map`,None,project,pairDelim,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringToMap,S,`str_to_map`,None,project,keyValueDelim,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringToMap,S,`str_to_map`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA +StringTranslate,S,`translate`,This is not 100% compatible with the Spark version because the GPU implementation supports all unicode code points. In Spark versions < 3.2.0; translate() does not support unicode characters with code point >= U+10000 (See SPARK-34094),project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTranslate,S,`translate`,This is not 100% compatible with the Spark version because the GPU implementation supports all unicode code points. In Spark versions < 3.2.0; translate() does not support unicode characters with code point >= U+10000 (See SPARK-34094),project,from,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTranslate,S,`translate`,This is not 100% compatible with the Spark version because the GPU implementation supports all unicode code points. In Spark versions < 3.2.0; translate() does not support unicode characters with code point >= U+10000 (See SPARK-34094),project,to,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTranslate,S,`translate`,This is not 100% compatible with the Spark version because the GPU implementation supports all unicode code points. In Spark versions < 3.2.0; translate() does not support unicode characters with code point >= U+10000 (See SPARK-34094),project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrim,S,`trim`,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrim,S,`trim`,None,project,trimStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrim,S,`trim`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrimLeft,S,`ltrim`,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrimLeft,S,`ltrim`,None,project,trimStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrimLeft,S,`ltrim`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrimRight,S,`rtrim`,None,project,src,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrimRight,S,`rtrim`,None,project,trimStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StringTrimRight,S,`rtrim`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StructsToJson,NS,`to_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,struct,S,S,S,S,S,S,S,S,PS,S,S,NA,NA,NA,PS,PS,PS,NA,NA,NA +StructsToJson,NS,`to_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Substring,S,`substr`; `substring`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA +Substring,S,`substr`; `substring`,None,project,pos,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Substring,S,`substr`; `substring`,None,project,len,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Substring,S,`substr`; `substring`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA +SubstringIndex,S,`substring_index`,None,project,str,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SubstringIndex,S,`substring_index`,None,project,delim,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SubstringIndex,S,`substring_index`,None,project,count,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +SubstringIndex,S,`substring_index`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Subtract,S,`-`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +Subtract,S,`-`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +Subtract,S,`-`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +Subtract,S,`-`,None,AST,lhs,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS +Subtract,S,`-`,None,AST,rhs,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS +Subtract,S,`-`,None,AST,result,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS +Tan,S,`tan`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Tan,S,`tan`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Tan,S,`tan`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Tan,S,`tan`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Tanh,S,`tanh`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Tanh,S,`tanh`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Tanh,S,`tanh`,None,AST,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Tanh,S,`tanh`,None,AST,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TimeAdd,S, ,None,project,start,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TimeAdd,S, ,None,project,interval,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,S,NA +TimeAdd,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToDegrees,S,`degrees`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToDegrees,S,`degrees`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToRadians,S,`radians`,None,project,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToRadians,S,`radians`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToUTCTimestamp,S,`to_utc_timestamp`,None,project,timestamp,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToUTCTimestamp,S,`to_utc_timestamp`,None,project,timezone,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToUTCTimestamp,S,`to_utc_timestamp`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToUnixTimestamp,S,`to_unix_timestamp`,None,project,timeExp,NA,NA,NA,NA,NA,NA,NA,S,PS,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToUnixTimestamp,S,`to_unix_timestamp`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ToUnixTimestamp,S,`to_unix_timestamp`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TransformKeys,S,`transform_keys`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TransformKeys,S,`transform_keys`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NS,NS +TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS +UnaryMinus,S,`negative`,None,AST,result,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS +UnaryPositive,S,`positive`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +UnaryPositive,S,`positive`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S +UnaryPositive,S,`positive`,None,AST,input,NA,S,S,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,S,S +UnaryPositive,S,`positive`,None,AST,result,NA,S,S,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,S,S +UnboundedFollowing$,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +UnboundedPreceding$,S, ,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA +UnixTimestamp,S,`unix_timestamp`,None,project,timeExp,NA,NA,NA,NA,NA,NA,NA,S,PS,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +UnixTimestamp,S,`unix_timestamp`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +UnixTimestamp,S,`unix_timestamp`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +UnscaledValue,S, ,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA +UnscaledValue,S, ,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Upper,S,`ucase`; `upper`,This is not 100% compatible with the Spark version because the Unicode version used by cuDF and the JVM may differ; resulting in some corner-case characters not changing case correctly.,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Upper,S,`ucase`; `upper`,This is not 100% compatible with the Spark version because the Unicode version used by cuDF and the JVM may differ; resulting in some corner-case characters not changing case correctly.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +WeekDay,S,`weekday`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +WeekDay,S,`weekday`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +WindowExpression,S, ,None,window,windowFunction,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +WindowExpression,S, ,None,window,windowSpec,NA,S,S,S,S,NS,NS,NA,NA,NA,PS,NA,NA,S,NA,NA,NA,NA,S,NS +WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS +WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS +WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +AggregateExpression,S, ,None,aggregation,aggFunc,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +AggregateExpression,S, ,None,aggregation,filter,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +AggregateExpression,S, ,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +AggregateExpression,S, ,None,reduction,aggFunc,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +AggregateExpression,S, ,None,reduction,filter,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +AggregateExpression,S, ,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +AggregateExpression,S, ,None,window,aggFunc,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +AggregateExpression,S, ,None,window,filter,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +AggregateExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +ApproximatePercentile,S,`approx_percentile`; `percentile_approx`,This is not 100% compatible with the Spark version because the GPU implementation of approx_percentile is not bit-for-bit compatible with Apache Spark,aggregation,input,NA,S,S,S,S,S,S,NS,NS,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +ApproximatePercentile,S,`approx_percentile`; `percentile_approx`,This is not 100% compatible with the Spark version because the GPU implementation of approx_percentile is not bit-for-bit compatible with Apache Spark,aggregation,percentage,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +ApproximatePercentile,S,`approx_percentile`; `percentile_approx`,This is not 100% compatible with the Spark version because the GPU implementation of approx_percentile is not bit-for-bit compatible with Apache Spark,aggregation,accuracy,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ApproximatePercentile,S,`approx_percentile`; `percentile_approx`,This is not 100% compatible with the Spark version because the GPU implementation of approx_percentile is not bit-for-bit compatible with Apache Spark,aggregation,result,NA,S,S,S,S,S,S,NS,NS,NA,S,NA,NA,NA,PS,NA,NA,NA,NA,NA +ApproximatePercentile,S,`approx_percentile`; `percentile_approx`,This is not 100% compatible with the Spark version because the GPU implementation of approx_percentile is not bit-for-bit compatible with Apache Spark,reduction,input,NA,S,S,S,S,S,S,NS,NS,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +ApproximatePercentile,S,`approx_percentile`; `percentile_approx`,This is not 100% compatible with the Spark version because the GPU implementation of approx_percentile is not bit-for-bit compatible with Apache Spark,reduction,percentage,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +ApproximatePercentile,S,`approx_percentile`; `percentile_approx`,This is not 100% compatible with the Spark version because the GPU implementation of approx_percentile is not bit-for-bit compatible with Apache Spark,reduction,accuracy,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ApproximatePercentile,S,`approx_percentile`; `percentile_approx`,This is not 100% compatible with the Spark version because the GPU implementation of approx_percentile is not bit-for-bit compatible with Apache Spark,reduction,result,NA,S,S,S,S,S,S,NS,NS,NA,S,NA,NA,NA,PS,NA,NA,NA,NA,NA +Average,S,`avg`; `mean`,None,aggregation,input,NA,S,S,S,S,S,S,NA,NA,NA,S,S,NA,NS,NA,NA,NA,NA,NS,NS +Average,S,`avg`; `mean`,None,aggregation,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Average,S,`avg`; `mean`,None,reduction,input,NA,S,S,S,S,S,S,NA,NA,NA,S,S,NA,NS,NA,NA,NA,NA,NS,NS +Average,S,`avg`; `mean`,None,reduction,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Average,S,`avg`; `mean`,None,window,input,NA,S,S,S,S,S,S,NA,NA,NA,S,S,NA,NS,NA,NA,NA,NA,NS,NS +Average,S,`avg`; `mean`,None,window,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +BloomFilterAggregate,S, ,None,reduction,child,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BloomFilterAggregate,S, ,None,reduction,estimatedItems,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BloomFilterAggregate,S, ,None,reduction,numBits,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +BloomFilterAggregate,S, ,None,reduction,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA +CollectList,S,`array_agg`; `collect_list`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +CollectList,S,`array_agg`; `collect_list`,None,aggregation,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +CollectList,S,`array_agg`; `collect_list`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +CollectList,S,`array_agg`; `collect_list`,None,reduction,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +CollectList,S,`array_agg`; `collect_list`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +CollectList,S,`array_agg`; `collect_list`,None,window,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +CollectSet,S,`collect_set`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +CollectSet,S,`collect_set`,None,aggregation,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +CollectSet,S,`collect_set`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +CollectSet,S,`collect_set`,None,reduction,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +CollectSet,S,`collect_set`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,PS,NS,NS,NS +CollectSet,S,`collect_set`,None,window,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA +Count,S,`count`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +Count,S,`count`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Count,S,`count`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +Count,S,`count`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Count,S,`count`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +Count,S,`count`,None,window,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +First,S,`first_value`; `first`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +First,S,`first_value`; `first`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Last,S,`last_value`; `last`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Last,S,`last_value`; `last`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Last,S,`last_value`; `last`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Max,S,`max`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +Max,S,`max`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +Max,S,`max`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +Max,S,`max`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +Max,S,`max`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +Max,S,`max`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +MaxBy,S,`max_by`,None,aggregation,value,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +MaxBy,S,`max_by`,None,aggregation,ordering,S,S,S,S,S,NS,NS,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +MaxBy,S,`max_by`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +MaxBy,S,`max_by`,None,reduction,value,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +MaxBy,S,`max_by`,None,reduction,ordering,S,S,S,S,S,NS,NS,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +MaxBy,S,`max_by`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Min,S,`min`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +Min,S,`min`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +Min,S,`min`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +Min,S,`min`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +Min,S,`min`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +Min,S,`min`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +MinBy,S,`min_by`,None,aggregation,value,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +MinBy,S,`min_by`,None,aggregation,ordering,S,S,S,S,S,NS,NS,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +MinBy,S,`min_by`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +MinBy,S,`min_by`,None,reduction,value,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +MinBy,S,`min_by`,None,reduction,ordering,S,S,S,S,S,NS,NS,S,PS,S,S,S,NS,NS,PS,NA,PS,NS,NA,NA +MinBy,S,`min_by`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +Percentile,S,`percentile`,None,aggregation,input,NA,S,S,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Percentile,S,`percentile`,None,aggregation,percentage,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +Percentile,S,`percentile`,None,aggregation,frequency,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +Percentile,S,`percentile`,None,aggregation,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +Percentile,S,`percentile`,None,reduction,input,NA,S,S,S,S,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Percentile,S,`percentile`,None,reduction,percentage,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +Percentile,S,`percentile`,None,reduction,frequency,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +Percentile,S,`percentile`,None,reduction,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA +PivotFirst,S, ,None,aggregation,pivotColumn,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +PivotFirst,S, ,None,aggregation,valueColumn,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +PivotFirst,S, ,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,NS,NS,NS,NS +PivotFirst,S, ,None,reduction,pivotColumn,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +PivotFirst,S, ,None,reduction,valueColumn,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +PivotFirst,S, ,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,NS,NS,NS,NS,NS +StddevPop,S,`stddev_pop`,None,reduction,input,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevPop,S,`stddev_pop`,None,reduction,result,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevPop,S,`stddev_pop`,None,aggregation,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevPop,S,`stddev_pop`,None,aggregation,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevPop,S,`stddev_pop`,None,window,input,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevPop,S,`stddev_pop`,None,window,result,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevSamp,S,`std`; `stddev_samp`; `stddev`,None,aggregation,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevSamp,S,`std`; `stddev_samp`; `stddev`,None,aggregation,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevSamp,S,`std`; `stddev_samp`; `stddev`,None,reduction,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevSamp,S,`std`; `stddev_samp`; `stddev`,None,reduction,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevSamp,S,`std`; `stddev_samp`; `stddev`,None,window,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +StddevSamp,S,`std`; `stddev_samp`; `stddev`,None,window,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sum,S,`sum`,None,aggregation,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sum,S,`sum`,None,aggregation,result,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sum,S,`sum`,None,reduction,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sum,S,`sum`,None,reduction,result,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sum,S,`sum`,None,window,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +Sum,S,`sum`,None,window,result,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA +VariancePop,S,`var_pop`,None,reduction,input,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VariancePop,S,`var_pop`,None,reduction,result,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VariancePop,S,`var_pop`,None,aggregation,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VariancePop,S,`var_pop`,None,aggregation,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VariancePop,S,`var_pop`,None,window,input,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VariancePop,S,`var_pop`,None,window,result,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VarianceSamp,S,`var_samp`; `variance`,None,reduction,input,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VarianceSamp,S,`var_samp`; `variance`,None,reduction,result,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VarianceSamp,S,`var_samp`; `variance`,None,aggregation,input,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VarianceSamp,S,`var_samp`; `variance`,None,aggregation,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VarianceSamp,S,`var_samp`; `variance`,None,window,input,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +VarianceSamp,S,`var_samp`; `variance`,None,window,result,NA,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +NormalizeNaNAndZero,S, ,None,project,input,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +NormalizeNaNAndZero,S, ,None,project,result,NA,NA,NA,NA,NA,S,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +InSubqueryExec,S, ,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA +InSubqueryExec,S, ,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +ScalarSubquery,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS +HiveGenericUDF,S, ,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,NS,NS,NS +HiveGenericUDF,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,NS,NS,NS +HiveSimpleUDF,S, ,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,NS,NS,NS +HiveSimpleUDF,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,NS,NS,NS From 60b68061f473f63e4193c0b7bd60cff3b4603d40 Mon Sep 17 00:00:00 2001 From: Peixin Date: Tue, 14 Jan 2025 09:34:38 +0800 Subject: [PATCH 47/47] Update to use vulnerability-scan runner [skip ci] (#11958) Required by ProdSec team to enable new malware scan. note: this change does not enable the malware scan directly but try to change to use the new shared runner within NVIDIA org. After merging this change, we will need to ask the Blossom team to enable the actual malware scan internally Signed-off-by: Peixin Li --- .github/workflows/blossom-ci.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/blossom-ci.yml b/.github/workflows/blossom-ci.yml index be5c238d687..06abf0828f3 100644 --- a/.github/workflows/blossom-ci.yml +++ b/.github/workflows/blossom-ci.yml @@ -90,7 +90,7 @@ jobs: Vulnerability-scan: name: Vulnerability scan needs: [Authorization] - runs-on: ubuntu-latest + runs-on: vulnerability-scan steps: - name: Checkout code uses: actions/checkout@v4