Skip to content

Commit 6beb467

Browse files
committed
Remove a bunch of overloaded methods to avoid default args. issue
1 parent 2bbac9c commit 6beb467

File tree

4 files changed

+25
-79
lines changed

4 files changed

+25
-79
lines changed

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

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

1818
package org.apache.spark.sql.execution
1919

20+
import org.apache.spark.sql.Row
2021
import org.apache.spark.sql.catalyst.dsl.expressions._
2122

2223
class SortSuite extends SparkPlanTest {
@@ -34,13 +35,13 @@ class SortSuite extends SparkPlanTest {
3435
checkAnswer(
3536
input.toDF("a", "b", "c"),
3637
ExternalSort('a.asc :: 'b.asc :: Nil, global = true, _: SparkPlan),
37-
input.sortBy(t => (t._1, t._2)),
38+
input.sortBy(t => (t._1, t._2)).map(Row.fromTuple),
3839
sortAnswers = false)
3940

4041
checkAnswer(
4142
input.toDF("a", "b", "c"),
4243
ExternalSort('b.asc :: 'a.asc :: Nil, global = true, _: SparkPlan),
43-
input.sortBy(t => (t._2, t._1)),
44+
input.sortBy(t => (t._2, t._1)).map(Row.fromTuple),
4445
sortAnswers = false)
4546
}
4647
}

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

