Skip to content

Commit a2460be

Browse files
committed
[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 Also fix the issue to fill Hive-generated RowCounts to our stats. This PR is to handle Hive-specific Stats metadata in `HiveClientImpl`. ### How was this patch tested? Added a few test cases. Author: Xiao Li <gatorsmile@gmail.com> Closes #14971 from gatorsmile/showCreateTableNew.
1 parent 9b09101 commit a2460be

File tree

15 files changed

+291
-59
lines changed

15 files changed

+291
-59
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -203,6 +203,8 @@ case class BucketSpec(
203203
* sensitive schema was unable to be read from the table properties.
204204
* Used to trigger case-sensitive schema inference at query time, when
205205
* configured.
206+
* @param ignoredProperties is a list of table properties that are used by the underlying table
207+
* but ignored by Spark SQL yet.
206208
*/
207209
case class CatalogTable(
208210
identifier: TableIdentifier,
@@ -221,7 +223,8 @@ case class CatalogTable(
221223
comment: Option[String] = None,
222224
unsupportedFeatures: Seq[String] = Seq.empty,
223225
tracksPartitionsInCatalog: Boolean = false,
224-
schemaPreservesCase: Boolean = true) {
226+
schemaPreservesCase: Boolean = true,
227+
ignoredProperties: Map[String, String] = Map.empty) {
225228

226229
import CatalogTable._
227230

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -492,7 +492,8 @@ class TreeNodeSuite extends SparkFunSuite {
492492
"tracksPartitionsInCatalog" -> false,
493493
"properties" -> JNull,
494494
"unsupportedFeatures" -> List.empty[String],
495-
"schemaPreservesCase" -> JBool(true)))
495+
"schemaPreservesCase" -> JBool(true),
496+
"ignoredProperties" -> JNull))
496497

497498
// For unknown case class, returns JNull.
498499
val bigValue = new Array[Int](10000)

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala

Lines changed: 1 addition & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -119,20 +119,7 @@ class DetermineTableStats(session: SparkSession) extends Rule[LogicalPlan] {
119119
case relation: CatalogRelation
120120
if DDLUtils.isHiveTable(relation.tableMeta) && relation.tableMeta.stats.isEmpty =>
121121
val table = relation.tableMeta
122-
// TODO: check if this estimate is valid for tables after partition pruning.
123-
// NOTE: getting `totalSize` directly from params is kind of hacky, but this should be
124-
// relatively cheap if parameters for the table are populated into the metastore.
125-
// Besides `totalSize`, there are also `numFiles`, `numRows`, `rawDataSize` keys
126-
// (see StatsSetupConst in Hive) that we can look at in the future.
127-
// When table is external,`totalSize` is always zero, which will influence join strategy
128-
// so when `totalSize` is zero, use `rawDataSize` instead.
129-
val totalSize = table.properties.get(StatsSetupConst.TOTAL_SIZE).map(_.toLong)
130-
val rawDataSize = table.properties.get(StatsSetupConst.RAW_DATA_SIZE).map(_.toLong)
131-
val sizeInBytes = if (totalSize.isDefined && totalSize.get > 0) {
132-
totalSize.get
133-
} else if (rawDataSize.isDefined && rawDataSize.get > 0) {
134-
rawDataSize.get
135-
} else if (session.sessionState.conf.fallBackToHdfsForStatsEnabled) {
122+
val sizeInBytes = if (session.sessionState.conf.fallBackToHdfsForStatsEnabled) {
136123
try {
137124
val hadoopConf = session.sessionState.newHadoopConf()
138125
val tablePath = new Path(table.location)

sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala

Lines changed: 63 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ import scala.collection.mutable.ArrayBuffer
2525

2626
import org.apache.hadoop.conf.Configuration
2727
import org.apache.hadoop.fs.Path
28+
import org.apache.hadoop.hive.common.StatsSetupConst
2829
import org.apache.hadoop.hive.conf.HiveConf
2930
import org.apache.hadoop.hive.metastore.{TableType => HiveTableType}
3031
import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema, Order}
@@ -414,6 +415,47 @@ private[hive] class HiveClientImpl(
414415

415416
val properties = Option(h.getParameters).map(_.asScala.toMap).orNull
416417

418+
// Hive-generated Statistics are also recorded in ignoredProperties
419+
val ignoredProperties = scala.collection.mutable.Map.empty[String, String]
420+
for (key <- HiveStatisticsProperties; value <- properties.get(key)) {
421+
ignoredProperties += key -> value
422+
}
423+
424+
val excludedTableProperties = HiveStatisticsProperties ++ Set(
425+
// The property value of "comment" is moved to the dedicated field "comment"
426+
"comment",
427+
// For EXTERNAL_TABLE, the table properties has a particular field "EXTERNAL". This is added
428+
// in the function toHiveTable.
429+
"EXTERNAL"
430+
)
431+
432+
val filteredProperties = properties.filterNot {
433+
case (key, _) => excludedTableProperties.contains(key)
434+
}
435+
val comment = properties.get("comment")
436+
437+
val totalSize = properties.get(StatsSetupConst.TOTAL_SIZE).map(BigInt(_))
438+
val rawDataSize = properties.get(StatsSetupConst.RAW_DATA_SIZE).map(BigInt(_))
439+
val rowCount = properties.get(StatsSetupConst.ROW_COUNT).map(BigInt(_)).filter(_ >= 0)
440+
// TODO: check if this estimate is valid for tables after partition pruning.
441+
// NOTE: getting `totalSize` directly from params is kind of hacky, but this should be
442+
// relatively cheap if parameters for the table are populated into the metastore.
443+
// Currently, only totalSize, rawDataSize, and rowCount are used to build the field `stats`
444+
// TODO: stats should include all the other two fields (`numFiles` and `numPartitions`).
445+
// (see StatsSetupConst in Hive)
446+
val stats =
447+
// When table is external, `totalSize` is always zero, which will influence join strategy
448+
// so when `totalSize` is zero, use `rawDataSize` instead. When `rawDataSize` is also zero,
449+
// return None. Later, we will use the other ways to estimate the statistics.
450+
if (totalSize.isDefined && totalSize.get > 0L) {
451+
Some(CatalogStatistics(sizeInBytes = totalSize.get, rowCount = rowCount))
452+
} else if (rawDataSize.isDefined && rawDataSize.get > 0) {
453+
Some(CatalogStatistics(sizeInBytes = rawDataSize.get, rowCount = rowCount))
454+
} else {
455+
// TODO: still fill the rowCount even if sizeInBytes is empty. Might break anything?
456+
None
457+
}
458+
417459
CatalogTable(
418460
identifier = TableIdentifier(h.getTableName, Option(h.getDbName)),
419461
tableType = h.getTableType match {
@@ -451,13 +493,15 @@ private[hive] class HiveClientImpl(
451493
),
452494
// For EXTERNAL_TABLE, the table properties has a particular field "EXTERNAL". This is added
453495
// in the function toHiveTable.
454-
properties = properties.filter(kv => kv._1 != "comment" && kv._1 != "EXTERNAL"),
455-
comment = properties.get("comment"),
496+
properties = filteredProperties,
497+
stats = stats,
498+
comment = comment,
456499
// In older versions of Spark(before 2.2.0), we expand the view original text and store
457500
// that into `viewExpandedText`, and that should be used in view resolution. So we get
458501
// `viewExpandedText` instead of `viewOriginalText` for viewText here.
459502
viewText = Option(h.getViewExpandedText),
460-
unsupportedFeatures = unsupportedFeatures)
503+
unsupportedFeatures = unsupportedFeatures,
504+
ignoredProperties = ignoredProperties.toMap)
461505
}
462506
}
463507

@@ -474,7 +518,12 @@ private[hive] class HiveClientImpl(
474518
}
475519

476520
override def alterTable(tableName: String, table: CatalogTable): Unit = withHiveState {
477-
val hiveTable = toHiveTable(table, Some(userName))
521+
// getTableOption removes all the Hive-specific properties. Here, we fill them back to ensure
522+
// these properties are still available to the others that share the same Hive metastore.
523+
// If users explicitly alter these Hive-specific properties through ALTER TABLE DDL, we respect
524+
// these user-specified values.
525+
val hiveTable = toHiveTable(
526+
table.copy(properties = table.ignoredProperties ++ table.properties), Some(userName))
478527
// Do not use `table.qualifiedName` here because this may be a rename
479528
val qualifiedTableName = s"${table.database}.$tableName"
480529
shim.alterTable(client, qualifiedTableName, hiveTable)
@@ -956,4 +1005,14 @@ private[hive] object HiveClientImpl {
9561005
parameters =
9571006
if (hp.getParameters() != null) hp.getParameters().asScala.toMap else Map.empty)
9581007
}
1008+
1009+
// Below is the key of table properties for storing Hive-generated statistics
1010+
private val HiveStatisticsProperties = Set(
1011+
StatsSetupConst.COLUMN_STATS_ACCURATE,
1012+
StatsSetupConst.NUM_FILES,
1013+
StatsSetupConst.NUM_PARTITIONS,
1014+
StatsSetupConst.ROW_COUNT,
1015+
StatsSetupConst.RAW_DATA_SIZE,
1016+
StatsSetupConst.TOTAL_SIZE
1017+
)
9591018
}

sql/hive/src/test/java/org/apache/spark/sql/hive/test/TestHiveSingleton.scala

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,13 +19,17 @@ package org.apache.spark.sql.hive.test
1919

2020
import org.scalatest.BeforeAndAfterAll
2121

22-
import org.apache.spark.sql.SparkSession
2322
import org.apache.spark.SparkFunSuite
23+
import org.apache.spark.sql.SparkSession
24+
import org.apache.spark.sql.hive.HiveExternalCatalog
25+
import org.apache.spark.sql.hive.client.HiveClient
2426

2527

2628
trait TestHiveSingleton extends SparkFunSuite with BeforeAndAfterAll {
2729
protected val spark: SparkSession = TestHive.sparkSession
2830
protected val hiveContext: TestHiveContext = TestHive
31+
protected val hiveClient: HiveClient =
32+
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
2933

3034
protected override def afterAll(): Unit = {
3135
try {

sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -646,7 +646,6 @@ class HiveDDLCommandSuite extends PlanTest with SQLTestUtils with TestHiveSingle
646646
}
647647

648648
test("SPARK-15887: hive-site.xml should be loaded") {
649-
val hiveClient = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
650649
assert(hiveClient.getConf("hive.in.test", "") == "true")
651650
}
652651

sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogBackwardCompatibilitySuite.scala

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -35,10 +35,6 @@ import org.apache.spark.util.Utils
3535
class HiveExternalCatalogBackwardCompatibilitySuite extends QueryTest
3636
with SQLTestUtils with TestHiveSingleton with BeforeAndAfterEach {
3737

38-
// To test `HiveExternalCatalog`, we need to read/write the raw table meta from/to hive client.
39-
val hiveClient: HiveClient =
40-
spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
41-
4238
val tempDir = Utils.createTempDir().getCanonicalFile
4339
val tempDirUri = tempDir.toURI
4440
val tempDirStr = tempDir.getAbsolutePath

sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -52,11 +52,6 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv
5252
jsonFilePath = Utils.getSparkClassLoader.getResource("sample.json").getFile
5353
}
5454

55-
// To test `HiveExternalCatalog`, we need to read the raw table metadata(schema, partition
56-
// columns and bucket specification are still in table properties) from hive client.
57-
private def hiveClient: HiveClient =
58-
sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
59-
6055
test("persistent JSON table") {
6156
withTable("jsonTable") {
6257
sql(

sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala

Lines changed: 4 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -325,26 +325,20 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing
325325
"last_modified_by",
326326
"last_modified_time",
327327
"Owner:",
328-
"COLUMN_STATS_ACCURATE",
329328
// The following are hive specific schema parameters which we do not need to match exactly.
330-
"numFiles",
331-
"numRows",
332-
"rawDataSize",
333-
"totalSize",
334329
"totalNumberFiles",
335330
"maxFileSize",
336-
"minFileSize",
337-
// EXTERNAL is not non-deterministic, but it is filtered out for external tables.
338-
"EXTERNAL"
331+
"minFileSize"
339332
)
340333

341334
table.copy(
342335
createTime = 0L,
343336
lastAccessTime = 0L,
344-
properties = table.properties.filterKeys(!nondeterministicProps.contains(_))
337+
properties = table.properties.filterKeys(!nondeterministicProps.contains(_)),
338+
stats = None,
339+
ignoredProperties = Map.empty
345340
)
346341
}
347-
348342
assert(normalize(actual) == normalize(expected))
349343
}
350344
}

0 commit comments

Comments
 (0)