File tree Expand file tree Collapse file tree 3 files changed +57
-3
lines changed
catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions
main/scala/org/apache/spark/sql/execution
test/scala/org/apache/spark/sql Expand file tree Collapse file tree 3 files changed +57
-3
lines changed Original file line number Diff line number Diff line change @@ -74,12 +74,14 @@ object SubqueryExpression {
7474 }
7575
7676 /**
77- * Returns true when an expression contains a subquery that has outer reference(s). The outer
77+ * Returns true when an expression contains a subquery that has outer reference(s) except
78+ * the [[org.apache.spark.sql.catalyst.expressions.DynamicPruningSubquery ]]. The outer
7879 * reference attributes are kept as children of subquery expression by
7980 * [[org.apache.spark.sql.catalyst.analysis.Analyzer.ResolveSubquery ]]
8081 */
8182 def hasCorrelatedSubquery (e : Expression ): Boolean = {
8283 e.find {
84+ case _ : DynamicPruningSubquery => false
8385 case s : SubqueryExpression => s.children.nonEmpty
8486 case _ => false
8587 }.isDefined
Original file line number Diff line number Diff line change @@ -44,8 +44,9 @@ class SparkOptimizer(
4444 Batch (" PartitionPruning" , Once ,
4545 PartitionPruning ,
4646 OptimizeSubqueries ) :+
47- Batch (" Pushdown Filters from PartitionPruning" , fixedPoint,
48- PushDownPredicates ) :+
47+ Batch (" Pushdown and infer Filters from PartitionPruning" , fixedPoint,
48+ PushDownPredicates ,
49+ InferFiltersFromConstraints ) :+
4950 Batch (" Cleanup filters that cannot be pushed down" , Once ,
5051 CleanupDynamicPruningFilters ,
5152 PruneFilters )) ++
Original file line number Diff line number Diff line change @@ -1280,6 +1280,57 @@ abstract class DynamicPartitionPruningSuiteBase
12801280 )
12811281 }
12821282 }
1283+
1284+ test(" Infer DPP filter to other partition column" ) {
1285+ val df = sql(
1286+ """
1287+ |SELECT t11.store_id,
1288+ | t11.code,
1289+ | t3.product_id
1290+ |FROM (SELECT t1.store_id,
1291+ | t2.code
1292+ | FROM fact_stats t1
1293+ | JOIN code_stats t2
1294+ | ON t1.store_id = t2.store_id) t11
1295+ | JOIN product t3
1296+ | ON t11.store_id = t3.store_id AND t3.product_id < 3
1297+ |""" .stripMargin)
1298+
1299+ assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 2 )
1300+ checkDistinctSubqueries(df, 1 )
1301+ checkPartitionPruningPredicate(df, false , true )
1302+
1303+ checkAnswer(df,
1304+ Row (2 , 20 , 2 ) ::
1305+ Row (2 , 20 , 2 ) ::
1306+ Row (1 , 10 , 1 ) :: Nil
1307+ )
1308+ }
1309+
1310+ test(" Should not infer DPP filter to other non-partition column" ) {
1311+ val df = sql(
1312+ """
1313+ |SELECT t11.store_id,
1314+ | t11.country,
1315+ | t3.product_id
1316+ |FROM (SELECT t1.store_id,
1317+ | t2.country
1318+ | FROM fact_stats t1
1319+ | JOIN dim_stats t2
1320+ | ON t1.store_id = t2.store_id) t11
1321+ | JOIN product t3
1322+ | ON t11.store_id = t3.store_id AND t3.product_id < 3
1323+ |""" .stripMargin)
1324+
1325+ assert(collectDynamicPruningExpressions(df.queryExecution.executedPlan).size === 1 )
1326+ checkPartitionPruningPredicate(df, false , true )
1327+
1328+ checkAnswer(df,
1329+ Row (2 , " NL" , 2 ) ::
1330+ Row (2 , " NL" , 2 ) ::
1331+ Row (1 , " NL" , 1 ) :: Nil
1332+ )
1333+ }
12831334}
12841335
12851336class DynamicPartitionPruningSuiteAEOff extends DynamicPartitionPruningSuiteBase {
You can’t perform that action at this time.
0 commit comments