Skip to content

Commit

Permalink
[TEST][MINOR] remove redundant EliminateSubqueryAliases in test code
Browse files Browse the repository at this point in the history
## What changes were proposed in this pull request?

The `analyze` method in `implicit class DslLogicalPlan` already includes `EliminateSubqueryAliases`. So there's no need to call `EliminateSubqueryAliases` again after calling `analyze` in some test code.

## How was this patch tested?

Existing tests.

Author: Zhenhua Wang <[email protected]>

Closes #20122 from wzhfy/redundant_code.
  • Loading branch information
wzhfy authored and gatorsmile committed Dec 30, 2017
1 parent 14c4a62 commit 234d943
Show file tree
Hide file tree
Showing 3 changed files with 10 additions and 11 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -256,7 +256,7 @@ class ColumnPruningSuite extends PlanTest {
x.select('a)
.sortBy(SortOrder('a, Ascending)).analyze

comparePlans(optimized, analysis.EliminateSubqueryAliases(correctAnswer))
comparePlans(optimized, correctAnswer)

// push down invalid
val originalQuery1 = {
Expand All @@ -271,7 +271,7 @@ class ColumnPruningSuite extends PlanTest {
.sortBy(SortOrder('a, Ascending))
.select('b).analyze

comparePlans(optimized1, analysis.EliminateSubqueryAliases(correctAnswer1))
comparePlans(optimized1, correctAnswer1)
}

test("Column pruning on Window with useless aggregate functions") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -504,7 +504,7 @@ class FilterPushdownSuite extends PlanTest {
}
val optimized = Optimize.execute(originalQuery.analyze)

comparePlans(analysis.EliminateSubqueryAliases(originalQuery.analyze), optimized)
comparePlans(originalQuery.analyze, optimized)
}

test("joins: conjunctive predicates") {
Expand All @@ -523,7 +523,7 @@ class FilterPushdownSuite extends PlanTest {
left.join(right, condition = Some("x.b".attr === "y.b".attr))
.analyze

comparePlans(optimized, analysis.EliminateSubqueryAliases(correctAnswer))
comparePlans(optimized, correctAnswer)
}

test("joins: conjunctive predicates #2") {
Expand All @@ -542,7 +542,7 @@ class FilterPushdownSuite extends PlanTest {
left.join(right, condition = Some("x.b".attr === "y.b".attr))
.analyze

comparePlans(optimized, analysis.EliminateSubqueryAliases(correctAnswer))
comparePlans(optimized, correctAnswer)
}

test("joins: conjunctive predicates #3") {
Expand All @@ -566,7 +566,7 @@ class FilterPushdownSuite extends PlanTest {
condition = Some("z.a".attr === "x.b".attr))
.analyze

comparePlans(optimized, analysis.EliminateSubqueryAliases(correctAnswer))
comparePlans(optimized, correctAnswer)
}

test("joins: push down where clause into left anti join") {
Expand All @@ -581,7 +581,7 @@ class FilterPushdownSuite extends PlanTest {
x.where("x.a".attr > 10)
.join(y, LeftAnti, Some("x.b".attr === "y.b".attr))
.analyze
comparePlans(optimized, analysis.EliminateSubqueryAliases(correctAnswer))
comparePlans(optimized, correctAnswer)
}

test("joins: only push down join conditions to the right of a left anti join") {
Expand All @@ -598,7 +598,7 @@ class FilterPushdownSuite extends PlanTest {
LeftAnti,
Some("x.b".attr === "y.b".attr && "x.a".attr > 10))
.analyze
comparePlans(optimized, analysis.EliminateSubqueryAliases(correctAnswer))
comparePlans(optimized, correctAnswer)
}

test("joins: only push down join conditions to the right of an existence join") {
Expand All @@ -616,7 +616,7 @@ class FilterPushdownSuite extends PlanTest {
ExistenceJoin(fillerVal),
Some("x.a".attr > 1))
.analyze
comparePlans(optimized, analysis.EliminateSubqueryAliases(correctAnswer))
comparePlans(optimized, correctAnswer)
}

val testRelationWithArrayType = LocalRelation('a.int, 'b.int, 'c_arr.array(IntegerType))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

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

import org.apache.spark.sql.catalyst.analysis
import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
Expand Down Expand Up @@ -118,7 +117,7 @@ class JoinOptimizationSuite extends PlanTest {

queryAnswers foreach { queryAnswerPair =>
val optimized = Optimize.execute(queryAnswerPair._1.analyze)
comparePlans(optimized, analysis.EliminateSubqueryAliases(queryAnswerPair._2.analyze))
comparePlans(optimized, queryAnswerPair._2.analyze)
}
}

Expand Down

0 comments on commit 234d943

Please sign in to comment.