Skip to content

Commit 99c9c16

Browse files
committed
Fix tests that use SQLConfEntry as a string
1 parent 88a03cc commit 99c9c16

File tree

6 files changed

+25
-20
lines changed

6 files changed

+25
-20
lines changed

sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -39,8 +39,12 @@ private[spark] object SQLConf {
3939

4040
def defaultValueString: String = defaultValue.map(stringConverter).getOrElse("<undefined>")
4141

42-
override def toString: String =
43-
s"SQLConfEntry(key = $key, defaultValue=$defaultValueString, doc=$doc, isPublic = $isPublic)"
42+
override def toString: String = {
43+
// Fail tests that use `SQLConfEntry` as a string.
44+
throw new IllegalStateException("Force to fail tests")
45+
// s"SQLConfEntry(key = $key, defaultValue=$defaultValueString, doc=$doc,
46+
// isPublic = $isPublic)"
47+
}
4448
}
4549

4650
private[sql] object SQLConfEntry {

sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -85,14 +85,14 @@ case class SetCommand(
8585
case Some((SQLConf.Deprecated.MAPRED_REDUCE_TASKS, Some(value))) =>
8686
logWarning(
8787
s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " +
88-
s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.")
88+
s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS.key} instead.")
8989
if (value.toInt < 1) {
9090
val msg = s"Setting negative ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} for automatically " +
9191
"determining the number of reducers is not supported."
9292
throw new IllegalArgumentException(msg)
9393
} else {
9494
sqlContext.setConf(SQLConf.SHUFFLE_PARTITIONS.key, value)
95-
Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$value"))
95+
Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS.key}=$value"))
9696
}
9797

9898
// Configures a single property.
@@ -110,8 +110,8 @@ case class SetCommand(
110110
case Some((SQLConf.Deprecated.MAPRED_REDUCE_TASKS, None)) =>
111111
logWarning(
112112
s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " +
113-
s"showing ${SQLConf.SHUFFLE_PARTITIONS} instead.")
114-
Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=${sqlContext.conf.numShufflePartitions}"))
113+
s"showing ${SQLConf.SHUFFLE_PARTITIONS.key} instead.")
114+
Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS.key}=${sqlContext.conf.numShufflePartitions}"))
115115

116116
// Queries a single property.
117117
case Some((key, None)) =>

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -416,15 +416,15 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
416416
ctx.sql("CACHE TABLE testData")
417417
val tmp = ctx.conf.autoBroadcastJoinThreshold
418418

419-
ctx.sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=1000000000")
419+
ctx.sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=1000000000")
420420
Seq(
421421
("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a",
422422
classOf[BroadcastLeftSemiJoinHash])
423423
).foreach {
424424
case (query, joinClass) => assertJoin(query, joinClass)
425425
}
426426

427-
ctx.sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1")
427+
ctx.sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=-1")
428428

429429
Seq(
430430
("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[LeftSemiJoinHash])

sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -238,7 +238,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest {
238238
// first session, we get the default value of the session status
239239
{ statement =>
240240

241-
val rs1 = statement.executeQuery(s"SET ${SQLConf.SHUFFLE_PARTITIONS}")
241+
val rs1 = statement.executeQuery(s"SET ${SQLConf.SHUFFLE_PARTITIONS.key}")
242242
rs1.next()
243243
defaultV1 = rs1.getString(1)
244244
assert(defaultV1 != "200")
@@ -256,12 +256,12 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest {
256256
{ statement =>
257257

258258
val queries = Seq(
259-
s"SET ${SQLConf.SHUFFLE_PARTITIONS}=291",
259+
s"SET ${SQLConf.SHUFFLE_PARTITIONS.key}=291",
260260
"SET hive.cli.print.header=true"
261261
)
262262

263263
queries.map(statement.execute)
264-
val rs1 = statement.executeQuery(s"SET ${SQLConf.SHUFFLE_PARTITIONS}")
264+
val rs1 = statement.executeQuery(s"SET ${SQLConf.SHUFFLE_PARTITIONS.key}")
265265
rs1.next()
266266
assert("spark.sql.shuffle.partitions=291" === rs1.getString(1))
267267
rs1.close()
@@ -276,7 +276,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest {
276276
// default value
277277
{ statement =>
278278

279-
val rs1 = statement.executeQuery(s"SET ${SQLConf.SHUFFLE_PARTITIONS}")
279+
val rs1 = statement.executeQuery(s"SET ${SQLConf.SHUFFLE_PARTITIONS.key}")
280280
rs1.next()
281281
assert(defaultV1 === rs1.getString(1))
282282
rs1.close()

sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -47,17 +47,18 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
4747
// Add Locale setting
4848
Locale.setDefault(Locale.US)
4949
// Set a relatively small column batch size for testing purposes
50-
TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, "5")
50+
TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE.key, "5")
5151
// Enable in-memory partition pruning for testing purposes
52-
TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, "true")
52+
TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING.key, "true")
5353
}
5454

5555
override def afterAll() {
5656
TestHive.cacheTables = false
5757
TimeZone.setDefault(originalTimeZone)
5858
Locale.setDefault(originalLocale)
59-
TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize.toString)
60-
TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning.toString)
59+
TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE.key, originalColumnBatchSize.toString)
60+
TestHive.setConf(
61+
SQLConf.IN_MEMORY_PARTITION_PRUNING.key, originalInMemoryPartitionPruning.toString)
6162
}
6263

6364
/** A list of tests deemed out of scope currently and thus completely disregarded. */

sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -167,7 +167,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
167167
ctx.conf.settings.synchronized {
168168
val tmp = ctx.conf.autoBroadcastJoinThreshold
169169

170-
sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1""")
170+
sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=-1""")
171171
df = sql(query)
172172
bhj = df.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin => j }
173173
assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off")
@@ -176,7 +176,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
176176
assert(shj.size === 1,
177177
"ShuffledHashJoin should be planned when BroadcastHashJoin is turned off")
178178

179-
sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=$tmp""")
179+
sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=$tmp""")
180180
}
181181

182182
after()
@@ -225,7 +225,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
225225
ctx.conf.settings.synchronized {
226226
val tmp = ctx.conf.autoBroadcastJoinThreshold
227227

228-
sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1")
228+
sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=-1")
229229
df = sql(leftSemiJoinQuery)
230230
bhj = df.queryExecution.sparkPlan.collect {
231231
case j: BroadcastLeftSemiJoinHash => j
@@ -238,7 +238,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
238238
assert(shj.size === 1,
239239
"LeftSemiJoinHash should be planned when BroadcastHashJoin is turned off")
240240

241-
sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=$tmp")
241+
sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=$tmp")
242242
}
243243

244244
}

0 commit comments

Comments
 (0)