-
- Downloads
[SPARK-19093][SQL] Cached tables are not used in SubqueryExpression
## What changes were proposed in this pull request? Consider the plans inside subquery expressions while looking up cache manager to make use of cached data. Currently CacheManager.useCachedData does not consider the subquery expressions in the plan. SQL ``` select * from rows where not exists (select * from rows) ``` Before the fix ``` == Optimized Logical Plan == Join LeftAnti :- InMemoryRelation [_1#3775, _2#3776], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas) : +- *FileScan parquet [_1#3775,_2#3776] Batched: true, Format: Parquet, Location: InMemoryFileIndex[dbfs:/tmp/rows], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<_1:string,_2:string> +- Project [_1#3775 AS _1#3775#4001, _2#3776 AS _2#3776#4002] +- Relation[_1#3775,_2#3776] parquet ``` After ``` == Optimized Logical Plan == Join LeftAnti :- InMemoryRelation [_1#256, _2#257], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas) : +- *FileScan parquet [_1#256,_2#257] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/tmp/rows], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<_1:string,_2:string> +- Project [_1#256 AS _1#256#298, _2#257 AS _2#257#299] +- InMemoryRelation [_1#256, _2#257], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas) +- *FileScan parquet [_1#256,_2#257] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/tmp/rows], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<_1:string,_2:string> ``` Query2 ``` SELECT * FROM t1 WHERE c1 IN (SELECT c1 FROM t2 WHERE c1 IN (SELECT c1 FROM t3 WHERE c1 = 1)) ``` Before ``` == Analyzed Logical Plan == c1: int Project [c1#3] +- Filter predicate-subquery#47 [(c1#3 = c1#10)] : +- Project [c1#10] : +- Filter predicate-subquery#46 [(c1#10 = c1#17)] : : +- Project [c1#17] : : +- Filter (c1#17 = 1) : : +- SubqueryAlias t3, `t3` : : +- Project [value#15 AS c1#17] : : +- LocalRelation [value#15] : +- SubqueryAlias t2, `t2` : +- Project [value#8 AS c1#10] : +- LocalRelation [value#8] +- SubqueryAlias t1, `t1` +- Project [value#1 AS c1#3] +- LocalRelation [value#1] == Optimized Logical Plan == Join LeftSemi, (c1#3 = c1#10) :- InMemoryRelation [c1#3], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), t1 : +- LocalTableScan [c1#3] +- Project [value#8 AS c1#10] +- Join LeftSemi, (value#8 = c1#17) :- LocalRelation [value#8] +- Project [value#15 AS c1#17] +- Filter (value#15 = 1) +- LocalRelation [value#15] ``` After ``` == Analyzed Logical Plan == c1: int Project [c1#3] +- Filter predicate-subquery#47 [(c1#3 = c1#10)] : +- Project [c1#10] : +- Filter predicate-subquery#46 [(c1#10 = c1#17)] : : +- Project [c1#17] : : +- Filter (c1#17 = 1) : : +- SubqueryAlias t3, `t3` : : +- Project [value#15 AS c1#17] : : +- LocalRelation [value#15] : +- SubqueryAlias t2, `t2` : +- Project [value#8 AS c1#10] : +- LocalRelation [value#8] +- SubqueryAlias t1, `t1` +- Project [value#1 AS c1#3] +- LocalRelation [value#1] == Optimized Logical Plan == Join LeftSemi, (c1#3 = c1#10) :- InMemoryRelation [c1#3], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), t1 : +- LocalTableScan [c1#3] +- Join LeftSemi, (c1#10 = c1#17) :- InMemoryRelation [c1#10], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), t2 : +- LocalTableScan [c1#10] +- Filter (c1#17 = 1) +- InMemoryRelation [c1#17], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), t1 +- LocalTableScan [c1#3] ``` ## How was this patch tested? Added new tests in CachedTableSuite. Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #16493 from dilipbiswal/SPARK-19093.
Showing
- sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala 6 additions, 1 deletion...n/scala/org/apache/spark/sql/execution/CacheManager.scala
- sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala 72 additions, 0 deletions...rc/test/scala/org/apache/spark/sql/CachedTableSuite.scala
Please register or sign in to comment