Skip to content

Commit 69f1763

Browse files
committed
wip
1 parent 603660b commit 69f1763

File tree

6 files changed

+52
-22
lines changed

6 files changed

+52
-22
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala

Lines changed: 7 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -207,7 +207,8 @@ abstract class Optimizer(catalogManager: CatalogManager)
207207
CollapseProject,
208208
RemoveNoopOperators) :+
209209
// This batch must be executed after the `RewriteSubquery` batch, which creates joins.
210-
Batch("NormalizeFloatingNumbers", Once, NormalizeFloatingNumbers)
210+
Batch("NormalizeFloatingNumbers", Once, NormalizeFloatingNumbers) :+
211+
Batch("Final Filter Convert CNF", Once, finalScanFilterConvertRules: _*)
211212

212213
// remove any batches with no rules. this may happen when subclasses do not add optional rules.
213214
batches.filter(_.rules.nonEmpty)
@@ -273,6 +274,11 @@ abstract class Optimizer(catalogManager: CatalogManager)
273274
*/
274275
def earlyScanPushDownRules: Seq[Rule[LogicalPlan]] = Nil
275276

277+
/**
278+
* Override to provide additional rules for final filter convert to CNF.
279+
*/
280+
def finalScanFilterConvertRules: Seq[Rule[LogicalPlan]] = Nil
281+
276282
/**
277283
* Returns (defaultBatches - (excludedRules - nonExcludableRules)), the rule batches that
278284
* eventually run in the Optimizer.

sql/core/src/main/scala/org/apache/spark/sql/execution/PushCNFPredicateThroughFileScan.scala

Lines changed: 13 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -32,14 +32,18 @@ import org.apache.spark.sql.execution.datasources.LogicalRelation
3232
*/
3333
object PushCNFPredicateThroughFileScan extends Rule[LogicalPlan] with PredicateHelper {
3434

35-
def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
36-
case ScanOperation(projectList, conditions, relation: LogicalRelation)
37-
if conditions.nonEmpty =>
38-
val predicates = conjunctiveNormalFormAndGroupExpsByReference(conditions.reduceLeft(And))
39-
if (predicates.isEmpty) {
40-
plan
41-
} else {
42-
Project(projectList, Filter(predicates.reduceLeft(And), relation))
43-
}
35+
def apply(plan: LogicalPlan): LogicalPlan = {
36+
var resolved = false
37+
plan resolveOperatorsDown {
38+
case ScanOperation(projectList, conditions, relation: LogicalRelation)
39+
if conditions.nonEmpty && !resolved =>
40+
resolved = true
41+
val predicates = conjunctiveNormalFormAndGroupExpsByReference(conditions.reduceLeft(And))
42+
if (predicates.isEmpty) {
43+
plan
44+
} else {
45+
Project(projectList, Filter(predicates.reduceLeft(And), relation))
46+
}
47+
}
4448
}
4549
}

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -37,8 +37,10 @@ class SparkOptimizer(
3737

3838
override def earlyScanPushDownRules: Seq[Rule[LogicalPlan]] =
3939
// TODO: move SchemaPruning into catalyst
40-
SchemaPruning :: V2ScanRelationPushDown :: PushCNFPredicateThroughFileScan ::
41-
PruneFileSourcePartitions :: Nil
40+
SchemaPruning :: V2ScanRelationPushDown :: PruneFileSourcePartitions :: Nil
41+
42+
override def finalScanFilterConvertRules: Seq[Rule[LogicalPlan]] =
43+
PushCNFPredicateThroughFileScan :: Nil
4244

4345
override def defaultBatches: Seq[Batch] = (preOptimizationBatches ++ super.defaultBatches :+
4446
Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)) :+

sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -235,6 +235,9 @@ abstract class BaseSessionStateBuilder(
235235
override def earlyScanPushDownRules: Seq[Rule[LogicalPlan]] =
236236
super.earlyScanPushDownRules ++ customEarlyScanPushDownRules
237237

238+
override def finalScanFilterConvertRules: Seq[Rule[LogicalPlan]] =
239+
super.finalScanFilterConvertRules ++ customFinalScanFilterConvertRules
240+
238241
override def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] =
239242
super.extendedOperatorOptimizationRules ++ customOperatorOptimizationRules
240243
}
@@ -258,6 +261,14 @@ abstract class BaseSessionStateBuilder(
258261
*/
259262
protected def customEarlyScanPushDownRules: Seq[Rule[LogicalPlan]] = Nil
260263

264+
/**
265+
* Custom final scan filter convert rules to add to the Optimizer. Prefer overriding this instead
266+
* of creating your own Optimizer.
267+
*
268+
* Note that this may NOT depend on the `optimizer` function.
269+
*/
270+
protected def customFinalScanFilterConvertRules: Seq[Rule[LogicalPlan]] = Nil
271+
261272
/**
262273
* Planner that converts optimized logical plans to physical plans.
263274
*

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -99,7 +99,10 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session
9999
}
100100

101101
override def customEarlyScanPushDownRules: Seq[Rule[LogicalPlan]] =
102-
Seq(PushCNFPredicateThroughHiveTableScan, new PruneHiveTablePartitions(session))
102+
Seq(new PruneHiveTablePartitions(session))
103+
104+
override def customFinalScanFilterConvertRules: Seq[Rule[LogicalPlan]] =
105+
Seq(PushCNFPredicateThroughHiveTableScan)
103106

104107
/**
105108
* Planner that takes into account Hive-specific strategies.

sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PushCNFPredicateThroughHiveTableScan.scala

Lines changed: 13 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -31,14 +31,18 @@ import org.apache.spark.sql.catalyst.rules.Rule
3131
*/
3232
object PushCNFPredicateThroughHiveTableScan extends Rule[LogicalPlan] with PredicateHelper {
3333

34-
def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
35-
case ScanOperation(projectList, conditions, relation: HiveTableRelation)
36-
if conditions.nonEmpty =>
37-
val predicates = conjunctiveNormalFormAndGroupExpsByReference(conditions.reduceLeft(And))
38-
if (predicates.isEmpty) {
39-
plan
40-
} else {
41-
Project(projectList, Filter(predicates.reduceLeft(And), relation))
42-
}
34+
def apply(plan: LogicalPlan): LogicalPlan = {
35+
var resolved = false
36+
plan resolveOperatorsDown {
37+
case ScanOperation(projectList, conditions, relation: HiveTableRelation)
38+
if conditions.nonEmpty && !resolved =>
39+
resolved = true
40+
val predicates = conjunctiveNormalFormAndGroupExpsByReference(conditions.reduceLeft(And))
41+
if (predicates.isEmpty) {
42+
plan
43+
} else {
44+
Project(projectList, Filter(predicates.reduceLeft(And), relation))
45+
}
46+
}
4347
}
4448
}

0 commit comments

Comments
 (0)