From ff383250d4d179e56397108551e41de679a5f321 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 28 Oct 2021 14:53:16 -0500 Subject: [PATCH 1/8] Allow changing yarn resource gpu name Signed-off-by: Thomas Graves --- .../spark/rapids/GpuDeviceManager.scala | 22 +++++++++++++------ .../com/nvidia/spark/rapids/Plugin.scala | 2 +- .../com/nvidia/spark/rapids/RapidsConf.scala | 8 +++++++ 3 files changed, 24 insertions(+), 8 deletions(-) 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 323e4ad38ec..b6222b58851 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 @@ -58,6 +58,8 @@ object GpuDeviceManager extends Logging { // Attempt to set and acquire the gpu, return true if acquired, false otherwise def tryToSetGpuDeviceAndAcquire(addr: Int): Boolean = { + logWarning(s"try to set: $addr") + try { GpuDeviceManager.setGpuDeviceAndAcquire(addr) } catch { @@ -80,6 +82,7 @@ object GpuDeviceManager extends Logging { // we couldn't acquire on first try var numRetries = 2 val addrsToTry = ArrayBuffer.empty ++= (0 until deviceCount) + logWarning(s"find gpu and acquire: $addrsToTry") while (numRetries > 0) { val addr = addrsToTry.find(tryToSetGpuDeviceAndAcquire) if (addr.isDefined) { @@ -100,9 +103,12 @@ object GpuDeviceManager extends Logging { addr } - def getGPUAddrFromResources(resources: Map[String, ResourceInformation]): Option[Int] = { - if (resources.contains("gpu")) { - val addrs = resources("gpu").addresses + def getGPUAddrFromResources(resources: Map[String, ResourceInformation], + conf: RapidsConf): Option[Int] = { + val sparkGpuResourceName = conf.getSparkGpuResourceName + if (resources.contains(sparkGpuResourceName)) { + logWarning(s"spark resources contains: $sparkGpuResourceName") + val addrs = resources(sparkGpuResourceName).addresses if (addrs.length > 1) { // Throw an exception since we assume one GPU per executor. // If multiple GPUs are allocated by spark, then different tasks could get assigned @@ -112,22 +118,24 @@ object GpuDeviceManager extends Logging { } Some(addrs.head.toInt) } else { + logWarning(s"spark resources do NOT contain: $sparkGpuResourceName") None } } // Initializes the GPU if Spark assigned one. // Returns either the GPU addr Spark assigned or None if Spark didn't assign one. - def initializeGpu(resources: Map[String, ResourceInformation]): Option[Int] = { - getGPUAddrFromResources(resources).map(setGpuDeviceAndAcquire(_)) + def initializeGpu(resources: Map[String, ResourceInformation], conf: RapidsConf): Option[Int] = { + getGPUAddrFromResources(resources, conf).map(setGpuDeviceAndAcquire(_)) } - def initializeGpuAndMemory(resources: Map[String, ResourceInformation]): Unit = { + def initializeGpuAndMemory(resources: Map[String, ResourceInformation], + conf: RapidsConf): Unit = { // Set the GPU before RMM is initialized if spark provided the GPU address so that RMM // uses that GPU. We only need to initialize RMM once per Executor because we are relying on // only 1 GPU per executor. // If Spark didn't provide the address we just use the default GPU. - val addr = initializeGpu(resources) + val addr = initializeGpu(resources, conf) initializeMemory(addr) } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala index ee18e4b8cc5..e5e5d74fa3b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala @@ -183,7 +183,7 @@ class RapidsExecutorPlugin extends ExecutorPlugin with Logging { // on executor startup. if (!GpuDeviceManager.rmmTaskInitEnabled) { logInfo("Initializing memory from Executor Plugin") - GpuDeviceManager.initializeGpuAndMemory(pluginContext.resources().asScala.toMap) + GpuDeviceManager.initializeGpuAndMemory(pluginContext.resources().asScala.toMap, conf) if (GpuShuffleEnv.isRapidsShuffleAvailable) { GpuShuffleEnv.initShuffleManager() if (conf.shuffleTransportEarlyStart) { 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 19dcd63465b..05022c255cd 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 @@ -1292,6 +1292,12 @@ object RapidsConf { .booleanConf .createWithDefault(value = true) + val SPARK_GPU_RESOURCE_NAME = conf("spark.rapids.gpu.resourceName") + .doc("The name of the Spark resource that represents a GPU") + .internal() + .stringConf + .createWithDefault("gpu") + private def printSectionHeader(category: String): Unit = println(s"\n### $category") @@ -1701,6 +1707,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isRangeWindowLongEnabled: Boolean = get(ENABLE_RANGE_WINDOW_LONG) + lazy val getSparkGpuResourceName: String = get(SPARK_GPU_RESOURCE_NAME) + private val optimizerDefaults = Map( // this is not accurate because CPU projections do have a cost due to appending values // to each row that is produced, but this needs to be a really small number because From 5c24230c4efbda406777d3137ad5ac1219b034f9 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 29 Oct 2021 10:08:26 -0500 Subject: [PATCH 2/8] fix initialize from task --- .../scala/com/nvidia/spark/rapids/GpuDeviceManager.scala | 5 +++-- .../src/main/scala/com/nvidia/spark/rapids/Plugin.scala | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) 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 b6222b58851..922ea477690 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 @@ -161,11 +161,12 @@ object GpuDeviceManager extends Logging { def initializeFromTask(): Unit = { if (threadGpuInitialized.get() == false) { val resources = getResourcesFromTaskContext + val conf = new RapidsConf(SparkEnv.get.conf) if (rmmTaskInitEnabled) { - initializeGpuAndMemory(resources) + initializeGpuAndMemory(resources, conf) } else { // just set the device if provided so task thread uses right GPU - initializeGpu(resources) + initializeGpu(resources, conf) } threadGpuInitialized.set(true) } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala index e5e5d74fa3b..f9be2a98477 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala @@ -196,7 +196,7 @@ class RapidsExecutorPlugin extends ExecutorPlugin with Logging { val concurrentGpuTasks = conf.concurrentGpuTasks logInfo(s"The number of concurrent GPU tasks allowed is $concurrentGpuTasks") - GpuSemaphore.initialize(concurrentGpuTasks) + GpuSemaphore.initialize(concurrentGpuTasks, conf) } catch { case e: Throwable => // Exceptions in executor plugin can cause a single thread to die but the executor process From ac8d68e280a0dfc0077e37752286e12448bc7631 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 1 Nov 2021 10:06:50 -0500 Subject: [PATCH 3/8] fix --- sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala index f9be2a98477..e5e5d74fa3b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala @@ -196,7 +196,7 @@ class RapidsExecutorPlugin extends ExecutorPlugin with Logging { val concurrentGpuTasks = conf.concurrentGpuTasks logInfo(s"The number of concurrent GPU tasks allowed is $concurrentGpuTasks") - GpuSemaphore.initialize(concurrentGpuTasks, conf) + GpuSemaphore.initialize(concurrentGpuTasks) } catch { case e: Throwable => // Exceptions in executor plugin can cause a single thread to die but the executor process From 42370287d4ec252a912cb8bcbdd79008a6065915 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 3 Nov 2021 14:53:12 -0500 Subject: [PATCH 4/8] cleanup --- .../scala/com/nvidia/spark/rapids/GpuDeviceManager.scala | 6 +----- .../src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala | 4 ++-- 2 files changed, 3 insertions(+), 7 deletions(-) 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 922ea477690..df1c297601c 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 @@ -58,8 +58,6 @@ object GpuDeviceManager extends Logging { // Attempt to set and acquire the gpu, return true if acquired, false otherwise def tryToSetGpuDeviceAndAcquire(addr: Int): Boolean = { - logWarning(s"try to set: $addr") - try { GpuDeviceManager.setGpuDeviceAndAcquire(addr) } catch { @@ -82,7 +80,6 @@ object GpuDeviceManager extends Logging { // we couldn't acquire on first try var numRetries = 2 val addrsToTry = ArrayBuffer.empty ++= (0 until deviceCount) - logWarning(s"find gpu and acquire: $addrsToTry") while (numRetries > 0) { val addr = addrsToTry.find(tryToSetGpuDeviceAndAcquire) if (addr.isDefined) { @@ -107,7 +104,7 @@ object GpuDeviceManager extends Logging { conf: RapidsConf): Option[Int] = { val sparkGpuResourceName = conf.getSparkGpuResourceName if (resources.contains(sparkGpuResourceName)) { - logWarning(s"spark resources contains: $sparkGpuResourceName") + logDebug(s"Spark resources contain: $sparkGpuResourceName") val addrs = resources(sparkGpuResourceName).addresses if (addrs.length > 1) { // Throw an exception since we assume one GPU per executor. @@ -118,7 +115,6 @@ object GpuDeviceManager extends Logging { } Some(addrs.head.toInt) } else { - logWarning(s"spark resources do NOT contain: $sparkGpuResourceName") None } } 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 05022c255cd..361220f4796 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 @@ -1293,8 +1293,8 @@ object RapidsConf { .createWithDefault(value = true) val SPARK_GPU_RESOURCE_NAME = conf("spark.rapids.gpu.resourceName") - .doc("The name of the Spark resource that represents a GPU") - .internal() + .doc("The name of the Spark resource that represents a GPU that you want the plugin to use " + "if using custom resources with Spark.") .stringConf .createWithDefault("gpu") From 09bb142b93433c2d3a64aad8e882a3d841dbe827 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 3 Nov 2021 15:31:48 -0500 Subject: [PATCH 5/8] Doc updates --- docs/configs.md | 1 + .../src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/configs.md b/docs/configs.md index 315f4bdf39a..ca35827be5a 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -31,6 +31,7 @@ Name | Description | Default Value -----|-------------|-------------- spark.rapids.alluxio.pathsToReplace|List of paths to be replaced with corresponding alluxio scheme. Eg, when configureis set to "s3:/foo->alluxio://0.1.2.3:19998/foo,gcs:/bar->alluxio://0.1.2.3:19998/bar", which means: s3:/foo/a.csv will be replaced to alluxio://0.1.2.3:19998/foo/a.csv and gcs:/bar/b.csv will be replaced to alluxio://0.1.2.3:19998/bar/b.csv|None 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.gpu.resourceName|The name of the Spark resource that represents a GPU that you want the plugin to use if using custom resources with Spark.|gpu spark.rapids.memory.gpu.allocFraction|The fraction of available 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.|1.0 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.batchWriteBuffer.size|The size of the GPU memory buffer used to batch small buffers when spilling to GDS. Note that this buffer is mapped to the PCI Base Address Register (BAR) space, which may be very limited on some GPUs (e.g. the NVIDIA T4 only has 256 MiB), and it is also used by UCX bounce buffers.|8388608 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 361220f4796..379f7705749 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 @@ -1293,7 +1293,7 @@ object RapidsConf { .createWithDefault(value = true) val SPARK_GPU_RESOURCE_NAME = conf("spark.rapids.gpu.resourceName") - .doc("The name of the Spark resource that represents a GPU that you want the plugin to use " + .doc("The name of the Spark resource that represents a GPU that you want the plugin to use " + "if using custom resources with Spark.") .stringConf .createWithDefault("gpu") From a910a0b4885e8e50829db07c950c14889432e2ec Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 3 Nov 2021 16:21:06 -0500 Subject: [PATCH 6/8] update configs --- docs/configs.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configs.md b/docs/configs.md index df5ae414a7a..2462605b163 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -32,7 +32,7 @@ Name | Description | Default Value spark.rapids.alluxio.pathsToReplace|List of paths to be replaced with corresponding alluxio scheme. Eg, when configureis set to "s3:/foo->alluxio://0.1.2.3:19998/foo,gcs:/bar->alluxio://0.1.2.3:19998/bar", which means: s3:/foo/a.csv will be replaced to alluxio://0.1.2.3:19998/foo/a.csv and gcs:/bar/b.csv will be replaced to alluxio://0.1.2.3:19998/bar/b.csv|None 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.gpu.resourceName|The name of the Spark resource that represents a GPU that you want the plugin to use if using custom resources with Spark.|gpu -spark.rapids.memory.gpu.allocFraction|The fraction of available 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.|1.0 +spark.rapids.memory.gpu.allocFraction|The fraction of available (free) GPU memory that should be allocated for pooled memory. This must be less than or equal to the maximum limit configured via spark.rapids.memory.gpu.maxAllocFraction, and greater than or equal to the minimum limit configured via spark.rapids.memory.gpu.minAllocFraction.|1.0 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.batchWriteBuffer.size|The size of the GPU memory buffer used to batch small buffers when spilling to GDS. Note that this buffer is mapped to the PCI Base Address Register (BAR) space, which may be very limited on some GPUs (e.g. the NVIDIA T4 only has 256 MiB), and it is also used by UCX bounce buffers.|8388608 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 From b39730b41f4f606223aa04f1d1bac88dc97f1b6f Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 3 Nov 2021 17:00:32 -0500 Subject: [PATCH 7/8] Add test --- sql-plugin/pom.xml | 10 +++- .../spark/rapids/GpuDeviceManagerSuite.scala | 55 +++++++++++++++++++ 2 files changed, 64 insertions(+), 1 deletion(-) create mode 100644 sql-plugin/src/test/scala/com/nvidia/spark/rapids/GpuDeviceManagerSuite.scala diff --git a/sql-plugin/pom.xml b/sql-plugin/pom.xml index 5d73fde5f55..f9e65d80c5a 100644 --- a/sql-plugin/pom.xml +++ b/sql-plugin/pom.xml @@ -44,6 +44,11 @@ org.scala-lang scala-library + + org.scalatest + scalatest_${scala.binary.version} + test + org.apache.orc orc-core @@ -368,7 +373,10 @@ org.apache.rat apache-rat-plugin - + + org.scalatest + scalatest-maven-plugin + diff --git a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/GpuDeviceManagerSuite.scala b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/GpuDeviceManagerSuite.scala new file mode 100644 index 00000000000..bf9e7d22bc8 --- /dev/null +++ b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/GpuDeviceManagerSuite.scala @@ -0,0 +1,55 @@ +/* + * Copyright (c) 2021, 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.scalatest.{Assertion, FunSuite} + +import org.apache.spark.SparkConf +import org.apache.spark.resource.ResourceInformation + +class GpuDeviceManagerSuite extends FunSuite { + + test("Test Spark gpu resource") { + val sparkConf = new SparkConf() + val conf = new RapidsConf(sparkConf) + val gpu = new ResourceInformation("gpu", Array("3")) + val resources = Map("gpu" -> gpu) + val gpuAddr = GpuDeviceManager.getGPUAddrFromResources(resources, conf) + assert(gpuAddr.nonEmpty) + assert(gpuAddr.get == 3) + } + + test("Test Spark custom resource missed") { + val sparkConf = new SparkConf() + val conf = new RapidsConf(sparkConf) + val gpu = new ResourceInformation("nvidia/gpu", Array("2")) + val resources = Map("nvidia/gpu" -> gpu) + val gpuAddr = GpuDeviceManager.getGPUAddrFromResources(resources, conf) + assert(gpuAddr.isEmpty) + } + + test("Test Spark custom resource") { + val sparkConf = new SparkConf() + sparkConf.set(RapidsConf.SPARK_GPU_RESOURCE_NAME.toString, "nvidia/gpu") + val conf = new RapidsConf(sparkConf) + val gpu = new ResourceInformation("nvidia/gpu", Array("1")) + val resources = Map("nvidia/gpu" -> gpu) + val gpuAddr = GpuDeviceManager.getGPUAddrFromResources(resources, conf) + assert(gpuAddr.nonEmpty) + assert(gpuAddr.get == 1) + } +} From 28aa6e4fde9c89ff1f45c17194c3558bd4b7ba9e Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 3 Nov 2021 17:26:56 -0500 Subject: [PATCH 8/8] Add test for throws --- .../com/nvidia/spark/rapids/GpuDeviceManagerSuite.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/GpuDeviceManagerSuite.scala b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/GpuDeviceManagerSuite.scala index bf9e7d22bc8..c830a433312 100644 --- a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/GpuDeviceManagerSuite.scala +++ b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/GpuDeviceManagerSuite.scala @@ -42,6 +42,15 @@ class GpuDeviceManagerSuite extends FunSuite { assert(gpuAddr.isEmpty) } + test("Test Spark multiple GPUs throws") { + val sparkConf = new SparkConf() + val conf = new RapidsConf(sparkConf) + val gpu = new ResourceInformation("gpu", Array("2", "3")) + val resources = Map("gpu" -> gpu) + assertThrows[IllegalArgumentException]( + GpuDeviceManager.getGPUAddrFromResources(resources, conf)) + } + test("Test Spark custom resource") { val sparkConf = new SparkConf() sparkConf.set(RapidsConf.SPARK_GPU_RESOURCE_NAME.toString, "nvidia/gpu")