Skip to content

Commit 5e494d3

Browse files
committed
[SPARK-38996][SQL] Use double quotes for types in error massages
### What changes were proposed in this pull request? In the PR, I propose to modify the method `QueryErrorsBase.toSQLType()` to use double quotes for types in error messages. ### Why are the changes needed? 1. To highlight types and make them more visible for users. 2. To be able to easily parse types from error text. 3. To be consistent to other outputs of identifiers, sql statement and etc. where Spark uses quotes or ticks. ### Does this PR introduce _any_ user-facing change? Yes, the PR changes user-facing errors. ### How was this patch tested? By running the modified test suites: ``` $ build/sbt "test:testOnly *QueryParsingErrorsSuite" $ build/sbt "test:testOnly *QueryCompilationErrorsSuite" $ build/sbt "test:testOnly *QueryExecutionErrorsSuite" $ build/sbt "testOnly *CastSuite" $ build/sbt "testOnly *AnsiCastSuiteWithAnsiModeOn" $ build/sbt "testOnly *EncoderResolutionSuite" $ build/sbt "test:testOnly *DatasetSuite" $ build/sbt "test:testOnly *InsertSuite" ``` Closes #36324 from MaxGekk/wrap-types-in-error-classes. Authored-by: Max Gekk <max.gekk@gmail.com> Signed-off-by: Max Gekk <max.gekk@gmail.com>
1 parent f4a81ae commit 5e494d3

27 files changed

+149
-149
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -62,6 +62,6 @@ trait QueryErrorsBase {
6262
}
6363

6464
def toSQLType(t: DataType): String = {
65-
t.sql
65+
"\"" + t.sql + "\""
6666
}
6767
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -88,7 +88,7 @@ class EncoderResolutionSuite extends PlanTest {
8888
val attrs = Seq($"arr".array(StringType))
8989
assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message ==
9090
s"""
91-
|[CANNOT_UP_CAST_DATATYPE] Cannot up cast array element from STRING to BIGINT.
91+
|[CANNOT_UP_CAST_DATATYPE] Cannot up cast array element from "STRING" to "BIGINT".
9292
|The type path of the target object is:
9393
|- array element class: "scala.Long"
9494
|- field (class: "scala.Array", name: "arr")
@@ -212,7 +212,7 @@ class EncoderResolutionSuite extends PlanTest {
212212
val attrs = Seq(attr)
213213
assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message ==
214214
s"""
215-
|[CANNOT_UP_CAST_DATATYPE] Cannot up cast a from ${attr.dataType.sql} to STRING.
215+
|[CANNOT_UP_CAST_DATATYPE] Cannot up cast a from "${attr.dataType.sql}" to "STRING".
216216
|The type path of the target object is:
217217
|- root class: "java.lang.String"
218218
|You can either add an explicit cast to the input data or choose a higher precision type
@@ -226,7 +226,7 @@ class EncoderResolutionSuite extends PlanTest {
226226
}.message
227227
assert(msg1 ==
228228
s"""
229-
|[CANNOT_UP_CAST_DATATYPE] Cannot up cast b from BIGINT to INT.
229+
|[CANNOT_UP_CAST_DATATYPE] Cannot up cast b from "BIGINT" to "INT".
230230
|The type path of the target object is:
231231
|- field (class: "scala.Int", name: "b")
232232
|- root class: "org.apache.spark.sql.catalyst.encoders.StringIntClass"
@@ -239,7 +239,7 @@ class EncoderResolutionSuite extends PlanTest {
239239
}.message
240240
assert(msg2 ==
241241
s"""
242-
|[CANNOT_UP_CAST_DATATYPE] Cannot up cast b.`b` from DECIMAL(38,18) to BIGINT.
242+
|[CANNOT_UP_CAST_DATATYPE] Cannot up cast b.`b` from "DECIMAL(38,18)" to "BIGINT".
243243
|The type path of the target object is:
244244
|- field (class: "scala.Long", name: "b")
245245
|- field (class: "org.apache.spark.sql.catalyst.encoders.StringLongClass", name: "b")

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AnsiCastSuiteBase.scala

Lines changed: 18 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -176,29 +176,29 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase {
176176
// cast to IntegerType
177177
Seq(IntegerType, ShortType, ByteType, LongType).foreach { dataType =>
178178
checkExceptionInExpression[NumberFormatException](cast("string", dataType),
179-
s"Invalid input syntax for type ${dataType.sql}: 'string'")
179+
s"""Invalid input syntax for type "${dataType.sql}": 'string'""")
180180
checkExceptionInExpression[NumberFormatException](cast("123-string", dataType),
181-
s"Invalid input syntax for type ${dataType.sql}: '123-string'")
181+
s"""Invalid input syntax for type "${dataType.sql}": '123-string'""")
182182
checkExceptionInExpression[NumberFormatException](cast("2020-07-19", dataType),
183-
s"Invalid input syntax for type ${dataType.sql}: '2020-07-19'")
183+
s"""Invalid input syntax for type "${dataType.sql}": '2020-07-19'""")
184184
checkExceptionInExpression[NumberFormatException](cast("1.23", dataType),
185-
s"Invalid input syntax for type ${dataType.sql}: '1.23'")
185+
s"""Invalid input syntax for type "${dataType.sql}": '1.23'""")
186186
}
187187

