Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[auto-merge] branch-24.10 to branch-24.12 [skip ci] [bot] #11517

Merged
merged 1 commit into from
Sep 27, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 4 additions & 1 deletion docs/dev/lore.md
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,10 @@ partitions.

You also need to set `spark.rapids.sql.lore.dumpPath` to tell LORE where to dump the data, the
value of which should point to a directory. All dumped data of a query will live in this
directory. A typical directory hierarchy would look like this:
directory. Note, the directory may either not exist, in which case it will be created, or it should be empty.
If the directory exists and contains files, an `IllegalArgumentException` will be thrown to prevent overwriting existing data.

A typical directory hierarchy would look like this:

```console
+ loreId-10/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,23 +89,21 @@ object GpuLore {
}

def dumpObject[T: ClassTag](obj: T, path: Path, hadoopConf: Configuration): Unit = {
withResource(path.getFileSystem(hadoopConf)) { fs =>
withResource(fs.create(path, false)) { fout =>
val serializerStream = SparkEnv.get.serializer.newInstance().serializeStream(fout)
withResource(serializerStream) { ser =>
ser.writeObject(obj)
}
val fs = path.getFileSystem(hadoopConf)
withResource(fs.create(path, true)) { fout =>
val serializerStream = SparkEnv.get.serializer.newInstance().serializeStream(fout)
withResource(serializerStream) { ser =>
ser.writeObject(obj)
}
}
}

def loadObject[T: ClassTag](path: Path, hadoopConf: Configuration): T = {
withResource(path.getFileSystem(hadoopConf)) { fs =>
withResource(fs.open(path)) { fin =>
val serializerStream = SparkEnv.get.serializer.newInstance().deserializeStream(fin)
withResource(serializerStream) { ser =>
ser.readObject().asInstanceOf[T]
}
val fs = path.getFileSystem(hadoopConf)
withResource(fs.open(path)) { fin =>
val serializerStream = SparkEnv.get.serializer.newInstance().deserializeStream(fin)
withResource(serializerStream) { ser =>
ser.readObject().asInstanceOf[T]
}
}
}
Expand Down Expand Up @@ -186,6 +184,12 @@ object GpuLore {
idGen.computeIfAbsent(executionId, _ => new AtomicInteger(0)).getAndIncrement()
}
}
/**
* Executions that have checked the lore output root path.
* Key is [[SQLExecution.EXECUTION_ID_KEY]].
*/
private val loreOutputRootPathChecked: ConcurrentHashMap[String, Boolean] =
new ConcurrentHashMap[String, Boolean]()

def tagForLore(sparkPlan: SparkPlan, rapidsConf: RapidsConf): SparkPlan = {
val loreDumpIds = rapidsConf.loreDumpIds
Expand All @@ -197,6 +201,20 @@ object GpuLore {
s"when ${RapidsConf.LORE_DUMP_IDS.key} is set."))

val spark = SparkShimImpl.sessionFromPlan(sparkPlan)

Option(spark.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)).foreach {
executionId =>
loreOutputRootPathChecked.computeIfAbsent(executionId, _ => {
val path = new Path(loreOutputRootPath)
val fs = path.getFileSystem(spark.sparkContext.hadoopConfiguration)
if (fs.exists(path) && fs.listStatus(path).nonEmpty) {
throw new IllegalArgumentException(
s"LORE dump path $loreOutputRootPath already exists and is not empty.")
}
true
})
}

val hadoopConf = {
val sc = spark.sparkContext
sc.broadcast(new SerializableConfiguration(sc.hadoopConfiguration))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ class GpuLoreDumpRDD(info: LoreDumpRDDInfo, input: RDD[ColumnarBatch])
private def dumpCurrentBatch(): ColumnarBatch = {
val outputPath = pathOfBatch(split.index, batchIdx)
val outputStream = outputPath.getFileSystem(info.hadoopConf.value.value)
.create(outputPath, false)
.create(outputPath, true)
DumpUtils.dumpToParquet(nextBatch.get, outputStream)
nextBatch.get
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
package com.nvidia.spark.rapids.lore

import com.nvidia.spark.rapids.{FunSuiteWithTempDir, GpuColumnarToRowExec, RapidsConf, SparkQueryCompareTestSuite}
import com.nvidia.spark.rapids.Arm.withResource
import org.apache.hadoop.fs.Path

import org.apache.spark.internal.Logging
Expand Down Expand Up @@ -147,6 +148,26 @@ class GpuLoreSuite extends SparkQueryCompareTestSuite with FunSuiteWithTempDir w
}
}

test("Non-empty lore dump path") {
withGpuSparkSession{ spark =>
spark.conf.set(RapidsConf.LORE_DUMP_PATH.key, TEST_FILES_ROOT.getAbsolutePath)
spark.conf.set(RapidsConf.LORE_DUMP_IDS.key, "3[*]")

//Create a file in the root path
val path = new Path(s"${TEST_FILES_ROOT.getAbsolutePath}/test")
val fs = path.getFileSystem(spark.sparkContext.hadoopConfiguration)
withResource(fs.create(path, true)) { _ =>
}

val df = spark.range(0, 1000, 1, 100)
.selectExpr("id % 10 as key", "id % 100 as value")

assertThrows[IllegalArgumentException] {
df.collect()
}
}
}

private def doTestReplay(loreDumpIds: String)(dfFunc: SparkSession => DataFrame) = {
val loreId = OutputLoreId.parse(loreDumpIds).head._1
withGpuSparkSession { spark =>
Expand Down