Skip to content

Commit 04d7265

Browse files
itholicHyukjinKwon
authored andcommitted
[SPARK-41979][SQL] Add missing dots for error messages in error classes
### What changes were proposed in this pull request? This PR proposes to add missing dots for error messages in error classes. This PR also fixes related tests, and includes a minor error message fix. ### Why are the changes needed? To keep consistency across all error messages. Error messages should end with a dot. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? `./build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*` Closes apache#39505 from itholic/missing_dots. Authored-by: itholic <[email protected]> Signed-off-by: Hyukjin Kwon <[email protected]>
1 parent 57d06f8 commit 04d7265

File tree

12 files changed

+399
-350
lines changed

12 files changed

+399
-350
lines changed

core/src/main/resources/error/error-classes.json

+307-307
Large diffs are not rendered by default.

core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala

+2-2
Original file line numberDiff line numberDiff line change
@@ -194,7 +194,7 @@ class SparkThrowableSuite extends SparkFunSuite {
194194
Map("objectName" -> "`foo`", "proposal" -> "`bar`, `baz`")
195195
) ==
196196
"[UNRESOLVED_COLUMN.WITH_SUGGESTION] A column or function parameter with " +
197-
"name `foo` cannot be resolved. Did you mean one of the following? [`bar`, `baz`]"
197+
"name `foo` cannot be resolved. Did you mean one of the following? [`bar`, `baz`]."
198198
)
199199

200200
assert(
@@ -206,7 +206,7 @@ class SparkThrowableSuite extends SparkFunSuite {
206206
""
207207
) ==
208208
"[UNRESOLVED_COLUMN.WITH_SUGGESTION] A column or function parameter with " +
209-
"name `foo` cannot be resolved. Did you mean one of the following? [`bar`, `baz`]"
209+
"name `foo` cannot be resolved. Did you mean one of the following? [`bar`, `baz`]."
210210
)
211211
}
212212

core/src/test/scala/org/apache/spark/metrics/sink/GraphiteSinkSuite.scala

+7-6
Original file line numberDiff line numberDiff line change
@@ -113,11 +113,12 @@ class GraphiteSinkSuite extends SparkFunSuite {
113113
props.put("protocol", "http")
114114
val registry = new MetricRegistry
115115

116-
val e = intercept[SparkException] {
117-
new GraphiteSink(props, registry)
118-
}
119-
assert(e.getErrorClass === "GRAPHITE_SINK_INVALID_PROTOCOL")
120-
assert(e.getMessage ===
121-
"[GRAPHITE_SINK_INVALID_PROTOCOL] Invalid Graphite protocol: http")
116+
checkError(
117+
exception = intercept[SparkException] {
118+
new GraphiteSink(props, registry)
119+
},
120+
errorClass = "GRAPHITE_SINK_INVALID_PROTOCOL",
121+
parameters = Map("protocol" -> "http")
122+
)
122123
}
123124
}

sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala

+10-5
Original file line numberDiff line numberDiff line change
@@ -153,11 +153,16 @@ class DataTypeSuite extends SparkFunSuite {
153153
val right = StructType(
154154
StructField("b", LongType) :: Nil)
155155

156-
val message = intercept[SparkException] {
157-
left.merge(right)
158-
}.getMessage
159-
assert(message.equals("Failed to merge fields 'b' and 'b'. " +
160-
"Failed to merge incompatible data types float and bigint"))
156+
checkError(
157+
exception = intercept[SparkException] {
158+
left.merge(right)
159+
},
160+
errorClass = "_LEGACY_ERROR_TEMP_2123",
161+
parameters = Map(
162+
"leftName" -> "b",
163+
"rightName" -> "b",
164+
"message" -> "Failed to merge incompatible data types float and bigint.")
165+
)
161166
}
162167

163168
test("existsRecursively") {

sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out

+3-3
Original file line numberDiff line numberDiff line change
@@ -163,7 +163,7 @@ org.apache.spark.SparkDateTimeException
163163
"sqlState" : "22007",
164164
"messageParameters" : {
165165
"ansiConfig" : "\"spark.sql.ansi.enabled\"",
166-
"message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31"
166+
"message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31."
167167
}
168168
}
169169

@@ -179,7 +179,7 @@ org.apache.spark.SparkDateTimeException
179179
"sqlState" : "22007",
180180
"messageParameters" : {
181181
"ansiConfig" : "\"spark.sql.ansi.enabled\"",
182-
"message" : "Conflict found: Field MonthOfYear 11 differs from MonthOfYear 12 derived from 1970-12-31"
182+
"message" : "Conflict found: Field MonthOfYear 11 differs from MonthOfYear 12 derived from 1970-12-31."
183183
}
184184
}
185185

