Skip to content
Draft
Original file line number Diff line number Diff line change
Expand Up @@ -853,6 +853,17 @@ object SQLConf {
.booleanConf
.createWithDefault(true)

val IN_MEMORY_CACHE_ENABLE_DSV2 =
buildConf("spark.sql.inMemoryColumnarStorage.enableDatasourceV2")
.doc("When true (default), cached DataFrames are represented as DataSourceV2Relation " +
"nodes, enabling Dynamic Partition Pruning and per-partition LIMIT pushdown on cached " +
"data. Set to false to revert to the pre-DSv2 InMemoryRelation path, which still " +
"supports column pruning, filter pushdown, and sort-order propagation via the " +
"InMemoryScans planner strategy.")
.version("4.0.0")
.booleanConf
.createWithDefault(true)

val COLUMN_VECTOR_OFFHEAP_ENABLED =
buildConf("spark.sql.columnVector.offheap.enabled")
.internal()
Expand Down Expand Up @@ -7984,6 +7995,8 @@ class SQLConf extends Serializable with Logging with SqlApiConf {

def inMemoryPartitionPruning: Boolean = getConf(IN_MEMORY_PARTITION_PRUNING)

def inMemoryCacheEnableDSv2: Boolean = getConf(IN_MEMORY_CACHE_ENABLE_DSV2)

def inMemoryTableScanStatisticsEnabled: Boolean = getConf(IN_MEMORY_TABLE_SCAN_STATISTICS_ENABLED)

def offHeapColumnVectorEnabled: Boolean = getConf(COLUMN_VECTOR_OFFHEAP_ENABLED)
Expand Down
46 changes: 46 additions & 0 deletions sql/core/benchmarks/InMemoryCacheDSv2Benchmark-jdk21-results.txt
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
================================================================================================
In-memory cache: column pruning
================================================================================================

OpenJDK 64-Bit Server VM 21.0.9 on Mac OS X 26.4
Apple M3 Max
Column pruning - 1000000 rows, 10 cols, select 2: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------
sum 2 of 10 cols (column pruning via DSv2) 19 24 4 52.2 19.2 1.0X
sum all 10 cols (no pruning - pre-DSv2 baseline) 54 57 3 18.4 54.5 0.4X


================================================================================================
In-memory cache: filter pushdown
================================================================================================

OpenJDK 64-Bit Server VM 21.0.9 on Mac OS X 26.4
Apple M3 Max
Filter pushdown - 1000000 rows, selective filter (c0 < 1000): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------------------------------------------------------------------
filter c0 < 1000 (pushed to scan, batch pruning) 7 10 2 151.3 6.6 1.0X
filter c0 < 1000 (count with full scan for comparison) 7 10 2 141.4 7.1 0.9X


================================================================================================
In-memory cache: planning overhead
================================================================================================

OpenJDK 64-Bit Server VM 21.0.9 on Mac OS X 26.4
Apple M3 Max
Planning overhead - 1000 plan-only iterations: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
---------------------------------------------------------------------------------------------------------------------------------
optimizedPlan (DSv2 path, V2ScanRelationPushDown) 262 323 55 0.0 261916.9 1.0X


================================================================================================
In-memory cache: end-to-end aggregate
================================================================================================

OpenJDK 64-Bit Server VM 21.0.9 on Mac OS X 26.4
Apple M3 Max
End-to-end aggregate (groupBy + sum) on 1000000 rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
------------------------------------------------------------------------------------------------------------------------------------
groupBy(key).sum(val) - DSv2 path 26 29 3 38.3 26.1 1.0X


Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ import org.apache.spark.internal.LogKeys._
import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
import org.apache.spark.sql.catalyst.analysis.Resolver
import org.apache.spark.sql.catalyst.catalog.HiveTableRelation
import org.apache.spark.sql.catalyst.expressions.{Attribute, SubqueryExpression}
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, SubqueryExpression}
import org.apache.spark.sql.catalyst.optimizer.EliminateResolvedHint
import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, ResolvedHint, View}
import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION
Expand All @@ -36,11 +36,12 @@ import org.apache.spark.sql.connector.catalog.CatalogPlugin
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.{IdentifierHelper, MultipartIdentifierHelper}
import org.apache.spark.sql.connector.catalog.Identifier
import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
import org.apache.spark.sql.execution.columnar.InMemoryRelation
import org.apache.spark.sql.execution.columnar.{InMemoryCacheTable, InMemoryRelation}
import org.apache.spark.sql.execution.command.CommandUtils
import org.apache.spark.sql.execution.datasources.{FileIndex, HadoopFsRelation, LogicalRelation, LogicalRelationWithTable}
import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, ExtractV2CatalogAndIdentifier, ExtractV2Table, FileTable, V2TableRefreshUtil}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.util.CaseInsensitiveStringMap
import org.apache.spark.storage.StorageLevel
import org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK

