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
Original file line number Diff line number Diff line change
Expand Up @@ -311,11 +311,14 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {

test("SPARK-32374: disallow setting properties for CREATE TEMPORARY VIEW") {
withTempView("myabcdview") {
val e = intercept[ParseException] {
sql("CREATE TEMPORARY VIEW myabcdview TBLPROPERTIES ('a' = 'b') AS SELECT * FROM jt")
}
assert(e.message.contains(
"Operation not allowed: TBLPROPERTIES can't coexist with CREATE TEMPORARY VIEW"))
val sqlText = "CREATE TEMPORARY VIEW myabcdview TBLPROPERTIES ('a' = 'b') AS SELECT * FROM jt"
checkError(
exception = intercept[ParseException] {
sql(sqlText)
},
errorClass = "_LEGACY_ERROR_TEMP_0035",
parameters = Map("message" -> "TBLPROPERTIES can't coexist with CREATE TEMPORARY VIEW"),
context = ExpectedContext(sqlText, 0, 77))
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,17 +46,17 @@ class SparkScriptTransformationSuite extends BaseScriptTransformationSuite with
val df = Seq("a", "b", "c").map(Tuple1.apply).toDF("a")
df.createTempView("v")

val e = intercept[ParseException] {
sql(
"""
|SELECT TRANSFORM (a)
|ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
|USING 'cat' AS (a)
|ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
|FROM v
""".stripMargin)
}.getMessage
assert(e.contains("TRANSFORM with SERDE is only supported in hive mode."))
val sqlText =
"""SELECT TRANSFORM (a)
|ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
|USING 'cat' AS (a)
|ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
|FROM v""".stripMargin
checkError(
exception = intercept[ParseException](sql(sqlText)),
errorClass = "UNSUPPORTED_FEATURE.TRANSFORM_NON_HIVE",
parameters = Map.empty,
context = ExpectedContext(sqlText, 0, 185))
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -53,16 +53,18 @@ class JdbcUtilsSuite extends SparkFunSuite {
"Found duplicate column(s) in the customSchema option value"))

// Throw ParseException
val dataTypeNotSupported = intercept[ParseException]{
JdbcUtils.getCustomSchema(tableSchema, "c3 DATEE, C2 STRING", caseInsensitive) ===
StructType(Seq(StructField("c3", DateType, false), StructField("C2", StringType, false)))
}
assert(dataTypeNotSupported.getMessage.contains("DataType datee is not supported"))
checkError(
exception = intercept[ParseException]{
JdbcUtils.getCustomSchema(tableSchema, "c3 DATEE, C2 STRING", caseInsensitive)
},
errorClass = "_LEGACY_ERROR_TEMP_0030",
parameters = Map("dataType" -> "datee"))

val mismatchedInput = intercept[ParseException]{
JdbcUtils.getCustomSchema(tableSchema, "c3 DATE. C2 STRING", caseInsensitive) ===
StructType(Seq(StructField("c3", DateType, false), StructField("C2", StringType, false)))
}
assert(mismatchedInput.getMessage.contains("Syntax error at or near '.'"))
checkError(
exception = intercept[ParseException]{
JdbcUtils.getCustomSchema(tableSchema, "c3 DATE. C2 STRING", caseInsensitive)
},
errorClass = "PARSE_SYNTAX_ERROR",
parameters = Map("error" -> "'.'", "hint" -> ""))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -267,10 +267,13 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession {
}.getMessage
assert(msg1.contains("Missing field bad_column in table h2.test.alt_table"))
// Update column to wrong type
val msg2 = intercept[ParseException] {
sql(s"ALTER TABLE $tableName ALTER COLUMN id TYPE bad_type")
}.getMessage
assert(msg2.contains("DataType bad_type is not supported"))
checkError(
exception = intercept[ParseException] {
sql(s"ALTER TABLE $tableName ALTER COLUMN id TYPE bad_type")
},
errorClass = "_LEGACY_ERROR_TEMP_0030",
parameters = Map("dataType" -> "bad_type"),
context = ExpectedContext("bad_type", 51, 58))
}
// Update column type in not existing table and namespace
Seq("h2.test.not_existing_table", "h2.bad_test.not_existing_table").foreach { table =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -473,8 +473,12 @@ class SQLConfSuite extends QueryTest with SharedSparkSession {
assert(zone === String.format("%+03d:00", Integer.valueOf(i)))
}
}
val e2 = intercept[ParseException](sql("set time zone interval 19 hours"))
assert(e2.getMessage contains "The interval value must be in the range of [-18, +18] hours")
val sqlText = "set time zone interval 19 hours"
checkError(
exception = intercept[ParseException](sql(sqlText)),
errorClass = "_LEGACY_ERROR_TEMP_0044",
parameters = Map.empty,
context = ExpectedContext(sqlText, 0, 30))
}