188188
Seq(DoubleType, FloatType, DecimalType.USER_DEFAULT).foreach { dataType =>
189189
checkExceptionInExpression[NumberFormatException](cast("string", dataType),
190-
s"Invalid input syntax for type ${dataType.sql}: 'string'")
190+
s"""Invalid input syntax for type "${dataType.sql}": 'string'""")
191191
checkExceptionInExpression[NumberFormatException](cast("123.000.00", dataType),
192-
s"Invalid input syntax for type ${dataType.sql}: '123.000.00'")
192+
s"""Invalid input syntax for type "${dataType.sql}": '123.000.00'""")
193193
checkExceptionInExpression[NumberFormatException](cast("abc.com", dataType),
194-
s"Invalid input syntax for type ${dataType.sql}: 'abc.com'")
194+
s"""Invalid input syntax for type "${dataType.sql}": 'abc.com'""")
195195
}
196196
}
197197

198198
protected def checkCastToNumericError(l: Literal, to: DataType,
199199
expectedDataTypeInErrorMsg: DataType, tryCastResult: Any): Unit = {
200200
checkExceptionInExpression[NumberFormatException](
201-
cast(l, to), s"Invalid input syntax for type ${expectedDataTypeInErrorMsg.sql}: 'true'")
201+
cast(l, to), s"""Invalid input syntax for type "${expectedDataTypeInErrorMsg.sql}": 'true'""")
202202
}
203203

204204
test("cast from invalid string array to numeric array should throw NumberFormatException") {
@@ -245,7 +245,7 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase {
245245

246246
checkExceptionInExpression[NumberFormatException](
247247
cast("abcd", DecimalType(38, 1)),
248-
s"Invalid input syntax for type ${DecimalType(38, 1).sql}: 'abcd'")
248+
s"""Invalid input syntax for type "${DecimalType(38, 1).sql}": 'abcd'""")
249249
}
250250

