-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-21031] [SQL] Add alterTableStats to store spark's stats and let alterTable keep existing stats
#18248
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 2 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -538,30 +538,10 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat | |
| requireTableExists(db, tableDefinition.identifier.table) | ||
| verifyTableProperties(tableDefinition) | ||
|
|
||
| // convert table statistics to properties so that we can persist them through hive api | ||
| val withStatsProps = if (tableDefinition.stats.isDefined) { | ||
| val stats = tableDefinition.stats.get | ||
| var statsProperties: Map[String, String] = | ||
| Map(STATISTICS_TOTAL_SIZE -> stats.sizeInBytes.toString()) | ||
| if (stats.rowCount.isDefined) { | ||
| statsProperties += STATISTICS_NUM_ROWS -> stats.rowCount.get.toString() | ||
| } | ||
| val colNameTypeMap: Map[String, DataType] = | ||
| tableDefinition.schema.fields.map(f => (f.name, f.dataType)).toMap | ||
| stats.colStats.foreach { case (colName, colStat) => | ||
| colStat.toMap(colName, colNameTypeMap(colName)).foreach { case (k, v) => | ||
| statsProperties += (columnStatKeyPropName(colName, k) -> v) | ||
| } | ||
| } | ||
| tableDefinition.copy(properties = tableDefinition.properties ++ statsProperties) | ||
| } else { | ||
| tableDefinition | ||
| } | ||
|
|
||
| if (tableDefinition.tableType == VIEW) { | ||
| client.alterTable(withStatsProps) | ||
| client.alterTable(tableDefinition) | ||
| } else { | ||
| val oldTableDef = getRawTable(db, withStatsProps.identifier.table) | ||
| val oldTableDef = getRawTable(db, tableDefinition.identifier.table) | ||
|
|
||
| val newStorage = if (DDLUtils.isHiveTable(tableDefinition)) { | ||
| tableDefinition.storage | ||
|
|
@@ -614,9 +594,11 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat | |
| // Sets the `schema`, `partitionColumnNames` and `bucketSpec` from the old table definition, | ||
|
||
| // to retain the spark specific format if it is. Also add old data source properties to table | ||
| // properties, to retain the data source table format. | ||
| val oldDataSourceProps = oldTableDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX)) | ||
| val newTableProps = oldDataSourceProps ++ withStatsProps.properties + partitionProviderProp | ||
| val newDef = withStatsProps.copy( | ||
| val propsFromOldTable = oldTableDef.properties.filter { case (k, v) => | ||
| k.startsWith(DATASOURCE_PREFIX) || k.startsWith(STATISTICS_PREFIX) | ||
| } | ||
| val newTableProps = propsFromOldTable ++ tableDefinition.properties + partitionProviderProp | ||
| val newDef = tableDefinition.copy( | ||
| storage = newStorage, | ||
| schema = oldTableDef.schema, | ||
| partitionColumnNames = oldTableDef.partitionColumnNames, | ||
|
|
@@ -647,6 +629,32 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat | |
| } | ||
| } | ||
|
|
||
| override def alterTableStats( | ||
| db: String, | ||
| table: String, | ||
| stats: CatalogStatistics): Unit = withClient { | ||
| requireTableExists(db, table) | ||
| val rawTable = getRawTable(db, table) | ||
|
|
||
| // convert table statistics to properties so that we can persist them through hive client | ||
| var statsProperties: Map[String, String] = | ||
| Map(STATISTICS_TOTAL_SIZE -> stats.sizeInBytes.toString()) | ||
| if (stats.rowCount.isDefined) { | ||
| statsProperties += STATISTICS_NUM_ROWS -> stats.rowCount.get.toString() | ||
| } | ||
| val colNameTypeMap: Map[String, DataType] = | ||
| rawTable.schema.fields.map(f => (f.name, f.dataType)).toMap | ||
| stats.colStats.foreach { case (colName, colStat) => | ||
| colStat.toMap(colName, colNameTypeMap(colName)).foreach { case (k, v) => | ||
| statsProperties += (columnStatKeyPropName(colName, k) -> v) | ||
| } | ||
| } | ||
|
|
||
| val oldTableNonStatsProps = rawTable.properties.filterNot(_._1.startsWith(STATISTICS_PREFIX)) | ||
| val updatedTable = rawTable.copy(properties = oldTableNonStatsProps ++ statsProperties) | ||
| client.alterTable(updatedTable) | ||
| } | ||
|
|
||
| override def getTable(db: String, table: String): CatalogTable = withClient { | ||
| restoreTableMetadata(getRawTable(db, table)) | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -25,7 +25,7 @@ import scala.util.matching.Regex | |
|
|
||
| import org.apache.spark.sql._ | ||
| import org.apache.spark.sql.catalyst.TableIdentifier | ||
| import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogStatistics} | ||
| import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogStatistics, CatalogTable} | ||
| import org.apache.spark.sql.catalyst.util.StringUtils | ||
| import org.apache.spark.sql.execution.command.DDLUtils | ||
| import org.apache.spark.sql.execution.datasources.LogicalRelation | ||
|
|
@@ -267,7 +267,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto | |
| } | ||
| } | ||
|
|
||
| test("get statistics when not analyzed in both Hive and Spark") { | ||
| test("get statistics when not analyzed in Hive or Spark") { | ||
| val tabName = "tab1" | ||
| withTable(tabName) { | ||
| createNonPartitionedTable(tabName, analyzedByHive = false, analyzedBySpark = false) | ||
|
|
@@ -313,60 +313,70 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto | |
| } | ||
| } | ||
|
|
||
| test("alter table SET TBLPROPERTIES after analyze table") { | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Here I found the logic is the same for two cases except only the command (set and unset, respectively), so I extracted the common logic. |
||
| Seq(true, false).foreach { analyzedBySpark => | ||
| val tabName = "tab1" | ||
| withTable(tabName) { | ||
| createNonPartitionedTable(tabName, analyzedByHive = true, analyzedBySpark = analyzedBySpark) | ||
| val fetchedStats1 = checkTableStats( | ||
| tabName, hasSizeInBytes = true, expectedRowCounts = Some(500)) | ||
| sql(s"ALTER TABLE $tabName SET TBLPROPERTIES ('foo' = 'a')") | ||
| val fetchedStats2 = checkTableStats( | ||
| tabName, hasSizeInBytes = true, expectedRowCounts = Some(500)) | ||
| assert(fetchedStats1 == fetchedStats2) | ||
|
|
||
| val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED $tabName") | ||
|
|
||
| val totalSize = extractStatsPropValues(describeResult, "totalSize") | ||
| assert(totalSize.isDefined && totalSize.get > 0, "totalSize is lost") | ||
| test("alter table should not have the side effect to store statistics in Spark side") { | ||
| def getCatalogTable(tableName: String): CatalogTable = { | ||
| spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) | ||
| } | ||
|
|
||
| // ALTER TABLE SET TBLPROPERTIES invalidates some Hive specific statistics | ||
| // This is triggered by the Hive alterTable API | ||
| val numRows = extractStatsPropValues(describeResult, "numRows") | ||
| assert(numRows.isDefined && numRows.get == -1, "numRows is lost") | ||
| val rawDataSize = extractStatsPropValues(describeResult, "rawDataSize") | ||
| assert(rawDataSize.isDefined && rawDataSize.get == -1, "rawDataSize is lost") | ||
| } | ||
| val table = "alter_table_side_effect" | ||
| withTable(table) { | ||
| sql(s"CREATE TABLE $table (i string, j string)") | ||
| sql(s"INSERT INTO TABLE $table SELECT 'a', 'b'") | ||
| val catalogTable1 = getCatalogTable(table) | ||
| val hiveSize1 = BigInt(catalogTable1.ignoredProperties(StatsSetupConst.TOTAL_SIZE)) | ||
|
|
||
| sql(s"ALTER TABLE $table SET TBLPROPERTIES ('prop1' = 'a')") | ||
|
|
||
| sql(s"INSERT INTO TABLE $table SELECT 'c', 'd'") | ||
| val catalogTable2 = getCatalogTable(table) | ||
| val hiveSize2 = BigInt(catalogTable2.ignoredProperties(StatsSetupConst.TOTAL_SIZE)) | ||
| // After insertion, Hive's stats should be changed. | ||
| assert(hiveSize2 > hiveSize1) | ||
| // We haven't generate stats in Spark, so we should still use Hive's stats here. | ||
| assert(catalogTable2.stats.get.sizeInBytes == hiveSize2) | ||
| } | ||
| } | ||
|
|
||
| test("alter table UNSET TBLPROPERTIES after analyze table") { | ||
| private def testAlterTableProperties(tabName: String, alterTablePropCmd: String): Unit = { | ||
| Seq(true, false).foreach { analyzedBySpark => | ||
| val tabName = "tab1" | ||
| withTable(tabName) { | ||
| createNonPartitionedTable(tabName, analyzedByHive = true, analyzedBySpark = analyzedBySpark) | ||
| val fetchedStats1 = checkTableStats( | ||
| tabName, hasSizeInBytes = true, expectedRowCounts = Some(500)) | ||
| sql(s"ALTER TABLE $tabName UNSET TBLPROPERTIES ('prop1')") | ||
| val fetchedStats2 = checkTableStats( | ||
| tabName, hasSizeInBytes = true, expectedRowCounts = Some(500)) | ||
| assert(fetchedStats1 == fetchedStats2) | ||
| checkTableStats(tabName, hasSizeInBytes = true, expectedRowCounts = Some(500)) | ||
|
|
||
| // Run ALTER TABLE command | ||
| sql(alterTablePropCmd) | ||
|
|
||
| val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED $tabName") | ||
|
|
||
| val totalSize = extractStatsPropValues(describeResult, "totalSize") | ||
| assert(totalSize.isDefined && totalSize.get > 0, "totalSize is lost") | ||
|
|
||
| // ALTER TABLE UNSET TBLPROPERTIES invalidates some Hive specific statistics | ||
| // This is triggered by the Hive alterTable API | ||
| // ALTER TABLE SET/UNSET TBLPROPERTIES invalidates some Hive specific statistics, but not | ||
| // Spark specific statistics. This is triggered by the Hive alterTable API. | ||
| val numRows = extractStatsPropValues(describeResult, "numRows") | ||
| assert(numRows.isDefined && numRows.get == -1, "numRows is lost") | ||
| val rawDataSize = extractStatsPropValues(describeResult, "rawDataSize") | ||
| assert(rawDataSize.isDefined && rawDataSize.get == -1, "rawDataSize is lost") | ||
|
|
||
| if (analyzedBySpark) { | ||
| checkTableStats(tabName, hasSizeInBytes = true, expectedRowCounts = Some(500)) | ||
| } else { | ||
| checkTableStats(tabName, hasSizeInBytes = true, expectedRowCounts = None) | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| test("alter table SET TBLPROPERTIES after analyze table") { | ||
| testAlterTableProperties("set_prop_table", | ||
| "ALTER TABLE set_prop_table SET TBLPROPERTIES ('foo' = 'a')") | ||
| } | ||
|
|
||
| test("alter table UNSET TBLPROPERTIES after analyze table") { | ||
| testAlterTableProperties("unset_prop_table", | ||
| "ALTER TABLE unset_prop_table UNSET TBLPROPERTIES ('prop1')") | ||
| } | ||
|
|
||
| test("add/drop partitions - managed table") { | ||
| val catalog = spark.sessionState.catalog | ||
| val managedTable = "partitionedTable" | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please update the description of this function too.