diff options
author | gatorsmile <gatorsmile@gmail.com> | 2016-12-20 23:40:02 -0800 |
---|---|---|
committer | Reynold Xin <rxin@databricks.com> | 2016-12-20 23:40:02 -0800 |
commit | 24c0c94128770be9034fb69518713d7f6aa1e041 (patch) | |
tree | f367c32b5005da96c0634ba5e1f8337e5d0aa86e /sql/hive/src/test/scala/org | |
parent | b2dd8ec6b2c05c996e2d7c0bf8db0073c1ee0b94 (diff) | |
download | spark-24c0c94128770be9034fb69518713d7f6aa1e041.tar.gz spark-24c0c94128770be9034fb69518713d7f6aa1e041.tar.bz2 spark-24c0c94128770be9034fb69518713d7f6aa1e041.zip |
[SPARK-18949][SQL] Add recoverPartitions API to Catalog
### What changes were proposed in this pull request?
Currently, we only have a SQL interface for recovering all the partitions in the directory of a table and update the catalog. `MSCK REPAIR TABLE` or `ALTER TABLE table RECOVER PARTITIONS`. (Actually, very hard for me to remember `MSCK` and have no clue what it means)
After the new "Scalable Partition Handling", the table repair becomes much more important for making visible the data in the created data source partitioned table.
Thus, this PR is to add it into the Catalog interface. After this PR, users can repair the table by
```Scala
spark.catalog.recoverPartitions("testTable")
```
### How was this patch tested?
Modified the existing test cases.
Author: gatorsmile <gatorsmile@gmail.com>
Closes #16356 from gatorsmile/repairTable.
Diffstat (limited to 'sql/hive/src/test/scala/org')
-rw-r--r-- | sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala | 6 |
1 files changed, 3 insertions, 3 deletions
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala index c2ac032760..3f84cbdb1b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala @@ -70,7 +70,7 @@ class PartitionProviderCompatibilitySuite } withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { verifyIsLegacyTable("test") - spark.sql("msck repair table test") + spark.catalog.recoverPartitions("test") spark.sql("show partitions test").count() // check we are a new table // sanity check table performance @@ -90,7 +90,7 @@ class PartitionProviderCompatibilitySuite setupPartitionedDatasourceTable("test", dir) spark.sql("show partitions test").count() // check we are a new table assert(spark.sql("select * from test").count() == 0) // needs repair - spark.sql("msck repair table test") + spark.catalog.recoverPartitions("test") assert(spark.sql("select * from test").count() == 5) } } @@ -160,7 +160,7 @@ class PartitionProviderCompatibilitySuite withTable("test") { withTempDir { dir => setupPartitionedDatasourceTable("test", dir) - sql("msck repair table test") + spark.catalog.recoverPartitions("test") spark.sql( """insert overwrite table test |partition (partCol=1) |