diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 889ecf9f7b08a..eeefe31ccc204 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4105,6 +4105,29 @@ }, "sqlState" : "42K0E" }, + "INVALID_LINE_SEPARATOR" : { + "message" : [ + "Invalid line separator configuration." + ], + "subClass" : { + "EMPTY" : { + "message" : [ + "The 'lineSep' option cannot be an empty string." + ] + }, + "NULL" : { + "message" : [ + "The 'lineSep' option cannot be a null value." + ] + }, + "TOO_LONG" : { + "message" : [ + "The 'lineSep' option can contain at most 2 characters, but got characters." + ] + } + }, + "sqlState" : "22023" + }, "INVALID_LOG_VERSION" : { "message" : [ "UnsupportedLogVersion." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala index 9edb1603f4638..4ae7aad19a9c7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala @@ -284,11 +284,15 @@ class CSVOptions( * A string between two consecutive JSON records. */ val lineSeparator: Option[String] = parameters.get(LINE_SEP).map { sep => - require(sep != null, "'lineSep' cannot be a null value.") - require(sep.nonEmpty, "'lineSep' cannot be an empty string.") - // Intentionally allow it up to 2 for Window's CRLF although multiple - // characters have an issue with quotes. This is intentionally undocumented. - require(sep.length <= 2, "'lineSep' can contain only 1 character.") + if (sep == null) { + throw QueryExecutionErrors.lineSepCannotBeNullError() + } + if (sep.isEmpty) { + throw QueryExecutionErrors.lineSepCannotBeEmptyError() + } + if (sep.length > 2) { + throw QueryExecutionErrors.lineSepTooLongError(sep.length) + } if (sep.length == 2) logWarning("It is not recommended to set 'lineSep' " + "with 2 characters due to the limitation of supporting multi-char 'lineSep' within quotes.") sep diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index 0aa8308276871..b97244c1c2805 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -3349,4 +3349,20 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE "expectedFamily" -> expectedFamily, "actualFamily" -> actualFamily)) } + + def lineSepCannotBeNullError(): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "INVALID_LINE_SEPARATOR.NULL") + } + + def lineSepCannotBeEmptyError(): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "INVALID_LINE_SEPARATOR.EMPTY") + } + + def lineSepTooLongError(length: Int): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "INVALID_LINE_SEPARATOR.TOO_LONG", + messageParameters = Map("length" -> length.toString)) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 24f80f4b928f6..22b291677cd8b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -2484,15 +2484,21 @@ abstract class CSVSuite // scalastyle:on nonascii test("lineSep restrictions") { - val errMsg1 = intercept[IllegalArgumentException] { - spark.read.option("lineSep", "").csv(testFile(carsFile)).collect() - }.getMessage - assert(errMsg1.contains("'lineSep' cannot be an empty string")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + spark.read.option("lineSep", "").csv(testFile(carsFile)).collect() + }, + condition = "INVALID_LINE_SEPARATOR.EMPTY", + parameters = Map.empty + ) - val errMsg2 = intercept[IllegalArgumentException] { - spark.read.option("lineSep", "123").csv(testFile(carsFile)).collect() - }.getMessage - assert(errMsg2.contains("'lineSep' can contain only 1 character")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + spark.read.option("lineSep", "123").csv(testFile(carsFile)).collect() + }, + condition = "INVALID_LINE_SEPARATOR.TOO_LONG", + parameters = Map("length" -> "3") + ) } Seq(true, false).foreach { multiLine =>