Skip to content
Closed
Show file tree
Hide file tree
Changes from all 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 @@ -26,7 +26,7 @@ import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{Dataset, SparkSession}
import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ResolvedHint}
import org.apache.spark.sql.catalyst.plans.logical.{AnalysisBarrier, LogicalPlan, ResolvedHint}
import org.apache.spark.sql.execution.columnar.InMemoryRelation
import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
import org.apache.spark.storage.StorageLevel
Expand Down Expand Up @@ -97,7 +97,7 @@ class CacheManager extends Logging {
val inMemoryRelation = InMemoryRelation(
sparkSession.sessionState.conf.useCompression,
sparkSession.sessionState.conf.columnBatchSize, storageLevel,
sparkSession.sessionState.executePlan(planToCache).executedPlan,
sparkSession.sessionState.executePlan(AnalysisBarrier(planToCache)).executedPlan,
tableName,
planToCache)
cachedData.add(CachedData(planToCache, inMemoryRelation))
Expand Down Expand Up @@ -142,7 +142,7 @@ class CacheManager extends Logging {
// Remove the cache entry before we create a new one, so that we can have a different
// physical plan.
it.remove()
val plan = spark.sessionState.executePlan(cd.plan).executedPlan
val plan = spark.sessionState.executePlan(AnalysisBarrier(cd.plan)).executedPlan
val newCache = InMemoryRelation(
cacheBuilder = cd.cachedRepresentation
.cacheBuilder.copy(cachedPlan = plan)(_cachedColumnBuffers = null),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ package org.apache.spark.sql
import org.scalatest.concurrent.TimeLimits
import org.scalatest.time.SpanSugar._

import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.storage.StorageLevel
Expand Down Expand Up @@ -132,4 +133,19 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits
df.unpersist()
assert(df.storageLevel == StorageLevel.NONE)
}

test("SPARK-24613 Cache with UDF could not be matched with subsequent dependent caches") {
val udf1 = udf({x: Int => x + 1})
val df = spark.range(0, 10).toDF("a").withColumn("b", udf1($"a"))
val df2 = df.agg(sum(df("b")))

df.cache()
df.count()
df2.cache()

val plan = df2.queryExecution.withCachedData
assert(plan.isInstanceOf[InMemoryRelation])
val internalPlan = plan.asInstanceOf[InMemoryRelation].cacheBuilder.cachedPlan
assert(internalPlan.find(_.isInstanceOf[InMemoryTableScanExec]).isDefined)
}
}