Repository: spark
Updated Branches:
  refs/heads/master 79fd0cc05 -> ccb115430


http://git-wip-us.apache.org/repos/asf/spark/blob/ccb11543/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..5f16960
--- /dev/null
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive
+
+import java.io.File
+
+import org.apache.spark.metrics.source.HiveCatalogMetrics
+import org.apache.spark.sql.{AnalysisException, QueryTest}
+import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SQLTestUtils
+
+class PartitionProviderCompatibilitySuite
+  extends QueryTest with TestHiveSingleton with SQLTestUtils {
+
+  private def setupPartitionedDatasourceTable(tableName: String, dir: File): 
Unit = {
+    spark.range(5).selectExpr("id as fieldOne", "id as partCol").write
+      .partitionBy("partCol")
+      .mode("overwrite")
+      .parquet(dir.getAbsolutePath)
+
+    spark.sql(s"""
+      |create table $tableName (fieldOne long, partCol int)
+      |using parquet
+      |options (path "${dir.getAbsolutePath}")
+      |partitioned by (partCol)""".stripMargin)
+  }
+
+  private def verifyIsLegacyTable(tableName: String): Unit = {
+    val unsupportedCommands = Seq(
+      s"ALTER TABLE $tableName ADD PARTITION (partCol=1) LOCATION '/foo'",
+      s"ALTER TABLE $tableName PARTITION (partCol=1) RENAME TO PARTITION 
(partCol=2)",
+      s"ALTER TABLE $tableName PARTITION (partCol=1) SET LOCATION '/foo'",
+      s"ALTER TABLE $tableName DROP PARTITION (partCol=1)",
+      s"DESCRIBE $tableName PARTITION (partCol=1)",
+      s"SHOW PARTITIONS $tableName")
+
+    withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") {
+      for (cmd <- unsupportedCommands) {
+        val e = intercept[AnalysisException] {
+          spark.sql(cmd)
+        }
+        assert(e.getMessage.contains("partition metadata is not stored in the 
Hive metastore"), e)
+      }
+    }
+  }
+
+  test("convert partition provider to hive with repair table") {
+    withTable("test") {
+      withTempDir { dir =>
+        withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "false") {
+          setupPartitionedDatasourceTable("test", dir)
+          assert(spark.sql("select * from test").count() == 5)
+        }
+        withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") {
+          verifyIsLegacyTable("test")
+          spark.sql("msck repair table test")
+          spark.sql("show partitions test").count()  // check we are a new 
table
+
+          // sanity check table performance
+          HiveCatalogMetrics.reset()
+          assert(spark.sql("select * from test where partCol < 2").count() == 
2)
+          assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 2)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 2)
+        }
+      }
+    }
+  }
+
+  test("when partition management is enabled, new tables have partition 
provider hive") {
+    withTable("test") {
+      withTempDir { dir =>
+        withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") {
+          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")
+          assert(spark.sql("select * from test").count() == 5)
+        }
+      }
+    }
+  }
+
+  test("when partition management is disabled, new tables have no partition 
provider") {
+    withTable("test") {
+      withTempDir { dir =>
+        withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "false") {
+          setupPartitionedDatasourceTable("test", dir)
+          verifyIsLegacyTable("test")
+          assert(spark.sql("select * from test").count() == 5)
+        }
+      }
+    }
+  }
+
+  test("when partition management is disabled, we preserve the old behavior 
even for new tables") {
+    withTable("test") {
+      withTempDir { dir =>
+        withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") {
+          setupPartitionedDatasourceTable("test", dir)
+          spark.sql("show partitions test").count()  // check we are a new 
table
+          spark.sql("refresh table test")
+          assert(spark.sql("select * from test").count() == 0)
+        }
+        // disabled
+        withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "false") {
+          val e = intercept[AnalysisException] {
+            spark.sql(s"show partitions test")
+          }
+          assert(e.getMessage.contains("filesource partition management is 
disabled"))
+          spark.sql("refresh table test")
+          assert(spark.sql("select * from test").count() == 5)
+        }
+        // then enabled again
+        withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") {
+          spark.sql("refresh table test")
+          assert(spark.sql("select * from test").count() == 0)
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/ccb11543/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala
new file mode 100644
index 0000000..476383a
--- /dev/null
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive
+
+import java.io.File
+
+import org.scalatest.BeforeAndAfterEach
+
+import org.apache.spark.metrics.source.HiveCatalogMetrics
+import org.apache.spark.sql.execution.datasources.FileStatusCache
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SQLTestUtils
+
+class PartitionedTablePerfStatsSuite
+  extends QueryTest with TestHiveSingleton with SQLTestUtils with 
BeforeAndAfterEach {
+
+  override def beforeEach(): Unit = {
+    super.beforeEach()
+    FileStatusCache.resetForTesting()
+  }
+
+  override def afterEach(): Unit = {
+    super.afterEach()
+    FileStatusCache.resetForTesting()
+  }
+
+  private case class TestSpec(setupTable: (String, File) => Unit, 
isDatasourceTable: Boolean)
+
+  /**
+   * Runs a test against both converted hive and native datasource tables. The 
test can use the
+   * passed TestSpec object for setup and inspecting test parameters.
+   */
+  private def genericTest(testName: String)(fn: TestSpec => Unit): Unit = {
+    test("hive table: " + testName) {
+      fn(TestSpec(setupPartitionedHiveTable, false))
+    }
+    test("datasource table: " + testName) {
+      fn(TestSpec(setupPartitionedDatasourceTable, true))
+    }
+  }
+
+  private def setupPartitionedHiveTable(tableName: String, dir: File): Unit = {
+    spark.range(5).selectExpr("id as fieldOne", "id as partCol1", "id as 
partCol2").write
+      .partitionBy("partCol1", "partCol2")
+      .mode("overwrite")
+      .parquet(dir.getAbsolutePath)
+
+    spark.sql(s"""
+      |create external table $tableName (fieldOne long)
+      |partitioned by (partCol1 int, partCol2 int)
+      |stored as parquet
+      |location "${dir.getAbsolutePath}"""".stripMargin)
+    spark.sql(s"msck repair table $tableName")
+  }
+
+  private def setupPartitionedDatasourceTable(tableName: String, dir: File): 
Unit = {
+    spark.range(5).selectExpr("id as fieldOne", "id as partCol1", "id as 
partCol2").write
+      .partitionBy("partCol1", "partCol2")
+      .mode("overwrite")
+      .parquet(dir.getAbsolutePath)
+
+    spark.sql(s"""
+      |create table $tableName (fieldOne long, partCol1 int, partCol2 int)
+      |using parquet
+      |options (path "${dir.getAbsolutePath}")
+      |partitioned by (partCol1, partCol2)""".stripMargin)
+    spark.sql(s"msck repair table $tableName")
+  }
+
+  genericTest("partitioned pruned table reports only selected files") { spec =>
+    assert(spark.sqlContext.getConf(HiveUtils.CONVERT_METASTORE_PARQUET.key) 
== "true")
+    withTable("test") {
+      withTempDir { dir =>
+        spec.setupTable("test", dir)
+        val df = spark.sql("select * from test")
+        assert(df.count() == 5)
+        assert(df.inputFiles.length == 5)  // unpruned
+
+        val df2 = spark.sql("select * from test where partCol1 = 3 or partCol2 
= 4")
+        assert(df2.count() == 2)
+        assert(df2.inputFiles.length == 2)  // pruned, so we have less files
+
+        val df3 = spark.sql("select * from test where PARTCOL1 = 3 or partcol2 
= 4")
+        assert(df3.count() == 2)
+        assert(df3.inputFiles.length == 2)
+
+        val df4 = spark.sql("select * from test where partCol1 = 999")
+        assert(df4.count() == 0)
+        assert(df4.inputFiles.length == 0)
+
+        // TODO(ekl) enable for hive tables as well once SPARK-17983 is fixed
+        if (spec.isDatasourceTable) {
+          val df5 = spark.sql("select * from test where fieldOne = 4")
+          assert(df5.count() == 1)
+          assert(df5.inputFiles.length == 5)
+        }
+      }
+    }
+  }
+
+  genericTest("lazy partition pruning reads only necessary partition data") { 
spec =>
+    withSQLConf(
+        SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true",
+        SQLConf.HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE.key -> "0") {
+      withTable("test") {
+        withTempDir { dir =>
+          spec.setupTable("test", dir)
+          HiveCatalogMetrics.reset()
+          spark.sql("select * from test where partCol1 = 999").count()
+          assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 0)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0)
+
+          HiveCatalogMetrics.reset()
+          spark.sql("select * from test where partCol1 < 2").count()
+          assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 2)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 2)
+
+          HiveCatalogMetrics.reset()
+          spark.sql("select * from test where partCol1 < 3").count()
+          assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 3)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 3)
+
+          // should read all
+          HiveCatalogMetrics.reset()
+          spark.sql("select * from test").count()
+          assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5)
+
+          // read all should not be cached
+          HiveCatalogMetrics.reset()
+          spark.sql("select * from test").count()
+          assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5)
+
+          // cache should be disabled
+          assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0)
+        }
+      }
+    }
+  }
+
+  genericTest("lazy partition pruning with file status caching enabled") { 
spec =>
+    withSQLConf(
+        SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true",
+        SQLConf.HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE.key -> "9999999") {
+      withTable("test") {
+        withTempDir { dir =>
+          spec.setupTable("test", dir)
+          HiveCatalogMetrics.reset()
+          assert(spark.sql("select * from test where partCol1 = 999").count() 
== 0)
+          assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 0)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0)
+          assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0)
+
+          HiveCatalogMetrics.reset()
+          assert(spark.sql("select * from test where partCol1 < 2").count() == 
2)
+          assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 2)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 2)
+          assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0)
+
+          HiveCatalogMetrics.reset()
+          assert(spark.sql("select * from test where partCol1 < 3").count() == 
3)
+          assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 3)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 1)
+          assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 2)
+
+          HiveCatalogMetrics.reset()
+          assert(spark.sql("select * from test").count() == 5)
+          assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 2)
+          assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 3)
+
+          HiveCatalogMetrics.reset()
+          assert(spark.sql("select * from test").count() == 5)
+          assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0)
+          assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 5)
+        }
+      }
+    }
+  }
+
+  genericTest("file status caching respects refresh table and refreshByPath") 
{ spec =>
+    withSQLConf(
+        SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true",
+        SQLConf.HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE.key -> "9999999") {
+      withTable("test") {
+        withTempDir { dir =>
+          spec.setupTable("test", dir)
+          HiveCatalogMetrics.reset()
+          assert(spark.sql("select * from test").count() == 5)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5)
+          assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0)
+
+          HiveCatalogMetrics.reset()
+          spark.sql("refresh table test")
+          assert(spark.sql("select * from test").count() == 5)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5)
+          assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0)
+
+          spark.catalog.cacheTable("test")
+          HiveCatalogMetrics.reset()
+          spark.catalog.refreshByPath(dir.getAbsolutePath)
+          assert(spark.sql("select * from test").count() == 5)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5)
+          assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0)
+        }
+      }
+    }
+  }
+
+  genericTest("file status cache respects size limit") { spec =>
+    withSQLConf(
+        SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true",
+        SQLConf.HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE.key -> "1" /* 1 byte 
*/) {
+      withTable("test") {
+        withTempDir { dir =>
+          spec.setupTable("test", dir)
+          HiveCatalogMetrics.reset()
+          assert(spark.sql("select * from test").count() == 5)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5)
+          assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0)
+          assert(spark.sql("select * from test").count() == 5)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 10)
+          assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0)
+        }
+      }
+    }
+  }
+
+  test("hive table: files read and cached when filesource partition management 
is off") {
+    withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "false") {
+      withTable("test") {
+        withTempDir { dir =>
+          setupPartitionedHiveTable("test", dir)
+
+          // We actually query the partitions from hive each time the table is 
resolved in this
+          // mode. This is kind of terrible, but is needed to preserve the 
legacy behavior
+          // of doing plan cache validation based on the entire partition set.
+          HiveCatalogMetrics.reset()
+          assert(spark.sql("select * from test where partCol1 = 999").count() 
== 0)
+          // 5 from table resolution, another 5 from ListingFileCatalog
+          assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 10)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5)
+
+          HiveCatalogMetrics.reset()
+          assert(spark.sql("select * from test where partCol1 < 2").count() == 
2)
+          assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0)
+
+          HiveCatalogMetrics.reset()
+          assert(spark.sql("select * from test").count() == 5)
+          assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0)
+        }
+      }
+    }
+  }
+
+  test("datasource table: all partition data cached in memory when partition 
management is off") {
+    withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "false") {
+      withTable("test") {
+        withTempDir { dir =>
+          setupPartitionedDatasourceTable("test", dir)
+          HiveCatalogMetrics.reset()
+          assert(spark.sql("select * from test where partCol1 = 999").count() 
== 0)
+
+          // not using metastore
+          assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 0)
+
+          // reads and caches all the files initially
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5)
+
+          HiveCatalogMetrics.reset()
+          assert(spark.sql("select * from test where partCol1 < 2").count() == 
2)
+          assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 0)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0)
+
+          HiveCatalogMetrics.reset()
+          assert(spark.sql("select * from test").count() == 5)
+          assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 0)
+          assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0)
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/ccb11543/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
index c351063..4f5ebc3 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
@@ -310,39 +310,50 @@ class StatisticsSuite extends QueryTest with 
TestHiveSingleton with SQLTestUtils
     }
   }
 
-  test("test table-level statistics for data source table created in 
HiveExternalCatalog") {
-    val parquetTable = "parquetTable"
-    withTable(parquetTable) {
-      sql(s"CREATE TABLE $parquetTable (key STRING, value STRING) USING 
PARQUET")
-      val catalogTable = 
spark.sessionState.catalog.getTableMetadata(TableIdentifier(parquetTable))
-      assert(DDLUtils.isDatasourceTable(catalogTable))
+  private def testUpdatingTableStats(tableDescription: String, createTableCmd: 
String): Unit = {
+    test("test table-level statistics for " + tableDescription) {
+      val parquetTable = "parquetTable"
+      withTable(parquetTable) {
+        sql(createTableCmd)
+        val catalogTable = spark.sessionState.catalog.getTableMetadata(
+          TableIdentifier(parquetTable))
+        assert(DDLUtils.isDatasourceTable(catalogTable))
+
+        sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src")
+        checkTableStats(
+          parquetTable, isDataSourceTable = true, hasSizeInBytes = false, 
expectedRowCounts = None)
 
-      sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src")
-      checkTableStats(
-        parquetTable, isDataSourceTable = true, hasSizeInBytes = false, 
expectedRowCounts = None)
+        // noscan won't count the number of rows
+        sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS noscan")
+        val fetchedStats1 = checkTableStats(
+          parquetTable, isDataSourceTable = true, hasSizeInBytes = true, 
expectedRowCounts = None)
 
-      // noscan won't count the number of rows
-      sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS noscan")
-      val fetchedStats1 = checkTableStats(
-        parquetTable, isDataSourceTable = true, hasSizeInBytes = true, 
expectedRowCounts = None)
+        sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src")
+        sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS noscan")
+        val fetchedStats2 = checkTableStats(
+          parquetTable, isDataSourceTable = true, hasSizeInBytes = true, 
expectedRowCounts = None)
+        assert(fetchedStats2.get.sizeInBytes > fetchedStats1.get.sizeInBytes)
 
-      sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src")
-      sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS noscan")
-      val fetchedStats2 = checkTableStats(
-        parquetTable, isDataSourceTable = true, hasSizeInBytes = true, 
expectedRowCounts = None)
-      assert(fetchedStats2.get.sizeInBytes > fetchedStats1.get.sizeInBytes)
-
-      // without noscan, we count the number of rows
-      sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS")
-      val fetchedStats3 = checkTableStats(
-        parquetTable,
-        isDataSourceTable = true,
-        hasSizeInBytes = true,
-        expectedRowCounts = Some(1000))
-      assert(fetchedStats3.get.sizeInBytes == fetchedStats2.get.sizeInBytes)
+        // without noscan, we count the number of rows
+        sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS")
+        val fetchedStats3 = checkTableStats(
+          parquetTable,
+          isDataSourceTable = true,
+          hasSizeInBytes = true,
+          expectedRowCounts = Some(1000))
+        assert(fetchedStats3.get.sizeInBytes == fetchedStats2.get.sizeInBytes)
+      }
     }
   }
 
+  testUpdatingTableStats(
+    "data source table created in HiveExternalCatalog",
+    "CREATE TABLE parquetTable (key STRING, value STRING) USING PARQUET")
+
+  testUpdatingTableStats(
+    "partitioned data source table",
+    "CREATE TABLE parquetTable (key STRING, value STRING) USING PARQUET 
PARTITIONED BY (key)")
+
   test("statistics collection of a table with zero column") {
     val table_no_cols = "table_no_cols"
     withTable(table_no_cols) {

http://git-wip-us.apache.org/repos/asf/spark/blob/ccb11543/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala
index ad1e9b1..46ed18c 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala
@@ -415,10 +415,7 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils 
with TestHiveSingleto
         .mode(SaveMode.Overwrite)
         .saveAsTable("part_datasrc")
 
-      val message1 = intercept[AnalysisException] {
-        sql("SHOW PARTITIONS part_datasrc")
-      }.getMessage
-      assert(message1.contains("is not allowed on a datasource table"))
+      assert(sql("SHOW PARTITIONS part_datasrc").count() == 3)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/ccb11543/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
index 495b4f8..01fa827 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
@@ -358,7 +358,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils 
with TestHiveSingleton {
         "# Partition Information",
         "# col_name",
         "Detailed Partition Information CatalogPartition(",
-        "Partition Values: [Us, 1]",
+        "Partition Values: [c=Us, d=1]",
         "Storage(Location:",
         "Partition Parameters")
 
@@ -399,10 +399,8 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils 
with TestHiveSingleton {
         .range(1).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd).write
         .partitionBy("d")
         .saveAsTable("datasource_table")
-      val m4 = intercept[AnalysisException] {
-        sql("DESC datasource_table PARTITION (d=2)")
-      }.getMessage()
-      assert(m4.contains("DESC PARTITION is not allowed on a datasource 
table"))
+
+      sql("DESC datasource_table PARTITION (d=0)")
 
       val m5 = intercept[AnalysisException] {
         spark.range(10).select('id as 'a, 'id as 'b).createTempView("view1")


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to