251251
protected def checkCastToBooleanError(l: Literal, to: DataType, tryCastResult: Any): Unit = {
@@ -260,7 +260,7 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase {
260260

261261
protected def checkCastToTimestampError(l: Literal, to: DataType): Unit = {
262262
checkExceptionInExpression[DateTimeException](
263-
cast(l, to), s"Invalid input syntax for type TIMESTAMP: ${toSQLValue(l)}")
263+
cast(l, to), s"""Invalid input syntax for type "TIMESTAMP": ${toSQLValue(l)}""")
264264
}
265265

266266
test("cast from timestamp II") {
@@ -281,7 +281,7 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase {
281281
assert(negativeTs.getTime < 0)
282282
Seq(ByteType, ShortType, IntegerType).foreach { dt =>
283283
checkExceptionInExpression[SparkArithmeticException](
284-
cast(negativeTs, dt), s"to ${dt.sql} causes overflow")
284+
cast(negativeTs, dt), s"""to "${dt.sql}" causes overflow""")
285285
}
286286
}
287287
}
@@ -292,7 +292,7 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase {
292292
assert(negativeTs.getTime < 0)
293293
Seq(ByteType, ShortType, IntegerType).foreach { dt =>
294294
checkExceptionInExpression[SparkArithmeticException](
295-
cast(negativeTs, dt), s"to ${dt.sql} causes overflow")
295+
cast(negativeTs, dt), s"""to "${dt.sql}" causes overflow""")
296296
}
297297
val expectedSecs = Math.floorDiv(negativeTs.getTime, MILLIS_PER_SECOND)
298298
checkEvaluation(cast(negativeTs, LongType), expectedSecs)
@@ -371,7 +371,7 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase {
371371
assert(ret.resolved == !isTryCast)
372372
if (!isTryCast) {
373373
checkExceptionInExpression[NumberFormatException](
374-
ret, s"Invalid input syntax for type ${IntegerType.sql}")
374+
ret, s"""Invalid input syntax for type "${IntegerType.sql}"""")
375375
}
376376
}
377377

@@ -389,7 +389,7 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase {
389389
assert(ret.resolved == !isTryCast)
390390
if (!isTryCast) {
391391
checkExceptionInExpression[NumberFormatException](
392-
ret, s"Invalid input syntax for type ${IntegerType.sql}")
392+
ret, s"""Invalid input syntax for type "${IntegerType.sql}"""")
393393
}
394394
}
395395
}
@@ -514,7 +514,7 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase {
514514
assert(ret.resolved === !isTryCast)
515515
if (!isTryCast) {
516516
checkExceptionInExpression[NumberFormatException](
517-
ret, s"Invalid input syntax for type ${IntegerType.sql}")
517+
ret, s"""Invalid input syntax for type "${IntegerType.sql}"""")
518518
}
519519
}
520520

@@ -523,7 +523,7 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase {
523523
def checkCastWithParseError(str: String): Unit = {
524524
checkExceptionInExpression[DateTimeException](
525525
cast(Literal(str), TimestampType, Option(zid.getId)),
526-
s"Invalid input syntax for type TIMESTAMP: '$str'")
526+
s"""Invalid input syntax for type "TIMESTAMP": '$str'""")
527527
}
528528

529529
checkCastWithParseError("123")
@@ -544,7 +544,7 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase {
544544
def checkCastWithParseError(str: String): Unit = {
545545
checkExceptionInExpression[DateTimeException](
546546
cast(Literal(str), DateType, Option(zid.getId)),
547-
s"Invalid input syntax for type DATE: '$str'")
547+
s"""Invalid input syntax for type "DATE": '$str'""")
548548
}
549549

550550
checkCastWithParseError("2015-13-18")
@@ -572,7 +572,7 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase {
572572
"2021-06-17 00:00:00ABC").foreach { invalidInput =>
573573
checkExceptionInExpression[DateTimeException](
574574
cast(invalidInput, TimestampNTZType),
575-
s"Invalid input syntax for type TIMESTAMP_NTZ: '$invalidInput'")
575+
s"""Invalid input syntax for type "TIMESTAMP_NTZ": '$invalidInput'""")
576576
}
577577
}
578578
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala

Lines changed: 33 additions & 33 deletions
Original file line numberDiff line numberDiff line change
@@ -592,15 +592,15 @@ class CastSuite extends CastSuiteBase {
592592
val e1 = intercept[ArithmeticException] {
593593
Cast(Literal(Byte.MaxValue + 1), ByteType).eval()
594594
}.getMessage
595-
assert(e1.contains("Casting 128 to TINYINT causes overflow"))
595+
assert(e1.contains("Casting 128 to \"TINYINT\" causes overflow"))
596596
val e2 = intercept[ArithmeticException] {
597597
Cast(Literal(Short.MaxValue + 1), ShortType).eval()
598598
}.getMessage
599-
assert(e2.contains("Casting 32768 to SMALLINT causes overflow"))
599+
assert(e2.contains("Casting 32768 to \"SMALLINT\" causes overflow"))
600600
val e3 = intercept[ArithmeticException] {
601601
Cast(Literal(Int.MaxValue + 1L), IntegerType).eval()
602602
}.getMessage
603-
assert(e3.contains("Casting 2147483648L to INT causes overflow"))
603+
assert(e3.contains("Casting 2147483648L to \"INT\" causes overflow"))
604604
}
605605
}
606606