test("SPARK-34454: configs from the legacy namespace should be internal") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -496,22 +496,26 @@ class JDBCWriteSuite extends SharedSparkSession with BeforeAndAfter {

test("SPARK-10849: jdbc CreateTableColumnTypes option with invalid data type") {
val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2)
val msg = intercept[ParseException] {
df.write.mode(SaveMode.Overwrite)
.option("createTableColumnTypes", "name CLOB(2000)")
.jdbc(url1, "TEST.USERDBTYPETEST", properties)
}.getMessage()
assert(msg.contains("DataType clob(2000) is not supported."))
checkError(
exception = intercept[ParseException] {
df.write.mode(SaveMode.Overwrite)
.option("createTableColumnTypes", "name CLOB(2000)")
.jdbc(url1, "TEST.USERDBTYPETEST", properties)
},
errorClass = "_LEGACY_ERROR_TEMP_0030",
parameters = Map("dataType" -> "clob(2000)"))
}

test("SPARK-10849: jdbc CreateTableColumnTypes option with invalid syntax") {
val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2)
val msg = intercept[ParseException] {
df.write.mode(SaveMode.Overwrite)
.option("createTableColumnTypes", "`name char(20)") // incorrectly quoted column
.jdbc(url1, "TEST.USERDBTYPETEST", properties)
}.getMessage()
assert(msg.contains("Syntax error at or near '`': extra input '`'"))
checkError(
exception = intercept[ParseException] {
df.write.mode(SaveMode.Overwrite)
.option("createTableColumnTypes", "`name char(20)") // incorrectly quoted column
.jdbc(url1, "TEST.USERDBTYPETEST", properties)
},
errorClass = "PARSE_SYNTAX_ERROR",
parameters = Map("error" -> "'`'", "hint" -> ": extra input '`'"))
}

