From 2e0fe6ab8bf8be20413dceec8e07fa20e0f7d283 Mon Sep 17 00:00:00 2001 From: Rong Ou Date: Thu, 17 Dec 2020 10:05:14 -0800 Subject: [PATCH] Support spilling to disk directly via cuFile/GDS (#1313) * support spilling to GDS Signed-off-by: Rong Ou * address review comments * review feedback Signed-off-by: Rong Ou --- docs/configs.md | 1 + .../nvidia/spark/rapids/RapidsBuffer.scala | 1 + .../spark/rapids/RapidsBufferCatalog.scala | 18 ++- .../com/nvidia/spark/rapids/RapidsConf.scala | 10 ++ .../nvidia/spark/rapids/RapidsGdsStore.scala | 106 ++++++++++++++++++ .../rapids/shuffle/BufferSendState.scala | 2 +- 6 files changed, 133 insertions(+), 5 deletions(-) create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsGdsStore.scala diff --git a/docs/configs.md b/docs/configs.md index 45d73ebb0a3..da168751ed3 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -32,6 +32,7 @@ Name | Description | Default Value spark.rapids.cloudSchemes|Comma separated list of additional URI schemes that are to be considered cloud based filesystems. Schemes already included: dbfs, s3, s3a, s3n, wasbs, gs. Cloud based stores generally would be total separate from the executors and likely have a higher I/O read cost. Many times the cloud filesystems also get better throughput when you have multiple readers in parallel. This is used with spark.rapids.sql.format.parquet.reader.type|None spark.rapids.memory.gpu.allocFraction|The fraction of total GPU memory that should be initially allocated for pooled memory. Extra memory will be allocated as needed, but it may result in more fragmentation. This must be less than or equal to the maximum limit configured via spark.rapids.memory.gpu.maxAllocFraction.|0.9 spark.rapids.memory.gpu.debug|Provides a log of GPU memory allocations and frees. If set to STDOUT or STDERR the logging will go there. Setting it to NONE disables logging. All other values are reserved for possible future expansion and in the mean time will disable logging.|NONE +spark.rapids.memory.gpu.direct.storage.spill.enabled|Should GPUDirect Storage (GDS) be used to spill GPU memory buffers directly to disk. GDS must be enabled and the directory `spark.local.dir` must support GDS. This is an experimental feature. For more information on GDS, see https://docs.nvidia.com/gpudirect-storage/.|false spark.rapids.memory.gpu.maxAllocFraction|The fraction of total GPU memory that limits the maximum size of the RMM pool. The value must be greater than or equal to the setting for spark.rapids.memory.gpu.allocFraction. Note that this limit will be reduced by the reserve memory configured in spark.rapids.memory.gpu.reserve.|1.0 spark.rapids.memory.gpu.oomDumpDir|The path to a local directory where a heap dump will be created if the GPU encounters an unrecoverable out-of-memory (OOM) error. The filename will be of the form: "gpu-oom-.hprof" where is the process ID.|None spark.rapids.memory.gpu.pool|Select the RMM pooling allocator to use. Valid values are "DEFAULT", "ARENA", and "NONE". With "DEFAULT", `rmm::mr::pool_memory_resource` is used; with "ARENA", `rmm::mr::arena_memory_resource` is used. If set to "NONE", pooling is disabled and RMM just passes through to CUDA memory allocation directly.|ARENA diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala index 7350f05ffce..c865dfaafc2 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala @@ -56,6 +56,7 @@ object StorageTier extends Enumeration { val DEVICE: StorageTier = Value(0, "device memory") val HOST: StorageTier = Value(1, "host memory") val DISK: StorageTier = Value(2, "local disk") + val GDS: StorageTier = Value(3, "GPUDirect Storage") } /** Interface provided by all types of RAPIDS buffers */ 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 index 675884d1037..e772827703f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala @@ -113,6 +113,7 @@ object RapidsBufferCatalog extends Logging with Arm { private var deviceStorage: RapidsDeviceMemoryStore = _ private var hostStorage: RapidsHostMemoryStore = _ private var diskStorage: RapidsDiskStore = _ + private var gdsStorage: RapidsGdsStore = _ private var memoryEventHandler: DeviceMemoryEventHandler = _ private lazy val conf: SparkConf = { @@ -130,11 +131,16 @@ object RapidsBufferCatalog extends Logging with Arm { closeImpl() assert(memoryEventHandler == null) deviceStorage = new RapidsDeviceMemoryStore() - hostStorage = new RapidsHostMemoryStore(rapidsConf.hostSpillStorageSize) val diskBlockManager = new RapidsDiskBlockManager(conf) - diskStorage = new RapidsDiskStore(diskBlockManager) - deviceStorage.setSpillStore(hostStorage) - hostStorage.setSpillStore(diskStorage) + if (rapidsConf.isGdsSpillEnabled) { + gdsStorage = new RapidsGdsStore(diskBlockManager) + deviceStorage.setSpillStore(gdsStorage) + } else { + hostStorage = new RapidsHostMemoryStore(rapidsConf.hostSpillStorageSize) + diskStorage = new RapidsDiskStore(diskBlockManager) + deviceStorage.setSpillStore(hostStorage) + hostStorage.setSpillStore(diskStorage) + } logInfo("Installing GPU memory handler for spill") memoryEventHandler = new DeviceMemoryEventHandler(deviceStorage, rapidsConf.gpuOomDumpDir) @@ -166,6 +172,10 @@ object RapidsBufferCatalog extends Logging with Arm { diskStorage.close() diskStorage = null } + if (gdsStorage != null) { + gdsStorage.close() + gdsStorage = null + } } def getDeviceStorage: RapidsDeviceMemoryStore = deviceStorage 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 f0da76f6654..2106964e630 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 @@ -333,6 +333,14 @@ object RapidsConf { .bytesConf(ByteUnit.BYTE) .createWithDefault(ByteUnit.GiB.toBytes(1)) + val GDS_SPILL = conf("spark.rapids.memory.gpu.direct.storage.spill.enabled") + .doc("Should GPUDirect Storage (GDS) be used to spill GPU memory buffers directly to disk. " + + "GDS must be enabled and the directory `spark.local.dir` must support GDS. This is an " + + "experimental feature. For more information on GDS, see " + + "https://docs.nvidia.com/gpudirect-storage/.") + .booleanConf + .createWithDefault(false) + val POOLED_MEM = conf("spark.rapids.memory.gpu.pooling.enabled") .doc("Should RMM act as a pooling allocator for GPU memory, or should it just pass " + "through to CUDA memory allocation directly. DEPRECATED: please use " + @@ -976,6 +984,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val hostSpillStorageSize: Long = get(HOST_SPILL_STORAGE_SIZE) + lazy val isGdsSpillEnabled: Boolean = get(GDS_SPILL) + lazy val hasNans: Boolean = get(HAS_NANS) lazy val gpuTargetBatchSizeBytes: Long = get(GPU_BATCH_SIZE_BYTES) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsGdsStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsGdsStore.scala new file mode 100644 index 00000000000..9238970753a --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsGdsStore.scala @@ -0,0 +1,106 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids + +import java.io.File +import java.util.concurrent.ConcurrentHashMap + +import ai.rapids.cudf._ +import com.nvidia.spark.rapids.StorageTier.StorageTier +import com.nvidia.spark.rapids.format.TableMeta + +import org.apache.spark.sql.rapids.RapidsDiskBlockManager +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.vectorized.ColumnarBatch + +/** A buffer store using GPUDirect Storage (GDS). */ +class RapidsGdsStore( + diskBlockManager: RapidsDiskBlockManager, + catalog: RapidsBufferCatalog = RapidsBufferCatalog.singleton) + extends RapidsBufferStore("gds", catalog) with Arm { + private[this] val sharedBufferFiles = new ConcurrentHashMap[RapidsBufferId, File] + + override def createBuffer( + other: RapidsBuffer, + stream: Cuda.Stream): RapidsBufferBase = { + withResource(other.getMemoryBuffer) { otherBuffer => + val deviceBuffer = otherBuffer match { + case d: DeviceMemoryBuffer => d + case _ => throw new IllegalStateException("copying from buffer without device memory") + } + val id = other.id + val path = if (id.canShareDiskPaths) { + sharedBufferFiles.computeIfAbsent(id, _ => id.getDiskPath(diskBlockManager)) + } else { + id.getDiskPath(diskBlockManager) + } + // When sharing files, append to the file; otherwise, write from the beginning. + val fileOffset = if (id.canShareDiskPaths) { + // only one writer at a time for now when using shared files + path.synchronized { + CuFile.appendDeviceBufferToFile(path, deviceBuffer) + } + } else { + CuFile.writeDeviceBufferToFile(path, 0, deviceBuffer) + 0 + } + logDebug(s"Spilled to $path $fileOffset:${other.size} via GDS") + new RapidsGdsBuffer(id, fileOffset, other.size, other.meta, other.getSpillPriority) + } + } + + class RapidsGdsBuffer( + id: RapidsBufferId, + fileOffset: Long, + size: Long, + meta: TableMeta, + spillPriority: Long) extends RapidsBufferBase(id, size, meta, spillPriority) { + override val storageTier: StorageTier = StorageTier.GDS + + // TODO(rongou): cache this buffer to avoid repeated reads from disk. + override def getMemoryBuffer: DeviceMemoryBuffer = synchronized { + val path = if (id.canShareDiskPaths) { + sharedBufferFiles.get(id) + } else { + id.getDiskPath(diskBlockManager) + } + closeOnExcept(DeviceMemoryBuffer.allocate(size)) { buffer => + CuFile.readFileToDeviceBuffer(buffer, path, fileOffset) + logDebug(s"Created device buffer for $path $fileOffset:$size via GDS") + buffer + } + } + + override protected def releaseResources(): Unit = { + // 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 GDS spill path $path") + } + } + } + + override def getColumnarBatch(sparkTypes: Array[DataType]): ColumnarBatch = { + withResource(getMemoryBuffer) { deviceBuffer => + columnarBatchFromDeviceBuffer(deviceBuffer, sparkTypes) + } + } + } +} 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 729bd8fa9e1..cee61bfe2e0 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 @@ -152,7 +152,7 @@ class BufferSendState( requestHandler.acquireShuffleBuffer(bufferId)) { rapidsBuffer => //these are closed later, after we synchronize streams rapidsBuffer.storageTier match { - case StorageTier.DEVICE => + case StorageTier.DEVICE | StorageTier.GDS => deviceBuffs += blockRange.rangeSize() case _ => // host/disk hostBuffs += blockRange.rangeSize()