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 1/5] 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 2/5] 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 3/5] 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 4/5] [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 5/5] 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__':