@@ -642,15 +642,15 @@ class CastSuite extends CastSuiteBase {
642642
checkEvaluation(cast(v2, LongType), 25L)
643643
case MINUTE =>
644644
checkExceptionInExpression[ArithmeticException](cast(v2, ByteType),
645-
s"Casting $v2 to TINYINT causes overflow")
645+
s"""Casting $v2 to "TINYINT" causes overflow""")
646646
checkEvaluation(cast(v2, ShortType), (MINUTES_PER_HOUR * 25 + 1).toShort)
647647
checkEvaluation(cast(v2, IntegerType), (MINUTES_PER_HOUR * 25 + 1).toInt)
648648
checkEvaluation(cast(v2, LongType), MINUTES_PER_HOUR * 25 + 1)
649649
case SECOND =>
650650
checkExceptionInExpression[ArithmeticException](cast(v2, ByteType),
651-
s"Casting $v2 to TINYINT causes overflow")
651+
s"""Casting $v2 to "TINYINT" causes overflow""")
652652
checkExceptionInExpression[ArithmeticException](cast(v2, ShortType),
653-
s"Casting $v2 to SMALLINT causes overflow")
653+
s"""Casting $v2 to "SMALLINT" causes overflow""")
654654
checkEvaluation(cast(v2, IntegerType), num.toInt)
655655
checkEvaluation(cast(v2, LongType), num)
656656
}
@@ -659,69 +659,69 @@ class CastSuite extends CastSuiteBase {
659659
dt.endField match {
660660
case DAY =>
661661
checkExceptionInExpression[ArithmeticException](cast(v3, ByteType),
662-
s"Casting $v3 to TINYINT causes overflow")
662+
s"""Casting $v3 to "TINYINT" causes overflow""")
663663
checkExceptionInExpression[ArithmeticException](cast(v3, ShortType),
664-
s"Casting $v3 to SMALLINT causes overflow")
664+
s"""Casting $v3 to "SMALLINT" causes overflow""")
665665
checkEvaluation(cast(v3, IntegerType), (Long.MaxValue / MICROS_PER_DAY).toInt)
666666
checkEvaluation(cast(v3, LongType), Long.MaxValue / MICROS_PER_DAY)
667667
case HOUR =>
668668
checkExceptionInExpression[ArithmeticException](cast(v3, ByteType),
669-
s"Casting $v3 to TINYINT causes overflow")
669+
s"""Casting $v3 to "TINYINT" causes overflow""")
670670
checkExceptionInExpression[ArithmeticException](cast(v3, ShortType),
671-
s"Casting $v3 to SMALLINT causes overflow")
671+
s"""Casting $v3 to "SMALLINT" causes overflow""")
672672
checkExceptionInExpression[ArithmeticException](cast(v3, IntegerType),
673-
s"Casting $v3 to INT causes overflow")
673+
s"""Casting $v3 to "INT" causes overflow""")
674674
checkEvaluation(cast(v3, LongType), Long.MaxValue / MICROS_PER_HOUR)
675675
case MINUTE =>
676676
checkExceptionInExpression[ArithmeticException](cast(v3, ByteType),
677-
s"Casting $v3 to TINYINT causes overflow")
677+
s"""Casting $v3 to "TINYINT" causes overflow""")
678678
checkExceptionInExpression[ArithmeticException](cast(v3, ShortType),
679-
s"Casting $v3 to SMALLINT causes overflow")
679+
s"""Casting $v3 to "SMALLINT" causes overflow""")
680680
checkExceptionInExpression[ArithmeticException](cast(v3, IntegerType),
681-
s"Casting $v3 to INT causes overflow")
681+
s"""Casting $v3 to "INT" causes overflow""")
682682
checkEvaluation(cast(v3, LongType), Long.MaxValue / MICROS_PER_MINUTE)
683683
case SECOND =>
684684
checkExceptionInExpression[ArithmeticException](cast(v3, ByteType),
685-
s"Casting $v3 to TINYINT causes overflow")
685+
s"""Casting $v3 to "TINYINT" causes overflow""")
686686
checkExceptionInExpression[ArithmeticException](cast(v3, ShortType),
687-
s"Casting $v3 to SMALLINT causes overflow")
687+
s"""Casting $v3 to "SMALLINT" causes overflow""")
688688
checkExceptionInExpression[ArithmeticException](cast(v3, IntegerType),
689-
s"Casting $v3 to INT causes overflow")
689+
s"""Casting $v3 to "INT" causes overflow""")
690690
checkEvaluation(cast(v3, LongType), Long.MaxValue / MICROS_PER_SECOND)
691691
}
692692

