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
The table of contents is too big for display.
Diff view
Diff view
  •  
  •  
  •  
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ class AvroCodecSuite extends FileSourceCodecSuite {
s"""CREATE TABLE avro_t
|USING $format OPTIONS('compression'='unsupported')
|AS SELECT 1 as id""".stripMargin)),
errorClass = "CODEC_SHORT_NAME_NOT_FOUND",
condition = "CODEC_SHORT_NAME_NOT_FOUND",
sqlState = Some("42704"),
parameters = Map("codecName" -> "unsupported")
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -329,7 +329,7 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession {
s"""
|select to_avro(s, 42) as result from t
|""".stripMargin)),
errorClass = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT",
condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT",
parameters = Map("sqlExpr" -> "\"to_avro(s, 42)\"",
"msg" -> ("The second argument of the TO_AVRO SQL function must be a constant string " +
"containing the JSON representation of the schema to use for converting the value to " +
Expand All @@ -344,7 +344,7 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession {
s"""
|select from_avro(s, 42, '') as result from t
|""".stripMargin)),
errorClass = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT",
condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT",
parameters = Map("sqlExpr" -> "\"from_avro(s, 42, )\"",
"msg" -> ("The second argument of the FROM_AVRO SQL function must be a constant string " +
"containing the JSON representation of the schema to use for converting the value " +
Expand All @@ -359,7 +359,7 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession {
s"""
|select from_avro(s, '$jsonFormatSchema', 42) as result from t
|""".stripMargin)),
errorClass = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT",
condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT",
parameters = Map(
"sqlExpr" ->
s"\"from_avro(s, $jsonFormatSchema, 42)\"".stripMargin,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -439,7 +439,7 @@ abstract class AvroLogicalTypeSuite extends QueryTest with SharedSparkSession {
assert(ex.getErrorClass.startsWith("FAILED_READ_FILE"))
checkError(
exception = ex.getCause.asInstanceOf[SparkArithmeticException],
errorClass = "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION",
condition = "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION",
parameters = Map(
"value" -> "0",
"precision" -> "4",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -894,7 +894,7 @@ abstract class AvroSuite
assert(ex.getErrorClass.startsWith("FAILED_READ_FILE"))
checkError(
exception = ex.getCause.asInstanceOf[AnalysisException],
errorClass = "AVRO_INCOMPATIBLE_READ_TYPE",
condition = "AVRO_INCOMPATIBLE_READ_TYPE",
parameters = Map("avroPath" -> "field 'a'",
"sqlPath" -> "field 'a'",
"avroType" -> "decimal\\(12,10\\)",
Expand Down Expand Up @@ -972,7 +972,7 @@ abstract class AvroSuite
assert(ex.getErrorClass.startsWith("FAILED_READ_FILE"))
checkError(
exception = ex.getCause.asInstanceOf[AnalysisException],
errorClass = "AVRO_INCOMPATIBLE_READ_TYPE",
condition = "AVRO_INCOMPATIBLE_READ_TYPE",
parameters = Map("avroPath" -> "field 'a'",
"sqlPath" -> "field 'a'",
"avroType" -> "interval day to second",
Expand Down Expand Up @@ -1009,7 +1009,7 @@ abstract class AvroSuite
assert(ex.getErrorClass.startsWith("FAILED_READ_FILE"))
checkError(
exception = ex.getCause.asInstanceOf[AnalysisException],
errorClass = "AVRO_INCOMPATIBLE_READ_TYPE",
condition = "AVRO_INCOMPATIBLE_READ_TYPE",
parameters = Map("avroPath" -> "field 'a'",
"sqlPath" -> "field 'a'",
"avroType" -> "interval year to month",
Expand Down Expand Up @@ -1673,15 +1673,15 @@ abstract class AvroSuite
exception = intercept[AnalysisException] {
sql("select interval 1 days").write.format("avro").mode("overwrite").save(tempDir)
},
errorClass = "_LEGACY_ERROR_TEMP_1136",
condition = "_LEGACY_ERROR_TEMP_1136",
parameters = Map.empty
)
checkError(
exception = intercept[AnalysisException] {
spark.udf.register("testType", () => new IntervalData())
sql("select testType()").write.format("avro").mode("overwrite").save(tempDir)
},
errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE",
condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE",
parameters = Map(
"columnName" -> "`testType()`",
"columnType" -> "UDT(\"INTERVAL\")",
Expand Down Expand Up @@ -2726,7 +2726,7 @@ abstract class AvroSuite
|LOCATION '${dir}'
|AS SELECT ID, IF(ID=1,1,0) FROM v""".stripMargin)
},
errorClass = "INVALID_COLUMN_NAME_AS_PATH",
condition = "INVALID_COLUMN_NAME_AS_PATH",
parameters = Map(
"datasource" -> "AvroFileFormat", "columnName" -> "`(IF((ID = 1), 1, 0))`")
)
Expand Down Expand Up @@ -2831,7 +2831,7 @@ class AvroV1Suite extends AvroSuite {
sql("SELECT ID, IF(ID=1,1,0) FROM v").write.mode(SaveMode.Overwrite)
.format("avro").save(dir.getCanonicalPath)
},
errorClass = "INVALID_COLUMN_NAME_AS_PATH",
condition = "INVALID_COLUMN_NAME_AS_PATH",
parameters = Map(
"datasource" -> "AvroFileFormat", "columnName" -> "`(IF((ID = 1), 1, 0))`")
)
Expand All @@ -2844,7 +2844,7 @@ class AvroV1Suite extends AvroSuite {
.write.mode(SaveMode.Overwrite)
.format("avro").save(dir.getCanonicalPath)
},
errorClass = "INVALID_COLUMN_NAME_AS_PATH",
condition = "INVALID_COLUMN_NAME_AS_PATH",
parameters = Map(
"datasource" -> "AvroFileFormat", "columnName" -> "`(IF((ID = 1), 1, 0))`")
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -490,7 +490,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite {
.option("query", "SELECT @myvariant1 as variant1, @myvariant2 as variant2")
.load()
},
errorClass = "UNRECOGNIZED_SQL_TYPE",
condition = "UNRECOGNIZED_SQL_TYPE",
parameters = Map("typeName" -> "sql_variant", "jdbcType" -> "-156"))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ class DB2IntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest {
exception = intercept[AnalysisException] {
sql(sql1)
},
errorClass = "NOT_SUPPORTED_CHANGE_COLUMN",
condition = "NOT_SUPPORTED_CHANGE_COLUMN",
parameters = Map(
"originType" -> "\"DOUBLE\"",
"newType" -> "\"VARCHAR(10)\"",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JD
exception = intercept[AnalysisException] {
sql(sql1)
},
errorClass = "NOT_SUPPORTED_CHANGE_COLUMN",
condition = "NOT_SUPPORTED_CHANGE_COLUMN",
parameters = Map(
"originType" -> "\"STRING\"",
"newType" -> "\"INT\"",
Expand All @@ -115,7 +115,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JD
exception = intercept[SparkSQLFeatureNotSupportedException] {
sql(s"ALTER TABLE $tbl ALTER COLUMN ID DROP NOT NULL")
},
errorClass = "_LEGACY_ERROR_TEMP_2271")
condition = "_LEGACY_ERROR_TEMP_2271")
}

test("SPARK-47440: SQLServer does not support boolean expression in binary comparison") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -98,7 +98,7 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest
exception = intercept[AnalysisException] {
sql(sql1)
},
errorClass = "NOT_SUPPORTED_CHANGE_COLUMN",
condition = "NOT_SUPPORTED_CHANGE_COLUMN",
parameters = Map(
"originType" -> "\"STRING\"",
"newType" -> "\"INT\"",
Expand Down Expand Up @@ -131,7 +131,7 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest
exception = intercept[SparkSQLFeatureNotSupportedException] {
sql(s"ALTER TABLE $tbl ALTER COLUMN ID DROP NOT NULL")
},
errorClass = "_LEGACY_ERROR_TEMP_2271")
condition = "_LEGACY_ERROR_TEMP_2271")
}

override def testCreateTableWithProperty(tbl: String): Unit = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ class MySQLNamespaceSuite extends DockerJDBCIntegrationSuite with V2JDBCNamespac
exception = intercept[SparkSQLFeatureNotSupportedException] {
catalog.createNamespace(Array("foo"), Map("comment" -> "test comment").asJava)
},
errorClass = "UNSUPPORTED_FEATURE.COMMENT_NAMESPACE",
condition = "UNSUPPORTED_FEATURE.COMMENT_NAMESPACE",
parameters = Map("namespace" -> "`foo`")
)
assert(catalog.namespaceExists(Array("foo")) === false)
Expand All @@ -74,23 +74,23 @@ class MySQLNamespaceSuite extends DockerJDBCIntegrationSuite with V2JDBCNamespac
Array("foo"),
NamespaceChange.setProperty("comment", "comment for foo"))
},
errorClass = "UNSUPPORTED_FEATURE.COMMENT_NAMESPACE",
condition = "UNSUPPORTED_FEATURE.COMMENT_NAMESPACE",
parameters = Map("namespace" -> "`foo`")
)

checkError(
exception = intercept[SparkSQLFeatureNotSupportedException] {
catalog.alterNamespace(Array("foo"), NamespaceChange.removeProperty("comment"))
},
errorClass = "UNSUPPORTED_FEATURE.REMOVE_NAMESPACE_COMMENT",
condition = "UNSUPPORTED_FEATURE.REMOVE_NAMESPACE_COMMENT",
parameters = Map("namespace" -> "`foo`")
)

checkError(
exception = intercept[SparkSQLFeatureNotSupportedException] {
catalog.dropNamespace(Array("foo"), cascade = false)
},
errorClass = "UNSUPPORTED_FEATURE.DROP_NAMESPACE",
condition = "UNSUPPORTED_FEATURE.DROP_NAMESPACE",
parameters = Map("namespace" -> "`foo`")
)
catalog.dropNamespace(Array("foo"), cascade = true)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,7 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTes
exception = intercept[AnalysisException] {
sql(sql1)
},
errorClass = "NOT_SUPPORTED_CHANGE_COLUMN",
condition = "NOT_SUPPORTED_CHANGE_COLUMN",
parameters = Map(
"originType" -> "\"DECIMAL(19,0)\"",
"newType" -> "\"INT\"",
Expand All @@ -139,7 +139,7 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTes
exception = intercept[SparkRuntimeException] {
sql(s"INSERT INTO $tableName SELECT rpad('hi', 256, 'spark')")
},
errorClass = "EXCEED_LIMIT_LENGTH",
condition = "EXCEED_LIMIT_LENGTH",
parameters = Map("limit" -> "255")
)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCT
exception = intercept[AnalysisException] {
sql(sql1)
},
errorClass = "NOT_SUPPORTED_CHANGE_COLUMN",
condition = "NOT_SUPPORTED_CHANGE_COLUMN",
parameters = Map(
"originType" -> "\"STRING\"",
"newType" -> "\"INT\"",
Expand Down Expand Up @@ -118,7 +118,7 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCT
sql(s"CREATE TABLE $t2(c int)")
checkError(
exception = intercept[TableAlreadyExistsException](sql(s"ALTER TABLE $t1 RENAME TO t2")),
errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS",
condition = "TABLE_OR_VIEW_ALREADY_EXISTS",
parameters = Map("relationName" -> "`t2`")
)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ private[v2] trait V2JDBCNamespaceTest extends SharedSparkSession with DockerInte
catalog.listNamespaces(Array("foo"))
}
checkError(e,
errorClass = "SCHEMA_NOT_FOUND",
condition = "SCHEMA_NOT_FOUND",
parameters = Map("schemaName" -> "`foo`"))
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu
exception = intercept[AnalysisException] {
sql(sqlText)
},
errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION",
condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION",
sqlState = "42703",
parameters = Map(
"objectName" -> "`bad_column`",
Expand All @@ -92,11 +92,11 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu

private def checkErrorFailedJDBC(
e: AnalysisException,
errorClass: String,
condition: String,
tbl: String): Unit = {
checkErrorMatchPVals(
exception = e,
errorClass = errorClass,
condition = condition,
parameters = Map(
"url" -> "jdbc:.*",
"tableName" -> s"`$tbl`")
Expand Down Expand Up @@ -126,7 +126,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu
exception = intercept[AnalysisException] {
sql(s"ALTER TABLE $catalogName.alt_table ADD COLUMNS (C3 DOUBLE)")
},
errorClass = "FIELD_ALREADY_EXISTS",
condition = "FIELD_ALREADY_EXISTS",
parameters = Map(
"op" -> "add",
"fieldNames" -> "`C3`",
Expand Down Expand Up @@ -159,7 +159,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu
exception = intercept[AnalysisException] {
sql(sqlText)
},
errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION",
condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION",
sqlState = "42703",
parameters = Map(
"objectName" -> "`bad_column`",
Expand All @@ -182,7 +182,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu
exception = intercept[AnalysisException] {
sql(sqlText)
},
errorClass = "UNRESOLVED_COLUMN.WITH_SUGGESTION",
condition = "UNRESOLVED_COLUMN.WITH_SUGGESTION",
sqlState = "42703",
parameters = Map(
"objectName" -> "`bad_column`",
Expand All @@ -206,7 +206,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu
exception = intercept[AnalysisException] {
sql(s"ALTER TABLE $catalogName.alt_table RENAME COLUMN ID1 TO ID2")
},
errorClass = "FIELD_ALREADY_EXISTS",
condition = "FIELD_ALREADY_EXISTS",
parameters = Map(
"op" -> "rename",
"fieldNames" -> "`ID2`",
Expand Down Expand Up @@ -308,7 +308,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu
exception = intercept[IndexAlreadyExistsException] {
sql(s"CREATE index i1 ON $catalogName.new_table (col1)")
},
errorClass = "INDEX_ALREADY_EXISTS",
condition = "INDEX_ALREADY_EXISTS",
parameters = Map("indexName" -> "`i1`", "tableName" -> "`new_table`")
)

Expand All @@ -333,7 +333,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu
exception = intercept[NoSuchIndexException] {
sql(s"DROP index i1 ON $catalogName.new_table")
},
errorClass = "INDEX_NOT_FOUND",
condition = "INDEX_NOT_FOUND",
parameters = Map("indexName" -> "`i1`", "tableName" -> "`new_table`")
)
}
Expand Down Expand Up @@ -975,7 +975,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu
exception = intercept[AnalysisException] {
sql(s"ALTER TABLE $catalogName.tbl2 RENAME TO tbl1")
},
errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS",
condition = "TABLE_OR_VIEW_ALREADY_EXISTS",
parameters = Map("relationName" -> "`tbl1`")
)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -153,7 +153,7 @@ class KafkaOffsetReaderSuite extends QueryTest with SharedSparkSession with Kafk
}
checkError(
exception = ex,
errorClass = "KAFKA_START_OFFSET_DOES_NOT_MATCH_ASSIGNED",
condition = "KAFKA_START_OFFSET_DOES_NOT_MATCH_ASSIGNED",
parameters = Map(
"specifiedPartitions" -> "Set\\(.*,.*\\)",
"assignedPartitions" -> "Set\\(.*,.*,.*\\)"),
Expand Down
Loading