diff --git a/docs/configs.md b/docs/configs.md index b6a2babac48..729dd52f2e4 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -31,8 +31,9 @@ Name | Description | Default Value -----|-------------|-------------- 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.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. If the value is 1 then no artificial limit will be applied.|1.0 +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.pooling.enabled|Should RMM act as a pooling allocator for GPU memory, or should it just pass through to CUDA memory allocation directly.|true +spark.rapids.memory.gpu.reserve|The amount of GPU memory that should remain unallocated by RMM and left for system use such as memory needed for kernels, kernel launches or JIT compilation.|1073741824 spark.rapids.memory.host.spillStorageSize|Amount of off-heap host memory to use for buffering spilled GPU data before spilling to local disk|1073741824 spark.rapids.memory.pinnedPool.size|The size of the pinned memory pool in bytes unless otherwise specified. Use 0 to disable the pool.|0 spark.rapids.memory.uvm.enabled|UVM or universal memory can allow main host memory to act essentially as swap for device(GPU) memory. This allows the GPU to process more data than fits in memory, but can result in slower processing. This is an experimental feature.|false 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 e0d9cd00fc4..33bbdd95a58 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 @@ -154,27 +154,52 @@ object GpuDeviceManager extends Logging { } } + private def toMB(x: Long): Double = x / 1024 / 1024.0 + + private def computeRmmInitSizes(conf: RapidsConf, info: CudaMemInfo): (Long, Long) = { + // Align workaround for https://github.com/rapidsai/rmm/issues/527 + def truncateToAlignment(x: Long): Long = x & ~511L + + var initialAllocation = truncateToAlignment((conf.rmmAllocFraction * info.total).toLong) + if (initialAllocation > info.free) { + logWarning(s"Initial RMM allocation (${toMB(initialAllocation)} MB) is " + + s"larger than free memory (${toMB(info.free)} MB)") + } + val maxAllocation = truncateToAlignment((conf.rmmAllocMaxFraction * info.total).toLong) + if (maxAllocation < initialAllocation) { + throw new IllegalArgumentException(s"${RapidsConf.RMM_ALLOC_MAX_FRACTION} " + + s"configured as ${conf.rmmAllocMaxFraction} which is less than the " + + s"${RapidsConf.RMM_ALLOC_FRACTION} setting of ${conf.rmmAllocFraction}") + } + val reserveAmount = conf.rmmAllocReserve + if (reserveAmount >= maxAllocation) { + throw new IllegalArgumentException(s"RMM reserve memory (${toMB(reserveAmount)} MB) " + + s"larger than maximum pool size (${toMB(maxAllocation)} MB). Check the settings for " + + s"${RapidsConf.RMM_ALLOC_MAX_FRACTION} (=${conf.rmmAllocFraction}) and " + + s"${RapidsConf.RMM_ALLOC_RESERVE} (=$reserveAmount)") + } + val adjustedMaxAllocation = truncateToAlignment(maxAllocation - reserveAmount) + if (initialAllocation > adjustedMaxAllocation) { + logWarning(s"Initial RMM allocation (${toMB(initialAllocation)} MB) is larger than " + + s"the adjusted maximum allocation (${toMB(adjustedMaxAllocation)} MB), " + + "lowering initial allocation to the adjusted maximum allocation.") + initialAllocation = adjustedMaxAllocation + } + + // Currently a max limit only works in pooled mode. Need a general limit resource wrapper + // as requested in https://github.com/rapidsai/rmm/issues/442 to support for all RMM modes. + if (conf.isPooledMemEnabled) { + (initialAllocation, adjustedMaxAllocation) + } else { + (initialAllocation, 0) + } + } + private def initializeRmm(gpuId: Int, rapidsConf: Option[RapidsConf] = None): Unit = { if (!Rmm.isInitialized) { val conf = rapidsConf.getOrElse(new RapidsConf(SparkEnv.get.conf)) val info = Cuda.memGetInfo() - val initialAllocation = (conf.rmmAllocFraction * info.total).toLong - if (initialAllocation > info.free) { - logWarning(s"Initial RMM allocation(${initialAllocation / 1024 / 1024.0} MB) is " + - s"larger than free memory(${info.free / 1024 / 1024.0} MB)") - } - val maxAllocation = if (conf.rmmAllocMaxFraction < 1) { - (conf.rmmAllocMaxFraction * info.total).toLong - } else { - // Do not attempt to enforce any artificial pool limit based on queried GPU memory size - // if config indicates all GPU memory should be used. - 0 - } - if (maxAllocation > 0 && maxAllocation < initialAllocation) { - throw new IllegalArgumentException(s"${RapidsConf.RMM_ALLOC_MAX_FRACTION} " + - s"configured as ${conf.rmmAllocMaxFraction} which is less than the " + - s"${RapidsConf.RMM_ALLOC_FRACTION} setting of ${conf.rmmAllocFraction}") - } + val (initialAllocation, maxAllocation) = computeRmmInitSizes(conf, info) var init = RmmAllocationMode.CUDA_DEFAULT val features = ArrayBuffer[String]() if (conf.isPooledMemEnabled) { @@ -202,7 +227,8 @@ object GpuDeviceManager extends Logging { deviceId = Some(gpuId) logInfo(s"Initializing RMM${features.mkString(" ", " ", "")} " + - s"${initialAllocation / 1024 / 1024.0} MB on gpuId $gpuId") + s"initial size = ${toMB(initialAllocation)} MB, " + + s"max size = ${toMB(maxAllocation)} MB on gpuId $gpuId") try { Cuda.setDevice(gpuId) 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 c59fa8fd3fa..65dc5dbe20c 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 @@ -253,6 +253,7 @@ object RapidsConf { .createWithDefault("NONE") private val RMM_ALLOC_MAX_FRACTION_KEY = "spark.rapids.memory.gpu.maxAllocFraction" + private val RMM_ALLOC_RESERVE_KEY = "spark.rapids.memory.gpu.reserve" val RMM_ALLOC_FRACTION = conf("spark.rapids.memory.gpu.allocFraction") .doc("The fraction of total GPU memory that should be initially allocated " + @@ -266,11 +267,18 @@ object RapidsConf { val RMM_ALLOC_MAX_FRACTION = conf(RMM_ALLOC_MAX_FRACTION_KEY) .doc("The fraction of total GPU memory that limits the maximum size of the RMM pool. " + s"The value must be greater than or equal to the setting for $RMM_ALLOC_FRACTION. " + - "If the value is 1 then no artificial limit will be applied.") + "Note that this limit will be reduced by the reserve memory configured in " + + s"$RMM_ALLOC_RESERVE_KEY.") .doubleConf .checkValue(v => v >= 0 && v <= 1, "The fraction value must be in [0, 1].") .createWithDefault(1) + val RMM_ALLOC_RESERVE = conf(RMM_ALLOC_RESERVE_KEY) + .doc("The amount of GPU memory that should remain unallocated by RMM and left for " + + "system use such as memory needed for kernels, kernel launches or JIT compilation.") + .bytesConf(ByteUnit.BYTE) + .createWithDefault(ByteUnit.MiB.toBytes(1024)) + val HOST_SPILL_STORAGE_SIZE = conf("spark.rapids.memory.host.spillStorageSize") .doc("Amount of off-heap host memory to use for buffering spilled GPU data " + "before spilling to local disk") @@ -800,6 +808,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val rmmAllocMaxFraction: Double = get(RMM_ALLOC_MAX_FRACTION) + lazy val rmmAllocReserve: Long = get(RMM_ALLOC_RESERVE) + lazy val hostSpillStorageSize: Long = get(HOST_SPILL_STORAGE_SIZE) lazy val hasNans: Boolean = get(HAS_NANS) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuDeviceManagerSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuDeviceManagerSuite.scala index 08a0fc38c95..16af976f5a1 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuDeviceManagerSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuDeviceManagerSuite.scala @@ -34,6 +34,7 @@ class GpuDeviceManagerSuite extends FunSuite with Arm { .set(RapidsConf.POOLED_MEM.key, "true") .set(RapidsConf.RMM_ALLOC_FRACTION.key, initPoolFraction.toString) .set(RapidsConf.RMM_ALLOC_MAX_FRACTION.key, maxPoolFraction.toString) + .set(RapidsConf.RMM_ALLOC_RESERVE.key, "0") try { TestUtils.withGpuSparkSession(conf) { _ => val initPoolSize = (totalGpuSize * initPoolFraction).toLong @@ -41,7 +42,7 @@ class GpuDeviceManagerSuite extends FunSuite with Arm { // initial allocation should fit within initial pool size withResource(DeviceMemoryBuffer.allocate(allocSize)) { _ => // this should grow the pool - withResource(DeviceMemoryBuffer.allocate(allocSize)) { _ => + withResource(DeviceMemoryBuffer.allocate(allocSize / 2)) { _ => assertThrows[OutOfMemoryError] { // this should exceed the specified pool limit DeviceMemoryBuffer.allocate(allocSize).close() @@ -53,4 +54,27 @@ class GpuDeviceManagerSuite extends FunSuite with Arm { GpuDeviceManager.shutdown() } } + + test("RMM reserve larger than max") { + SparkSession.getActiveSession.foreach(_.close()) + GpuDeviceManager.shutdown() + val rapidsConf = new RapidsConf(Map(RapidsConf.RMM_ALLOC_RESERVE.key -> "200g")) + assertThrows[IllegalArgumentException] { + GpuDeviceManager.initializeMemory(None, Some(rapidsConf)) + } + } + + test("RMM init equals max") { + SparkSession.getActiveSession.foreach(_.close()) + GpuDeviceManager.shutdown() + val rapidsConf = new RapidsConf(Map( + RapidsConf.RMM_ALLOC_RESERVE.key -> "0", + RapidsConf.RMM_ALLOC_FRACTION.key -> "0.3", + RapidsConf.RMM_ALLOC_MAX_FRACTION.key -> "0.3")) + try { + GpuDeviceManager.initializeMemory(None, Some(rapidsConf)) + } finally { + GpuDeviceManager.shutdown() + } + } }