693693
val v4 = Literal.create(Duration.of(Long.MinValue, ChronoUnit.MICROS), dt)
694694
dt.endField match {
695695
case DAY =>
696696
checkExceptionInExpression[ArithmeticException](cast(v4, ByteType),
697-
s"Casting $v4 to TINYINT causes overflow")
697+
s"""Casting $v4 to "TINYINT" causes overflow""")
698698
checkExceptionInExpression[ArithmeticException](cast(v4, ShortType),
699-
s"Casting $v4 to SMALLINT causes overflow")
699+
s"""Casting $v4 to "SMALLINT" causes overflow""")
700700
checkEvaluation(cast(v4, IntegerType), (Long.MinValue / MICROS_PER_DAY).toInt)
701701
checkEvaluation(cast(v4, LongType), Long.MinValue / MICROS_PER_DAY)
702702
case HOUR =>
703703
checkExceptionInExpression[ArithmeticException](cast(v4, ByteType),
704-
s"Casting $v4 to TINYINT causes overflow")
704+
s"""Casting $v4 to "TINYINT" causes overflow""")
705705
checkExceptionInExpression[ArithmeticException](cast(v4, ShortType),
706-
s"Casting $v4 to SMALLINT causes overflow")
706+
s"""Casting $v4 to "SMALLINT" causes overflow""")
707707
checkExceptionInExpression[ArithmeticException](cast(v4, IntegerType),
708-
s"Casting $v4 to INT causes overflow")
708+
s"""Casting $v4 to "INT" causes overflow""")
709709
checkEvaluation(cast(v4, LongType), Long.MinValue / MICROS_PER_HOUR)
710710
case MINUTE =>
711711
checkExceptionInExpression[ArithmeticException](cast(v4, ByteType),
712-
s"Casting $v4 to TINYINT causes overflow")
712+
s"""Casting $v4 to "TINYINT" causes overflow""")
713713
checkExceptionInExpression[ArithmeticException](cast(v4, ShortType),
714-
s"Casting $v4 to SMALLINT causes overflow")
714+
s"""Casting $v4 to "SMALLINT" causes overflow""")
715715
checkExceptionInExpression[ArithmeticException](cast(v4, IntegerType),
716-
s"Casting $v4 to INT causes overflow")
716+
s"""Casting $v4 to "INT" causes overflow""")
717717
checkEvaluation(cast(v4, LongType), Long.MinValue / MICROS_PER_MINUTE)
718718
case SECOND =>
719719
checkExceptionInExpression[ArithmeticException](cast(v4, ByteType),
720-
s"Casting $v4 to TINYINT causes overflow")
720+
s"""Casting $v4 to "TINYINT" causes overflow""")
721721
checkExceptionInExpression[ArithmeticException](cast(v4, ShortType),
722-
s"Casting $v4 to SMALLINT causes overflow")
722+
s"""Casting $v4 to "SMALLINT" causes overflow""")
723723
checkExceptionInExpression[ArithmeticException](cast(v4, IntegerType),
724-
s"Casting $v4 to INT causes overflow")
724+
s"""Casting $v4 to "INT" causes overflow""")
725725
checkEvaluation(cast(v4, LongType), Long.MinValue / MICROS_PER_SECOND)
726726
}
727727
}
@@ -777,7 +777,7 @@ class CastSuite extends CastSuiteBase {
777777
).foreach {
778778
case (v, toType) =>
779779
checkExceptionInExpression[ArithmeticException](cast(v, toType),
780-
s"Casting $v to ${toType.sql} causes overflow")
780+
s"""Casting $v to "${toType.sql}" causes overflow""")
781781
}
782782

