-
Notifications
You must be signed in to change notification settings - Fork 244
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Create a PrioritySemaphore to back the GpuSemaphore (#11376)
* priority semaphore implementation and tests Signed-off-by: Zach Puller <zpuller@nvidia.com> --------- Signed-off-by: Zach Puller <zpuller@nvidia.com>
- Loading branch information
Showing
3 changed files
with
195 additions
and
6 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
92 changes: 92 additions & 0 deletions
92
sql-plugin/src/main/scala/com/nvidia/spark/rapids/PrioritySemaphore.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,92 @@ | ||
/* | ||
* Copyright (c) 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. | ||
* 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.util.concurrent.locks.{Condition, ReentrantLock} | ||
|
||
import scala.collection.mutable.PriorityQueue | ||
|
||
object PrioritySemaphore { | ||
private val DEFAULT_MAX_PERMITS = 1000 | ||
} | ||
|
||
class PrioritySemaphore[T](val maxPermits: Int)(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 | ||
// won't work together properly, and we see things like deadlocks. | ||
private val lock = new ReentrantLock() | ||
private var occupiedSlots: Int = 0 | ||
|
||
private case class ThreadInfo(priority: T, condition: Condition) | ||
|
||
// We expect a relatively small number of threads to be contending for this lock at any given | ||
// time, therefore we are not concerned with the insertion/removal time complexity. | ||
private val waitingQueue: PriorityQueue[ThreadInfo] = PriorityQueue()(Ordering.by(_.priority)) | ||
|
||
def this()(implicit ordering: Ordering[T]) = this(PrioritySemaphore.DEFAULT_MAX_PERMITS)(ordering) | ||
|
||
def tryAcquire(numPermits: Int): Boolean = { | ||
lock.lock() | ||
try { | ||
if (canAcquire(numPermits)) { | ||
commitAcquire(numPermits) | ||
true | ||
} else { | ||
false | ||
} | ||
} finally { | ||
lock.unlock() | ||
} | ||
} | ||
|
||
def acquire(numPermits: Int, priority: T): Unit = { | ||
lock.lock() | ||
try { | ||
val condition = lock.newCondition() | ||
while (!canAcquire(numPermits)) { | ||
waitingQueue.enqueue(ThreadInfo(priority, condition)) | ||
condition.await() | ||
} | ||
commitAcquire(numPermits) | ||
|
||
} finally { | ||
lock.unlock() | ||
}} | ||
|
||
private def commitAcquire(numPermits: Int): Unit = { | ||
occupiedSlots += numPermits | ||
} | ||
|
||
def release(numPermits: Int): Unit = { | ||
lock.lock() | ||
try { | ||
occupiedSlots -= numPermits | ||
if (waitingQueue.nonEmpty) { | ||
val nextThread = waitingQueue.dequeue() | ||
nextThread.condition.signal() | ||
} | ||
} finally { | ||
lock.unlock() | ||
} | ||
} | ||
|
||
private def canAcquire(numPermits: Int): Boolean = { | ||
occupiedSlots + numPermits <= maxPermits | ||
} | ||
|
||
} |
91 changes: 91 additions & 0 deletions
91
tests/src/test/scala/com/nvidia/spark/rapids/PrioritySemaphoreSuite.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,91 @@ | ||
/* | ||
* Copyright (c) 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. | ||
* 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.util.concurrent.{CountDownLatch, TimeUnit} | ||
|
||
import scala.collection.JavaConverters._ | ||
|
||
import org.scalatest.funsuite.AnyFunSuite | ||
|
||
class PrioritySemaphoreSuite extends AnyFunSuite { | ||
type TestPrioritySemaphore = PrioritySemaphore[Long] | ||
|
||
test("tryAcquire should return true if permits are available") { | ||
val semaphore = new TestPrioritySemaphore(10) | ||
|
||
assert(semaphore.tryAcquire(5)) | ||
assert(semaphore.tryAcquire(3)) | ||
assert(semaphore.tryAcquire(2)) | ||
assert(!semaphore.tryAcquire(1)) | ||
} | ||
|
||
test("acquire and release should work correctly") { | ||
val semaphore = new TestPrioritySemaphore(1) | ||
|
||
assert(semaphore.tryAcquire(1)) | ||
|
||
val latch = new CountDownLatch(1) | ||
val t = new Thread(() => { | ||
try { | ||
semaphore.acquire(1, 1) | ||
fail("Should not acquire permit") | ||
} catch { | ||
case _: InterruptedException => | ||
semaphore.acquire(1, 1) | ||
} finally { | ||
latch.countDown() | ||
} | ||
}) | ||
t.start() | ||
|
||
Thread.sleep(100) | ||
t.interrupt() | ||
|
||
semaphore.release(1) | ||
|
||
latch.await(1, TimeUnit.SECONDS) | ||
} | ||
|
||
test("multiple threads should handle permits and priority correctly") { | ||
val semaphore = new TestPrioritySemaphore(0) | ||
val latch = new CountDownLatch(3) | ||
val results = new java.util.ArrayList[Int]() | ||
|
||
def taskWithPriority(priority: Int) = new Runnable { | ||
override def run(): Unit = { | ||
try { | ||
semaphore.acquire(1, priority) | ||
results.add(priority) | ||
semaphore.release(1) | ||
} finally { | ||
latch.countDown() | ||
} | ||
} | ||
} | ||
|
||
new Thread(taskWithPriority(2)).start() | ||
new Thread(taskWithPriority(1)).start() | ||
new Thread(taskWithPriority(3)).start() | ||
|
||
Thread.sleep(100) | ||
semaphore.release(1) | ||
|
||
latch.await(1, TimeUnit.SECONDS) | ||
assert(results.asScala.toList == List(3, 2, 1)) | ||
} | ||
} |