Skip to content

Commit 2189ce0

Browse files
committed
Remove ResolvedHint case in PhysicalOperation.unapply instead
1 parent 2fcfb79 commit 2189ce0

File tree

2 files changed

+2
-11
lines changed

2 files changed

+2
-11
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala

Lines changed: 0 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -66,9 +66,6 @@ object PhysicalOperation extends PredicateHelper {
6666
val substitutedCondition = substitute(aliases)(condition)
6767
(fields, filters ++ splitConjunctivePredicates(substitutedCondition), other, aliases)
6868

69-
case h: ResolvedHint =>
70-
collectProjectsAndFilters(h.child)
71-
7269
case other =>
7370
(None, Nil, other, Map.empty)
7471
}

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala

Lines changed: 2 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources
2020
import org.apache.spark.sql.catalyst.catalog.CatalogStatistics
2121
import org.apache.spark.sql.catalyst.expressions._
2222
import org.apache.spark.sql.catalyst.planning.PhysicalOperation
23-
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project, ResolvedHint}
23+
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
2424
import org.apache.spark.sql.catalyst.rules.Rule
2525

2626
private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] {
@@ -71,13 +71,7 @@ private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] {
7171
// Keep partition-pruning predicates so that they are visible in physical planning
7272
val filterExpression = filters.reduceLeft(And)
7373
val filter = Filter(filterExpression, prunedLogicalRelation)
74-
op match {
75-
case h: ResolvedHint =>
76-
// Restore the ResolvedHint removed by PhysicalOperation.collectProjectsAndFilters
77-
h.copy(child = Project(projects, filter))
78-
case _ =>
79-
Project(projects, filter)
80-
}
74+
Project(projects, filter)
8175
} else {
8276
op
8377
}

0 commit comments

Comments
 (0)