|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.sql.catalyst.optimizer |
| 19 | + |
| 20 | +import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries |
| 21 | +import org.apache.spark.sql.catalyst.dsl.plans._ |
| 22 | +import org.apache.spark.sql.catalyst.dsl.expressions._ |
| 23 | +import org.apache.spark.sql.catalyst.expressions.Rand |
| 24 | +import org.apache.spark.sql.catalyst.plans.PlanTest |
| 25 | +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} |
| 26 | +import org.apache.spark.sql.catalyst.rules.RuleExecutor |
| 27 | + |
| 28 | + |
| 29 | +class ProjectCollapsingSuite extends PlanTest { |
| 30 | + object Optimize extends RuleExecutor[LogicalPlan] { |
| 31 | + val batches = |
| 32 | + Batch("Subqueries", FixedPoint(10), EliminateSubQueries) :: |
| 33 | + Batch("ProjectCollapsing", Once, ProjectCollapsing) :: Nil |
| 34 | + } |
| 35 | + |
| 36 | + val testRelation = LocalRelation('a.int, 'b.int) |
| 37 | + |
| 38 | + test("collapse two deterministic, independent projects into one") { |
| 39 | + val query = testRelation |
| 40 | + .select(('a + 1).as('a_plus_1), 'b) |
| 41 | + .select('a_plus_1, ('b + 1).as('b_plus_1)) |
| 42 | + |
| 43 | + val optimized = Optimize.execute(query.analyze) |
| 44 | + val correctAnswer = testRelation.select(('a + 1).as('a_plus_1), ('b + 1).as('b_plus_1)).analyze |
| 45 | + |
| 46 | + comparePlans(optimized, correctAnswer) |
| 47 | + } |
| 48 | + |
| 49 | + test("collapse two deterministic, dependent projects into one") { |
| 50 | + val query = testRelation |
| 51 | + .select(('a + 1).as('a_plus_1), 'b) |
| 52 | + .select(('a_plus_1 + 1).as('a_plus_2), 'b) |
| 53 | + |
| 54 | + val optimized = Optimize.execute(query.analyze) |
| 55 | + |
| 56 | + val correctAnswer = testRelation.select( |
| 57 | + (('a + 1).as('a_plus_1) + 1).as('a_plus_2), |
| 58 | + 'b).analyze |
| 59 | + |
| 60 | + comparePlans(optimized, correctAnswer) |
| 61 | + } |
| 62 | + |
| 63 | + test("do not collapse nondeterministic projects") { |
| 64 | + val query = testRelation |
| 65 | + .select(Rand(10).as('rand)) |
| 66 | + .select(('rand + 1).as('rand1), ('rand + 2).as('rand2)) |
| 67 | + |
| 68 | + val optimized = Optimize.execute(query.analyze) |
| 69 | + val correctAnswer = query.analyze |
| 70 | + |
| 71 | + comparePlans(optimized, correctAnswer) |
| 72 | + } |
| 73 | +} |
0 commit comments