Expand Down Expand Up @@ -332,8 +333,19 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper {
cachedData: CachedData,
column: Seq[Attribute]): Unit = {
val relation = cachedData.cachedRepresentation
val planToAnalyze: LogicalPlan = if (SQLConf.get.inMemoryCacheEnableDSv2) {
DataSourceV2Relation(
table = new InMemoryCacheTable(relation),
output = relation.output.map { case ar: AttributeReference => ar },
catalog = None,
identifier = None,
options = CaseInsensitiveStringMap.empty()
)
} else {
relation
}
val (rowCount, newColStats) =
CommandUtils.computeColumnStats(sparkSession, relation, column)
CommandUtils.computeColumnStats(sparkSession, planToAnalyze, column)
relation.updateStats(rowCount, newColStats)
}

Expand Down Expand Up @@ -494,6 +506,7 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper {
* plan must be normalized.
*/
private[sql] def useCachedData(plan: LogicalPlan): LogicalPlan = {
val dsv2Enabled = SQLConf.get.inMemoryCacheEnableDSv2
val newPlan = plan transformDown {
case command: Command => command

Expand All @@ -502,9 +515,24 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper {
// After cache lookup, we should still keep the hints from the input plan.
val hints = EliminateResolvedHint.extractHintsFromPlan(currentFragment)._2
val cachedPlan = cached.cachedRepresentation.withOutput(currentFragment.output)
// When DSv2 is enabled (default), wrap in DataSourceV2Relation so that
// V2ScanRelationPushDown can apply DPP and per-partition LIMIT pushdown.
// When disabled, return InMemoryRelation directly; InMemoryScans handles it via
// pruneFilterProject (column pruning, filter pushdown, sort-order propagation).
val substitutedPlan: LogicalPlan = if (dsv2Enabled) {
DataSourceV2Relation(
table = new InMemoryCacheTable(cachedPlan),
output = cachedPlan.output.map { case ar: AttributeReference => ar },
catalog = None,
identifier = None,
options = CaseInsensitiveStringMap.empty()
)
} else {
cachedPlan
}
// The returned hint list is in top-down order, we should create the hint nodes from
// right to left.
hints.foldRight[LogicalPlan](cachedPlan) { case (hint, p) =>
hints.foldRight[LogicalPlan](substitutedPlan) { case (hint, p) =>
ResolvedHint(p, hint)
}
}.getOrElse(currentFragment)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -704,6 +704,15 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
}
}

/**
* Strategy for cached in-memory tables.
*
* Under the default DSv2 path (spark.sql.inMemoryColumnarStorage.enableDatasourceV2 = true),
* CacheManager wraps InMemoryRelation in DataSourceV2Relation, so this strategy is only
* reached when DSv2 is disabled. Both paths produce InMemoryTableScanExec and support column
* pruning, filter pushdown, and sort-order propagation; the DSv2 path additionally enables
* Dynamic Partition Pruning and per-partition LIMIT pushdown.
*/
object InMemoryScans extends Strategy {
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case PhysicalOperation(projectList, filters, mem: InMemoryRelation) =>
Expand Down
Loading