diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index e4c837a7ab13..eb6c77a5d181 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -758,9 +758,18 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { BucketSpec( ctx.INTEGER_VALUE.getText.toInt, visitIdentifierList(ctx.identifierList), - Option(ctx.orderedIdentifierList).toSeq + Option(ctx.orderedIdentifierList) + .toSeq .flatMap(_.orderedIdentifier.asScala) - .map(_.identifier.getText)) + .map { orderedIdCtx => + Option(orderedIdCtx.ordering).map(_.getText).foreach { dir => + if (dir.toLowerCase != "asc") { + throw parseException("Only ASC ordering is supported for sorting columns", ctx) + } + } + + orderedIdCtx.identifier.getText + }) } /** diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index cb100021be3d..d6c98ea61968 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -948,9 +948,9 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sql( s"""CREATE TABLE t USING PARQUET - |OPTIONS (PATH '$path') - |PARTITIONED BY (a) - |AS SELECT 1 AS a, 2 AS b + |OPTIONS (PATH '$path') + |PARTITIONED BY (a) + |AS SELECT 1 AS a, 2 AS b """.stripMargin ) @@ -972,9 +972,9 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sql( s"""CREATE TABLE t USING PARQUET - |OPTIONS (PATH '$path') - |CLUSTERED BY (a) SORTED BY (b) INTO 2 BUCKETS - |AS SELECT 1 AS a, 2 AS b + |OPTIONS (PATH '$path') + |CLUSTERED BY (a) SORTED BY (b) INTO 2 BUCKETS + |AS SELECT 1 AS a, 2 AS b """.stripMargin ) @@ -992,9 +992,9 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sql( s"""CREATE TABLE t USING PARQUET - |OPTIONS (PATH '$path') - |CLUSTERED BY (a) INTO 2 BUCKETS - |AS SELECT 1 AS a, 2 AS b + |OPTIONS (PATH '$path') + |CLUSTERED BY (a) INTO 2 BUCKETS + |AS SELECT 1 AS a, 2 AS b """.stripMargin ) @@ -1016,10 +1016,10 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sql( s"""CREATE TABLE t USING PARQUET - |OPTIONS (PATH '$path') - |PARTITIONED BY (a) - |CLUSTERED BY (b) SORTED BY (c) INTO 2 BUCKETS - |AS SELECT 1 AS a, 2 AS b, 3 AS c + |OPTIONS (PATH '$path') + |PARTITIONED BY (a) + |CLUSTERED BY (b) SORTED BY (c) INTO 2 BUCKETS + |AS SELECT 1 AS a, 2 AS b, 3 AS c """.stripMargin ) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 100cb3cef8bf..f20ab36efbf0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -24,6 +24,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, FunctionRegistry} +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.{HiveUtils, MetastoreRelation} @@ -1488,4 +1489,20 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { "Once a managed table has been dropped, " + "dirs of this table should also have been deleted.") } + + test("SPARK-14981: DESC not supported for sorting columns") { + withTable("t") { + val cause = intercept[ParseException] { + sql( + """CREATE TABLE t USING PARQUET + |OPTIONS (PATH '/path/to/file') + |CLUSTERED BY (a) SORTED BY (b DESC) INTO 2 BUCKETS + |AS SELECT 1 AS a, 2 AS b + """.stripMargin + ) + } + + assert(cause.getMessage.contains("Only ASC ordering is supported for sorting columns")) + } + } }