Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 0 additions & 5 deletions common/utils/src/main/resources/error/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -6125,11 +6125,6 @@
"<className> is not a valid Spark SQL Data Source."
]
},
"_LEGACY_ERROR_TEMP_1136" : {
"message" : [
"Cannot save interval data type into external storage."
]
},
"_LEGACY_ERROR_TEMP_1137" : {
"message" : [
"Unable to resolve <name> given [<outputStr>]."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1673,8 +1673,12 @@ abstract class AvroSuite
exception = intercept[AnalysisException] {
sql("select interval 1 days").write.format("avro").mode("overwrite").save(tempDir)
},
condition = "_LEGACY_ERROR_TEMP_1136",
parameters = Map.empty
condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE",
parameters = Map(
"format" -> "Avro",
"columnName" -> "`INTERVAL '1 days'`",
"columnType" -> "\"INTERVAL\""
)
)
checkError(
exception = intercept[AnalysisException] {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1691,12 +1691,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
messageParameters = Map("className" -> className))
}

def cannotSaveIntervalIntoExternalStorageError(): Throwable = {
new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_1136",
messageParameters = Map.empty)
}

def cannotResolveAttributeError(name: String, outputStr: String): Throwable = {
new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_1137",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -514,7 +514,8 @@ case class DataSource(
dataSource.createRelation(
sparkSession.sqlContext, mode, caseInsensitiveOptions, Dataset.ofRows(sparkSession, data))
case format: FileFormat =>
disallowWritingIntervals(outputColumns.map(_.dataType), forbidAnsiIntervals = false)
disallowWritingIntervals(
outputColumns.toStructType.asNullable, format.toString, forbidAnsiIntervals = false)
val cmd = planForWritingFileFormat(format, mode, data)
val qe = sparkSession.sessionState.executePlan(cmd)
qe.assertCommandExecuted()
Expand All @@ -539,7 +540,7 @@ case class DataSource(
}
SaveIntoDataSourceCommand(data, dataSource, caseInsensitiveOptions, mode)
case format: FileFormat =>
disallowWritingIntervals(data.schema.map(_.dataType), forbidAnsiIntervals = false)
disallowWritingIntervals(data.schema, format.toString, forbidAnsiIntervals = false)
DataSource.validateSchema(data.schema, sparkSession.sessionState.conf)
planForWritingFileFormat(format, mode, data)
case _ => throw SparkException.internalError(
Expand All @@ -566,12 +567,15 @@ case class DataSource(
}

private def disallowWritingIntervals(
dataTypes: Seq[DataType],
outputColumns: Seq[StructField],
format: String,
forbidAnsiIntervals: Boolean): Unit = {
dataTypes.foreach(
TypeUtils.invokeOnceForInterval(_, forbidAnsiIntervals) {
throw QueryCompilationErrors.cannotSaveIntervalIntoExternalStorageError()
})
outputColumns.foreach { field =>
TypeUtils.invokeOnceForInterval(field.dataType, forbidAnsiIntervals) {
throw QueryCompilationErrors.dataTypeUnsupportedByDataSourceError(
format, field
)}
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -506,14 +506,23 @@ class FileBasedDataSourceSuite extends QueryTest
withSQLConf(
SQLConf.USE_V1_SOURCE_LIST.key -> useV1List,
SQLConf.LEGACY_INTERVAL_ENABLED.key -> "true") {
val formatMapping = Map(
"csv" -> "CSV",
"json" -> "JSON",
"parquet" -> "Parquet",
"orc" -> "ORC"
)
// write path
Seq("csv", "json", "parquet", "orc").foreach { format =>
checkError(
exception = intercept[AnalysisException] {
sql("select interval 1 days").write.format(format).mode("overwrite").save(tempDir)
},
condition = "_LEGACY_ERROR_TEMP_1136",
parameters = Map.empty
condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE",
parameters = Map(
"format" -> formatMapping(format),
"columnName" -> "`INTERVAL '1 days'`",
"columnType" -> "\"INTERVAL\"")
)
}

Expand Down