Skip to content

Commit dd4f2d8

Browse files
committed
Add a SchemaPruningTest trait for running tests with schema pruning
turned on
1 parent 0cd85b6 commit dd4f2d8

5 files changed

Lines changed: 69 additions & 13 deletions

File tree

Lines changed: 52 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,52 @@
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
19+
20+
import org.scalatest.BeforeAndAfterAll
21+
22+
import org.apache.spark.sql.catalyst.plans.PlanTest
23+
import org.apache.spark.sql.internal.SQLConf.NESTED_SCHEMA_PRUNING_ENABLED
24+
25+
/**
26+
* A PlanTest that ensures that all tests in this suite are run with nested schema pruning enabled.
27+
* Remove this trait once the default value of SQLConf.NESTED_SCHEMA_PRUNING_ENABLED is set to true.
28+
*/
29+
private[sql] trait SchemaPruningTest extends PlanTest with BeforeAndAfterAll {
30+
private var originalConfSchemaPruningEnabled = false
31+
32+
override protected def beforeAll(): Unit = {
33+
// Call `withSQLConf` eagerly because some subtypes of `PlanTest` (I'm looking at you,
34+
// `SQLTestUtils`) override `withSQLConf` to reset the existing `SQLConf` with a new one without
35+
// copying existing settings first. This here is an awful, ugly way to get around that behavior
36+
// by initializing the "real" `SQLConf` with an noop call to `withSQLConf`. I don't want to risk
37+
// "fixing" the downstream behavior, breaking everything else that's expecting these semantics.
38+
// Oh well...
39+
withSQLConf()(())
40+
originalConfSchemaPruningEnabled = conf.nestedSchemaPruningEnabled
41+
conf.setConf(NESTED_SCHEMA_PRUNING_ENABLED, true)
42+
super.beforeAll()
43+
}
44+
45+
override protected def afterAll(): Unit = {
46+
try {
47+
super.afterAll()
48+
} finally {
49+
conf.setConf(NESTED_SCHEMA_PRUNING_ENABLED, originalConfSchemaPruningEnabled)
50+
}
51+
}
52+
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateFieldExtractionPushdownSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,15 +17,15 @@
1717

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

20+
import org.apache.spark.sql.catalyst.SchemaPruningTest
2021
import org.apache.spark.sql.catalyst.dsl.expressions._
2122
import org.apache.spark.sql.catalyst.dsl.plans._
2223
import org.apache.spark.sql.catalyst.expressions.aggregate.Count
23-
import org.apache.spark.sql.catalyst.plans.PlanTest
2424
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
2525
import org.apache.spark.sql.catalyst.rules.RuleExecutor
2626
import org.apache.spark.sql.types._
2727

28-
class AggregateFieldExtractionPushdownSuite extends PlanTest {
28+
class AggregateFieldExtractionPushdownSuite extends SchemaPruningTest {
2929
private val testRelation =
3030
LocalRelation(
3131
StructField("a", StructType(

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinFieldExtractionPushdownSuite.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717

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

20+
import org.apache.spark.sql.catalyst.SchemaPruningTest
2021
import org.apache.spark.sql.catalyst.dsl.expressions._
2122
import org.apache.spark.sql.catalyst.dsl.plans._
2223
import org.apache.spark.sql.catalyst.expressions._
@@ -25,7 +26,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
2526
import org.apache.spark.sql.catalyst.rules.RuleExecutor
2627
import org.apache.spark.sql.types._
2728

28-
class JoinFieldExtractionPushdownSuite extends PlanTest {
29+
class JoinFieldExtractionPushdownSuite extends SchemaPruningTest {
2930
private val leftRelation =
3031
LocalRelation(
3132
StructField("la", StructType(

sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala

Lines changed: 11 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -2267,15 +2267,17 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
22672267
}
22682268

22692269
test("SPARK-4502: Nested column pruning shouldn't fail filter") {
2270-
withTempPath { dir =>
2271-
val path = dir.getCanonicalPath
2272-
val data =
2273-
"""{"a":{"b":1,"c":2}}
2274-
|{}""".stripMargin
2275-
Seq(data).toDF().repartition(1).write.text(path)
2276-
checkAnswer(
2277-
spark.read.json(path).filter($"a.b" > 1).select($"a.b"),
2278-
Seq.empty)
2270+
withSQLConf(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key -> "true") {
2271+
withTempPath { dir =>
2272+
val path = dir.getCanonicalPath
2273+
val data =
2274+
"""{"a":{"b":1,"c":2}}
2275+
|{}""".stripMargin
2276+
Seq(data).toDF().repartition(1).write.text(path)
2277+
checkAnswer(
2278+
spark.read.json(path).filter($"a.b" > 1).select($"a.b"),
2279+
Seq.empty)
2280+
}
22792281
}
22802282
}
22812283
}

sql/core/src/test/scala/org/apache/spark/sql/execution/FileSchemaPruningTest.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,10 +21,11 @@ import org.scalactic.Equality
2121
import org.scalatest.Assertions
2222

2323
import org.apache.spark.sql.DataFrame
24+
import org.apache.spark.sql.catalyst.SchemaPruningTest
2425
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
2526
import org.apache.spark.sql.types.StructType
2627

27-
private[sql] trait FileSchemaPruningTest {
28+
private[sql] trait FileSchemaPruningTest extends SchemaPruningTest {
2829
_: Assertions =>
2930

3031
private val schemaEquality = new Equality[StructType] {

0 commit comments

Comments
 (0)