[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-22 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/14971


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-18 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117393090
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -215,6 +218,215 @@ class StatisticsSuite extends 
StatisticsCollectionTestBase with TestHiveSingleto
 }
   }
 
+  private def createNonPartitionedTable(
+  tabName: String,
+  analyzedBySpark: Boolean = true,
+  analyzedByHive: Boolean = true): Unit = {
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+sql(
+  s"""
+ |CREATE TABLE $tabName (key STRING, value STRING)
+ |STORED AS TEXTFILE
+ |TBLPROPERTIES ('prop1' = 'val1', 'prop2' = 'val2')
+   """.stripMargin)
+sql(s"INSERT INTO TABLE $tabName SELECT * FROM src")
+if (analyzedBySpark) sql(s"ANALYZE TABLE $tabName COMPUTE STATISTICS")
+// This is to mimic the scenario in which Hive genrates statistics 
before we reading it
+if (analyzedByHive) hiveClient.runSqlHive(s"ANALYZE TABLE $tabName 
COMPUTE STATISTICS")
+val describeResult1 = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+
+val tableMetadata =
+  
spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName)).properties
+// statistics info is not contained in the metadata of the original 
table
+assert(Seq(StatsSetupConst.COLUMN_STATS_ACCURATE,
+  StatsSetupConst.NUM_FILES,
+  StatsSetupConst.NUM_PARTITIONS,
+  StatsSetupConst.ROW_COUNT,
+  StatsSetupConst.RAW_DATA_SIZE,
+  StatsSetupConst.TOTAL_SIZE).forall(!tableMetadata.contains(_)))
+
+if (analyzedByHive) {
+  assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").nonEmpty)
+} else {
+  assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").isEmpty)
+}
+  }
+
+  private def extractStatsPropValues(
+  descOutput: Seq[String],
+  propKey: String): Option[BigInt] = {
+val str = descOutput
+  .filterNot(_.contains(HiveExternalCatalog.STATISTICS_PREFIX))
+  .filter(_.contains(propKey))
+if (str.isEmpty) {
+  None
+} else {
+  assert(str.length == 1, "found more than one matches")
+  val pattern = new Regex(s"""$propKey\\s+(-?\\d+)""")
+  val pattern(value) = str.head.trim
+  Option(BigInt(value))
+}
+  }
+
+  test("get statistics when not analyzed in both Hive and Spark") {
+val tabName = "tab1"
+withTable(tabName) {
+  createNonPartitionedTable(tabName, analyzedByHive = false, 
analyzedBySpark = false)
+  checkTableStats(
+tabName, hasSizeInBytes = true, expectedRowCounts = None)
+
+  // ALTER TABLE SET TBLPROPERTIES invalidates some contents of Hive 
specific statistics
+  // This is triggered by the Hive alterTable API
+  val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+  val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+
+  val rawDataSize = extractStatsPropValues(describeResult, 
"rawDataSize")
+  val numRows = extractStatsPropValues(describeResult, "numRows")
+  val totalSize = extractStatsPropValues(describeResult, "totalSize")
+  assert(rawDataSize.isEmpty, "rawDataSize should not be shown without 
table analysis")
+  assert(numRows.isEmpty, "numRows should not be shown without table 
analysis")
+  assert(totalSize.isDefined && totalSize.get > 0, "totalSize is lost")
+}
+  }
+
+  test("alter table rename after analyze table") {
+Seq(true, false).foreach { analyzedBySpark =>
+  val oldName = "tab1"
+  val newName = "tab2"
+  withTable(oldName, newName) {
+createNonPartitionedTable(oldName, analyzedByHive = true, 
analyzedBySpark = analyzedBySpark)
+val fetchedStats1 = checkTableStats(
+  oldName, hasSizeInBytes = true, expectedRowCounts = Some(500))
+sql(s"ALTER TABLE $oldName RENAME TO $newName")
+val fetchedStats2 = checkTableStats(
+  newName, hasSizeInBytes = true, expectedRowCounts = Some(500))
+assert(fetchedStats1 == fetchedStats2)
+
+// ALTER TABLE RENAME does not affect the contents of Hive 
specific statistics
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$newName")
+
+val rawDataSize = extractStatsPropValues(describeResult, 
"rawDataSize")
+val numRows = extractStatsPropValues(describeResult, "numRows")
   

[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-18 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117392683
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala 
---
@@ -414,6 +415,50 @@ private[hive] class HiveClientImpl(
 
   val properties = Option(h.getParameters).map(_.asScala.toMap).orNull
 
+  // Hive-generated Statistics are also recorded in ignoredProperties
+  val ignoredProperties = scala.collection.mutable.Map.empty[String, 
String]
+  for (key <- HiveStatisticsProperties; value <- properties.get(key)) {
+ignoredProperties += key -> value
+  }
+
+  val excludedTableProperties = HiveStatisticsProperties ++ Set(
+// The property value of "comment" is moved to the dedicated field 
"comment"
+"comment",
+// For EXTERNAL_TABLE, the table properties has a particular field 
"EXTERNAL". This is added
+// in the function toHiveTable.
+"EXTERNAL"
+  )
+
+  val filteredProperties = properties.filterNot {
+case (key, _) => excludedTableProperties.contains(key)
+  }
+  val comment = properties.get("comment")
+
+  val totalSize = 
properties.get(StatsSetupConst.TOTAL_SIZE).map(BigInt(_))
+  val rawDataSize = 
properties.get(StatsSetupConst.RAW_DATA_SIZE).map(BigInt(_))
+  lazy val rowCount = 
properties.get(StatsSetupConst.ROW_COUNT).map(BigInt(_)) match {
--- End diff --

1. I think we can just use val, no need to bother about performance here.
2. can be simplified to `xxx.filter(_ >= 0)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-18 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117371477
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -175,7 +178,7 @@ class StatisticsSuite extends 
StatisticsCollectionTestBase with TestHiveSingleto
   sql(s"INSERT INTO TABLE $textTable SELECT * FROM src")
   checkTableStats(
 textTable,
-hasSizeInBytes = false,
+hasSizeInBytes = true,
--- End diff --

It sounds like Hive does online stats updates. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-18 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117370839
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -215,6 +218,217 @@ class StatisticsSuite extends 
StatisticsCollectionTestBase with TestHiveSingleto
 }
   }
 
+  private def createNonPartitionedTable(
+  tabName: String,
+  analyzedBySpark: Boolean = true,
+  analyzedByHive: Boolean = true): Unit = {
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+sql(
+  s"""
+ |CREATE TABLE $tabName (key STRING, value STRING)
+ |STORED AS TEXTFILE
+ |TBLPROPERTIES ('prop1' = 'val1', 'prop2' = 'val2')
+   """.stripMargin)
+sql(s"INSERT INTO TABLE $tabName SELECT * FROM src")
+if (analyzedBySpark) sql(s"ANALYZE TABLE $tabName COMPUTE STATISTICS")
+// This is to mimic the scenario in which Hive genrates statistics 
before we reading it
+if (analyzedByHive) hiveClient.runSqlHive(s"ANALYZE TABLE $tabName 
COMPUTE STATISTICS")
+val describeResult1 = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+
+val tableMetadata =
+  
spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName)).properties
+// statistics info is not contained in the metadata of the original 
table
+assert(Seq(StatsSetupConst.COLUMN_STATS_ACCURATE,
+  StatsSetupConst.NUM_FILES,
+  StatsSetupConst.NUM_PARTITIONS,
+  StatsSetupConst.ROW_COUNT,
+  StatsSetupConst.RAW_DATA_SIZE,
+  StatsSetupConst.TOTAL_SIZE).forall(!tableMetadata.contains(_)))
+
+if (analyzedByHive) {
+  assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").nonEmpty)
+} else {
+  assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").isEmpty)
+}
+  }
+
+  private def extractStatsPropValues(
+  descOutput: Seq[String],
+  propKey: String): Option[BigInt] = {
+val str = descOutput
+  .filterNot(_.contains(HiveExternalCatalog.STATISTICS_PREFIX))
+  .filter(_.contains(propKey))
+if (str.isEmpty) {
+  None
+} else {
+  assert(str.length == 1, "found more than one matches")
+  val pattern = new Regex(s"""$propKey\\s+(-?\\d+)""")
+  val pattern(value) = str.head.trim
+  Option(BigInt(value))
+}
+  }
+
+  test("get statistics when not analyzed in both Hive and Spark") {
+val tabName = "tab1"
+withTable(tabName) {
+  createNonPartitionedTable(tabName, analyzedByHive = false, 
analyzedBySpark = false)
+  checkTableStats(
+tabName, hasSizeInBytes = true, expectedRowCounts = None)
+
+  // ALTER TABLE SET TBLPROPERTIES invalidates some contents of Hive 
specific statistics
+  // This is triggered by the Hive alterTable API
+  val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+  val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+
+  val rawDataSize = extractStatsPropValues(describeResult, 
"rawDataSize")
+  val numRows = extractStatsPropValues(describeResult, "numRows")
+  val totalSize = extractStatsPropValues(describeResult, "totalSize")
+  assert(rawDataSize.isEmpty, "rawDataSize should not be shown without 
table analysis")
+  assert(numRows.isEmpty, "numRows should not be shown without table 
analysis")
+  assert(totalSize.isDefined && totalSize.get > 0, "totalSize is lost")
+}
+  }
+
+  test("alter table rename after analyze table") {
+Seq(true, false).foreach { analyzedBySpark =>
+  val oldName = "tab1"
+  val newName = "tab2"
+  withTable(oldName, newName) {
+createNonPartitionedTable(oldName, analyzedByHive = true, 
analyzedBySpark = analyzedBySpark)
+val fetchedStats1 = checkTableStats(
+  oldName, hasSizeInBytes = true, expectedRowCounts = Some(500))
+sql(s"ALTER TABLE $oldName RENAME TO $newName")
+val fetchedStats2 = checkTableStats(
+  newName, hasSizeInBytes = true, expectedRowCounts = Some(500))
+assert(fetchedStats1 == fetchedStats2)
+
+// ALTER TABLE RENAME does not affect the contents of Hive 
specific statistics
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$newName")
+
+val rawDataSize = extractStatsPropValues(describeResult, 
"rawDataSize")
+val numRows = extractStatsPropValues(describeResult, "numRows")
  

[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-18 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117366221
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala 
---
@@ -414,6 +415,50 @@ private[hive] class HiveClientImpl(
 
   val properties = Option(h.getParameters).map(_.asScala.toMap).orNull
 
+  // Hive-generated Statistics are also recorded in ignoredProperties
+  val ignoredProperties = scala.collection.mutable.Map.empty[String, 
String]
+  for (key <- HiveStatisticsProperties; value <- properties.get(key)) {
+ignoredProperties += key -> value
+  }
+
+  val excludedTableProperties = HiveStatisticsProperties ++ Set(
+// The property value of "comment" is moved to the dedicated field 
"comment"
+"comment",
+// For EXTERNAL_TABLE, the table properties has a particular field 
"EXTERNAL". This is added
+// in the function toHiveTable.
+"EXTERNAL"
+  )
+
+  val filteredProperties = properties.filterNot {
+case (key, _) => excludedTableProperties.contains(key)
+  }
+  val comment = properties.get("comment")
+
+  val totalSize = 
properties.get(StatsSetupConst.TOTAL_SIZE).map(BigInt(_))
+  val rawDataSize = 
properties.get(StatsSetupConst.RAW_DATA_SIZE).map(BigInt(_))
+  def rowCount = 
properties.get(StatsSetupConst.ROW_COUNT).map(BigInt(_)) match {
--- End diff --

only used once. We also can use lazy val


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-18 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117365748
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
 ---
@@ -192,13 +192,7 @@ abstract class HiveComparisonTest
 "last_modified_by",
 "last_modified_time",
 "Owner:",
-"COLUMN_STATS_ACCURATE",
 // The following are hive specific schema parameters which we do not 
need to match exactly.
-"numFiles",
-"numRows",
-"rawDataSize",
-"totalSize",
-"totalNumberFiles",
--- End diff --

I think we should keep it unchanged.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-18 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117365577
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -232,7 +446,8 @@ class StatisticsSuite extends 
StatisticsCollectionTestBase with TestHiveSingleto
 checkTableStats(parquetTable, hasSizeInBytes = true, 
expectedRowCounts = Some(500))
   }
   withSQLConf("spark.sql.hive.convertMetastoreOrc" -> "true") {
-checkTableStats(orcTable, hasSizeInBytes = false, 
expectedRowCounts = None)
+// We still can get tableSize from Hive before Analyze
+checkTableStats(orcTable, hasSizeInBytes = true, expectedRowCounts 
= None)
--- End diff --

A good question. This is from Hive. : ( I did not investigate the root 
cause inside Hive.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-18 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117365344
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
 ---
@@ -192,13 +192,7 @@ abstract class HiveComparisonTest
 "last_modified_by",
 "last_modified_time",
 "Owner:",
-"COLUMN_STATS_ACCURATE",
 // The following are hive specific schema parameters which we do not 
need to match exactly.
-"numFiles",
-"numRows",
-"rawDataSize",
-"totalSize",
-"totalNumberFiles",
--- End diff --

`TextMetaDataFormatter` and `JsonMetaDataFormatter` insert these info based 
on `numFiles`. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-18 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117175951
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -175,7 +178,7 @@ class StatisticsSuite extends 
StatisticsCollectionTestBase with TestHiveSingleto
   sql(s"INSERT INTO TABLE $textTable SELECT * FROM src")
   checkTableStats(
 textTable,
-hasSizeInBytes = false,
+hasSizeInBytes = true,
--- End diff --

Because now in `HiveClientImpl.getTableOption`, we respect Hive's stats.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-18 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117176108
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -175,7 +178,7 @@ class StatisticsSuite extends 
StatisticsCollectionTestBase with TestHiveSingleto
   sql(s"INSERT INTO TABLE $textTable SELECT * FROM src")
   checkTableStats(
 textTable,
-hasSizeInBytes = false,
+hasSizeInBytes = true,
--- End diff --

Hive will alter totalSize after inserting data.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-18 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117174966
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala 
---
@@ -414,6 +415,50 @@ private[hive] class HiveClientImpl(
 
   val properties = Option(h.getParameters).map(_.asScala.toMap).orNull
 
+  // Hive-generated Statistics are also recorded in ignoredProperties
+  val ignoredProperties = scala.collection.mutable.Map.empty[String, 
String]
+  for (key <- HiveStatisticsProperties; value <- properties.get(key)) {
+ignoredProperties += key -> value
+  }
+
+  val excludedTableProperties = HiveStatisticsProperties ++ Set(
+// The property value of "comment" is moved to the dedicated field 
"comment"
+"comment",
+// For EXTERNAL_TABLE, the table properties has a particular field 
"EXTERNAL". This is added
+// in the function toHiveTable.
+"EXTERNAL"
+  )
+
+  val filteredProperties = properties.filterNot {
+case (key, _) => excludedTableProperties.contains(key)
+  }
+  val comment = properties.get("comment")
+
+  val totalSize = 
properties.get(StatsSetupConst.TOTAL_SIZE).map(BigInt(_))
+  val rawDataSize = 
properties.get(StatsSetupConst.RAW_DATA_SIZE).map(BigInt(_))
+  def rowCount = 
properties.get(StatsSetupConst.ROW_COUNT).map(BigInt(_)) match {
--- End diff --

why use def?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-18 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117169136
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
 ---
@@ -192,13 +192,7 @@ abstract class HiveComparisonTest
 "last_modified_by",
 "last_modified_time",
 "Owner:",
-"COLUMN_STATS_ACCURATE",
 // The following are hive specific schema parameters which we do not 
need to match exactly.
-"numFiles",
-"numRows",
-"rawDataSize",
-"totalSize",
-"totalNumberFiles",
--- End diff --

Is `totalNumberFiles` the same as `numFiles`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-18 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117168812
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -215,6 +218,217 @@ class StatisticsSuite extends 
StatisticsCollectionTestBase with TestHiveSingleto
 }
   }
 
+  private def createNonPartitionedTable(
+  tabName: String,
+  analyzedBySpark: Boolean = true,
+  analyzedByHive: Boolean = true): Unit = {
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+sql(
+  s"""
+ |CREATE TABLE $tabName (key STRING, value STRING)
+ |STORED AS TEXTFILE
+ |TBLPROPERTIES ('prop1' = 'val1', 'prop2' = 'val2')
+   """.stripMargin)
+sql(s"INSERT INTO TABLE $tabName SELECT * FROM src")
+if (analyzedBySpark) sql(s"ANALYZE TABLE $tabName COMPUTE STATISTICS")
+// This is to mimic the scenario in which Hive genrates statistics 
before we reading it
+if (analyzedByHive) hiveClient.runSqlHive(s"ANALYZE TABLE $tabName 
COMPUTE STATISTICS")
+val describeResult1 = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+
+val tableMetadata =
+  
spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName)).properties
+// statistics info is not contained in the metadata of the original 
table
+assert(Seq(StatsSetupConst.COLUMN_STATS_ACCURATE,
+  StatsSetupConst.NUM_FILES,
+  StatsSetupConst.NUM_PARTITIONS,
+  StatsSetupConst.ROW_COUNT,
+  StatsSetupConst.RAW_DATA_SIZE,
+  StatsSetupConst.TOTAL_SIZE).forall(!tableMetadata.contains(_)))
+
+if (analyzedByHive) {
+  assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").nonEmpty)
+} else {
+  assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").isEmpty)
+}
+  }
+
+  private def extractStatsPropValues(
+  descOutput: Seq[String],
+  propKey: String): Option[BigInt] = {
+val str = descOutput
+  .filterNot(_.contains(HiveExternalCatalog.STATISTICS_PREFIX))
+  .filter(_.contains(propKey))
+if (str.isEmpty) {
+  None
+} else {
+  assert(str.length == 1, "found more than one matches")
+  val pattern = new Regex(s"""$propKey\\s+(-?\\d+)""")
+  val pattern(value) = str.head.trim
+  Option(BigInt(value))
+}
+  }
+
+  test("get statistics when not analyzed in both Hive and Spark") {
+val tabName = "tab1"
+withTable(tabName) {
+  createNonPartitionedTable(tabName, analyzedByHive = false, 
analyzedBySpark = false)
+  checkTableStats(
+tabName, hasSizeInBytes = true, expectedRowCounts = None)
+
+  // ALTER TABLE SET TBLPROPERTIES invalidates some contents of Hive 
specific statistics
+  // This is triggered by the Hive alterTable API
+  val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+  val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+
+  val rawDataSize = extractStatsPropValues(describeResult, 
"rawDataSize")
+  val numRows = extractStatsPropValues(describeResult, "numRows")
+  val totalSize = extractStatsPropValues(describeResult, "totalSize")
+  assert(rawDataSize.isEmpty, "rawDataSize should not be shown without 
table analysis")
+  assert(numRows.isEmpty, "numRows should not be shown without table 
analysis")
+  assert(totalSize.isDefined && totalSize.get > 0, "totalSize is lost")
+}
+  }
+
+  test("alter table rename after analyze table") {
+Seq(true, false).foreach { analyzedBySpark =>
+  val oldName = "tab1"
+  val newName = "tab2"
+  withTable(oldName, newName) {
+createNonPartitionedTable(oldName, analyzedByHive = true, 
analyzedBySpark = analyzedBySpark)
+val fetchedStats1 = checkTableStats(
+  oldName, hasSizeInBytes = true, expectedRowCounts = Some(500))
+sql(s"ALTER TABLE $oldName RENAME TO $newName")
+val fetchedStats2 = checkTableStats(
+  newName, hasSizeInBytes = true, expectedRowCounts = Some(500))
+assert(fetchedStats1 == fetchedStats2)
+
+// ALTER TABLE RENAME does not affect the contents of Hive 
specific statistics
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$newName")
+
+val rawDataSize = extractStatsPropValues(describeResult, 
"rawDataSize")
+val numRows = extractStatsPropValues(describeResult, "numRows")
+  

[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-18 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117156770
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala 
---
@@ -414,6 +415,50 @@ private[hive] class HiveClientImpl(
 
   val properties = Option(h.getParameters).map(_.asScala.toMap).orNull
 
+  // Hive-generated Statistics are also recorded in ignoredProperties
+  val ignoredProperties = scala.collection.mutable.Map.empty[String, 
String]
+  for (key <- HiveStatisticsProperties; value <- properties.get(key)) {
+ignoredProperties += key -> value
+  }
+
+  val excludedTableProperties = HiveStatisticsProperties ++ Set(
+// The property value of "comment" is moved to the dedicated field 
"comment"
+"comment",
+// For EXTERNAL_TABLE, the table properties has a particular field 
"EXTERNAL". This is added
+// in the function toHiveTable.
+"EXTERNAL"
+  )
+
+  val filteredProperties = properties.filterNot {
+case (key, _) => excludedTableProperties.contains(key)
+  }
+  val comment = properties.get("comment")
+
+  val totalSize = 
properties.get(StatsSetupConst.TOTAL_SIZE).map(BigInt(_))
+  val rawDataSize = 
properties.get(StatsSetupConst.RAW_DATA_SIZE).map(BigInt(_))
+  def rowCount = 
properties.get(StatsSetupConst.ROW_COUNT).map(BigInt(_)) match {
+case Some(c) if c >= 0 => Some(c)
+case _ => None
+  }
+  // TODO: check if this estimate is valid for tables after partition 
pruning.
+  // NOTE: getting `totalSize` directly from params is kind of hacky, 
but this should be
+  // relatively cheap if parameters for the table are populated into 
the metastore.
+  // Currently, only totalSize, rawDataSize, and row_count are used to 
build the field `stats`
--- End diff --

nit: rowCount


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-18 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117169168
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -232,7 +446,8 @@ class StatisticsSuite extends 
StatisticsCollectionTestBase with TestHiveSingleto
 checkTableStats(parquetTable, hasSizeInBytes = true, 
expectedRowCounts = Some(500))
   }
   withSQLConf("spark.sql.hive.convertMetastoreOrc" -> "true") {
-checkTableStats(orcTable, hasSizeInBytes = false, 
expectedRowCounts = None)
+// We still can get tableSize from Hive before Analyze
+checkTableStats(orcTable, hasSizeInBytes = true, expectedRowCounts 
= None)
--- End diff --

Orc table has size from Hive, while parquet table doesn't?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-18 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117172781
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -215,6 +218,217 @@ class StatisticsSuite extends 
StatisticsCollectionTestBase with TestHiveSingleto
 }
   }
 
+  private def createNonPartitionedTable(
+  tabName: String,
+  analyzedBySpark: Boolean = true,
+  analyzedByHive: Boolean = true): Unit = {
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+sql(
+  s"""
+ |CREATE TABLE $tabName (key STRING, value STRING)
+ |STORED AS TEXTFILE
+ |TBLPROPERTIES ('prop1' = 'val1', 'prop2' = 'val2')
+   """.stripMargin)
+sql(s"INSERT INTO TABLE $tabName SELECT * FROM src")
+if (analyzedBySpark) sql(s"ANALYZE TABLE $tabName COMPUTE STATISTICS")
+// This is to mimic the scenario in which Hive genrates statistics 
before we reading it
+if (analyzedByHive) hiveClient.runSqlHive(s"ANALYZE TABLE $tabName 
COMPUTE STATISTICS")
+val describeResult1 = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+
+val tableMetadata =
+  
spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName)).properties
+// statistics info is not contained in the metadata of the original 
table
+assert(Seq(StatsSetupConst.COLUMN_STATS_ACCURATE,
+  StatsSetupConst.NUM_FILES,
+  StatsSetupConst.NUM_PARTITIONS,
+  StatsSetupConst.ROW_COUNT,
+  StatsSetupConst.RAW_DATA_SIZE,
+  StatsSetupConst.TOTAL_SIZE).forall(!tableMetadata.contains(_)))
+
+if (analyzedByHive) {
+  assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").nonEmpty)
+} else {
+  assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").isEmpty)
+}
+  }
+
+  private def extractStatsPropValues(
+  descOutput: Seq[String],
+  propKey: String): Option[BigInt] = {
+val str = descOutput
+  .filterNot(_.contains(HiveExternalCatalog.STATISTICS_PREFIX))
+  .filter(_.contains(propKey))
+if (str.isEmpty) {
+  None
+} else {
+  assert(str.length == 1, "found more than one matches")
+  val pattern = new Regex(s"""$propKey\\s+(-?\\d+)""")
+  val pattern(value) = str.head.trim
+  Option(BigInt(value))
+}
+  }
+
+  test("get statistics when not analyzed in both Hive and Spark") {
+val tabName = "tab1"
+withTable(tabName) {
+  createNonPartitionedTable(tabName, analyzedByHive = false, 
analyzedBySpark = false)
+  checkTableStats(
+tabName, hasSizeInBytes = true, expectedRowCounts = None)
+
+  // ALTER TABLE SET TBLPROPERTIES invalidates some contents of Hive 
specific statistics
+  // This is triggered by the Hive alterTable API
+  val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+  val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+
+  val rawDataSize = extractStatsPropValues(describeResult, 
"rawDataSize")
+  val numRows = extractStatsPropValues(describeResult, "numRows")
+  val totalSize = extractStatsPropValues(describeResult, "totalSize")
+  assert(rawDataSize.isEmpty, "rawDataSize should not be shown without 
table analysis")
+  assert(numRows.isEmpty, "numRows should not be shown without table 
analysis")
+  assert(totalSize.isDefined && totalSize.get > 0, "totalSize is lost")
+}
+  }
+
+  test("alter table rename after analyze table") {
+Seq(true, false).foreach { analyzedBySpark =>
+  val oldName = "tab1"
+  val newName = "tab2"
+  withTable(oldName, newName) {
+createNonPartitionedTable(oldName, analyzedByHive = true, 
analyzedBySpark = analyzedBySpark)
+val fetchedStats1 = checkTableStats(
+  oldName, hasSizeInBytes = true, expectedRowCounts = Some(500))
+sql(s"ALTER TABLE $oldName RENAME TO $newName")
+val fetchedStats2 = checkTableStats(
+  newName, hasSizeInBytes = true, expectedRowCounts = Some(500))
+assert(fetchedStats1 == fetchedStats2)
+
+// ALTER TABLE RENAME does not affect the contents of Hive 
specific statistics
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$newName")
+
+val rawDataSize = extractStatsPropValues(describeResult, 
"rawDataSize")
+val numRows = extractStatsPropValues(describeResult, "numRows")
+  

[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-17 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117158766
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -175,7 +178,7 @@ class StatisticsSuite extends 
StatisticsCollectionTestBase with TestHiveSingleto
   sql(s"INSERT INTO TABLE $textTable SELECT * FROM src")
   checkTableStats(
 textTable,
-hasSizeInBytes = false,
+hasSizeInBytes = true,
--- End diff --

why the behavior is changed?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-17 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117158738
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala ---
@@ -325,26 +325,24 @@ class ShowCreateTableSuite extends QueryTest with 
SQLTestUtils with TestHiveSing
 "last_modified_by",
 "last_modified_time",
 "Owner:",
-"COLUMN_STATS_ACCURATE",
 // The following are hive specific schema parameters which we do 
not need to match exactly.
-"numFiles",
-"numRows",
-"rawDataSize",
-"totalSize",
 "totalNumberFiles",
 "maxFileSize",
-"minFileSize",
-// EXTERNAL is not non-deterministic, but it is filtered out for 
external tables.
-"EXTERNAL"
+"minFileSize"
   )
 
   table.copy(
 createTime = 0L,
 lastAccessTime = 0L,
-properties = 
table.properties.filterKeys(!nondeterministicProps.contains(_))
+properties = 
table.properties.filterKeys(!nondeterministicProps.contains(_)),
+stats = None,
+ignoredProperties = Map.empty
   )
 }
 
+val e = normalize(actual)
+val m = normalize(expected)
--- End diff --

remove this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-17 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r117158531
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala 
---
@@ -414,6 +415,50 @@ private[hive] class HiveClientImpl(
 
   val properties = Option(h.getParameters).map(_.asScala.toMap).orNull
 
+  // Hive-generated Statistics are also recorded in ignoredProperties
+  val ignoredProperties = scala.collection.mutable.Map.empty[String, 
String]
+  for (key <- HiveStatisticsProperties; value <- properties.get(key)) {
+ignoredProperties += key -> value
+  }
+
+  val excludedTableProperties = HiveStatisticsProperties ++ Set(
+// The property value of "comment" is moved to the dedicated field 
"comment"
+"comment",
+// For EXTERNAL_TABLE, the table properties has a particular field 
"EXTERNAL". This is added
+// in the function toHiveTable.
+"EXTERNAL"
+  )
+
+  val filteredProperties = properties.filterNot {
+case (key, _) => excludedTableProperties.contains(key)
+  }
+  val comment = properties.get("comment")
+
+  val totalSize = 
properties.get(StatsSetupConst.TOTAL_SIZE).map(BigInt(_))
+  val rawDataSize = 
properties.get(StatsSetupConst.RAW_DATA_SIZE).map(BigInt(_))
+  def rowCount = 
properties.get(StatsSetupConst.ROW_COUNT).map(BigInt(_)) match {
+case Some(c) if c >= 0 => Some(c)
+case _ => None
+  }
+  // TODO: check if this estimate is valid for tables after partition 
pruning.
+  // NOTE: getting `totalSize` directly from params is kind of hacky, 
but this should be
+  // relatively cheap if parameters for the table are populated into 
the metastore.
+  // Currently, only totalSize, rawDataSize, and row_count are used to 
build the field `stats`
+  // TODO: stats should include all the other two fields (`numFiles` 
and `numPartitions`).
+  // (see StatsSetupConst in Hive)
+  val stats =
+  // When table is external, `totalSize` is always zero, which will 
influence join strategy
+  // so when `totalSize` is zero, use `rawDataSize` instead. When 
`rawDataSize` is also zero,
+  // return None. Later, we will use the other ways to estimate the 
statistics.
+  if (totalSize.isDefined && totalSize.get > 0L) {
--- End diff --

the indention is wrong


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-17 Thread gatorsmile
GitHub user gatorsmile reopened a pull request:

https://github.com/apache/spark/pull/14971

[SPARK-17410] [SPARK-17284] Move Hive-generated Stats Info to HiveClientImpl

### What changes were proposed in this pull request?

After we adding a new field `stats` into `CatalogTable`, we should not 
expose Hive-specific Stats metadata to `MetastoreRelation`. It complicates all 
the related codes. It also introduces a bug in `SHOW CREATE TABLE`. The 
statistics-related table properties should be skipped by `SHOW CREATE TABLE`, 
since it could be incorrect in the newly created table. See the Hive JIRA: 
https://issues.apache.org/jira/browse/HIVE-13792

This PR is to handle Hive-specific Stats metadata in `HiveClientImpl`. 
### How was this patch tested?

Added a few test cases.


You can merge this pull request into a Git repository by running:

$ git pull https://github.com/gatorsmile/spark showCreateTableNew

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/14971.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #14971


commit 92474c5a142fb9db2c86549c8347f910fc01fcbd
Author: gatorsmile 
Date:   2016-08-28T22:28:15Z

remove stats-related props

commit ce8e8b89a5b61648daaa59578e2b6a99ec2f6d74
Author: gatorsmile 
Date:   2016-08-29T05:24:05Z

address comments

commit 9ce526b7729c4111292d6abb69bd81aec0ecf5de
Author: gatorsmile 
Date:   2016-09-06T00:25:56Z

Merge remote-tracking branch 'upstream/master' into showCreateTable

commit efd879dbda12e235d00d9b6bc4891a591832912b
Author: gatorsmile 
Date:   2016-09-06T07:07:12Z

move stats from metastorerelation to hiveclientimpl

commit 491c0cd2580cfd358b2fd4d94a6f41b4063f172f
Author: gatorsmile 
Date:   2016-09-06T07:34:51Z

improve the comments.

commit c9cdf44b561c5e88a108cd09ad17842332d14162
Author: gatorsmile 
Date:   2016-09-06T07:35:26Z

improve the comments.

commit 4b0aed54b0aef6675f4f9fac82f6647563afb5cc
Author: gatorsmile 
Date:   2016-09-08T03:21:28Z

Merge remote-tracking branch 'upstream/master' into showCreateTable

commit 552101af418e4a30febf7d09938022ecc4c08da9
Author: gatorsmile 
Date:   2016-09-08T03:25:00Z

merge

commit d3dcb564509fd2a32a3fadefb811495affaaa466
Author: gatorsmile 
Date:   2016-09-11T05:55:20Z

Merge remote-tracking branch 'upstream/master' into showCreateTable

commit 9e18ba104527d2bb14331f4b51194002dabb2556
Author: gatorsmile 
Date:   2016-09-11T21:48:36Z

fix and add more test cases

commit c6a85bcd4b6b58c46787d1ca1510418cef79a8d5
Author: gatorsmile 
Date:   2016-09-16T04:50:18Z

Merge remote-tracking branch 'upstream/master' into showCreateTable

commit 3ed68e0b0aa6aff19a5e31c89fed7e5c814e83f3
Author: gatorsmile 
Date:   2016-09-16T06:16:40Z

improve the test case

commit 2e4d398388cd64f3e1d130af81d5e7ddc23a2a19
Author: gatorsmile 
Date:   2016-09-17T06:52:32Z

also utilizes Hive-generated row counts when not analyzed in Spark

commit 5dfa17efa84ed180e68b4922cfaf85e3d50f14ad
Author: gatorsmile 
Date:   2016-09-17T07:30:57Z

more comments

commit 2f40c7f5532c8b6e66c786f3b1506bd4efdcf711
Author: gatorsmile 
Date:   2016-09-18T00:08:48Z

address comments.

commit 3376bd6a57a65fa004abd43237f8f3c87f07064a
Author: gatorsmile 
Date:   2016-09-18T03:31:11Z

fix test cases

commit 90cd18e9d7bad6462fb0254d7981e23341795c11
Author: gatorsmile 
Date:   2016-09-21T04:58:42Z

Merge remote-tracking branch 'upstream/master' into showCreateTable

commit 7ad08fe2a488fa759b4abf4e99a7206e031379d9
Author: gatorsmile 
Date:   2016-09-21T05:23:58Z

test case fix

commit f4c0ebb0901216ea09eaf3f77e4fdcd431b15d37
Author: gatorsmile 
Date:   2016-09-22T23:08:55Z

address comments

commit 4c89d92ab65d7f4f061e32aa22780fd6e4b7c798
Author: gatorsmile 
Date:   2016-09-22T23:12:57Z

address comments

commit 699b5d8aa4d9370009c73f45d1618f1e5bb92210
Author: gatorsmile 
Date:   2016-09-24T01:10:04Z

fix.

commit 8c90d0b7364c46de5a4a59fa89457000bb283dd9
Author: gatorsmile 
Date:   2016-09-25T05:33:44Z

Merge remote-tracking branch 'upstream/master' into showCreateTable

commit 50ce04e51aebd5f68f7e50ec7c3bbe72275bf629
Author: gatorsmile 
Date:   2016-09-25T07:10:30Z

address comments.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-05-05 Thread gatorsmile
Github user gatorsmile closed the pull request at:

https://github.com/apache/spark/pull/14971


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-01-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r95977972
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala 
---
@@ -427,7 +473,14 @@ private[hive] class HiveClientImpl(
   }
 
   override def alterTable(tableName: String, table: CatalogTable): Unit = 
withHiveState {
-val hiveTable = toHiveTable(table)
+// getTableOption removes all the Hive-specific properties. Here, we 
fill them back to ensure
+// these properties are still available to the others that share the 
same Hive metastore.
+// If users explicitly alter these Hive-specific properties through 
ALTER TABLE DDL, we respect
+// these user-specified values.
+val hiveSpecificProps = table.ignoredProperties.filter {
--- End diff --

do we really need to do `filter` here? I think the semantic of 
`ignoredProperties` is: ignore these properties in Spark SQL, but get them back 
when saving the metastore


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-01-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r95977105
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala 
---
@@ -427,7 +473,14 @@ private[hive] class HiveClientImpl(
   }
 
   override def alterTable(tableName: String, table: CatalogTable): Unit = 
withHiveState {
-val hiveTable = toHiveTable(table)
+// getTableOption removes all the Hive-specific properties. Here, we 
fill them back to ensure
+// these properties are still available to the others that share the 
same Hive metastore.
+// If users explicitly alter these Hive-specific properties through 
ALTER TABLE DDL, we respect
+// these user-specified values.
+val hiveSpecificProps = table.ignoredProperties.filter {
+  case (key, _) => HiveStatisticsProperties.contains(key)
+}
+val hiveTable = toHiveTable(table.copy(properties = table.properties 
++ hiveSpecificProps))
--- End diff --

`table.properties ++ hiveSpecificProps`, by doing this, we won't respect 
user-specified Hive-specific properties, which are in `table.properties`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-01-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r95976667
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala 
---
@@ -378,6 +380,50 @@ private[hive] class HiveClientImpl(
 
   val properties = Option(h.getParameters).map(_.asScala.toMap).orNull
 
+  // Hive-generated Statistics are also recorded in ignoredProperties
+  val ignoredProperties = scala.collection.mutable.Map.empty[String, 
String]
+  properties.collect {
--- End diff --

how about
```
for (key <- HiveStatisticsProperties; value <- properties.get(key)) {
  ignoredProperties += key -> value
}
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2017-01-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r95975787
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala 
---
@@ -378,6 +380,50 @@ private[hive] class HiveClientImpl(
 
   val properties = Option(h.getParameters).map(_.asScala.toMap).orNull
 
+  // Hive-generated Statistics are also recorded in ignoredProperties
+  val ignoredProperties = scala.collection.mutable.Map.empty[String, 
String]
+  properties.collect {
+case (key, value) if HiveStatisticsProperties.contains(key) =>
+  ignoredProperties += (key -> value)
+  }
+
+  val excludedTableProperties = HiveStatisticsProperties ++ Set(
+// The property value of "comment" is moved to the dedicated field 
"comment"
+"comment",
+// For EXTERNAL_TABLE, the table properties has a particular field 
"EXTERNAL". This is added
+// in the function toHiveTable.
+"EXTERNAL"
+  )
+
+  val filteredProperties = properties.filterNot {
+case (key, _) => excludedTableProperties.contains(key)
+  }
+  val comment = properties.get("comment")
+
+  val totalSize = 
properties.get(StatsSetupConst.TOTAL_SIZE).map(BigInt(_))
+  val rawDataSize = 
properties.get(StatsSetupConst.RAW_DATA_SIZE).map(BigInt(_))
+  val rowCount = 
properties.get(StatsSetupConst.ROW_COUNT).map(BigInt(_)) match {
+case Some(c) if c >= 0 => Some(c)
+case _ => None
+  }
+  // TODO: check if this estimate is valid for tables after partition 
pruning.
+  // NOTE: getting `totalSize` directly from params is kind of hacky, 
but this should be
+  // relatively cheap if parameters for the table are populated into 
the metastore.
+  // Currently, only totalSize, rawDataSize, and row_count are used to 
build the field `stats`
+  // TODO: stats should include all the other two fields (`numFiles` 
and `numPartitions`).
+  // (see StatsSetupConst in Hive)
+  val stats =
+// When table is external, `totalSize` is always zero, which will 
influence join strategy
+// so when `totalSize` is zero, use `rawDataSize` instead. When 
`rawDataSize` is also zero,
+// return None. Later, we will use the other ways to estimate the 
statistics.
+if (totalSize.isDefined && totalSize.get > 0L) {
+  Some(Statistics(sizeInBytes = totalSize.get, rowCount = 
rowCount))
+} else if (rawDataSize.isDefined && rawDataSize.get > 0) {
+  Some(Statistics(sizeInBytes = rawDataSize.get, rowCount = 
rowCount))
+} else {
+  None
--- End diff --

shall we respect `rowCount` in this case?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-25 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r80375588
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -216,16 +218,15 @@ class StatisticsSuite extends QueryTest with 
TestHiveSingleton with SQLTestUtils
 hasSizeInBytes = false,
 expectedRowCounts = None)
   sql(s"INSERT INTO TABLE $textTable SELECT * FROM src")
-  checkStats(
-textTable,
-isDataSourceTable = false,
-hasSizeInBytes = false,
-expectedRowCounts = None)
+
+  val fetchedStats = checkStats(
+textTable, isDataSourceTable = false, hasSizeInBytes = true, 
expectedRowCounts = None)
--- End diff --

Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-25 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r80375572
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -259,6 +260,230 @@ class StatisticsSuite extends QueryTest with 
TestHiveSingleton with SQLTestUtils
 }
   }
 
+  private def createNonPartitionedTable(
+  tabName: String,
+  analyzedBySpark: Boolean = true,
+  analyzedByHive: Boolean = true): Unit = {
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+sql(
+  s"""
+ |CREATE TABLE $tabName (key STRING, value STRING)
+ |STORED AS TEXTFILE
+ |TBLPROPERTIES ('prop1' = 'val1', 'prop2' = 'val2')
+   """.stripMargin)
+sql(s"INSERT INTO TABLE $tabName SELECT * FROM src")
+if (analyzedBySpark) sql(s"ANALYZE TABLE $tabName COMPUTE STATISTICS")
+if (analyzedByHive) hiveClient.runSqlHive(s"ANALYZE TABLE $tabName 
COMPUTE STATISTICS")
+val describeResult1 = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+
+val tableMetadata =
+  
spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName)).properties
+// statistics info is not contained in the metadata of the original 
table
+assert(Seq(StatsSetupConst.COLUMN_STATS_ACCURATE,
+  StatsSetupConst.NUM_FILES,
+  StatsSetupConst.NUM_PARTITIONS,
+  StatsSetupConst.ROW_COUNT,
+  StatsSetupConst.RAW_DATA_SIZE,
+  StatsSetupConst.TOTAL_SIZE).forall(!tableMetadata.contains(_)))
+
+if (analyzedByHive) {
+  assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").nonEmpty)
+} else {
+  assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").isEmpty)
+}
+  }
+
+  private def extractStatsPropValues(
+  descOutput: Seq[String],
+  propKey: String): Option[BigInt] = {
+val str = descOutput
+  .filterNot(_.contains(HiveExternalCatalog.STATISTICS_PREFIX))
+  .filter(_.contains(propKey))
+if (str.isEmpty) {
+  None
+} else {
+  assert(str.length == 1, "found more than one matches")
+  val pattern = new Regex(s"""$propKey\\s+(-?\\d+)""")
+  val pattern(value) = str.head.trim
+  Option(BigInt(value))
+}
+  }
+
+  test("get statistics when not analyzed in both Hive and Spark") {
+val tabName = "tab1"
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+hiveClient.runSqlHive("set hive.stats.autogather=false")
--- End diff --

This is not related to the code. Let me remove it. 

No matter whether this is true or false, totalSize is a positive value. 
Also tried `INSERT OVERWRITE`. Still the same. Maybe a bug in Hive?

Originally, I want to see the impact. It does not behave consistently with 
my installed Hive version. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-24 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r80374991
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -259,6 +260,230 @@ class StatisticsSuite extends QueryTest with 
TestHiveSingleton with SQLTestUtils
 }
   }
 
+  private def createNonPartitionedTable(
+  tabName: String,
+  analyzedBySpark: Boolean = true,
+  analyzedByHive: Boolean = true): Unit = {
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+sql(
+  s"""
+ |CREATE TABLE $tabName (key STRING, value STRING)
+ |STORED AS TEXTFILE
+ |TBLPROPERTIES ('prop1' = 'val1', 'prop2' = 'val2')
+   """.stripMargin)
+sql(s"INSERT INTO TABLE $tabName SELECT * FROM src")
+if (analyzedBySpark) sql(s"ANALYZE TABLE $tabName COMPUTE STATISTICS")
+if (analyzedByHive) hiveClient.runSqlHive(s"ANALYZE TABLE $tabName 
COMPUTE STATISTICS")
+val describeResult1 = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+
+val tableMetadata =
+  
spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName)).properties
+// statistics info is not contained in the metadata of the original 
table
+assert(Seq(StatsSetupConst.COLUMN_STATS_ACCURATE,
+  StatsSetupConst.NUM_FILES,
+  StatsSetupConst.NUM_PARTITIONS,
+  StatsSetupConst.ROW_COUNT,
+  StatsSetupConst.RAW_DATA_SIZE,
+  StatsSetupConst.TOTAL_SIZE).forall(!tableMetadata.contains(_)))
+
+if (analyzedByHive) {
+  assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").nonEmpty)
--- End diff --

Hive does not recognize Spark-generated statistics. Thus, if 
`analyzedByHive` is `false`, we do not expect to see the collected `numRows`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-23 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r80191528
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -259,6 +260,230 @@ class StatisticsSuite extends QueryTest with 
TestHiveSingleton with SQLTestUtils
 }
   }
 
+  private def createNonPartitionedTable(
+  tabName: String,
+  analyzedBySpark: Boolean = true,
+  analyzedByHive: Boolean = true): Unit = {
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+sql(
+  s"""
+ |CREATE TABLE $tabName (key STRING, value STRING)
+ |STORED AS TEXTFILE
+ |TBLPROPERTIES ('prop1' = 'val1', 'prop2' = 'val2')
+   """.stripMargin)
+sql(s"INSERT INTO TABLE $tabName SELECT * FROM src")
+if (analyzedBySpark) sql(s"ANALYZE TABLE $tabName COMPUTE STATISTICS")
+if (analyzedByHive) hiveClient.runSqlHive(s"ANALYZE TABLE $tabName 
COMPUTE STATISTICS")
+val describeResult1 = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+
+val tableMetadata =
+  
spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName)).properties
+// statistics info is not contained in the metadata of the original 
table
+assert(Seq(StatsSetupConst.COLUMN_STATS_ACCURATE,
+  StatsSetupConst.NUM_FILES,
+  StatsSetupConst.NUM_PARTITIONS,
+  StatsSetupConst.ROW_COUNT,
+  StatsSetupConst.RAW_DATA_SIZE,
+  StatsSetupConst.TOTAL_SIZE).forall(!tableMetadata.contains(_)))
+
+if (analyzedByHive) {
+  assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").nonEmpty)
--- End diff --

why we test the stats at hive side not spark side?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-23 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r80191227
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -259,6 +260,230 @@ class StatisticsSuite extends QueryTest with 
TestHiveSingleton with SQLTestUtils
 }
   }
 
+  private def createNonPartitionedTable(
+  tabName: String,
+  analyzedBySpark: Boolean = true,
+  analyzedByHive: Boolean = true): Unit = {
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+sql(
+  s"""
+ |CREATE TABLE $tabName (key STRING, value STRING)
+ |STORED AS TEXTFILE
+ |TBLPROPERTIES ('prop1' = 'val1', 'prop2' = 'val2')
+   """.stripMargin)
+sql(s"INSERT INTO TABLE $tabName SELECT * FROM src")
+if (analyzedBySpark) sql(s"ANALYZE TABLE $tabName COMPUTE STATISTICS")
+if (analyzedByHive) hiveClient.runSqlHive(s"ANALYZE TABLE $tabName 
COMPUTE STATISTICS")
+val describeResult1 = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+
+val tableMetadata =
+  
spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName)).properties
+// statistics info is not contained in the metadata of the original 
table
+assert(Seq(StatsSetupConst.COLUMN_STATS_ACCURATE,
+  StatsSetupConst.NUM_FILES,
+  StatsSetupConst.NUM_PARTITIONS,
+  StatsSetupConst.ROW_COUNT,
+  StatsSetupConst.RAW_DATA_SIZE,
+  StatsSetupConst.TOTAL_SIZE).forall(!tableMetadata.contains(_)))
+
+if (analyzedByHive) {
+  assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").nonEmpty)
+} else {
+  assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").isEmpty)
+}
+  }
+
+  private def extractStatsPropValues(
+  descOutput: Seq[String],
+  propKey: String): Option[BigInt] = {
+val str = descOutput
+  .filterNot(_.contains(HiveExternalCatalog.STATISTICS_PREFIX))
+  .filter(_.contains(propKey))
+if (str.isEmpty) {
+  None
+} else {
+  assert(str.length == 1, "found more than one matches")
+  val pattern = new Regex(s"""$propKey\\s+(-?\\d+)""")
+  val pattern(value) = str.head.trim
+  Option(BigInt(value))
+}
+  }
+
+  test("get statistics when not analyzed in both Hive and Spark") {
+val tabName = "tab1"
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+hiveClient.runSqlHive("set hive.stats.autogather=false")
--- End diff --

we should reset this conf back after test


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-23 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r80190792
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -216,16 +218,15 @@ class StatisticsSuite extends QueryTest with 
TestHiveSingleton with SQLTestUtils
 hasSizeInBytes = false,
 expectedRowCounts = None)
   sql(s"INSERT INTO TABLE $textTable SELECT * FROM src")
-  checkStats(
-textTable,
-isDataSourceTable = false,
-hasSizeInBytes = false,
-expectedRowCounts = None)
+
+  val fetchedStats = checkStats(
+textTable, isDataSourceTable = false, hasSizeInBytes = true, 
expectedRowCounts = None)
--- End diff --

we should also update the comment in L212


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-22 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r80137168
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
 ---
@@ -134,7 +134,7 @@ case class CatalogTable(
 viewOriginalText: Option[String] = None,
 viewText: Option[String] = None,
 comment: Option[String] = None,
-unsupportedFeatures: Seq[String] = Seq.empty) {
+unsupportedFeatures: Map[String, String] = Map.empty) {
--- End diff --

👍 Let me add a new field called `ignoredProperties`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-22 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r79991975
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
 ---
@@ -134,7 +134,7 @@ case class CatalogTable(
 viewOriginalText: Option[String] = None,
 viewText: Option[String] = None,
 comment: Option[String] = None,
-unsupportedFeatures: Seq[String] = Seq.empty) {
+unsupportedFeatures: Map[String, String] = Map.empty) {
--- End diff --

should we give it a new name? hive stats properties are not unsupported but 
ignored...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-22 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r79992002
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
 ---
@@ -134,7 +134,7 @@ case class CatalogTable(
 viewOriginalText: Option[String] = None,
 viewText: Option[String] = None,
 comment: Option[String] = None,
-unsupportedFeatures: Seq[String] = Seq.empty) {
+unsupportedFeatures: Map[String, String] = Map.empty) {
--- End diff --

or we may just add a new field


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-18 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r79300109
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -259,6 +260,230 @@ class StatisticsSuite extends QueryTest with 
TestHiveSingleton with SQLTestUtils
 }
   }
 
+  private def createNonPartitionedTable(
+  tabName: String,
+  analyzedBySpark: Boolean = true,
+  analyzedByHive: Boolean = true): Unit = {
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+sql(
+  s"""
+ |CREATE TABLE $tabName (key STRING, value STRING)
+ |STORED AS TEXTFILE
+ |TBLPROPERTIES ('prop1' = 'val1', 'prop2' = 'val2')
+   """.stripMargin)
+sql(s"INSERT INTO TABLE $tabName SELECT * FROM src")
+if (analyzedBySpark) sql(s"ANALYZE TABLE $tabName COMPUTE STATISTICS")
+if (analyzedByHive) hiveClient.runSqlHive(s"ANALYZE TABLE $tabName 
COMPUTE STATISTICS")
+val describeResult1 = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+
+val tableMetadata =
+  
spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName)).properties
+// statistics info is not contained in the metadata of the original 
table
+assert(Seq(StatsSetupConst.COLUMN_STATS_ACCURATE,
+  StatsSetupConst.NUM_FILES,
+  StatsSetupConst.NUM_PARTITIONS,
+  StatsSetupConst.ROW_COUNT,
+  StatsSetupConst.RAW_DATA_SIZE,
+  StatsSetupConst.TOTAL_SIZE).forall(!tableMetadata.contains(_)))
+
+if (analyzedByHive) {
+  assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").nonEmpty)
+} else {
+  assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").isEmpty)
+}
+  }
+
+  private def extractStatsPropValues(
+  descOutput: Seq[String],
+  propKey: String): Option[BigInt] = {
+val str = descOutput
+  .filterNot(_.contains(HiveExternalCatalog.STATISTICS_PREFIX))
+  .filter(_.contains(propKey))
+if (str.isEmpty) {
+  None
+} else {
+  assert(str.length == 1, "found more than one matches")
+  val pattern = new Regex(s"""$propKey\\s+(-?\\d+)""") // 
s"""${propKey}\s+(-?\d+)""".r
+  val pattern(value) = str.head.trim
+  Option(BigInt(value))
+}
+  }
+
+  test("get statistics when not analyzed in both Hive and Spark") {
+val tabName = "tab1"
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+hiveClient.runSqlHive("set hive.stats.autogather=false")
+withTable(tabName) {
+  createNonPartitionedTable(tabName, analyzedByHive = false, 
analyzedBySpark = false)
+  checkStats(
+tabName, isDataSourceTable = false, hasSizeInBytes = true, 
expectedRowCounts = None)
+
+  // ALTER TABLE SET TBLPROPERTIES invalidates some contents of Hive 
specific statistics
+  // This is triggered by the Hive alterTable API
+  val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+  val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+
+  val rawDataSize = extractStatsPropValues(describeResult, 
"rawDataSize")
+  val numRows = extractStatsPropValues(describeResult, "numRows")
+  val totalSize = extractStatsPropValues(describeResult, "totalSize")
+  assert(rawDataSize.isEmpty, "rawDataSize should not be shown without 
table analysis")
+  assert(numRows.isEmpty, "numRows should not be shown without table 
analysis")
+  assert(totalSize.isDefined && totalSize.get > 0, "totalSize is lost")
+}
+  }
+
+  test("alter table rename after analyze table") {
+Seq(true, false).foreach { analyzedBySpark =>
+  val oldName = "tab1"
+  val newName = "tab2"
+  withTable(oldName, newName) {
+createNonPartitionedTable(oldName, analyzedByHive = true, 
analyzedBySpark = analyzedBySpark)
+val fetchedStats1 = checkStats(
+  oldName, isDataSourceTable = false, hasSizeInBytes = true, 
expectedRowCounts = Some(500))
+sql(s"ALTER TABLE $oldName RENAME TO $newName")
+val fetchedStats2 = checkStats(
+  newName, isDataSourceTable = false, hasSizeInBytes = true, 
expectedRowCounts = Some(500))
+assert(fetchedStats1 == fetchedStats2)
+
+// ALTER TABLE RENAME does not affect the contents of Hive 
specific statistics
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$newName")

[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-17 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r79286533
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala 
---
@@ -378,6 +380,47 @@ private[hive] class HiveClientImpl(
 
   val properties = Option(h.getParameters).map(_.asScala.toMap).orNull
 
+  val excludedTableProperties = Set(
+// The following are hive-generated statistics fields. Currently, 
only total_size and
+// row_count are used to populate the dedicated field `stats`.
+// TODO: stats should include all the other three fields.
+StatsSetupConst.COLUMN_STATS_ACCURATE,
+StatsSetupConst.NUM_FILES,
+StatsSetupConst.NUM_PARTITIONS,
+StatsSetupConst.ROW_COUNT,
+StatsSetupConst.RAW_DATA_SIZE,
--- End diff --

Using `unsupportedFeatures` sounds a pretty good idea! Let me make a try. 
Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-17 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r79281449
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala 
---
@@ -378,6 +380,47 @@ private[hive] class HiveClientImpl(
 
   val properties = Option(h.getParameters).map(_.asScala.toMap).orNull
 
+  val excludedTableProperties = Set(
+// The following are hive-generated statistics fields. Currently, 
only total_size and
+// row_count are used to populate the dedicated field `stats`.
+// TODO: stats should include all the other three fields.
+StatsSetupConst.COLUMN_STATS_ACCURATE,
+StatsSetupConst.NUM_FILES,
+StatsSetupConst.NUM_PARTITIONS,
+StatsSetupConst.ROW_COUNT,
+StatsSetupConst.RAW_DATA_SIZE,
--- End diff --

`CatalogTable` have a field `unsupportedFeatures`, can we extend it to hide 
this kind of hive specific properties which are only useful in alter table?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-17 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r79281395
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala 
---
@@ -378,6 +380,47 @@ private[hive] class HiveClientImpl(
 
   val properties = Option(h.getParameters).map(_.asScala.toMap).orNull
 
+  val excludedTableProperties = Set(
+// The following are hive-generated statistics fields. Currently, 
only total_size and
+// row_count are used to populate the dedicated field `stats`.
+// TODO: stats should include all the other three fields.
+StatsSetupConst.COLUMN_STATS_ACCURATE,
+StatsSetupConst.NUM_FILES,
+StatsSetupConst.NUM_PARTITIONS,
+StatsSetupConst.ROW_COUNT,
+StatsSetupConst.RAW_DATA_SIZE,
--- End diff --

ah I see. So our targets are:

1. recognize hive statistics, i.e. we should set the `CatalogTable.stats` 
according to hive stats properties
2. don't overwrite hive stats properties.
3. SHOW CREATE TABLE shouldn't print hive stats properties.

My proposal: In `HiveClientImpl`, set `CatalogTable.stats` by hive stats 
properties, and still keep them in table properties. In SHOW CREATE TABLE, hide 
the hive stats properties.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-17 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r79278095
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala 
---
@@ -378,6 +380,51 @@ private[hive] class HiveClientImpl(
 
   val properties = Option(h.getParameters).map(_.asScala.toMap).orNull
 
+  val excludedTableProperties = Set(
+// The following are hive-generated statistics fields. Currently, 
only total_size and
+// row_count are used to populate the dedicated field `stats`.
+// TODO: stats should include all the other three fields.
+StatsSetupConst.COLUMN_STATS_ACCURATE,
+StatsSetupConst.NUM_FILES,
+StatsSetupConst.NUM_PARTITIONS,
+StatsSetupConst.ROW_COUNT,
+StatsSetupConst.RAW_DATA_SIZE,
+StatsSetupConst.TOTAL_SIZE,
+// The property value of "comment" is moved to the dedicated field 
"comment"
+"comment",
+// For EXTERNAL_TABLE, the table properties has a particular field 
"EXTERNAL". This is added
+// in the function toHiveTable.
+"EXTERNAL"
+  )
+
+  val filteredProperties = properties.filterNot {
+case (key, _) => excludedTableProperties.contains(key)
+  }
+  val comment = properties.get("comment")
+
+  val totalSize = properties.get(StatsSetupConst.TOTAL_SIZE).map(s => 
BigInt(s.toLong))
+  val rawDataSize = 
properties.get(StatsSetupConst.RAW_DATA_SIZE).map(s => BigInt(s.toLong))
+  val rowCount = properties.get(StatsSetupConst.ROW_COUNT).map(s => 
BigInt(s.toLong)) match {
--- End diff --

Here, we also utilize Hive-generated row counts when users have not run 
ANALYZE TABLE through Spark.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-16 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r79121955
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala 
---
@@ -378,6 +380,47 @@ private[hive] class HiveClientImpl(
 
   val properties = Option(h.getParameters).map(_.asScala.toMap).orNull
 
+  val excludedTableProperties = Set(
+// The following are hive-generated statistics fields. Currently, 
only total_size and
+// row_count are used to populate the dedicated field `stats`.
+// TODO: stats should include all the other three fields.
+StatsSetupConst.COLUMN_STATS_ACCURATE,
+StatsSetupConst.NUM_FILES,
+StatsSetupConst.NUM_PARTITIONS,
+StatsSetupConst.ROW_COUNT,
+StatsSetupConst.RAW_DATA_SIZE,
--- End diff --

how about we only handle the 2 we need? i.e. `TOTAL_SIZE` and 
`RAW_DATA_SIZE`. Then we don't need to do an extra `getTable` call in 
`alterTable`, which may cause performance regression.

Ideally the rule is, we only drop the hive properties that we moved to 
other places, so that we can reconstruct them without an extra `getTable` call.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-15 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r79113972
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala ---
@@ -111,23 +111,8 @@ private[hive] case class MetastoreRelation(
   @transient override lazy val statistics: Statistics = {
 catalogTable.stats.getOrElse(Statistics(
   sizeInBytes = {
-val totalSize = 
hiveQlTable.getParameters.get(StatsSetupConst.TOTAL_SIZE)
-val rawDataSize = 
hiveQlTable.getParameters.get(StatsSetupConst.RAW_DATA_SIZE)
--- End diff --

In the master branch, we do not use Hive-generated `numRows`... Let me fix 
it in this PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-11 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r78305536
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -259,6 +259,156 @@ class StatisticsSuite extends QueryTest with 
TestHiveSingleton with SQLTestUtils
 }
   }
 
+  private def createNonPartitionedTable(tabName: String): 
Option[Statistics] = {
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+sql(
+  s"""
+ |CREATE TABLE $tabName (key STRING, value STRING)
+ |STORED AS TEXTFILE
+ |TBLPROPERTIES ('prop1' = 'val1', 'prop2' = 'val2')
+   """.stripMargin)
+sql(s"INSERT INTO TABLE $tabName SELECT * FROM src")
+sql(s"ANALYZE TABLE $tabName COMPUTE STATISTICS")
+hiveClient.runSqlHive(s"ANALYZE TABLE $tabName COMPUTE STATISTICS")
+val describeResult1 = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+
+val tableMetadata =
+  
spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName)).properties
+// statistics info is not contained in the metadata of the original 
table
+assert(Seq(StatsSetupConst.COLUMN_STATS_ACCURATE,
+  StatsSetupConst.NUM_FILES,
+  StatsSetupConst.NUM_PARTITIONS,
+  StatsSetupConst.ROW_COUNT,
+  StatsSetupConst.RAW_DATA_SIZE,
+  StatsSetupConst.TOTAL_SIZE).forall(!tableMetadata.contains(_)))
+
+assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").nonEmpty)
+checkStats(
+  tabName, isDataSourceTable = false, hasSizeInBytes = true, 
expectedRowCounts = Some(500))
+  }
+
+  test("alter table rename after analyze table") {
--- End diff --

We can add more ALTER TABLE commands, if you think it is necessary.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-11 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14971#discussion_r78305513
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala ---
@@ -259,6 +259,156 @@ class StatisticsSuite extends QueryTest with 
TestHiveSingleton with SQLTestUtils
 }
   }
 
+  private def createNonPartitionedTable(tabName: String): 
Option[Statistics] = {
+val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+sql(
+  s"""
+ |CREATE TABLE $tabName (key STRING, value STRING)
+ |STORED AS TEXTFILE
+ |TBLPROPERTIES ('prop1' = 'val1', 'prop2' = 'val2')
+   """.stripMargin)
+sql(s"INSERT INTO TABLE $tabName SELECT * FROM src")
+sql(s"ANALYZE TABLE $tabName COMPUTE STATISTICS")
+hiveClient.runSqlHive(s"ANALYZE TABLE $tabName COMPUTE STATISTICS")
+val describeResult1 = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+
+val tableMetadata =
+  
spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName)).properties
+// statistics info is not contained in the metadata of the original 
table
+assert(Seq(StatsSetupConst.COLUMN_STATS_ACCURATE,
+  StatsSetupConst.NUM_FILES,
+  StatsSetupConst.NUM_PARTITIONS,
+  StatsSetupConst.ROW_COUNT,
+  StatsSetupConst.RAW_DATA_SIZE,
+  StatsSetupConst.TOTAL_SIZE).forall(!tableMetadata.contains(_)))
+
+assert(StringUtils.filterPattern(describeResult1, 
"*numRows\\s+500*").nonEmpty)
+checkStats(
+  tabName, isDataSourceTable = false, hasSizeInBytes = true, 
expectedRowCounts = Some(500))
+  }
+
+  test("alter table rename after analyze table") {
+val oldName = "tab1"
+val newName = "tab2"
+withTable(oldName, newName) {
+  val fetchedStats1 = createNonPartitionedTable(oldName)
+  sql(s"ALTER TABLE $oldName RENAME TO $newName")
+  val fetchedStats2 = checkStats(
+newName, isDataSourceTable = false, hasSizeInBytes = true, 
expectedRowCounts = Some(500))
+  assert(fetchedStats1 == fetchedStats2)
+
+  // ALTER TABLE RENAME does not affect the contents of Hive specific 
statistics
+  val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+  val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$newName")
+  assert(StringUtils.filterPattern(describeResult, 
"*numRows\\s+500*").nonEmpty)
+}
+  }
+
+  test("alter table SET TBLPROPERTIES after analyze table") {
+val tabName = "tab1"
+withTable(tabName) {
+  val fetchedStats1 = createNonPartitionedTable(tabName)
+
+  sql(s"ALTER TABLE $tabName SET TBLPROPERTIES ('foo' = 'a')")
+  val fetchedStats2 = checkStats(
+tabName, isDataSourceTable = false, hasSizeInBytes = true, 
expectedRowCounts = Some(500))
+  assert(fetchedStats1 == fetchedStats2)
+
+  // ALTER TABLE SET TBLPROPERTIES invalidates some contents of Hive 
specific statistics
+  // This is triggered by the Hive alterTable API
+  val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+  val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+  assert(StringUtils.filterPattern(describeResult, 
"*numRows\\s+-1*").nonEmpty)
+}
+  }
+
+  test("alter table UNSET TBLPROPERTIES after analyze table") {
+val tabName = "tab1"
+withTable(tabName) {
+  val fetchedStats1 = createNonPartitionedTable(tabName)
+
+  sql(s"ALTER TABLE $tabName UNSET TBLPROPERTIES ('prop1')")
+  val fetchedStats2 = checkStats(
+tabName, isDataSourceTable = false, hasSizeInBytes = true, 
expectedRowCounts = Some(500))
+  assert(fetchedStats1 == fetchedStats2)
+
+  // ALTER TABLE UNSET TBLPROPERTIES invalidates some contents of Hive 
specific statistics
+  // This is triggered by the Hive alterTable API
+  val hiveClient = 
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
+  val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED 
$tabName")
+  assert(StringUtils.filterPattern(describeResult, 
"*numRows\\s+-1*").nonEmpty)
+}
+  }
+
+  test("add/drop partitions - managed table") {
--- End diff --

FYI, when we drop partitions of EXTERNAL tables, `ANALYZE TABLE` is unable 
to exclude them from statistics. This should be fixed with 
https://issues.apache.org/jira/browse/SPARK-17129, if my understanding is right.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not h

[GitHub] spark pull request #14971: [SPARK-17410] [SPARK-17284] Move Hive-generated S...

2016-09-06 Thread gatorsmile
GitHub user gatorsmile opened a pull request:

https://github.com/apache/spark/pull/14971

[SPARK-17410] [SPARK-17284] Move Hive-generated Stats Info to HiveClientImpl

### What changes were proposed in this pull request?
After we adding a new field `stats` into `CatalogTable`, we should not 
expose Hive-specific Stats metadata to `MetastoreRelation`. It complicates all 
the related codes. It also introduces a bug in `SHOW CREATE TABLE`. The 
statistics-related table properties should be skipped by `SHOW CREATE TABLE`, 
since it could be incorrect in the newly created table. See the Hive JIRA: 
https://issues.apache.org/jira/browse/HIVE-13792

This PR is to handle Hive-specific Stats metadata in `HiveClientImpl`. 

### How was this patch tested?
Added a few test cases.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/gatorsmile/spark showCreateTableNew

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/14971.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #14971


commit 92474c5a142fb9db2c86549c8347f910fc01fcbd
Author: gatorsmile 
Date:   2016-08-28T22:28:15Z

remove stats-related props

commit ce8e8b89a5b61648daaa59578e2b6a99ec2f6d74
Author: gatorsmile 
Date:   2016-08-29T05:24:05Z

address comments

commit 9ce526b7729c4111292d6abb69bd81aec0ecf5de
Author: gatorsmile 
Date:   2016-09-06T00:25:56Z

Merge remote-tracking branch 'upstream/master' into showCreateTable

commit efd879dbda12e235d00d9b6bc4891a591832912b
Author: gatorsmile 
Date:   2016-09-06T07:07:12Z

move stats from metastorerelation to hiveclientimpl




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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