test("SPARK-10849: jdbc CreateTableColumnTypes duplicate columns") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -155,18 +155,23 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSparkSession {

test("disallows CREATE TEMPORARY TABLE ... USING ... AS query") {
withTable("t") {
val error = intercept[ParseException] {
sql(
s"""
|CREATE TEMPORARY TABLE t USING PARQUET
|OPTIONS (PATH '${path.toURI}')
|PARTITIONED BY (a)
|AS SELECT 1 AS a, 2 AS b
""".stripMargin
)
}.getMessage
assert(error.contains("Operation not allowed") &&
error.contains("CREATE TEMPORARY TABLE"))
val pathUri = path.toURI.toString
val sqlText =
s"""CREATE TEMPORARY TABLE t USING PARQUET
|OPTIONS (PATH '$pathUri')
|PARTITIONED BY (a)
|AS SELECT 1 AS a, 2 AS b""".stripMargin
checkError(
exception = intercept[ParseException] {
sql(sqlText)
},
errorClass = "_LEGACY_ERROR_TEMP_0035",
parameters = Map(
"message" -> "CREATE TEMPORARY TABLE ... AS ..., use CREATE TEMPORARY VIEW instead"),
context = ExpectedContext(
fragment = sqlText,
start = 0,
stop = 99 + pathUri.length))
}
}

Expand Down Expand Up @@ -282,10 +287,18 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSparkSession {

test("specifying the column list for CTAS") {
withTable("t") {
val e = intercept[ParseException] {
sql("CREATE TABLE t (a int, b int) USING parquet AS SELECT 1, 2")
}.getMessage
assert(e.contains("Schema may not be specified in a Create Table As Select (CTAS)"))
val sqlText = "CREATE TABLE t (a int, b int) USING parquet AS SELECT 1, 2"
checkError(
exception = intercept[ParseException] {
sql(sqlText)
},
errorClass = "_LEGACY_ERROR_TEMP_0035",
parameters = Map(
"message" -> "Schema may not be specified in a Create Table As Select (CTAS) statement"),
context = ExpectedContext(
fragment = sqlText,
start = 0,
stop = 57))
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1134,10 +1134,16 @@ class InsertSuite extends DataSourceTest with SharedSparkSession {
withTable("t") {
sql("create table t(i boolean default true, s bigint, q int default 42) " +
"using parquet partitioned by (i)")
assert(intercept[ParseException] {
sql("insert into t partition(i=default) values(5, default)")
}.getMessage.contains(
"References to DEFAULT column values are not allowed within the PARTITION clause"))
checkError(
exception = intercept[ParseException] {
sql("insert into t partition(i=default) values(5, default)")
},
errorClass = "_LEGACY_ERROR_TEMP_0059",
parameters = Map.empty,
context = ExpectedContext(
fragment = "partition(i=default)",
start = 14,
stop = 33))
}
// The configuration option to append missing NULL values to the end of the INSERT INTO
// statement is not enabled.
Expand Down Expand Up @@ -1500,7 +1506,6 @@ class InsertSuite extends DataSourceTest with SharedSparkSession {
val BAD_SUBQUERY = "subquery expressions are not allowed in DEFAULT values"
}
val createTable = "create table t(i boolean, s bigint) using parquet"
val insertDefaults = "insert into t values (default, default)"
withTable("t") {
sql(createTable)
// The default value fails to analyze.
Expand All @@ -1522,9 +1527,17 @@ class InsertSuite extends DataSourceTest with SharedSparkSession {
}.getMessage.contains("provided a value of incompatible type"))
// The default value is disabled per configuration.
withSQLConf(SQLConf.ENABLE_DEFAULT_COLUMNS.key -> "false") {
assert(intercept[ParseException] {
sql("alter table t alter column s set default 41 + 1")
}.getMessage.contains("Support for DEFAULT column values is not allowed"))
val sqlText = "alter table t alter column s set default 41 + 1"
checkError(
exception = intercept[ParseException] {
sql(sqlText)
},
errorClass = "_LEGACY_ERROR_TEMP_0058",
parameters = Map.empty,
context = ExpectedContext(
fragment = sqlText,
start = 0,
stop = 46))
}
}
// Attempting to set a default value for a partitioning column is not allowed.
Expand Down Expand Up @@ -2061,10 +2074,16 @@ class InsertSuite extends DataSourceTest with SharedSparkSession {
}

test("SPARK-29174 fail LOCAL in INSERT OVERWRITE DIRECT remote path") {
val message = intercept[ParseException] {
sql("insert overwrite local directory 'hdfs:/abcd' using parquet select 1")
}.getMessage
assert(message.contains("LOCAL is supported only with file: scheme"))
checkError(
exception = intercept[ParseException] {
sql("insert overwrite local directory 'hdfs:/abcd' using parquet select 1")
},
errorClass = "_LEGACY_ERROR_TEMP_0050",
parameters = Map.empty,
context = ExpectedContext(
fragment = "insert overwrite local directory 'hdfs:/abcd' using parquet",
start = 0,
stop = 58))
}

test("SPARK-32508 " +
Expand Down