@@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.parser._
3030import org .apache .spark .sql .catalyst .parser .SqlBaseParser ._
3131import org .apache .spark .sql .catalyst .plans .logical .{LogicalPlan , OneRowRelation , ScriptInputOutputSchema }
3232import org .apache .spark .sql .execution .command ._
33- import org .apache .spark .sql .execution .datasources .{CreateTempViewUsing , _ }
33+ import org .apache .spark .sql .execution .datasources .{CreateTable , CreateTempViewUsing , _ }
3434import org .apache .spark .sql .internal .{HiveSerDe , SQLConf , VariableSubstitution }
3535import org .apache .spark .sql .types .{DataType , StructType }
3636
@@ -310,7 +310,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
310310 }
311311
312312 /**
313- * Create a [[CreateTableUsing ]] or a [[ CreateTableUsingAsSelect ]] logical plan.
313+ * Create a [[CreateTable ]] logical plan.
314314 */
315315 override def visitCreateTableUsing (ctx : CreateTableUsingContext ): LogicalPlan = withOrigin(ctx) {
316316 val (table, temp, ifNotExists, external) = visitCreateTableHeader(ctx.createTableHeader)
@@ -319,12 +319,26 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
319319 }
320320 val options = Option (ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map .empty)
321321 val provider = ctx.tableProvider.qualifiedName.getText
322+ val schema = Option (ctx.colTypeList()).map(createStructType)
322323 val partitionColumnNames =
323324 Option (ctx.partitionColumnNames)
324325 .map(visitIdentifierList(_).toArray)
325326 .getOrElse(Array .empty[String ])
326327 val bucketSpec = Option (ctx.bucketSpec()).map(visitBucketSpec)
327328
329+ val tableDesc = CatalogTable (
330+ identifier = table,
331+ tableType = CatalogTableType .MANAGED ,
332+ storage = CatalogStorageFormat .empty.copy(properties = options),
333+ schema = schema.getOrElse(new StructType ),
334+ provider = Some (provider),
335+ partitionColumnNames = partitionColumnNames,
336+ bucketSpec = bucketSpec
337+ )
338+
339+ // Determine the storage mode.
340+ val mode = if (ifNotExists) SaveMode .Ignore else SaveMode .ErrorIfExists
341+
328342 if (ctx.query != null ) {
329343 // Get the backing query.
330344 val query = plan(ctx.query)
@@ -333,32 +347,19 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
333347 operationNotAllowed(" CREATE TEMPORARY TABLE ... USING ... AS query" , ctx)
334348 }
335349
336- // Determine the storage mode.
337- val mode = if (ifNotExists) {
338- SaveMode .Ignore
339- } else {
340- SaveMode .ErrorIfExists
341- }
342-
343- CreateTableUsingAsSelect (
344- table, provider, partitionColumnNames, bucketSpec, mode, options, query)
350+ CreateTable (tableDesc, mode, Some (query))
345351 } else {
346- val struct = Option (ctx.colTypeList()).map(createStructType)
347- if (struct.isEmpty && bucketSpec.nonEmpty) {
348- throw new ParseException (
349- " Expected explicit specification of table schema when using CLUSTERED BY clause." , ctx)
350- }
352+ if (temp) {
353+ if (ifNotExists) {
354+ operationNotAllowed(" CREATE TEMPORARY TABLE IF NOT EXISTS" , ctx)
355+ }
351356
352- CreateTableUsing (
353- table,
354- struct,
355- provider,
356- temp,
357- options,
358- partitionColumnNames,
359- bucketSpec,
360- ifNotExists,
361- managedIfNoPath = true )
357+ logWarning(s " CREATE TEMPORARY TABLE ... USING ... is deprecated, please use " +
358+ " CREATE TEMPORARY VIEW ... USING ... instead" )
359+ CreateTempViewUsing (table, schema, replace = true , provider, options)
360+ } else {
361+ CreateTable (tableDesc, mode, None )
362+ }
362363 }
363364 }
364365
@@ -891,8 +892,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
891892 }
892893
893894 /**
894- * Create a table, returning either a [[CreateTableCommand ]] or a
895- * [[CreateHiveTableAsSelectLogicalPlan ]].
895+ * Create a table, returning a [[CreateTable ]] logical plan.
896896 *
897897 * This is not used to create datasource tables, which is handled through
898898 * "CREATE TABLE ... USING ...".
@@ -933,23 +933,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
933933 val properties = Option (ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map .empty)
934934 val selectQuery = Option (ctx.query).map(plan)
935935
936- // Ensuring whether no duplicate name is used in table definition
937- val colNames = dataCols.map(_.name)
938- if (colNames.length != colNames.distinct.length) {
939- val duplicateColumns = colNames.groupBy(identity).collect {
940- case (x, ys) if ys.length > 1 => " \" " + x + " \" "
941- }
942- operationNotAllowed(s " Duplicated column names found in table definition of $name: " +
943- duplicateColumns.mkString(" [" , " ," , " ]" ), ctx)
944- }
945-
946- // For Hive tables, partition columns must not be part of the schema
947- val badPartCols = partitionCols.map(_.name).toSet.intersect(colNames.toSet)
948- if (badPartCols.nonEmpty) {
949- operationNotAllowed(s " Partition columns may not be specified in the schema: " +
950- badPartCols.map(" \" " + _ + " \" " ).mkString(" [" , " ," , " ]" ), ctx)
951- }
952-
953936 // Note: Hive requires partition columns to be distinct from the schema, so we need
954937 // to include the partition columns here explicitly
955938 val schema = StructType (dataCols ++ partitionCols)
@@ -1001,10 +984,13 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
1001984 tableType = tableType,
1002985 storage = storage,
1003986 schema = schema,
987+ provider = Some (" hive" ),
1004988 partitionColumnNames = partitionCols.map(_.name),
1005989 properties = properties,
1006990 comment = comment)
1007991
992+ val mode = if (ifNotExists) SaveMode .Ignore else SaveMode .ErrorIfExists
993+
1008994 selectQuery match {
1009995 case Some (q) =>
1010996 // Just use whatever is projected in the select statement as our schema
@@ -1025,27 +1011,24 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
10251011
10261012 val hasStorageProperties = (ctx.createFileFormat != null ) || (ctx.rowFormat != null )
10271013 if (conf.convertCTAS && ! hasStorageProperties) {
1028- val mode = if (ifNotExists) SaveMode .Ignore else SaveMode .ErrorIfExists
10291014 // At here, both rowStorage.serdeProperties and fileStorage.serdeProperties
10301015 // are empty Maps.
10311016 val optionsWithPath = if (location.isDefined) {
10321017 Map (" path" -> location.get)
10331018 } else {
10341019 Map .empty[String , String ]
10351020 }
1036- CreateTableUsingAsSelect (
1037- tableIdent = tableDesc.identifier,
1038- provider = conf.defaultDataSourceName,
1039- partitionColumns = tableDesc.partitionColumnNames.toArray,
1040- bucketSpec = None ,
1041- mode = mode,
1042- options = optionsWithPath,
1043- q
1021+
1022+ val newTableDesc = tableDesc.copy(
1023+ storage = CatalogStorageFormat .empty.copy(properties = optionsWithPath),
1024+ provider = Some (conf.defaultDataSourceName)
10441025 )
1026+
1027+ CreateTable (newTableDesc, mode, Some (q))
10451028 } else {
1046- CreateHiveTableAsSelectLogicalPlan (tableDesc, q, ifNotExists )
1029+ CreateTable (tableDesc, mode, Some (q) )
10471030 }
1048- case None => CreateTableCommand (tableDesc, ifNotExists )
1031+ case None => CreateTable (tableDesc, mode, None )
10491032 }
10501033 }
10511034
0 commit comments