Skip to content

Comments

[SPARK-3137][Core]Replace the global TorrentBroadcast lock with fine grained KeyLock#25612

Closed
zsxwing wants to merge 5 commits intoapache:masterfrom
zsxwing:SPARK-3137
Closed

[SPARK-3137][Core]Replace the global TorrentBroadcast lock with fine grained KeyLock#25612
zsxwing wants to merge 5 commits intoapache:masterfrom
zsxwing:SPARK-3137

Conversation

@zsxwing
Copy link
Member

@zsxwing zsxwing commented Aug 28, 2019

What changes were proposed in this pull request?

This PR provides a new lock mechanism KeyLock to lock with a given key. Also use this new lock in TorrentBroadcast to avoid blocking tasks from fetching different broadcast values.

Why are the changes needed?

TorrentBroadcast.readObject uses a global lock so only one task can be fetching the blocks at the same time. This is not optimal if we are running multiple stages concurrently because they should be able to independently fetch their own blocks.

Does this PR introduce any user-facing change?

No.

How was this patch tested?

Existing tests.

@zsxwing zsxwing requested a review from jiangxb1987 August 28, 2019 18:00
new ReferenceMap(AbstractReferenceMap.HARD, AbstractReferenceMap.WEAK)
Collections.synchronizedMap(
new ReferenceMap(AbstractReferenceMap.HARD, AbstractReferenceMap.WEAK)
.asInstanceOf[java.util.Map[Any, Any]]
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This one now needs protection as the global lock is removed. Casting it because Scala compiler cannot figure it out.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: would it make compiler happy and be nicer if the val was typed (val cachedValues: JMap[Any, Any]) instead of doing asInstanceOf?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No. The compiler doesn't like Java classes :P

* Otherwise, if not running in a task then immediately release the lock.
*/
private def releaseLock(blockId: BlockId): Unit = {
private def releaseBlockManagerLock(blockId: BlockId): Unit = {
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Rename this method to avoid confusing.

@gatorsmile
Copy link
Member

cc @Ngone51

@SparkQA
Copy link

SparkQA commented Aug 28, 2019

Test build #109880 has finished for PR 25612 at commit 290a3e5.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

private def acquireTorrentBroadcastLock(broadcastId: BroadcastBlockId): Unit = {
while (true) {
val lock = torrentBroadcastLock.putIfAbsent(broadcastId, new Object)
if (lock == null) return
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can this happen?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yep when no such key in ConcurrentHashMap.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Right OK, but then it isn't locking on anything the first time, how does this work?

Copy link
Member Author

@zsxwing zsxwing Aug 28, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • The caller adding an object to ConcurrentHashMap successfully wins the lock and acquireTorrentBroadcastLock will return. Others should wait until they add an object to ConcurrentHashMap successfully.
  • In releaseTorrentBroadcastLock, it will remove the object from ConcurrentHashMap and wake up threads waiting for this object to retry.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry I get it now. The Object isn't really the lock. It's the presence of absence of the Object. The Object is used to queue up waiting callers.

OK but this seems simpler if you computeIfAbsent a ReentrantLock and just use the lock as a lock directly. It manages all the details of allowing one caller, waking up waiters, etc. I'd imagine it's more straightforward to understand?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK but this seems simpler if you computeIfAbsent a ReentrantLock and just use the lock as a lock directly. It manages all the details of allowing one caller, waking up waiters, etc. I'd imagine it's more straightforward to understand?

The unlock code path is pretty tricky. Needs to unlock and remove from the map (the removal will allow a thread to put a new lock). I can try to make these two actions atomic by adding a global lock. But that's more complicated than the current way.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Isn't this all there is to it, more or less?

val lock = torrentBroadcastLock.computeIfAbsent(broadcastId, new ReentrantLock())
lock.lock()
try {
  func
} finally {
  lock.unlock()
}

The downside I suppose is that you have a map of locks that keeps growing, but how many broadcast IDs would there be? vs a relatively small lock object for each.

Copy link
Member Author

@zsxwing zsxwing Aug 29, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The number of broadcast IDs is significant enough for a long running application. Even if the user doesn't use broadcast, we will still use it internally, for example, task binary is a broadcast object, a Spark SQL query may use broadcast join...

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK, makes sense. Well, could still make sense to use a Condition but it wouldn't change much.

if (lock == null) return
lock.synchronized {
while (torrentBroadcastLock.get(broadcastId) eq lock) {
lock.wait()
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about just using a Java Lock implementation rather than re-roll the signalling?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Using a <ReentrantLock, Condition> pair? The codes would be more complicated.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, just a ReentrantLock. I think you don't need to deal with spurious wakeups then either

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I just tried ReentrantLock but didn't find a simple way to implement the same function.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why not just stripe over a fixed number of Reentrant locks? This should reduce contention enough (and potential collissions can be controlled with numLocks) is easy to understand and doesn't reimplement its own lock queues. It's also fully interruptible, while in the current implementation the threads that are queueing for the lock are not interruptible.

private val numLocks = 100
private val locks = (1 to numLocks).map(_ => new java.util.concurrent.locks.ReentrantLock)

private def withTorrentBroadcastLock[T](broadcastId: BroadcastBlockId)(func: => T): T = {
  val lock = locks(broadcastId.broadcastId % numLocks)
  lock.lockInterruptibly()
  try {
    func
  } finally {
    lock.unlock()
  }
}

@zsxwing zsxwing changed the title [SPARK-3137][Core]Replace the global TorrentBroadcast lock with fine grained locks [SPARK-3137][Core]Replace the global TorrentBroadcast lock with fine grained KeyLock Aug 29, 2019
@SparkQA
Copy link

SparkQA commented Aug 29, 2019

Test build #109916 has finished for PR 25612 at commit 2cf1826.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Aug 29, 2019

Test build #109917 has finished for PR 25612 at commit 844be5c.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

val lock = lockMap.putIfAbsent(key, new Object)
if (lock == null) return
lock.synchronized {
while (lockMap.get(key) eq lock) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

After releasing keylock, if a new thread for the same broadcastId enters to put a new object before another queueing thread to check lockMap.get(key) eq lock, both threads could get the keylock finally ?

Copy link
Member Author

@zsxwing zsxwing Aug 30, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The thread inside this inner while loop will end, and will retry the outer while loop. The line if (lock == null) return is the only exit. When acquireLock returns, this thread has putIfAbsent a new object to lockMap successfully.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh, I see.

@zsxwing
Copy link
Member Author

zsxwing commented Aug 30, 2019

@juliuszsompolski @srowen I added tests. Please take another look.

@@ -56,7 +57,10 @@ private[spark] class BroadcastManager(
private val nextBroadcastId = new AtomicLong(0)

private[broadcast] val cachedValues = {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You don't need the braces here, but I wouldn't bother unless you're changing this file again

@SparkQA
Copy link

SparkQA commented Aug 30, 2019

Test build #109955 has finished for PR 25612 at commit 2e9639c.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

@juliuszsompolski juliuszsompolski left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM!
Made two tiny nits reading through it.

new ReferenceMap(AbstractReferenceMap.HARD, AbstractReferenceMap.WEAK)
Collections.synchronizedMap(
new ReferenceMap(AbstractReferenceMap.HARD, AbstractReferenceMap.WEAK)
.asInstanceOf[java.util.Map[Any, Any]]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: would it make compiler happy and be nicer if the val was typed (val cachedValues: JMap[Any, Any]) instead of doing asInstanceOf?

val broadcastCache = SparkEnv.get.broadcastManager.cachedValues
broadcastCache.synchronized {
TorrentBroadcast.torrentBroadcastLock.withLock(broadcastId) {
// As we just lock based on `broadcastId`, whenever using `broadcastCache`, we should just
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: just -> only

@SparkQA
Copy link

SparkQA commented Sep 3, 2019

Test build #110051 has finished for PR 25612 at commit 9b59ca2.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@zsxwing
Copy link
Member Author

zsxwing commented Sep 3, 2019

Thanks! Merging to master.

@asfgit asfgit closed this in 8980093 Sep 3, 2019
@zsxwing zsxwing deleted the SPARK-3137 branch September 3, 2019 21:09
@jiangxb1987
Copy link
Contributor

A late LGTM :)

asfgit pushed a commit that referenced this pull request Sep 5, 2019
### What changes were proposed in this pull request?

Use `KeyLock` added in #25612 to simplify `MapOutputTracker.getStatuses`. It also has some improvement after the refactoring:
- `InterruptedException` is no longer sallowed.
- When a shuffle block is fetched, we don't need to wake up unrelated sleeping threads.

### Why are the changes needed?

`MapOutputTracker.getStatuses` is pretty hard to maintain right now because it has a special lock mechanism which we needs to pay attention to whenever updating this method. As we can use `KeyLock` to hide the complexity of locking behind a dedicated lock class, it's better to refactor it to make it easy to understand and maintain.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Existing tests.

Closes #25680 from zsxwing/getStatuses.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
PavithraRamachandran pushed a commit to PavithraRamachandran/spark that referenced this pull request Sep 15, 2019
### What changes were proposed in this pull request?

Use `KeyLock` added in apache#25612 to simplify `MapOutputTracker.getStatuses`. It also has some improvement after the refactoring:
- `InterruptedException` is no longer sallowed.
- When a shuffle block is fetched, we don't need to wake up unrelated sleeping threads.

### Why are the changes needed?

`MapOutputTracker.getStatuses` is pretty hard to maintain right now because it has a special lock mechanism which we needs to pay attention to whenever updating this method. As we can use `KeyLock` to hide the complexity of locking behind a dedicated lock class, it's better to refactor it to make it easy to understand and maintain.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Existing tests.

Closes apache#25680 from zsxwing/getStatuses.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

8 participants