aboutsummaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
authorgatorsmile <gatorsmile@gmail.com>2016-12-20 23:40:02 -0800
committerReynold Xin <rxin@databricks.com>2016-12-20 23:40:02 -0800
commit24c0c94128770be9034fb69518713d7f6aa1e041 (patch)
treef367c32b5005da96c0634ba5e1f8337e5d0aa86e /sql
parentb2dd8ec6b2c05c996e2d7c0bf8db0073c1ee0b94 (diff)
downloadspark-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')
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala7
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala14
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala6
3 files changed, 24 insertions, 3 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala
index aecdda1c36..6b061f8ab2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala
@@ -301,6 +301,13 @@ abstract class Catalog {
def dropGlobalTempView(viewName: String): Boolean
/**
+ * Recover all the partitions in the directory of a table and update the catalog.
+ *
+ * @since 2.1.1
+ */
+ def recoverPartitions(tableName: String): Unit
+
+ /**
* Returns true if the table is currently cached in-memory.
*
* @since 2.0.0
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
index 6d984621cc..41ed9d7180 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
@@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.{DefinedByConstructorParams, FunctionIdenti
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.execution.command.AlterTableRecoverPartitionsCommand
import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource}
import org.apache.spark.sql.types.StructType
@@ -394,6 +395,19 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
}
/**
+ * Recover all the partitions in the directory of a table and update the catalog.
+ *
+ * @param tableName the name of the table to be repaired.
+ * @group ddl_ops
+ * @since 2.1.1
+ */
+ override def recoverPartitions(tableName: String): Unit = {
+ val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
+ sparkSession.sessionState.executePlan(
+ AlterTableRecoverPartitionsCommand(tableIdent)).toRdd
+ }
+
+ /**
* Returns true if the table is currently cached in-memory.
*
* @group cachemgmt
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)