Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-21726][SQL] Check for structural integrity of the plan in Optimzer in test mode. #18956

Closed
wants to merge 6 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils

/**
* Abstract class all optimizers should inherit of, contains the standard batches (extending
Expand All @@ -37,6 +38,12 @@ import org.apache.spark.sql.types._
abstract class Optimizer(sessionCatalog: SessionCatalog)
extends RuleExecutor[LogicalPlan] {

// Check for structural integrity of the plan in test mode. Currently we only check if a plan is
// still resolved after the execution of each rule.
override protected def isPlanIntegral(plan: LogicalPlan): Boolean = {
Utils.isTesting && plan.resolved
}

protected def fixedPoint = FixedPoint(SQLConf.get.optimizerMaxIterations)

def batches: Seq[Batch] = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,14 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging {
/** Defines a sequence of rule batches, to be overridden by the implementation. */
protected def batches: Seq[Batch]

/**
* Defines a check function that checks for structural integrity of the plan after the execution
* of each rule. For example, we can check whether a plan is still resolved after each rule in
* `Optimizer`, so we can catch rules that return invalid plans. The check function returns
* `false` if the given plan doesn't pass the structural integrity check.
*/
protected def isPlanIntegral(plan: TreeType): Boolean = true

/**
* Executes the batches of rules defined by the subclass. The batches are executed serially
* using the defined execution strategy. Within each batch, rules are also executed serially.
Expand Down Expand Up @@ -93,6 +101,13 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging {
""".stripMargin)
}

// Run the structural integrity checker against the plan after each rule.
if (!isPlanIntegral(result)) {
val message = s"After applying rule ${rule.ruleName} in batch ${batch.name}, " +
"the structural integrity of the plan is broken."
throw new TreeNodeException(result, message, null)
}

result
}
iteration += 1
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import org.scalatest.prop.GeneratorDrivenPropertyChecks
import org.apache.spark.{SparkConf, SparkFunSuite}
import org.apache.spark.serializer.JavaSerializer
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
import org.apache.spark.sql.catalyst.analysis.ResolveTimeZone
import org.apache.spark.sql.catalyst.analysis.{ResolveTimeZone, SimpleAnalyzer}
import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.optimizer.SimpleTestOptimizer
import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Project}
Expand Down Expand Up @@ -188,7 +188,9 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks {
expected: Any,
inputRow: InternalRow = EmptyRow): Unit = {
val plan = Project(Alias(expression, s"Optimized($expression)")() :: Nil, OneRowRelation())
val optimizedPlan = SimpleTestOptimizer.execute(plan)
// We should analyze the plan first, otherwise we possibly optimize an unresolved plan.
val analyzedPlan = SimpleAnalyzer.execute(plan)
val optimizedPlan = SimpleTestOptimizer.execute(analyzedPlan)
checkEvaluationWithoutCodegen(optimizedPlan.expressions.head, expected, inputRow)
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.catalyst.optimizer

import org.apache.spark.sql.catalyst.analysis.{EmptyFunctionRegistry, UnresolvedAttribute}
import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog}
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.expressions.{Alias, Literal}
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation, Project}
import org.apache.spark.sql.catalyst.rules._
import org.apache.spark.sql.internal.SQLConf


class OptimizerStructuralIntegrityCheckerSuite extends PlanTest {

object OptimizeRuleBreakSI extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case Project(projectList, child) =>
val newAttr = UnresolvedAttribute("unresolvedAttr")
Project(projectList ++ Seq(newAttr), child)
}
}

object Optimize extends Optimizer(
new SessionCatalog(
new InMemoryCatalog,
EmptyFunctionRegistry,
new SQLConf())) {
val newBatch = Batch("OptimizeRuleBreakSI", Once, OptimizeRuleBreakSI)
override def batches: Seq[Batch] = Seq(newBatch) ++ super.batches
}

test("check for invalid plan after execution of rule") {
val analyzed = Project(Alias(Literal(10), "attr")() :: Nil, OneRowRelation()).analyze
assert(analyzed.resolved)
val message = intercept[TreeNodeException[LogicalPlan]] {
Optimize.execute(analyzed)
}.getMessage
val ruleName = OptimizeRuleBreakSI.ruleName
assert(message.contains(s"After applying rule $ruleName in batch OptimizeRuleBreakSI"))
assert(message.contains("the structural integrity of the plan is broken"))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -56,4 +56,21 @@ class RuleExecutorSuite extends SparkFunSuite {
}.getMessage
assert(message.contains("Max iterations (10) reached for batch fixedPoint"))
}

test("structural integrity checker") {
object WithSIChecker extends RuleExecutor[Expression] {
override protected def isPlanIntegral(expr: Expression): Boolean = expr match {
case IntegerLiteral(_) => true
case _ => false
}
val batches = Batch("once", Once, DecrementLiterals) :: Nil
}

assert(WithSIChecker.execute(Literal(10)) === Literal(9))

val message = intercept[TreeNodeException[LogicalPlan]] {
WithSIChecker.execute(Literal(10.1))
}.getMessage
assert(message.contains("the structural integrity of the plan is broken"))
}
}