783783
Seq(
@@ -792,7 +792,7 @@ class CastSuite extends CastSuiteBase {
792792
).foreach {
793793
case (v, toType) =>
794794
checkExceptionInExpression[ArithmeticException](cast(v, toType),
795-
s"Casting ${v}L to ${toType.sql} causes overflow")
795+
s"""Casting ${v}L to "${toType.sql}" causes overflow""")
796796
}
797797
}
798798

@@ -829,7 +829,7 @@ class CastSuite extends CastSuiteBase {
829829
case (v, dt, toType) =>
830830
val value = Literal.create(v, dt)
831831
checkExceptionInExpression[ArithmeticException](cast(value, toType),
832-
s"Casting $value to ${toType.sql} causes overflow")
832+
s"""Casting $value to "${toType.sql}" causes overflow""")
833833
}
834834

835835
Seq(
@@ -887,7 +887,7 @@ class CastSuite extends CastSuiteBase {
887887
).foreach {
888888
case (v, toType) =>
889889
checkExceptionInExpression[ArithmeticException](cast(v, toType),
890-
s"Casting $v to ${toType.sql} causes overflow")
890+
s"""Casting $v to "${toType.sql}" causes overflow""")
891891
}
892892

893893
Seq(
@@ -898,7 +898,7 @@ class CastSuite extends CastSuiteBase {
898898
).foreach {
899899
case (v, toType) =>
900900
checkExceptionInExpression[ArithmeticException](cast(v, toType),
901-
s"Casting ${v}L to ${toType.sql} causes overflow")
901+
s"""Casting ${v}L to "${toType.sql}" causes overflow""")
902902
}
903903
}
904904
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateFormatterSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -208,6 +208,6 @@ class DateFormatterSuite extends DatetimeFormatterSuite {
208208
val errMsg = intercept[DateTimeException] {
209209
formatter.parse("x123")
210210
}.getMessage
211-
assert(errMsg.contains("Invalid input syntax for type DATE: 'x123'"))
211+
assert(errMsg.contains("""Invalid input syntax for type "DATE": 'x123'"""))
212212
}
213213
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TimestampFormatterSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -453,7 +453,7 @@ class TimestampFormatterSuite extends DatetimeFormatterSuite {
453453
val errMsg = intercept[DateTimeException] {
454454
formatter.parse("x123")
455455
}.getMessage
456-
assert(errMsg.contains("Invalid input syntax for type TIMESTAMP: 'x123'"))
456+
assert(errMsg.contains("""Invalid input syntax for type "TIMESTAMP": 'x123'"""))
457457
}
458458
}
459459
}

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -285,7 +285,7 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester with SQLHelper
285285
assert(Decimal.fromString(UTF8String.fromString("str")) === null)
286286
val e = intercept[NumberFormatException](Decimal.fromStringANSI(UTF8String.fromString("str")))
287287
assert(e.getMessage.contains("Invalid input syntax for type " +
288-
s"${DecimalType.USER_DEFAULT.sql}: 'str'"))
288+
s""""${DecimalType.USER_DEFAULT.sql}": 'str'"""))
289289
}
290290

291291
test("SPARK-35841: Casting string to decimal type doesn't work " +

0 commit comments

Comments
 (0)