diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index 9aaf09c2d9d38..1ac8d4222a22a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -204,7 +204,7 @@ trait AnalysisTest extends PlanTest { protected def interceptParseException(parser: String => Any)( sqlCommand: String, messages: String*)( errorClass: Option[String] = None): Unit = { - val e = intercept[ParseException](parser(sqlCommand)) + val e = parseException(parser)(sqlCommand) messages.foreach { message => assert(e.message.contains(message)) } @@ -212,4 +212,9 @@ trait AnalysisTest extends PlanTest { assert(e.getErrorClass == errorClass.get) } } + + protected def parseException(parser: String => Any)( + sqlText: String): ParseException = { + intercept[ParseException](parser(sqlText)) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index a9376121b6878..7faf20063cfcb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.parser import java.util.Locale -import org.apache.spark.sql.AnalysisException +import org.apache.spark.SparkThrowable import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions.{EqualTo, Hex, Literal} import org.apache.spark.sql.catalyst.plans.logical.{TableSpec => LogicalTableSpec, _} @@ -34,22 +34,10 @@ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class DDLParserSuite extends AnalysisTest { import CatalystSqlParser._ - private def assertUnsupported(sql: String, containsThesePhrases: Seq[String] = Seq()): Unit = { - val e = intercept[ParseException] { - parsePlan(sql) - } - assert(e.getMessage.toLowerCase(Locale.ROOT).contains("operation not allowed")) - containsThesePhrases.foreach { p => - assert(e.getMessage.toLowerCase(Locale.ROOT).contains(p.toLowerCase(Locale.ROOT))) - } + private def parseException(sqlText: String): SparkThrowable = { + super.parseException(parsePlan)(sqlText) } - private def intercept(sqlCommand: String, messages: String*): Unit = - interceptParseException(parsePlan)(sqlCommand, messages: _*)() - - private def intercept(sqlCommand: String, errorClass: Option[String], messages: String*): Unit = - interceptParseException(parsePlan)(sqlCommand, messages: _*)(errorClass) - private def parseCompare(sql: String, expected: LogicalPlan): Unit = { comparePlans(parsePlan(sql), expected, checkAnalysis = false) } @@ -74,8 +62,11 @@ class DDLParserSuite extends AnalysisTest { testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } - intercept("CREATE TABLE my_tab(a: INT COMMENT 'test', b: STRING) USING parquet", - "Syntax error at or near ':': extra input ':'") + val sql = "CREATE TABLE my_tab(a: INT COMMENT 'test', b: STRING) USING parquet" + checkError( + exception = parseException(sql), + errorClass = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "':'", "hint" -> ": extra input ':'")) } test("create/replace table - with IF NOT EXISTS") { @@ -324,23 +315,53 @@ class DDLParserSuite extends AnalysisTest { test("create/replace table - mixed partition references and column definitions") { val createSql = "CREATE TABLE my_tab (id bigint, p1 string) PARTITIONED BY (p1, p2 string)" + val value1 = + """PARTITION BY: Cannot mix partition expressions and partition columns: + |Expressions: p1 + |Columns: p2 string""".stripMargin + checkError( + exception = parseException(createSql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> value1), + context = ExpectedContext( + fragment = createSql, + start = 0, + stop = 72)) + val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") - Seq(createSql, replaceSql).foreach { sql => - assertUnsupported(sql, Seq( - "PARTITION BY: Cannot mix partition expressions and partition columns", - "Expressions: p1", - "Columns: p2 string")) - } + checkError( + exception = parseException(replaceSql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> value1), + context = ExpectedContext( + fragment = replaceSql, + start = 0, + stop = 73)) val createSqlWithExpr = "CREATE TABLE my_tab (id bigint, p1 string) PARTITIONED BY (p2 string, truncate(p1, 16))" + val value2 = + """PARTITION BY: Cannot mix partition expressions and partition columns: + |Expressions: truncate(p1, 16) + |Columns: p2 string""".stripMargin + checkError( + exception = parseException(createSqlWithExpr), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> value2), + context = ExpectedContext( + fragment = createSqlWithExpr, + start = 0, + stop = 86)) + val replaceSqlWithExpr = createSqlWithExpr.replaceFirst("CREATE", "REPLACE") - Seq(createSqlWithExpr, replaceSqlWithExpr).foreach { sql => - assertUnsupported(sql, Seq( - "PARTITION BY: Cannot mix partition expressions and partition columns", - "Expressions: truncate(p1, 16)", - "Columns: p2 string")) - } + checkError( + exception = parseException(replaceSqlWithExpr), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> value2), + context = ExpectedContext( + fragment = replaceSqlWithExpr, + start = 0, + stop = 87)) } test("create/replace table - stored as") { @@ -397,12 +418,27 @@ class DDLParserSuite extends AnalysisTest { |PARTITIONED BY (part string) |STORED AS otherFormat |ROW FORMAT SERDE 'customSerde' - |WITH SERDEPROPERTIES ('prop'='value') - """.stripMargin + |WITH SERDEPROPERTIES ('prop'='value')""".stripMargin + val value = "ROW FORMAT SERDE is incompatible with format 'otherformat', " + + "which also specifies a serde" + checkError( + exception = parseException(createSql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> value), + context = ExpectedContext( + fragment = createSql, + start = 0, + stop = 150)) + val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") - Seq(createSql, replaceSql).foreach { sql => - assertUnsupported(sql, Seq("ROW FORMAT SERDE is incompatible with format 'otherFormat'")) - } + checkError( + exception = parseException(replaceSql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> value), + context = ExpectedContext( + fragment = replaceSql, + start = 0, + stop = 151)) } test("create/replace table - stored as format with delimited clauses") { @@ -439,13 +475,26 @@ class DDLParserSuite extends AnalysisTest { |PARTITIONED BY (part string) |STORED AS otherFormat |ROW FORMAT DELIMITED - |FIELDS TERMINATED BY ',' - """.stripMargin + |FIELDS TERMINATED BY ','""".stripMargin + val value = "ROW FORMAT DELIMITED is only compatible with 'textfile', not 'otherformat'" + checkError( + exception = parseException(createFailSql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> value), + context = ExpectedContext( + fragment = createFailSql, + start = 0, + stop = 127)) + val replaceFailSql = createFailSql.replaceFirst("CREATE", "REPLACE") - Seq(createFailSql, replaceFailSql).foreach { sql => - assertUnsupported(sql, Seq( - "ROW FORMAT DELIMITED is only compatible with 'textfile', not 'otherFormat'")) - } + checkError( + exception = parseException(replaceFailSql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> value), + context = ExpectedContext( + fragment = replaceFailSql, + start = 0, + stop = 128)) } test("create/replace table - stored as inputformat/outputformat") { @@ -499,54 +548,122 @@ class DDLParserSuite extends AnalysisTest { val createSql = """CREATE TABLE my_tab (id bigint, part string) |USING parquet - |STORED AS parquet - """.stripMargin - assertUnsupported(createSql, Seq("CREATE TABLE ... USING ... STORED AS")) + |STORED AS parquet""".stripMargin + checkError( + exception = parseException(createSql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "CREATE TABLE ... USING ... STORED AS parquet "), + context = ExpectedContext( + fragment = createSql, + start = 0, + stop = 75)) val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") - assertUnsupported(replaceSql, Seq("REPLACE TABLE ... USING ... STORED AS")) + checkError( + exception = parseException(replaceSql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "REPLACE TABLE ... USING ... STORED AS parquet "), + context = ExpectedContext( + fragment = replaceSql, + start = 0, + stop = 76)) } test("create/replace table - using with row format serde") { val createSql = """CREATE TABLE my_tab (id bigint, part string) |USING parquet - |ROW FORMAT SERDE 'customSerde' - """.stripMargin - assertUnsupported(createSql, Seq("CREATE TABLE ... USING ... ROW FORMAT SERDE")) + |ROW FORMAT SERDE 'customSerde'""".stripMargin + checkError( + exception = parseException(createSql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "CREATE TABLE ... USING ... ROW FORMAT SERDE customSerde"), + context = ExpectedContext( + fragment = createSql, + start = 0, + stop = 88)) val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") - assertUnsupported(replaceSql, Seq("REPLACE TABLE ... USING ... ROW FORMAT SERDE")) + checkError( + exception = parseException(replaceSql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "REPLACE TABLE ... USING ... ROW FORMAT SERDE customSerde"), + context = ExpectedContext( + fragment = replaceSql, + start = 0, + stop = 89)) } test("create/replace table - using with row format delimited") { val createSql = """CREATE TABLE my_tab (id bigint, part string) |USING parquet - |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' - """.stripMargin - assertUnsupported(createSql, Seq("CREATE TABLE ... USING ... ROW FORMAT DELIMITED")) + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ','""".stripMargin + checkError( + exception = parseException(createSql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "CREATE TABLE ... USING ... ROW FORMAT DELIMITED"), + context = ExpectedContext( + fragment = createSql, + start = 0, + stop = 103)) val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") - assertUnsupported(replaceSql, Seq("REPLACE TABLE ... USING ... ROW FORMAT DELIMITED")) + checkError( + exception = parseException(replaceSql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "REPLACE TABLE ... USING ... ROW FORMAT DELIMITED"), + context = ExpectedContext( + fragment = replaceSql, + start = 0, + stop = 104)) } test("create/replace table - stored by") { val createSql = """CREATE TABLE my_tab (id bigint, p1 string) - |STORED BY 'handler' - """.stripMargin + |STORED BY 'handler'""".stripMargin + val fragment = "STORED BY 'handler'" + checkError( + exception = parseException(createSql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "STORED BY"), + context = ExpectedContext( + fragment = fragment, + start = 43, + stop = 61)) + val replaceSql = createSql.replaceFirst("CREATE", "REPLACE") - Seq(createSql, replaceSql).foreach { sql => - assertUnsupported(sql, Seq("stored by")) - } + checkError( + exception = parseException(replaceSql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "STORED BY"), + context = ExpectedContext( + fragment = fragment, + start = 44, + stop = 62)) } test("Unsupported skew clause - create/replace table") { - intercept("CREATE TABLE my_tab (id bigint) SKEWED BY (id) ON (1,2,3)", - "CREATE TABLE ... SKEWED BY") - intercept("REPLACE TABLE my_tab (id bigint) SKEWED BY (id) ON (1,2,3)", - "CREATE TABLE ... SKEWED BY") + val sql1 = "CREATE TABLE my_tab (id bigint) SKEWED BY (id) ON (1,2,3)" + checkError( + exception = parseException(sql1), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "CREATE TABLE ... SKEWED BY"), + context = ExpectedContext( + fragment = sql1, + start = 0, + stop = 56)) + + val sql2 = "REPLACE TABLE my_tab (id bigint) SKEWED BY (id) ON (1,2,3)" + checkError( + exception = parseException(sql2), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "CREATE TABLE ... SKEWED BY"), + context = ExpectedContext( + fragment = sql2, + start = 0, + stop = 57)) } test("Duplicate clauses - create/replace table") { @@ -555,46 +672,188 @@ class DDLParserSuite extends AnalysisTest { } def replaceTableHeader(duplicateClause: String): String = { - s"CREATE TABLE my_tab(a INT, b STRING) $duplicateClause $duplicateClause" + s"REPLACE TABLE my_tab(a INT, b STRING) $duplicateClause $duplicateClause" } - intercept(createTableHeader("TBLPROPERTIES('test' = 'test2')"), - "Found duplicate clauses: TBLPROPERTIES") - intercept(createTableHeader("LOCATION '/tmp/file'"), - "Found duplicate clauses: LOCATION") - intercept(createTableHeader("COMMENT 'a table'"), - "Found duplicate clauses: COMMENT") - intercept(createTableHeader("CLUSTERED BY(b) INTO 256 BUCKETS"), - "Found duplicate clauses: CLUSTERED BY") - intercept(createTableHeader("PARTITIONED BY (b)"), - "Found duplicate clauses: PARTITIONED BY") - intercept(createTableHeader("PARTITIONED BY (c int)"), - "Found duplicate clauses: PARTITIONED BY") - intercept(createTableHeader("STORED AS parquet"), - "Found duplicate clauses: STORED AS") - intercept(createTableHeader("STORED AS INPUTFORMAT 'in' OUTPUTFORMAT 'out'"), - "Found duplicate clauses: STORED AS") - intercept(createTableHeader("ROW FORMAT SERDE 'serde'"), - "Found duplicate clauses: ROW FORMAT") - - intercept(replaceTableHeader("TBLPROPERTIES('test' = 'test2')"), - "Found duplicate clauses: TBLPROPERTIES") - intercept(replaceTableHeader("LOCATION '/tmp/file'"), - "Found duplicate clauses: LOCATION") - intercept(replaceTableHeader("COMMENT 'a table'"), - "Found duplicate clauses: COMMENT") - intercept(replaceTableHeader("CLUSTERED BY(b) INTO 256 BUCKETS"), - "Found duplicate clauses: CLUSTERED BY") - intercept(replaceTableHeader("PARTITIONED BY (b)"), - "Found duplicate clauses: PARTITIONED BY") - intercept(replaceTableHeader("PARTITIONED BY (c int)"), - "Found duplicate clauses: PARTITIONED BY") - intercept(replaceTableHeader("STORED AS parquet"), - "Found duplicate clauses: STORED AS") - intercept(replaceTableHeader("STORED AS INPUTFORMAT 'in' OUTPUTFORMAT 'out'"), - "Found duplicate clauses: STORED AS") - intercept(replaceTableHeader("ROW FORMAT SERDE 'serde'"), - "Found duplicate clauses: ROW FORMAT") + val sql1 = createTableHeader("TBLPROPERTIES('test' = 'test2')") + checkError( + exception = parseException(sql1), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "TBLPROPERTIES"), + context = ExpectedContext( + fragment = sql1, + start = 0, + stop = 99)) + + val sql2 = createTableHeader("LOCATION '/tmp/file'") + checkError( + exception = parseException(sql2), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "LOCATION"), + context = ExpectedContext( + fragment = sql2, + start = 0, + stop = 77)) + + val sql3 = createTableHeader("COMMENT 'a table'") + checkError( + exception = parseException(sql3), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "COMMENT"), + context = ExpectedContext( + fragment = sql3, + start = 0, + stop = 71)) + + val sql4 = createTableHeader("CLUSTERED BY(b) INTO 256 BUCKETS") + checkError( + exception = parseException(sql4), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "CLUSTERED BY"), + context = ExpectedContext( + fragment = sql4, + start = 0, + stop = 101)) + + val sql5 = createTableHeader("PARTITIONED BY (b)") + checkError( + exception = parseException(sql5), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "PARTITIONED BY"), + context = ExpectedContext( + fragment = sql5, + start = 0, + stop = 73)) + + val sql6 = createTableHeader("PARTITIONED BY (c int)") + checkError( + exception = parseException(sql6), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "PARTITIONED BY"), + context = ExpectedContext( + fragment = sql6, + start = 0, + stop = 81)) + + val sql7 = createTableHeader("STORED AS parquet") + checkError( + exception = parseException(sql7), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "STORED AS/BY"), + context = ExpectedContext( + fragment = sql7, + start = 0, + stop = 71)) + + val sql8 = createTableHeader("STORED AS INPUTFORMAT 'in' OUTPUTFORMAT 'out'") + checkError( + exception = parseException(sql8), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "STORED AS/BY"), + context = ExpectedContext( + fragment = sql8, + start = 0, + stop = 127)) + + val sql9 = createTableHeader("ROW FORMAT SERDE 'serde'") + checkError( + exception = parseException(sql9), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "ROW FORMAT"), + context = ExpectedContext( + fragment = sql9, + start = 0, + stop = 85)) + + val sql10 = replaceTableHeader("TBLPROPERTIES('test' = 'test2')") + checkError( + exception = parseException(sql10), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "TBLPROPERTIES"), + context = ExpectedContext( + fragment = sql10, + start = 0, + stop = 100)) + + val sql11 = replaceTableHeader("LOCATION '/tmp/file'") + checkError( + exception = parseException(sql11), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "LOCATION"), + context = ExpectedContext( + fragment = sql11, + start = 0, + stop = 78)) + + val sql12 = replaceTableHeader("COMMENT 'a table'") + checkError( + exception = parseException(sql12), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "COMMENT"), + context = ExpectedContext( + fragment = sql12, + start = 0, + stop = 72)) + + val sql13 = replaceTableHeader("CLUSTERED BY(b) INTO 256 BUCKETS") + checkError( + exception = parseException(sql13), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "CLUSTERED BY"), + context = ExpectedContext( + fragment = sql13, + start = 0, + stop = 102)) + + val sql14 = replaceTableHeader("PARTITIONED BY (b)") + checkError( + exception = parseException(sql14), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "PARTITIONED BY"), + context = ExpectedContext( + fragment = sql14, + start = 0, + stop = 74)) + + val sql15 = replaceTableHeader("PARTITIONED BY (c int)") + checkError( + exception = parseException(sql15), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "PARTITIONED BY"), + context = ExpectedContext( + fragment = sql15, + start = 0, + stop = 82)) + + val sql16 = replaceTableHeader("STORED AS parquet") + checkError( + exception = parseException(sql16), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "STORED AS/BY"), + context = ExpectedContext( + fragment = sql16, + start = 0, + stop = 72)) + + val sql17 = replaceTableHeader("STORED AS INPUTFORMAT 'in' OUTPUTFORMAT 'out'") + checkError( + exception = parseException(sql17), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "STORED AS/BY"), + context = ExpectedContext( + fragment = sql17, + start = 0, + stop = 128)) + + val sql18 = replaceTableHeader("ROW FORMAT SERDE 'serde'") + checkError( + exception = parseException(sql18), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "ROW FORMAT"), + context = ExpectedContext( + fragment = sql18, + start = 0, + stop = 86)) } test("support for other types in OPTIONS") { @@ -923,10 +1182,16 @@ class DDLParserSuite extends AnalysisTest { } test("alter table: update column type invalid type") { - val msg = intercept[ParseException] { - parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c TYPE bad_type") - }.getMessage - assert(msg.contains("DataType bad_type is not supported")) + val sql = "ALTER TABLE table_name ALTER COLUMN a.b.c TYPE bad_type" + val fragment = "bad_type" + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0030", + parameters = Map("dataType" -> "bad_type"), + context = ExpectedContext( + fragment = fragment, + start = 47, + stop = 54)) } test("alter table: update column type") { @@ -969,17 +1234,23 @@ class DDLParserSuite extends AnalysisTest { } test("alter table: multiple property changes are not allowed") { - intercept[ParseException] { - parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c " + - "TYPE bigint COMMENT 'new comment'")} + val sql1 = "ALTER TABLE table_name ALTER COLUMN a.b.c TYPE bigint COMMENT 'new comment'" + checkError( + exception = parseException(sql1), + errorClass = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'COMMENT'", "hint" -> "")) - intercept[ParseException] { - parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c " + - "TYPE bigint COMMENT AFTER d")} + val sql2 = "ALTER TABLE table_name ALTER COLUMN a.b.c TYPE bigint COMMENT AFTER d" + checkError( + exception = parseException(sql2), + errorClass = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'COMMENT'", "hint" -> "")) - intercept[ParseException] { - parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c " + - "TYPE bigint COMMENT 'new comment' AFTER d")} + val sql3 = "ALTER TABLE table_name ALTER COLUMN a.b.c TYPE bigint COMMENT 'new comment' AFTER d" + checkError( + exception = parseException(sql3), + errorClass = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'COMMENT'", "hint" -> "")) } test("alter table: SET/DROP NOT NULL") { @@ -1087,11 +1358,29 @@ class DDLParserSuite extends AnalysisTest { None)) // renaming column not supported in hive style ALTER COLUMN. - intercept("ALTER TABLE table_name CHANGE COLUMN a.b.c new_name INT", - "please run RENAME COLUMN instead") + val sql4 = "ALTER TABLE table_name CHANGE COLUMN a.b.c new_name INT" + checkError( + exception = parseException(sql4), + errorClass = "_LEGACY_ERROR_TEMP_0034", + parameters = Map( + "operation" -> "Renaming column", + "command" -> "ALTER COLUMN", + "msg" -> ", please run RENAME COLUMN instead"), + context = ExpectedContext( + fragment = sql4, + start = 0, + stop = 54)) // ALTER COLUMN for a partition is not supported. - intercept("ALTER TABLE table_name PARTITION (a='1') CHANGE COLUMN a.b.c c INT") + val sql5 = "ALTER TABLE table_name PARTITION (a='1') CHANGE COLUMN a.b.c c INT" + checkError( + exception = parseException(sql5), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE table PARTITION partition_spec CHANGE COLUMN"), + context = ExpectedContext( + fragment = sql5, + start = 0, + stop = 65)) } test("alter table: hive style replace columns") { @@ -1130,17 +1419,51 @@ class DDLParserSuite extends AnalysisTest { QualifiedColType(None, "y", IntegerType, true, Some("y1"), None, None) ))) - intercept("ALTER TABLE table_name PARTITION (a='1') REPLACE COLUMNS (x string)", - "Operation not allowed: ALTER TABLE table PARTITION partition_spec REPLACE COLUMNS") - - intercept("ALTER TABLE table_name REPLACE COLUMNS (x string NOT NULL)", - "NOT NULL is not supported in Hive-style REPLACE COLUMNS") - - intercept("ALTER TABLE table_name REPLACE COLUMNS (x string FIRST)", - "Column position is not supported in Hive-style REPLACE COLUMNS") - - intercept("ALTER TABLE table_name REPLACE COLUMNS (a.b.c string)", - "Replacing with a nested column is not supported in Hive-style REPLACE COLUMNS") + val sql5 = "ALTER TABLE table_name PARTITION (a='1') REPLACE COLUMNS (x string)" + checkError( + exception = parseException(sql5), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE table PARTITION partition_spec REPLACE COLUMNS"), + context = ExpectedContext( + fragment = sql5, + start = 0, + stop = 66)) + + val sql6 = "ALTER TABLE table_name REPLACE COLUMNS (x string NOT NULL)" + checkError( + exception = parseException(sql6), + errorClass = "_LEGACY_ERROR_TEMP_0034", + parameters = Map("operation" -> "NOT NULL", "command" -> "REPLACE COLUMNS", "msg" -> ""), + context = ExpectedContext( + fragment = sql6, + start = 0, + stop = 57)) + + val sql7 = "ALTER TABLE table_name REPLACE COLUMNS (x string FIRST)" + checkError( + exception = parseException(sql7), + errorClass = "_LEGACY_ERROR_TEMP_0034", + parameters = Map( + "operation" -> "Column position", + "command" -> "REPLACE COLUMNS", + "msg" -> ""), + context = ExpectedContext( + fragment = sql7, + start = 0, + stop = 54)) + + val sql8 = "ALTER TABLE table_name REPLACE COLUMNS (a.b.c string)" + checkError( + exception = parseException(sql8), + errorClass = "_LEGACY_ERROR_TEMP_0034", + parameters = Map( + "operation" -> "Replacing with a nested column", + "command" -> "REPLACE COLUMNS", + "msg" -> ""), + context = ExpectedContext( + fragment = sql8, + start = 0, + stop = 52)) } test("alter view: rename view") { @@ -1298,30 +1621,39 @@ class DDLParserSuite extends AnalysisTest { } test("insert table: if not exists with dynamic partition fails") { - val exc = intercept[AnalysisException] { - parsePlan( - """ - |INSERT OVERWRITE TABLE testcat.ns1.ns2.tbl - |PARTITION (p1 = 3, p2) IF NOT EXISTS - |SELECT * FROM source - """.stripMargin) - } - - assert(exc.getMessage.contains("IF NOT EXISTS with dynamic partitions")) - assert(exc.getMessage.contains("p2")) + val sql = + """INSERT OVERWRITE TABLE testcat.ns1.ns2.tbl + |PARTITION (p1 = 3, p2) IF NOT EXISTS + |SELECT * FROM source""".stripMargin + val fragment = + """INSERT OVERWRITE TABLE testcat.ns1.ns2.tbl + |PARTITION (p1 = 3, p2) IF NOT EXISTS""".stripMargin + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "IF NOT EXISTS with dynamic partitions: p2"), + context = ExpectedContext( + fragment = fragment, + start = 0, + stop = 78)) } test("insert table: if not exists without overwrite fails") { - val exc = intercept[AnalysisException] { - parsePlan( - """ - |INSERT INTO TABLE testcat.ns1.ns2.tbl - |PARTITION (p1 = 3) IF NOT EXISTS - |SELECT * FROM source - """.stripMargin) - } - - assert(exc.getMessage.contains("INSERT INTO ... IF NOT EXISTS")) + val sql = + """INSERT INTO TABLE testcat.ns1.ns2.tbl + |PARTITION (p1 = 3) IF NOT EXISTS + |SELECT * FROM source""".stripMargin + val fragment = + """INSERT INTO TABLE testcat.ns1.ns2.tbl + |PARTITION (p1 = 3) IF NOT EXISTS""".stripMargin + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "INSERT INTO ... IF NOT EXISTS"), + context = ExpectedContext( + fragment = fragment, + start = 0, + stop = 69)) } test("delete from table: delete all") { @@ -1339,11 +1671,15 @@ class DDLParserSuite extends AnalysisTest { } test("delete from table: columns aliases is not allowed") { - val exc = intercept[ParseException] { - parsePlan("DELETE FROM testcat.ns1.ns2.tbl AS t(a,b,c,d) WHERE d = 2") - } - - assert(exc.getMessage.contains("Columns aliases are not allowed in DELETE.")) + val sql = "DELETE FROM testcat.ns1.ns2.tbl AS t(a,b,c,d) WHERE d = 2" + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0003", + parameters = Map("op" -> "DELETE"), + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 56)) } test("update table: basic") { @@ -1374,16 +1710,18 @@ class DDLParserSuite extends AnalysisTest { } test("update table: columns aliases is not allowed") { - val exc = intercept[ParseException] { - parsePlan( - """ - |UPDATE testcat.ns1.ns2.tbl AS t(a,b,c,d) - |SET b='Robert', c=32 - |WHERE d=2 - """.stripMargin) - } - - assert(exc.getMessage.contains("Columns aliases are not allowed in UPDATE.")) + val sql = + """UPDATE testcat.ns1.ns2.tbl AS t(a,b,c,d) + |SET b='Robert', c=32 + |WHERE d=2""".stripMargin + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0003", + parameters = Map("op" -> "UPDATE"), + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 70)) } test("merge into table: basic") { @@ -1506,20 +1844,22 @@ class DDLParserSuite extends AnalysisTest { test("merge into table: columns aliases are not allowed") { Seq("target(c1, c2)" -> "source", "target" -> "source(c1, c2)").foreach { case (targetAlias, sourceAlias) => - val exc = intercept[ParseException] { - parsePlan( - s""" - |MERGE INTO testcat1.ns1.ns2.tbl AS $targetAlias - |USING testcat2.ns1.ns2.tbl AS $sourceAlias - |ON target.col1 = source.col1 - |WHEN MATCHED AND (target.col2='delete') THEN DELETE - |WHEN MATCHED AND (target.col2='update') THEN UPDATE SET target.col2 = source.col2 - |WHEN NOT MATCHED AND (target.col2='insert') - |THEN INSERT (target.col1, target.col2) values (source.col1, source.col2) - """.stripMargin) - } - - assert(exc.getMessage.contains("Columns aliases are not allowed in MERGE.")) + val sql = s"""MERGE INTO testcat1.ns1.ns2.tbl AS $targetAlias + |USING testcat2.ns1.ns2.tbl AS $sourceAlias + |ON target.col1 = source.col1 + |WHEN MATCHED AND (target.col2='delete') THEN DELETE + |WHEN MATCHED AND (target.col2='update') THEN UPDATE SET target.col2 = source.col2 + |WHEN NOT MATCHED AND (target.col2='insert') + |THEN INSERT (target.col1, target.col2) values (source.col1, source.col2)""" + .stripMargin + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0003", + parameters = Map("op" -> "MERGE"), + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 365)) } } @@ -1555,55 +1895,61 @@ class DDLParserSuite extends AnalysisTest { } test("merge into table: only the last matched clause can omit the condition") { - val exc = intercept[ParseException] { - parsePlan( - """ - |MERGE INTO testcat1.ns1.ns2.tbl AS target - |USING testcat2.ns1.ns2.tbl AS source - |ON target.col1 = source.col1 - |WHEN MATCHED AND (target.col2 == 'update1') THEN UPDATE SET target.col2 = 1 - |WHEN MATCHED THEN UPDATE SET target.col2 = 2 - |WHEN MATCHED THEN DELETE - |WHEN NOT MATCHED AND (target.col2='insert') - |THEN INSERT (target.col1, target.col2) values (source.col1, source.col2) - """.stripMargin) - } - - assert(exc.getMessage.contains("only the last MATCHED clause can omit the condition")) + val sql = + """MERGE INTO testcat1.ns1.ns2.tbl AS target + |USING testcat2.ns1.ns2.tbl AS source + |ON target.col1 = source.col1 + |WHEN MATCHED AND (target.col2 == 'update1') THEN UPDATE SET target.col2 = 1 + |WHEN MATCHED THEN UPDATE SET target.col2 = 2 + |WHEN MATCHED THEN DELETE + |WHEN NOT MATCHED AND (target.col2='insert') + |THEN INSERT (target.col1, target.col2) values (source.col1, source.col2)""".stripMargin + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0009", + parameters = Map.empty, + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 369)) } test("merge into table: only the last not matched clause can omit the condition") { - val exc = intercept[ParseException] { - parsePlan( - """ - |MERGE INTO testcat1.ns1.ns2.tbl AS target - |USING testcat2.ns1.ns2.tbl AS source - |ON target.col1 = source.col1 - |WHEN MATCHED AND (target.col2 == 'update') THEN UPDATE SET target.col2 = source.col2 - |WHEN MATCHED THEN DELETE - |WHEN NOT MATCHED AND (target.col2='insert1') - |THEN INSERT (target.col1, target.col2) values (source.col1, 1) - |WHEN NOT MATCHED - |THEN INSERT (target.col1, target.col2) values (source.col1, 2) - |WHEN NOT MATCHED - |THEN INSERT (target.col1, target.col2) values (source.col1, source.col2) - """.stripMargin) - } - - assert(exc.getMessage.contains("only the last NOT MATCHED clause can omit the condition")) + val sql = + """MERGE INTO testcat1.ns1.ns2.tbl AS target + |USING testcat2.ns1.ns2.tbl AS source + |ON target.col1 = source.col1 + |WHEN MATCHED AND (target.col2 == 'update') THEN UPDATE SET target.col2 = source.col2 + |WHEN MATCHED THEN DELETE + |WHEN NOT MATCHED AND (target.col2='insert1') + |THEN INSERT (target.col1, target.col2) values (source.col1, 1) + |WHEN NOT MATCHED + |THEN INSERT (target.col1, target.col2) values (source.col1, 2) + |WHEN NOT MATCHED + |THEN INSERT (target.col1, target.col2) values (source.col1, source.col2)""".stripMargin + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0010", + parameters = Map.empty, + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 494)) } test("merge into table: there must be a when (not) matched condition") { - val exc = intercept[ParseException] { - parsePlan( - """ - |MERGE INTO testcat1.ns1.ns2.tbl AS target - |USING testcat2.ns1.ns2.tbl AS source - |ON target.col1 = source.col1 - """.stripMargin) - } - - assert(exc.getMessage.contains("There must be at least one WHEN clause in a MERGE statement")) + val sql = + """MERGE INTO testcat1.ns1.ns2.tbl AS target + |USING testcat2.ns1.ns2.tbl AS source + |ON target.col1 = source.col1""".stripMargin + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0008", + parameters = Map.empty, + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 106)) } test("show views") { @@ -1686,10 +2032,25 @@ class DDLParserSuite extends AnalysisTest { UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false), Map("ds" -> None, "hr" -> None), noScan = true)) - intercept("analyze table a.b.c compute statistics xxxx", - "Expected `NOSCAN` instead of `xxxx`") - intercept("analyze table a.b.c partition (a) compute statistics xxxx", - "Expected `NOSCAN` instead of `xxxx`") + val sql1 = "analyze table a.b.c compute statistics xxxx" + checkError( + exception = parseException(sql1), + errorClass = "_LEGACY_ERROR_TEMP_0036", + parameters = Map("ctx" -> "xxxx"), + context = ExpectedContext( + fragment = sql1, + start = 0, + stop = 42)) + + val sql2 = "analyze table a.b.c partition (a) compute statistics xxxx" + checkError( + exception = parseException(sql2), + errorClass = "_LEGACY_ERROR_TEMP_0036", + parameters = Map("ctx" -> "xxxx"), + context = ExpectedContext( + fragment = sql2, + start = 0, + stop = 56)) } test("SPARK-33687: analyze tables statistics") { @@ -1697,12 +2058,24 @@ class DDLParserSuite extends AnalysisTest { AnalyzeTables(UnresolvedNamespace(Seq("a", "b", "c")), noScan = false)) comparePlans(parsePlan("ANALYZE TABLES FROM a COMPUTE STATISTICS NOSCAN"), AnalyzeTables(UnresolvedNamespace(Seq("a")), noScan = true)) - intercept("ANALYZE TABLES IN a.b.c COMPUTE STATISTICS xxxx", - "Expected `NOSCAN` instead of `xxxx`") + + val sql = "ANALYZE TABLES IN a.b.c COMPUTE STATISTICS xxxx" + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0036", + parameters = Map("ctx" -> "xxxx"), + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 46)) } test("analyze table column statistics") { - intercept("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR COLUMNS", "") + val sql1 = "ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR COLUMNS" + checkError( + exception = parseException(sql1), + errorClass = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "end of input", "hint" -> "")) comparePlans( parsePlan("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR COLUMNS key, value"), @@ -1735,10 +2108,17 @@ class DDLParserSuite extends AnalysisTest { None, allColumns = true)) - intercept("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL COLUMNS key, value", - Some("PARSE_SYNTAX_ERROR"), "Syntax error at or near 'key'") // expecting {, ';'} - intercept("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL", - "Syntax error at or near end of input: missing 'COLUMNS'") + val sql2 = "ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL COLUMNS key, value" + checkError( + exception = parseException(sql2), + errorClass = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'key'", "hint" -> "")) // expecting {, ';'} + + val sql3 = "ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL" + checkError( + exception = parseException(sql3), + errorClass = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "end of input", "hint" -> ": missing 'COLUMNS'")) } test("LOAD DATA INTO table") { @@ -1811,8 +2191,15 @@ class DDLParserSuite extends AnalysisTest { true, Map("storageLevel" -> "DISK_ONLY"))) - intercept("CACHE TABLE a.b.c AS SELECT * FROM testData", - "It is not allowed to add catalog/namespace prefix a.b") + val sql = "CACHE TABLE a.b.c AS SELECT * FROM testData" + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0037", + parameters = Map("quoted" -> "a.b"), + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 42)) } test("UNCACHE TABLE") { @@ -1855,12 +2242,18 @@ class DDLParserSuite extends AnalysisTest { } test("alter view: add partition (not supported)") { - assertUnsupported( - """ - |ALTER VIEW a.b.c ADD IF NOT EXISTS PARTITION + val sql = + """ALTER VIEW a.b.c ADD IF NOT EXISTS PARTITION |(dt='2008-08-08', country='us') PARTITION - |(dt='2009-09-09', country='uk') - """.stripMargin) + |(dt='2009-09-09', country='uk')""".stripMargin + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER VIEW ... ADD PARTITION"), + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 117)) } test("alter view: AS Query") { @@ -2108,12 +2501,19 @@ class DDLParserSuite extends AnalysisTest { Some("''"))) // It is not possible to pass an empty string default value without using quotes. // This results in a parsing error. - intercept("ALTER TABLE t1 ALTER COLUMN a.b.c SET DEFAULT ", - "Syntax error at or near end of input") + val sql1 = "ALTER TABLE t1 ALTER COLUMN a.b.c SET DEFAULT " + checkError( + exception = parseException(sql1), + errorClass = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "end of input", "hint" -> "")) // It is not possible to both SET DEFAULT and DROP DEFAULT at the same time. // This results in a parsing error. - intercept("ALTER TABLE t1 ALTER COLUMN a.b.c DROP DEFAULT SET DEFAULT 42", - "Syntax error at or near 'SET'") + val sql2 = "ALTER TABLE t1 ALTER COLUMN a.b.c DROP DEFAULT SET DEFAULT 42" + checkError( + exception = parseException(sql2), + errorClass = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'SET'", "hint" -> "")) + comparePlans( parsePlan("ALTER TABLE t1 ALTER COLUMN a.b.c DROP DEFAULT"), AlterColumn( @@ -2126,9 +2526,16 @@ class DDLParserSuite extends AnalysisTest { Some(""))) // Make sure that the parser returns an exception when the feature is disabled. withSQLConf(SQLConf.ENABLE_DEFAULT_COLUMNS.key -> "false") { - intercept( - "CREATE TABLE my_tab(a INT, b STRING NOT NULL DEFAULT \"abc\") USING parquet", - "Support for DEFAULT column values is not allowed") + val sql = "CREATE TABLE my_tab(a INT, b STRING NOT NULL DEFAULT \"abc\") USING parquet" + val fragment = "b STRING NOT NULL DEFAULT \"abc\"" + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0058", + parameters = Map.empty, + context = ExpectedContext( + fragment = fragment, + start = 27, + stop = 57)) } // In each of the following cases, the DEFAULT reference parses as an unresolved attribute diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 6774995867107..e1316c5b132e6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -59,7 +59,7 @@ class ExpressionParserSuite extends AnalysisTest { } private def parseException(sqlText: String): SparkThrowable = { - intercept[ParseException](defaultParser.parseExpression(sqlText)) + super.parseException(defaultParser.parseExpression)(sqlText) } def assertEval( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 8ef2cf9887c14..5e73a994d1d06 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -17,16 +17,13 @@ package org.apache.spark.sql.execution.command -import java.util.Locale - -import org.apache.spark.sql.AnalysisException +import org.apache.spark.SparkThrowable import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, GlobalTempView, LocalTempView, UnresolvedAttribute, UnresolvedFunc, UnresolvedIdentifier} import org.apache.spark.sql.catalyst.catalog.{ArchiveResource, FileResource, FunctionResource, JarResource} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans import org.apache.spark.sql.catalyst.dsl.plans.DslLogicalPlan import org.apache.spark.sql.catalyst.expressions.JsonTuple -import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.test.SharedSparkSession @@ -35,19 +32,10 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ private lazy val parser = new SparkSqlParser() - private def assertUnsupported(sql: String, containsThesePhrases: Seq[String] = Seq()): Unit = { - val e = intercept[ParseException] { - parser.parsePlan(sql) - } - assert(e.getMessage.toLowerCase(Locale.ROOT).contains("operation not allowed")) - containsThesePhrases.foreach { p => - assert(e.getMessage.toLowerCase(Locale.ROOT).contains(p.toLowerCase(Locale.ROOT))) - } + private def parseException(sqlText: String): SparkThrowable = { + super.parseException(parser.parsePlan)(sqlText) } - private def intercept(sqlCommand: String, messages: String*): Unit = - interceptParseException(parser.parsePlan)(sqlCommand, messages: _*)() - private def compareTransformQuery(sql: String, expected: LogicalPlan): Unit = { val plan = parser.parsePlan(sql).asInstanceOf[ScriptTransformation].copy(ioschema = null) comparePlans(plan, expected, checkAnalysis = false) @@ -70,11 +58,14 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { } val v2 = "INSERT OVERWRITE DIRECTORY USING parquet SELECT 1 as a" - val e2 = intercept[ParseException] { - parser.parsePlan(v2) - } - assert(e2.message.contains( - "Directory path and 'path' in OPTIONS should be specified one, but not both")) + checkError( + exception = parseException(v2), + errorClass = "_LEGACY_ERROR_TEMP_0049", + parameters = Map.empty, + context = ExpectedContext( + fragment = "INSERT OVERWRITE DIRECTORY USING parquet", + start = 0, + stop = 39)) val v3 = """ @@ -96,95 +87,271 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { } val v4 = - """ - | INSERT OVERWRITE DIRECTORY '/tmp/file' USING json + """INSERT OVERWRITE DIRECTORY '/tmp/file' USING json | OPTIONS ('path' '/tmp/file', a 1, b 0.1, c TRUE) - | SELECT 1 as a - """.stripMargin - val e4 = intercept[ParseException] { - parser.parsePlan(v4) - } - assert(e4.message.contains( - "Directory path and 'path' in OPTIONS should be specified one, but not both")) + | SELECT 1 as a""".stripMargin + val fragment4 = + """INSERT OVERWRITE DIRECTORY '/tmp/file' USING json + | OPTIONS ('path' '/tmp/file', a 1, b 0.1, c TRUE)""".stripMargin + checkError( + exception = parseException(v4), + errorClass = "_LEGACY_ERROR_TEMP_0049", + parameters = Map.empty, + context = ExpectedContext( + fragment = fragment4, + start = 0, + stop = 98)) } test("alter table - property values must be set") { - assertUnsupported( - sql = "ALTER TABLE my_tab SET TBLPROPERTIES('key_without_value', 'key_with_value'='x')", - containsThesePhrases = Seq("key_without_value")) + val sql = "ALTER TABLE my_tab SET TBLPROPERTIES('key_without_value', 'key_with_value'='x')" + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "Values must be specified for key(s): [key_without_value]"), + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 78)) } test("alter table unset properties - property values must NOT be set") { - assertUnsupported( - sql = "ALTER TABLE my_tab UNSET TBLPROPERTIES('key_without_value', 'key_with_value'='x')", - containsThesePhrases = Seq("key_with_value")) + val sql = "ALTER TABLE my_tab UNSET TBLPROPERTIES('key_without_value', 'key_with_value'='x')" + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "Values should not be specified for key(s): [key_with_value]"), + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 80)) } test("alter table: exchange partition (not supported)") { - assertUnsupported( - """ - |ALTER TABLE table_name_1 EXCHANGE PARTITION - |(dt='2008-08-08', country='us') WITH TABLE table_name_2 - """.stripMargin) + val sql = + """ALTER TABLE table_name_1 EXCHANGE PARTITION + |(dt='2008-08-08', country='us') WITH TABLE table_name_2""".stripMargin + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE EXCHANGE PARTITION"), + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 98)) } test("alter table: archive partition (not supported)") { - assertUnsupported("ALTER TABLE table_name ARCHIVE PARTITION (dt='2008-08-08', country='us')") + val sql = "ALTER TABLE table_name ARCHIVE PARTITION (dt='2008-08-08', country='us')" + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE ARCHIVE PARTITION"), + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 71)) } test("alter table: unarchive partition (not supported)") { - assertUnsupported("ALTER TABLE table_name UNARCHIVE PARTITION (dt='2008-08-08', country='us')") + val sql = "ALTER TABLE table_name UNARCHIVE PARTITION (dt='2008-08-08', country='us')" + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE UNARCHIVE PARTITION"), + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 73)) } test("alter table: set file format (not allowed)") { - assertUnsupported( - "ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' OUTPUTFORMAT 'test'") - assertUnsupported( - "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + - "SET FILEFORMAT PARQUET") + val sql1 = "ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' OUTPUTFORMAT 'test'" + checkError( + exception = parseException(sql1), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE SET FILEFORMAT"), + context = ExpectedContext( + fragment = sql1, + start = 0, + stop = 75)) + + val sql2 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + + "SET FILEFORMAT PARQUET" + checkError( + exception = parseException(sql2), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE SET FILEFORMAT"), + context = ExpectedContext( + fragment = sql2, + start = 0, + stop = 86)) } test("alter table: touch (not supported)") { - assertUnsupported("ALTER TABLE table_name TOUCH") - assertUnsupported("ALTER TABLE table_name TOUCH PARTITION (dt='2008-08-08', country='us')") + val sql1 = "ALTER TABLE table_name TOUCH" + checkError( + exception = parseException(sql1), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE TOUCH"), + context = ExpectedContext( + fragment = sql1, + start = 0, + stop = 27)) + + val sql2 = "ALTER TABLE table_name TOUCH PARTITION (dt='2008-08-08', country='us')" + checkError( + exception = parseException(sql2), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE TOUCH"), + context = ExpectedContext( + fragment = sql2, + start = 0, + stop = 69)) } test("alter table: compact (not supported)") { - assertUnsupported("ALTER TABLE table_name COMPACT 'compaction_type'") - assertUnsupported( - """ - |ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') - |COMPACT 'MAJOR' - """.stripMargin) + val sql1 = "ALTER TABLE table_name COMPACT 'compaction_type'" + checkError( + exception = parseException(sql1), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE COMPACT"), + context = ExpectedContext( + fragment = sql1, + start = 0, + stop = 47)) + + val sql2 = + """ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') + |COMPACT 'MAJOR'""".stripMargin + checkError( + exception = parseException(sql2), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE COMPACT"), + context = ExpectedContext( + fragment = sql2, + start = 0, + stop = 79)) } test("alter table: concatenate (not supported)") { - assertUnsupported("ALTER TABLE table_name CONCATENATE") - assertUnsupported( - "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') CONCATENATE") + val sql1 = "ALTER TABLE table_name CONCATENATE" + checkError( + exception = parseException(sql1), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE CONCATENATE"), + context = ExpectedContext( + fragment = sql1, + start = 0, + stop = 33)) + + val sql2 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') CONCATENATE" + checkError( + exception = parseException(sql2), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE CONCATENATE"), + context = ExpectedContext( + fragment = sql2, + start = 0, + stop = 75)) } test("alter table: cluster by (not supported)") { - assertUnsupported( - "ALTER TABLE table_name CLUSTERED BY (col_name) SORTED BY (col2_name) INTO 3 BUCKETS") - assertUnsupported("ALTER TABLE table_name CLUSTERED BY (col_name) INTO 3 BUCKETS") - assertUnsupported("ALTER TABLE table_name NOT CLUSTERED") - assertUnsupported("ALTER TABLE table_name NOT SORTED") + val sql1 = "ALTER TABLE table_name CLUSTERED BY (col_name) SORTED BY (col2_name) INTO 3 BUCKETS" + checkError( + exception = parseException(sql1), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE CLUSTERED BY"), + context = ExpectedContext( + fragment = sql1, + start = 0, + stop = 82)) + + val sql2 = "ALTER TABLE table_name CLUSTERED BY (col_name) INTO 3 BUCKETS" + checkError( + exception = parseException(sql2), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE CLUSTERED BY"), + context = ExpectedContext( + fragment = sql2, + start = 0, + stop = 60)) + + val sql3 = "ALTER TABLE table_name NOT CLUSTERED" + checkError( + exception = parseException(sql3), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE NOT CLUSTERED"), + context = ExpectedContext( + fragment = sql3, + start = 0, + stop = 35)) + + val sql4 = "ALTER TABLE table_name NOT SORTED" + checkError( + exception = parseException(sql4), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE NOT SORTED"), + context = ExpectedContext( + fragment = sql4, + start = 0, + stop = 32)) } test("alter table: skewed by (not supported)") { - assertUnsupported("ALTER TABLE table_name NOT SKEWED") - assertUnsupported("ALTER TABLE table_name NOT STORED AS DIRECTORIES") - assertUnsupported("ALTER TABLE table_name SET SKEWED LOCATION (col_name1=\"location1\"") - assertUnsupported("ALTER TABLE table_name SKEWED BY (key) ON (1,5,6) STORED AS DIRECTORIES") + val sql1 = "ALTER TABLE table_name NOT SKEWED" + checkError( + exception = parseException(sql1), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE NOT SKEWED"), + context = ExpectedContext( + fragment = sql1, + start = 0, + stop = 32)) + + val sql2 = "ALTER TABLE table_name NOT STORED AS DIRECTORIES" + checkError( + exception = parseException(sql2), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE NOT STORED AS DIRECTORIES"), + context = ExpectedContext( + fragment = sql2, + start = 0, + stop = 47)) + + val sql3 = "ALTER TABLE table_name SET SKEWED LOCATION (col_name1=\"location1\"" + checkError( + exception = parseException(sql3), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE SET SKEWED LOCATION"), + context = ExpectedContext( + fragment = sql3, + start = 0, + stop = 64)) + + val sql4 = "ALTER TABLE table_name SKEWED BY (key) ON (1,5,6) STORED AS DIRECTORIES" + checkError( + exception = parseException(sql4), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE SKEWED BY"), + context = ExpectedContext( + fragment = sql4, + start = 0, + stop = 70)) } test("alter table: replace columns (not allowed)") { - assertUnsupported( - """ - |ALTER TABLE table_name REPLACE COLUMNS (new_col1 INT - |COMMENT 'test_comment', new_col2 LONG COMMENT 'test_comment2') RESTRICT - """.stripMargin) + val sql = + """ALTER TABLE table_name REPLACE COLUMNS (new_col1 INT + |COMMENT 'test_comment', new_col2 LONG COMMENT 'test_comment2') RESTRICT""".stripMargin + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "ALTER TABLE REPLACE COLUMNS"), + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 123)) } test("SPARK-14383: DISTRIBUTE and UNSET as non-keywords") { @@ -194,59 +361,113 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { } test("unsupported operations") { - intercept[ParseException] { - parser.parsePlan( - """ - |CREATE TEMPORARY TABLE ctas2 - |ROW FORMAT SERDE "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" - |WITH SERDEPROPERTIES("serde_p1"="p1","serde_p2"="p2") - |STORED AS RCFile - |TBLPROPERTIES("tbl_p1"="p11", "tbl_p2"="p22") - |AS SELECT key, value FROM src ORDER BY key, value - """.stripMargin) - } - intercept[ParseException] { - parser.parsePlan( - """ - |CREATE TABLE user_info_bucketed(user_id BIGINT, firstname STRING, lastname STRING) - |CLUSTERED BY(user_id) INTO 256 BUCKETS - |AS SELECT key, value FROM src ORDER BY key, value - """.stripMargin) - } - intercept[ParseException] { - parser.parsePlan( - """ - |CREATE TABLE user_info_bucketed(user_id BIGINT, firstname STRING, lastname STRING) - |SKEWED BY (key) ON (1,5,6) - |AS SELECT key, value FROM src ORDER BY key, value - """.stripMargin) - } - intercept[ParseException] { - parser.parsePlan( - """ - |SELECT TRANSFORM (key, value) USING 'cat' AS (tKey, tValue) - |ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe' - |RECORDREADER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordReader' - |FROM testData - """.stripMargin) - } + val sql1 = + """CREATE TEMPORARY TABLE ctas2 + |ROW FORMAT SERDE "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" + |WITH SERDEPROPERTIES("serde_p1"="p1","serde_p2"="p2") + |STORED AS RCFile + |TBLPROPERTIES("tbl_p1"="p11", "tbl_p2"="p22") + |AS SELECT key, value FROM src ORDER BY key, value""".stripMargin + + checkError( + exception = parseException(sql1), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map( + "message" -> "CREATE TEMPORARY TABLE ... AS ..., use CREATE TEMPORARY VIEW instead"), + context = ExpectedContext( + fragment = sql1, + start = 0, + stop = 266)) + + val sql2 = + """CREATE TABLE user_info_bucketed(user_id BIGINT, firstname STRING, lastname STRING) + |CLUSTERED BY(user_id) INTO 256 BUCKETS + |AS SELECT key, value FROM src ORDER BY key, value""".stripMargin + checkError( + exception = parseException(sql2), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map( + "message" -> "Schema may not be specified in a Create Table As Select (CTAS) statement"), + context = ExpectedContext( + fragment = sql2, + start = 0, + stop = 170)) + + val sql3 = + """CREATE TABLE user_info_bucketed(user_id BIGINT, firstname STRING, lastname STRING) + |SKEWED BY (key) ON (1,5,6) + |AS SELECT key, value FROM src ORDER BY key, value""".stripMargin + checkError( + exception = parseException(sql3), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "CREATE TABLE ... SKEWED BY"), + context = ExpectedContext( + fragment = sql3, + start = 0, + stop = 158)) + + val sql4 = """SELECT TRANSFORM (key, value) USING 'cat' AS (tKey, tValue) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe' + |RECORDREADER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordReader' + |FROM testData""".stripMargin + checkError( + exception = parseException(sql4), + errorClass = "_LEGACY_ERROR_TEMP_0048", + parameters = Map.empty, + context = ExpectedContext( + fragment = sql4, + start = 0, + stop = 230)) } test("Invalid interval term should throw AnalysisException") { - def assertError(sql: String, errorMessage: String): Unit = { - val e = intercept[AnalysisException] { - parser.parsePlan(sql) - } - assert(e.getMessage.contains(errorMessage)) - } - assertError("select interval '42-32' year to month", - "month 32 outside range [0, 11]") - assertError("select interval '5 49:12:15' day to second", - "hour 49 outside range [0, 23]") - assertError("select interval '23:61:15' hour to second", - "minute 61 outside range [0, 59]") - assertError("select interval '.1111111111' second", - "'.1111111111' is out of range") + val sql1 = "select interval '42-32' year to month" + val value1 = "Error parsing interval year-month string: " + + "requirement failed: month 32 outside range [0, 11]" + val fragment1 = "'42-32' year to month" + checkError( + exception = parseException(sql1), + errorClass = "_LEGACY_ERROR_TEMP_0063", + parameters = Map("msg" -> value1), + context = ExpectedContext( + fragment = fragment1, + start = 16, + stop = 36)) + + val sql2 = "select interval '5 49:12:15' day to second" + val fragment2 = "'5 49:12:15' day to second" + checkError( + exception = parseException(sql2), + errorClass = "_LEGACY_ERROR_TEMP_0063", + parameters = Map("msg" -> "requirement failed: hour 49 outside range [0, 23]"), + context = ExpectedContext( + fragment = fragment2, + start = 16, + stop = 41)) + + val sql3 = "select interval '23:61:15' hour to second" + val fragment3 = "'23:61:15' hour to second" + checkError( + exception = parseException(sql3), + errorClass = "_LEGACY_ERROR_TEMP_0063", + parameters = Map("msg" -> "requirement failed: minute 61 outside range [0, 59]"), + context = ExpectedContext( + fragment = fragment3, + start = 16, + stop = 40)) + + val sql4 = "select interval '.1111111111' second" + val value4 = "Error parsing ' .1111111111 second' to interval, " + + "interval can only support nanosecond precision, '.1111111111' is out of range" + val fragment4 = "'.1111111111' second" + checkError( + exception = parseException(sql4), + errorClass = "_LEGACY_ERROR_TEMP_0062", + parameters = Map("msg" -> value4), + context = ExpectedContext( + fragment = fragment4, + start = 16, + stop = 35)) } test("use native json_tuple instead of hive's UDTF in LATERAL VIEW") { @@ -333,7 +554,14 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { comparePlans(parsed2, expected2) val v3 = "CREATE TEMPORARY VIEW a.b AS SELECT 1" - intercept(v3, "It is not allowed to add database prefix") + checkError( + exception = parseException(v3), + errorClass = "_LEGACY_ERROR_TEMP_0054", + parameters = Map("database" -> "a"), + context = ExpectedContext( + fragment = v3, + start = 0, + stop = 36)) } test("create temp view - full") { @@ -380,25 +608,44 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { test("create view -- partitioned view") { val v1 = "CREATE VIEW view1 partitioned on (ds, hr) as select * from srcpart" - intercept[ParseException] { - parser.parsePlan(v1) - } + checkError( + exception = parseException(v1), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "CREATE VIEW ... PARTITIONED ON"), + context = ExpectedContext( + fragment = v1, + start = 0, + stop = 65)) } test("create view - duplicate clauses") { def createViewStatement(duplicateClause: String): String = { - s""" - |CREATE OR REPLACE VIEW view1 + s"""CREATE OR REPLACE VIEW view1 |(col1, col3 COMMENT 'hello') |$duplicateClause |$duplicateClause - |AS SELECT * FROM tab1 - """.stripMargin + |AS SELECT * FROM tab1""".stripMargin } + val sql1 = createViewStatement("COMMENT 'BLABLA'") + checkError( + exception = parseException(sql1), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "COMMENT"), + context = ExpectedContext( + fragment = sql1, + start = 0, + stop = 112)) + val sql2 = createViewStatement("TBLPROPERTIES('prop1Key'=\"prop1Val\")") - intercept(sql1, "Found duplicate clauses: COMMENT") - intercept(sql2, "Found duplicate clauses: TBLPROPERTIES") + checkError( + exception = parseException(sql2), + errorClass = "_LEGACY_ERROR_TEMP_0041", + parameters = Map("clauseName" -> "TBLPROPERTIES"), + context = ExpectedContext( + fragment = sql2, + start = 0, + stop = 152)) } test("CREATE FUNCTION") { @@ -436,8 +683,15 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { FunctionResource(ArchiveResource, "a"), FunctionResource(FileResource, "f")), false, false)) - intercept("CREATE FUNCTION a as 'fun' USING OTHER 'o'", - "Operation not allowed: CREATE FUNCTION with resource type 'other'") + val sql = "CREATE FUNCTION a as 'fun' USING OTHER 'o'" + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "CREATE FUNCTION with resource type 'other'"), + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 41)) } test("DROP FUNCTION") { @@ -461,21 +715,43 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { parser.parsePlan("DROP TEMPORARY FUNCTION IF EXISTS a"), DropFunctionCommand(Seq("a").asFunctionIdentifier, true, true)) - intercept("DROP TEMPORARY FUNCTION a.b", - "DROP TEMPORARY FUNCTION requires a single part name") - intercept("DROP TEMPORARY FUNCTION IF EXISTS a.b", - "DROP TEMPORARY FUNCTION requires a single part name") + val sql1 = "DROP TEMPORARY FUNCTION a.b" + checkError( + exception = parseException(sql1), + errorClass = "INVALID_SQL_SYNTAX", + parameters = Map( + "inputString" -> "DROP TEMPORARY FUNCTION requires a single part name but got: `a`.`b`"), + context = ExpectedContext( + fragment = sql1, + start = 0, + stop = 26)) + + val sql2 = "DROP TEMPORARY FUNCTION IF EXISTS a.b" + checkError( + exception = parseException(sql2), + errorClass = "INVALID_SQL_SYNTAX", + parameters = Map( + "inputString" -> "DROP TEMPORARY FUNCTION requires a single part name but got: `a`.`b`"), + context = ExpectedContext( + fragment = sql2, + start = 0, + stop = 36)) } test("SPARK-32374: create temporary view with properties not allowed") { - assertUnsupported( - sql = """ - |CREATE OR REPLACE TEMPORARY VIEW a.b.c - |(col1, col3 COMMENT 'hello') - |TBLPROPERTIES('prop1Key'="prop1Val") - |AS SELECT * FROM tab1 - """.stripMargin, - containsThesePhrases = Seq("TBLPROPERTIES can't coexist with CREATE TEMPORARY VIEW")) + val sql = + """CREATE OR REPLACE TEMPORARY VIEW a.b.c + |(col1, col3 COMMENT 'hello') + |TBLPROPERTIES('prop1Key'="prop1Val") + |AS SELECT * FROM tab1""".stripMargin + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "TBLPROPERTIES can't coexist with CREATE TEMPORARY VIEW"), + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 125)) } test("create table like") {