aboutsummaryrefslogtreecommitdiff
path: root/sql/core
diff options
context:
space:
mode:
authorgatorsmile <gatorsmile@gmail.com>2016-09-01 16:36:14 +0800
committerWenchen Fan <wenchen@databricks.com>2016-09-01 16:36:14 +0800
commit1f06a5b6a0584d0c9656f58eaf54e54e2383c82b (patch)
tree9ed5ed2249d690e4752b61e0da3f7e0c852a155b /sql/core
parentdd859f95c0aaa0b7c8fbff0a5f108cf3c9bf520a (diff)
downloadspark-1f06a5b6a0584d0c9656f58eaf54e54e2383c82b.tar.gz
spark-1f06a5b6a0584d0c9656f58eaf54e54e2383c82b.tar.bz2
spark-1f06a5b6a0584d0c9656f58eaf54e54e2383c82b.zip
[SPARK-17353][SPARK-16943][SPARK-16942][SQL] Fix multiple bugs in CREATE TABLE LIKE command
### What changes were proposed in this pull request? The existing `CREATE TABLE LIKE` command has multiple issues: - The generated table is non-empty when the source table is a data source table. The major reason is the data source table is using the table property `path` to store the location of table contents. Currently, we keep it unchanged. Thus, we still create the same table with the same location. - The table type of the generated table is `EXTERNAL` when the source table is an external Hive Serde table. Currently, we explicitly set it to `MANAGED`, but Hive is checking the table property `EXTERNAL` to decide whether the table is `EXTERNAL` or not. (See https://github.com/apache/hive/blob/master/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L1407-L1408) Thus, the created table is still `EXTERNAL`. - When the source table is a `VIEW`, the metadata of the generated table contains the original view text and view original text. So far, this does not break anything, but it could cause something wrong in Hive. (For example, https://github.com/apache/hive/blob/master/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L1405-L1406) - The issue regarding the table `comment`. To follow what Hive does, the table comment should be cleaned, but the column comments should be still kept. - The `INDEX` table is not supported. Thus, we should throw an exception in this case. - `owner` should not be retained. `ToHiveTable` set it [here](https://github.com/apache/spark/blob/e679bc3c1cd418ef0025d2ecbc547c9660cac433/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala#L793) no matter which value we set in `CatalogTable`. We set it to an empty string for avoiding the confusing output in Explain. - Add a support for temp tables - Like Hive, we should not copy the table properties from the source table to the created table, especially for the statistics-related properties, which could be wrong in the created table. - `unsupportedFeatures` should not be copied from the source table. The created table does not have these unsupported features. - When the type of source table is a view, the target table is using the default format of data source tables: `spark.sql.sources.default`. This PR is to fix the above issues. ### How was this patch tested? Improve the test coverage by adding more test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #14531 from gatorsmile/createTableLike.
Diffstat (limited to 'sql/core')
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala57
1 files changed, 45 insertions, 12 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
index b4a15b8b28..67b2329eff 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
@@ -29,17 +29,23 @@ import org.apache.hadoop.fs.Path
import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType}
+import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType}
import org.apache.spark.sql.catalyst.catalog.CatalogTableType._
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.catalyst.util.quoteIdentifier
import org.apache.spark.sql.execution.datasources.PartitioningUtils
+import org.apache.spark.sql.internal.HiveSerDe
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
/**
- * A command to create a table with the same definition of the given existing table.
+ * A command to create a MANAGED table with the same definition of the given existing table.
+ * In the target table definition, the table comment is always empty but the column comments
+ * are identical to the ones defined in the source table.
+ *
+ * The CatalogTable attributes copied from the source table are storage(inputFormat, outputFormat,
+ * serde, compressed, properties), schema, provider, partitionColumnNames, bucketSpec.
*
* The syntax of using this command in SQL is:
* {{{
@@ -58,18 +64,45 @@ case class CreateTableLikeCommand(
throw new AnalysisException(
s"Source table in CREATE TABLE LIKE does not exist: '$sourceTable'")
}
- if (catalog.isTemporaryTable(sourceTable)) {
- throw new AnalysisException(
- s"Source table in CREATE TABLE LIKE cannot be temporary: '$sourceTable'")
+
+ val sourceTableDesc = catalog.getTableMetadata(sourceTable)
+
+ // Storage format
+ val newStorage =
+ if (sourceTableDesc.tableType == CatalogTableType.VIEW) {
+ val newPath = catalog.defaultTablePath(targetTable)
+ CatalogStorageFormat.empty.copy(properties = Map("path" -> newPath))
+ } else if (DDLUtils.isDatasourceTable(sourceTableDesc)) {
+ val newPath = catalog.defaultTablePath(targetTable)
+ val newSerdeProp =
+ sourceTableDesc.storage.properties.filterKeys(_.toLowerCase != "path") ++
+ Map("path" -> newPath)
+ sourceTableDesc.storage.copy(
+ locationUri = None,
+ properties = newSerdeProp)
+ } else {
+ sourceTableDesc.storage.copy(
+ locationUri = None,
+ properties = sourceTableDesc.storage.properties)
+ }
+
+ val newProvider = if (sourceTableDesc.tableType == CatalogTableType.VIEW) {
+ Some(sparkSession.sessionState.conf.defaultDataSourceName)
+ } else {
+ sourceTableDesc.provider
}
- val tableToCreate = catalog.getTableMetadata(sourceTable).copy(
- identifier = targetTable,
- tableType = CatalogTableType.MANAGED,
- createTime = System.currentTimeMillis,
- lastAccessTime = -1).withNewStorage(locationUri = None)
+ val newTableDesc =
+ CatalogTable(
+ identifier = targetTable,
+ tableType = CatalogTableType.MANAGED,
+ storage = newStorage,
+ schema = sourceTableDesc.schema,
+ provider = newProvider,
+ partitionColumnNames = sourceTableDesc.partitionColumnNames,
+ bucketSpec = sourceTableDesc.bucketSpec)
- catalog.createTable(tableToCreate, ifNotExists)
+ catalog.createTable(newTableDesc, ifNotExists)
Seq.empty[Row]
}
}
@@ -517,7 +550,7 @@ case class ShowTablesCommand(
/**
- * A command for users to list the properties for a table If propertyKey is specified, the value
+ * A command for users to list the properties for a table. If propertyKey is specified, the value
* for the propertyKey is returned. If propertyKey is not specified, all the keys and their
* corresponding values are returned.
* The syntax of using this command in SQL is: