aboutsummaryrefslogtreecommitdiff
path: root/sql/hive/src/main/scala/org/apache/spark
diff options
context:
space:
mode:
Diffstat (limited to 'sql/hive/src/main/scala/org/apache/spark')
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala2
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala17
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala8
3 files changed, 14 insertions, 13 deletions
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
index 313093818f..ee048b2588 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
@@ -36,7 +36,7 @@ import org.apache.spark.sql.hive.client.HiveClient
* All public methods must be synchronized for thread-safety.
*/
private[spark] class HiveExternalCatalog(client: HiveClient) extends ExternalCatalog with Logging {
- import ExternalCatalog._
+ import CatalogTypes.TablePartitionSpec
// Exceptions thrown by the hive client that we would like to wrap
private val clientExceptions = Set(
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala
index ef08a39c17..b224664050 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala
@@ -21,6 +21,7 @@ import java.io.PrintStream
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.catalog._
+import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.Expression
@@ -119,7 +120,7 @@ private[hive] trait HiveClient {
def dropPartitions(
db: String,
table: String,
- specs: Seq[ExternalCatalog.TablePartitionSpec],
+ specs: Seq[TablePartitionSpec],
ignoreIfNotExists: Boolean): Unit
/**
@@ -128,8 +129,8 @@ private[hive] trait HiveClient {
def renamePartitions(
db: String,
table: String,
- specs: Seq[ExternalCatalog.TablePartitionSpec],
- newSpecs: Seq[ExternalCatalog.TablePartitionSpec]): Unit
+ specs: Seq[TablePartitionSpec],
+ newSpecs: Seq[TablePartitionSpec]): Unit
/**
* Alter one or more table partitions whose specs match the ones specified in `newParts`,
@@ -144,7 +145,7 @@ private[hive] trait HiveClient {
final def getPartition(
dbName: String,
tableName: String,
- spec: ExternalCatalog.TablePartitionSpec): CatalogTablePartition = {
+ spec: TablePartitionSpec): CatalogTablePartition = {
getPartitionOption(dbName, tableName, spec).getOrElse {
throw new NoSuchPartitionException(dbName, tableName, spec)
}
@@ -154,14 +155,14 @@ private[hive] trait HiveClient {
final def getPartitionOption(
db: String,
table: String,
- spec: ExternalCatalog.TablePartitionSpec): Option[CatalogTablePartition] = {
+ spec: TablePartitionSpec): Option[CatalogTablePartition] = {
getPartitionOption(getTable(db, table), spec)
}
/** Returns the specified partition or None if it does not exist. */
def getPartitionOption(
table: CatalogTable,
- spec: ExternalCatalog.TablePartitionSpec): Option[CatalogTablePartition]
+ spec: TablePartitionSpec): Option[CatalogTablePartition]
/**
* Returns the partitions for the given table that match the supplied partition spec.
@@ -170,7 +171,7 @@ private[hive] trait HiveClient {
final def getPartitions(
db: String,
table: String,
- partialSpec: Option[ExternalCatalog.TablePartitionSpec]): Seq[CatalogTablePartition] = {
+ partialSpec: Option[TablePartitionSpec]): Seq[CatalogTablePartition] = {
getPartitions(getTable(db, table), partialSpec)
}
@@ -180,7 +181,7 @@ private[hive] trait HiveClient {
*/
def getPartitions(
table: CatalogTable,
- partialSpec: Option[ExternalCatalog.TablePartitionSpec] = None): Seq[CatalogTablePartition]
+ partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition]
/** Returns partitions filtered by predicates for the given table. */
def getPartitionsByFilter(
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
index d651791f9c..c98eaa0d15 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
@@ -40,7 +40,7 @@ import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchPartitionException}
import org.apache.spark.sql.catalyst.catalog._
-import org.apache.spark.sql.catalyst.catalog.ExternalCatalog.TablePartitionSpec
+import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.execution.QueryExecutionException
import org.apache.spark.util.{CircularBuffer, Utils}
@@ -375,7 +375,7 @@ private[hive] class HiveClientImpl(
override def dropPartitions(
db: String,
table: String,
- specs: Seq[ExternalCatalog.TablePartitionSpec],
+ specs: Seq[TablePartitionSpec],
ignoreIfNotExists: Boolean): Unit = withHiveState {
// TODO: figure out how to drop multiple partitions in one call
val hiveTable = client.getTable(db, table, true /* throw exception */)
@@ -399,8 +399,8 @@ private[hive] class HiveClientImpl(
override def renamePartitions(
db: String,
table: String,
- specs: Seq[ExternalCatalog.TablePartitionSpec],
- newSpecs: Seq[ExternalCatalog.TablePartitionSpec]): Unit = withHiveState {
+ specs: Seq[TablePartitionSpec],
+ newSpecs: Seq[TablePartitionSpec]): Unit = withHiveState {
require(specs.size == newSpecs.size, "number of old and new partition specs differ")
val catalogTable = getTable(db, table)
val hiveTable = toHiveTable(catalogTable)