Lines changed: 7 additions & 63 deletions
Original file line numberDiff line numberDiff line change
@@ -54,7 +54,7 @@ class SparkPlanTest extends SparkFunSuite {
5454
planFunction: SparkPlan => SparkPlan,
5555
expectedAnswer: Seq[Row],
5656
sortAnswers: Boolean = true): Unit = {
57-
checkAnswer(
57+
doCheckAnswer(
5858
input :: Nil,
5959
(plans: Seq[SparkPlan]) => planFunction(plans.head),
6060
expectedAnswer,
@@ -71,13 +71,13 @@ class SparkPlanTest extends SparkFunSuite {
7171
* @param sortAnswers if true, the answers will be sorted by their toString representations prior
7272
* to being compared.
7373
*/
74-
protected def checkAnswer(
74+
protected def checkAnswer2(
7575
left: DataFrame,
7676
right: DataFrame,
7777
planFunction: (SparkPlan, SparkPlan) => SparkPlan,
7878
expectedAnswer: Seq[Row],
7979
sortAnswers: Boolean = true): Unit = {
80-
checkAnswer(
80+
doCheckAnswer(
8181
left :: right :: Nil,
8282
(plans: Seq[SparkPlan]) => planFunction(plans(0), plans(1)),
8383
expectedAnswer,
@@ -87,13 +87,13 @@ class SparkPlanTest extends SparkFunSuite {
8787
/**
8888
* Runs the plan and makes sure the answer matches the expected result.
8989
* @param input the input data to be used.
90-
* @param planFunction a function which accepts the input SparkPlan and uses it to instantiate
91-
* the physical operator that's being tested.
90+
* @param planFunction a function which accepts a sequence of input SparkPlans and uses them to
91+
* instantiate the physical operator that's being tested.
9292
* @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s.
9393
* @param sortAnswers if true, the answers will be sorted by their toString representations prior
9494
* to being compared.
9595
*/
96-
protected def checkAnswer(
96+
protected def doCheckAnswer(
9797
input: Seq[DataFrame],
9898
planFunction: Seq[SparkPlan] => SparkPlan,
9999
expectedAnswer: Seq[Row],
@@ -104,62 +104,6 @@ class SparkPlanTest extends SparkFunSuite {
104104
}
105105
}
106106

107-
/**
108-
* Runs the plan and makes sure the answer matches the expected result.
109-
* @param input the input data to be used.
110-
* @param planFunction a function which accepts the input SparkPlan and uses it to instantiate
111-
* the physical operator that's being tested.
112-
* @param expectedAnswer the expected result in a [[Seq]] of [[Product]]s.
113-
* @param sortAnswers if true, the answers will be sorted by their toString representations prior
114-
* to being compared.
115-
*/
116-
protected def checkAnswer[A <: Product : TypeTag](
117-
input: DataFrame,
118-
planFunction: SparkPlan => SparkPlan,
119-
expectedAnswer: Seq[A],
120-
sortAnswers: Boolean = true): Unit = {
121-
val expectedRows = expectedAnswer.map(Row.fromTuple)
122-
checkAnswer(input, planFunction, expectedRows, sortAnswers)
123-
}
124-
125-
/**
126-
* Runs the plan and makes sure the answer matches the expected result.
127-
* @param left the left input data to be used.
128-
* @param right the right input data to be used.
129-
* @param planFunction a function which accepts the input SparkPlan and uses it to instantiate
130-
* the physical operator that's being tested.
131-
* @param expectedAnswer the expected result in a [[Seq]] of [[Product]]s.
132-
* @param sortAnswers if true, the answers will be sorted by their toString representations prior
133-
* to being compared.
134-
*/
135-
protected def checkAnswer[A <: Product : TypeTag](
136-
left: DataFrame,
137-
right: DataFrame,
138-
planFunction: (SparkPlan, SparkPlan) => SparkPlan,
139-
expectedAnswer: Seq[A],
140-
sortAnswers: Boolean = true): Unit = {
141-
val expectedRows = expectedAnswer.map(Row.fromTuple)
142-
checkAnswer(left, right, planFunction, expectedRows, sortAnswers)
143-
}
144-
145-
/**
146-
* Runs the plan and makes sure the answer matches the expected result.
147-
* @param input the input data to be used.
148-
* @param planFunction a function which accepts the input SparkPlan and uses it to instantiate
149-
* the physical operator that's being tested.
150-
* @param expectedAnswer the expected result in a [[Seq]] of [[Product]]s.
151-
* @param sortAnswers if true, the answers will be sorted by their toString representations prior
152-
* to being compared.
153-
*/
154-
protected def checkAnswer[A <: Product : TypeTag](
155-
input: Seq[DataFrame],
156-
planFunction: Seq[SparkPlan] => SparkPlan,
157-
expectedAnswer: Seq[A],
158-
sortAnswers: Boolean = true): Unit = {
159-
val expectedRows = expectedAnswer.map(Row.fromTuple)
160-
checkAnswer(input, planFunction, expectedRows, sortAnswers)
161-
}
162-
163107
/**
164108
* Runs the plan and makes sure the answer matches the result produced by a reference plan.
165109
* @param input the input data to be used.
@@ -172,7 +116,7 @@ class SparkPlanTest extends SparkFunSuite {
172116
* @param sortAnswers if true, the answers will be sorted by their toString representations prior
173117
* to being compared.
174118
*/
175-
protected def checkAnswer(
119+
protected def checkThatPlansAgree(
176120
input: DataFrame,
177121
planFunction: SparkPlan => SparkPlan,
178122
expectedPlanFunction: SparkPlan => SparkPlan,

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

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,7 @@ class UnsafeExternalSortSuite extends SparkPlanTest with BeforeAndAfterAll {
4040
// TODO: this test is going to fail until we implement a proper iterator interface
4141
// with a close() method.
4242
TestSQLContext.sparkContext.conf.set("spark.unsafe.exceptionOnMemoryLeak", "true")
43-
checkAnswer(
43+
checkThatPlansAgree(
4444
(1 to 100).map(v => Tuple1(v)).toDF("a"),
4545
(child: SparkPlan) => Limit(10, UnsafeExternalSort('a.asc :: Nil, true, child)),
4646
(child: SparkPlan) => Limit(10, Sort('a.asc :: Nil, global = true, child)),
@@ -51,7 +51,7 @@ class UnsafeExternalSortSuite extends SparkPlanTest with BeforeAndAfterAll {
5151
test("sort followed by limit") {
5252
TestSQLContext.sparkContext.conf.set("spark.unsafe.exceptionOnMemoryLeak", "false")
5353
try {
54-
checkAnswer(
54+
checkThatPlansAgree(
5555
(1 to 100).map(v => Tuple1(v)).toDF("a"),
5656
(child: SparkPlan) => Limit(10, UnsafeExternalSort('a.asc :: Nil, true, child)),
5757
(child: SparkPlan) => Limit(10, Sort('a.asc :: Nil, global = true, child)),
@@ -66,7 +66,7 @@ class UnsafeExternalSortSuite extends SparkPlanTest with BeforeAndAfterAll {
6666
test("sorting does not crash for large inputs") {
6767
val sortOrder = 'a.asc :: Nil
6868
val stringLength = 1024 * 1024 * 2
69-
checkAnswer(
69+
checkThatPlansAgree(
7070
Seq(Tuple1("a" * stringLength), Tuple1("b" * stringLength)).toDF("a").repartition(1),
7171
UnsafeExternalSort(sortOrder, global = true, _: SparkPlan, testSpillFrequency = 1),
7272
Sort(sortOrder, global = true, _: SparkPlan),
@@ -93,7 +93,7 @@ class UnsafeExternalSortSuite extends SparkPlanTest with BeforeAndAfterAll {
9393
StructType(StructField("a", dataType, nullable = true) :: Nil)
9494
)
9595
assert(UnsafeExternalSort.supportsSchema(inputDf.schema))
96-
checkAnswer(
96+
checkThatPlansAgree(
9797
inputDf,
9898
UnsafeExternalSort(sortOrder, global = true, _: SparkPlan, testSpillFrequency = 23),
9999
Sort(sortOrder, global = true, _: SparkPlan),

sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala

Lines changed: 11 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717

1818
package org.apache.spark.sql.execution.joins
1919

20+
import org.apache.spark.sql.Row
2021
import org.apache.spark.sql.catalyst.dsl.expressions._
2122
import org.apache.spark.sql.catalyst.expressions.{Expression, LessThan}
2223
import org.apache.spark.sql.catalyst.plans.{FullOuter, LeftOuter, RightOuter}
@@ -41,48 +42,48 @@ class OuterJoinSuite extends SparkPlanTest {
4142
val condition = Some(LessThan('b, 'd))
4243

4344
test("shuffled hash outer join") {
44-
checkAnswer(left, right, (left: SparkPlan, right: SparkPlan) =>
45+
checkAnswer2(left, right, (left: SparkPlan, right: SparkPlan) =>
4546
ShuffledHashOuterJoin(leftKeys, rightKeys, LeftOuter, condition, left, right),
4647
Seq(
4748
(1, 2.0, null, null),
4849
(2, 1.0, 2, 3.0),
4950
(3, 3.0, null, null)
50-
))
51+
).map(Row.fromTuple))
5152

52-
checkAnswer(left, right, (left: SparkPlan, right: SparkPlan) =>
53+
checkAnswer2(left, right, (left: SparkPlan, right: SparkPlan) =>
5354
ShuffledHashOuterJoin(leftKeys, rightKeys, RightOuter, condition, left, right),
5455
Seq(
5556
(2, 1.0, 2, 3.0),
5657
(null, null, 3, 2.0),
5758
(null, null, 4, 1.0)
58-
))
59+
).map(Row.fromTuple))
5960

60-
checkAnswer(left, right, (left: SparkPlan, right: SparkPlan) =>
61+
checkAnswer2(left, right, (left: SparkPlan, right: SparkPlan) =>
6162
ShuffledHashOuterJoin(leftKeys, rightKeys, FullOuter, condition, left, right),
6263
Seq(
6364
(1, 2.0, null, null),
6465
(2, 1.0, 2, 3.0),
6566
(3, 3.0, null, null),
6667
(null, null, 3, 2.0),
6768
(null, null, 4, 1.0)
68-
))
69+
).map(Row.fromTuple))
6970
}
7071

7172
test("broadcast hash outer join") {
72-
checkAnswer(left, right, (left: SparkPlan, right: SparkPlan) =>
73+
checkAnswer2(left, right, (left: SparkPlan, right: SparkPlan) =>
7374
BroadcastHashOuterJoin(leftKeys, rightKeys, LeftOuter, condition, left, right),
7475
Seq(
7576
(1, 2.0, null, null),
7677
(2, 1.0, 2, 3.0),
7778
(3, 3.0, null, null)
78-
))
79+
).map(Row.fromTuple))
7980

80-
checkAnswer(left, right, (left: SparkPlan, right: SparkPlan) =>
81+
checkAnswer2(left, right, (left: SparkPlan, right: SparkPlan) =>
8182
BroadcastHashOuterJoin(leftKeys, rightKeys, RightOuter, condition, left, right),
8283
Seq(
8384
(2, 1.0, 2, 3.0),
8485
(null, null, 3, 2.0),
8586
(null, null, 4, 1.0)
86-
))
87+
).map(Row.fromTuple))
8788
}
8889
}

0 commit comments

Comments
 (0)