@@ -211,7 +211,7 @@ org.apache.spark.SparkDateTimeException
211211
"sqlState" : "22007",
212212
"messageParameters" : {
213213
"ansiConfig" : "\"spark.sql.ansi.enabled\"",
214-
"message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31"
214+
"message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31."
215215
}
216216
}
217217

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

+15-9
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,7 @@ import scala.util.Random
2929

3030
import org.scalatest.matchers.should.Matchers._
3131

32-
import org.apache.spark.SparkException
32+
import org.apache.spark.{SparkException, SparkIllegalArgumentException}
3333
import org.apache.spark.api.python.PythonEvalType
3434
import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd}
3535
import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
@@ -1142,15 +1142,21 @@ class DataFrameSuite extends QueryTest
11421142
val onlyPercentiles = person2.summary("0.1%", "99.9%")
11431143
assert(onlyPercentiles.count() === 2)
11441144

1145-
val fooE = intercept[IllegalArgumentException] {
1146-
person2.summary("foo")
1147-
}
1148-
assert(fooE.getMessage === "foo is not a recognised statistic")
1145+
checkError(
1146+
exception = intercept[SparkIllegalArgumentException] {
1147+
person2.summary("foo")
1148+
},
1149+
errorClass = "_LEGACY_ERROR_TEMP_2114",
1150+
parameters = Map("stats" -> "foo")
1151+
)
11491152

1150-
val parseE = intercept[IllegalArgumentException] {
1151-
person2.summary("foo%")
1152-
}
1153-
assert(parseE.getMessage === "Unable to parse foo% as a percentile")
1153+
checkError(
1154+
exception = intercept[SparkIllegalArgumentException] {
1155+
person2.summary("foo%")
1156+
},
1157+
errorClass = "_LEGACY_ERROR_TEMP_2113",
1158+
parameters = Map("stats" -> "foo%")
1159+
)
11541160
}
11551161

