diff options
author | Wenchen Fan <wenchen@databricks.com> | 2016-07-25 22:05:48 +0800 |
---|---|---|
committer | Cheng Lian <lian@databricks.com> | 2016-07-25 22:05:48 +0800 |
commit | 64529b186a1c33740067cc7639d630bc5b9ae6e8 (patch) | |
tree | ab923216bc0a6480b21df76b024e7d232033eb1e /sql/hive | |
parent | d27d362ebae0c4a5cc6c99f13ef20049214dd4f9 (diff) | |
download | spark-64529b186a1c33740067cc7639d630bc5b9ae6e8.tar.gz spark-64529b186a1c33740067cc7639d630bc5b9ae6e8.tar.bz2 spark-64529b186a1c33740067cc7639d630bc5b9ae6e8.zip |
[SPARK-16691][SQL] move BucketSpec to catalyst module and use it in CatalogTable
## What changes were proposed in this pull request?
It's weird that we have `BucketSpec` to abstract bucket info, but don't use it in `CatalogTable`. This PR moves `BucketSpec` into catalyst module.
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes #14331 from cloud-fan/check.
Diffstat (limited to 'sql/hive')
3 files changed, 7 insertions, 13 deletions
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 9f5782f045..2392cc0bdd 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 @@ -365,9 +365,9 @@ private[hive] class HiveClientImpl( }, schema = schema, partitionColumnNames = partCols.map(_.name), - sortColumnNames = Seq(), // TODO: populate this - bucketColumnNames = h.getBucketCols.asScala, - numBuckets = h.getNumBuckets, + // We can not populate bucketing information for Hive tables as Spark SQL has a different + // implementation of hash function from Hive. + bucketSpec = None, owner = h.getOwner, createTime = h.getTTable.getCreateTime.toLong * 1000, lastAccessTime = h.getLastAccessTime.toLong * 1000, @@ -764,10 +764,7 @@ private[hive] class HiveClientImpl( hiveTable.setFields(schema.asJava) } hiveTable.setPartCols(partCols.asJava) - // TODO: set sort columns here too - hiveTable.setBucketCols(table.bucketColumnNames.asJava) hiveTable.setOwner(conf.getUser) - hiveTable.setNumBuckets(table.numBuckets) hiveTable.setCreateTime((table.createTime / 1000).toInt) hiveTable.setLastAccessTime((table.lastAccessTime / 1000).toInt) table.storage.locationUri.foreach { loc => shim.setDataLocation(hiveTable, loc) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index a708434f5e..5450fba753 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -293,9 +293,7 @@ class HiveDDLCommandSuite extends PlanTest { assert(desc.tableType == CatalogTableType.MANAGED) assert(desc.schema == Seq(CatalogColumn("id", "int"), CatalogColumn("name", "string"))) assert(desc.partitionColumnNames.isEmpty) - assert(desc.sortColumnNames.isEmpty) - assert(desc.bucketColumnNames.isEmpty) - assert(desc.numBuckets == -1) + assert(desc.bucketSpec.isEmpty) assert(desc.viewText.isEmpty) assert(desc.viewOriginalText.isEmpty) assert(desc.storage.locationUri.isEmpty) @@ -453,9 +451,7 @@ class HiveDDLCommandSuite extends PlanTest { CatalogColumn("name", "string"), CatalogColumn("month", "int"))) assert(desc.partitionColumnNames == Seq("month")) - assert(desc.sortColumnNames.isEmpty) - assert(desc.bucketColumnNames.isEmpty) - assert(desc.numBuckets == -1) + assert(desc.bucketSpec.isEmpty) assert(desc.viewText.isEmpty) assert(desc.viewOriginalText.isEmpty) assert(desc.storage.locationUri == Some("/path/to/mercury")) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index fc01ff3f5a..e461490310 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -20,10 +20,11 @@ package org.apache.spark.sql.sources import java.io.File import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.execution.DataSourceScanExec -import org.apache.spark.sql.execution.datasources.{BucketSpec, DataSourceStrategy} +import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.functions._ |