Github user windpiger commented on a diff in the pull request:
https://github.com/apache/spark/pull/17097#discussion_r103411010
--- Diff:
sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala ---
@@ -88,46 +90,93 @@ class CacheManager extends Logging {
query: Dataset[_],
tableName: Option[String] = None,
storageLevel: StorageLevel = MEMORY_AND_DISK): Unit = writeLock {
- val planToCache = query.queryExecution.analyzed
+ val planToCache = query.logicalPlan
if (lookupCachedData(planToCache).nonEmpty) {
logWarning("Asked to cache already cached data.")
} else {
val sparkSession = query.sparkSession
- cachedData +=
- CachedData(
- planToCache,
- InMemoryRelation(
- sparkSession.sessionState.conf.useCompression,
- sparkSession.sessionState.conf.columnBatchSize,
- storageLevel,
-
sparkSession.sessionState.executePlan(planToCache).executedPlan,
- tableName))
+ cachedData.add(CachedData(
+ planToCache,
+ InMemoryRelation(
+ sparkSession.sessionState.conf.useCompression,
+ sparkSession.sessionState.conf.columnBatchSize,
+ storageLevel,
+ query.queryExecution.executedPlan,
+ tableName)))
}
}
/**
- * Tries to remove the data for the given [[Dataset]] from the cache.
- * No operation, if it's already uncached.
+ * Tries to remove the cache entry of the given query, no operation, if
it's already uncached.
+ * Note that all other caches that refer to this plan will be re-cached.
+ *
+ * @return true if a cache entry is found and removed, false otherwise.
*/
def uncacheQuery(query: Dataset[_], blocking: Boolean = true): Boolean =
writeLock {
- val planToCache = query.queryExecution.analyzed
- val dataIndex = cachedData.indexWhere(cd =>
planToCache.sameResult(cd.plan))
- val found = dataIndex >= 0
+ uncacheQuery(query.sparkSession, query.logicalPlan, blocking)
+ }
+
+ /**
+ * Tries to remove the cache entry of the given plan, no operation, if
it's already uncached.
+ * Note that all other caches that refer to this plan will be re-cached.
+ *
+ * @return true if a cache entry is found and removed, false otherwise.
+ */
+ def uncacheQuery(spark: SparkSession, plan: LogicalPlan, blocking:
Boolean): Boolean = writeLock {
+ val it = cachedData.iterator()
+ var found = false
+ while (it.hasNext && !found) {
+ val cd = it.next()
+ if (cd.plan.sameResult(plan)) {
+ cd.cachedRepresentation.cachedColumnBuffers.unpersist(blocking)
+ it.remove()
+ found = true
+ }
+ }
if (found) {
-
cachedData(dataIndex).cachedRepresentation.cachedColumnBuffers.unpersist(blocking)
- cachedData.remove(dataIndex)
+ recacheByPlan(spark, plan)
--- End diff --
this will change the action of unpersist.
https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala#L2530
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]