Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-26917][SQL] Further reduce locks in CacheManager #24028

Closed
wants to merge 17 commits into from
Closed
Changes from 14 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,7 @@ package org.apache.spark.sql.execution

import java.util.concurrent.locks.ReentrantReadWriteLock

import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.collection.immutable.IndexedSeq

import org.apache.hadoop.fs.{FileSystem, Path}

Expand All @@ -46,38 +45,17 @@ case class CachedData(plan: LogicalPlan, cachedRepresentation: InMemoryRelation)
*/
class CacheManager extends Logging {

@transient
private val cachedData = new java.util.LinkedList[CachedData]

@transient
private val cacheLock = new ReentrantReadWriteLock

/** Acquires a read lock on the cache for the duration of `f`. */
private def readLock[A](f: => A): A = {
val lock = cacheLock.readLock()
lock.lock()
try f finally {
lock.unlock()
}
}

/** Acquires a write lock on the cache for the duration of `f`. */
private def writeLock[A](f: => A): A = {
val lock = cacheLock.writeLock()
lock.lock()
try f finally {
lock.unlock()
}
}
@transient @volatile
private var cachedData = IndexedSeq[CachedData]()
Copy link
Contributor

Choose a reason for hiding this comment

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

we'd better add some comments to explain the access pattern of cachedData. e.g. adding/removing elements should be done by creating a new seq, and locked by this.


/** Clears all cached tables. */
def clearCache(): Unit = writeLock {
cachedData.asScala.foreach(_.cachedRepresentation.cacheBuilder.clearCache())
cachedData.clear()
def clearCache(): Unit = this.synchronized {
cachedData.foreach(_.cachedRepresentation.cacheBuilder.clearCache())
cachedData = IndexedSeq[CachedData]()
}

/** Checks if the cache is empty. */
def isEmpty: Boolean = readLock {
Copy link
Contributor

Choose a reason for hiding this comment

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

can we remove the readLock method?

Copy link
Contributor

Choose a reason for hiding this comment

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

I think we can remove writeLock as well, and simply use this.synchronized

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I made this change, but while doing that I realized there was an issue with the earlier change to the "partition" function. See comment below

val (plansToUncache, remainingPlans) = cachedData.partition(cd => shouldRemove(cd.plan))
// If a new plan is cached by a different thread at this point, it will be in the cachedData object,
// but not in plansToUncache or remainingPlans. So the next line will remove it.
writeLock {
cachedData = remainingPlans
}

Copy link
Contributor Author

Choose a reason for hiding this comment

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

So I reverted this back to the previous behavior.

def isEmpty: Boolean = {
cachedData.isEmpty
}

Expand All @@ -101,11 +79,11 @@ class CacheManager extends Logging {
sparkSession.sessionState.executePlan(planToCache).executedPlan,
tableName,
planToCache)
writeLock {
this.synchronized {
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: please use space instead of tab.

if (lookupCachedData(planToCache).nonEmpty) {
logWarning("Data has already been cached.")
} else {
cachedData.add(CachedData(planToCache, inMemoryRelation))
cachedData = CachedData(planToCache, inMemoryRelation) +: cachedData
}
}
}
Expand Down Expand Up @@ -144,22 +122,12 @@ class CacheManager extends Logging {
} else {
_.sameResult(plan)
}
val plansToUncache = mutable.Buffer[CachedData]()
readLock {
val it = cachedData.iterator()
while (it.hasNext) {
val cd = it.next()
if (shouldRemove(cd.plan)) {
plansToUncache += cd
}
}
}
plansToUncache.foreach { cd =>
writeLock {
cachedData.remove(cd)
}
cd.cachedRepresentation.cacheBuilder.clearCache(blocking)
val plansToUncache = cachedData.filter(cd => shouldRemove(cd.plan))
this.synchronized {
Copy link
Contributor

Choose a reason for hiding this comment

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

ditto

cachedData = cachedData.filter(!plansToUncache.contains(_))
}
plansToUncache.foreach { _.cachedRepresentation.cacheBuilder.clearCache(blocking) }

// Re-compile dependent cached queries after removing the cached query.
if (!cascade) {
recacheByCondition(spark, cd => {
Expand Down Expand Up @@ -194,46 +162,36 @@ class CacheManager extends Logging {
private def recacheByCondition(
spark: SparkSession,
condition: CachedData => Boolean): Unit = {
val needToRecache = scala.collection.mutable.ArrayBuffer.empty[CachedData]
readLock {
val it = cachedData.iterator()
while (it.hasNext) {
val cd = it.next()
if (condition(cd)) {
needToRecache += cd
}
}
val needToRecache = cachedData.filter(condition)
this.synchronized {
// Remove the cache entry before creating a new ones.
cachedData = cachedData.filter(!needToRecache.contains(_))
}
needToRecache.map { cd =>
writeLock {
// Remove the cache entry before we create a new one, so that we can have a different
// physical plan.
cachedData.remove(cd)
}
cd.cachedRepresentation.cacheBuilder.clearCache()
val plan = spark.sessionState.executePlan(cd.plan).executedPlan
val newCache = InMemoryRelation(
cacheBuilder = cd.cachedRepresentation.cacheBuilder.copy(cachedPlan = plan),
logicalPlan = cd.plan)
val recomputedPlan = cd.copy(cachedRepresentation = newCache)
writeLock {
this.synchronized {
if (lookupCachedData(recomputedPlan.plan).nonEmpty) {
logWarning("While recaching, data was already added to cache.")
} else {
cachedData.add(recomputedPlan)
cachedData = recomputedPlan +: cachedData
}
}
}
}

/** Optionally returns cached data for the given [[Dataset]] */
def lookupCachedData(query: Dataset[_]): Option[CachedData] = readLock {
def lookupCachedData(query: Dataset[_]): Option[CachedData] = {
lookupCachedData(query.logicalPlan)
}

/** Optionally returns cached data for the given [[LogicalPlan]]. */
def lookupCachedData(plan: LogicalPlan): Option[CachedData] = readLock {
cachedData.asScala.find(cd => plan.sameResult(cd.plan))
def lookupCachedData(plan: LogicalPlan): Option[CachedData] = {
cachedData.find(cd => plan.sameResult(cd.plan))
}

/** Replaces segments of the given logical plan with cached versions where possible. */
Expand Down