11561162
test("apply on query results (SPARK-5462)") {

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

+11-2
Original file line numberDiff line numberDiff line change
@@ -584,8 +584,17 @@ class PersistedViewTestSuite extends SQLViewTestSuite with SharedSparkSession {
584584
val e = intercept[AnalysisException] {
585585
sql(s"SELECT * FROM test_view")
586586
}
587-
assert(e.getMessage.contains("re-create the view by running: CREATE OR REPLACE"))
588-
val ddl = e.getMessage.split(": ").last
587+
checkError(
588+
exception = e,
589+
errorClass = "_LEGACY_ERROR_TEMP_1176",
590+
parameters = Map(
591+
"viewName" -> "`spark_catalog`.`default`.`test_view`",
592+
"viewDDL" ->
593+
"CREATE OR REPLACE VIEW spark_catalog.default.test_view AS SELECT * FROM t",
594+
"actualCols" -> "[]", "colName" -> "col_j",
595+
"expectedNum" -> "1")
596+
)
597+
val ddl = e.getMessageParameters.get("viewDDL")
589598
sql(ddl)
590599
checkAnswer(sql("select * FROM test_view"), Row(1))
591600
}

sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala

+8-5
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,7 @@ import org.apache.hadoop.fs.viewfs.ViewFileSystem
2929
import org.mockito.ArgumentMatchers.any
3030
import org.mockito.Mockito.{mock, when}
3131

32-
import org.apache.spark.SparkException
32+
import org.apache.spark.{SparkException, SparkRuntimeException}
3333
import org.apache.spark.metrics.source.HiveCatalogMetrics
3434
import org.apache.spark.sql.SparkSession
3535
import org.apache.spark.sql.catalyst.util._
@@ -133,10 +133,13 @@ class FileIndexSuite extends SharedSparkSession {
133133
val schema = StructType(Seq(StructField("a", IntegerType, false)))
134134
withSQLConf(SQLConf.VALIDATE_PARTITION_COLUMNS.key -> "true") {
135135
val fileIndex = new InMemoryFileIndex(spark, Seq(path), Map.empty, Some(schema))
136-
val msg = intercept[RuntimeException] {
137-
fileIndex.partitionSpec()
138-
}.getMessage
139-
assert(msg == "Failed to cast value `foo` to `IntegerType` for partition column `a`")
136+
checkError(
137+
exception = intercept[SparkRuntimeException] {
138+
fileIndex.partitionSpec()
139+
},
140+
errorClass = "_LEGACY_ERROR_TEMP_2058",
141+
parameters = Map("value" -> "foo", "dataType" -> "IntegerType", "columnName" -> "a")
142+
)
140143
}
141144

142145
withSQLConf(SQLConf.VALIDATE_PARTITION_COLUMNS.key -> "false") {

sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala

+22-2
Original file line numberDiff line numberDiff line change
@@ -1001,9 +1001,29 @@ class ParquetSchemaSuite extends ParquetSchemaTest {
10011001
val col = spark.read.parquet(file).schema.fields.filter(_.name == "a")
10021002
assert(col.length == 1)
10031003
if (col(0).dataType == StringType) {
1004-
assert(errMsg.contains("Column: [a], Expected: int, Found: BINARY"))
1004+
checkError(
1005+
exception = e.getCause.asInstanceOf[SparkException],
1006+
errorClass = "_LEGACY_ERROR_TEMP_2063",
1007+
parameters = Map(
1008+
"filePath" ->
1009+
s".*${dir.getCanonicalPath}.*",
1010+
"column" -> "\\[a\\]",
1011+
"logicalType" -> "int",
1012+
"physicalType" -> "BINARY"),
1013+
matchPVals = true
1014+
)
10051015
} else {
1006-
assert(errMsg.endsWith("Column: [a], Expected: string, Found: INT32"))
1016+
checkError(
1017+
exception = e.getCause.asInstanceOf[SparkException],
1018+
errorClass = "_LEGACY_ERROR_TEMP_2063",
1019+
parameters = Map(
1020+
"filePath" ->
1021+
s".*${dir.getCanonicalPath}.*",
1022+
"column" -> "\\[a\\]",
1023+
"logicalType" -> "string",
1024+
"physicalType" -> "INT32"),
1025+
matchPVals = true
1026+
)
10071027
}
10081028
}
10091029
}

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

+7-4
Original file line numberDiff line numberDiff line change
@@ -535,10 +535,13 @@ class HashedRelationSuite extends SharedSparkSession {
535535
buffer.append(keyIterator.next().getLong(0))
536536
}
537537
// attempt an illegal next() call
538-
val caught = intercept[SparkException] {
539-
keyIterator.next()
540-
}
541-
assert(caught.getLocalizedMessage === "End of the iterator")
538+
checkError(
539+
exception = intercept[SparkException] {
540+
keyIterator.next()
541+
},
542+
errorClass = "_LEGACY_ERROR_TEMP_2104",
543+
parameters = Map.empty
544+
)
542545
assert(buffer.sortWith(_ < _) === randomArray)
543546
buffer.clear()
544547

sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala

+6-4
Original file line numberDiff line numberDiff line change
@@ -261,7 +261,7 @@ class ForeachWriterSuite extends StreamTest with SharedSparkSession with BeforeA
261261
}
262262
}
263263

264-
testQuietly("foreach with error not caused by ForeachWriter") {
264+
test("foreach with error not caused by ForeachWriter") {
265265
withTempDir { checkpointDir =>
266266
val input = MemoryStream[Int]
267267
val query = input.toDS().repartition(1).map(_ / 0).writeStream
@@ -283,9 +283,11 @@ class ForeachWriterSuite extends StreamTest with SharedSparkSession with BeforeA
283283
assert(allEvents(0)(0) === ForeachWriterSuite.Open(partition = 0, version = 0))
284284
// `close` should be called with the error
285285
val errorEvent = allEvents(0)(1).asInstanceOf[ForeachWriterSuite.Close]
286-
assert(errorEvent.error.get.isInstanceOf[SparkException])
287-
assert(errorEvent.error.get.getMessage ===
288-
"Foreach writer has been aborted due to a task failure")
286+
checkError(
287+
exception = errorEvent.error.get.asInstanceOf[SparkException],
288+
errorClass = "_LEGACY_ERROR_TEMP_2256",
289+
parameters = Map.empty
290+
)
289291
}
290292
}
291293
}

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

+1-1
Original file line numberDiff line numberDiff line change
@@ -641,7 +641,7 @@ class CliSuite extends SparkFunSuite {
641641

642642
test("SPARK-37694: delete [jar|file|archive] shall use spark sql processor") {
643643
runCliWithin(2.minute, errorResponses = Seq("ParseException"))(
644-
"delete jar dummy.jar;" -> "Syntax error at or near 'jar': missing 'FROM'(line 1, pos 7)")
644+
"delete jar dummy.jar;" -> "Syntax error at or near 'jar': missing 'FROM'.(line 1, pos 7)")
645645
}
646646

647647
test("SPARK-37906: Spark SQL CLI should not pass final comment") {

0 commit comments

Comments
 (0)